From 973de13fef5e787be1f2867f2a4fbb4af20ee77d Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 26 May 2022 21:12:47 +0800 Subject: [PATCH 001/104] Refine function test framework (#4861) close pingcap/tiflash#4830 --- dbms/src/Debug/astToExecutor.cpp | 190 ++++++++-------- dbms/src/Debug/astToExecutor.h | 2 + .../Coprocessor/DAGExpressionAnalyzer.cpp | 24 +-- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 4 - .../DAGExpressionAnalyzerHelper.cpp | 33 ++- .../Coprocessor/DAGExpressionAnalyzerHelper.h | 9 + dbms/src/Flash/Coprocessor/DAGUtils.cpp | 18 +- dbms/src/Flash/Coprocessor/DAGUtils.h | 1 + dbms/src/TestUtils/ColumnsToTiPBExpr.cpp | 202 ++++++++++++++++++ dbms/src/TestUtils/ColumnsToTiPBExpr.h | 35 +++ dbms/src/TestUtils/FunctionTestUtils.cpp | 136 ++++++++---- dbms/src/TestUtils/FunctionTestUtils.h | 27 ++- 12 files changed, 521 insertions(+), 160 deletions(-) create mode 100644 dbms/src/TestUtils/ColumnsToTiPBExpr.cpp create mode 100644 dbms/src/TestUtils/ColumnsToTiPBExpr.h diff --git a/dbms/src/Debug/astToExecutor.cpp b/dbms/src/Debug/astToExecutor.cpp index 999eb6d2e68..82f894905e6 100644 --- a/dbms/src/Debug/astToExecutor.cpp +++ b/dbms/src/Debug/astToExecutor.cpp @@ -31,6 +31,93 @@ namespace DB { +void literalFieldToTiPBExpr(const ColumnInfo & ci, const Field & val_field, tipb::Expr * expr, Int32 collator_id) +{ + *(expr->mutable_field_type()) = columnInfoToFieldType(ci); + expr->mutable_field_type()->set_collate(collator_id); + if (!val_field.isNull()) + { + WriteBufferFromOwnString ss; + switch (ci.tp) + { + case TiDB::TypeLongLong: + case TiDB::TypeLong: + case TiDB::TypeShort: + case TiDB::TypeTiny: + case TiDB::TypeInt24: + if (ci.hasUnsignedFlag()) + { + expr->set_tp(tipb::ExprType::Uint64); + UInt64 val = val_field.safeGet(); + encodeDAGUInt64(val, ss); + } + else + { + expr->set_tp(tipb::ExprType::Int64); + Int64 val = val_field.safeGet(); + encodeDAGInt64(val, ss); + } + break; + case TiDB::TypeFloat: + { + expr->set_tp(tipb::ExprType::Float32); + auto val = static_cast(val_field.safeGet()); + encodeDAGFloat32(val, ss); + break; + } + case TiDB::TypeDouble: + { + expr->set_tp(tipb::ExprType::Float64); + Float64 val = val_field.safeGet(); + encodeDAGFloat64(val, ss); + break; + } + case TiDB::TypeString: + { + expr->set_tp(tipb::ExprType::String); + const auto & val = val_field.safeGet(); + encodeDAGString(val, ss); + break; + } + case TiDB::TypeNewDecimal: + { + expr->set_tp(tipb::ExprType::MysqlDecimal); + encodeDAGDecimal(val_field, ss); + break; + } + case TiDB::TypeDate: + { + expr->set_tp(tipb::ExprType::MysqlTime); + UInt64 val = val_field.safeGet(); + encodeDAGUInt64(MyDate(val).toPackedUInt(), ss); + break; + } + case TiDB::TypeDatetime: + case TiDB::TypeTimestamp: + { + expr->set_tp(tipb::ExprType::MysqlTime); + UInt64 val = val_field.safeGet(); + encodeDAGUInt64(MyDateTime(val).toPackedUInt(), ss); + break; + } + case TiDB::TypeTime: + { + expr->set_tp(tipb::ExprType::MysqlDuration); + Int64 val = val_field.safeGet(); + encodeDAGInt64(val, ss); + break; + } + default: + throw Exception(fmt::format("Type {} does not support literal in function unit test", getDataTypeByColumnInfo(ci)->getName())); + } + expr->set_val(ss.releaseStr()); + } + else + { + expr->set_tp(tipb::ExprType::Null); + } +} + namespace { std::unordered_map func_name_to_sig({ @@ -112,76 +199,9 @@ DAGColumnInfo toNullableDAGColumnInfo(const DAGColumnInfo & input) void literalToPB(tipb::Expr * expr, const Field & value, uint32_t collator_id) { - WriteBufferFromOwnString ss; - switch (value.getType()) - { - case Field::Types::Which::Null: - { - expr->set_tp(tipb::Null); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeNull); - ft->set_collate(collator_id); - // Null literal expr doesn't need value. - break; - } - case Field::Types::Which::UInt64: - { - expr->set_tp(tipb::Uint64); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeLongLong); - ft->set_flag(TiDB::ColumnFlagUnsigned | TiDB::ColumnFlagNotNull); - ft->set_collate(collator_id); - encodeDAGUInt64(value.get(), ss); - break; - } - case Field::Types::Which::Int64: - { - expr->set_tp(tipb::Int64); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeLongLong); - ft->set_flag(TiDB::ColumnFlagNotNull); - ft->set_collate(collator_id); - encodeDAGInt64(value.get(), ss); - break; - } - case Field::Types::Which::Float64: - { - expr->set_tp(tipb::Float64); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeFloat); - ft->set_flag(TiDB::ColumnFlagNotNull); - ft->set_collate(collator_id); - encodeDAGFloat64(value.get(), ss); - break; - } - case Field::Types::Which::Decimal32: - case Field::Types::Which::Decimal64: - case Field::Types::Which::Decimal128: - case Field::Types::Which::Decimal256: - { - expr->set_tp(tipb::MysqlDecimal); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeNewDecimal); - ft->set_flag(TiDB::ColumnFlagNotNull); - ft->set_collate(collator_id); - encodeDAGDecimal(value, ss); - break; - } - case Field::Types::Which::String: - { - expr->set_tp(tipb::String); - auto * ft = expr->mutable_field_type(); - ft->set_tp(TiDB::TypeString); - ft->set_flag(TiDB::ColumnFlagNotNull); - ft->set_collate(collator_id); - // TODO: Align with TiDB. - encodeDAGBytes(value.get(), ss); - break; - } - default: - throw Exception(String("Unsupported literal type: ") + value.getTypeName(), ErrorCodes::LOGICAL_ERROR); - } - expr->set_val(ss.releaseStr()); + DataTypePtr type = applyVisitor(FieldToDataType(), value); + ColumnInfo ci = reverseGetColumnInfo({"", type}, 0, Field(), true); + literalFieldToTiPBExpr(ci, value, expr, collator_id); } String getFunctionNameForConstantFolding(tipb::Expr * expr) @@ -262,15 +282,15 @@ void identifierToPB(const DAGSchema & input, ASTIdentifier * id, tipb::Expr * ex void astToPB(const DAGSchema & input, ASTPtr ast, tipb::Expr * expr, uint32_t collator_id, const Context & context) { - if (ASTIdentifier * id = typeid_cast(ast.get())) + if (auto * id = typeid_cast(ast.get())) { identifierToPB(input, id, expr, collator_id); } - else if (ASTFunction * func = typeid_cast(ast.get())) + else if (auto * func = typeid_cast(ast.get())) { functionToPB(input, func, expr, collator_id, context); } - else if (ASTLiteral * lit = typeid_cast(ast.get())) + else if (auto * lit = typeid_cast(ast.get())) { literalToPB(expr, lit->value, collator_id); } @@ -505,7 +525,7 @@ void identifierToPB(const DAGSchema & input, ASTIdentifier * id, tipb::Expr * ex void collectUsedColumnsFromExpr(const DAGSchema & input, ASTPtr ast, std::unordered_set & used_columns) { - if (ASTIdentifier * id = typeid_cast(ast.get())) + if (auto * id = typeid_cast(ast.get())) { auto column_name = splitQualifiedName(id->getColumnName()); if (!column_name.first.empty()) @@ -526,7 +546,7 @@ void collectUsedColumnsFromExpr(const DAGSchema & input, ASTPtr ast, std::unorde } } } - else if (ASTFunction * func = typeid_cast(ast.get())) + else if (auto * func = typeid_cast(ast.get())) { if (AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) { @@ -559,7 +579,7 @@ void collectUsedColumnsFromExpr(const DAGSchema & input, ASTPtr ast, std::unorde TiDB::ColumnInfo compileExpr(const DAGSchema & input, ASTPtr ast) { TiDB::ColumnInfo ci; - if (ASTIdentifier * id = typeid_cast(ast.get())) + if (auto * id = typeid_cast(ast.get())) { /// check column auto ft = std::find_if(input.begin(), input.end(), [&](const auto & field) { @@ -574,7 +594,7 @@ TiDB::ColumnInfo compileExpr(const DAGSchema & input, ASTPtr ast) throw Exception("No such column " + id->getColumnName(), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); ci = ft->second; } - else if (ASTFunction * func = typeid_cast(ast.get())) + else if (auto * func = typeid_cast(ast.get())) { /// check function String func_name_lowercase = Poco::toLower(func->name); @@ -692,7 +712,7 @@ TiDB::ColumnInfo compileExpr(const DAGSchema & input, ASTPtr ast) compileExpr(input, child_ast); } } - else if (ASTLiteral * lit = typeid_cast(ast.get())) + else if (auto * lit = typeid_cast(ast.get())) { switch (lit->value.getType()) { @@ -909,7 +929,7 @@ bool TopN::toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t collator_id, tipb::TopN * topn = tipb_executor->mutable_topn(); for (const auto & child : order_columns) { - ASTOrderByElement * elem = typeid_cast(child.get()); + auto * elem = typeid_cast(child.get()); if (!elem) throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); tipb::ByItem * by = topn->add_order_by(); @@ -954,7 +974,7 @@ bool Aggregation::toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t collat auto & input_schema = children[0]->output_schema; for (const auto & expr : agg_exprs) { - const ASTFunction * func = typeid_cast(expr.get()); + const auto * func = typeid_cast(expr.get()); if (!func || !AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) throw Exception("Only agg function is allowed in select for a query with aggregation", ErrorCodes::LOGICAL_ERROR); @@ -1024,7 +1044,7 @@ void Aggregation::columnPrune(std::unordered_set & used_columns) { if (used_columns.find(func->getColumnName()) != used_columns.end()) { - const ASTFunction * agg_func = typeid_cast(func.get()); + const auto * agg_func = typeid_cast(func.get()); if (agg_func != nullptr) { /// agg_func should not be nullptr, just double check @@ -1075,7 +1095,7 @@ void Aggregation::toMPPSubPlan(size_t & executor_index, const DAGProperties & pr /// re-construct agg_exprs and gby_exprs in final_agg for (size_t i = 0; i < partial_agg->agg_exprs.size(); i++) { - const ASTFunction * agg_func = typeid_cast(partial_agg->agg_exprs[i].get()); + const auto * agg_func = typeid_cast(partial_agg->agg_exprs[i].get()); ASTPtr update_agg_expr = agg_func->clone(); auto * update_agg_func = typeid_cast(update_agg_expr.get()); if (agg_func->name == "count") @@ -1368,7 +1388,7 @@ ExecutorPtr compileTopN(ExecutorPtr input, size_t & executor_index, ASTPtr order std::vector order_columns; for (const auto & child : order_exprs->children) { - ASTOrderByElement * elem = typeid_cast(child.get()); + auto * elem = typeid_cast(child.get()); if (!elem) throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); order_columns.push_back(child); @@ -1399,7 +1419,7 @@ ExecutorPtr compileAggregation(ExecutorPtr input, size_t & executor_index, ASTPt { for (const auto & expr : agg_funcs->children) { - const ASTFunction * func = typeid_cast(expr.get()); + const auto * func = typeid_cast(expr.get()); if (!func || !AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) { need_append_project = true; @@ -1490,7 +1510,7 @@ ExecutorPtr compileProject(ExecutorPtr input, size_t & executor_index, ASTPtr se output_schema.emplace_back(ft->first, ft->second); continue; } - const ASTFunction * func = typeid_cast(expr.get()); + const auto * func = typeid_cast(expr.get()); if (func && AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) { throw Exception("No such agg " + func->getColumnName(), ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); diff --git a/dbms/src/Debug/astToExecutor.h b/dbms/src/Debug/astToExecutor.h index 54839e60dc6..37d3f22b6e1 100644 --- a/dbms/src/Debug/astToExecutor.h +++ b/dbms/src/Debug/astToExecutor.h @@ -294,6 +294,8 @@ ExecutorPtr compileExchangeSender(ExecutorPtr input, size_t & executor_index, ti ExecutorPtr compileExchangeReceiver(size_t & executor_index, DAGSchema schema); +void literalFieldToTiPBExpr(const ColumnInfo & ci, const Field & field, tipb::Expr * expr, Int32 collator_id); + //TODO: add compileWindow } // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index cffae76cb81..aa269469cdb 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -1279,15 +1279,7 @@ String DAGExpressionAnalyzer::getActions(const tipb::Expr & expr, const Expressi } else if (isScalarFunctionExpr(expr)) { - const String & func_name = getFunctionName(expr); - if (DAGExpressionAnalyzerHelper::function_builder_map.count(func_name) != 0) - { - ret = DAGExpressionAnalyzerHelper::function_builder_map[func_name](this, expr, actions); - } - else - { - ret = buildFunction(expr, actions); - } + ret = DAGExpressionAnalyzerHelper::buildFunction(this, expr, actions); } else { @@ -1341,18 +1333,4 @@ String DAGExpressionAnalyzer::buildTupleFunctionForGroupConcat( return applyFunction(func_name, argument_names, actions, nullptr); } -String DAGExpressionAnalyzer::buildFunction( - const tipb::Expr & expr, - const ExpressionActionsPtr & actions) -{ - const String & func_name = getFunctionName(expr); - Names argument_names; - for (const auto & child : expr.children()) - { - String name = getActions(child, actions); - argument_names.push_back(name); - } - return applyFunction(func_name, argument_names, actions, getCollatorFromExpr(expr)); -} - } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index f565e7a6348..3b7112af02d 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -275,10 +275,6 @@ class DAGExpressionAnalyzer : private boost::noncopyable const ExpressionActionsPtr & actions, const String & column_name); - String buildFunction( - const tipb::Expr & expr, - const ExpressionActionsPtr & actions); - String buildFilterColumn( const ExpressionActionsPtr & actions, const std::vector & conditions); diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.cpp index cabd88e0ba7..ee529680d28 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.cpp @@ -248,7 +248,7 @@ String DAGExpressionAnalyzerHelper::buildCastFunctionInternal( return result_name; FunctionBuilderPtr function_builder = FunctionFactory::instance().get(tidb_cast_name, analyzer->getContext()); - FunctionBuilderTiDBCast * function_builder_tidb_cast = dynamic_cast(function_builder.get()); + auto * function_builder_tidb_cast = dynamic_cast(function_builder.get()); function_builder_tidb_cast->setInUnion(in_union); function_builder_tidb_cast->setTiDBFieldType(field_type); @@ -401,6 +401,37 @@ String DAGExpressionAnalyzerHelper::buildRegexpFunction( return analyzer->applyFunction(func_name, argument_names, actions, collator); } +String DAGExpressionAnalyzerHelper::buildDefaultFunction( + DAGExpressionAnalyzer * analyzer, + const tipb::Expr & expr, + const ExpressionActionsPtr & actions) +{ + const String & func_name = getFunctionName(expr); + Names argument_names; + for (const auto & child : expr.children()) + { + String name = analyzer->getActions(child, actions); + argument_names.push_back(name); + } + return analyzer->applyFunction(func_name, argument_names, actions, getCollatorFromExpr(expr)); +} + +String DAGExpressionAnalyzerHelper::buildFunction( + DAGExpressionAnalyzer * analyzer, + const tipb::Expr & expr, + const ExpressionActionsPtr & actions) +{ + const String & func_name = getFunctionName(expr); + if (function_builder_map.count(func_name) != 0) + { + return function_builder_map[func_name](analyzer, expr, actions); + } + else + { + return buildDefaultFunction(analyzer, expr, actions); + } +} + DAGExpressionAnalyzerHelper::FunctionBuilderMap DAGExpressionAnalyzerHelper::function_builder_map( {{"in", DAGExpressionAnalyzerHelper::buildInFunction}, {"notIn", DAGExpressionAnalyzerHelper::buildInFunction}, diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.h index c8f2b658388..fcafcc57819 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.h @@ -25,6 +25,10 @@ class DAGExpressionAnalyzer; class DAGExpressionAnalyzerHelper { public: + static String buildFunction( + DAGExpressionAnalyzer * analyzer, + const tipb::Expr & expr, + const ExpressionActionsPtr & actions); static String buildInFunction( DAGExpressionAnalyzer * analyzer, const tipb::Expr & expr, @@ -83,6 +87,11 @@ class DAGExpressionAnalyzerHelper const tipb::Expr & expr, const ExpressionActionsPtr & actions); + static String buildDefaultFunction( + DAGExpressionAnalyzer * analyzer, + const tipb::Expr & expr, + const ExpressionActionsPtr & actions); + using FunctionBuilder = std::function; using FunctionBuilderMap = std::unordered_map; diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 66f5d7031d7..87f58131c8c 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -707,7 +707,16 @@ void assertBlockSchema( actual->getName())); } } - +/// used by test +std::unordered_map getFuncNameToSigMap() +{ + std::unordered_map ret; + for (const auto & element : scalar_func_map) + { + ret[element.second] = element.first; + } + return ret; +} } // namespace bool isScalarFunctionExpr(const tipb::Expr & expr) @@ -1420,5 +1429,12 @@ tipb::EncodeType analyzeDAGEncodeType(DAGContext & dag_context) return tipb::EncodeType::TypeDefault; return encode_type; } +tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name) +{ + static std::unordered_map func_name_sig_map = getFuncNameToSigMap(); + if (func_name_sig_map.find(name) == func_name_sig_map.end()) + throw Exception(fmt::format("Unsupported function {}", name)); + return func_name_sig_map[name]; +} } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.h b/dbms/src/Flash/Coprocessor/DAGUtils.h index 4d6a62bbe6f..5776edf0098 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.h +++ b/dbms/src/Flash/Coprocessor/DAGUtils.h @@ -104,5 +104,6 @@ class UniqueNameGenerator tipb::DAGRequest getDAGRequestFromStringWithRetry(const String & s); tipb::EncodeType analyzeDAGEncodeType(DAGContext & dag_context); +tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name); } // namespace DB diff --git a/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp b/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp new file mode 100644 index 00000000000..2c3bf243176 --- /dev/null +++ b/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp @@ -0,0 +1,202 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include + +namespace DB +{ +namespace tests +{ +namespace +{ +void columnToTiPBExpr(tipb::Expr * expr, const ColumnWithTypeAndName column, size_t index) +{ + ColumnInfo ci = reverseGetColumnInfo({column.name, column.type}, 0, Field(), true); + bool is_const = false; + if (column.column != nullptr) + { + is_const = column.column->isColumnConst(); + if (!is_const) + { + if (column.column->isColumnNullable()) + { + auto [col, null_map] = removeNullable(column.column.get()); + is_const = col->isColumnConst(); + } + } + } + if (is_const) + { + Field val_field; + column.column->get(0, val_field); + literalFieldToTiPBExpr(ci, val_field, expr, 0); + } + else + { + *(expr->mutable_field_type()) = columnInfoToFieldType(ci); + expr->set_tp(tipb::ExprType::ColumnRef); + WriteBufferFromOwnString ss; + encodeDAGInt64(index, ss); + expr->set_val(ss.releaseStr()); + } +} +void columnsToTiPBExprForRegExp( + tipb::Expr * expr, + const String &, + const ColumnNumbers & argument_column_number, + const ColumnsWithTypeAndName & columns, + const TiDB::TiDBCollatorPtr & collator) +{ + expr->set_tp(tipb::ExprType::ScalarFunc); + if (collator == nullptr || !collator->isBinary()) + expr->set_sig(tipb::ScalarFuncSig::RegexpUTF8Sig); + else + expr->set_sig(tipb::ScalarFuncSig::RegexpSig); + for (size_t i = 0; i < argument_column_number.size(); ++i) + { + auto * argument_expr = expr->add_children(); + columnToTiPBExpr(argument_expr, columns[argument_column_number[i]], i); + } + /// since we don't know the type, just set a fake one + expr->mutable_field_type()->set_tp(TiDB::TypeLongLong); + if (collator != nullptr) + expr->mutable_field_type()->set_collate(-collator->getCollatorId()); +} +void columnsToTiPBExprForTiDBCast( + tipb::Expr * expr, + const String & func_name, + const ColumnNumbers & argument_column_number, + const ColumnsWithTypeAndName & columns, + const TiDB::TiDBCollatorPtr & collator) +{ + expr->set_tp(tipb::ExprType::ScalarFunc); + expr->set_sig(reverseGetFuncSigByFuncName(func_name)); + assert(argument_column_number.size() == 2); + const auto & type_column = columns[argument_column_number[1]]; + bool is_const = false; + if (type_column.column != nullptr) + { + is_const = type_column.column->isColumnConst(); + if (!is_const) + { + if (type_column.column->isColumnNullable()) + { + auto [col, null_map] = removeNullable(type_column.column.get()); + is_const = col->isColumnConst(); + } + } + } + assert(is_const && removeNullable(type_column.type)->isString()); + Field val; + type_column.column->get(0, val); + String type_string = val.safeGet(); + DataTypePtr target_type = DataTypeFactory::instance().get(type_string); + auto * argument_expr = expr->add_children(); + columnToTiPBExpr(argument_expr, columns[argument_column_number[0]], 0); + ColumnInfo ci = reverseGetColumnInfo({type_string, target_type}, 0, Field(), true); + *(expr->mutable_field_type()) = columnInfoToFieldType(ci); + if (collator != nullptr) + expr->mutable_field_type()->set_collate(-collator->getCollatorId()); +} + +const std::unordered_map date_add_sub_map({ + {"addDays", "DAY"}, + {"addWeeks", "WEEK"}, + {"addMonths", "MONTH"}, + {"addYears", "YEAR"}, + {"addHours", "HOUR"}, + {"addMinutes", "MINUTE"}, + {"addSeconds", "SECOND"}, + {"subtractDays", "DAY"}, + {"subtractWeeks", "WEEK"}, + {"subtractMonths", "MONTH"}, + {"subtractYears", "YEAR"}, + {"subtractHours", "HOUR"}, + {"subtractMinutes", "MINUTE"}, + {"subtractSeconds", "SECOND"}, +}); + +void columnsToTiPBExprForDateAddSub( + tipb::Expr * expr, + const String & func_name, + const ColumnNumbers & argument_column_number, + const ColumnsWithTypeAndName & columns, + const TiDB::TiDBCollatorPtr & collator) +{ + String name = func_name.substr(0, 3) == "add" ? "date_add" : "date_sub"; + expr->set_tp(tipb::ExprType::ScalarFunc); + expr->set_sig(reverseGetFuncSigByFuncName(name)); + for (size_t i = 0; i < argument_column_number.size(); ++i) + { + auto * argument_expr = expr->add_children(); + columnToTiPBExpr(argument_expr, columns[argument_column_number[i]], i); + } + String unit = date_add_sub_map.find(func_name)->second; + *(expr->add_children()) = constructStringLiteralTiExpr(unit); + /// since we don't know the type, just set a fake one + expr->mutable_field_type()->set_tp(TiDB::TypeLongLong); + if (collator != nullptr) + expr->mutable_field_type()->set_collate(-collator->getCollatorId()); +} +void columnsToTiPBExpr( + tipb::Expr * expr, + const String & func_name, + const ColumnNumbers & argument_column_number, + const ColumnsWithTypeAndName & columns, + const TiDB::TiDBCollatorPtr & collator) +{ + if (func_name == "tidb_cast") + { + columnsToTiPBExprForTiDBCast(expr, func_name, argument_column_number, columns, collator); + } + else if (func_name == "regexp") + { + columnsToTiPBExprForRegExp(expr, func_name, argument_column_number, columns, collator); + } + else if (date_add_sub_map.find(func_name) != date_add_sub_map.end()) + { + columnsToTiPBExprForDateAddSub(expr, func_name, argument_column_number, columns, collator); + } + else + { + expr->set_tp(tipb::ExprType::ScalarFunc); + expr->set_sig(reverseGetFuncSigByFuncName(func_name)); + for (size_t i = 0; i < argument_column_number.size(); ++i) + { + auto * argument_expr = expr->add_children(); + columnToTiPBExpr(argument_expr, columns[argument_column_number[i]], i); + } + /// since we don't know the type, just set a fake one + expr->mutable_field_type()->set_tp(TiDB::TypeLongLong); + if (collator != nullptr) + expr->mutable_field_type()->set_collate(-collator->getCollatorId()); + } +} +} // namespace + +tipb::Expr columnsToTiPBExpr( + const String & func_name, + const ColumnNumbers & argument_column_number, + const ColumnsWithTypeAndName & columns, + const TiDB::TiDBCollatorPtr & collator) +{ + tipb::Expr ret; + columnsToTiPBExpr(&ret, func_name, argument_column_number, columns, collator); + return ret; +} +} // namespace tests +} // namespace DB diff --git a/dbms/src/TestUtils/ColumnsToTiPBExpr.h b/dbms/src/TestUtils/ColumnsToTiPBExpr.h new file mode 100644 index 00000000000..e7a2e81d59e --- /dev/null +++ b/dbms/src/TestUtils/ColumnsToTiPBExpr.h @@ -0,0 +1,35 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace tests +{ +tipb::Expr columnsToTiPBExpr( + const String & func_name, + const ColumnNumbers & argument_column_number, + const ColumnsWithTypeAndName & columns, + const TiDB::TiDBCollatorPtr & collator); +} // namespace tests +} // namespace DB diff --git a/dbms/src/TestUtils/FunctionTestUtils.cpp b/dbms/src/TestUtils/FunctionTestUtils.cpp index 6aa7541ee59..dae07f7123b 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.cpp +++ b/dbms/src/TestUtils/FunctionTestUtils.cpp @@ -12,10 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include +#include +#include +#include #include #include +#include #include #include #include @@ -115,60 +120,113 @@ void blockEqual( } } +std::pair buildFunction( + Context & context, + const String & func_name, + const ColumnNumbers & argument_column_numbers, + const ColumnsWithTypeAndName & columns, + const TiDB::TiDBCollatorPtr & collator) +{ + tipb::Expr tipb_expr = columnsToTiPBExpr(func_name, argument_column_numbers, columns, collator); + NamesAndTypes source_columns; + for (size_t index : argument_column_numbers) + source_columns.emplace_back(columns[index].name, columns[index].type); + DAGExpressionAnalyzer analyzer(source_columns, context); + ExpressionActionsChain chain; + auto & last_step = analyzer.initAndGetLastStep(chain); + auto result_name = DB::DAGExpressionAnalyzerHelper::buildFunction(&analyzer, tipb_expr, last_step.actions); + last_step.required_output.push_back(result_name); + chain.finalize(); + return std::make_pair(last_step.actions, result_name); +} -ColumnWithTypeAndName executeFunction(Context & context, const String & func_name, const ColumnsWithTypeAndName & columns, const TiDB::TiDBCollatorPtr & collator) +ColumnsWithTypeAndName toColumnsWithUniqueName(const ColumnsWithTypeAndName & columns) { - auto & factory = FunctionFactory::instance(); + ColumnsWithTypeAndName columns_with_distinct_name = columns; + std::string base_name = "col"; + for (size_t i = 0; i < columns.size(); ++i) + { + columns_with_distinct_name[i].name = fmt::format("{}_{}", base_name, i); + } + return columns_with_distinct_name; +} - Block block(columns); - ColumnNumbers cns; +ColumnWithTypeAndName executeFunction( + Context & context, + const String & func_name, + const ColumnsWithTypeAndName & columns, + const TiDB::TiDBCollatorPtr & collator, + bool raw_function_test) +{ + ColumnNumbers argument_column_numbers; for (size_t i = 0; i < columns.size(); ++i) - cns.push_back(i); - - auto bp = factory.tryGet(func_name, context); - if (!bp) - throw TiFlashTestException(fmt::format("Function {} not found!", func_name)); - auto func = bp->build(columns, collator); - block.insert({nullptr, func->getReturnType(), "res"}); - func->execute(block, cns, columns.size()); - return block.getByPosition(columns.size()); + argument_column_numbers.push_back(i); + return executeFunction(context, func_name, argument_column_numbers, columns, collator, raw_function_test); } -ColumnWithTypeAndName executeFunction(Context & context, const String & func_name, const ColumnNumbers & argument_column_numbers, const ColumnsWithTypeAndName & columns) +ColumnWithTypeAndName executeFunction( + Context & context, + const String & func_name, + const ColumnNumbers & argument_column_numbers, + const ColumnsWithTypeAndName & columns, + const TiDB::TiDBCollatorPtr & collator, + bool raw_function_test) { - auto & factory = FunctionFactory::instance(); - Block block(columns); - ColumnsWithTypeAndName arguments; - for (size_t i = 0; i < argument_column_numbers.size(); ++i) - arguments.push_back(columns.at(i)); - auto bp = factory.tryGet(func_name, context); - if (!bp) - throw TiFlashTestException(fmt::format("Function {} not found!", func_name)); - auto func = bp->build(arguments); - block.insert({nullptr, func->getReturnType(), "res"}); - func->execute(block, argument_column_numbers, columns.size()); - return block.getByPosition(columns.size()); + if (raw_function_test) + { + auto & factory = FunctionFactory::instance(); + Block block(columns); + ColumnsWithTypeAndName arguments; + for (size_t i = 0; i < argument_column_numbers.size(); ++i) + arguments.push_back(columns.at(i)); + auto bp = factory.tryGet(func_name, context); + if (!bp) + throw TiFlashTestException(fmt::format("Function {} not found!", func_name)); + auto func = bp->build(arguments, collator); + block.insert({nullptr, func->getReturnType(), "res"}); + func->execute(block, argument_column_numbers, columns.size()); + return block.getByPosition(columns.size()); + } + auto columns_with_unique_name = toColumnsWithUniqueName(columns); + auto [actions, result_name] = buildFunction(context, func_name, argument_column_numbers, columns_with_unique_name, collator); + Block block(columns_with_unique_name); + actions->execute(block); + return block.getByName(result_name); } DataTypePtr getReturnTypeForFunction( Context & context, const String & func_name, const ColumnsWithTypeAndName & columns, - const TiDB::TiDBCollatorPtr & collator) + const TiDB::TiDBCollatorPtr & collator, + bool raw_function_test) { - auto & factory = FunctionFactory::instance(); - - Block block(columns); - ColumnNumbers cns; - for (size_t i = 0; i < columns.size(); ++i) - cns.push_back(i); - - auto bp = factory.tryGet(func_name, context); - if (!bp) - throw TiFlashTestException(fmt::format("Function {} not found!", func_name)); - auto func = bp->build(columns, collator); - return func->getReturnType(); + if (raw_function_test) + { + auto & factory = FunctionFactory::instance(); + + Block block(columns); + ColumnNumbers cns; + for (size_t i = 0; i < columns.size(); ++i) + cns.push_back(i); + + auto bp = factory.tryGet(func_name, context); + if (!bp) + throw TiFlashTestException(fmt::format("Function {} not found!", func_name)); + auto func = bp->build(columns, collator); + return func->getReturnType(); + } + else + { + ColumnNumbers argument_column_numbers; + for (size_t i = 0; i < columns.size(); ++i) + argument_column_numbers.push_back(i); + auto columns_with_unique_name = toColumnsWithUniqueName(columns); + auto [actions, result_name] = buildFunction(context, func_name, argument_column_numbers, columns_with_unique_name, collator); + return actions->getSampleBlock().getByName(result_name).type; + } } + ColumnWithTypeAndName createOnlyNullColumnConst(size_t size, const String & name) { DataTypePtr data_type = std::make_shared(std::make_shared()); diff --git a/dbms/src/TestUtils/FunctionTestUtils.h b/dbms/src/TestUtils/FunctionTestUtils.h index e88f33a5ca7..615a58ebda5 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.h +++ b/dbms/src/TestUtils/FunctionTestUtils.h @@ -535,13 +535,16 @@ ColumnWithTypeAndName executeFunction( Context & context, const String & func_name, const ColumnsWithTypeAndName & columns, - const TiDB::TiDBCollatorPtr & collator = nullptr); + const TiDB::TiDBCollatorPtr & collator = nullptr, + bool raw_function_test = true); ColumnWithTypeAndName executeFunction( Context & context, const String & func_name, const ColumnNumbers & argument_column_numbers, - const ColumnsWithTypeAndName & columns); + const ColumnsWithTypeAndName & columns, + const TiDB::TiDBCollatorPtr & collator = nullptr, + bool raw_function_test = true); template ColumnWithTypeAndName executeFunction( @@ -558,7 +561,8 @@ DataTypePtr getReturnTypeForFunction( Context & context, const String & func_name, const ColumnsWithTypeAndName & columns, - const TiDB::TiDBCollatorPtr & collator = nullptr); + const TiDB::TiDBCollatorPtr & collator = nullptr, + bool raw_function_test = true); template ColumnWithTypeAndName createNullableColumn(InferredDataVector init_vec, const std::vector & null_map, const String name = "") @@ -679,9 +683,13 @@ class FunctionTest : public ::testing::Test context.setDAGContext(dag_context_ptr.get()); } - ColumnWithTypeAndName executeFunction(const String & func_name, const ColumnsWithTypeAndName & columns, const TiDB::TiDBCollatorPtr & collator = nullptr) + ColumnWithTypeAndName executeFunction( + const String & func_name, + const ColumnsWithTypeAndName & columns, + const TiDB::TiDBCollatorPtr & collator = nullptr, + bool raw_function_test = true) { - return DB::tests::executeFunction(context, func_name, columns, collator); + return DB::tests::executeFunction(context, func_name, columns, collator, raw_function_test); } template @@ -691,9 +699,14 @@ class FunctionTest : public ::testing::Test return executeFunction(func_name, vec); } - ColumnWithTypeAndName executeFunction(const String & func_name, const ColumnNumbers & argument_column_numbers, const ColumnsWithTypeAndName & columns) + ColumnWithTypeAndName executeFunction( + const String & func_name, + const ColumnNumbers & argument_column_numbers, + const ColumnsWithTypeAndName & columns, + const TiDB::TiDBCollatorPtr & collator = nullptr, + bool raw_function_test = true) { - return DB::tests::executeFunction(context, func_name, argument_column_numbers, columns); + return DB::tests::executeFunction(context, func_name, argument_column_numbers, columns, collator, raw_function_test); } template From f10b6d27c6e0986d5db28f3f372491d5972464db Mon Sep 17 00:00:00 2001 From: yibin Date: Thu, 26 May 2022 22:26:02 +0800 Subject: [PATCH 002/104] Add mutex to protect exchange receiver's async client (#5008) * Add to_seconds support for tiflash Signed-off-by: yibin * Fix format issue Signed-off-by: yibin * Add mutex lock to protect async reciever async reader Signed-off-by: yibin * Fix a rebase issue Signed-off-by: yibin * Change raw pointer to unique ptr Signed-off-by: yibin * Fix format issue Signed-off-by: yibin Co-authored-by: Ti Chi Robot --- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 11 +++++++++-- dbms/src/Functions/FunctionsDateTime.cpp | 1 - dbms/src/Functions/FunctionsDateTime.h | 1 - 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 9639771c586..f194afee31f 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -92,10 +92,14 @@ class AsyncRequestHandler : public UnaryCallback switch (stage) { case AsyncRequestStage::WAIT_MAKE_READER: + { + // Use lock to ensure reader is created already in reactor thread + std::unique_lock lock(mu); if (!ok) reader.reset(); notifyReactor(); break; + } case AsyncRequestStage::WAIT_BATCH_READ: if (ok) ++read_packet_index; @@ -227,6 +231,8 @@ class AsyncRequestHandler : public UnaryCallback void start() { stage = AsyncRequestStage::WAIT_MAKE_READER; + // Use lock to ensure async reader is unreachable from grpc thread before this function returns + std::unique_lock lock(mu); rpc_context->makeAsyncReader(*request, reader, thisAsUnaryCallback()); } @@ -283,6 +289,7 @@ class AsyncRequestHandler : public UnaryCallback size_t read_packet_index = 0; Status finish_status = RPCContext::getStatusOK(); LoggerPtr log; + std::mutex mu; }; } // namespace @@ -393,10 +400,10 @@ void ExchangeReceiverBase::reactor(const std::vector & asyn MPMCQueue ready_requests(alive_async_connections * 2); std::vector waiting_for_retry_requests; - std::vector> handlers; + std::vector> handlers; handlers.reserve(alive_async_connections); for (const auto & req : async_requests) - handlers.emplace_back(&ready_requests, &msg_channel, rpc_context, req, exc_log->identifier()); + handlers.emplace_back(std::make_unique(&ready_requests, &msg_channel, rpc_context, req, exc_log->identifier())); while (alive_async_connections > 0) { diff --git a/dbms/src/Functions/FunctionsDateTime.cpp b/dbms/src/Functions/FunctionsDateTime.cpp index dd072a00f76..607f6bc4c99 100644 --- a/dbms/src/Functions/FunctionsDateTime.cpp +++ b/dbms/src/Functions/FunctionsDateTime.cpp @@ -254,7 +254,6 @@ void registerFunctionsDateTime(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); - factory.registerFunction(); factory.registerFunction(); } diff --git a/dbms/src/Functions/FunctionsDateTime.h b/dbms/src/Functions/FunctionsDateTime.h index df579a1bab8..08f5a5887d4 100644 --- a/dbms/src/Functions/FunctionsDateTime.h +++ b/dbms/src/Functions/FunctionsDateTime.h @@ -3450,7 +3450,6 @@ using FunctionToTiDBDayOfYear = FunctionMyDateOrMyDateTimeToSomething; using FunctionToTiDBToSeconds = FunctionMyDateOrMyDateTimeToSomething; using FunctionToTiDBToDays = FunctionMyDateOrMyDateTimeToSomething; - using FunctionToRelativeYearNum = FunctionDateOrDateTimeToSomething; using FunctionToRelativeQuarterNum = FunctionDateOrDateTimeToSomething; using FunctionToRelativeMonthNum = FunctionDateOrDateTimeToSomething; From d84f273182aabc70e445497cb51bad9ecfb7deec Mon Sep 17 00:00:00 2001 From: jiaqizho Date: Fri, 27 May 2022 17:42:48 +0800 Subject: [PATCH 003/104] Optimize blobstore gc when current stat total size is 0 (#5015) close pingcap/tiflash#5016 --- dbms/src/Storages/Page/V3/BlobStore.cpp | 19 +++++++++++++++++-- dbms/src/Storages/Page/V3/BlobStore.h | 2 +- 2 files changed, 18 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/Page/V3/BlobStore.cpp b/dbms/src/Storages/Page/V3/BlobStore.cpp index dc5ed536f9e..f03bc5bcf73 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.cpp +++ b/dbms/src/Storages/Page/V3/BlobStore.cpp @@ -914,7 +914,22 @@ std::vector BlobStore::getGCStats() // Avoid divide by zero if (right_margin == 0) { - LOG_FMT_TRACE(log, "Current blob is empty [blob_id={}, total size(all invalid)={}].", stat->id, stat->sm_total_size); + if (unlikely(stat->sm_valid_rate != 0)) + { + throw Exception(fmt::format("Current blob is empty, but valid rate is not 0. [blob_id={}][valid_size={}][valid_rate={}]", + stat->id, + stat->sm_valid_size, + stat->sm_valid_rate)); + } + + LOG_FMT_TRACE(log, "Current blob is empty [blob_id={}, total size(all invalid)={}] [valid_rate={}].", stat->id, stat->sm_total_size, stat->sm_valid_rate); + + // If current blob empty, the size of in disk blob may not empty + // So we need truncate current blob, and let it be reused. + auto blobfile = getBlobFile(stat->id); + LOG_FMT_TRACE(log, "Truncate empty blob file [blob_id={}] to 0.", stat->id); + blobfile->truncate(right_margin); + blobstore_gc_info.appendToTruncatedBlob(stat->id, stat->sm_valid_rate); continue; } @@ -1468,7 +1483,7 @@ void BlobStore::BlobStats::BlobStat::recalculateSpaceMap() const auto & [total_size, valid_size] = smap->getSizes(); sm_total_size = total_size; sm_valid_size = valid_size; - sm_valid_rate = valid_size * 1.0 / total_size; + sm_valid_rate = total_size == 0 ? 0.0 : valid_size * 1.0 / total_size; recalculateCapacity(); } diff --git a/dbms/src/Storages/Page/V3/BlobStore.h b/dbms/src/Storages/Page/V3/BlobStore.h index e527eb0f3bf..16c775d0667 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.h +++ b/dbms/src/Storages/Page/V3/BlobStore.h @@ -107,7 +107,7 @@ class BlobStore : private Allocator UInt64 sm_max_caps = 0; UInt64 sm_total_size = 0; UInt64 sm_valid_size = 0; - double sm_valid_rate = 1.0; + double sm_valid_rate = 0.0; public: BlobStat(BlobFileId id_, SpaceMap::SpaceMapType sm_type, UInt64 sm_max_caps_, BlobStatType type_ = BlobStatType::NORMAL) From ca3e1c6be883b479f7169e5ce4a317266010e66c Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger <60744015+Lloyd-Pottiger@users.noreply.github.com> Date: Fri, 27 May 2022 19:24:48 +0800 Subject: [PATCH 004/104] Fix an invalid default value cause bootstrap failed (#4916) close pingcap/tiflash#3157 --- dbms/src/Storages/Transaction/TiDB.cpp | 29 ++++++++++++++++++++++++-- 1 file changed, 27 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/Transaction/TiDB.cpp b/dbms/src/Storages/Transaction/TiDB.cpp index 763dcac39fc..3810e25372f 100644 --- a/dbms/src/Storages/Transaction/TiDB.cpp +++ b/dbms/src/Storages/Transaction/TiDB.cpp @@ -26,6 +26,8 @@ #include #include +#include + namespace DB { namespace ErrorCodes @@ -110,14 +112,28 @@ Field ColumnInfo::defaultValueToField() const } switch (tp) { - // TODO: Consider unsigned? // Integer Type. case TypeTiny: case TypeShort: case TypeLong: case TypeLongLong: case TypeInt24: - return value.convert(); + { + // In c++, cast a unsigned integer to signed integer will not change the value. + // like 9223372036854775808 which is larger than the maximum value of Int64, + // static_cast(static_cast(9223372036854775808)) == 9223372036854775808 + // so we don't need consider unsigned here. + try + { + return value.convert(); + } + catch (...) + { + // due to https://github.com/pingcap/tidb/issues/34881 + // we do this to avoid exception in older version of TiDB. + return static_cast(std::llround(value.convert())); + } + } case TypeBit: { // TODO: We shall use something like `orig_default_bit`, which will never change once created, @@ -615,6 +631,8 @@ catch (const Poco::Exception & e) /////////////////////// IndexColumnInfo::IndexColumnInfo(Poco::JSON::Object::Ptr json) + : offset() + , length() { deserialize(json); } @@ -664,6 +682,13 @@ catch (const Poco::Exception & e) /////////////////////// IndexInfo::IndexInfo(Poco::JSON::Object::Ptr json) + : id() + , state() + , index_type() + , is_unique() + , is_primary() + , is_invisible() + , is_global() { deserialize(json); } From d1ecebd92d8d88a224f28125d2c31c4f17d3c069 Mon Sep 17 00:00:00 2001 From: JaySon Date: Fri, 27 May 2022 19:58:48 +0800 Subject: [PATCH 005/104] PageStorage: Report the file usage of BlobStore (#4999) close pingcap/tiflash#4922 --- dbms/src/Interpreters/AsynchronousMetrics.cpp | 32 + dbms/src/Interpreters/AsynchronousMetrics.h | 5 + dbms/src/Server/Server.cpp | 11 +- dbms/src/Storages/DeltaMerge/StoragePool.h | 2 + dbms/src/Storages/Page/FileUsage.h | 29 + dbms/src/Storages/Page/PageStorage.cpp | 18 + dbms/src/Storages/Page/PageStorage.h | 9 + dbms/src/Storages/Page/Snapshot.h | 2 +- dbms/src/Storages/Page/V3/BlobStore.cpp | 45 +- dbms/src/Storages/Page/V3/BlobStore.h | 14 +- dbms/src/Storages/Page/V3/PageStorageImpl.cpp | 5 + dbms/src/Storages/Page/V3/PageStorageImpl.h | 2 + dbms/src/Storages/Transaction/KVStore.h | 5 + .../Storages/Transaction/RegionPersister.cpp | 6 + .../Storages/Transaction/RegionPersister.h | 3 + metrics/grafana/tiflash_summary.json | 986 ++++++++++-------- 16 files changed, 721 insertions(+), 453 deletions(-) create mode 100644 dbms/src/Storages/Page/FileUsage.h diff --git a/dbms/src/Interpreters/AsynchronousMetrics.cpp b/dbms/src/Interpreters/AsynchronousMetrics.cpp index e96d57e0370..8095fbb0e59 100644 --- a/dbms/src/Interpreters/AsynchronousMetrics.cpp +++ b/dbms/src/Interpreters/AsynchronousMetrics.cpp @@ -21,8 +21,12 @@ #include #include #include +#include #include +#include #include +#include +#include #include #include @@ -125,6 +129,26 @@ static void calculateMaxAndSum(Max & max, Sum & sum, T x) max = x; } +FileUsageStatistics AsynchronousMetrics::getPageStorageFileUsage() +{ + // Get from RegionPersister + auto & tmt = context.getTMTContext(); + auto & kvstore = tmt.getKVStore(); + FileUsageStatistics usage = kvstore->getFileUsageStatistics(); + + // Get the blob file status from all PS V3 instances + if (auto global_storage_pool = context.getGlobalStoragePool(); global_storage_pool != nullptr) + { + const auto log_usage = global_storage_pool->log_storage->getFileUsageStatistics(); + const auto meta_usage = global_storage_pool->meta_storage->getFileUsageStatistics(); + const auto data_usage = global_storage_pool->data_storage->getFileUsageStatistics(); + + usage.total_file_num += log_usage.total_file_num + meta_usage.total_file_num + data_usage.total_file_num; + usage.total_disk_size += log_usage.total_disk_size + meta_usage.total_disk_size + data_usage.total_disk_size; + usage.total_valid_size += log_usage.total_valid_size + meta_usage.total_valid_size + data_usage.total_valid_size; + } + return usage; +} void AsynchronousMetrics::update() { @@ -147,6 +171,7 @@ void AsynchronousMetrics::update() set("Uptime", context.getUptimeSeconds()); { + // Get the snapshot status from all delta tree tables auto databases = context.getDatabases(); double max_dt_stable_oldest_snapshot_lifetime = 0.0; @@ -177,6 +202,13 @@ void AsynchronousMetrics::update() set("MaxDTBackgroundTasksLength", max_dt_background_tasks_length); } + { + const FileUsageStatistics usage = getPageStorageFileUsage(); + set("BlobFileNums", usage.total_file_num); + set("BlobDiskBytes", usage.total_disk_size); + set("BlobValidBytes", usage.total_valid_size); + } + #if USE_TCMALLOC { /// tcmalloc related metrics. Remove if you switch to different allocator. diff --git a/dbms/src/Interpreters/AsynchronousMetrics.h b/dbms/src/Interpreters/AsynchronousMetrics.h index 5de328601a6..536e6a6b6f6 100644 --- a/dbms/src/Interpreters/AsynchronousMetrics.h +++ b/dbms/src/Interpreters/AsynchronousMetrics.h @@ -14,6 +14,8 @@ #pragma once +#include + #include #include #include @@ -47,6 +49,9 @@ class AsynchronousMetrics /// Returns copy of all values. Container getValues() const; +private: + FileUsageStatistics getPageStorageFileUsage(); + private: Context & context; diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 705b8a533f3..fcf820eb958 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -464,7 +464,7 @@ struct RaftStoreProxyRunner : boost::noncopyable } RunRaftStoreProxyParms parms; - pthread_t thread; + pthread_t thread{}; Poco::Logger * log; }; @@ -477,6 +477,11 @@ void initStores(Context & global_context, Poco::Logger * log, bool lazily_init_s int err_cnt = 0; for (auto & [table_id, storage] : storages) { + // This will skip the init of storages that do not contain any data. TiFlash now sync the schema and + // create all tables regardless the table have define TiFlash replica or not, so there may be lots + // of empty tables in TiFlash. + // Note that we still need to init stores that contains data (defined by the stable dir of this storage + // is exist), or the data used size reported to PD is not correct. try { init_cnt += storage->initStoreIfDataDirExist() ? 1 : 0; @@ -498,6 +503,7 @@ void initStores(Context & global_context, Poco::Logger * log, bool lazily_init_s if (lazily_init_store) { LOG_FMT_INFO(log, "Lazily init store."); + // apply the inited in another thread to shorten the start time of TiFlash std::thread(do_init_stores).detach(); } else @@ -1149,7 +1155,7 @@ int Server::main(const std::vector & /*args*/) /// Try to increase limit on number of open files. { - rlimit rlim; + rlimit rlim{}; if (getrlimit(RLIMIT_NOFILE, &rlim)) throw Poco::Exception("Cannot getrlimit"); @@ -1437,6 +1443,7 @@ int Server::main(const std::vector & /*args*/) } /// This object will periodically calculate some metrics. + /// should init after `createTMTContext` cause we collect some data from the TiFlash context object. AsynchronousMetrics async_metrics(*global_context); attachSystemTablesAsync(*global_context->getDatabase("system"), async_metrics); diff --git a/dbms/src/Storages/DeltaMerge/StoragePool.h b/dbms/src/Storages/DeltaMerge/StoragePool.h index d05454a5431..77684ea46cb 100644 --- a/dbms/src/Storages/DeltaMerge/StoragePool.h +++ b/dbms/src/Storages/DeltaMerge/StoragePool.h @@ -28,6 +28,7 @@ struct Settings; class Context; class StoragePathPool; class StableDiskDelegator; +class AsynchronousMetrics; namespace DM { @@ -50,6 +51,7 @@ class GlobalStoragePool : private boost::noncopyable void restore(); friend class StoragePool; + friend class ::DB::AsynchronousMetrics; // GC immediately // Only used on dbgFuncMisc diff --git a/dbms/src/Storages/Page/FileUsage.h b/dbms/src/Storages/Page/FileUsage.h new file mode 100644 index 00000000000..6319f4a4acf --- /dev/null +++ b/dbms/src/Storages/Page/FileUsage.h @@ -0,0 +1,29 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once +#include + +#include + +namespace DB +{ +struct FileUsageStatistics +{ + size_t total_disk_size = 0; + size_t total_valid_size = 0; + size_t total_file_num = 0; +}; + +} // namespace DB diff --git a/dbms/src/Storages/Page/PageStorage.cpp b/dbms/src/Storages/Page/PageStorage.cpp index 6e1addae093..d8b767a6c15 100644 --- a/dbms/src/Storages/Page/PageStorage.cpp +++ b/dbms/src/Storages/Page/PageStorage.cpp @@ -66,6 +66,8 @@ class PageReaderImpl : private boost::noncopyable // Get some statistics of all living snapshots and the oldest living snapshot. virtual SnapshotsStatistics getSnapshotsStat() const = 0; + virtual FileUsageStatistics getFileUsageStatistics() const = 0; + virtual void traverse(const std::function & acceptor, bool only_v2, bool only_v3) const = 0; }; @@ -137,6 +139,11 @@ class PageReaderImplNormal : public PageReaderImpl storage->traverse(acceptor, nullptr); } + FileUsageStatistics getFileUsageStatistics() const override + { + return storage->getFileUsageStatistics(); + } + private: NamespaceId ns_id; PageStoragePtr storage; @@ -294,6 +301,11 @@ class PageReaderImplMixed : public PageReaderImpl return statistics_total; } + FileUsageStatistics getFileUsageStatistics() const override + { + return storage_v3->getFileUsageStatistics(); + } + void traverse(const std::function & acceptor, bool only_v2, bool only_v3) const override { // Used by RegionPersister::restore @@ -424,6 +436,12 @@ SnapshotsStatistics PageReader::getSnapshotsStat() const return impl->getSnapshotsStat(); } + +FileUsageStatistics PageReader::getFileUsageStatistics() const +{ + return impl->getFileUsageStatistics(); +} + void PageReader::traverse(const std::function & acceptor, bool only_v2, bool only_v3) const { impl->traverse(acceptor, only_v2, only_v3); diff --git a/dbms/src/Storages/Page/PageStorage.h b/dbms/src/Storages/Page/PageStorage.h index cec6e297d0e..0059c0570c1 100644 --- a/dbms/src/Storages/Page/PageStorage.h +++ b/dbms/src/Storages/Page/PageStorage.h @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -251,6 +252,12 @@ class PageStorage : private boost::noncopyable // Get some statistics of all living snapshots and the oldest living snapshot. virtual SnapshotsStatistics getSnapshotsStat() const = 0; + virtual FileUsageStatistics getFileUsageStatistics() const + { + // return all zeros by default + return FileUsageStatistics{}; + } + virtual size_t getNumberOfPages() = 0; virtual std::set getAliveExternalPageIds(NamespaceId ns_id) = 0; @@ -380,6 +387,8 @@ class PageReader : private boost::noncopyable // Get some statistics of all living snapshots and the oldest living snapshot. SnapshotsStatistics getSnapshotsStat() const; + FileUsageStatistics getFileUsageStatistics() const; + void traverse(const std::function & acceptor, bool only_v2 = false, bool only_v3 = false) const; private: diff --git a/dbms/src/Storages/Page/Snapshot.h b/dbms/src/Storages/Page/Snapshot.h index 77e68f1b054..073fc0a2830 100644 --- a/dbms/src/Storages/Page/Snapshot.h +++ b/dbms/src/Storages/Page/Snapshot.h @@ -61,7 +61,7 @@ class PageStorageSnapshotMixed : public PageStorageSnapshot }; using PageStorageSnapshotMixedPtr = std::shared_ptr; -static inline PageStorageSnapshotMixedPtr +inline PageStorageSnapshotMixedPtr toConcreteMixedSnapshot(const PageStorageSnapshotPtr & ptr) { return std::static_pointer_cast(ptr); diff --git a/dbms/src/Storages/Page/V3/BlobStore.cpp b/dbms/src/Storages/Page/V3/BlobStore.cpp index f03bc5bcf73..d5f71841b91 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.cpp +++ b/dbms/src/Storages/Page/V3/BlobStore.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -66,7 +67,7 @@ using ChecksumClass = Digest::CRC64; * BlobStore methods * *********************/ -BlobStore::BlobStore(String storage_name, const FileProviderPtr & file_provider_, PSDiskDelegatorPtr delegator_, BlobStore::Config config_) +BlobStore::BlobStore(String storage_name, const FileProviderPtr & file_provider_, PSDiskDelegatorPtr delegator_, const BlobStore::Config & config_) : delegator(std::move(delegator_)) , file_provider(file_provider_) , config(config_) @@ -115,6 +116,38 @@ void BlobStore::registerPaths() } } +FileUsageStatistics BlobStore::getFileUsageStatistics() const +{ + FileUsageStatistics usage; + + // Get a copy of stats map to avoid the big lock on stats map + const auto stats_list = blob_stats.getStats(); + + for (const auto & [path, stats] : stats_list) + { + (void)path; + for (const auto & stat : stats) + { + // We can access to these type without any locking. + if (stat->isReadOnly() || stat->isBigBlob()) + { + usage.total_disk_size += stat->sm_total_size; + usage.total_valid_size += stat->sm_valid_size; + } + else + { + // Else the stat may being updated, acquire a lock to avoid data race. + auto lock = stat->lock(); + usage.total_disk_size += stat->sm_total_size; + usage.total_valid_size += stat->sm_valid_size; + } + } + usage.total_file_num += stats.size(); + } + + return usage; +} + PageEntriesEdit BlobStore::handleLargeWrite(DB::WriteBatch & wb, const WriteLimiterPtr & write_limiter) { auto ns_id = wb.getNamespaceId(); @@ -872,6 +905,7 @@ struct BlobStoreGCInfo std::vector BlobStore::getGCStats() { + // Get a copy of stats map to avoid the big lock on stats map const auto stats_list = blob_stats.getStats(); std::vector blob_need_gc; BlobStoreGCInfo blobstore_gc_info; @@ -1211,7 +1245,7 @@ BlobStatPtr BlobStore::BlobStats::createStat(BlobFileId blob_file_id, const std: // New blob file id won't bigger than roll_id if (blob_file_id > roll_id) { - throw Exception(fmt::format("BlobStats won't create [blob_id={}], which is bigger than [RollMaxId={}]", + throw Exception(fmt::format("BlobStats won't create [blob_id={}], which is bigger than [roll_id={}]", blob_file_id, roll_id), ErrorCodes::LOGICAL_ERROR); @@ -1274,8 +1308,7 @@ BlobStatPtr BlobStore::BlobStats::createBigPageStatNotChecking(BlobFileId blob_f BlobStatPtr stat = std::make_shared( blob_file_id, SpaceMap::SpaceMapType::SMAP64_BIG, - config.file_limit_size, - BlobStatType::BIG_BLOB); + config.file_limit_size); PageFileIdAndLevel id_lvl{blob_file_id, 0}; stats_map[delegator->choosePath(id_lvl)].emplace_back(stat); @@ -1453,7 +1486,7 @@ bool BlobStore::BlobStats::BlobStat::removePosFromStat(BlobFileOffset offset, si if (!smap->markFree(offset, buf_size)) { smap->logDebugString(); - throw Exception(fmt::format("Remove postion from BlobStat failed, [offset={} , buf_size={}, blob_id={}] is invalid.", + throw Exception(fmt::format("Remove postion from BlobStat failed, invalid position [offset={}] [buf_size={}] [blob_id={}]", offset, buf_size, id), @@ -1470,7 +1503,7 @@ void BlobStore::BlobStats::BlobStat::restoreSpaceMap(BlobFileOffset offset, size if (!smap->markUsed(offset, buf_size)) { smap->logDebugString(); - throw Exception(fmt::format("Restore postion from BlobStat failed, [offset={}] [buf_size={}] [blob_id={}] is used or subspan is used", + throw Exception(fmt::format("Restore postion from BlobStat failed, the space/subspace is already being used [offset={}] [buf_size={}] [blob_id={}]", offset, buf_size, id), diff --git a/dbms/src/Storages/Page/V3/BlobStore.h b/dbms/src/Storages/Page/V3/BlobStore.h index 16c775d0667..5a3e98400d1 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.h +++ b/dbms/src/Storages/Page/V3/BlobStore.h @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -110,12 +111,17 @@ class BlobStore : private Allocator double sm_valid_rate = 0.0; public: - BlobStat(BlobFileId id_, SpaceMap::SpaceMapType sm_type, UInt64 sm_max_caps_, BlobStatType type_ = BlobStatType::NORMAL) + BlobStat(BlobFileId id_, SpaceMap::SpaceMapType sm_type, UInt64 sm_max_caps_) : id(id_) - , type(type_) + , type(BlobStatType::NORMAL) , smap(SpaceMap::createSpaceMap(sm_type, 0, sm_max_caps_)) , sm_max_caps(sm_max_caps_) { + if (sm_type == SpaceMap::SpaceMapType::SMAP64_BIG) + { + type = BlobStatType::BIG_BLOB; + } + // Won't create read-only blob by default. assert(type != BlobStatType::READ_ONLY); } @@ -246,10 +252,12 @@ class BlobStore : private Allocator std::map> stats_map; }; - BlobStore(String storage_name, const FileProviderPtr & file_provider_, PSDiskDelegatorPtr delegator_, BlobStore::Config config); + BlobStore(String storage_name, const FileProviderPtr & file_provider_, PSDiskDelegatorPtr delegator_, const BlobStore::Config & config); void registerPaths(); + FileUsageStatistics getFileUsageStatistics() const; + std::vector getGCStats(); PageEntriesEdit gc(std::map & entries_need_gc, diff --git a/dbms/src/Storages/Page/V3/PageStorageImpl.cpp b/dbms/src/Storages/Page/V3/PageStorageImpl.cpp index cfa07199637..a568bb5087f 100644 --- a/dbms/src/Storages/Page/V3/PageStorageImpl.cpp +++ b/dbms/src/Storages/Page/V3/PageStorageImpl.cpp @@ -80,6 +80,11 @@ DB::PageStorage::SnapshotPtr PageStorageImpl::getSnapshot(const String & tracing return page_directory->createSnapshot(tracing_id); } +FileUsageStatistics PageStorageImpl::getFileUsageStatistics() const +{ + return blob_store.getFileUsageStatistics(); +} + SnapshotsStatistics PageStorageImpl::getSnapshotsStat() const { return page_directory->getSnapshotsStat(); diff --git a/dbms/src/Storages/Page/V3/PageStorageImpl.h b/dbms/src/Storages/Page/V3/PageStorageImpl.h index 50d160e81da..f49601ce2ad 100644 --- a/dbms/src/Storages/Page/V3/PageStorageImpl.h +++ b/dbms/src/Storages/Page/V3/PageStorageImpl.h @@ -72,6 +72,8 @@ class PageStorageImpl : public DB::PageStorage SnapshotsStatistics getSnapshotsStat() const override; + FileUsageStatistics getFileUsageStatistics() const override; + size_t getNumberOfPages() override; std::set getAliveExternalPageIds(NamespaceId ns_id) override; diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index 9d30f249e60..ef851d67958 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -162,6 +162,11 @@ class KVStore final : private boost::noncopyable ~KVStore(); + FileUsageStatistics getFileUsageStatistics() const + { + return region_persister.getFileUsageStatistics(); + } + private: friend class MockTiDB; friend struct MockTiDBTable; diff --git a/dbms/src/Storages/Transaction/RegionPersister.cpp b/dbms/src/Storages/Transaction/RegionPersister.cpp index c3db88daece..7ce52c6caa1 100644 --- a/dbms/src/Storages/Transaction/RegionPersister.cpp +++ b/dbms/src/Storages/Transaction/RegionPersister.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -379,4 +380,9 @@ bool RegionPersister::gc() return stable_page_storage->gc(); } +FileUsageStatistics RegionPersister::getFileUsageStatistics() const +{ + return page_reader->getFileUsageStatistics(); +} + } // namespace DB diff --git a/dbms/src/Storages/Transaction/RegionPersister.h b/dbms/src/Storages/Transaction/RegionPersister.h index f2828add202..a6b400345f8 100644 --- a/dbms/src/Storages/Transaction/RegionPersister.h +++ b/dbms/src/Storages/Transaction/RegionPersister.h @@ -16,6 +16,7 @@ #include #include +#include #include #include #include @@ -57,6 +58,8 @@ class RegionPersister final : private boost::noncopyable PageStorage::Config getPageStorageSettings() const; + FileUsageStatistics getFileUsageStatistics() const; + #ifndef DBMS_PUBLIC_GTEST private: #endif diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index a6d1abac46f..f899a47ed10 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -33,12 +33,6 @@ "id": "prometheus", "name": "Prometheus", "version": "1.0.0" - }, - { - "type": "panel", - "id": "timeseries", - "name": "Time series", - "version": "" } ], "annotations": { @@ -58,7 +52,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1652861766192, + "iteration": 1653635389238, "links": [], "panels": [ { @@ -5336,14 +5330,30 @@ "align": false, "alignLevel": null } - }, + } + ], + "repeat": null, + "title": "Storage", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 5 + }, + "id": 119, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The states of BlobStore (an internal component of storage engine)", + "description": "The states of BlobStore (an internal component of PageStorage)", "fieldConfig": { "defaults": {}, "overrides": [] @@ -5354,20 +5364,20 @@ "h": 8, "w": 12, "x": 0, - "y": 69 + "y": 6 }, "hiddenSeries": false, "id": 85, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, - "current": false, + "current": true, "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -5383,11 +5393,11 @@ "renderer": "flot", "seriesOverrides": [ { - "alias": "/^BlobAllocated/", + "alias": "/^allocated/", "yaxis": 1 }, { - "alias": "/^BlobExpandRate/", + "alias": "/^expand_rate/", "yaxis": 2 } ], @@ -5402,7 +5412,7 @@ "hide": false, "interval": "", "intervalFactor": 2, - "legendFormat": "BlobAllocated-{{instance}}", + "legendFormat": "allocated-{{instance}}", "refId": "A" }, { @@ -5412,7 +5422,7 @@ "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "BlobExpandRate-{{instance}}", + "legendFormat": "expand_rate-{{instance}}", "refId": "B" } ], @@ -5420,7 +5430,7 @@ "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "BlobStore Status", + "title": "PageStorage Blob Status", "tooltip": { "shared": true, "sort": 0, @@ -5457,23 +5467,7 @@ "align": false, "alignLevel": null } - } - ], - "repeat": null, - "title": "Storage", - "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 5 - }, - "id": 64, - "panels": [ + }, { "aliasColors": {}, "bars": false, @@ -5481,42 +5475,40 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The throughput of write and delta's background management", + "description": "The disk usage of PageStorage instances in each TiFlash node", + "editable": true, + "error": false, "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, + "grid": {}, "gridPos": { - "h": 9, - "w": 24, - "x": 0, + "h": 8, + "w": 12, + "x": 12, "y": 6 }, - "height": "", "hiddenSeries": false, - "id": 70, + "id": 128, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, "sideWidth": null, - "sort": "current", - "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -5525,11 +5517,14 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, "seriesOverrides": [ { - "alias": "/total/", + "alias": "/^valid_rate/", "yaxis": 2 + }, + { + "alias": "/size/", + "linewidth": 3 } ], "spaceLength": 10, @@ -5538,47 +5533,51 @@ "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}[1m]))", - "format": "time_series", + "expr": "tiflash_system_asynchronous_metric_BlobDiskBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "throughput_write+ingest", - "refId": "A", - "step": 10 + "intervalFactor": 2, + "legendFormat": "disk_size-{{instance}}", + "refId": "A" }, { - "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"write|ingest\"}[1m]))", - "format": "time_series", - "intervalFactor": 1, - "legendFormat": "throughput_delta-management", + "exemplar": true, + "expr": "sum(tiflash_system_asynchronous_metric_BlobValidBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "valid_size-{{instance}}", "refId": "B" }, { "exemplar": true, - "expr": "sum(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"})", - "format": "time_series", + "expr": "sum((tiflash_system_asynchronous_metric_BlobValidBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) / (tiflash_system_asynchronous_metric_BlobDiskBytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"})) by (instance)", + "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "total_write+ingest", + "legendFormat": "valid_rate-{{instance}}", "refId": "C" }, { - "expr": "sum(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"write|ingest\"})", + "exemplar": true, + "expr": "sum(tiflash_system_asynchronous_metric_BlobFileNums{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "total_delta-management", - "refId": "D" + "hide": true, + "interval": "", + "intervalFactor": 2, + "legendFormat": "num_file-{{instance}}", + "refId": "E", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write & Delta Management Throughput", + "title": "PageStorage Disk Usage", "tooltip": { + "msResolution": false, "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -5591,7 +5590,7 @@ }, "yaxes": [ { - "format": "binBps", + "format": "bytes", "label": null, "logBase": 1, "max": null, @@ -5599,11 +5598,11 @@ "show": true }, { - "format": "bytes", + "format": "percentunit", "label": null, "logBase": 1, - "max": null, - "min": null, + "max": "1.1", + "min": "0", "show": true } ], @@ -5618,29 +5617,34 @@ "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The stall duration of write and delete range", + "decimals": 1, + "description": "The number of files of PageStorage instances in each TiFlash node", + "editable": true, + "error": false, "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, + "grid": {}, "gridPos": { "h": 8, - "w": 24, + "w": 12, "x": 0, - "y": 15 + "y": 14 }, "hiddenSeries": false, - "id": 62, + "id": 129, "legend": { "alignAsTable": true, "avg": false, "current": true, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, + "sideWidth": null, "total": false, "values": true }, @@ -5656,40 +5660,32 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "99-delta_merge", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_write_stall_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type, instance))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "99-{{type}}-{{instance}}", - "refId": "B" - }, - { - "expr": "histogram_quantile(1, sum(rate(tiflash_storage_write_stall_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type, instance))", + "exemplar": true, + "expr": "sum(tiflash_system_asynchronous_metric_BlobFileNums{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "format": "time_series", - "intervalFactor": 1, - "legendFormat": "max-{{type}}-{{instance}}", - "refId": "A" + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "num_file-{{instance}}", + "refId": "A", + "step": 10 } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write Stall Duration", + "title": "PageStorage File Num", "tooltip": { + "msResolution": false, "shared": true, - "sort": 0, + "sort": 2, "value_type": "individual" }, "type": "graph", @@ -5702,7 +5698,7 @@ }, "yaxes": [ { - "format": "s", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5710,10 +5706,10 @@ "show": true }, { - "format": "s", + "format": "percentunit", "label": null, "logBase": 1, - "max": null, + "max": "1.1", "min": "0", "show": true } @@ -5730,29 +5726,29 @@ "dashes": false, "datasource": "${DS_TEST-CLUSTER}", "decimals": 1, - "description": "The throughput of write by instance", + "description": "The number of tables running under different mode in DeltaTree", + "editable": true, + "error": false, "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, + "grid": {}, "gridPos": { - "h": 9, - "w": 24, - "x": 0, - "y": 23 + "h": 8, + "w": 12, + "x": 12, + "y": 14 }, - "height": "", "hiddenSeries": false, - "id": 89, + "id": 123, "legend": { "alignAsTable": true, "avg": false, "current": true, - "hideEmpty": false, - "hideZero": false, - "max": true, + "max": false, "min": false, "rightSide": true, "show": true, @@ -5765,7 +5761,7 @@ "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null", + "nullPointMode": "null as zero", "options": { "alertThreshold": true }, @@ -5774,43 +5770,45 @@ "pointradius": 5, "points": false, "renderer": "flot", - "repeatedByRow": true, - "seriesOverrides": [ - { - "alias": "/total/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, "steppedLine": false, "targets": [ { "exemplar": true, - "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write\"}[1m])) by (instance)", + "expr": "sum(tiflash_system_current_metric_StoragePoolV2Only{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "format": "time_series", - "hide": false, "interval": "", - "intervalFactor": 1, - "legendFormat": "throughput_write-{{instance}}", + "intervalFactor": 2, + "legendFormat": "{{instance}}-OnlyV2", "refId": "A", "step": 10 }, { "exemplar": true, - "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"ingest\"}[1m])) by (instance)", + "expr": "sum(tiflash_system_current_metric_StoragePoolV3Only{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", "hide": false, "interval": "", - "legendFormat": "throughput_ingest-{{instance}}", + "legendFormat": "{{instance}}-OnlyV3", "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(tiflash_system_current_metric_StoragePoolMixMode{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "{{instance}}-MixMode", + "refId": "C" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Write Throughput By Instance", + "title": "StoragePool Runmode", "tooltip": { + "msResolution": false, "shared": true, "sort": 0, "value_type": "individual" @@ -5825,7 +5823,7 @@ }, "yaxes": [ { - "format": "binBps", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5833,7 +5831,7 @@ "show": true }, { - "format": "bytes", + "format": "short", "label": null, "logBase": 1, "max": null, @@ -5845,43 +5843,65 @@ "align": false, "alignLevel": null } - }, + } + ], + "title": "StoragePool", + "type": "row" + }, + { + "collapsed": true, + "datasource": null, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 6 + }, + "id": 64, + "panels": [ { "aliasColors": {}, "bars": false, "dashLength": 10, "dashes": false, "datasource": "${DS_TEST-CLUSTER}", - "description": "The total count of different kinds of commands received", + "decimals": 1, + "description": "The throughput of write and delta's background management", "fieldConfig": { "defaults": {}, "overrides": [] }, - "fill": 0, + "fill": 1, "fillGradient": 0, "gridPos": { "h": 9, "w": 24, "x": 0, - "y": 32 + "y": 71 }, + "height": "", "hiddenSeries": false, - "id": 90, + "id": 70, "legend": { "alignAsTable": true, "avg": false, "current": true, + "hideEmpty": false, + "hideZero": false, "max": true, "min": false, "rightSide": true, "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, "total": false, "values": true }, "lines": true, "linewidth": 1, "links": [], - "nullPointMode": "null as zero", + "nullPointMode": "null", "options": { "alertThreshold": true }, @@ -5890,9 +5910,10 @@ "pointradius": 5, "points": false, "renderer": "flot", + "repeatedByRow": true, "seriesOverrides": [ { - "alias": "/delete_range|ingest/", + "alias": "/total/", "yaxis": 2 } ], @@ -5901,8 +5922,372 @@ "steppedLine": false, "targets": [ { - "expr": "sum(rate(tiflash_system_profile_event_DMWriteBlock{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance, type)", - "format": "time_series", + "exemplar": true, + "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"}[1m]))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "throughput_write+ingest", + "refId": "A", + "step": 10 + }, + { + "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"write|ingest\"}[1m]))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "throughput_delta-management", + "refId": "B" + }, + { + "exemplar": true, + "expr": "sum(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write|ingest\"})", + "format": "time_series", + "interval": "", + "intervalFactor": 1, + "legendFormat": "total_write+ingest", + "refId": "C" + }, + { + "expr": "sum(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"write|ingest\"})", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "total_delta-management", + "refId": "D" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Write & Delta Management Throughput", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "binBps", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The stall duration of write and delete range", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 8, + "w": 24, + "x": 0, + "y": 80 + }, + "hiddenSeries": false, + "id": 62, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "99-delta_merge", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_write_stall_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type, instance))", + "format": "time_series", + "hide": true, + "intervalFactor": 1, + "legendFormat": "99-{{type}}-{{instance}}", + "refId": "B" + }, + { + "expr": "histogram_quantile(1, sum(rate(tiflash_storage_write_stall_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le, type, instance))", + "format": "time_series", + "intervalFactor": 1, + "legendFormat": "max-{{type}}-{{instance}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Write Stall Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "decimals": 1, + "description": "The throughput of write by instance", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 1, + "fillGradient": 0, + "gridPos": { + "h": 9, + "w": 24, + "x": 0, + "y": 88 + }, + "height": "", + "hiddenSeries": false, + "id": 89, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "hideEmpty": false, + "hideZero": false, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sideWidth": null, + "sort": "current", + "sortDesc": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "repeatedByRow": true, + "seriesOverrides": [ + { + "alias": "/total/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"write\"}[1m])) by (instance)", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 1, + "legendFormat": "throughput_write-{{instance}}", + "refId": "A", + "step": 10 + }, + { + "exemplar": true, + "expr": "sum(rate(tiflash_storage_throughput_bytes{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"ingest\"}[1m])) by (instance)", + "hide": false, + "interval": "", + "legendFormat": "throughput_ingest-{{instance}}", + "refId": "B" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Write Throughput By Instance", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "format": "binBps", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true + }, + { + "format": "bytes", + "label": null, + "logBase": 1, + "max": null, + "min": null, + "show": true + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "The total count of different kinds of commands received", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 9, + "w": 24, + "x": 0, + "y": 97 + }, + "hiddenSeries": false, + "id": 90, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [ + { + "alias": "/delete_range|ingest/", + "yaxis": 2 + } + ], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ + { + "expr": "sum(rate(tiflash_system_profile_event_DMWriteBlock{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (instance, type)", + "format": "time_series", "hide": false, "intervalFactor": 1, "legendFormat": "write block-{{instance}}", @@ -5970,7 +6355,7 @@ "h": 1, "w": 24, "x": 0, - "y": 6 + "y": 7 }, "id": 34, "panels": [ @@ -5990,7 +6375,7 @@ "h": 7, "w": 12, "x": 0, - "y": 7 + "y": 8 }, "hiddenSeries": false, "id": 35, @@ -6088,7 +6473,7 @@ "h": 7, "w": 12, "x": 12, - "y": 7 + "y": 8 }, "hiddenSeries": false, "id": 36, @@ -6206,7 +6591,7 @@ "h": 7, "w": 12, "x": 0, - "y": 14 + "y": 15 }, "hiddenSeries": false, "id": 37, @@ -6340,7 +6725,7 @@ "h": 7, "w": 12, "x": 12, - "y": 14 + "y": 15 }, "hiddenSeries": false, "id": 75, @@ -6444,7 +6829,7 @@ "h": 7, "w": 24, "x": 0, - "y": 21 + "y": 22 }, "hiddenSeries": false, "id": 82, @@ -6597,7 +6982,7 @@ "h": 7, "w": 12, "x": 0, - "y": 28 + "y": 29 }, "heatmap": {}, "hideZeroBuckets": true, @@ -6667,7 +7052,7 @@ "h": 7, "w": 12, "x": 12, - "y": 28 + "y": 29 }, "heatmap": {}, "hideZeroBuckets": true, @@ -6737,7 +7122,7 @@ "h": 7, "w": 12, "x": 0, - "y": 35 + "y": 36 }, "heatmap": {}, "hideZeroBuckets": true, @@ -6807,7 +7192,7 @@ "h": 7, "w": 12, "x": 12, - "y": 35 + "y": 36 }, "heatmap": {}, "hideZeroBuckets": true, @@ -6871,7 +7256,7 @@ "h": 7, "w": 24, "x": 0, - "y": 42 + "y": 43 }, "height": "", "hiddenSeries": false, @@ -6985,7 +7370,7 @@ "h": 7, "w": 12, "x": 0, - "y": 49 + "y": 50 }, "heatmap": {}, "hideZeroBuckets": true, @@ -7054,7 +7439,7 @@ "h": 7, "w": 12, "x": 12, - "y": 49 + "y": 50 }, "heatmap": {}, "hideZeroBuckets": true, @@ -7124,7 +7509,7 @@ "h": 7, "w": 12, "x": 0, - "y": 56 + "y": 57 }, "heatmap": {}, "hideZeroBuckets": true, @@ -7194,7 +7579,7 @@ "h": 7, "w": 12, "x": 12, - "y": 56 + "y": 57 }, "heatmap": {}, "hideZeroBuckets": true, @@ -7264,7 +7649,7 @@ "h": 7, "w": 12, "x": 0, - "y": 63 + "y": 64 }, "heatmap": {}, "hideZeroBuckets": true, @@ -7330,7 +7715,7 @@ "h": 7, "w": 12, "x": 12, - "y": 63 + "y": 64 }, "hiddenSeries": false, "id": 91, @@ -7453,7 +7838,7 @@ "h": 1, "w": 24, "x": 0, - "y": 7 + "y": 8 }, "id": 95, "panels": [ @@ -7675,290 +8060,9 @@ ], "title": "Rough Set Filter Rate Histogram", "type": "row" - }, - { - "collapsed": true, - "datasource": null, - "gridPos": { - "h": 1, - "w": 24, - "x": 0, - "y": 8 - }, - "id": 119, - "panels": [ - { - "datasource": "${DS_TEST-CLUSTER}", - "description": "The Global StoragePool and KVStore Runmode", - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisLabel": "", - "axisPlacement": "auto", - "axisSoftMax": 5, - "barAlignment": 0, - "drawStyle": "line", - "fillOpacity": 0, - "gradientMode": "none", - "hideFrom": { - "graph": false, - "legend": false, - "tooltip": false - }, - "lineInterpolation": "linear", - "lineStyle": { - "fill": "solid" - }, - "lineWidth": 1, - "pointSize": 11, - "scaleDistribution": { - "type": "linear" - }, - "showPoints": "never", - "spanNulls": false - }, - "decimals": 0, - "mappings": [ - { - "from": "", - "id": 1, - "text": "ONLY_V2", - "to": "", - "type": 1, - "value": "1" - }, - { - "from": "", - "id": 2, - "text": "ONLY_V3", - "to": "", - "type": 1, - "value": "2" - }, - { - "from": "", - "id": 3, - "text": "MIX_MODE", - "to": "", - "type": 1, - "value": "3" - }, - { - "from": "", - "id": 4, - "text": " ", - "to": "", - "type": 1, - "value": "4" - }, - { - "from": "", - "id": 5, - "text": " ", - "to": "", - "type": 1, - "value": "5" - } - ], - "min": 0, - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - }, - { - "color": "red", - "value": 80 - } - ] - }, - "unit": "short" - }, - "overrides": [] - }, - "gridPos": { - "h": 8, - "w": 12, - "x": 0, - "y": 9 - }, - "id": 126, - "links": [], - "options": { - "graph": {}, - "legend": { - "calcs": [ - "lastNotNull" - ], - "displayMode": "table", - "placement": "right" - }, - "tooltipOptions": { - "mode": "multi" - } - }, - "pluginVersion": "7.5.11", - "targets": [ - { - "exemplar": true, - "expr": "tiflash_system_current_metric_GlobalStorageRunMode{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "format": "time_series", - "interval": "", - "intervalFactor": 2, - "legendFormat": "{{instance}}-GlobalRunMode", - "refId": "A", - "step": 10 - }, - { - "exemplar": false, - "expr": "tiflash_system_current_metric_RegionPersisterRunMode{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}", - "hide": false, - "interval": "", - "legendFormat": "{{instance}}-KVStoreRunMode", - "refId": "B" - } - ], - "timeFrom": null, - "timeShift": null, - "title": "Global Runmode", - "type": "timeseries" - }, - { - "aliasColors": {}, - "bars": false, - "dashLength": 10, - "dashes": false, - "datasource": "${DS_TEST-CLUSTER}", - "decimals": 1, - "description": "The StoragePool Runmode in DeltaMerge Storage", - "editable": true, - "error": false, - "fieldConfig": { - "defaults": {}, - "overrides": [] - }, - "fill": 0, - "fillGradient": 0, - "grid": {}, - "gridPos": { - "h": 8, - "w": 12, - "x": 12, - "y": 9 - }, - "hiddenSeries": false, - "id": 123, - "legend": { - "alignAsTable": true, - "avg": false, - "current": true, - "max": false, - "min": false, - "rightSide": true, - "show": true, - "sideWidth": null, - "sort": "current", - "sortDesc": true, - "total": false, - "values": true - }, - "lines": true, - "linewidth": 1, - "links": [], - "nullPointMode": "null as zero", - "options": { - "alertThreshold": true - }, - "percentage": false, - "pluginVersion": "7.5.11", - "pointradius": 5, - "points": false, - "renderer": "flot", - "seriesOverrides": [], - "spaceLength": 10, - "stack": false, - "steppedLine": false, - "targets": [ - { - "exemplar": true, - "expr": "sum(tiflash_system_current_metric_StoragePoolV2Only{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "format": "time_series", - "interval": "", - "intervalFactor": 2, - "legendFormat": "{{instance}}-OnlyV2", - "refId": "A", - "step": 10 - }, - { - "exemplar": true, - "expr": "sum(tiflash_system_current_metric_StoragePoolV3Only{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "{{instance}}-OnlyV3", - "refId": "B" - }, - { - "exemplar": true, - "expr": "sum(tiflash_system_current_metric_StoragePoolMixMode{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", - "hide": false, - "interval": "", - "legendFormat": "{{instance}}-MixMode", - "refId": "C" - } - ], - "thresholds": [], - "timeFrom": null, - "timeRegions": [], - "timeShift": null, - "title": "StoragePool Runmode", - "tooltip": { - "msResolution": false, - "shared": true, - "sort": 0, - "value_type": "individual" - }, - "type": "graph", - "xaxis": { - "buckets": null, - "mode": "time", - "name": null, - "show": true, - "values": [] - }, - "yaxes": [ - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": "0", - "show": true - }, - { - "format": "short", - "label": null, - "logBase": 1, - "max": null, - "min": null, - "show": true - } - ], - "yaxis": { - "align": false, - "alignLevel": null - } - } - ], - "title": "StoragePool", - "type": "row" } ], - "refresh": false, + "refresh": "30s", "schemaVersion": 27, "style": "dark", "tags": [], From 7dff46338e179a047127abb151d2db586a5b9cce Mon Sep 17 00:00:00 2001 From: hehechen Date: Fri, 27 May 2022 20:38:50 +0800 Subject: [PATCH 006/104] Fix unstable ut in the master branch (#5024) close pingcap/tiflash#5023 --- dbms/src/TestUtils/TiFlashTestEnv.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/TestUtils/TiFlashTestEnv.cpp b/dbms/src/TestUtils/TiFlashTestEnv.cpp index 264fd6009a3..bd05e5826db 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.cpp +++ b/dbms/src/TestUtils/TiFlashTestEnv.cpp @@ -88,7 +88,7 @@ Context TiFlashTestEnv::getContext(const DB::Settings & settings, Strings testda Context context = *global_context; context.setGlobalContext(*global_context); // Load `testdata_path` as path if it is set. - const String root_path = testdata_path.empty() ? getTemporaryPath() : testdata_path[0]; + const String root_path = testdata_path.empty() ? (DB::toString(getpid()) + "/" + getTemporaryPath()) : testdata_path[0]; if (testdata_path.empty()) testdata_path.push_back(root_path); context.setPath(root_path); From c8624a048783d6db172b000a9ec180848f6fb1c4 Mon Sep 17 00:00:00 2001 From: xufei Date: Mon, 30 May 2022 11:00:26 +0800 Subject: [PATCH 007/104] refine get type from decimal literal (#5014) close pingcap/tiflash#5007 --- dbms/src/Common/Decimal.h | 31 +++++- dbms/src/Core/Field.h | 15 ++- dbms/src/DataTypes/DataTypeDecimal.h | 2 +- dbms/src/DataTypes/FieldToDataType.h | 3 +- .../tests/gtest_decimal_literal_datatype.cpp | 100 ++++++++++++++++++ 5 files changed, 143 insertions(+), 8 deletions(-) create mode 100644 dbms/src/DataTypes/tests/gtest_decimal_literal_datatype.cpp diff --git a/dbms/src/Common/Decimal.h b/dbms/src/Common/Decimal.h index 71dcf82ea39..d4a3f382208 100644 --- a/dbms/src/Common/Decimal.h +++ b/dbms/src/Common/Decimal.h @@ -306,10 +306,6 @@ using Decimal64 = Decimal; using Decimal128 = Decimal; using Decimal256 = Decimal; -static constexpr PrecType minDecimalPrecision() -{ - return 1; -} template static constexpr PrecType maxDecimalPrecision() { @@ -336,6 +332,33 @@ constexpr PrecType maxDecimalPrecision() return 65; } +template +constexpr PrecType minDecimalPrecision() +{ + /// return a invalid value + return maxDecimalPrecision() + 1; +} +template <> +constexpr PrecType minDecimalPrecision() +{ + return 1; +} +template <> +constexpr PrecType minDecimalPrecision() +{ + return maxDecimalPrecision() + 1; +} +template <> +constexpr PrecType minDecimalPrecision() +{ + return maxDecimalPrecision() + 1; +} +template <> +constexpr PrecType minDecimalPrecision() +{ + return maxDecimalPrecision() + 1; +} + template struct PromoteType { diff --git a/dbms/src/Core/Field.h b/dbms/src/Core/Field.h index 3d2673a3412..038a9c3fe90 100644 --- a/dbms/src/Core/Field.h +++ b/dbms/src/Core/Field.h @@ -122,7 +122,20 @@ class DecimalField } if (cnt == 0) cnt = 1; - return cnt; + return std::max(cnt, scale); + } + + /// In TiFlash there are 4 subtype of decimal: + /// Decimal32, Decimal64, Decimal128 and Decimal256 + /// they are not compatible with each other. So a DecimalField + /// can not be inserted into a decimal column with DecimalType + /// getPrecWithCurrentDecimalType will return the prec that fit + /// current decimal type, that is to say, current DecimalField can be + /// inserted into a decimal column with type `Decimal(getPrecWithCurrentDecimalType, getScale)` + UInt32 getPrecWithCurrentDecimalType() const + { + auto raw_prec = getPrec(); + return std::max(raw_prec, minDecimalPrecision()); } template diff --git a/dbms/src/DataTypes/DataTypeDecimal.h b/dbms/src/DataTypes/DataTypeDecimal.h index c8f32c03117..47f6602c9a3 100644 --- a/dbms/src/DataTypes/DataTypeDecimal.h +++ b/dbms/src/DataTypes/DataTypeDecimal.h @@ -192,7 +192,7 @@ using DataTypeDecimal256 = DataTypeDecimal; inline DataTypePtr createDecimal(UInt64 prec, UInt64 scale) { - if (prec < minDecimalPrecision() || prec > maxDecimalPrecision()) + if (prec < minDecimalPrecision() || prec > maxDecimalPrecision()) throw Exception("Wrong precision:" + DB::toString(prec), ErrorCodes::ARGUMENT_OUT_OF_BOUND); if (static_cast(scale) > prec) diff --git a/dbms/src/DataTypes/FieldToDataType.h b/dbms/src/DataTypes/FieldToDataType.h index 9903172f860..9f4b80b7324 100644 --- a/dbms/src/DataTypes/FieldToDataType.h +++ b/dbms/src/DataTypes/FieldToDataType.h @@ -40,8 +40,7 @@ class FieldToDataType : public StaticVisitor template DataTypePtr operator()(const DecimalField & x) const { - PrecType prec = maxDecimalPrecision(); - return std::make_shared>(prec, x.getScale()); + return std::make_shared>(x.getPrecWithCurrentDecimalType(), x.getScale()); } }; diff --git a/dbms/src/DataTypes/tests/gtest_decimal_literal_datatype.cpp b/dbms/src/DataTypes/tests/gtest_decimal_literal_datatype.cpp new file mode 100644 index 00000000000..0ae32502679 --- /dev/null +++ b/dbms/src/DataTypes/tests/gtest_decimal_literal_datatype.cpp @@ -0,0 +1,100 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include + +namespace DB +{ +namespace tests +{ +TEST(DecimalLiteralDataTypeTest, getPrec) +try +{ + /// Decimal32 + ASSERT_TRUE(DecimalField(0, 0).getPrec() == 1); + ASSERT_TRUE(DecimalField(0, 1).getPrec() == 1); + ASSERT_TRUE(DecimalField(0, 2).getPrec() == 2); + ASSERT_TRUE(DecimalField(123, 0).getPrec() == 3); + ASSERT_TRUE(DecimalField(123, 2).getPrec() == 3); + ASSERT_TRUE(DecimalField(123, 4).getPrec() == 4); + + /// Decimal64 + ASSERT_TRUE(DecimalField(0, 0).getPrec() == 1); + ASSERT_TRUE(DecimalField(0, 1).getPrec() == 1); + ASSERT_TRUE(DecimalField(0, 2).getPrec() == 2); + ASSERT_TRUE(DecimalField(123, 0).getPrec() == 3); + ASSERT_TRUE(DecimalField(123, 2).getPrec() == 3); + ASSERT_TRUE(DecimalField(123, 4).getPrec() == 4); + ASSERT_TRUE(DecimalField(1234567891011ll, 4).getPrec() == 13); + + /// Decimal128 + ASSERT_TRUE(DecimalField(0, 0).getPrec() == 1); + ASSERT_TRUE(DecimalField(0, 1).getPrec() == 1); + ASSERT_TRUE(DecimalField(0, 2).getPrec() == 2); + ASSERT_TRUE(DecimalField(123, 0).getPrec() == 3); + ASSERT_TRUE(DecimalField(123, 2).getPrec() == 3); + ASSERT_TRUE(DecimalField(123, 4).getPrec() == 4); + ASSERT_TRUE(DecimalField(Int128(123123123123123ll) * 1000000, 4).getPrec() == 21); + + /// Decimal256 + ASSERT_TRUE(DecimalField(Int256(0), 0).getPrec() == 1); + ASSERT_TRUE(DecimalField(Int256(0), 1).getPrec() == 1); + ASSERT_TRUE(DecimalField(Int256(0), 2).getPrec() == 2); + ASSERT_TRUE(DecimalField(Int256(123), 0).getPrec() == 3); + ASSERT_TRUE(DecimalField(Int256(123), 2).getPrec() == 3); + ASSERT_TRUE(DecimalField(Int256(123), 4).getPrec() == 4); + ASSERT_TRUE(DecimalField(Int256(123123123123123123ll) * Int256(1000000000ll) * Int256(100000000000000ll), 4).getPrec() == 41); +} +CATCH + +TEST(DecimalLiteralDataTypeTest, fieldToDataType) +try +{ + /// Decimal32 + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(1,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(0, 0))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(1,1)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(0, 1))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(2,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(0, 2))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(3,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(123, 0))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(3,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(123, 2))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(4,4)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(123, 4))))); + + /// Decimal64 + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(10,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(0, 0))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(10,1)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(0, 1))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(10,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(0, 2))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(10,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(123, 0))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(10,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(123, 2))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(13,4)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(1234567891011ll, 4))))); + + /// Decimal128 + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(19,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(0, 0))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(19,1)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(0, 1))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(19,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(0, 2))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(19,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(123, 0))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(19,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(123, 2))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(21,4)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(Int128(123123123123123ll) * 1000000, 4))))); + + /// Decimal256 + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(39,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(Int256(0), 0))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(39,1)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(Int256(0), 1))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(39,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(Int256(0), 2))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(39,0)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(Int256(123), 0))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(39,2)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(Int256(123), 2))))); + ASSERT_TRUE(DataTypeFactory::instance().get("Decimal(41,4)")->equals(*applyVisitor(FieldToDataType(), Field(DecimalField(Int256(123123123123123123ll) * Int256(1000000000ll) * Int256(100000000000000ll), 4))))); +} +CATCH +} // namespace tests +} // namespace DB From 516fa158f887b81db8cdb5d3e0f8a016391ed6de Mon Sep 17 00:00:00 2001 From: hzh0425 <642256541@qq.com> Date: Mon, 30 May 2022 11:54:26 +0800 Subject: [PATCH 008/104] Refactor: add new module schema (#4932) ref pingcap/tiflash#4646 --- dbms/CMakeLists.txt | 1 + dbms/src/Databases/test/gtest_database.cpp | 2 +- dbms/src/Debug/MockSchemaGetter.h | 3 +- dbms/src/Debug/MockSchemaNameMapper.h | 2 +- dbms/src/Debug/MockTiDB.h | 4 +- dbms/src/Debug/dbgFuncCoprocessor.cpp | 20 ++-- dbms/src/Debug/dbgFuncMockTiDBTable.cpp | 2 +- dbms/src/Debug/dbgFuncSchema.cpp | 4 +- dbms/src/Debug/dbgFuncSchemaName.cpp | 6 +- dbms/src/Flash/BatchCoprocessorHandler.cpp | 2 +- .../Coprocessor/DAGStorageInterpreter.cpp | 2 +- dbms/src/Flash/CoprocessorHandler.cpp | 2 +- dbms/src/Interpreters/Context.cpp | 3 +- dbms/src/Interpreters/IDAsPathUpgrader.cpp | 20 ++-- dbms/src/Interpreters/IDAsPathUpgrader.h | 4 +- .../Interpreters/InterpreterSelectQuery.cpp | 16 ++-- dbms/src/Interpreters/loadMetadata.cpp | 4 +- dbms/src/Server/Server.cpp | 2 +- dbms/src/Storages/StorageDeltaMerge.cpp | 2 +- .../System/StorageSystemDTSegments.cpp | 21 +++-- .../Storages/System/StorageSystemDTTables.cpp | 2 +- .../System/StorageSystemDatabases.cpp | 17 ++-- .../Storages/System/StorageSystemTables.cpp | 25 ++--- .../Storages/Transaction/ApplySnapshot.cpp | 2 +- .../Storages/Transaction/PartitionStreams.cpp | 2 +- dbms/src/Storages/Transaction/RegionTable.cpp | 2 +- dbms/src/Storages/Transaction/TMTContext.cpp | 6 +- dbms/src/Storages/Transaction/TiDB.cpp | 20 ++-- .../tests/gtest_rename_resolver.cpp | 12 +-- .../Transaction/tests/gtest_table_info.cpp | 92 +++++++++---------- .../Storages/tests/gtest_filter_parser.cpp | 4 +- .../Schema}/SchemaBuilder-internal.h | 0 .../Schema}/SchemaBuilder.cpp | 8 +- .../Schema}/SchemaBuilder.h | 2 +- .../Schema}/SchemaGetter.cpp | 60 +++++++----- .../Schema}/SchemaGetter.h | 6 +- .../Schema}/SchemaNameMapper.h | 0 .../Schema}/SchemaSyncService.cpp | 6 +- .../Schema}/SchemaSyncService.h | 0 .../Schema}/SchemaSyncer.h | 0 .../Schema}/TiDBSchemaSyncer.h | 2 +- 41 files changed, 207 insertions(+), 183 deletions(-) rename dbms/src/{Storages/Transaction => TiDB/Schema}/SchemaBuilder-internal.h (100%) rename dbms/src/{Storages/Transaction => TiDB/Schema}/SchemaBuilder.cpp (99%) rename dbms/src/{Storages/Transaction => TiDB/Schema}/SchemaBuilder.h (98%) rename dbms/src/{Storages/Transaction => TiDB/Schema}/SchemaGetter.cpp (83%) rename dbms/src/{Storages/Transaction => TiDB/Schema}/SchemaGetter.h (96%) rename dbms/src/{Storages/Transaction => TiDB/Schema}/SchemaNameMapper.h (100%) rename dbms/src/{Storages/Transaction => TiDB/Schema}/SchemaSyncService.cpp (98%) rename dbms/src/{Storages/Transaction => TiDB/Schema}/SchemaSyncService.h (100%) rename dbms/src/{Storages/Transaction => TiDB/Schema}/SchemaSyncer.h (100%) rename dbms/src/{Storages/Transaction => TiDB/Schema}/TiDBSchemaSyncer.h (99%) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 718d18c4954..21cf06cbe31 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -90,6 +90,7 @@ add_headers_and_sources(dbms src/Storages/Page/V2/mvcc) add_headers_and_sources(dbms src/Storages/Page/V2/VersionSet) add_headers_and_sources(dbms src/Storages/Page/V2/gc) add_headers_and_sources(dbms src/WindowFunctions) +add_headers_and_sources(dbms src/TiDB/Schema) if (ENABLE_V3_PAGESTORAGE) add_headers_and_sources(dbms src/Storages/Page/V3) add_headers_and_sources(dbms src/Storages/Page/V3/LogFile) diff --git a/dbms/src/Databases/test/gtest_database.cpp b/dbms/src/Databases/test/gtest_database.cpp index 72915b8644f..6b8bbc17348 100644 --- a/dbms/src/Databases/test/gtest_database.cpp +++ b/dbms/src/Databases/test/gtest_database.cpp @@ -25,11 +25,11 @@ #include #include #include -#include #include #include #include #include +#include #include #include diff --git a/dbms/src/Debug/MockSchemaGetter.h b/dbms/src/Debug/MockSchemaGetter.h index cdbaed97223..f02699866ce 100644 --- a/dbms/src/Debug/MockSchemaGetter.h +++ b/dbms/src/Debug/MockSchemaGetter.h @@ -15,14 +15,13 @@ #pragma once #include -#include +#include namespace DB { struct MockSchemaGetter { - TiDB::DBInfoPtr getDatabase(DatabaseID db_id) { return MockTiDB::instance().getDBInfoByID(db_id); } Int64 getVersion() { return MockTiDB::instance().getVersion(); } diff --git a/dbms/src/Debug/MockSchemaNameMapper.h b/dbms/src/Debug/MockSchemaNameMapper.h index b3fabab198a..003525aad89 100644 --- a/dbms/src/Debug/MockSchemaNameMapper.h +++ b/dbms/src/Debug/MockSchemaNameMapper.h @@ -14,7 +14,7 @@ #pragma once -#include +#include namespace DB { diff --git a/dbms/src/Debug/MockTiDB.h b/dbms/src/Debug/MockTiDB.h index cb09f9e305a..36d2af90859 100644 --- a/dbms/src/Debug/MockTiDB.h +++ b/dbms/src/Debug/MockTiDB.h @@ -15,10 +15,10 @@ #pragma once #include -#include -#include #include #include +#include +#include #include diff --git a/dbms/src/Debug/dbgFuncCoprocessor.cpp b/dbms/src/Debug/dbgFuncCoprocessor.cpp index a4a1f6730c9..e9335d1e2bd 100644 --- a/dbms/src/Debug/dbgFuncCoprocessor.cpp +++ b/dbms/src/Debug/dbgFuncCoprocessor.cpp @@ -336,7 +336,7 @@ void dbgFuncTiDBQueryFromNaturalDag(Context & context, const ASTs & args, DBGInv if (args.size() != 1) throw Exception("Args not matched, should be: json_dag_path", ErrorCodes::BAD_ARGUMENTS); - String json_dag_path = safeGet(typeid_cast(*args[0]).value); + auto json_dag_path = safeGet(typeid_cast(*args[0]).value); auto dag = NaturalDag(json_dag_path, &Poco::Logger::get("MockDAG")); dag.init(); dag.build(context); @@ -431,7 +431,7 @@ BlockInputStreamPtr dbgFuncTiDBQuery(Context & context, const ASTs & args) if (args.empty() || args.size() > 3) throw Exception("Args not matched, should be: query[, region-id, dag_prop_string]", ErrorCodes::BAD_ARGUMENTS); - String query = safeGet(typeid_cast(*args[0]).value); + auto query = safeGet(typeid_cast(*args[0]).value); RegionID region_id = InvalidRegionID; if (args.size() >= 2) region_id = safeGet(typeid_cast(*args[1]).value); @@ -464,8 +464,8 @@ BlockInputStreamPtr dbgFuncMockTiDBQuery(Context & context, const ASTs & args) if (args.size() < 2 || args.size() > 4) throw Exception("Args not matched, should be: query, region-id[, start-ts, dag_prop_string]", ErrorCodes::BAD_ARGUMENTS); - String query = safeGet(typeid_cast(*args[0]).value); - RegionID region_id = safeGet(typeid_cast(*args[1]).value); + auto query = safeGet(typeid_cast(*args[0]).value); + auto region_id = safeGet(typeid_cast(*args[1]).value); Timestamp start_ts = DEFAULT_MAX_READ_TSO; if (args.size() >= 3) start_ts = safeGet(typeid_cast(*args[2]).value); @@ -671,14 +671,14 @@ const ASTTablesInSelectQueryElement * getJoin(ASTSelectQuery & ast_query) if (!ast_query.tables) return nullptr; - const ASTTablesInSelectQuery & tables_in_select_query = static_cast(*ast_query.tables); + const auto & tables_in_select_query = static_cast(*ast_query.tables); if (tables_in_select_query.children.empty()) return nullptr; const ASTTablesInSelectQueryElement * joined_table = nullptr; for (const auto & child : tables_in_select_query.children) { - const ASTTablesInSelectQueryElement & tables_element = static_cast(*child); + const auto & tables_element = static_cast(*child); if (tables_element.table_join) { if (!joined_table) @@ -737,7 +737,7 @@ std::pair compileQueryBlock( bool append_pk_column = false; for (const auto & expr : ast_query.select_expression_list->children) { - if (ASTIdentifier * identifier = typeid_cast(expr.get())) + if (auto * identifier = typeid_cast(expr.get())) { if (identifier->getColumnName() == MutableSupport::tidb_pk_column_name) { @@ -756,7 +756,7 @@ std::pair compileQueryBlock( String right_table_alias; { String database_name, table_name; - const ASTTableExpression & table_to_join = static_cast(*joined_table->table_expression); + const auto & table_to_join = static_cast(*joined_table->table_expression); if (table_to_join.database_and_table_name) { auto identifier = static_cast(*table_to_join.database_and_table_name); @@ -788,7 +788,7 @@ std::pair compileQueryBlock( bool right_append_pk_column = false; for (const auto & expr : ast_query.select_expression_list->children) { - if (ASTIdentifier * identifier = typeid_cast(expr.get())) + if (auto * identifier = typeid_cast(expr.get())) { auto names = splitQualifiedName(identifier->getColumnName()); if (names.second == MutableSupport::tidb_pk_column_name) @@ -831,7 +831,7 @@ std::pair compileQueryBlock( bool has_agg_func = false; for (const auto & child : ast_query.select_expression_list->children) { - const ASTFunction * func = typeid_cast(child.get()); + const auto * func = typeid_cast(child.get()); if (func && AggregateFunctionFactory::instance().isAggregateFunctionName(func->name)) { has_agg_func = true; diff --git a/dbms/src/Debug/dbgFuncMockTiDBTable.cpp b/dbms/src/Debug/dbgFuncMockTiDBTable.cpp index 5b5cc004b58..65d0b2eadaa 100644 --- a/dbms/src/Debug/dbgFuncMockTiDBTable.cpp +++ b/dbms/src/Debug/dbgFuncMockTiDBTable.cpp @@ -24,8 +24,8 @@ #include #include #include -#include #include +#include #include namespace DB diff --git a/dbms/src/Debug/dbgFuncSchema.cpp b/dbms/src/Debug/dbgFuncSchema.cpp index 00ba5ab7335..8b73ddc23a3 100644 --- a/dbms/src/Debug/dbgFuncSchema.cpp +++ b/dbms/src/Debug/dbgFuncSchema.cpp @@ -22,10 +22,10 @@ #include #include #include -#include -#include #include #include +#include +#include #include #include diff --git a/dbms/src/Debug/dbgFuncSchemaName.cpp b/dbms/src/Debug/dbgFuncSchemaName.cpp index a4dac1ae050..4c2ad86bd62 100644 --- a/dbms/src/Debug/dbgFuncSchemaName.cpp +++ b/dbms/src/Debug/dbgFuncSchemaName.cpp @@ -20,9 +20,9 @@ #include #include #include -#include -#include #include +#include +#include #include #include @@ -97,7 +97,7 @@ BlockInputStreamPtr dbgFuncQueryMapped(Context & context, const ASTs & args) if (args.size() < 2 || args.size() > 3) throw Exception("Args not matched, should be: query, database-name[, table-name]", ErrorCodes::BAD_ARGUMENTS); - String query = safeGet(typeid_cast(*args[0]).value); + auto query = safeGet(typeid_cast(*args[0]).value); const String & database_name = typeid_cast(*args[1]).name; if (args.size() == 3) diff --git a/dbms/src/Flash/BatchCoprocessorHandler.cpp b/dbms/src/Flash/BatchCoprocessorHandler.cpp index 273ceec8f08..0fd41832711 100644 --- a/dbms/src/Flash/BatchCoprocessorHandler.cpp +++ b/dbms/src/Flash/BatchCoprocessorHandler.cpp @@ -18,8 +18,8 @@ #include #include #include -#include #include +#include #include diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index d91b18254f6..11a1b7e2d3e 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -33,7 +33,7 @@ #include #include #include -#include +#include #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wunused-parameter" diff --git a/dbms/src/Flash/CoprocessorHandler.cpp b/dbms/src/Flash/CoprocessorHandler.cpp index e432dd37083..3d653025b83 100644 --- a/dbms/src/Flash/CoprocessorHandler.cpp +++ b/dbms/src/Flash/CoprocessorHandler.cpp @@ -22,8 +22,8 @@ #include #include #include -#include #include +#include #include diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 5d967b388d0..a0adef5b50d 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -59,9 +59,9 @@ #include #include #include -#include #include #include +#include #include #include #include @@ -207,6 +207,7 @@ struct ContextShared explicit ContextShared(std::shared_ptr runtime_components_factory_) : runtime_components_factory(std::move(runtime_components_factory_)) + , storage_run_mode(PageStorageRunMode::ONLY_V3) { /// TODO: make it singleton (?) static std::atomic num_calls{0}; diff --git a/dbms/src/Interpreters/IDAsPathUpgrader.cpp b/dbms/src/Interpreters/IDAsPathUpgrader.cpp index ce72625fd46..8c807b537e9 100644 --- a/dbms/src/Interpreters/IDAsPathUpgrader.cpp +++ b/dbms/src/Interpreters/IDAsPathUpgrader.cpp @@ -33,10 +33,10 @@ #include #include #include -#include #include #include -#include +#include +#include #include #include @@ -71,7 +71,7 @@ std::shared_ptr getDatabaseEngine(const FileProviderPtr & file_prov ParserCreateQuery parser; ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "in file " + filename, 0); ASTCreateQuery & ast_create_query = typeid_cast(*ast); - auto storage = ast_create_query.storage; + auto * storage = ast_create_query.storage; if (storage == nullptr || storage->engine == nullptr || storage->engine->name.empty()) { throw Exception("Can not get database engine for file: " + filename, ErrorCodes::LOGICAL_ERROR); @@ -97,7 +97,7 @@ std::pair getTableInfo(const FileProviderPtr & file_pro ParserCreateQuery parser; ASTPtr ast = parseQuery(parser, definition.data(), definition.data() + definition.size(), "in file " + table_metadata_file, 0); ASTCreateQuery & ast_create_query = typeid_cast(*ast); - auto storage = ast_create_query.storage; + auto * storage = ast_create_query.storage; if (storage == nullptr || storage->engine == nullptr || storage->engine->name.empty()) { throw Exception("Can not get table engine for file: " + table_metadata_file, ErrorCodes::LOGICAL_ERROR); @@ -105,7 +105,7 @@ std::pair getTableInfo(const FileProviderPtr & file_pro TiDB::TableInfo info; ASTFunction * engine = storage->engine; - auto * args = typeid_cast(engine->arguments.get()); + const auto * args = typeid_cast(engine->arguments.get()); if (args == nullptr) throw Exception("Can not cast table engine arguments", ErrorCodes::BAD_ARGUMENTS); @@ -399,12 +399,12 @@ String IDAsPathUpgrader::DatabaseDiskInfo::getNewMetaDirectory(const String & ro return root_path + (endsWith(root_path, "/") ? "" : "/") + "/metadata/" + escapeForFileName(newName()) + "/"; } // "data/" -String IDAsPathUpgrader::DatabaseDiskInfo::getNewDataDirectory(const String & root_path) const +String IDAsPathUpgrader::DatabaseDiskInfo::getNewDataDirectory(const String & root_path) { return root_path + "/data/"; } // "extra_data/" -String IDAsPathUpgrader::DatabaseDiskInfo::getNewExtraDirectory(const String & extra_root) const +String IDAsPathUpgrader::DatabaseDiskInfo::getNewExtraDirectory(const String & extra_root) { return extra_root + "/"; } @@ -457,11 +457,11 @@ IDAsPathUpgrader::IDAsPathUpgrader(Context & global_ctx_, bool is_mock_, std::un bool IDAsPathUpgrader::needUpgrade() { - const auto metadataPath = global_context.getPath() + "/metadata"; + const auto metadata_path = global_context.getPath() + "/metadata"; // For old version, we have database directories and its `.sql` file Poco::DirectoryIterator dir_end; - for (Poco::DirectoryIterator it(metadataPath); it != dir_end; ++it) + for (Poco::DirectoryIterator it(metadata_path); it != dir_end; ++it) { if (!it->isDirectory()) continue; @@ -893,7 +893,7 @@ void IDAsPathUpgrader::renameTable( args->children.emplace_back(literal); else if (args->children.size() >= 2) args->children.at(1) = literal; - } while (0); + } while (false); } const String new_tbl_meta_file = table.getNewMetaFilePath(root_path, db_info); diff --git a/dbms/src/Interpreters/IDAsPathUpgrader.h b/dbms/src/Interpreters/IDAsPathUpgrader.h index 8ef57f7f2cc..38dc37536aa 100644 --- a/dbms/src/Interpreters/IDAsPathUpgrader.h +++ b/dbms/src/Interpreters/IDAsPathUpgrader.h @@ -137,9 +137,9 @@ class IDAsPathUpgrader // "metadata/db_${id}/" String getNewMetaDirectory(const String & root_path) const; // "data/" - String getNewDataDirectory(const String & root_path) const; + static String getNewDataDirectory(const String & root_path); // "extra_data/" - String getNewExtraDirectory(const String & extra_root) const; + static String getNewExtraDirectory(const String & extra_root); private: // "metadata/${db_name}.sql" diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index a56fdb849e3..51b55f65bd4 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -55,12 +55,12 @@ #include #include #include -#include #include #include #include #include #include +#include #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wunused-parameter" @@ -496,7 +496,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt { if (expressions.has_join) { - const ASTTableJoin & join = static_cast(*query.join()->table_join); + const auto & join = static_cast(*query.join()->table_join); if (join.kind == ASTTableJoin::Kind::Full || join.kind == ASTTableJoin::Kind::Right) pipeline.stream_with_non_joined_data = expressions.before_join->createStreamWithNonJoinedDataIfFullOrRightJoin( pipeline.firstStream()->getHeader(), @@ -816,7 +816,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline for (size_t i = 0; i < arr->size(); i++) { - String str = arr->getElement(i); + auto str = arr->getElement(i); ::metapb::Region region; ::google::protobuf::TextFormat::ParseFromString(str, ®ion); @@ -839,7 +839,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline } /// PARTITION SELECT only supports MergeTree family now. - if (const ASTSelectQuery * select_query = typeid_cast(query_info.query.get())) + if (const auto * select_query = typeid_cast(query_info.query.get())) { if (select_query->partition_expression_list) { @@ -860,7 +860,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline if (auto managed_storage = std::dynamic_pointer_cast(storage); managed_storage && managed_storage->engineType() == TiDB::StorageEngine::DT) { - if (const ASTSelectQuery * select_query = typeid_cast(query_info.query.get())) + if (const auto * select_query = typeid_cast(query_info.query.get())) { // With `no_kvsotre` is true, we do not do learner read if (likely(!select_query->no_kvstore)) @@ -910,7 +910,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline QuotaForIntervals & quota = context.getQuota(); pipeline.transform([&](auto & stream) { - if (IProfilingBlockInputStream * p_stream = dynamic_cast(stream.get())) + if (auto * p_stream = dynamic_cast(stream.get())) { p_stream->setLimits(limits); @@ -1275,7 +1275,7 @@ void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline) // NOLINT for (const auto & elem : query.limit_by_expression_list->children) columns.emplace_back(elem->getColumnName()); - size_t value = safeGet(typeid_cast(*query.limit_by_value).value); + auto value = safeGet(typeid_cast(*query.limit_by_value).value); pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, value, columns); @@ -1347,7 +1347,7 @@ void InterpreterSelectQuery::executeExtremes(Pipeline & pipeline) return; pipeline.transform([&](auto & stream) { - if (IProfilingBlockInputStream * p_stream = dynamic_cast(stream.get())) + if (auto * p_stream = dynamic_cast(stream.get())) p_stream->enableExtremes(); }); } diff --git a/dbms/src/Interpreters/loadMetadata.cpp b/dbms/src/Interpreters/loadMetadata.cpp index 77c2bda6cda..54167f364e7 100644 --- a/dbms/src/Interpreters/loadMetadata.cpp +++ b/dbms/src/Interpreters/loadMetadata.cpp @@ -26,9 +26,9 @@ #include #include #include -#include -#include #include +#include +#include #include #include diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index fcf820eb958..04676ef969d 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -65,10 +65,10 @@ #include #include #include -#include #include #include #include +#include #include #include #include diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 38a947a027f..fc73e28e23a 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -44,10 +44,10 @@ #include #include #include -#include #include #include #include +#include #include #include #include diff --git a/dbms/src/Storages/System/StorageSystemDTSegments.cpp b/dbms/src/Storages/System/StorageSystemDTSegments.cpp index 37f2dbe1b99..f84a19a005c 100644 --- a/dbms/src/Storages/System/StorageSystemDTSegments.cpp +++ b/dbms/src/Storages/System/StorageSystemDTSegments.cpp @@ -23,12 +23,13 @@ #include #include #include -#include #include +#include namespace DB { -StorageSystemDTSegments::StorageSystemDTSegments(const std::string & name_) : name(name_) +StorageSystemDTSegments::StorageSystemDTSegments(const std::string & name_) + : name(name_) { setColumns(ColumnsDescription({ {"database", std::make_shared()}, @@ -61,11 +62,11 @@ StorageSystemDTSegments::StorageSystemDTSegments(const std::string & name_) : na } BlockInputStreams StorageSystemDTSegments::read(const Names & column_names, - const SelectQueryInfo &, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const SelectQueryInfo &, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) { check(column_names); processed_stage = QueryProcessingStage::FetchColumns; @@ -78,19 +79,19 @@ BlockInputStreams StorageSystemDTSegments::read(const Names & column_names, for (const auto & d : databases) { String database_name = d.first; - auto & database = d.second; + const auto & database = d.second; const DatabaseTiFlash * db_tiflash = typeid_cast(database.get()); auto it = database->getIterator(context); for (; it->isValid(); it->next()) { - auto & table_name = it->name(); + const auto & table_name = it->name(); auto & storage = it->table(); if (storage->getName() != MutableSupport::delta_tree_storage_name) continue; auto dm_storage = std::dynamic_pointer_cast(storage); - auto & table_info = dm_storage->getTableInfo(); + const auto & table_info = dm_storage->getTableInfo(); auto table_id = table_info.id; auto segment_stats = dm_storage->getStore()->getSegmentStats(); for (auto & stat : segment_stats) diff --git a/dbms/src/Storages/System/StorageSystemDTTables.cpp b/dbms/src/Storages/System/StorageSystemDTTables.cpp index b3f9cf5b29e..b700cfb5324 100644 --- a/dbms/src/Storages/System/StorageSystemDTTables.cpp +++ b/dbms/src/Storages/System/StorageSystemDTTables.cpp @@ -23,7 +23,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/dbms/src/Storages/System/StorageSystemDatabases.cpp b/dbms/src/Storages/System/StorageSystemDatabases.cpp index 6c665fec900..5ba249a0aba 100644 --- a/dbms/src/Storages/System/StorageSystemDatabases.cpp +++ b/dbms/src/Storages/System/StorageSystemDatabases.cpp @@ -21,16 +21,17 @@ #include #include #include -#include #include #include +#include namespace DB { -StorageSystemDatabases::StorageSystemDatabases(const std::string & name_) : name(name_) +StorageSystemDatabases::StorageSystemDatabases(const std::string & name_) + : name(name_) { setColumns(ColumnsDescription({ {"name", std::make_shared()}, @@ -45,11 +46,11 @@ StorageSystemDatabases::StorageSystemDatabases(const std::string & name_) : name BlockInputStreams StorageSystemDatabases::read(const Names & column_names, - const SelectQueryInfo &, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const SelectQueryInfo &, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) { check(column_names); processed_stage = QueryProcessingStage::FetchColumns; @@ -79,7 +80,7 @@ BlockInputStreams StorageSystemDatabases::read(const Names & column_names, res_columns[j++]->insert(Int64(database_id)); res_columns[j++]->insert(database.second->getEngineName()); - res_columns[j++]->insert((UInt64)tombstone); + res_columns[j++]->insert(static_cast(tombstone)); res_columns[j++]->insert(database.second->getDataPath()); res_columns[j++]->insert(database.second->getMetadataPath()); } diff --git a/dbms/src/Storages/System/StorageSystemTables.cpp b/dbms/src/Storages/System/StorageSystemTables.cpp index dd24d426e80..4abf0044c87 100644 --- a/dbms/src/Storages/System/StorageSystemTables.cpp +++ b/dbms/src/Storages/System/StorageSystemTables.cpp @@ -27,10 +27,10 @@ #include #include #include -#include #include #include #include +#include namespace DB { @@ -71,7 +71,8 @@ NameAndTypePair tryGetColumn(const ColumnsWithTypeAndName & columns, const Strin struct VirtualColumnsProcessor { explicit VirtualColumnsProcessor(const ColumnsWithTypeAndName & all_virtual_columns_) - : all_virtual_columns(all_virtual_columns_), virtual_columns_mask(all_virtual_columns_.size(), 0) + : all_virtual_columns(all_virtual_columns_) + , virtual_columns_mask(all_virtual_columns_.size(), 0) {} /// Separates real and virtual column names, returns real ones @@ -131,7 +132,8 @@ struct VirtualColumnsProcessor } // namespace -StorageSystemTables::StorageSystemTables(const std::string & name_) : name(name_) +StorageSystemTables::StorageSystemTables(const std::string & name_) + : name(name_) { setColumns(ColumnsDescription({ {"database", std::make_shared()}, @@ -147,7 +149,8 @@ StorageSystemTables::StorageSystemTables(const std::string & name_) : name(name_ })); virtual_columns = {{std::make_shared(), "metadata_modification_time"}, - {std::make_shared(), "create_table_query"}, {std::make_shared(), "engine_full"}}; + {std::make_shared(), "create_table_query"}, + {std::make_shared(), "engine_full"}}; } @@ -164,11 +167,11 @@ static ColumnPtr getFilteredDatabases(const ASTPtr & query, const Context & cont BlockInputStreams StorageSystemTables::read(const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - const size_t /*max_block_size*/, - const unsigned /*num_streams*/) + const SelectQueryInfo & query_info, + const Context & context, + QueryProcessingStage::Enum & processed_stage, + const size_t /*max_block_size*/, + const unsigned /*num_streams*/) { processed_stage = QueryProcessingStage::FetchColumns; @@ -226,7 +229,7 @@ BlockInputStreams StorageSystemTables::read(const Names & column_names, { if (db_tiflash) tidb_database_name = mapper.displayDatabaseName(db_tiflash->getDatabaseInfo()); - auto & table_info = managed_storage->getTableInfo(); + const auto & table_info = managed_storage->getTableInfo(); tidb_table_name = mapper.displayTableName(table_info); table_id = table_info.id; tombstone = managed_storage->getTombstone(); @@ -279,7 +282,7 @@ BlockInputStreams StorageSystemTables::read(const Names & column_names, { Tables external_tables = context.getSessionContext().getExternalTables(); - for (auto table : external_tables) + for (const auto & table : external_tables) { size_t j = 0; res_columns[j++]->insertDefault(); diff --git a/dbms/src/Storages/Transaction/ApplySnapshot.cpp b/dbms/src/Storages/Transaction/ApplySnapshot.cpp index 6106dda6f4b..3ed04d5ecbf 100644 --- a/dbms/src/Storages/Transaction/ApplySnapshot.cpp +++ b/dbms/src/Storages/Transaction/ApplySnapshot.cpp @@ -28,8 +28,8 @@ #include #include #include -#include #include +#include #include diff --git a/dbms/src/Storages/Transaction/PartitionStreams.cpp b/dbms/src/Storages/Transaction/PartitionStreams.cpp index 9142aad5358..13840159ebb 100644 --- a/dbms/src/Storages/Transaction/PartitionStreams.cpp +++ b/dbms/src/Storages/Transaction/PartitionStreams.cpp @@ -26,10 +26,10 @@ #include #include #include -#include #include #include #include +#include #include namespace DB diff --git a/dbms/src/Storages/Transaction/RegionTable.cpp b/dbms/src/Storages/Transaction/RegionTable.cpp index 8b5ca5746f0..c855d5b3226 100644 --- a/dbms/src/Storages/Transaction/RegionTable.cpp +++ b/dbms/src/Storages/Transaction/RegionTable.cpp @@ -20,9 +20,9 @@ #include #include #include -#include #include #include +#include namespace DB { diff --git a/dbms/src/Storages/Transaction/TMTContext.cpp b/dbms/src/Storages/Transaction/TMTContext.cpp index 006be9d7a92..719784edaf2 100644 --- a/dbms/src/Storages/Transaction/TMTContext.cpp +++ b/dbms/src/Storages/Transaction/TMTContext.cpp @@ -22,9 +22,9 @@ #include #include #include -#include #include -#include +#include +#include #include namespace DB @@ -55,6 +55,8 @@ TMTContext::TMTContext(Context & context_, const TiFlashRaftConfig & raft_config , engine(raft_config.engine) , replica_read_max_thread(1) , batch_read_index_timeout_ms(DEFAULT_BATCH_READ_INDEX_TIMEOUT_MS) + , wait_index_timeout_ms(DEFAULT_WAIT_INDEX_TIMEOUT_MS) + , read_index_worker_tick_ms(10) , wait_region_ready_timeout_sec(DEFAULT_WAIT_REGION_READY_TIMEOUT_SEC) {} diff --git a/dbms/src/Storages/Transaction/TiDB.cpp b/dbms/src/Storages/Transaction/TiDB.cpp index 3810e25372f..15bf2a3fb58 100644 --- a/dbms/src/Storages/Transaction/TiDB.cpp +++ b/dbms/src/Storages/Transaction/TiDB.cpp @@ -23,8 +23,8 @@ #include #include #include -#include #include +#include #include @@ -631,8 +631,8 @@ catch (const Poco::Exception & e) /////////////////////// IndexColumnInfo::IndexColumnInfo(Poco::JSON::Object::Ptr json) - : offset() - , length() + : offset(0) + , length(0) { deserialize(json); } @@ -682,13 +682,13 @@ catch (const Poco::Exception & e) /////////////////////// IndexInfo::IndexInfo(Poco::JSON::Object::Ptr json) - : id() - , state() - , index_type() - , is_unique() - , is_primary() - , is_invisible() - , is_global() + : id(0) + , state(TiDB::SchemaState::StateNone) + , index_type(0) + , is_unique(true) + , is_primary(true) + , is_invisible(true) + , is_global(true) { deserialize(json); } diff --git a/dbms/src/Storages/Transaction/tests/gtest_rename_resolver.cpp b/dbms/src/Storages/Transaction/tests/gtest_rename_resolver.cpp index f149fd40e3a..4d1afe2653c 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_rename_resolver.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_rename_resolver.cpp @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include -#include #include +#include +#include namespace DB::tests { @@ -138,10 +138,10 @@ inline ::testing::AssertionResult ColumnNameWithIDPairsCompare( // return ::testing::AssertionSuccess(); else return ::testing::internal::EqFailure(lhs_expr, - rhs_expr, - "<" + lhs.first.toString() + "," + lhs.second.toString() + ">", - "<" + rhs.first.toString() + "," + rhs.second.toString() + ">", - false); + rhs_expr, + "<" + lhs.first.toString() + "," + lhs.second.toString() + ">", + "<" + rhs.first.toString() + "," + rhs.second.toString() + ">", + false); } #define ASSERT_COLUMN_NAME_ID_PAIR_EQ(val1, val2) ASSERT_PRED_FORMAT2(::DB::tests::ColumnNameWithIDPairsCompare, val1, val2) diff --git a/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp b/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp index 0c9747cc24c..516a173b151 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp @@ -18,10 +18,10 @@ #include #include #include -#include #include #include #include +#include using TableInfo = TiDB::TableInfo; @@ -65,20 +65,20 @@ try R"json({"id":45,"name":{"O":"t","L":"t"},"charset":"utf8mb4","collate":"utf8mb4_bin","cols":[{"id":1,"name":{"O":"t","L":"t"},"offset":0,"origin_default":"\u0000\u00124","origin_default_bit":null,"default":null,"default_bit":null,"default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":254,"Flag":129,"Flen":4,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2}],"index_info":null,"constraint_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"is_common_handle":false,"comment":"","auto_inc_id":0,"auto_id_cache":0,"auto_rand_id":0,"max_col_id":1,"max_idx_id":0,"max_cst_id":0,"update_timestamp":418683341902184450,"ShardRowIDBits":0,"max_shard_row_id_bits":0,"auto_random_bits":0,"pre_split_regions":0,"partition":null,"compression":"","view":null,"sequence":null,"Lock":null,"version":3,"tiflash_replica":{"Count":1,"LocationLabels":[],"Available":false,"AvailablePartitionIDs":null}})json", [](const TableInfo & table_info) { ASSERT_EQ(table_info.columns[0].defaultValueToField().get(), - Field(String("\0\x12" - "4\0", - 4)) - .get()); + Field(String("\0\x12" + "4\0", + 4)) + .get()); }}, // Test binary default value with exact length having the full content. ParseCase{ R"json({"id":45,"name":{"O":"t","L":"t"},"charset":"utf8mb4","collate":"utf8mb4_bin","cols":[{"id":1,"name":{"O":"t","L":"t"},"offset":0,"origin_default":"\u0000\u00124","origin_default_bit":null,"default":null,"default_bit":null,"default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":254,"Flag":129,"Flen":3,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2}],"index_info":null,"constraint_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"is_common_handle":false,"comment":"","auto_inc_id":0,"auto_id_cache":0,"auto_rand_id":0,"max_col_id":1,"max_idx_id":0,"max_cst_id":0,"update_timestamp":418683341902184450,"ShardRowIDBits":0,"max_shard_row_id_bits":0,"auto_random_bits":0,"pre_split_regions":0,"partition":null,"compression":"","view":null,"sequence":null,"Lock":null,"version":3,"tiflash_replica":{"Count":1,"LocationLabels":[],"Available":false,"AvailablePartitionIDs":null}})json", [](const TableInfo & table_info) { ASSERT_EQ(table_info.columns[0].defaultValueToField().get(), - Field(String("\0\x12" - "4", - 3)) - .get()); + Field(String("\0\x12" + "4", + 3)) + .get()); }}, }; @@ -141,47 +141,47 @@ try { auto cases = // {StmtCase{ - 1145, // + 1145, // R"json({"id":1939,"db_name":{"O":"customer","L":"customer"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // R"json({"id":1145,"name":{"O":"customerdebt","L":"customerdebt"},"cols":[{"id":1,"name":{"O":"id","L":"id"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"type":{"Tp":8,"Flag":515,"Flen":20,"Decimal":0},"state":5,"comment":"i\"d"}],"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"负债信息","partition":null})json", // R"stmt(CREATE TABLE `customer`.`customerdebt`(`id` Int64) Engine = DeltaMerge((`id`), '{"cols":[{"comment":"i\\"d","default":null,"default_bit":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":515,"Flen":20,"Tp":8}}],"comment":"\\u8D1F\\u503A\\u4FE1\\u606F","id":1145,"index_info":[],"is_common_handle":false,"name":{"L":"customerdebt","O":"customerdebt"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":0}'))stmt", // }, - StmtCase{ - 2049, // - R"json({"id":1939,"db_name":{"O":"customer","L":"customer"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":2049,"name":{"O":"customerdebt","L":"customerdebt"},"cols":[{"id":1,"name":{"O":"id","L":"id"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"type":{"Tp":8,"Flag":515,"Flen":20,"Decimal":0},"state":5,"comment":"i\"d"}],"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"负债信息","update_timestamp":404545295996944390,"partition":null})json", // - R"stmt(CREATE TABLE `customer`.`customerdebt`(`id` Int64) Engine = DeltaMerge((`id`), '{"cols":[{"comment":"i\\"d","default":null,"default_bit":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":515,"Flen":20,"Tp":8}}],"comment":"\\u8D1F\\u503A\\u4FE1\\u606F","id":2049,"index_info":[],"is_common_handle":false,"name":{"L":"customerdebt","O":"customerdebt"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}'))stmt", // - }, - StmtCase{ - 31, // - R"json({"id":1,"db_name":{"O":"db1","L":"db1"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":31,"name":{"O":"simple_t","L":"simple_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545295996944390,"ShardRowIDBits":0,"partition":null})json", // - R"stmt(CREATE TABLE `db1`.`simple_t`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":31,"index_info":[],"is_common_handle":false,"name":{"L":"simple_t","O":"simple_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}'))stmt", // - }, - StmtCase{ - 33, // - R"json({"id":2,"db_name":{"O":"db2","L":"db2"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":33,"name":{"O":"pk_t","L":"pk_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":3,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545312978108418,"ShardRowIDBits":0,"partition":null})json", // - R"stmt(CREATE TABLE `db2`.`pk_t`(`i` Int32) Engine = DeltaMerge((`i`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":3,"Flen":11,"Tp":3}}],"comment":"","id":33,"index_info":[],"is_common_handle":false,"name":{"L":"pk_t","O":"pk_t"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545312978108418}'))stmt", // - }, - StmtCase{ - 35, // - R"json({"id":1,"db_name":{"O":"db1","L":"db1"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":35,"name":{"O":"not_null_t","L":"not_null_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":4097,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545324922961926,"ShardRowIDBits":0,"partition":null})json", // - R"stmt(CREATE TABLE `db1`.`not_null_t`(`i` Int32, `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":35,"index_info":[],"is_common_handle":false,"name":{"L":"not_null_t","O":"not_null_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545324922961926}'))stmt", // - }, - StmtCase{ - 37, // - R"json({"id":2,"db_name":{"O":"db2","L":"db2"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", - R"json({"id":37,"name":{"O":"mytable","L":"mytable"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"mycol","L":"mycol"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":15,"Flag":4099,"Flen":256,"Decimal":0,"Charset":"utf8","Collate":"utf8_bin","Elems":null},"state":5,"comment":""}],"index_info":[{"id":1,"idx_name":{"O":"PRIMARY","L":"primary"},"tbl_name":{"O":"","L":""},"idx_cols":[{"name":{"O":"mycol","L":"mycol"},"offset":0,"length":-1}],"is_unique":true,"is_primary":true,"state":5,"comment":"","index_type":1}],"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":1,"update_timestamp":404566455285710853,"ShardRowIDBits":0,"partition":null})json", // - R"stmt(CREATE TABLE `db2`.`mytable`(`mycol` String) Engine = DeltaMerge((`mycol`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"mycol","O":"mycol"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"utf8","Collate":"utf8_bin","Decimal":0,"Elems":null,"Flag":4099,"Flen":256,"Tp":15}}],"comment":"","id":37,"index_info":[],"is_common_handle":false,"name":{"L":"mytable","O":"mytable"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404566455285710853}'))stmt", // - }, - StmtCase{ - 32, // - R"json({"id":1,"db_name":{"O":"test","L":"test"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":31,"name":{"O":"range_part_t","L":"range_part_t"},"charset":"utf8mb4","collate":"utf8mb4_bin","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","version":0}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":407445773801488390,"ShardRowIDBits":0,"partition":{"type":1,"expr":"`i`","columns":null,"enable":true,"definitions":[{"id":32,"name":{"O":"p0","L":"p0"},"less_than":["0"]},{"id":33,"name":{"O":"p1","L":"p1"},"less_than":["100"]}],"num":0},"compression":"","version":1})json", // - R"stmt(CREATE TABLE `test`.`range_part_t_32`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"belonging_table_id":31,"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":32,"index_info":[],"is_common_handle":false,"is_partition_sub_table":true,"name":{"L":"range_part_t_32","O":"range_part_t_32"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":407445773801488390}'))stmt", // - }}; + StmtCase{ + 2049, // + R"json({"id":1939,"db_name":{"O":"customer","L":"customer"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // + R"json({"id":2049,"name":{"O":"customerdebt","L":"customerdebt"},"cols":[{"id":1,"name":{"O":"id","L":"id"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"type":{"Tp":8,"Flag":515,"Flen":20,"Decimal":0},"state":5,"comment":"i\"d"}],"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"负债信息","update_timestamp":404545295996944390,"partition":null})json", // + R"stmt(CREATE TABLE `customer`.`customerdebt`(`id` Int64) Engine = DeltaMerge((`id`), '{"cols":[{"comment":"i\\"d","default":null,"default_bit":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":515,"Flen":20,"Tp":8}}],"comment":"\\u8D1F\\u503A\\u4FE1\\u606F","id":2049,"index_info":[],"is_common_handle":false,"name":{"L":"customerdebt","O":"customerdebt"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}'))stmt", // + }, + StmtCase{ + 31, // + R"json({"id":1,"db_name":{"O":"db1","L":"db1"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // + R"json({"id":31,"name":{"O":"simple_t","L":"simple_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545295996944390,"ShardRowIDBits":0,"partition":null})json", // + R"stmt(CREATE TABLE `db1`.`simple_t`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":31,"index_info":[],"is_common_handle":false,"name":{"L":"simple_t","O":"simple_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}'))stmt", // + }, + StmtCase{ + 33, // + R"json({"id":2,"db_name":{"O":"db2","L":"db2"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // + R"json({"id":33,"name":{"O":"pk_t","L":"pk_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":3,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545312978108418,"ShardRowIDBits":0,"partition":null})json", // + R"stmt(CREATE TABLE `db2`.`pk_t`(`i` Int32) Engine = DeltaMerge((`i`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":3,"Flen":11,"Tp":3}}],"comment":"","id":33,"index_info":[],"is_common_handle":false,"name":{"L":"pk_t","O":"pk_t"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545312978108418}'))stmt", // + }, + StmtCase{ + 35, // + R"json({"id":1,"db_name":{"O":"db1","L":"db1"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // + R"json({"id":35,"name":{"O":"not_null_t","L":"not_null_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":4097,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545324922961926,"ShardRowIDBits":0,"partition":null})json", // + R"stmt(CREATE TABLE `db1`.`not_null_t`(`i` Int32, `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":35,"index_info":[],"is_common_handle":false,"name":{"L":"not_null_t","O":"not_null_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545324922961926}'))stmt", // + }, + StmtCase{ + 37, // + R"json({"id":2,"db_name":{"O":"db2","L":"db2"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", + R"json({"id":37,"name":{"O":"mytable","L":"mytable"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"mycol","L":"mycol"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":15,"Flag":4099,"Flen":256,"Decimal":0,"Charset":"utf8","Collate":"utf8_bin","Elems":null},"state":5,"comment":""}],"index_info":[{"id":1,"idx_name":{"O":"PRIMARY","L":"primary"},"tbl_name":{"O":"","L":""},"idx_cols":[{"name":{"O":"mycol","L":"mycol"},"offset":0,"length":-1}],"is_unique":true,"is_primary":true,"state":5,"comment":"","index_type":1}],"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":1,"update_timestamp":404566455285710853,"ShardRowIDBits":0,"partition":null})json", // + R"stmt(CREATE TABLE `db2`.`mytable`(`mycol` String) Engine = DeltaMerge((`mycol`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"mycol","O":"mycol"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"utf8","Collate":"utf8_bin","Decimal":0,"Elems":null,"Flag":4099,"Flen":256,"Tp":15}}],"comment":"","id":37,"index_info":[],"is_common_handle":false,"name":{"L":"mytable","O":"mytable"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404566455285710853}'))stmt", // + }, + StmtCase{ + 32, // + R"json({"id":1,"db_name":{"O":"test","L":"test"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // + R"json({"id":31,"name":{"O":"range_part_t","L":"range_part_t"},"charset":"utf8mb4","collate":"utf8mb4_bin","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","version":0}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":407445773801488390,"ShardRowIDBits":0,"partition":{"type":1,"expr":"`i`","columns":null,"enable":true,"definitions":[{"id":32,"name":{"O":"p0","L":"p0"},"less_than":["0"]},{"id":33,"name":{"O":"p1","L":"p1"},"less_than":["100"]}],"num":0},"compression":"","version":1})json", // + R"stmt(CREATE TABLE `test`.`range_part_t_32`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"belonging_table_id":31,"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":32,"index_info":[],"is_common_handle":false,"is_partition_sub_table":true,"name":{"L":"range_part_t_32","O":"range_part_t_32"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":407445773801488390}'))stmt", // + }}; for (auto & c : cases) { diff --git a/dbms/src/Storages/tests/gtest_filter_parser.cpp b/dbms/src/Storages/tests/gtest_filter_parser.cpp index a027ea71cfc..8820c05d2da 100644 --- a/dbms/src/Storages/tests/gtest_filter_parser.cpp +++ b/dbms/src/Storages/tests/gtest_filter_parser.cpp @@ -25,10 +25,10 @@ #include #include #include -#include -#include #include #include +#include +#include #include #include diff --git a/dbms/src/Storages/Transaction/SchemaBuilder-internal.h b/dbms/src/TiDB/Schema/SchemaBuilder-internal.h similarity index 100% rename from dbms/src/Storages/Transaction/SchemaBuilder-internal.h rename to dbms/src/TiDB/Schema/SchemaBuilder-internal.h diff --git a/dbms/src/Storages/Transaction/SchemaBuilder.cpp b/dbms/src/TiDB/Schema/SchemaBuilder.cpp similarity index 99% rename from dbms/src/Storages/Transaction/SchemaBuilder.cpp rename to dbms/src/TiDB/Schema/SchemaBuilder.cpp index 1ed8b33d415..64d118eec3e 100644 --- a/dbms/src/Storages/Transaction/SchemaBuilder.cpp +++ b/dbms/src/TiDB/Schema/SchemaBuilder.cpp @@ -35,11 +35,11 @@ #include #include #include -#include -#include -#include #include #include +#include +#include +#include #include #include @@ -1078,7 +1078,7 @@ void SchemaBuilder::applyCreatePhysicalTable(DBInfoPtr db_in ParserCreateQuery parser; ASTPtr ast = parseQuery(parser, stmt.data(), stmt.data() + stmt.size(), "from syncSchema " + table_info->name, 0); - ASTCreateQuery * ast_create_query = typeid_cast(ast.get()); + auto * ast_create_query = typeid_cast(ast.get()); ast_create_query->attach = true; ast_create_query->if_not_exists = true; ast_create_query->database = name_mapper.mapDatabaseName(*db_info); diff --git a/dbms/src/Storages/Transaction/SchemaBuilder.h b/dbms/src/TiDB/Schema/SchemaBuilder.h similarity index 98% rename from dbms/src/Storages/Transaction/SchemaBuilder.h rename to dbms/src/TiDB/Schema/SchemaBuilder.h index fcfba7db57b..8446765f74a 100644 --- a/dbms/src/Storages/Transaction/SchemaBuilder.h +++ b/dbms/src/TiDB/Schema/SchemaBuilder.h @@ -15,8 +15,8 @@ #pragma once #include -#include #include +#include namespace DB { diff --git a/dbms/src/Storages/Transaction/SchemaGetter.cpp b/dbms/src/TiDB/Schema/SchemaGetter.cpp similarity index 83% rename from dbms/src/Storages/Transaction/SchemaGetter.cpp rename to dbms/src/TiDB/Schema/SchemaGetter.cpp index a8dbf8befb7..7f52f9301b1 100644 --- a/dbms/src/Storages/Transaction/SchemaGetter.cpp +++ b/dbms/src/TiDB/Schema/SchemaGetter.cpp @@ -14,7 +14,7 @@ #include #include -#include +#include #include namespace DB @@ -103,14 +103,14 @@ struct TxnStructure } public: - static String Get(pingcap::kv::Snapshot & snap, const String & key) + static String get(pingcap::kv::Snapshot & snap, const String & key) { String encode_key = encodeStringDataKey(key); String value = snap.Get(encode_key); return value; } - static String HGet(pingcap::kv::Snapshot & snap, const String & key, const String & field) + static String hGet(pingcap::kv::Snapshot & snap, const String & key, const String & field) { String encode_key = encodeHashDataKey(key, field); String value = snap.Get(encode_key); @@ -118,7 +118,7 @@ struct TxnStructure } // For convinient, we only return values. - static std::vector> HGetAll(pingcap::kv::Snapshot & snap, const String & key) + static std::vector> hGetAll(pingcap::kv::Snapshot & snap, const String & key) { auto tikv_key_prefix = hashDataKeyPrefix(key); String tikv_key_end = pingcap::kv::prefixNext(tikv_key_prefix); @@ -137,7 +137,14 @@ struct TxnStructure } }; -AffectedOption::AffectedOption(Poco::JSON::Object::Ptr json) { deserialize(json); } +AffectedOption::AffectedOption(Poco::JSON::Object::Ptr json) + : schema_id(0) + , table_id(0) + , old_table_id(0) + , old_schema_id(0) +{ + deserialize(json); +} void AffectedOption::deserialize(Poco::JSON::Object::Ptr json) { @@ -175,19 +182,22 @@ void SchemaDiff::deserialize(const String & data) Int64 SchemaGetter::getVersion() { - String ver = TxnStructure::Get(snap, schemaVersionKey); - if (ver == "") + String ver = TxnStructure::get(snap, schemaVersionKey); + if (ver.empty()) return 0; return std::stoll(ver); } -String SchemaGetter::getSchemaDiffKey(Int64 ver) { return std::string(schemaDiffPrefix) + ":" + std::to_string(ver); } +String SchemaGetter::getSchemaDiffKey(Int64 ver) +{ + return std::string(schemaDiffPrefix) + ":" + std::to_string(ver); +} SchemaDiff SchemaGetter::getSchemaDiff(Int64 ver) { String key = getSchemaDiffKey(ver); - String data = TxnStructure::Get(snap, key); - if (data == "") + String data = TxnStructure::get(snap, key); + if (data.empty()) { throw TiFlashException("cannot find schema diff for version: " + std::to_string(ver), Errors::Table::SyncError); } @@ -196,16 +206,22 @@ SchemaDiff SchemaGetter::getSchemaDiff(Int64 ver) return diff; } -String SchemaGetter::getDBKey(DatabaseID db_id) { return String(DBPrefix) + ":" + std::to_string(db_id); } +String SchemaGetter::getDBKey(DatabaseID db_id) +{ + return String(DBPrefix) + ":" + std::to_string(db_id); +} -String SchemaGetter::getTableKey(TableID table_id) { return String(TablePrefix) + ":" + std::to_string(table_id); } +String SchemaGetter::getTableKey(TableID table_id) +{ + return String(TablePrefix) + ":" + std::to_string(table_id); +} TiDB::DBInfoPtr SchemaGetter::getDatabase(DatabaseID db_id) { String key = getDBKey(db_id); - String json = TxnStructure::HGet(snap, DBs, key); + String json = TxnStructure::hGet(snap, DBs, key); - if (json == "") + if (json.empty()) return nullptr; LOG_DEBUG(log, "Get DB Info from TiKV : " + json); @@ -221,8 +237,8 @@ TiDB::TableInfoPtr SchemaGetter::getTableInfo(DatabaseID db_id, TableID table_id throw Exception(); } String table_key = getTableKey(table_id); - String table_info_json = TxnStructure::HGet(snap, db_key, table_key); - if (table_info_json == "") + String table_info_json = TxnStructure::hGet(snap, db_key, table_key); + if (table_info_json.empty()) return nullptr; LOG_DEBUG(log, "Get Table Info from TiKV : " + table_info_json); TiDB::TableInfoPtr table_info = std::make_shared(table_info_json); @@ -232,8 +248,8 @@ TiDB::TableInfoPtr SchemaGetter::getTableInfo(DatabaseID db_id, TableID table_id std::vector SchemaGetter::listDBs() { std::vector res; - auto pairs = TxnStructure::HGetAll(snap, DBs); - for (auto pair : pairs) + auto pairs = TxnStructure::hGetAll(snap, DBs); + for (const auto & pair : pairs) { auto db_info = std::make_shared(pair.second); res.push_back(db_info); @@ -243,8 +259,8 @@ std::vector SchemaGetter::listDBs() bool SchemaGetter::checkDBExists(const String & key) { - String value = TxnStructure::HGet(snap, DBs, key); - return value.size() > 0; + String value = TxnStructure::hGet(snap, DBs, key); + return !value.empty(); } std::vector SchemaGetter::listTables(DatabaseID db_id) @@ -257,9 +273,9 @@ std::vector SchemaGetter::listTables(DatabaseID db_id) std::vector res; - auto kv_pairs = TxnStructure::HGetAll(snap, db_key); + auto kv_pairs = TxnStructure::hGetAll(snap, db_key); - for (auto kv_pair : kv_pairs) + for (const auto & kv_pair : kv_pairs) { const String & key = kv_pair.first; if (key.rfind(TablePrefix, 0) != 0) diff --git a/dbms/src/Storages/Transaction/SchemaGetter.h b/dbms/src/TiDB/Schema/SchemaGetter.h similarity index 96% rename from dbms/src/Storages/Transaction/SchemaGetter.h rename to dbms/src/TiDB/Schema/SchemaGetter.h index efcac11f626..cfa5e1c6335 100644 --- a/dbms/src/Storages/Transaction/SchemaGetter.h +++ b/dbms/src/TiDB/Schema/SchemaGetter.h @@ -139,13 +139,13 @@ struct SchemaGetter SchemaDiff getSchemaDiff(Int64 ver); - String getSchemaDiffKey(Int64 ver); + static String getSchemaDiffKey(Int64 ver); bool checkDBExists(const String & key); - String getDBKey(DatabaseID db_id); + static String getDBKey(DatabaseID db_id); - String getTableKey(TableID table_id); + static String getTableKey(TableID table_id); TiDB::DBInfoPtr getDatabase(DatabaseID db_id); diff --git a/dbms/src/Storages/Transaction/SchemaNameMapper.h b/dbms/src/TiDB/Schema/SchemaNameMapper.h similarity index 100% rename from dbms/src/Storages/Transaction/SchemaNameMapper.h rename to dbms/src/TiDB/Schema/SchemaNameMapper.h diff --git a/dbms/src/Storages/Transaction/SchemaSyncService.cpp b/dbms/src/TiDB/Schema/SchemaSyncService.cpp similarity index 98% rename from dbms/src/Storages/Transaction/SchemaSyncService.cpp rename to dbms/src/TiDB/Schema/SchemaSyncService.cpp index a22578b51fc..92eb700766b 100644 --- a/dbms/src/Storages/Transaction/SchemaSyncService.cpp +++ b/dbms/src/TiDB/Schema/SchemaSyncService.cpp @@ -19,10 +19,10 @@ #include #include #include -#include -#include -#include #include +#include +#include +#include #include namespace DB diff --git a/dbms/src/Storages/Transaction/SchemaSyncService.h b/dbms/src/TiDB/Schema/SchemaSyncService.h similarity index 100% rename from dbms/src/Storages/Transaction/SchemaSyncService.h rename to dbms/src/TiDB/Schema/SchemaSyncService.h diff --git a/dbms/src/Storages/Transaction/SchemaSyncer.h b/dbms/src/TiDB/Schema/SchemaSyncer.h similarity index 100% rename from dbms/src/Storages/Transaction/SchemaSyncer.h rename to dbms/src/TiDB/Schema/SchemaSyncer.h diff --git a/dbms/src/Storages/Transaction/TiDBSchemaSyncer.h b/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h similarity index 99% rename from dbms/src/Storages/Transaction/TiDBSchemaSyncer.h rename to dbms/src/TiDB/Schema/TiDBSchemaSyncer.h index c9b2f2aa567..b682abf1af4 100644 --- a/dbms/src/Storages/Transaction/TiDBSchemaSyncer.h +++ b/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h @@ -18,9 +18,9 @@ #include #include #include -#include #include #include +#include #include #include From 335b124cab124828aadc318b7355f8390950f032 Mon Sep 17 00:00:00 2001 From: Wenxuan Date: Mon, 30 May 2022 14:54:26 +0800 Subject: [PATCH 009/104] docs: Add rfc for compact table via sql (#4929) ref pingcap/tiflash#4897 --- .../2022-05-19-compact-table-via-sql.md | 224 ++++++++++++++++++ .../2022-05-19-compact-table-via-sql-1.png | Bin 0 -> 96999 bytes .../2022-05-19-compact-table-via-sql-2.png | Bin 0 -> 75093 bytes .../2022-05-19-compact-table-via-sql-3.png | Bin 0 -> 124588 bytes .../2022-05-19-compact-table-via-sql-4.png | Bin 0 -> 52004 bytes .../2022-05-19-compact-table-via-sql-5.png | Bin 0 -> 63735 bytes .../2022-05-19-compact-table-via-sql-6.png | Bin 0 -> 101035 bytes .../2022-05-19-compact-table-via-sql-7.png | Bin 0 -> 122437 bytes 8 files changed, 224 insertions(+) create mode 100644 docs/design/2022-05-19-compact-table-via-sql.md create mode 100644 docs/design/images/2022-05-19-compact-table-via-sql-1.png create mode 100644 docs/design/images/2022-05-19-compact-table-via-sql-2.png create mode 100644 docs/design/images/2022-05-19-compact-table-via-sql-3.png create mode 100644 docs/design/images/2022-05-19-compact-table-via-sql-4.png create mode 100644 docs/design/images/2022-05-19-compact-table-via-sql-5.png create mode 100644 docs/design/images/2022-05-19-compact-table-via-sql-6.png create mode 100644 docs/design/images/2022-05-19-compact-table-via-sql-7.png diff --git a/docs/design/2022-05-19-compact-table-via-sql.md b/docs/design/2022-05-19-compact-table-via-sql.md new file mode 100644 index 00000000000..c2f9aeff22f --- /dev/null +++ b/docs/design/2022-05-19-compact-table-via-sql.md @@ -0,0 +1,224 @@ +# Compact Table via SQL + +- Author(s): [Wish](http://github.com/breezewish) + +## Introduction + +This RFC introduces a compaction SQL command in TiDB. The command triggers a +compaction on TiFlash replicas, which can be used to: + +1. Migrate from PageStorage v2 to PageStorage v3 +2. Optimize performance by better organizing the data + +## Motivation or Background + +Recently the PageStorage v3 engine was introduced to TiFlash. By allowing +users to manually trigger the compaction, the migration from PageStorage v2 to +PageStorage v3 can be done easily, as a compaction command will merge and clear +all delta layer data (which was stored in v2) into the stable layer, while new +delta layer data will be stored in PageStorage v3. + +As a bonus, even when the Delta layer is already stored in PageStorage v3, users +can also benefit from this manual compaction command, considering that +compaction will rewrite the stored data into a better organized state. + +## Product Behavior + +New SQL syntax: + +```sql +ALTER TABLE table_name COMPACT [engine_type REPLICA] +-- engine_type could be either TIKV or TIFLASH +``` + +Sample SQLs: + +```sql +ALTER TABLE `users` COMPACT; -- Implicit: Not recommended +ALTER TABLE `users` COMPACT TIFLASH REPLICA; -- Explicit: Recommended +``` + +- The compaction is triggered immediately and runs in the foreground. The SQL + won’t return until the compaction is done. +- When a table is already in compacting progress, the new compaction SQL command + involving this table will be exited immediately by producing a “canceled” + warning. +- The compact SQL commands can be executed on different tables simultaneously, + resulting in multiple tables being compacted simultaneously. +- When `engine_type` is specified as `TIKV`, “unsupported” error will be + returned. When `engine_type` is not specified, the SQL will run as compact + TiFlash replicas only. This behavior will change when we support TiKV compaction + in future. +- When the table contains multiple partitions, all partitions will be compacted. +- The compact command will exit in the following ways: + 1. User kill SQL via `KILL [TIDB] `: stop task immediately + 2. TiDB stopped: the compaction run on TiFlash should be stopped. There will + be no retries after TiDB is restarted. + 3. TiFlash stopped: the compaction command should stop and return error + immediately. There will be no retries after TiFlash is restarted. + 4. Compaction is finished. + +## Detailed Design + +### Protocol + +New APIs will be added to TiFlash: + +```protobuf +// Pseudo code + +message Error { + oneof { + ErrorCompactInProgress, + ErrorTooManyPendingTasks, + // More errors can be added in future + } +} + +message ErrorCompactInProgress {} + +message ErrorTooManyPendingTasks {} + +message CompactRequest { + bytes id + bytes start_key // Optional + bytes max_end_key // Optional + int64 physical_table_id // Used to locate the TiFlsh table +} + +message CompactResponse { + optional Error error + bool has_remaining + bytes compacted_start_key + bytes compacted_end_key +} +``` + +### General Flow + +![](./images/2022-05-19-compact-table-via-sql-1.png) + +TiDB sends `CompactRequest` to one TiFlash instance in series. Each request +compacts one or multiple Segments in the TiFlash instance. The segments may +change (e.g. split or merge) during the SQL execution process so that TiFlash +will respond with the end key for each request, which will be then used as the +`StartKey` for the next request sent from TiDB. + +When there are multiple TiFlash instances, TiDB talks with each TiFlash +concurrently. However, for each TiFlash connection, requests are still sent in +series. Newly added TiFlash instances during a compaction SQL execution are +discarded. + +### Interrupt Execution (Kill) + +![](./images/2022-05-19-compact-table-via-sql-2.png) + +When user executes `KILL [TIDB] ` to interrupt the compaction command, +TiDB simply stops the execution by not sending future `CompactRequest`. +There will be `CompactRequest` running in TiFlash instances at the moment when +user initiates the `KILL`. These running "small" compaction tasks will be +untouched and kept running until finished. There is no way to stop a running +`CompactRequest` in TiFlash. + +### On TiDB Restart + +Similar to Kill, TiDB does not need to do anything extra after the restart. +TiFlash will be returned to idle after currently running `CompactRequest` are +finished. + +### On Request Error + +The `CompactRequest` from TiDB to TiFlash may fail for several reasons, for +example, encountering network failures or receiving errors from TiFlash. +In such case, there will be no more future `CompactRequest` sent to this +failed TiFlash instance during the compaction command. Requests will +continue sending to other TiFlash instances. + +![](./images/2022-05-19-compact-table-via-sql-3.png) + +### Service Endpoint + +A new gRPC endpoint will be added to the tikv gRPC service: + +```protobuf +service Tikv { + // Existing endpoints + rpc KvGet(kvrpcpb.GetRequest) returns (kvrpcpb.GetResponse) {} + rpc KvScan(kvrpcpb.ScanRequest) returns (kvrpcpb.ScanResponse) {} + rpc KvPrewrite(kvrpcpb.PrewriteRequest) returns (kvrpcpb.PrewriteResponse) {} + ... + + // Newly added management endpoint + rpc Compact(managementpb.CompactRequest) returns (managementpb.CompactResponse) {} +} +``` + +### Handling CompactRequest in TiFlash + +![](./images/2022-05-19-compact-table-via-sql-4.png) + +The `CompactRequest` will be processed one by one in a new thread pool with 1 +worker thread. When one request is being processed, TiFlash locates the Segment +according to `start_key` and then performs a foreground Delta Merge. + +The number of worker thread can be configured in case of users want more table +compaction concurrency. Note that compaction is always stepped in serial for a +single table, even if there are more than 1 worker threads. + +### Compact Multiple Segments in One Request + +When Delta Merge takes a short time, TiFlash repeats itself to compact more +Segments, until either: + +- 1 minute (can be configured) has elapsed since receiving the request +- There are no more Segments in `[start_key, max_end_key)` + This can speed up the compaction process when there are many compacted + segments by reducing round trips. + +![](./images/2022-05-19-compact-table-via-sql-5.png) + +### Multiple Compaction Command from Same Table + +Compacting the same table concurrently doesn't make sense and leads to extra +resource costs, so that we would like to avoid concurrent compactions for the same +table, allowing only one to be executed. + +In order to detect such cases, an `ID` field is attached to the `CompactRequest`, +which will be set as the Table ID in TiDB. TiFlash adds ID to a map when the +request is received, and removes ID from the map when the response is going +to be returned. If ID exists in the map, `ErrorCompactInProgress` will be +returned immediately, without processing the request in the thread pool at all. + +![](./images/2022-05-19-compact-table-via-sql-6.png) + +### Multiple Compaction Command from Different Tables + +As there is only one worker thread, when user invokes multiple compaction +command concurrently, these compactions will be stepped evenly, instead of +following a FIFO order, as demostrated below: + +![](./images/2022-05-19-compact-table-via-sql-7.png) + +If there are too many queued requests (>N), new requests will be rejected +and `ErrorTooManyPendingTasks` will be returned. This effectively means, the +number of concurrent running compact commands will be limited to max N. + +# Investigation & Alternatives + +- The Compact API can be placed in TiKV Debug Service (`debugpb`). There is even + existing `debugpb.CompactRequest`. However, + - All TiKV Debug Service endpoints are only used by tikv-ctl, instead of + TiDB or any other clients now. + - `debugpb.CompactRequest` is not suitable to be used by TiFlash, containing + too many TiKV specific fields and lacking fields for TiFlash. It will also be + hard to modify it while keeping compatibility + clean. +- The Compact API can be provided via `DBGInvoker`, which means it available via + Clickhouse Client SQL or TiFlash HTTP Service. This is also the home of most + management or debug APIs of TiFlash. However, + - Currently TiDB does not talk to TiFlash in this way. + - We require this API to be stable for TiDB to use. All APIs in DBGInvoker + are human-facing, not producing machine readable and stable output now. + +# Unresolved Questions + +None. diff --git a/docs/design/images/2022-05-19-compact-table-via-sql-1.png b/docs/design/images/2022-05-19-compact-table-via-sql-1.png new file mode 100644 index 0000000000000000000000000000000000000000..7c07de7e433c5f36ce76659b7cdf080ab81c56d4 GIT binary patch literal 96999 zcmeFZbySpX-!44B00T;w^w3B*(p}PEfTX~HNJ&UHh;*lvq@r}EfFL=7Al-^|gMjot z=YF2|eeb>B`;Tvb>-%G`^=;Qe#<{NRtY06$<1#`=O9c;?3Ks%_;Hjx9>OmmT0SE*m z42A{1p&jyl3xV9WvsF;gQBzQW>$o^s+u9=_5Y>p3$Jhq?{p2}DiShB@&|#`LZA^L? zsyKO=@e2;2%<(!F8ZeHN>i0`CN~E049Qo#kw(i>l`BN7ba!Q62FRStnS()E%eL8GA z@^jtG6|}k5-__Waf;{tIy3c&}o*aVtQ{GnIlaGm8Dn!m3_k}rn?T`JHLo4D(#>PT{ z0-iT5tu1JTU#>pX1aIXY-8^*rgHA~RvEu4B^rftT{9J}CZFv#fGC&kxEx5SkDF)dn zlybg8lW)BrPa%lwDurS`gauq_&mzTM52*COu5BIG6QlP7s7l37GT z^5E0AbNjoQMWeaP_pvWIe5K;oSZW?zI)$RgM$&(0Cuc2^*9e;WpqfmKQH9mbQ{CFl zp7Qm*1%rh3H)RfOoo**g=U`bHy3$|0k<~>{aX2nU(WlKnOBf)Bv#dQgU-1{gL!l{RC-M#+?7THNph!bf4%w)f#6hbO@mBvtZxOux8+d2yC z(XXI)gcO17@dN>b*pDJ!w-yDm=`fwY_TZtByyGNI6L}U;K|kjIz49&VlcEFyZ>wkc zPly#F7`kB=cSh($KV!U*k)$#WS$ffx(Ku{Qx44a?jk6P~61o+N|9J-|ziE^{p9W7H z9+3Wl+;DLcy&)(o$CGcDV46X&JdPU8W)9>0>5rgqHEd(m51lzbeEt)3;Y8_sD)hkJ z>aoCmI%Ryu*{epqF1aq(a3$TXy{&?1VY~MW+j$xES2gV)rRf(6{A;_(EwC$Vrf{|; z-S5~zZg1b22(PnHA@zLbAw@Z560$&y^|L=jGC$xCwQqIg{Q+Y;;}&BX&NW<2eOxl^ zI4NC$ha3Y$=SC$aU}e#RY-i zcO^Ou^ITEMYt2PBcMVAGINH6*?j-f7WN~9>hK>LnP0!KWL3b;m?tJv3OPH6kgfPqm zI5~5W8y}rAZ|H?FE^Ek!IX*iY!kp3oZz4$3oO252B=G7j8Vwd(2jvSQW30(elSUu$ zPW5jX3qjIQIaL^59#dF2rA5p=Mn@P4j;wsbGiEe36@dgA#pu3-jXuf3+lE1<3ia^u zKIuXnv7juKnS|Lsg~D52A-?kb$&s7+tncx}(GmGvLb#gpy?J8m%(Zv2<9Osazj3&U zK8eeb<;|ZQ!T-%N6NRQh`@I**ZxbC+Rb{}1G1yZ$rBO=C+54gLNhziD+D8NIi5S1` zhZc@ zRc614e?gT~r1e>XG2y*-EsJ+oO}ZA6MT?1t)e$~H6h|@}5f{LoekWQq~CIyCSNtaYI19ED`=LAJT5bKNSo2E<9t3mcHZ~I%C4GMEEUP7 z&BDv##3Y);n;_BeN9qvC{UtY1vA+nJ_dw_Q-FJ6S*`3D6$GOM+#+JvGHfzQ?$4)=w z>Ib~PPWYhk@@;jAWr@os@h1MJ^QOb55mk(}!mWJ7H(XowXXCX=wb-=;RBm__c(GV! zmZ{1`%56PlF^}I^Y1(O~u6lm8`0DZ%<4u}SOI_a+ktg=t=dvEt~aOCSa z_xJgN+2YxPcLi&CY5Af`6?%K&rmv-ooeG9Wh9}KFw42?8UB7$E+GWrM?IOqfNb8Q6 z@(KBYC5MF`q8B0_vW1m$LbK$Z_Z09jVoxRiagwBkom)RX_52@_uqx?oF;;BR#H+)alfZTt-|zrN~lc zBh7{nRn3-~_0~^rl}#Gt*7-YG+9f@&=_u}8Z7K`m?yaAy?>Wh#{mS#rG0`MtUf3sJ z>bvx?f5;UAd#v-j+`7)ji+-Y)zgQy|c;Bn&I)7U#+5Y1>t-dnX;6{8Q z#;v~n6Wf8nyQkV_rP{s56te|OJ%j+O-R7d&rZVoeIl~WzA_j*%@x1=LEqVG-!_bw` z@zCQ?pD^uPGIVm`;gJS(&PkS^e>i^-lS)`f5M! zv`^#Gc2pbDpv=_EK9m+_eyQ|aWhT6q(w5?T%&5{v3}s<-a`X;mo27hjDcyIm30M36 zDe7hWUQJ35|JIQ|mj^1W&$O1b?!~M1X?&eaIm@6PEc zPwOzNl;7u?mGMzJIik7AN)a>pEc>*n(x{ufjHGQSrf)v^LYh~^NyRk4|GB?O&=Wap zxg-OXEH-K35x1Yo5j(^xY7vE&iF<|PN#2FJqdNkMB3cs$?{CR$8m+h==KdmxLXYH# zlGf15@HLM-Aw5W*D8bjK(DZvm^XPI=$d}6L$)MZOhHJ4*R_B{41gn6co%JBspy83t z%OU*SHdlOv)V0q^j*WC+u&fdgEu1EINJ+ezHyK2>H-1SDhK+JnDA4g5J zFLo@xKv;h3nbBm;91!uF;hOQaQ~#5C&XM`yu&ldG$tc%RecJtmVwHD%Hl{w!;ig)l zwZPKm^N{}d!uWVgvei;{z+i^3C zE`eTePabcXP0|Dc8eW#wyWXyc>=IRXror_rQa^=~N0VZ#iRJtOX z?10)EgM+Gt<;3NAHWw{P$Bb_-r&crlcJF>_*Ja0w`|WB>WhB`c+Qela$$0!}-8(D3 zg}jAF(Z(~;;9XC5`SqwF#KF|zIIWde*z}}nZch5c)p|+?CtG=#v6O3!>(0;P^sODy zM5noCu@mcF?(4F(h9jr;Px@}}_I9^@LVb!(>vq#*`22N_TdwwA(wfq$(1l6b`z8Nb z-?dy=PSdv1mi8D5xbO4i8o6zF?AoyQy5(~-Py3yt{G*~@2s4j<`k8>z0L1C$q)f=Q z{f^q!hA+3@{EhX2QCEY+_2Hk7MWW@Woh`v0etYUNcl@W$63z~`-;HSxpR)$A`j!8& zxG6fb8=LgG!o0d7E>D)&zZ=*YWNm}7Z;cJ9b%U%|Iu%6go&;kAvtLZ2L9a<3okU^$ z#FXW}^pD#MGZPg>n`*(UH;XqTIr-o$1WELTZ0<@6JzyMk_t1CIeVQxp`EBur{KUR6 z1>$QsvAFnneDMasd7Jzsw)e{b-2(>2DM9<#v&4t-nAb7v7vJ|>E)VcYi97@8_ioU` zT@xtqg8XPe^%03MP_x$5gm8e*FbEWl3W5nfp@AQ1H0uBStc=DE!T9TYbO_|NEd=`S zd$hoB)Sr0pgL>v)zcCWSAlTqv#NfyCE&6}l4IOxk@gJWt!oW3%oW6pZ8u+bm>4HEw zx;}GqW2#sX24CR3P&IUgK*(88KWJ)t%==*cN!teoZU&kf;+9ShJWs8hED$`N4lhvS zKqNiI!AA##+f%rwgT14xxTh53Uw4Ru ?8R383;$|nsXrQSBS8#Gcz(siYc=#Bl zap7>dq>GicxSpc&zaIzxl45-3=JrCIm)FC?gU3UV$H~QpmtRawjF(S>a*liSgi>0g8V$2f`zS4$V$7jCvrj&RhtPc58Yx=ArIq9*#!KmS@M!qfJ@W^#1> z_qMHk3MuoxF|R|fH~cy|4Ux~e)#Vf|NWpOFKX+5JBoju^IzYBgOfN#l}>o!tOA(%33$KLEdhmZAQjb%DRw|MeU7xdBujW1SF)EJRIF?tv%Tb~eu3 z!|&(4xlsZH2p-D;9eI*8yGTX$4Ea|&KY!*McReirNhN3^@Jcm+l4$^$!XTPKtgHLp zs#xRImDzB!$Igk^p4ghyTI)rt4Z-5^S!;INq^aFb-KXK13#q*d8>ubr95wl12ow%M zgULeB2?PK3QFZ|55+{#qMKJJ(!vAvVUw0Xx!w?$K|MuL!F5W?1#Q47#|L-CE|4b7K z13MB~)CH8M-^ztVyK4P|Wy!Kz;>*ITpbG&vei?jEgEg}5=@x5dvI<%wD=i&H@?X_H zAHVNDjiA^VEsCn2S{%S|L~=bP&sJT}Kl1P}wH zUgsO{R<1Ah8uAU_X=VQ^tr;S|VwQuTzhqmwu3V0KF>g33e+b+6Wgae0p&>ddN_4Yr z`GmulyOnwGG(^@5N;ojEtjrBlj17#mq%if&YyCZ{K3Zq@SnOoVvdHIj7xO~QecPbv z>SPu%*XUe*G$b*2ww4=b@F@Nc1%nO^+9w!AZ-m|kbH`DFR!f@1^HuexU+*cD(2;BG zPhLsOy=6rUt$w4?7bLq4TXz{G(Uu(XPCVY8*5_$lLLV`(`obJzQoN!PN9Uka{iEx3 z&Fz{yFOq{~(a?Q~cp$IUOW44;=XZ68A*3OTtZbyP-h7C z;I}-5GPdc5k4+xds;-8WTTFKZVU*dR^u($!o?(|bmoQM468eYH7)$6wVHjjs<+%p! zfNTbDo_S$HIH9?;N;jb!26`Q#Ec-`H!ZZfLm*~wcE1xcQIbGL=QZ9B|DkfG^&w2_5 z`Fwt_GiYXtq#P{w)=*nZ>-Y`7d%TI?g$OTqqc7Abn9P{I-S#hw|lxdKFHJgprRHH18b~=n?nKq6|6GLdQ0$R z1_*6TIebeF9hnBli(S?VJfrcrHF{GE7O5UpL}_DzL*uy=l)qr^G3WgH_r~{kTFb3K zJS*vzu~nwF1TvQ|vS;!Ipraaryq|6Br7bIBz`h+q8|j!kd~rsX?(Ks0IXA$sv~Pjs z)f&8UW=%wySK-5tj|6;9>@x%qk+qKV&05P@;x9kc+D&SSOk0s;h&a3vdiL$D){ZVE zx-Clw#gE!2InERcfolbK?k77#(|Kc|xoYqZU#(y=N(CwyJh`ZIAez4l`heYTsxt5D z?0C{lfpYC|ke4@#g124*8(RF?!x?aoX`>q&-~g<3wp*KEQI^_1SkbhU8&?5h@imol88!`GTh7ETA$ zSVnz4mCpM^u;|Msk-7BaI4O@X*}Gt} z`*})+Fva+e%BNjcOC3R@H8$E!YdJofV_l|g-u>$%1uGvumM8wd5_9X#^+omGr;C9( z;ERA#Chg9*qzO+{S@zzKqmwGM{a!5Kf9W~>{A1a`^Vj!5qQS44jzP$-7$`$}Yvjih z5wulx5u%}G4WK|aTO+4JL_mR(>L~3jaBkB{8ie7~a+tKZ*DaWHKdEK6Ek(IXQ`_a9 zm?XIntp0cgh5H}0pTcC}N#MiNX zls*THh%!y%a)U(VEV{j?@LD6u-w{s>`ij9Woqa2TQf z81#rmsYk0*Cu8_7pMZh}6i8e#JwZ-(sU7-&t7W@-u++vd1PkxiHwA)}wv!q4nL0bR zGNXq2SH)V{9Qwr?3GZ_KpX|2Z_#0MRMUeFliEnW!3a}DGV-lxpY-YK5W-%AW9s^$%Gr_)FY$-$Y?n_H`& zc7@f~eTkxo@=i~HLxKiS1FTGqx?F;7cQ1w=i=nTo`;D%erm%MSmJ207o$q4L)7bbkVph$W4?w?N>$p3vP|+x~`sY1eT=Xio z4&K$N%y_i!aLBMxbhOgrt!%KpHm0~^ybRtd5&G<5#tz>h^v`#pjQ2sr%@4~nBK!uq z1V-lg42}<6=j=q==>qkWOe3eZy7@csS}a=_It~MnaOj=Hrw>BGU`my&^}4I-7Mhs@ zL15Lk?pU-6s8tJ5nP_1L-QsQ40U<6}mEh5Z$MxBoN^_hWKKeKujPP7%Ee8PlM?@6r zN(H&dv}aR6rlAVJi11aO1j!cRz>^=o;oDJotAS1+xgWT$ZD5N`=dUm6!k6GGWnGC^ zz^{_Y6IN9qRD}V5#?~EwyMPh+Gmc>4su%$?V!N3K_|a!s_iF^#?-0wfN9B#I3qS`L zz@Z?Hc3huzh zEIIh*(aLh1Oe*iEKa2&#U$WMHtojpK^7qP=W2pL9Go5xHe?LuMrN|VSGRLr;sV(t6 zTS+?viCM&_clA>iQ5GLx>+J$howVqGq3OQhWpk5FbH5A6j(PJ)>}ew;l<&8`!(vz^P2YqO~x8|7vi zoN}FGl4gLGj{z~L<0$%l`n@yzSULJmRc)6N-Ms6tlt93>uh??5;IBXo{CE%t&V^s& zQL|!TU5cPXi#hSzd~6%2oQf}KD_dtZ%cZooolL_;CT3}-<_!h zB&q7dIZ_I=3VuT5$7ja$w;QpcKV^YQ;6KWbBZLJ(+S3&RRY?fh6M-S^ydAT&7?wIM z)!lzh?@vfJB2=QHSid%*Ozpr-!m7c}o?KQvz%g2lP;Yk_KC;@KYZ7o;kd)EJEJvlU zVFnbW1LE%6?1S75S!OD)nha~K&lZ?C(&s%E{ZJ`q`)%wxlBKiv*+eX<#Np?@x&e_t8jz+0>ZCpz6x)EElyW4|?g{DGQHoyV4;fb{h-cFQsIEx<0qts&gK=`>t+yt+k?SI7WU%mm^oaG>{2<3jny*&2g9u1VVGBm>5@SIaL!=}@IE z3dChhDem^Z#R^lT;kQ7pPVp;x|36Q~*58Jd_Ga{8j;6~N{sgHeDahh)SO_Q2^rCfA zhkIZM_5SthiC#J%WKWUXLXS6Dp8Z{>Sm5yYfq!^~N{k5A(4DXHDb>>5a6#q_ECN#} zXz&XzBF)shWIYT5!dYv(P>DgFhEu?1$-9A}B|Qh(zODq*eXf76<7D$=#lUxM8HcnCdiSZP*Kau)l5OAmk^R{N;dcZN zHm0C13el-BI368k$)8pmED@a>E74bKTXgF}o7YfUted_OBaGH9R#&F?`>p#|G7_Nb zw*E~H<>5z$^^$lT9_MjM*k<*EyqQ-*;&9D5dpH4G9h-qr1nnphRYu)~n=jDjT*sp# z9k6=N+$x-B4vP;5zInQ>IP`Th)U0Um1++&KwcFT zt6dq8>lh7PI&Wmi$X=b(jA?;X*!}?6|ITaN=kh+&T6jn zmvDWNMXokvU&YQZp!0Rd8hkg=`K*uLGytzMZ8z|J`+AGAvqwgB@S9YP+I?kOCO0o@ z4AOQam~xNasd=L~=K%SYeQ%+k)jxeLHhbsy_if4TtBb{(q4)a8h-uxii(g6Fm$|g= zQ=KPWFvOP!amJLr%r3MA3ZPEHLc4xG)X|9pO;iswL-yBkGeyy%7XV~=R zM48d7q^L~xy?Ug;pr^!MYt8AL>#&!#VY7{U$?xAXyuYul-z`Urmpk6g)!J8J=X^zl zV1NYa0ToTTAALv|emAJme@V{^6(%0jdQ&q)W6L+K}R4N2P4(|a#;*p2*-$J>^) zzZJ;0@oSjdw$ovM0X~R5hm7UxxhzxVy7rvj9r+By&+8?R4fAq=QqQR*4HuMxBsTSF zxyQuK)#A;xo#q>%gIV1K^^8nBtmiPJTNFIMrFxK~8tt(p1IR-SZK}rCb5}bsJ?)8T>FUI3WpYvD6==UscD_O3cVCod4cf z_PEEMyg~23`Eu3&k*OErCQhpE0E^mwDTYpJreaXe8RP(w%Zep8#K7a=D6qR)^8U89 zh&|J10Vc0b$EK0KlrisrRlRHevpeFXt@!5d(x|?+6ynHjqv#V!&G&qzs8W?DFQ%JF z1{{F`ZU3AI`O3ZQd+>$A&X|`4Bd?V7N%AYeJz|mN3pmPr4Gj_^gnK@tDn;HYkViRF zby8~NB4(@z|3LSp+;b;``+NaqZ;u!JFC4_yvRu2lc&i=8$dNZ&74789L=4tV#gh9S z81+Mq&dXLeSEmiVm--SQ6`Y~ewE!|3O0x()vwM3fICM_|S*^lw6;~i&2xM`^d;a*k`4F)-lQd>O3Msmvy-}Zdmt8x2H!$gbHhOL_ab^aHYIuw;Js< zMu*i}thnq5_>z8LP!1v2oB{JJjYbWQ7T2zFZpK5boz#sU)mZzBAh?F4ZC9PeVe+fr z`@(gaJd`alE7{QXqOvmftP`95wl#tVWQn^Lp)w5V0a)Y!gbJ#}N&Z-=OMp%oPJwo` zTX`dKr^58R5v&_z)JA9SA9>@d&HT^42nHC>oc?UTKIU!jl8dXB@4T!F?)j~2)ZnzZ zP%$jG1LXJIMlqrY`cvHOGk}>m({JKruAeR6rAiq5R70bGHD+LWPWhW%|J-#Id2q7e zvq!ZeynkbMbN-mR!oDO$GC*wAXRnO{c@j5{@p!MmdBhm^q>FaTWmp~uOy|~!g$4^% zg2V|NJz|xE<3ZDzJzZ=Ap}mc;LG3Jxmc3drv9s@^&;Py&cp zbi+1g8!Ad0e@0F)bwR7}O;_&`0BK2@_YJ85H^y>1PJk&nf~eHRr9UNOumH&wM|wPW zJ{)k(v?*-e&8u5cy81P95QvJCb?MVFosngX@UK~?_Og*u@1PQonWJGD-eKvB(sUDc z2g+l@Od;DuA%~f<9ErGU0Ba1A8`s6hSY_>Yjk&Z~8RZ``b`De?d>J_eN3L_DiPZ@y68byuBt+ z0M58lXBs5@`z$AEXP^xosvDZG!lJ?w=q@Aq*PYW02hLEp6 zX8g4##$-H7f$mRRpx%ws04hceLV6pWvA5S}9JZ;V^hvV-zaeq9Y`4TRk~#i6d0cEC zs4_q>6ULIqf)Z>uW;^#y;653L;ov%AqI)c_2ro<05Hlb$3a%^1XAG>l{CK1 z04h|jr%$9QMnEOh=L5V>n^F|HS93z(>c-)3b>6lyeQo0!h04i#2ct|Ui!4N|0?|OE zkHxZhZWJ|b5j>s-nAU2;=KdNU$8FsX2C~dLVa^cg)Xd?87EFDLW9$wB#{zPG3j&H+X#+?zz%0;K8 z&lkC%5SY{l_XktCzf_G&tna4g`k$BF3DS;!ozYf;v%p)?e%*|GBdi{6^G3*aw0oD( zzIEX9#sgdEH9RNVq3Fa&M zf^v4*lGFQ*Baqofy-a~N5t!*+-_-58O)%33M|5$*r(h-`5iMv-k8i|V-F3ZbfYTrf zKry|U0+Bat`?+dO_#Qt9lpGIZWv)Gaj(~M2ogZ&K&W?TY7*-ex&=`G-XJLlqz;{$v z7S$6UvD6i7|LVvXju7Guxeo_e1e7X;K@P6E6eGG2f3h(MH3)8vaj)JOZiq?`^H){ zUe)v`vGwNPxDlf}GlN0yJ&BQ?qyP&!XUvYi;4@2ns-k>ghK{tcMu+~C1J+OHo}Wg@ z9@T-vmIX&y;xU~*uYe{;Vo93e&Gpqvai$~3M-vW^@4mv>S8C*=FE7pn1x)jf+2(o% zZxL~UZ!%a?fj#XxR!|XGQfl=Q3_XI8Fj~EM!IC51RdADQDt$q8}Fz@{bQjL(%^CmckWuSHE}) zezn<|$jM{)DHeMODqfl2C(gzS`^*+g;PHP2uc-1r+FBVv*YFaY7|*}JIqetn(U^%sG`O1;nW}b_ef{Z zIYHIv>O)FB!6vYzM7!Dh(a~pmI3H>axk0z)dBWN!LiQLSZ`)ROyr4cJc44q9G{Ool zddfov9TX;?HWW>@0RXWCI-}S8daLSpNNt%ofHCS;L9YU3p8^Yhb@qY`J%RPDI`_-| zqGG+Di7V8}QICr!AKqg!HI&u^O&C5@BWks4+zmuc)rKo|TB)O=2Nub30zgd3WeSV% z%4CTAv>rK)9NZncNu0*DXddx6HSYEqA$tzclPDXd{@*Bd&HcJ1V6m!hvaT!o9JPcR z*j11$%84{J#or3*BYAIUT83#ku(3lZ2NBro+yB@rsuu(fQ+OUoYSiIIPi5HmE^yTasNMi#>zgQy3t%Pnf5E5WseZC-alQ2{Hb%>+zxmljDqF#%x@h%@Neh|IZ)i;V~3Wlfg>!`(W_D z^Pn@jF7f+(_>|I|u^Rz;D$tM&7-1hsfS6>c%f~kqNHbnm45Z%Gp%sZoTJ zGPNMOuF;{v4$E79LHc=3exQ@@rvBV+djxE(jxk?05OTx<3F(?50naUj-iEV)=k!1f zp_D<_B`eBjUHCFw{resssso6{DcIuzf?BdVPxf&-??$bUp2}BWtq%yw$0)DG@-;u2 z4YZ3}k^J`u(| zNwCxm`SgT48Eq5>XfQ#N(9ixiD@}OQphX3>P!Bw{ve#7U1(`c8s_zPU@FrB26lWik zJ@QeWHqa?Q!lkRn^vO-wP&lCtdMn$qz0^5As9`igJhRWQ#SRK5Ub>~Dh^Np1#*QQb zH#>LF0X9vz7}`P-zbJejcuRSp&g2|gR3Wl%sF9oN^K8J#p#6URk<07}xZ$TZSjMG8 z9}JAK4-WQKZ(=H(&=%bnEwa{Yodycmkwb62;ee$TgLNtrE=gs;1O;HA#$X1A$q-aX z`HwW5AWz^tqZ+Q(rT&@zqY4BskFF+&I*9+yVmJVZGr7~nh|fQd9gkWjG|~FMKZqLc zC+c_>P6|%{d29wCt6MnCUuEH&(5VxNP4fHza3Dn}h67515NM)2I1oLq{(Im+_)s&+ zC5WKx5FAKQ!a5^35F*rp1g9v20|60UDg$mQKnjz6I|TN$86A%WCnP}K{5pG`1qewu+yO7ECSA3N*^7W+MU|@d1JM*Z40h5DQC|L-EV&-Hpa41#kQ!p?Bm4@D;PWk79OjHI3X-kff{O5!G zKj2|P8Fc##TA9odOK`e0Kq`MV*#c#O-^kG@cc7#M!V(7uJ({75fnEjjhPMW9VzdC^ zi~(2{1-jxJpf1FkK-~FX2baJ(5~E-w}Qm`NtV64h1{Fsjv4v_F8X7EF1Juy{eeVIa zurL}ZbTVbu4|xbWChDBKyOdzepjRB2XsNCYP9|9y+>6S+z`b$c-rv(%L_i15z`ZLr zU1TukNYuCpEi4RhFA$m;P9-}oxHl2p``}Hs9JtpD_<%Ba+8*q4KVpjbLC zDh%|JLm<%6`#{!DxQ82otc}5XU33`;&~Z=-_g4i3L03ZQ1t}{o3Q%VSd;8L3vkvx# z2LvP>9}hP4zY-)-;!_4!ZEM9XfinGr_}+uErWA`<;2$mlwH{Rk#vAj<2Kmp%D-1R? zE(@TMzn+FuLH)0eAD@9Kg$HuFS_ss9YZ4jK9VX1Fe@R0 z41RFvN+jzGZ7o~|m9iqU_JWkc&mSy!^S15>XSoiEjRO3k3ABq`QZIgg*DeAyC$Wl^ z!R@h7#gFsBXmxz`V0of-_!H zs#In>FZcAbq#0Y(T%QdG3;+m(u63JU1}s#-ZC&R^(zEh2E@vNrMp+zYY7?aW&Q$>@ z5u8@!v9};#H^J2e_zhN*CLUiKQ@0cyOnBCecWf8FP?&MWNu3yc8l*fMD6s5}rSuZC z8A=9?{{CP6Nh_dBlK}etgVlpPoS9Can(SM4>POoHZd$j4@#fvWYYG6K!R*lG+0jl> zPYm^}i2IB!+lZJx#R0b`Z!E*#*st+-(CKi{2~{!Cjz$lEzfR8W_*n7jIi=VmR6}av zGlBGE2QLTvDPj(-thC#y{Subicl$E#{dx!t?UT?{}oF{JcDBvETA&ZyIP57Gv zvF1Ji^lVtH04y6!UIfjeb1$ubllmsJ+zZNIE;Sbbx64qDOyeoo*Z_TPv)?NTOuU!@*Fwj2&{3T|R*M^~=W zz8ur_r^cfe`5j%kezyXU&BlipL6=u@xs1H@&6K5R9I=Ef7C>2L-^KzTH9-U)2lk*y z)KP%Ed;~`!Gh<1rNnRmEPa6?22caZd@!&KKsf3?1gVu9dmdApZ%a2zStM*%Yujvv) z2&AXH*HB$UvB66~?>huS8fz@3Q>6FZ54ue+ZH#|!DKL6#^G8h&m0cfOt}i31k;v74 zd3bm->tvJM|FU|)d#fKb&>VTQfka$(>L#BI7Yy>YOBOWF1e|P7A0h>Z`#`snTba3L zK^8DW=4tvs2yq(b_%LZY>QG#C+9dmmi~E-XMsmE_ArqNxIVL) z1_Ym9fO(jf^4Qd{ylIt6?0mhzZ&;%4CW`kHyRvGHtA-TvcPk*Tby^d9ATM~GSrM|p z$MgYFgxyYF1@X<3L^XN4*BpFmg(5Qtyc2(M#~$# z*(!AfVlcISTMnC+v^BZUav=Z(I)Tbao*B4T#dIM|G+1{IWUJD`NgfLl1RA`U@A=-1 z6sQ=DECB||f!oHL1CE+CK%&wcic}*U(BR^eP$prQUb|yA$fd+Ccu=?97Tu8M@rCU1 zlqE?V_ZoL3(ZGV=smW{@uCbi%-E#Si1FPEQ5ky+gG7lR?TFLVK)7P~2X9@&TV~ANV zyVeu?Yb6SkUu$a;cVzrqxjzPH&am`HoSG$;4Nd_8@WJ+TZlj2_5Og&L09TNz3*f`X z9oMFN*RTp!?f6!lQf-FzW zaWQnL8*f!hVr_bUQe&J~N;5<$il5b3lxCCVtSr}pDXMGWvs1@SK-Lkd)`>@+h5aVN z#Clfe;N#R7ARrp4tZ5{Y?6or+rB#8@`xKNflvqkLAU-C_gZOwi)rtmeZyNf*bA&g% z8KY30!vgS!#YN1NsE@4W4ef zZB1}r5U>4DQJ4kIy-g_KA*wm=M#6VcW7QK)c{pQN(GOhUUe~v5vX`&rm?Wo6NUv_K zzD+mb4Ot24RYnq#SgM1jz1VhTS8&>;Wdn5qQSC3#tE$P(-Rkf(jUc+XRCk5Af>mmZz<}IB|OOatSQn)FQtnVO70h!TTek zg3DNJS&EI?zEbWJhySPD=n{$7Z=87Cu^(o4vE_K?r_0`0q~gue3SNE2#2*q{qiL~Q zCsLtdc+N@u_XKGa)$Q343_MVPuRu-x&k)mIYuR6*uhKx5O(vwL*7o}^kTs-+;ZYBO zm|X<&iRAa60Uy>G!grnZXh#ef~b0T2w35$WaW8dnC^Oo|!AKgJV5b(05+LCdlzS zG29p_c%9BtC}54e<5N00NH#F=gJxTMe+ma-J*f^TD$n-y7M4*IZgTD?%+^QYR9P+$ z^{R9ntR;`PFTaU>2+pe(%47~V6wmtjB&M~CKxXcc8st0ac#!ADY#4uk!KQ>M{=SSj z7RY#I0Z;t=WM_-lN3b1oNN3Y~@S&n8t@)#6Nk4a+!{z zHnH>#NUjoY12f)cJWe;J5aX7(i5Up!ApKVg9xn^>ZX6*F1rSXlnfRWMv2)gv60=EP z|FJYu{*sj;XmgIoCdV$Qe!M+}B-RWtE}|J@O%`@qn5#Eatp)wsIBwTp2`c9eJdfCC z0kg}&VH`+xS6Wh%7qJI{n1C!jt>vfZbH9q0w<&lpz+YQR!o}8+?mr zzk*ecy=5iUXuDCP!xyzaQsMztt*mFWB66Ej^KwFF0{ngUO(2h$7UXZXuL6zEc{Nw? znt;3EyIudY^R0@Due}f1zoXsT`vPouL4Wqw{pRoQaIMa{GG#A?Bh$Y^F1J`uOz{{r zIoWMUB!=oo-s>_AcQ1Y%(HaH+^$7}_!$v2BWr6~#GGdhOFZc?2B-&1)zg4b!rW?x;ma5Hku4q zJN&a8XJagCp*7NC`e9yV?nPJhd6U-p=^pRRpI(`577FBIH+vE&H+UX9XM&SIoi-7D zX$+FOsqDKmZ;P&F|*kf=+R_7 zQqNp$qcF+pS6@Q?26w~UsH4_fxr>*gNYJZ_a^%bFhrRmL=AgVpKQGzox!%8)8^F`7 zCJQnpMTeqZK&-CXS!acnJ*gumnM4k4$sQ}x9bT&!8d8#i17rlznBjBbT~PmDLw}YY zj(C79+Y^1&0U@OtS&^`vxS3F>(fo4j(U$DEdarL#?TQ(oRvImMC5x4xNzc~W>K9ze z=NOb%_GAuBT9^CQ%;tUEBqKqZ-O_#|)ZgA)*08WS$$J7{yeD*$NrbSU1u;mu>}%6X z@KDWHujCb~ciuTK*S~dsXUPN{lt(f3*(!$R&7G3R$k7KBL&A>N?a?m5XV|l&ES)HJ zpcD0^Q+5cdSULs$dd#RbNz+R6pzK-e$KR=T-aB|7Z?13~H}cucGh2&?snJkd0$YO< zVu4LQ)hWaSo4gBNG!p!`!d$=O!+T^Mt;T5nGp--7CLE;J$3B)i%+_0oUE*WeCaCXhltC(n}sxq&c12_7a%pN7dwp}F3N+fGWCrv)cY5J zlM|08I#-$eyo#3^t7S?-wG%QYn0LYooH#fdO>i7`sHP2}Dlb^^@0py0chM*?#%3R+ zho-3a2E4Y)8ms`D!95T~e6Dp`H05c(beRVEGcNu)=)@-Ckg)cF!W#i-4E6Sa$AAFk zQ3@#H)P?|;(-5#u>}s8tg0KgG1vws}Qut|yhWY2oIM&-Od!G(LX*^R3ia}jKIJ1nS z;4*;Ew|Eecf4v8ybsYk*W0OJNYYIx-9A|FNe1~U7nGW?C(Nw}5o_F`0?`}(9{niJO zT4kZ$#bo_oKn3$8%i#wUGDP;)RUIhjBT!QH{FM)*(&W%;T#RBbnBpt2%Y2K7=F?@s zV@U(ZL*G=DrGq8P9PAtB-4DUHgW!jEleYLo8c@Q10Rm3Ba~B{5sxE%=`-`YGNd+j< zC=Wn3z2LP*Yv%S{gx-6dVH$u8f-_S9o3R?o5);`2FE}LtJCIuT-v0$&iA$2XIUDw$ z0F3*kTXwm=hY|Lz2R+wyzSyAHZGGS24*1p+>;kr<%+mlHxVi*TfX!#K<2;Vvk2 zQYd+=;;awE=a9d`tLUjsxmN7(Alwzu^m3Z4*Nh zdy-B2pKqB6Qs%By>!TflqPg#xx_0sfAX3A-UK6BY;NOEUH=lUK2v79&Ef;zNc>N*j z~N!#w;}krrialK>0cM^D%!T!01Z%} zyoH2G!;5t_eV=1zuR#8EgvA1U}lzM-gLELpE$+x$-1FDDL=F87^K1kUZB zU!vM#L}vOX+5yWRcVaRMkPaIVUvC6_03-Bx5R^9E_i84=dy_k6x9D0g4+l@Y0l&V@ z#AAM(E{!AE*5_T0_lGhcP}pK-o+^Xa9*+pbei3T-TDJ~tp8*b1vaXi)=0Le}2rEV& z7e&gm)r0G(Vv}_;;Av@rYbmh2*v$!;zkLTY@yRI}QYiOMB%w zpiRld-#X5L0#@Rc-+(tVQZ(c+e&)K3sy(v*?rHX=5I}kn2Xobo8Ig!LSvtkk*zXC8 z^OH|BJ-XcbFJ5ozw05V95bML&;`j*FfGE*riZU-=BI(aCvE>B|rdvQ;|YeL4QB@yipiXeag; zpkWx;$?!idTK?_%N0|+@zrG_?=m2~{%Em3oqED;==}95?$jgN6^_)4&o+UnYbq z!^l-bO@QKIN@fS?_HvES84d+6E2Elr7w+~KsJ5^D?0Vzn2S{RXn}D+_i~TB>h@_H@ z2X4lv!-m+>ZR|wZo1Ek=W$AWQ&cl?UEB_<<%n}4VDBZ)A z@T|?8|4YAJg~1lDv}S|BZXrYLH!ziVtnM*T^7=AZHr}+SdT{LRWx(1|1^F?ryHYRT z&OhJ=O6~^(epX*(UIE8`eYoc`z>^CV4rv`^2DQD~aZ-E0zONMi@O!Un{9)Zo-9)We zK}91)+dcR0VvEZ)3zfr8W`y@lG7qXx2}&dy7`QPU+g~DvE#QCaNK))_<=KWOvR%Zc z4xE$3CN5{l6~p}6Cc2Y&=mpLBP`K{0Fm&PW5l9MH4Z^-4zaS0eyi9-g06BjSn`B7k zDgreY8I8DVCjQhWP61C^q2aA4_9)hi5{SmAG==G`ntP7ka~_XpuYyW}6{y^yqq+I@ z$=Ufk(GiRn@`Zy?8Uk^d2kml_985*{wG7-Y+_A)<+)s!N^HS5Tp-1LmCV=zjjghEK z<%HfOCVlHY#npf~OzbzzFN(S8mP~Qes}n)lQn{PMcw@r`ShlH!Oe0IX_6oZmjtn*M z3mhA%i=-x7W58Eg0gLAaxrao4Z>T3ZEa=&M3cDCEHvllJbeVOxiSx$`B16UeibiRT z)!sjU>XJJ#D}dM>+*@ozq9H4mZ5-TQ`-I+n4yW+Tsl4X6$Zq9QvhWEx!zr` z4mAKYmVQOH;TYnykfUrfgQ6I-{zH~+DA(>0F*m(%*bNx*CTZ{I&(G}sxV-$<(>Q)BsY)r@Io=F4fi$(H4CfAyq(RLxDNKfjp&b6H z(ULR3+}7nOKHnDIq0Hw`99qjieUFf02j$bfKWz)vR(2)tLQUKl%o(Nyjc!R&win>Z z2QkHeg}!B{zA=-v`BFAOmXUNqC*?G66Dr-WakdrqH68N6LT5IhRE^uO`e5!g zQ#7N~36tx6@~q9lT01}0n|xqJGQ5@`(AE&uYFajS{y6Zbw}kSP(Cc*{aI5y#YD-s{ zFh}?rD7u5!g_oLs4!#Ti{dY6VRF0NU(Kcg#sahRDUpQYoSL7B=#k#IkJSjZ?ys>Bd z_e^xv4`7kJQeAU(j)fYvu(s@QIdBf+Z=LezP)~RdB?YvP1lVgS>p5v1%?|@%0H>15 zW)#r^!{$I1?;g1JHe}(zPmjX~Y27;{q0r!^dXnR4jsU;tl^qd+Z3<-TUe7_rmaTja zqvNUOj=8>qxT}XQ^8{=)1yZXxu(lR?@A0HF#HG~;?iCe#? zGBMvaGtuc3U(Fhi^__|_z=RIOB-sNCUg`W+%Hm+~L(QJ9t7dWykem6_*;Jav6ivn5+b`nxPc1j3teL6i zOC7u@$m7eohyWUvt^YMD#9zPOSq#2k=U#E0mq}h+^yZ=RpUnw|)|GoO`24evvd6Gc z)1B+8^fCP|)z<)5EdAHCt_nWLrI-5YE%);E#GY{<(MCzmCi>Zk{JLe06q!-4u-c;t zvAVU{cirS8b{YS~&ntkzzummg%>S9>;-#j{;gB!bLMUon{Cww6`cg}(w9BD5sBs(o zXt5Kp=J5-%l}68Q>^5Y6AjM)_XM4TouSnc&^~NN*+4jQt{1=(-f4l;S?#;{x)2<%z z1moveVE?irRtn~(+Y{rDLidhP6b1RD?-rU%7i=##(mMWuHgPJb5iUT#?yZ>C14i;=ZKOp z@?XFql;7BR#z&OoEU#aiVY&D2jQe}2)10HcKFOYNOk&5`{d0r2J8$YtjThne3!XT= zQ+40~%FpEwM7Q16dnOvule@`i%Nq~%CKB3|?CLf@Ot$S}9}-~CwQKkS)f#t^3#BmW zVg}n30&>EPYVS-k?d>l4(l3OmqEb@up~jPE!G@i~c>TWR`7`Z_!2f?6Wp~p7bYXhOcL_whDJB=soPA`>Wymk|&Vk~3Z+9B1+ z%3D;OzDC;r@bdo7s-@dzh5m)MM6XG(rM_ijhER!ODu=I2kqKk8(9s1s0r`}!8;ru1 zL1&6rw4duJXg0(RVh2}_#`amQIn==fo&unFK9W2b`DXvVFy)1!o8TbqignOWg-`y4>ss^`>o|fNEZTxalc{reoDKFNU$LSg zxlqoy>9q@$J!`{=*T0Lg!PgEzAhthJk(J`RJ7=*g;gu?Rgy}1ftr33{fv3R3CgXXy zM48RUm5#?eo^$cXbfj1_2W>e$mL`_}EKfQnf^A`p!&pd6_#N8WW_X5aiXXEDcz3m! zWGU_47KQD*a*81d!!bUxwGJKQCjLq6&+&J`KFQP<7M@W{?_%C^ty16r2UJ~?m)%zi z1{CIBg z@0Q@q|(41|S7{)ux5C_pE5-9wo;vZb7ef$eVE*jQ1l!h3Bc|! zr%-XSSf|IxAV(NAK`-~{(_xRa&C!}a-@7aQ(yN`=?JBh)W+~HlBzC7#+@* zPb!4;N8=yBH4Jc~m8oAc%x|I$ki|X}+m&0!BwkfIyg!(1H2w1j zlr(HTeE4h75JZ3);w&Bxn)mG0j6PbE0lLKU+)s_x#W9-?87aj_uDBU1O81RC3ppBl zE|QzJ%(SogF43+xk4kN&I-kgXLdb<5KkTJP)s3PnX9xv;HnHtP57s1xZ4w=K?2X%m zt4N6US#Tsb?vef-_%%q%6plT2zWQIYAi&aZJ4516$c@-~aZAhFo993L`uawd@rD-5 zEMTq?wLBBmn^tbFzl)DNDut@G(awQDK+3mWQJ4=8kDlT$>%hP$mc43-<;VFzBK_eR zX)XT`YgZb|OuL;d9xOllK>#A>#`PU?ulFSFN2IHNi<5FVD#fwSJ_~^q^(fE6lui(4 z;GJp8Y!Yzqjiw2B!+UNuIph|LN^yR*a}WnKnAyyJS`+y>nzJz5O_*>EDrIr}hta{p z12ms>OR}wl{^YdxEYx@(FeQLv$D(z@AK>(jPvlpHoE_=SW~ z6nte0#Z(t&A$ZT(bFJ8bhd8zqKf=(Z2G~70r&a1+IV6JsMKJbviEZso(#a8dUqBC*}&5vvvOdjDyZ@^5?%oYJHqALw%UwkKUsQ z6+LQm2g2+V%BhBF9SwD}=i_U{LgyJxZBo$LaC+OzC}r5%kGsk7Sp zMgT0zB%?^c$HlE*^Vm)?SSJUNOk!|69EZ(6kj)L{o7qHzNcM!m=bkF{$08iVX040f z#Y2}M54E$Cs#OUg(e`F@>ek@K(PVmF+)Q<8bg2HJ!xX=k^2S_>mJG~XV^WGu^}iD~ zGSZU=5B3TcFOQZ+d!>^Mp2uNn^T%quALb7nTBJa}f&>~Mgcd+u93Zy}%iWh?c+UIl zz`{Gut0d&?syctZ^(u!DkQglFwM*WI7!6i+WyO550T8jSp-@t}wtk)+XFLJ4n2q8& z8qF#;VT!a4l2%G`EQA1r#mdq9J!{&B2JE*W+bSZJ3rTm3W<$Q{$9Um4OU_zcN1y6n zu?*O{@Yl<;?AdZULelUD1G5t2hDx+ooVes{={tyMxD7upuKX-o zs-AqhSD`|N^(scSy-m-UPy77tdU*E2tp<+K-R?D2_dXY*Wb>no$6>ntt$KdiM&isw zbt~yPtJ2m0$Qk4siIF_QVCmGFZ|v1b5y}fg-HTGTHlfCOh$CAwW$}mod=;>|1iqBJ zLmiv#G z2zJ!Uq%=Ept`6P^JoX*YxB!2(2=}loW-3tzH=3>Os#-^sw6I)s{KT*I1nM^ql5J(G z9Zr`Sz|-FQQQRN~vO*}#m6n-~xB3oVRrxy(MLh6x_Fui*dPXKMSlqClAbn9)(DCw! z!sIds7|2BF3(Y%Nh}LOX83#2D+TKpkOzyIySR_6P)5=HY8u!-WILq$V8ZB+Tc{t0h zRip?aY;M;LDH`E#F)!}5kRfr7L3Rx8ZTBo4A%3BSSRKI$6xabVlt zLhXTsuz9AXnAER@Op@Myo4g!;rk!^lr6ET7_GRToA++@M9rsB{tCw6B#q(eD-=aZ7 z*wm*Y6NwGW!@~%Nu zWcZQzhIcKGj_!Kb1VYt(n2{wH4>MpEyv+DBf(ux>M|vlEybu{6sv!;5b5~5 z80^zQh}D2~L?;xenS51NkXOGr=8+M9hF65Z@a&?nr2R>DoSni#+;VT3BcPUNEbEu& zOF}()j3vH)va#892Fqh|MXl_|rzHDF!jnF+`YMI5ZKRIwJ_y2`Rp_InBA{A3u=?`B zfcpi7Uol|O5`bmw^Z^x2=bnW9sMNu~V~r`T<9YDF52TX?{tGQ9OU>Ba**S!j<^D>S z({a-|I2+9RG!npjp~)z6wOL`YdlRzveSg{e3k6~5-XkR!iZrCVM_u+-TdiJHxOs24 zF*nlIEfj~sm`;&)<%M`X+3rkByq8_xK+HplV$514=HTA0*KAC7ct^MDWdLEtbYVz_A5ws??1@-k>aO^GEFiT?LUf9sBOgke^PS zLeYbRJx&ALxj$Znq{TYVWY*=7JJh=9x94kRw;Q<{6{v0oVce56|M6C25nWdQufo)O zuqtwgRh(ghw~|5HzmFrb=JIL(9W39>S2h20zKI@_StR?1-o6k(P~M8`)85~|eD z4<{HB)8tsr5wDP8Tgk1DACd#?%@*k$^(6_(8I3Y@6a59^^patwOffhgA=P^UpIWJ% zp)Xmii3a72@`&(r$Nw)c)8OJYU~wJ=lh5|2Q1syF%loJxSPVs(JF6-Cg}?b}2O8Vh zoa^jnCtR)uPD~#SVG=ro60!&U4UvLU*5FT3Vhl7ByE1{WA{l>JdtdyflO)}XbZw~# zvq8@YXv^k1$Z0sMNX3Ra%?|vD*mZ{2e`pbQOl@OL1wF<9ok#Ffq(eV2g3qOX_Yl!K z0bVlyW9LNle{iz$;NL3jik|4FCm~P4GgYf*TRyBM}adirApB=qkkmk?FT^ zE7cdu3{>DLu)!0c%&?HfA6QuJ0VGGQA;S<2waIjRiEqOoVO1gpyiSEtz3j>B-h`7UBQOmy%DM{n!YMSOgYx*c3>r}Z4=vzCp_x!nbzOr!sD#+pe;8WuG zJ57O1FK4vn@F|!Uan1b|u?RENhSrC+=yk~*IepI$`LZiOOEmtPMsv{{BK0gt+Cvhh zE|~n>&1_nYOIPkRZra%O=01M_gk~p5d4`1g+No%8NrJRf=H2l|k|$%SR}R_r7aBEV zrpzg>a`kv`%nfKl$BNkv6eaBd09j&b_(~JmT{_os?}W=_B{GNQgF{iPfCLJOj# zxDSvrF{(&=!j6NHbuR_HZ)mE#HnSAOmKdyR0X&pfxU%*XqQHbg_={mc9IbrB$b-pL zwaefm2fGJ5$dm!n!i9%E`|v)r=ocu}3{QC=nmKIG>c$nf+0kBK?l9P0DGX?}_b%vL z(3utG4j>}@CSd0R1%;)d{22j{1ZH~J@6EoiA4tM(ky z-eGOy-47c9NCp1*PS%Lct`{WytJiKx09()+GJz$@?x1w_s^qiY4Hs1zL|jL*Lf;`o zt4MZk@LD6(?C3hC3-{{1H-R07PjxC!RmYVqEgRC@!Q|#!d8$XttbFo*qzqjpX_#;LeLfgaM7lMrlh z*jB}tqQ@&1W#a!dP@=xmR11bonki>-I(2#5r z-}pmyO&Asz;gp3!0Sy1RSR<^*Ffa4G>a_Ge@DsU_Tvt7$i@RZAGoH~pXW6jU9}OMx zh5QO5TN8K=*K@c|IsX%i+Y(Qa@)`14x5WXgyGXn(4ys8^9x~oD zDJzMfVF=1^BZ9@r<^nQ_p?7$G=bDMyjRO<@iV;$**aVTvc^VzMJr~b3TK2w^JT4Dm zu{lBTYvQlPi{r)BEYEc7uXXEO;fWWuLHcPw#x06{K$Q=vYK98r9Ls)LIYc0TI3ze~ zm^yL_+bmXbEXfuLSecm@4Cn>vY3sFWH_$`v0s^wM0{xW0JXmT@3{ZBix9o&SQyqrA zH}mo9&bu9JEj!A`s)-Jvr6OH+DB1Qcf&#kjzAR&U-ZYgxEgB%4_9?9IQA`MbnZ^4v*T8J-!{?N{P1Lf;>46$J$Ey zOHGPT*;ydrq|RXe^{tfs9b0co)}c+;c4?t8ljP-=mGF1bRUXTh!yPa$7dGAmyHANN z3`hdN^BpC>hPES?Sqk z)hVJ%;@G%8=yBJxExO%YBp}ja?Crz~uDJ59)b7sCAxf#w(|Gpmhsn&Ro(X|$Mw!tV zL2CTj^2yb?!|!aR-q(XW`KI&*h{+~#dihSWPsa@$qBR{KD*$EPF;MGj0O#j}i2s^& z*<|j=0SQH}_FUb?4===>>k;BNWW>ODe_-q?9#oWzM{M!{=?9ZX#_6Yo-*i+9kR2hL z&^`&OxW0c&|2%Lpssg^!#{!C)Kf4Yl2&hwJL-yAsJf5v~q?E(p&g(;zU#j(sBE38rs5=+4 zJ%NZf>|loZ0z{8ZFcpPQ`hkD{Jv44@Myg0`{DHG@ddj8ajB4f$t1h0I>o!3$NDj>=15JT|gRp z15R*NG&?qwOF(=%2M81CC2wYQRIzie?wg!2Z!Qs4Lxnc^kmi5e%G_=!g5CFu0wY8j~sqcCO^PN zSlKEf?^HsPo=1;N{odBxdQ@Ao?)2a)rp|om(-}(WCZptKC<3bi8#k3MhssXD8L&B= zQ3uYzEwH$9S7$C-$Nmm6O1Eu@6xBVnxdaYCKkc>FV93O7V%Jzeoom45qVk%IuAu0# zz2eJ^Icoc-4DO~oq^Vii;iq00Utpc0Az;XH9_d$>^&$DHXP&0{Y_EU3bzOXx<52B#fYJJ`;u)##2$~L>_l$`Y zw+^l)fM&=K?bxmxXEgLE^h?yD+v<5+X-ke*Md<4Xl^Wv1K0!~=A%5Y_*P@jFiTYA0 zz(k5E3HADVE7E(kL?><`??)8Ju9dNLyy!T8Vmk~HXXIfI13?bDOeFr<<+s2nyAR_~ zZf-pYjE)lbgZqK3M@_hTva$x-B?C8{-)%~zjI0gGBt>R6Pd>Z!Ejav}2G=%J@Xg;z zTeO69Yv6bL6hJ{0Ow`=>d$Wppd4o{AjOaLX^yinV`G0~FvF zvX8E;Eq0&LDvS0So2e&54=b-N@#h#NZeAcH`0R_Ou;Q`?x5Cd5QN4$kj8zYECVd+T z4h=y;$Zi8^PhSndc>j zfc^9_+o|kVL70Viu#9m>cNj47v zO`Le|8SsA)O0>rIPW+fX25uMq;Oq5fF99J$cbZD}UMz`oBEt&2t{1Qutz$l%`0C1d zUm4#(cROIt5&qNlLzp{}KiwWp)e<+Dr_=E?d{7}&g7PYt z?oAL61&{MnsP~m-ZOODm0pNvXw{@$UwdAXwg^ZRnll=nVMHR_@j45i9IFiYvK1kpS zT<*gr)GD{Yej4AbwU8$lx8f#;>L6_2#b^t4Dp%bZ1&TWJ2TwXs%8CtF3o4~J{Q?XN zHwxQU$X@@7{twE}KdQ>tO(Omkb2Y9rNrm!=ws+lr^Ua0^JYKgxd(&BpKE`uY1!-I) z(xQIHzII-6HaEk;oEx^0?p*&rkjCOg4Cjw_J1^&}4Hso*uxmhRNbTNAt?D? z?7M4UlrE$9~RxZ$h8rGFR`dqr_l&x?T;3PLCwQYBs zvG=tLde3B)g6q5eR~R^DRy(_0`p3sa1FMY=DC#gR6k5Bdn>hQJL}2vc_JZ3xU1@hY z{2iUtF`w2P$GnP#+g(14K=@{T=vRSZmif&D(QF*K6%xvB7wQB}Y~G(g7XxOq(wE=6 zd`Kc^on2n-Gq4zZ5Yd{okf)R&kZ5xA5gl8K5}w?FQP09EE3di&6yW}(zgUoezW-Ao zs&-zhp-l8-g21tlaW6JrwDon53SY?knVGCrE3Y+K-WS_KMflCk^H@DzsQGZ(!07qD z0=3DQnQrZh7;-J|`I0B2Yo$=hnyz}zdu^7T=USe0OaTX7s$Pd)|Lv=swc=@N?Q_$| zD%c%$<4R64S`?ka4lTlrK!V&rIQw*Ls-WG(HLt+ywcRTpPF!FAGs|Muf0f<91x$sV zJ^l#QwFA&K#HXI0gQ@e69)QWq7-o-&gS-L2Fks0j^s*lHn2ace`p(TwRJb`xw+f`x za!(%FJfV!1C*+Q`t63atEV_3E=mTegR9De50Ea$UwN}f?;rj8AKL7wU5dmBxa zKT4Ax$3U)~oWIBUV_sy=q<-gxHMU_0b_+OgGmCMEMo0J7bRSrl{58NkbZm6$OJaBB z)THawE8hXobKx-wB(+}@SYGbS&~#tZTat=c?uYR5GSHBki8z5rN)sVs@Kh`N{tR$U zZ%vNYnEV{k8opgv3!?QbIB}r|u)Bj*(??(siR8scvenQFEmO+&@Z|&a<)`blFErf~ zcObjHpf0Krw^*7RaNSMknP9|R);HyzZ1vrC#&s%Mky_HLb_&pFTTIBk|7BKWrdt)8*Wto&+ik+DqEBM3FuJ)(X4_!4 zL2pMlO{0O#m4R1MJ60ufc{n&;<%BFF8_)+jQ2vkF4+ z_fQ2fZ~N^?$ANzBDv*LY31;sh zr0@N&^gYa8U_8fL;a66_Oz#@ zX?%JwM4|Fq9=QX=5pr=XT0GFXIw9I1ba9OwK4uz5-G|p1Bvzp#yf|V6aJfi=99KJ% z*@fmCBZx0F7=erM+n>Y1%<#0BbKpra6qN%;sFxxOHKQ{A9T5irC!au_3<6RBHBHKM zlgc9iG4tcqA8;dbGzVEs6Cm%NuL1*8DCXbc%KU$E4fvzi3uUJzo@q^kP9)66)JrgS zt@qPUfPxjcw%yZd2>eA#6Z%vF>Ij=-kL1V2@I=;pVnsMG+9bH1LW}Od;3DzAhIDZL zTS$i#OBbZt1DETQT}mr7{l9G54Uoa_hDxgsW38pV8t48|`1O-Tdi7a8hlTd!(ZMJ| zpn3ue_rFhCpt?o}J*tOKY=(xNgI^pycwSvLbqG2y-|^rgCG>3=G`8Zsvjwa?0r>ix zCHEG*xI7X#(c`;vnMLae^nk0Ml0N*PAAVrzC?f&a^&$-jBx?uVE|2yMoH&CT8py1^ zm2ePF`MYRPXQ;va0YyF3G51C0UfoCY3uJbnyP8!psB$gf(^`9C9c*J= zFegr)G5JgjZ9)2A>2~y9Q0gro2aR=cyxPg4B?_%d*Vd+jd3Opv1~K5i!n{j^8y`Kx zeGD1_!LwHEy+b1PE~vkrBr$+Yk5)8Xug>uuen~|NQ|M^-#3?k#&`t!Rv}C062$TP= zE`J34T%_JuKxLQ!!a6jFEGwLC%>uubMqhJ&9yv@D!v+&qxwahEnLGmg{_3wC@aKZ( zg$2<4JOqn}6JF_l=}DhQvdr+w^K~TrmJL1Qn>yN&{Uy*mW|hr{`IQIQZ>&VL$OIdF zr<$(MKnv}qfLF8R(grD^Ya;LwZ3pnvDR=_!6Gm4SDq*-QwFv1bY*4TjfSTh!TY(x3 zPiyZz7o=;$2-oGO{b-{iefE04%$?(&f%^9T{$nk~d4u1bYgwjmg=G^m{ z+Mo11F0|hmc~lId2sq&otuzYk$taw#4&%?IvPIv8x|ANz40_{?a(uVaxa)bReCN^OPMiQ7c@}E8y#eh_`V6>r(5#fg$iV5dP16-x3GU zcOhjVq|+W%B=RqWmXs>K3xh8^;G-q8>|>bf-;?0COTjZRJ77}D8?_J-TvVbDq!)@# zekz0=TtuFsi3kBRbhNTK{f;L&Xb07S>vckYogort^WzG;`S}zEk`A=c))zN2siE1; z&{neJ0=l3dvmys?LtiLL?%zTqi=&IZ*znI(=L*x{I|Umgl{#E@tL>3C`WoIQ?2f}J z24V~c^vfwaI+!j&)Ur5szEm|RyLvAU4gB=Hn+2YB4=xKj5+Cco2LF-GCErAmyKB1kDwO--eK})544* zmxo?t`RsDY{{b|23RL9b4%*O~FejH&Owe2sxQt8QAF2A>KM!B)mY?EEf2W2fK1^a)jj){6diq3|Dp zUYdH!&JVX;M$Iy)KgAE1dB9~YA~{NM*%(~5{Pb+b6F<1^FZZ`D&@i@en3FdTVkNSa;r$Zs+xAoW`7xyP$utE35N-k*#^`vru+CHl?*CjZ zso=U}Nm|uysoqmUb01cB^T2@lKyz=s`{{p#`4yZdHVh0Hdn60eS7_AZHgaES7$V07 z(DNyQB0TOP+TgCAa2C*Qcm$98+r9#WDuf=RDgjj6;|OW8ss4zSKoBJ#g#1(?RI#`L zMOe`@_c;SoIv*wt1p54B)nLh4(r`T{^ACkjJn3&l;7vcE*AFi$efR0oBn2Y56V6P4 zyK9A%aiID%ZA3zfOLYQt@=G8CS)B+NSS8f*$>gGJiU}lO0w85_%@8st4Q^0DZlK5S z)QQepfShy%(&?`pe~83#HqKvc<2p{NeX=!I2qe>32=G+xr^l@VWgo*QhJbXiK9QGD zk39YjCz=SdeqggPu5_?N2_he$XI8q*=v@4DyDuLdvS#LHBx+0ZQk$E09%O5r0&4+L zF9c|}`GBs%Ab{PcrA$gJAE8>3btr;6ywMN2sYGF`$Bs2qvCR!!7U5?vkFk-DL$%*U zHF=}yE;$_bQ)mL3VX(;BmcU&7Zxd`~s~czn?bac7;uB5#yLS<_%d818-hj<~e;tl| zsa=9nfkp)IS3+!^eI|~r6Am_0v&`vL%}#fK3b$843XW-pTG{ic)cz#|ivGDMn`VcK zU+`Xo51$7`|LJWqjRkUM|8xg_lFh!_Ox`G7v2`H0G<5S9Xpi~HqLslz-FTG+b(=5+ z<5if<|4w#s$4p3)tV6_GBenr@IgrBmrm~=!DE1L0)F$>&!UE8oqm8QJa28&VxyM(M zxV3UX?5mn$*F}4L7Ru-ZfcGVYJ)MU%t9_0$iFF_e1^dsb{%3$9+n=FC*&}1NyS?E) zcxUM1OArV_Hy#8eWMQl;1xSuVpJcWjHq3TSfyE*QxfBSA-;J2tGX}=SsH|t5{E!RN zau7=IY>+&~98x_J+C|FmqovKg?&aaXVCY<4l;iHbtmrRAs!rTFFDwyly)Re$4Uiya z+IG8D1qb1q*@Q0utG)z^jvLVAYq7y!!%JP#J2r-)Cej;5VAOzr^fd5^eXWg$xID9W zKlMI~6^GPZ(bIR3$2{4)N>89d2*^B~yW+n6SowB;vBmgD{mJg_Mo_!q{KFsIwKCOy z2H3pshTtlDR96x9^7sQ%`sH5?!C$w7ONAmri>a}6t6#a1E3F3;SMU6TC$bdCew`A1 z47ToHTj<>X9x8Z8Ip!L#VB;RKDc z1fZNx(lX0$X%BI`0|R^*7Q$C+qd%h`#fCdz^bDpWk9@Qs(wKq?>1x$~AzkeRhBvsh z&@*X7*)$m4HKC?#o+GaYR zNXR)le*T!B-KQ^{v_RTVRWprmqcF`;(60QQr&TQl+~`A zFNV$NlWQHErFMyX9TGN&t*Du>39#&2`kqJjMj6LIhO#q4yHx9l=mSu-T72lycvE|t zd2daLGnL0%2DCf6aqTK4CIX@)Z-(qjtp=C&YS-)B_^Y5jSqMRyut3=GQo zGbjDg0`Je0UyLo{Ff_WQ2u4LnWULrra95;#j)M`i9=ByB0k(N_1Fu z?l-xZDFSaX1#uTtKAwK@>*uhRTdYQ}URU1J)gPVi12?sS<#v(NeGJc&+l@3cNC0|N(1%^ePPjheZdVjZ!84tlU+OXSZJ->LH z=JwE=cH8%l=IkvnWWsdVEai#KbQzd9cQ*8+l+UX!TvxGtNGnGE0TdC{hN z7oZRcNOoL~{;6vdA3DuG%@|ht#U&uGNieN{x;XsOf-2|Nc;l9Iz6#&_x>D@t^>Il# zv;w+E)A?eq*^EP9+)qCVljxx40qtpJIF8{IxOT0VJSY&{N&{vk=GWBD6mV?@Q3=XW zj9QoC+Fp6PnnaINE3*9ggxrCS5A4kJV0j=#VUox(SIQmWB>IU0WHcvgamGbdH9JlUuFU`(3>Bw@6 z0U>y_5{_(q%{FmepNExDzLY2!6<|qk{y6_C1)G?D`ENZ2TRaXnw~x2`3ul6|-`do6 z!^whiM!8ohe@kkSg<+IseQyN#T0CJz6qrT(*}c1s$J0Bab4NJS3!Un7kuVQf_q+eE z)_uJjcpX;8TR+1xqcG39>69ye+3ZQE>!_ZqVADazy&T0+PlVzH4h=# z<68&|tGeTD&op{0VXxb%^%YmohZn{Yrv&njoa{)v*g4z(a;~jFebz^NcqmX+m7#(7 zXc&e}vdO~8L)T~jVQr_Hf>G_qofvUW@oe@u9)ezvsi@l-!-+VoS4m8f?`ZTXJqjEN zlp1f~HOhW_M`bl;EW>#Wy`#s&O7(8$k<-``{^*LscIXJISc?)nN3>67Ki|BN&gXo+ z7tyU)h=MT>|02fnMnnPcPrwdR2rHnpCAKOK=Lp0UrkZ0zeIehUSje4P)blp6)Ajjq zl5Qa+>)6){w$7tbz3jMrPP;4zcDkguxqEDGPUi^qz3>WWBog61nxZM;5X zpx7G&o5Yl7-=DAFj`r&=)#TsTBMU42W%@==c?}nmQz%MD_7ZBihs_KWaCrwIxo+8b ztLK4Zvf#N(jzgmECfjM7$F_YM^nFl=F1|G`*2>36a9Q~Y+HNO$ zyA(V+MV9A)HG}`L6~JXermT1o9LSsLf6LPSU|W=JCL#U?k;2LGFTzd`^WFweNq;i# zSrz*wM12}4k2O1fR%XPJbZOfdP6Dk{(WWpBZ z^2=aZT`b<@uuiMu@>uzEdS`HrF0;Mdm*iy`tO)5nHtSDLI|Ub48l@(7sM~ zT;?ONg|QvduN=C~Ta|)eSJUnK@ElRl>iW6UqcnMLYG<=2o7KA9-&o^(KqXfjj_K=a zw3B$=L*GQUugsbB{W#y}-QKJM8XuWe^>3Q(+!Yt`mTBqYc28-%c#OwtK%|m*yGNN% zCF+p^KK2dN&K3wvd;I`;g;!vB=ep@Ca6(HLcf-S?Yzv;I5 z6~$_NJ1=4(redlcj+ULr>1y1!h~qQ*hWW^*p~v>7{A&ZIj!m=N;j^t{UJ`J4>%3~H zuv3oH#T4QT_$*!iijs!dpsjC|Ml2CK%FA4;&{0){ zHnK{s432(jzCs#hZ<)Et&8p6MW9#a1HX8O5U?Lo64g|uH zR<&8hvu{#T2La6+u&8ZVpu(Ov*bLw*kP;aK&tD}GoR9L!&UsoCI!4N3KoKXk8>UW- z;`K;+HXK_MTObe5j97_5HvTv(44;&EtrYwI&&&r$#IplC; zqw$7D?PcRqaNOU@-mx>vk9Bw)*6GnMkDHN>u!qwgi^!7hzvUGDVSVl}A`m#_mcGuXZXqot*|*1k{oaQbXEIA1P$*L^6pD{m|I z!W*+t0eWz>XM&MP7{MEM6Q{%=Fvo%`%fs4XawzI5qKBARGQ@JuYH=FyNOq(|;w4DQ zxcHC!Hd5THA&xGMt94&u=5y>VGY!U&VHTq4PGY038h^`Wj~bdEr;k8#gznyvo8Chy zsth{j2b3A@#xL+^^yYN(FbUo8a|JeB)3n$t&~P@6nhgtjuHlb+-z{^i@!DJE^P5@m zwH8L&{UF@I`R8}h+f?GTP|Hk;8YI-YD5~wgy1y?D$EY^-0~*%pMD&Tia+&`d|13yr_8C zIn)B}N#Ek&L$<%w7x*p)JgfY0p`~Nj4%lTp2;tzTb{MO#F%jUxm*%6>syC8joo9cm zg`7!=sK|FO^GW6*0KsA^pOC{)Y%cJ$7+ZeX2g1s^{4Yf8T|0 z-8pgYNtWJipAYQHBG^0#=sS1DZ1UDhH;bl<8K1Ca4PBoe$!|-~!b|w;ErEG_-8x{K zyeS-{fDej~z0YH>b%gv@ibQMwlJoTVSCNY_QESj?w|=rS0N%uC6yogyp!GDTO|v0I zpmz=6igV1&Hdj)mv||-?>uJtMYTzC*JL8pLxTRXM>>wn64)+M5;Em?F0N?3^U0MYk zMe3kDp0nAu2&{&aLaBpfaA~O&`Q|b0>(_>U0|d%## z=YJt3A>b=aE)L}BzC(w?#mwHzSRr=NdJVj-(J+vpe7pb-O3dA$i=}`kwUaXm0<^@g z2G4J4ww?U|fS0NUA07ZaEpd1c!g4yA)n4#!S@g~}5SM{!Q+na;9S(roI7_Ci@2UY*lLN<+ zE>v_EKsS&S}>`T3D^OaS3T+EKn(LGq6(m;-JsI@yxTqQwoZRQU?0y0b)rM^Q>l z5}juVWCnEb={l-70@Ctz_VY(i^1VVNpLk&_Q^`b!(vSr>82<11(vT&-0a_LJjj$l0 znl%d2*7~zyaT)5+w+nxSBw*^H;pqw^V+}wj5fgYK!s+34g5Y)7*&@~6P`{7i+J@Nb zB>*S_VF=q2bNC`O426b=2wpPaIZDnImZ_;?5HVH~%7X)bn6ibzq~(LeG@H1xL3)hIRnY1RsPxqX;K(2E%oB2SoW?Y!oU0B)X#g>&-j#9NnJQ3gow6M47b zwiML(w#NxpUFbJ_Q!c!Rln*-Q0g7-RG)DB%3OJMp$pcs6x(q&lhyn-nnd?k<>v$D>9Dc($)V@`C~J9WcAUvVKF`R0O9xAnOB( z5qYD&Kg(jop!1}CGDJuSj=M-Dg+WsAOP&yD+Fq0#M&e2v@7G7&IaqKqcD^ z@X8@j6dl!R1)|JH?O*9?a_Z;R)8}FCIWtWIalI3C@30a#b=K-%-A2eqveY0Pu5OFO zPuusp-9MD0l~!Rwr=6`T5zLiLv*dF}>X7?*e9 z{x^BF^c+C=yaEnas<6&}ofCtQ&w*Qwh#o zK>{yA$Y2QFG)g_Sv$1$%M!cQp?=B_B`7b4IP_tb9ar@n~vnewLy{baca{+AmN2$(YY_erUC>+y^mYi&@YJ$ViMkG&Nfoecnvwj< zuw@v(X?DCA!2(F+^Pv90%i8#R<=m6gme56o2uQlqglY2J16e738?v0GYumWIJBGMT zHBpP4bWO5)D9`9xP<4Y=4g*xmoByed24x?f%ZjMOg0qDdiCzs?ws#XHAGc&lWz_n+ILoWdgGpg3WRQK9KFR*yV_Fz z8Ej5gEL5To0q8{CtLc;cVFPe*!rw#GLD*;>YBLDm>1IK>B%SZ;K^w>($bN!t!09RO z@VWoYJupJ6x`MCLC<^;*xqyfDXxP5+O5kBS4OC~&$4W>LbG&UVfj8DsFpB0-ACa-X z+qjtxTqsOMtr;LnZm>YFE92}^MAkCGFJYTzWc@=!OGl7qbyZ3NW{B$pq#RVu)$qXU zh_ry0-9g66&mG(G3+YNp^sYij}*oWlh(y^4VU6J84OAuOeN zpi5T*A11!^osMdbM$$Z_47gYcF66e^3fp))hTP*+hP5tv(jaNz_Lf=+x4i{>`^e3| zb_5P6ZvhPx!#ifoZy}Ax*H~ip9XjJaEWOY-N#^Gwb)i)+zGVu-5Gg`u2$Nr@cx;@b zaVG9j)>M>)IK~tybyRJ{X}qlm;ObCXwlo1lhefX8@b9LpPmYV-?#;conl1e~;ex#l z(5mfN2rh#4Uw!fP-48qg|1i5gg77tsUWdN?80GSqfi@T)8&nzLt+b>Q1$mU%j%o}q zJl+=`ziFk~PQ``Nea%`u&}ooU4K58PVVi!_r5p!L(F%yK`W2s^HnI^z3O=Zg!143t zJ0q7i3QHNVh&>Jnjmg4L#i*0g8PbXN)GL7qURW#`?@_~>L&&wPe|^&vWa$%PBLvLr zzo26S(6Piw*%F;N8plt6M^;s;qF0w@0iZ+o9M8uo&Pk`C5#QfN;lav3LQ|T;*2`bk z9GR8SxHhouq+szCE?p8nJp{#&jv#;IoMX)F`Tcozhaf-p0pQV&0Nqr6l9;PLwi0tC zTgON{+Z~E{Z-KO)bB+!07HjTYE`i{yMtujCowqj#9tS{7*jf4Wvmb;t;&$1)1Q2?@ z$k{pFnX>dYeqFC;WN)q57vFY8SpfRV$*R2rmj4b~{!c(B@PG16ezHL@(&tZv2T@?5 z20Z)NqECN(O6>ZU&dFHoSi!;IQn0Bt>@SQPXTnNr%kK_C%3>aJJs(22bcaFTBMUZ& zkDwU0P)9CP!KZ8FAUXf)>b+-S*5D|xixhQ`L+enP!2#E^mym(`KS z6Ni*~b9J3djaaO>I{<+COOPtcj#zOvLfMUV%GE)0;{wJ*n)(kH@qJUq2S_`KYEWnBMuRyVf9?vlfu9Pz5 zepM~?@Q|~Q)P$jX(4FN-dIAvBLfO5v(|Ew%noB+fYJs9jxzr zVH;Uk9siRWCq0o2m;LIPp(YXs!wA5Nr5wwhbcfztZ7@Pm#Pal2Y_lt5Oi-Cj zJe6%c++hS-|Lvr38#-Joc-xD+~E>9rmft@;dF2ZBDlke^w+0Yv=q=Ch`zhR9V&^_|3X+~obQod z_q$4PtKbeG{Z@kg_!{ou0Yls>awnV-GOPc+WjnzT$BO(t)aY%vY_I$voaQ`5JP1zA z_iM;m*NC(`TMAja)^LG0;6Fzp#y}7k<_H+`NLb#FGSw}p-qS*Rl3#c8!SePb(x%3O z`{Nui4`)rdbO2(;L|O_N1L>913@90nEX4P=SUmXno3QlSDGiJt$zFk_M;@B^3FbU0 zHCH8~XLAFJo0i*{a$a7B zGd@x^svp7{?CIZ!NPg63yvR2PW~kkqCV)^COLLnWP$Cvn*{HFkm(;rj*8T3#7SsZ^ zsJO1}fmw{I+yhgZT>0?H@iX21z7SaU^k0TzfS&%dqPMhO_jKyh0VlPvd<~WRm#?)C zI~PHDo!y)TFr@8F6?X#1l>^n_tyk0uDGyasY$NX5`jPYRu0n(W~teNP}-{5}KlvHm5 zWjWvzUMn5do64g*cS@~>uPu#Vrow_POh&s9I*we(yE#L1cN=!*o<;3Sz%R32UcNh4 zU%plarP$?bwjUE>D)-<-oJow@&ZXa_MYbLB&nx#pB;wl;^o%#Kp_NU1AsMY<;@c7g z{dv~q^Od5;y92%;K-&X z1+OjvS>{m!1Xu&4TPmr>9{3duayMv1k``}vaOiCR zHl695Ry6`Ya{XV!LgSeq5 zP_Uo-XXZ4#2l7SU61Cj>5Bdw$H6s7qFihtkl-cx-k6@M_8b$fVK=u8La2k_E#NQHj z1>`A^yGbAzK?P!(uGd0*qKP%78|2+)ePJxZG9uoo6*a6hg%R z?<)l$?#hXaNwE1*qtMgC?-VmJ25o~uG_LT9l~Trx72>{JHY)04t5>o_~WQ@jPC z>2lR3N*`!x!R8zAEgP{!{y$dgeR`}62OQ-rf{oEeeGVQHqEUnb{C?eZ@Aq;47sw|e zW{rS<2(v|=&^~$fHwDAr-=LqfMS(?peiN&cg zLlMmL{lI59;ZREy8ke4yWA$IpWQn{6-^f3XQI0{uWS+u{O<-Pu`R<7R%U%8Vek}B` zDr!)zHx~we_WY1699k&b1-%sdIQf50j*>kOlac2C;qA@isr=fv;gX6*L?tCe=2?`X zRFo;1MMAQbsfi>qHkm5pW|PP~r81R>2Fds>q#{%*AwwlYnVw@w<-ULSb3gC%dEd|b z-~Q}+U+|vx=qW{jIKw!pL!g?)+F~Z zhP*V(G;n;e9=NJ-?})TJLpXVa;Lf;0baWv>tJu_=!R7kBEnW!4s0#mxn>C9|CkKz= z<8WJ#IQWNP^vVSZTR2yKO1MGK_6m9~%ku*7jnN@9xlx6@xFz4NW3qkDpRF04Hy0c{!mI0JwjN^|!Ez0XBVY!x6v?Yjbs$FefecBcj4gun zehPSa%esmY?k_85tFUhYZW3>RZqeBAtp;R90wJS8A1>ezTg+_P=;i`e5EeKr4{{w3 zh&0QuSxTTr0Lr)(fI30xhm*!26^#5juhjT5kauG2Rzn;iT{HWPmtx>@< z{3{2()FA8CVBOQ=X<9u7)Dr{x?NNjg_smJW4X??(VBU5N!SoaV5`CB`WAm^^^vF9j zR!ASgoR6f>FwE?T2A4=xEGAi$yvi-~cp&A~N*Veptj&mz59H7%az-8pF>kjdj$1_VQIdFxi@iM|IvzdM14wD+llGIBJ zhDlSn(3Eom(u5=T1}*=wUTYL@b_GvsXB919;6t#Ts@abm@%vZ&?!Q!W8E(4(9c(nK z!g)4s3u_}Hd(A4I)IChdIxVY6CLeG&YR`z20$MJCzny!hC63!FlG`%#?_l&MzaMoaGy!@{Q*G{wW8>e|0V&ooTttZj;lXc-Gu zFZJzPsSST7#b4O<1E5lomnycy<0$g#b_)G43&dU1uR2g0!PjsK*+whOShl+kG2d$%~%hVP8IPE{ev3ntEwP zJ~q=2=p#OUY7Gwkf8A*=CMC>=&c|n&w@$~J$`|4W5HF}=VNpxL5dNRHB6Jo@Ld9wf zgB3PSHU+xxAkV6RN+Y6CXDPx|LDsY8(c*cS4lsryL)_L_NYKSa8=2_OV*#s>Su({C ztRNS-yA*?oUWQYPNZZ!~-+wPUWiDa}bpsjR`zRBK&_ww?uxu`7eDr5n7dE{#(NiJV zp8~Z6Rx;4%kQ39@HAi5EzY?PA$+4AxOVj!hBmJD`p(|G~G4IQtxnhAMJ4D5V=V1l! z^lU>zk?C8-vuwdUI-eeNYtF$Zu(dM}Xw&v30UOBjd_M_J25XEDb*_v#raBAx=qDk? z_-Dc~v7^5$&ieW&bGhSFPt#Y>Lm(9Gm*e{6aviL69hc5^(+zULY{Et=2b6`Z3S$Ql<)SyvmaO^`wvp`P-gz zfg~G%nara@mMe)X#fq;JEK~IvK9EqzZAII~nEZqwdfMu-8m7z)G+qFK7?XFjhIRDE z2^-ZZNd^*sWrS_L-p7g@IFrbnTf<3IX;~4<%DR8QYYE*CEY|AzhRs-e?XU{VF9=BP zz?gNTjk=ea-os8WAUswQ%kfdfRQ<>(Qj0duAR}XXkPh}EEZeMyk1e@_$rMpq(TB?_ zF}#|dLA1XL-~zn3jN7(h1^qYJe=7e?8El$h@h6CUXCCQIZ)KnnEds__dzjDENvV(*|AEn}7Dvry<*k^8 z3w&wXSZ2so2v?O5SOm(kA3-L1wv~Hhv=s66M`mQF#OvVx=XiFJQYO#ww}j=)<#TZV z5WL_e-2huH4w7*8vL!Z%r^vW{7Pojk)D;CaC9Pbx`h-zW(Dw>^|qo5WN$wP{7r@Kfdrbq zhq%@9rxzhXs6rB7U-GG6O&>rE+M@37B>#lO^A_l)HLvuX96#Am3Z&flI(HlV=II5w ziOi_kD$(A`x5%PcGy5kY_`?huiF`BsJPc8)TgVY^ZgYLVn80czYyKvRewiZC&jB=+ zeALT$W@Oq{h}cO9{ckM$si-bKwhqZh@eoD@htJ;V?MZyzpME3&tsM#g+x zZ1c$5Im0oppf&|blqI>4ZxMad+OYXovyea<{&lz0o_r3#S-@XIe(g8si#u8&^-CUbpMTlMW z;%&0iOwF$fX6kHn7#sO2oL>rMv*x!C&Je(FZx_4VbQcxE)KzcLZ9b#Sj8P7l6)T3x zp$6-(%Cm@lsTGvx3s4CC6%P9B#fc6n5Ey<1V%;I2l#Bc1>|fLvctQ7MVc)TktKL!% zc)1LrV6+*cZB9CphT)DVGV&l7w&@i*cF%fF1eOYj^dTwR5pT{d##@OR3|;ZU6jq^z zXOU3bdj&eHBdH~Edjv*+wD7~9aMd|X9AS|oZZHE0+cDo__hi}ehH_Rd0_Yu`Nibjz z1(z(s^y@21s)bBP7j<8^>BqH`BNw1wGagr@J)S79NfnXwkJS^W1>{Txi;5-9NPKJe z<{wEXdS2C0#rAb`jzjS(@AHk)&1RxEUYZ8%E%o$xc_D{kuqark@eU!b#-J{Ydx}FL z`d#tHfosR6&2>W18`y9du76EBVf)A@8cWoQd`m3AVMdWd$1klGL_twIywe@AB?o91 zO@u5vfFrrgz09ms)R8S3&(FIY(Ptlb{V2PHrSi@28wi^9)e&+F4%R1397rZBk@(Ns zZH(mCUQ(`+2!Mv)a|XqkR^DDsL_-h8c@&IhcoMl~BATxREvmjX7q<=|>h^w2_(7_* z0qXSiKn^39XqFH1GbecwQ5zHukV*S2p2rH{=|~Tz^yn4E2vUoHjIcD9Cpf8tO zK7o3$wvRE?6rg9b;+kjST5z4rtTJH zjOhG{uu6*~h21V}0zyC!h;q}(-N(NWl5Ek7xcQu}TpNRw2T-K><(Y$KA1-;HO%8b4 zQM0c)kDfqRo-oT_1KHP|zpsH$!rbrXVMpg7Uoz&&h8w$t1?0qnYhIwScysZEh@_|_ z6JTv*F-cc$V596)SL>qR6#>HF$=-7XU{W{3blM_c;@;u8s;NqcVFI+3Nm-b_l5R~? z+tRF?0t4cmhUX+07+Acg8h`}-l-kgeSMzKIg?15IDISgW zdbb+>GM6U^Is3nGmqk-}jwLVCFyz%=BHzg+H-S(vPw=HR^3FGb&=ks7-3;3^psFbp zaV@n$#A7|pt3#DGd9_cU-MOkozm7$plCFCk3=pb&N*>us{{)&CFn{P+&9F=H5&taC z`$$2PO-heY1m1R2@J?C{mxz0w*70v2Dqo;z^>D_Azz-Lx8!`kq)J@a~OHM?VvQAyG z_NvlrN2AiyKaLyk?${S{UA=&PEB{Do@j|W(Nv8pFg+FYtm!* z>d?Nh(^FNduJG0m?mcpkQq+?ZDAxK-cQBJjwmDT3`7!2-2OY%b;!8#JN*VdHtFo_UoD0;VSBj3^!JVJcKm?eb=HAueO zY3!oy911&@_>eO#VJ>p%SWQhL*V^a}tyis9-J&+x3Rzole=KXKai_7EQ_F>1@X0k1-3aalIqAeFSUJVtp z%&UDLwo^O=2P#)kTpjPcJ5Q0S;!WYDp04vWuF}#!HTLyM?-UuirQ`eN*ILy=`GTH% z!W|K!kUiPg5?><58)EQuMWwvD0-L{L?@3uygo$Wt7{G$uY>wjP^^QeveHUuc*%>BF zC1}Nb8RZcaaCt{blP`-0hMyf!e(kHZ-uE`$oogtt<+n3SfJH~x8q4u9=9|5!j&e@W zx)rvlG1Sz;S=pCrW@jtPw*{m6MS@G-82BL`rCf1>T@>3qef++#{*Z@oOOq#ornXa3!c zbWMR1`XG5{8ugum`g6-w)1PZ6K{ccV*y@kn6nIS%QLyDWkN?>6Zagoq#$eQq6fP0V zt*yR*QgC#o|$GaIOm<%`qyj7Y&Pw7e|a3Sv84oH9}c>b7|F zRsW0D8AH@Wa#X@W_`e;hd}j+i()jl1$m%x*j(cx^3a7UJkOJZ1wq)tkje6w~)U|C` zrVlu+Oo!4M*7B8|){VvP+TaoK_AAHer~kk}?k^s=qy$dRgN^}tL;ca%A2qRGx`L6Y zQ0g_wbw*>ec8ZK5x*hy&>|ZNTZT?)pxV<9A!%#Cwc~yTH1VCG2Khjuu;`M6XjyjTj@pDD(H@J2*bvKJuomvHF~V@DXXR z-UlZjTUiL2v4Mf6h{x9pbJVQW88XydNTOgL{Vv9>YizjQr6zpzt18h$P8A|Dl%w@Y zRH=+O;+MrP?`9Wh(DKz%<@fkmlZ%oCb)v}*&3WlS&4t$y9?(tK@2PtO(q(&Is$UU8 z-0z0Mb%Pwe@gK>FEJvgFf(UY@BJ7#I?xg99^n0(JCGMkms_(|G!W9(xiw6AeKNB`b zoQO2d%WGU~6Z|u%f4W=D$N@Wmw^upd58a}Ab~h>Qr%O&z0j2e^niT@C1qYv3JwKnv zT*^zch)(;|yi+Sx$SyAPTNRR>j=h3aNxCPz1iX(k_hdesF^)7{U`URWpXmDQr6nvJ2g zN58Tkp;`Z77yBa*HAd=#0q+2>=!TMS0kV^kO!}z3duiDLQ8RKrDcjxpi&6XrjAE?(N@YYZ zeplt`2d7v(Y;m&E3mJY!Ubadpik9GKjkwL3T1`fnwS&88IrYvA_fawLjC*v-akXi5%lqku86e3=EC|5nCr7I|@3I=Yv#e;H<5S8L4iq(6XLaH@EkNL`x@Vt~ z-lLL25nrS{MVCb-ro&!5%aH9J|JHoV2Xw=i$}T-EH*G9x+Ps^M0=B{NT`=rr!SumY z-xT5BJYkkc-@21aG8ZW_3Xg*mR@<^;mJ;;;Kjxuo~m zp5dEPm7eoXR0*>>ZhNRZL1M>XD|#L^#D8cU+X*&PC3ea~Ujf01gEzC`{8? z;gR5%lW_r!IC1`m-E3W|cupR`pTF3M8m`lv{m>YQDnb4<)<0K`OVhS5hN4-^Mg+DG z$U&O5?g{Jd*@t#Zc}1s`XU|z&9Pj_yEL7Hf?R={FdeNL3l8{d@O%A)_IX-?^cB&#= zi}0nuUtUqLCE7?oKQbMmwFtanri_%ixAJ#9F=55B9dPjfk|mkcWHGck@2h*Tnf(ED znbdIuwe#n;;s$fky~4As@BYO=`AD+=QN%j46okV%-{CuT5| zd4R($ue>QCfcJ({4rQ*l8Qs1^a|UM6fH!m0L@L+Nc&6r{a=ap*|E=QaS~^J#LZI%= zl|mSoUt}(k!g7cUyO4n3E-)Ezf-#nx1U<_XJmM_4F9k4Zk@R7pI*{Dm*so?cN`BfY zAj?xaJI+@1aXehv0xa~`P%D!1xUO{%as5V8XyN1$P!UjYG)(I0BMahVD}WQ&dL7KP zS2z-{@~qh~)wXZ}yr}=YtG4_zNVpw%DOzsT&RvxK`;I=CtL@NodZ6}4=fnIdeM?l1 z-A@wBk?NjaHBJg;fyQA?!n2N!IhE_DvK5IOEs?XTrAa@bM353QE}$J46qcsReL*1B z1Vj#L&JJvEN5B!^Q9dYrJ?+l(?0v8OtJ3vTZGJXu3@%apqU;W@v=^9AKInpW$)6^6 zO!Mnp>ibcl?8~pfC!~EyVD9VMSq-}h{YI+2LWB%HHa5enCSq@*0Fd-TQn3KT^Qz2$ znNyQ9t$>tLb%)y-JGK$^d#V&Odm@23H6V$-E2jdy*Mn$aE}FOH(2l3`7q7A=6gpe= z4mk1-{8x@{`=aQ-!aN+A&!$6wM6!+t#2mhxejSBlGP8`E78J5jh+cayT;U<`a9^X! zZ7&L`nb``UjOuND?M;u#(OBB1<|go@EQ1RiQ=5)XGjZMJa*>@#>@`|`=JA?Q2obbnj(NtMU1Mz+u{Fid2kmAJ93VVfwA5u~{7YRUj?@@@)maz@3Y!h0Fvjf2=r_hRdb9eKuSP(`2NU;^eOfgXo|2);%EaVr!q7QfBd z`#K*=$Nm-#FFe2fG@cfT3iMOr#S33sUcq>_O>$efuO$o5L|LgyZS< zrAi-MyGMc6eCmkKZqVff_?SHW95WjczbsweIV~;sUzXG36#stFYO_-zM-^K-<6&qU#;XUXoPBAHT`>2 zWWyilLONhrOYjDJy>nB>pM*aEpGmx-eiNH|vU7EZ45#R1rbCEKNO(E`UO{(t zc7q=x1mvjR^BUL)fpVSZ)1Ms;^A10%2nIT$rZ-e>qcR{?BEBv2}?YleD*ZK+67PHqzltehk$G>Uuv)MH`_a4g!+cPTXoU?B{ z^_~(EmBcszX;D&=qk#fBg?3|+xtJKPqOPDc8(1_cMVcM=uXe6a=?S%O*6haN{TA{Y zUqjER2h4P0&B78#ab719@xJX?oznIq~MGjeEN zu4C}HPBev7IfuJO^cZ%;BqipI3rzAU?_e*bC!5^lx-XNnX3d(MR6cQWx0$0W!?ds&U8gRUcGcj&1?y-v?%Ev;qkt+*_H`d z&04me{!z=R&n?{z((X3>O}Rw}<+I#4hrT--8}E73GHL#D($Fwbt6=2JvFS0!=9d12d-hn|L>NbbAMaQcHm-7c9SlHS6@-bo(He#rU?o`Pa z83j6IO`C=YEa`NA_Iy{zI1Ee>|8%Vv!U(0HC*5 ziJXCJ$nHrCxUS(r>yF*NWK&rS43G_~lH3|-J(PH{hGaIAqPB=_xEkL22oxagL5ouL zWDB^|H7Hox<2X4+DttBI$atk2MCc6`UgP;RiT%EHR}6^4w6sgl9w$hP$G`x{rB1*D zKXQKQ+QWC9yR6o0Upa&qTAPdM{^5CgZ(w;ol1DxoAO*srW|e)`{*tk8noclhdpSu( zz*D%(E@(13b*KE$1XADg{m1S{GLs9UATRWmL1tU{Cf3v2xJt0-zFu6j4xy!0fDKoEiomB@J{gj$O$uZxdju zS}Y~oI0aoIT`~YR<)`9A@pCFPl?E_G4H6Gu<3JI=um}ZG<0|A1$*-w}2{%ZyiyywC z7J3$>&@Eax?G+F87k=VwAUXjk$2fo?I5m8N`yJU+n!G|4ga#=YCp{;A92Aa51t_On zC^mCE);#6`;$P6Rbs3*jXzlqx{y!Vl13e${AWAhu{VOPeVt3O@D@||keRu&Ix9S8ZQk^)QhLuYt$6p9c zu6ygb^k?i80V|2HdwNS}bwEZNY^bv#>R5*mS`3evef^R#k#os~^uyj;i5cK~rqtet zV-gu5;@PCH6<#O?5g#Qdkk6w@HDN8djuWGBSbE=YOXwjA4n)b*y>(QCx2pKU;2l@^ zI%M;uK;Nd-Zk^XQ zJ=LIeo4dgib6kFvzCuq^lp>u@-FtY`@GDPw-SvVW+r#v)RjiQPlAjuMeJlr+B}}&d zz>=e_ae4K?hF;^jdci%^3HZ@ii*%qP#KEG3H>N%F=J+|T@d{Y_;SMBkI4Elt|8lj} zNu1j4EX_g)?!6wf&3XhnS*KifKJUGuoWw1weoNfh8>)6sk}CCqH;9*Fw*2B}{>cFp zn$OW>g8`shSEu);U8PZkx$dgbnhwjr8p?5g+WsNTW75x#R+SJ4R&1HE5BMf zS7!Fqk%AFByr!iNP46e(v;J4IBav|O1P|?i z<0FUKSkh#V?|OFajNCJ#Hgg4%@jJeCq0sov;nw=eb`vIC0EKBV4d(*XScbd#%!Ir*#GY`FVXrl~!(USLbQ-cxcUboHawHM8CM=CqM2c zF>!6nww2*D*xvYORN< z_49Vn99+UYuC`X!cs@P6vA^+YpJ7C0--(DyvAP=T+^FKwJ}Wqm1^w32RV~-GT+Vp7 ziwjp3yH5o6=;@3q7ST#ij-uLU&alkYyAxd=#yRY&1)fIb)>z}>CyUBIA)ICXnKC?- z@ipr?2eJeU9Id6Z?TSemh}Q?3f^t?pWt~$iSoFPqv+#v<^O`henKj|S_Die*!u_O$IUFn8&q{4;*H)Tx!J~`#{1|#r)o|iox~qK zEC!=yw{$B5HsA2vC;7r@oHHF{$tGgqR>XsMwfq3=mmwR3KTwjV>d zrgsp^U$J@i12B~lHO;%pzd>kkCJJhst5W!F{bO@H^G@dwJtY(9LBFhwJpA-Vq0^^m z1X^0I?nxWZb8e3p#xXIY3A6N~Hc!^m`f6-Zt%pncpVBq>%sNNyFDx00BY3OQIsIXo z%@<8HS@J_BaD;TAUMVeE!l+XaBb#b$b4B5n?g?}E-VH^}_oU)vTaIp@wBL|@rV0{@ zT^}mLdPj05xh;1Gv(AaEO%2<1qB}9C#y}`SH>HGi5^U*u&WyJfbm?!-E@645XA!BJ za_duEQo3wFgwsEpo*bebxwdVG3Aa!SBHB2v5w_pCw4+PuB$0#_XmWog!6pa-H#Ik5@#c;o8f)`Bx>9qEvK3sFG zjM?RlV6QQe<6vjxh&_Azn_WjL$Pj%E@c;tkk31VJ4!+rtet|{D zlq&d+?^Mc1ZS9Yp=DT7zT=Tsio|!k2|&cYCu|zLx&i zutn82pJ~{vUoF{{#s1g0VbLPsrGve!2NLavJWYJ9@eadR>-kt+pv( zkIj-l(x%vef-|Erb^SZ551-i0e}HhMjr!6(b>`|WXPsi8as|`TnQsl^U za?7Zf-oczvJqoF2q$SHC8Wyo}$i!t*WN#@tg(T#r0KPYOZ}Mn3Uas>`q=SvwdUv{n zmnr3)6SiFz*OGm@4?+g)=3vg~jm9QNN6lrQbHD4YX}n<48J z2!};@51ld^X*zhgD>=q%Uz{m5L+|z2cSWzRf1|!)bXh^%1%`Icm zv~H=do_w*SLtInb{!^!wxx469!Us~DkA;ylxz~pdJaep;DKf0Lav^<%fN-Q)YrkNo zkZ@X%?0^NlqMB&?Y+KZvdF(4Lw2UYq-Q;LHI9!->f_ObKY)Ol_DfYI~9$s}@XAv24 z$I<>3MMgu>R1Yyk3i(R)vqUE$ELA<8cgkyYsP|&diHrTy@{^9yBP)>?MN@g_C1zWUIg3Dn4d4Yur(RnaGWrU8X zU1ycWBmk1M5&gdjQl|60RiVvcN|D(9TU%OUo^JV#e^KhuS&la`nG$;Ro9%O2x8wZk z6bUDfDY;;-S}b+jOPif`FTzDu+bUHsmEFL{YFxIm5#Zx1CV}-;E%Zz_m*}UIsC|LT zEu`{a-x@E3KN>#JhIZ&oM&#LT@nX6i@~TnO(!vY`OBir!d-ZK;%Z8a&5lKNo1Va zmdpcT%ph1fUxlW?o$afi41U-wxqSMrtG1zr|Y&|l@ z-l@n;@%fb(>WhXmFIuqOSJ?x@-mMw*6R)6$286Pxcfj}fuq^F{fD^kl(wHi27wA&Y zoB^R28n|S}En{Z;rF7Z(#E%(M_8n&LIl{}H<wu|E9e|KcyK%P7eUq~u&xQ2B2vQ=}eE*LbfY zN<^#^MC#w|KZ^GuK^g8TA&mG8`fsWx9&>3NiV44UuWj*bGSSb19<}cBt}hZBK>;C( zsZ8(cWdLm%(rJfB2v@YcWu{EeF%4`w!WWn2b0OakNb!XW4}cVNWBQz9h)luTXRxe< zK%t3Uda~ApAa!sJYeH5@SsK>LfAQci8lyW5&_~HaEn3T=noLLov(Q&Qb_AOIMg4`e zbV8QtuOytmOlqGrbi67bOVAv;#V`-{vlPt%3R#i2`TRxFl|eb44zI}Ot+fJuk$?_j zVbfk8W`w5a{nBBDlGGeBs&??EGh>(TCQW_3BR|4_X2=G+c)WbS&^m;b1sRnzep9a|j7#YHU_nCDHDr;(lE!w^9pR-};E^1kt6hFG1kbw23v{4``tzr_L z__PJ@HAU^eR<1B3|7ey^07l+$(S|jcNRqH%zZHAPqRt?RF!UtQw)%f)K9kpt_;<}` zqNGbSpVfr%P}8Z@2_GV!tih+MGzHz@0q-Lr4vul-R($>LZ1=^UcWJ6Q-p;tT0&xdS{x5lSXU0r6}i`aFdLycv4ACINm zsc{B9@DLwaI4x5mm*7h0Uy9GDpIK!m^$z)rvu1CBh)&!*gnMBVNvtV`c>~FO{ljDC z#<-rNr`llL%Sd;x57$D&)0ptEbIJ{i=^vt53;!PqJ_q0iZ2u!CWECdd3+1V>U(zKE z-3ji44`*aa>ny;d1jEy?=_Jwcb8X$hKD^f^jO<69_iK>NjW4Zc@85_oAEOV2e&v%z zVWB;iIlAbVusNcD&l)7@ePQ<&{!hJU^14<3uJ=sl7}0x9LjI1(^^k*YxPLkxnjD(I zNxacXJm9OQ_e^4MIN=&Emo-PqlOExiCs;}w&NMn1B=c+^GS9GoUdDWM>*<33pnp61np6>Gy1_@kP}hi#Hv?9X=?iAq56h#BK6l8F zZ8%O!1soz_y6ueT2TF3DEZIC_IRLn8MXx1*PH6x(8;B%OoxEZNpM>T8S<-Ib54jIO z7eEqGx*?%rgf(oS75c5@r3?4|Q`MrkqW}%wF0c!6C4xUI8HrT%^OaAbGGeLP0}tBd zJ-gfltkckz6XWogZ(8vfz7vPDno6vFBJ5WmVtIbvMDkQXiRI0X? z>VU5<0SNH&$s;&uOCt4Gk|2;L#RFD^TJKi;dR8tEwD4zR0Jz$Qp$->a8OG1yZ>+H& zBld7ABSATyEO$w%mJ7;3nDkbxBIf}p(GzOEz(2~X+&iybIG$6MxB3etOQm*2%iQWE zdE3^cm7`*kM3R$WolRM(KmwXUpiZTMKrQN3h**4a@+qRfahc%2pq$(0 zK3pIiMI8Sr*l5%lMT1`16xqI!p{$EN{jJglA~=K5qlkdjf_#6tG&2q;`%Bzhi9mIE z?$>P>nHmW-MOyGCLPUXXpxOQg4N==>Xik1V0zg*|byzhfgm-8z6h~mOx(m?52H>bS z30&3n`u2@Yo$+UC5(35SrlQh)E6iOV;Hmrt7s+`9+{MP#SaLQ9u1ZY=v5`&3f?M@}SI)GDXu5|L+q6Y-9Rs2@5&IZ`#4n2J< zO{c~?V*~fF2JXG_T?u-y1vT@-ODO#w1~x6tv3JeL+bY27ivO@201S1{-CA3|i`NP8 zVRJv?m#ORs%MS+y5@7=~&eOy#*Fkor$-c9$OJ-!~GBFVD1Zx&Wjz}2C6ga+i>lZGn z5x-VEziHvW2uPE_-2eg#P5nQ-^2)SF?Mal$#wDpPm$p*}h3721 zmkS6xhDiKqNK}?=Q>|u!3Zr3;fW9Jcs=*jLrAk)6ycu$XcFyZez+fA0x^?$LKZRk41G;%VdTGJNl5g<)nokywhiTq2#~dLFAY6P;?9P=%a(3@ zJL|?>&nE+gs$m%<{R5)!5BbhC+D6|DCth$`AX70QQY*mZC+v z>amAg@1E+nw-8|r`^h?a)gaU=)P}&1L{i`fiuF0@SKXb`jSjGw(v5zx=fL8k;sF@E z2GJXxc~3wKZd#BA@sr1G#vbazy(vcD z^R(C79%j+Y78ua5Qd8B`%x5p{NV|3CIIMcpu3)#2qf(i6naiR{vE0M zJ3Kxx7$`L=F$&FS4fCHL6LR{)LyV1e%VVX+4R}Yu*E4!4obGW+K-004wh=Roh`G_; zx-O>EC2SErnUxFiC84lr)Mpoo`!8c;WmC^4VV}Op(;LQec2@I3lAzz8UDUn>^GnG; zp@)aUmOi4|1XFu3H<0&gUbs$7qdhGype?e+)Y4~e5brY*nd$$08Fbes%MyXth%PzE zMo^`ZNi=L^1K}(8p?gFOSZ#CqA=?C%&?}sF`nE9jwaUD5Q8vNdimAs0NE&6RIsumgO zdWbq1*5&Nx@69VS)RZ?2XMTm)jk~C>!1DH9L|t!aPX)&ulqUOk$nmPKk>1PQ{}hdk zak5IaK7ew919o2x8YO zd1rHvxm0Z|eY(f}JbsfB0exzO*PAS4_`K*VP2Q1rn>y5`r`r)-7Al)-?9m#j)9`KU zNxQ_-5-N*^UphoD>7KF;o`N1#^Nmm~78AgT2{rkKodcNg1)6JHr4c*$m`td@m(GUH z6p$uvLx0KHW}_yeApJ@9?d_$Llb0>RKMso@4oFo=3dk8i;npvcM<)8W50>pRak*=x zTiRS5q(?H$Hga4g?u7uMJNY4LnMoQ`Dr#=fJNWfJ=)Jk;rGn3s+H{E*84CbS5ock{Itj;{GQh~6j&^^ahM~wxgcd5qw(~oimeId2c z$D3?Sjna309ZzsTHqM6TleHEpS~RQAh3+gnLluQsPBr&CK+j`CTZ2=nfC5syYQRj- zevMGWB}tZ}Cn6BM=pF3{NQ!^;Bl21|(;WX@Y@$krVXpAhRr2~kx^a3{Z4U;pZrQy&GN!@N44dt8YA(8zSjjMX9O^KH`cS9a`m#8?H z2Qd zp(O)>Ij8rh5A*&L{x$VDs#Qjl%ouYlYFDAmTFN|)5{)Dw_h3Can3L9^G+;l<_lcZ% z>Vpk4a~9)C-uU7jD-X@7k~MYfO75N4Yq#gyBT??}CwaeK+cntwZPRa&X~xhozHm2- zdW|LxA$fn@arGX0?y4)#2c|`)#~zn%p-N@S%yT?{qefP4ZRNV&#`UxL3KKuZ33=*v z#AR9tq+gZ#8Xl3tb1ff);@6+g`$=u!>TBLx!{!)$qjrFgo_|-g^%46mGPn1aW*Wcz zgdl@sN+k8jC3flZ8hDlNs-p#hHlDgEVLRcCyWY4YHs*fS{|zZU5eHq_CobaB{DzOe zA?W>-IcL-2ugQcEwwSjC4i%RUIXqu?b!G5z#i8QyI_)5+oG_4lSJ#xqj=H|@3FW6& zU|=WzGP|{c#vT8}+7Gq4A4&Wdwdvb`)TWK0HjR{@+ZN-g>(294JrAMkcROnsLm&Us z8lLzdNlKlqwEJyZF@5}yf)mYz6ljt2!kQ5|&aOL!;y%>^)w}aU1@Yj_KeOsq^0So+ zG_X9K7$1E-`t$u3hHE-IXx)e2$i!B31-w363V(IzK3(fmi@QQ~pQPUKPUKzDa}N_- zE)*TsA#e=@>R6Iscza81Spq1@;kKK4klbl^Ap2y>?zqG9&Ykg^d!JvWzMFs#ZNzd3 zX-91@=S^0r7{rCwV{vF4RE6;0EIRBs?@#_GB;i_#3`#LJpvcCR@tb)L+Q(H)wl=>q zH`$xL2>RULdtblwj74T(txVWP-H=5M=ZC9$LKW+BwnpYByAV~!&vUt}0{%^O`fmFY zh)y5>chTv`UVlZWd%&wHh=Y4+TGzSqx9IfuDbZ;|Ba_FS&XtfZlWH7m*QL|Cbqdl_ z_wl zK&kxXIg5tgczG3lH}tr(Oy>3NBS%KVtUch)^wt|T8tcBeX1&joNH{&Hc|7K3JC4oT z{sT=8vx-Q%6@GACO2Z+O7Y70W{qpRdLnF7=FA?e9{%>a zv$g0pJMhGl3JkjI3zHorChIjL3qK6?f0M5<7qHjRQ)}8Ax-ImK{B>9t!_XaWyHmXa z&d|qoYXlX}Spv?WHIf5Yk7{3o`t%59t^0A!YQ22x1`D(Dy@Cl^{kCQ(tWo5+jo(aY zyS8eAeMK<$(A{Ti8AU|rpP0Ww%~Ip&kHC&zZRSP~O1pZwV9`I~RlLkl*y{3XG|#*$ z#bju#O7lO=m~v0|!)59L`8uN{jf^Afs`S=)t|!q+l7T!ADf=_?r40hG!RRG@eslT5 zGaP7bBKQ3K=xy@; z9(&HB?l{yRQ0@Vu#z$*8TQ4z++NRmOwn>WwhzfPJP!Uct#EJMEox}0!Gt(?l^%JTO z)%fK#tZmoI81uIChP3EuwcfZPfc)EBw$pCLRyw{qs*-x=TS)k3cT}Dnwo2`o@T39p z*Uxj*dimDVl9igrR84FH{AF8=W#i#t`gakoreGvta(ZOfC{Z$d5$qc(nt;zBHk72- zzawO!iV3sCq!)*-k;2-k096Gbv1Q$7#xZfR@ps%^hk;UxI9JXvG8U(Fmn1OBWGaaDscAsq_VVx9B~8N zz$qo_)d11fJvc~m>$ir~IL_ClRwstO0kTOX<-C1WOId5*gz z%YMJMYqjq3C8jDS8_hIr*>sH-@O2ZPWdyQ?O_O_$&h>dkFcDejMNoS>MC3T<%>L%K zn2wx>HbGaoXPm%1h{N95vF7meIPWD3NF7y=uOxJaEDQBTy)Js|>%3D$oHg2zh^j)Ec9oaw+C zHBS&7YCS;T)b20Rr+_!pxjJa2WI~n^ zkt22+?sFt0Swa$3Ty*w~I*f z_7x9kHy|fSno`9&+BHuaito87avntr{Fs5n-hmU!(aR%RWYW-`sS1Z9sDn4y_E_^h zQyPgxQZ_c=ob^ZJC1N+1nl3346a$RtHOY^(0WDPqs@&h_r6kjSW>c;biiHVojT0{~ zMn~k7B!yIyMbTr*J0cxxK%`@NW=h9~26GcKR0n{j4QN>Q6eQA|pGiPK0*ORRKLRAz zf&lpgo1>zIA3|+F;&%~z59G;F_Lb!v0E}B2paOd3)g2J{8f#(|URGrhp4E$xVEYQ2iHTtNnrl;_hS7og z@i#ptfbyp7yv*4FrtpyQd-)kkOxw`dg;OfeIq2hve^Yr@(X$7eJ-2!-AJv&Ix%3cw zV$P$ymg4}?ZePA4(D1Eh>6Wo)GPd<)?y3pBsI+(c#NeeWqHzv2QTskrAj-uoayxP2 z7PP2ikg)a1Ivfo?(ShWEnsnnHO;BTtwKjUi5@xysO33V}-nxNbbl40C<(T0e?HQK- zzeHPj7}pULv4$kJO(J8}vk zewHoDf)a2Fh^jWw?bu;*XhI4Lz34&7mP8`^Exlr&b4aOa-Df0|RQ7G9TD$|~C|hNw zA5zh9)y|YfL!ATz3Ll82*Rk3H%)HkK1kyIgcIH60sw081desW_~S>-cOPJrIT%ad>!4NwupZk zB_QqY(kn{t1+`TD7congKC9RE8rr+baX|ZztQC}3h>Dv0ElXY|0iqY{;9N`+>4|)G z$Lye2Um{K<_aJ-6lm5F!8FqEWK)Ba>OTx{*LF3 zE7tJFKbqsqO6k_D-pn@)*<{QU?<^^rBSzG2 zRPm+Izo^|va@M2j`@!LnjLRw+mjy@~&BY;RBGaIBDhC#B&9`H^MgAWJ_H^Z!09iBbWCKmEZkC->>**M}x^^{S9y;~?E`FbwjgETOFZ!0u5C z`o7(gO&^$UFrGZ*t^MURR!pyLRNHnj76dM+cY>w3vE`CjVDRh)#c36=s_!@gs}NzI z0A2t7D2VWK#DomOAiZ8B(ktFk1$f=+aRZ2zHgX*==Zy;uGE*{cG)Tpr{-!RC(hN>W zCg`M!+F3Z3V6|;ZZ~!VYH`4a*@K_wn0TvIh@xjO@qyYA$j3bX*iUci6qwCBM5$0~) z1f^;thm^J2KTahxiye%RokBWgKEqie2q3FfCWep;LcumH?ylLc{XU+Tq=V9qQH-!3xRQq5N~SOv zAefkCmDa_8{eT2m%IHFa-yLF1qK}fQHY3FeH$mE^h9qD^iofUZONB%&aSdNm^JB;8 z<-&w8+59~HRLAU`Dxrc)?nZG>zrKGGGnK0pu^6SXI5~BC#!(@#LGjD|ImAX7rP*GV z*7!fqafeU}hzg(Q^=-a|vtil=f~KWnC1LVmkZ#{PxK&Dq^&t5Gzd+9ng}Qk7M= z?ypMaW*EUbIW2LANbBejm1V}@G)RNOZBkX*7DU{pK!{W~fdp9tP}#Pi&CQ-<37kFW z?*iMyitCa$j%XoX{wK8Va|o!JsxW5|WUAAMOm$Akx}DKjMpZ#LUO%kYD}s_Q4XIH^ zOE{GfNlS+drl}f5{I^4HqE+Nu_`Kga=3hiD{eKiUu z<+jS;*AWubA0tDLx8@Uw(&D>^S<-b;N%{gRtg9G?D?y{iD42v=WK}Sh`MwUwwYFW# zJ^)^t5YYnN|JHdv>?DC)riAULd1Ih8_@w#g5XWah4v7Yxw+3*CzrVfjlLBjHyLcHX za{VhCcElNksWKvyN8Nwm8B+zK($!PKjyS(0@KjyJVim4HCB9P&OOY9Qm1{5jvxa3% zHV6i)kYM_8?)^kG{7)p)M-MNiaZQCt@cPqdh*psR8SVh?ec)}#t#=^MD%YG;3i9px zu*z3nh{NsKI!0t$jd!w{jl;^bWo>~F(2Yx_o<%Ob3tmjZocDOV%yvIG*k~-i#|v^P z1s=vv@1YWCr0?Jfx76+rFZ-bj^x!9U1G9^F-Aa+}J@@k##H{9i#%|s>_wyT|(fEwL z2%kaW<=ic2Y^aC?tOYNK5cPr=;su{tAk1%ca{>;B`OR^J=5AY#qaa_reEdL8U@16V%y zvmCUYEyqb%!g3s$eymwr93B??4p!f5x~=*CxW&gQrphgH3Q>(($vcCZrpal80uH~0^8Kc9p}Dd_Jm9A2*#rE_xv z-kAB#2^=~1+?>GS#{P4WOY0}d6zJY0Qr3DmClFm+@S-0i-v>Vtsu$gZ#GQ$`pO>;r zC0P=E>O04Zh(7hFNKz7|?f78J!CQ-Nk*XG179w+c1B9~$_LS~9AJdWCUjAR1b3d=B)*&AH!WX5VYD;^H&-^?iu#l)A z%#_sZCiu_NX-Um^(}bF>24Ghds$WTm>8M}kjY%LzCrx(tkU*vjeJ82I!~uiX z!c*?bt%EpV)-2Y;QwCATQMZH3ewi1eG-XG+!29hvT9bJqVZ`3Wn_>_fi8!OTvKbC) z?W;DGFP;q?fVOlVVP$48sHlc->NH(76gN5c#X*hYEIdCmH*7|VKUV1h`NWEh|!d=bekQhA8aaoTt6*sUA!T4%VbwWdk zSP=6?J zqPfKA+l8uDzrnt~cmE3zTAZ+q?!G<^z=1TF{L^5nQ1IKX<#bzt-ON=FE5OlvBeP$T zfKwJIAG;H?qyU-X!e5O%fw9i0j%(wdK@0Qa_DS-~S($JU(Auw?{w;u}K_*r+&p)R@ z8-(xIp*BS_jz3K&zej~HtTOx!1CcakQX?d@ZFH3~^#oq&BYMv>*;BueswA=tFyiIZ z{dkM3@gw4l6#&>jS41xRG$PP z;{QDV{%|#C^vYi$b?`2wJm+D|2m)9COYE7R&>MOGO+eZjzmKq^>uj+TPGi#3eaS#h zDXX*d7Wn+3H~~8Oi&9Tt_LmC<=wA+B&UlO8o&ARyB}CZhx-HK0u$>ZG;kE~s(@HWJ zKd#w$XGR7deXMYz2FQhF_UAQ!jhxR>%)y65St*S$npBT9Kx~1DFgO|Jp1C$P1IROd z{xTVVQCa`BAL8K^6clQ$UuK<4M6#8+sCe+LMI2|&D-$?gY-tIMOBYVwn!B@Tf-_Gk zOn#&Q%9vHTAGgjK*r9nWhw)HBa;bVejmXQAn(b+>wnR;y9Up7x%8zpE5OI1Ywm#g| zmiTbn!)fbqDUq6Zb=HD3w2eVa?NiG%i@9*?7YR$|(!c zR$0PwNrQ9KG;2P?e9R(-?5!0)F?!h)-7v zobE5M#&nJ5b~-cq#N#*L^&0ND%nY|_XAvw%{@nCs?J(TSFK?98V0cbOQJ9kx3xjbz z8hP|6ZPlM8m_~eKw)h4u}<&{h5tI$Eh7gqJ7uV-X*=s4nO zcE1WV2#~{q;HsdxbQBC`1mmTbxYsK}W!!vL3l{1xY&^@Sg3unS$&YZKv}PmifCc^gxel}qWXFw2GKth#}>@h21hm;A%;ktb-W^|lAf zTsoWa=|_(%q{RJ5l)3o*lTO0ko|@#$&FL*z_p zIuc9_;V`I1n-$t{LTiVWUfw?<>4t``qF=Sa&U$`mrFDWN*ADW>_yDxskccudAnqh0 z5SoYKE+d_BnE604p2CWCkg$@i9P{Ka!+ti|ei*`1(CstPdSZuE<~+JdMwY_k<^=O~8w;YZESOlf3ks!B9~+rhLIW5qD<95*c1Ei#$+ zRjkD6mWc;f_#L%fMHfL1+pM-f;UWIjAXnzPgg!hLwCVs>|3h4{Fi@oTf3f%G;Z(M5 z`*67_Ljy7tkuojwOp#FLAv2j%7M7_pWe8C!M3PzNd6qGA8d*pzL*^#REV5+&j`MUel`s?($L@5GKPopSD(+zzZ_?Ov)HnA z$qgW`EGlo9LXzMI;J4f_wWz`+{)7qlO3L#-oF#vF29@N2p|FY#kDg+I=|qDzRaLgb zAVvx*2%tKC_dyGs8@u5hWc#8g*>v{9A02dZLE!c8CWb$&wGACwV4kY>uPWAoz-ahn z_U7JaKmIcVp21!0{wMnsxdGZCn0lwsT17kjFOR8zuT@u=Psn*P>Jt1U-QLvvb8Hjv zNuHhE!hZi+sBF{-FMcMEJ3!J0n?m6pk9An75b5hO1Y^XV_90{6?q;a5OBMaCLcJRn z71^$bTx@@Dh7Pb98iw6?vAzYI8fY1OZ2gNBz5Vm@GIT2^0g#gb5$U))06M?epe1ch(-id^J5z4ob z^d4+XwE!A>F`r2MGd2KRCfp68`b+=xru+*S1*X@fOjm#tIr@u`1YOYj-SeEvNY}aI zOOysf@rzqKG>3&R-bAVejn|Nj1bCVG9<=hA;zUZ<-s6t-_8?CO6hN~M$hgeRs&rC- zwA48Zg=8dnRNB35o8)YZu(wzX&q1cHr!@Nl;r|Bjh$d(X@9#meaNuFI)t~g*$3g;_ z#qYKzRTSqooHzplH5F}KN$~);zhQn_o?vvrnrsv79hH+1`8Ll!OUF9zxJ}eqwjr<86 z&peFVH;AMzKzwR8&8*r9hk&Jushp&th4q6!P^Cz)-hJnZKWHu_Lblobfy40spj#(^ z7?arE0f6RRkhN+BDsMnf{jeQsdp`v-imubaYf4*)a#kG%N)Q7hz|gk^aUDs(HcDCI zX3RtlNn&WxzPYH0gJzI>Lh$m&yd-19TV48e*@QHHmHDL1^a8MTF5qTZD-v{maOKt$lFo}vfR()L{ zHS7ooE$cE|$k281J)Kud8;P!^Ua3*KjOJGZkqKf42(UW`;EM>&Kv5ej0!Y=;;(Gc& z2atNxGO$3B05uxm2gkDhM6Sjbm=Py5H3C zNC0$T>rSrEbZ6d0+?hFJfzhKtO!2jmyHYE^AWmh3*`Tr zflUgeV|O8?67q-t)RN0!r`e>22~~+UOzx#9O(a>!vIz9muh`g$t`ESl`vs_9-$2y_ z<_+YHW($6+VpkehVplr?o`9j);8qnf9UG0+(esQ7`4T zb8};{6BqX8~TU}=&E;B+JZ~MA-V<_6v_x7p5&-3+NPytUs?xgm|1dY#Z%!={F za{NOT1I5<&4NzV*3Q{A|^Im#iXj~ellwhT^E{5meUVVT0ARi6YO6GP|K>Xy#P`Up3 zlrvQv21zTB-*yqJ%l9!}xVU>kXYVND=eiM3m;u9DmIBj_sa2);O+fpvO3aI}521ch z$V|{Fru~dFEP6C6!zmY$NGxxS1NeVBkc)>4ha~u+s?v4%KQh&bM22LlQSDo5%yM!We0YB!NV=lm%CU@1$bF#{3zmp%`aA(!EkQ#pMQi zWrM@`i_CKVr+uSOsqPLqfhXCc?gpoE5uE&&SkCM@_^+0uU8Fsz&m*<92G}u}di4po z%hfNsJYtLOtOAdsg-zgl=L(~QqNMPWdZ$3|tpkdr*wU>GO%o^BHS0zRPOykQE7qZ4 z+1^jsh%&>w^xXOW%jCCI&eWA>5bAXtx7GG`o)mT-=Th#-waI(Vg3 zHw?B4?uEaMgi3hBCpBIQX@N!qU+G;6&((8H3+w?EEa{cS|44b6!e@GxhXG6&1!?v1 zpY+$DzS;>_egzO;sSOKJZ9Q!e@wV9aV+^W*S~LB{&KvVz)G5U0#6Z>W5c^to(J6sZ z_w0C#e$kl!hO==eNMA^MQp_Q#q0!IC3k3puXcAX6xnP51B%$YkBb&{s4%O#?f5MP| zwO2YGY7;}UuMBWdoA>0gDHi(4y1g;JLnM7zo;O1Apd_@s!V@Z#8-}^kZxbcjEjN?~ zp;rG5%RQAIpq3hYn-%n7h%5q~1`KgBzm=<(l?G=AE5Z+OT%7=ZM^%FBdK0V~-*&2) zTCU^S_gVS|%Fj{!GL~qMTpn1pyUO!lxX?bRjPLw68~TXDd;&-E`AmcEsVq~a80_dI8D=~wTt<~(BR%uhiH_BY+RPPCx1G`<-#qmiIpf_YJ> z6vl8$Lx>YZtIfO5SRv7BAlx?SGp<;eI0_DHGEX_`-)P2|c@APbp^v1v!1AG@o~$v) z49A=S$5Q3^;BAUS4h^|qfx7C~@R(W}f{UV?ddWlTeu@dP!gWZndK(E=e*m6Yg`xO? z@Z9K(RroT-3dED0#&0jx6edO8HWlQ%mZzr~?Fak=$#YsFt3ushM5rem z!K8o@G6?G=-jbJqU8F!E8G6x`uZ6vgm#d#ag~N@9YhrkZL)f7!?Lcp~0PgAt!Bq*imnAiU%pZ&)6{ydp z22^Iv>gjYp%Yv7C!-j*I21V6lJ(CMS_^k#++j}EsO{#B|33FAq0K@10Td{h?@*>`V zU|xA&Z*`MfXT73r>2A7{I5wC*?H<*!{*LsoCqS}$dE?7T&#PR`8vknk)n zvOlvLbx6b`hH-)xv&$ec)jOZhnSbauSUSf`zFG|{&yF?4LkHKvYPnSv-)lINywKPL z2dmOwEg;pk`wCD6LGTo6%9-7)+SGp~gx_?F*~KjB6knVvhhpJQbwbtc*!D4pdl-E) zY4VG(qxEGA%)6_p@nA#pf1RBWw^a|k+P3c!PbM!ou{{*0$#0$@j*F^MNt!>UoAW&Y zf{E(HlhqJ~AYS*>^hFZ-ohB#Zo`63|z;Sr6!MkF(j{l8!^TK(m8=^S+S?5m7%gEuA zO;Fsi1p{19o)lj*?(+_}eB-WV-RBdKYXDJ{M%kB@v%0x*>i23hq)`jhRvmZlI;dqek1c+D(Kr!P zP&!fSGygIzMdL=@r?~98uZn{(TrQQ`kJYzjwhzg#0&&duimIGKu6QVmj?U0sy|BsLS6ChU71#E;d3#03m+2E^2*szn#oD->>4c6;LiL>8 zIuo0~8)ZH8RWreY_PptdZ(ll~kYoKfds+5r^i&qovhHn;iXDlWYY@%ZI zP;nfA?VKT^*n+iJnKGk#npM`#=RX-t0W(bYmzZ_!Z!zmP5iG^O#jKgah%SVdT?6KB z;f&m}*E2d&!?};=@{5YRTQX0MEb!sQi_a2rA2d#Cmytex=CR{O=QM+YE_pUf7^Ux_ zjG17}3wvVFg{J@zW&vQ!Z>bO)savhTogcGS@lJU&%Ekqkk>YqvB)K5ZuOu`EvYaFu zc4e;zx2OTXigMr1j>GTLzczTy^6nrNi)k9?y(krYZ`A1?Tnw+NAP-N@YOIOv5x!R( zTn5G(5=r8?OEdeK#9JSO_zxl@3y8&dx&F;tK7#ep{=5!5lk*|eajSdhacKD}QntRN zl(eI9FTdmQDMF}-fJ~NL* z=FB_tC>=j=pb3RiuvLWFmBk#%+CQHx?^^LLpa}AdZDOxdNY>_w-P!j~%;irA(P|DA zSuuCj?|c|o{!LI`0D|&ddNyZ->4~ZHO9lb7-qH!WzLB*V|53HZAysQyU0!^(G_;4# zRc%9u&wXehm>DIExKjry#s(}1T{GZH5Pff+Uy31&UHVjbLIk7mEtL3(U><{;6bgMf zS^wgI%+D*sr$T~>9B$bV!b@&wZ``l_q?pj8@iOGm?L0G|!VT$n)=*PdUV0~>(C4RK zMmn-H5~<6;Z)^%E(h_bg^3RzfXx_r0-mwgUSAa6v}K<+F(AfEb`JE^ z*M*KfN=pQ-a6gqxZ?Kp9R$OYk;OGWoiIcu5CC?z|?yT&z;LzU;2mGPxpmb1 zdzk2TZ8v`iMFsRJ)LvCrpQ>N|>4XR+zl2xVBKhGuZC*VZ%tG)=cA(u55MC9czq9Aj z1zrMT`4)G2svcP9O&)o33ShIt60i3G0}6#P3?@Ut@?;PZ@{%W&KnkIXP*&cfA*}l{ ziavf+ik5h6eL(<_q7xeR2y}xt{`z#_Y52j@Y&U>)p$GV5Y@&`a61%>07h3PH-l_bJ zi+qIqHn=OE^>3+zd%xkrN6)OMDqlmu)&8~!u>_!@^!x|@7s*lZ@N#hl+${DVL=&FR zCG`J68?xc6V%~-(Qr|-~Mw@dlp24+7!quGJqq+PapcV`BmIowYwa@`4`WasY@2Z1$ zP1`E&|2GV?EAZb98;U5MDo+PC@=f_x`0Vk2a)37cr}h>p3ilXI_IQfxFJ{juyvsu@ z10bj0_Xr=Hfc)2^HIEsysh5Ye#c%CBSl#1tm&G%0VWxT;5nDMEni_x+X`2}?$@~5- zFljF$j#0%k5~}w;(79#Z;)AOV!DwjwQf>dY%0u8im^w*z+D9kdOX=obcX zE4da7QxNR-k(HB32q{PWvL6fM&Hdi$tR-+_RrT|%T2p6e`W=G^e-ibbCCglwJjA70 zCfXsFgVH4RQNZcO5jHo8y5i){Pe^3&$5`9-z=T4!sV6Qz-T$|*JvpN6s3uNS(X0H2 zRc)(`JeF*s>OA(PN?d;&^3}`B@O6o(+21#|mx>$zORN>olWX9ds1Q<)g0CVVWK2SI zU!}ndAlBmJQ%GH0Sx;y18f`YvFudL_i~B1?XNydCfR+RXFdn1(`&N%rv%^IlCJRdk z+%*st0z$=dp8(D^G~tEfL=y<(eyN9{CZHte>`+K4YPuEx^;?F-bz%ERILr zS@t{*VEY_?VLFwlOQ1a)1Lzt-)oV&$gKBwoU2ih=)A9bhM_upv<(lhGPUh}wghJ*2A=sgV4l_i4{!KGc3g>2%?+bwv8^-Ug@)Vs z1^}NV;*ug+S*O!X>oTreueJ{TF+7%ML&nc+xE==50@R4gx#$j-0tTo8@|arHG#SE! z>T3epk3pLJ;Yg%>Pn6Q!%nIzTiO?tBclf5_OGuC$A#Jr)iY)g@gs4I(I3=II?pwwr zQiLZ)^>A&3@t5E0w!9c6HS|DJsr?g5j*# zk_S&O-ZZgK51w6x0U1Y(c4g)TNAL7UQK7-HF^EDT>3Dw7op&NsRH_;D;oD$opT%2M z+KtAGTyEH2nT3fX0`s_07Sm#N9mrxe&$S0{NMA@Bk;QL17wq49dq8GZI8&rR_3RjA zFI;ku-tDmd(xqXGbR`+wu@VQUnA(kG`HZx{p)bwxpD+G|wHHd>;pb?})f&Ka*4VuO zS_3S(k}RUy@QEE@n)jiv-S#ZDJcA!ekN65DJpqfljRi5#wC%9j>6+4%loLtvwUPzc#UQckEG*G?_g7oLP z_{?UPXI)i?Rg8wb7bL%&;7b%WQMf#833AM{b`aw;_1L_vS2^Za=`5Bs7t+!fVzsjsMbt~vcB>!o_ezJlYM9#NBzH^y`U9HQeVncM0U&hyJ z!A8mCt;~e~8o+pBp@l!gtc~V}&b&m8mAo}3V@M7HsapNg@C^U&H9P~flhPc;+>_V6 z;8{!>oSv&BbO5#4v2vtGK!d+pC84~seCK*)CtJ^+#S;5dIk8*$Pw!(uEYlZ{AspV! zerLOFT3!Q!%|8qKe2TF#Lp8k3HrWxfM|9iCz=_eN-1RL#$fU3J;n(LyON>mg^ur~e z(>lMzB{~ zArNCYnOgovoSR-iXcX{G=c6wNn|G347fv*W-fG7X8Zg1^EMwiO6P@^>h9{hNdX7}oE~y6N1)Y2cT7@}v?QzzeN@ zcFe+5+N22D(r8VFR$J%%zKjh-Juj{v!jHuX>wOPHLd2cJ_3e$Im5igX%U8-h=D>5& zzuVs);)TM-VsD1RxE)KL@nlteoVAbf(k#;U)!?t{isu6{iR@7uOXf}lGZU+CXKvT} zwVICt6SH#`IGMJ<`&MojfYuw$)mDE?FbhB&(V986cwAdH(RK#7o#$n_@f_XLQMW;^ zc=>#z`!Zke>eFkVosN27f{2Rx*K)%h+VY7R?LtGjvj9;qDVj*W=oB>TEH(aCPh@(u z4|1Fe7=Bv$md8YR5KX zeAPGZ;x7Bk>2u?Us+}E|R>k!hVu|}iH-xVW~%P)#nfdU@jxd`(R_&Sm?zjG<4fi&ryz9(H2XrIzK{upi+koo@mXm221i7jo zhP1r(aq$)@7gVG3OGx{jYesW4GAAM(xSPcdz-P+}w5VTDaO{a0?E{&fP3!mAy!BC1 z0ld5Xtpk!e-N(u24f>h>`S2$?XM=eBbr*1S5Cub?I)Mae2MEtKc|W}pWfL3Au8zSS z#Dv@dBdB3cKZm?X%p;pe)Mn7b@4P3Pn2mQaU06vPy*P?-)ADuCf0`Y)LXR~aGj`Br zct|93Q!=qM-Ojy#YObDh?F(@)fmTI;W)$KWiOu$+$$}IU}0cSd%wOuGNHFiToRhQx$zv}J!tSxfmD+nS`q zUD9BO{^`V4O}OiP(I6@+*Vi_(I$kLQtgFjqzU0tp^XOb;5MHB(}RsE`i9i zyl~wp|GubzT%RbqqES@mI(30wJEUZ`EiDBj15R&q4X(TlYvpjDG0#aE=YcJ?Zd>Es ze^J&^GBy<0rBJ6&5%1{=q77y{QOEtXboDANhjg~d9c ze@h$cnfR}12)(jwS6dx%)82sF?wo;Ru;7}HZMI^jx*HthSy z$)wFQqECD`!DwhLoWM_W-TN)3w=3`mJmzff{tofK`s5sdT2n}`ZE~b`ur4h)u*# zdcHm??UM{9HZ=PDnVLab%v+A7m8j77V|I{mlE-eZ?gKgeRRS(FN`lEQU7yv%=MzX= zW6k_PgUUHz{lrw3CPsj!_l&bg3;;muk^2ZID{mMqoi_m;8l*;0@g(Zz5JVAk`Eg*~ zA!bTB2wnp+DzEg-4LQT~f|vRIi-+qCt<1#1U@P?CnE+d#?r{gtWt&JA*txVH2@h?-W4W z#WodI)8AFjs(sQn!2;q`)sypn=6on_E&q0+O-j8nPB-VDJrFA#%=}*8Q;yYegSL#p zjZX!Sq$Q-<#MIg$80`Kc#Mf7k|At`i&bI{CE8u5KBx2m02UgY~4`32d6Htl9S=E6> zZMx5vd6$+~TK#i5t?+u7EpL zAA)m9pMkRy5VhcTQvDvf77l$*4dpc-(z~w_+C6E==^IB(mwSd#30*flD&@ zNL<64`^Ai21B$DH$xiGx*C0IdgYr0HwC zT6A_4RW9pq3N4hY5r~p7K-wAjp_Ozc3;50Rr0Voom-G8+Y(Z}L;V)Bcr^&7>|AoP* zgY;mIyz_!O(mk?3=$(_wcfEZiqoeyX(4zjZmT7?|8+%V!lmfnfA5vSMm*Z_eEQ+|0 zz%>h8Cl_1Y$$9dl{6u;>8k$d}7krL<-z9RIJQ}HG{}wDzMma~Or}0=P{iirY8}}}jr|`BBIjvD z_i_I3G?YQtTb=%YCkOo{tbF}{DF^)+!)F{>+rp0jms~f3fS9-)6#k}Uo`nZQMI)#M zA+;f=FlY)^^y+COZ0hsR$Y2yr{6ch*XN3$*mZ5;%9da}VjcD7Jf>V-irY`ZOk=JO_e%Ad&`r7(GjR_&tP zdsF@kn$O2h>;Vq!ZCKO{v=ZZaz0Q{q^MvqB#0f6e0WQ|?oc0@Pa4aTqMr;_&|wf$>c%-|Hj+ zeUExe`EH{q3%?N%STR64abmoDI-@@uh*Qu{=DXGK8jeZQI>O%55 zMm}!y_jZxAF|c7YJa@W%NQ-tU963fxN}Xr0JE&ZJ&O3#9H)%ON8>ZbT)Q6ny=Cc4& za{NyYHk~~rq*&~`kkce2AD-wkz7f+{7pq2{i^=KFGn~e)Qj*aq#~08 z+9F6{7dg#|fi#8$L~CXZ$Ji8#kxTC;kZbtWVLz4isN=lMT=Wx{k+Gjp ztLlKVll_?2=lvCzUQrUj$Fl;y8_6%fDb9c5bWp*Bzdl$Q2JAlE#ifx`;`;-g;8l4{ z1}h19bP``ok02=nFE22P`iM4|Wjb^8@42;56K_hbyAfx|vpFUrA$oV&%+WJA*;?_QhN4haPjRw3} z_5ToOpB1AM748a#BvjGTtxMi1cK3O5;19f^Ysk?-nUm#c?UHKciV-a zxxq01T$hwup6w9~7T1vo_DD9E8KfQzTJT&|VJuHf-2ZY3lI&tpbxp$VH=yT@%#KS@ ze{>QksrDnuZK5yPv7Oa`9Xl}PTYk5}_hk*(EBAwv)rAK4U?DYX0?LOLeF>6>1QN-M z#~^|1uj36FRsvGKR&Tt~tV^C-#*2TZJ|_#k;m(i5wXTC~<}W1`1c{$AHyii<%2EOH z+jUpg61iYT?|EK>mXS2f%>Lakhsik{5G%7OFfI2n|{VN@T*+f&hX_!A{jHdvs z=p3naE3{YyxdRd3*(Y}c%y+9Uj5Tg8(BDVu3`pqLvzGvPi`~G0-i3_MrgnT6N}9KQ zkU8J*xqtTTE3-lc2Bbxv1V~r^QU9MGk*bgKjC(k6q9Z&BbvRVuI0D@0Qvlv>wfjK< zck%)3JzJKmM?`Jz)xwZiZLi(s0?78$Dsq^Po#Tp?PO(VG=-%!9bQ4y$5ar!U9yn_Mcr z241X#SHHo-m#UvF52J{{P*LGW!UI<8#}mK+eE5olha0F|%?niASaZ!>)SOA4I^%E! z)EA2H_LFJ|F&+QeytC0f3C~Y5@kG8(1t_%od?Uwi><$u54Pr!V!U!IWiHI&m zpx3VrBfy>o)`T{!39srk^EIgPi2!}SDxg3)j_e#}Xa7Pl!}=@0an54JrF|84+B*Vd>XgFXWzypnXzRyA9V*A?2eZpc)ZyA*{nOT$pj}=atP>CGeT+gy}pfmwUjGV^mj#1^eRJ zkM(tb(1jKO0JLa-3mq{y1}ZweKnCZWNz>d)zpBo8GD9=7M*Nb%$>Q!&ZR=5xx4bw! z9g)BACG#eKkrTk#lTbDk1LdeES5g7u^2c2N7|fW6K%P6;7B@r`0t)uTm8C^^UP%Gl z%c<4s0B{r(DWTQ{IdKEweBr47*&^ltd5O}a(l(mUlu;v4O>hF3>cW%(26Ul3;*Rgq z0=+>8phIr^`$1px%n~|P+>b~$`KC}-%qL#>7@JxlhY2=iTc1s*pSxN0AlN;>d!}3`zTkvq7TpO$=INb6{qbu;y(Mndyfz0?2n?s`xt_$6=mOu*tetMnn-(n z?prAWna8T3*Hw~^Fl1FCP~NRtfpErtUABClDqd3+O=p7(dHb1=6Qu9ypA9W(9ZgeP zIV?I7KEj6uc^CB->5Ys^EuGImTaE8HGO6XIG+`7pKRgxPRp!gRc38nA%J)Gv`%QT* za_Z>5+tRCwY?olCyQffll!Tl^8Fmj#i|h}IyF)|I3X(C`MqhlNiHo9?Hy6&tg)VU3h>XfTJ7pk;#Uxma=^Ft`33@rWEumKMFIS;0rc$q}9(^Ql) zi|O?8F}Hi@3@x-B{kA5A0z`)T^pv_5ln1c&$_a1X*lT~7>Yu=VBoUPj3(5h!Rr^X3 zVrZIs^cfPXhSI2Yx23haN*@OD7L?*jcufy=1bZfaD&R@xWuoyyip~R+l-T177j<%0 zrhha*k-esvadYI`&9l08wtSnbQY&+xUWIL)qZ|^S@C1s*;)CxQb#OX(9ExDOxr@Z{ z5F5rU>A=GANNO0r+wOFoiqOP~P6}>;Z$Y3LC!Y8zFGE+2$vRqN4Ju%>dli||zj%7N z54`)vKbM!EVZnkrjW(nYM_ageD@l@;U-4=t%l$SACm#~ks=R`G;v4?SRYsu+U~JFZ zMm{<@x<)M?6$k{(T&Xdrtd8xU((7)OAk#4Iuw3yl4@~S|bY=e%v{EIQKq3O+o&L?Q zq4UW7;r$hQQJ&Tw4>t<@NY(Qugz-n{xGmt^6Z_@#YhmQ(z6r0U1f2=rr{C??X71dL zk(|3Y9l@s~^OBh+*JFzAWKP^goml^X+I%^B#R7YG%bo2lf{=C3Em1?Aanw!IH|_et z7{R0MC4$p}u zQ90CoLPRvf@I2S_BpKt(CfGBWSBv)>RhrofWAufyFSgMaFK0i*@tV4i&1K{*NI2lb zB?*=40-?wD;X#MUsdC=54Dl?gQ)F^Js2+b18g#n&IuY1L1~S?UdvBG4D*Na`ueYGT zlDW&pf-^2nQ=isC7lWz)Aa3A%Dea=ZlD@XYC;p@*kSdA?`sbj$ev9pewYe(+jw%ce zgn#@F7z8OKoRzP!4wA7-Vn6-k-8tg$kCkzAmSSV%Lps>i3Qb904-898xeKX3g}Z)A zr9vaWD^L?Q`vjYl5%NKB`?S3#;`F;oPe6ba7G3cn3}^aSdFyG0jvD)e=b>k-K3v)! z($dpEUL_bC8bmGS+MZlWqteoC0ZfjHubz>(?oXp&DM%WB*DQ zQCrn0uBX+NQSJDGL`r}_W@f!QAVPL)`k!zT=)h>`c8PV-D*xfuJBCc%Sr6>NP6cN3 zVonn7$;BX{E~NZN^vD^9IyFDD`m~?PbYk>Xm;qNABTm5bNPxQ)ZM%9jkQ8Eg&b$hH zUJ`OeUWl`J>{=KJ_Uy%s*l$I$eIbOmkcxkn{?N(c-Q}5Ha z&+b;|9SYH1)h|1wq%(Q85!PxrdF641GSm~8nwa;{(BT%ti-A_j)zg#@Dvgpz@;EhO z-%q91DGOt$&NvF|=*fn^B;<+HlbIrtfS-k`P7YP8g=1LlC(kegwx2+>dH=%nU}rJa z*XiOJRNZhGpeOv$Dn3Gl;_!N|{ix)M2^!=~)RJYTA{6A9(sl*n9nXS%s*hjBpciN7 zF_ei0?Y|_bUuBVMCK37{p7WL_yWjK~TRCeFRJJ41nIQQ>2~;!26X18p?A_x+mBn#^ zz5@28H0mzI02 zfV2R1{asaK3CsyAHO*U{0oyQcWvbR`>M)V!d~8iYyj@0xI6pev)GwGpnTlq1j#tXH z*GGs?{i$4TQ%h!OdQ`e3dVX<#CGdP4!i#$Az=&LJ3&VF`m792e66jQlwQ`zty2YlB z(0|Lf_pt+&Ah7IB*Jhd zKy=fD_qkxY&+_BF98(8t-|!(<&ciqDITC}o$hdhaUsQf*RQBXK-Ce05<)ATu6yu8- zm2_fpA_3u%gjUWuhy38VTaK4&J9pFB)F!jid-ACE1QD5davx$THw}Vh0v#wbJ-_tc z6ORFI*6;@S&(=%YyYES?%e$$mB;&UySJZ)b0$4g1E9-3}VHFmT-BKaDIGqGRr_hGQf+ z6UC8Su-r67kE*aKzz|YQ^CyOa<|`$2v=aWv;43-|?#F}YX;m?0O0yC@n#?4mxeM7= zpP?mH3@o#k4yw>Z^Ni@JGCuan@7CjW*%I8jGe#AjsE5hwHlqxtJK#{GP9I*PXP>oP zxsiL6ZZW?kNTjBvUwD5K}Mi z-GMl!+ZFZzW zU6L&Av6P#hGcd49IUz0@5oH4RTd{uu9l6$Uo7TKX&#|Ew6f?q!VQ{}NWt5bUticuI zT0F?CS{4;eGbK2t?4~+hEb5X#O{gO{&)vSxBxyYP zB^coz!AQ#T$^(gfl%MEM)aJ)ar#R|VLoPdXQ~N9LVFUTRj59stwa|M|>M5n`yV%^k zGTql+!ov8tI=|c8R6~>CRHbZIW^a(Vt|$}dAf5)X=%cC4wdFa%I7qas7aYH&C>Mf4 zQK$cczkV6U*e_z_K(>#zzQ@Mu!WBBnI-bJ$qfJH#4w-Jbj?%z7lm>FX+#u(Whf&*| ztau30Knx6)l6Clx49c68I!!@bS~N}|;+HI#Cycegt}RUvPiuYcoll_j9I-F*NKRVV zm{J(*$<~<57N^%sHsboSYnS+CHAMvj>o|sp>mRk(WySS0m=D-_2rM`I1 zN@}U2N*0bMYJ&LraHifnB&NCN{bu@4xqT&V#(zWVM1E9X)VC{95hE{#s zYB-9kAmlA3yIVu88VY~Z^9l7laZ|n*hDSHdRW_*n(~Uh4pnOj3G^D_n>t-AvA!l1G z+@&yB%Fu9@?FqasWKNn1P$x22V)W##Q+@E01rHTnVW??eHwd2nH);i0o#EB5%iks- z3Nkw5*Y*h3stbI+)?3$il!VWyNEk7N>4C!qRz-%!#Ty(BMf&p0WT@M;+A{TwAL=5n zf9sY<9S+wi9$`+l8=BZ4KXhGBfL($IN-)+TfK8$O%c`v56*t04@MvB)&!9oY!eGkvh!06<(Gd}(^n;|KBx zS-bR(0*um09TeePuM!a;5ORn+5y>lp=~~7j6@ynWGFyR0`8oJH{m8^aW zXvUUWVV{`1M^8R@#OnQp#;sRZWl+ql#>IIapp}X@tiE5|v%#9VrMK8@K-^{IJ+jI= zm2B1)LxJb5UG?^a@r2pHe@x=70-G(&E0G!M#9O;aI3Xk9dpr~@z%+FzH*FSSE-Vua zH=YANhHST%;gCs`+nUmzgpNzmOqB4)`6&5=|IkVls1;`j*ZSf3C0B zie34B8T&W!P?1!a;13c-NU>|R*BD614NBnQO%P?ZEtTOCu)PVVDPd`Hz|uTqY(MdI z4oS!TYD#8ktYOW>)Jp9FGCJkv+l~p|9HMWViyk%2?jDA z+nZ~i>mx`5q8hC&P?ddeh2Woj@VK_$l38dV@SBvsbgG-m0Xngyr0^g}2o&~u=nD$t zHIGI?i@NvuL8b?1fFmgmfn2IL5*%K?02JJ-_V1VB$YhfJ49W8Sp)xojOdo9#hr66W zsiMWq>@(=VRjJnlGK;hH`bg$Bgv(7B-<*_^G%$U7HNAV?aykqVHBIZO@>3}r@6`80 zTH5(ziqHuI=-4}kO!xpk8T^X8pO!6~T_*4`)k4sxdwk}4j~aIZ64pRLdx$3 zg@>7$PEJ$%CQkTy6li z4nz8hd^INFf-A-yS$GBXN*$kyx_{Gg1Hw2Qx&Y&vU5CT*7f=p}=Opy}dl+g}Lwv=x z0%5BA!r16omxa7_Ka!wkyl;Fl*D5h1pEv@I*hu!Eq+o5Jc^liy8th4_R++9)hG0X< z8@d|+gM9vAhFW!q`+wYdUIiK-1DL}F`neJNdq<>ADXr*cSwJOYTjH_r)o{`4PS79% zvezXdJ3rsZZLa$~frBamR?E=AQS+^{CFwJOTZHF;n4@yoM14m1q24_#3qTa#PaN^B zy^T`f1_Q~FtLHLj3O0!~%~c9Gh`?r7CzhBM zOlp5{biuC7d1$fUCk@Vt=Vbmr-=ym4ZmXZ%+o`fEkdpSNOSeisCcuz7c_(j7wIF(x zQF~*imHmNcBP8xTS4j1zA!XI{&TO3hjxv$b9k|<}%VYD8sw2i0pWls^TH=`q#t~(O z=$2-DXRc;9yyKh@c$VS=`pH|LD{MQ}y%kLG%vE2G5yC)(@R{Wc@VS`qT1OtZ+{{AqYTn=_QrGnMy5*QEX2;a>o$UkP`U^P?&)Qg5gUX{BZ3jR5 z&Q--(84EqGXLMbAGb&Q`h6wsr|7_&su z5-l|?>iMQJtJyeT!kNLHM!dxl~SL*R31gfn|+% z7#(sNx^42|%Q+{CedCsL?hJhvFss(g@0>80_xt|FAi5dfcE?Tu4%X+Shjk~n!fyr1 z_mCadjs3xzwMEVrS-OiIzZA2va$1TP{hTLLqVoHvu88G|lH-WyOV34pFJwLt7jT6x zTw4e{OA5B88Jv!RTnGqzYs`v->&bwO!yKj*PC28((ryP!d8~T?y;O!1>>f2#qkFaA zwH^yU?2C-~7_YwHaI{HMmdnp8c5qwVp5H`{k2IPPU{8hfx+o@Cjsy5G3D{h>h8Z;T zP<{^9^W;d*iF2UL;j4QMpqPz&Co&yk?f;>_AT(_IDwN*#)_%#WhT%E%b&z59CXcz# zTW7X3Q1^HRX0s$r3TCY|R=zbJD6t67u~+ukzshf&bn2*EhX^uRtUe|lrKPkb^w}v? zlJ_cm%?J|~DkBnz=O5rB=wi)|cU`*yquJr|I4%FBrM;Q6`gb^=9aLu9jNeY^9o97x zlx=f^$(|3#zB*^8A2lWe4ilPdwc;e?FHK?lU!1=d7=E?{wUpq=stPmuUGxrT>8L7M zNBZN`FPC|-Ke^wZs#mrox3eraSsJia=x0VX=O~>s&39jV{^{oeQu!9gZ&g(Io(H)B zrDdN%n_3LvdpoiHy9q-V$L@($4B95Gy;(Wil9&_XlbM7%B`zY&Ne-j`ZZh1cphu8Q zg*(M7L)RUPlb1q;=B?K&m7M3ozW zf?;Rml0k$M6f4&Cs{0UpzO4QQTOYfp*@xVF!9zA2{vPSyU<@ntu&L=5RlSnVmWr zhRi+IoC0c}P%$;~3aBkp%DhcyOB%)yI?yUuJo-hec3%^=f#=HWq&xO1lV17*_x*9# zZp2)Cmno4hE~@6}o^O9Vfb0rnT%weErj_>A59-Iwb^5`2liBp?=k)?doO>L@h_QTz z#~YwaN)YNi9OZuW*q4m&Y_GNuHU|J>|M;%KdwG^*M-bC4H%8PuX1@mT3gqI8sECnpBa)R(34 zCpp?@?jpM~CT{duYhmBm+opczG$XYMZ3?N6UD#y&4^YIpktpXF2Q}gYkW0cdq9WM{Esl74R zsjjvVlAu&$Zxi3rWxpk^-dbh&F=42(I^%utaxJ$d<*+D(;Y>JagbnuE&iGqpbz&pXX#61a3$Af&0g6~3fBH*AC z8)kMvC#TwWW38uRI-x4?Rt;B@Z-;}`ib;w%POs{-nk+l2Q#Lva6ar*#UuQ#s zL<3}lOF7Hh=4K}!u||74Mh-rElO&q3ltZ#U@Q&(A_j$0^FogR<)+USD+*XH7xZx@75yInyt#7zU>M{pfeo;C|@ zcU1#KN#4cMV#YD@ylAR}pASdphc7)cDeH0{89vQ65guBx0MHAW#n`1)ldUC&u7PYP z3M#y-2P)Bi{jKm-K0zO7pJjJuFWZ5a~6WNR^4X&aXu+#9nG-!DC;kI zj(Qjq5QF=b&P)uBW&~GHakI#3nKH9SneCZNGndrDT=~vq5X3$gHMhg}_C@TkGE>K5DElTy<_5 z+B2Nwt`B)B$XtSKq}MAj<%pEa<3`Wg&DXu{u2X6${{Rrcr88|4u>pX|50d+su(4=9 zfIvhc!ZGran-I;=yuM758eJ3F#Nz#VE4a_UueTBU$k?HyF4bYYsZn;4?2`ZIhT!k| z;Y<<*t`Ton7)G@_fyz*t%jk0ZU7d0??&YKU=9(9o0pav~MpF z#EPEOK;;>sm){o(X>7*ousl_En?Jwv2xV0)$O7}Ay7+A)SFWXWB`N{)gjc%F4fHB& z39SU_4NoHDb;HgL$M^R{E%+I0%4o24%5Z4*txCErs>IfIsX*VAK4jg=astLH_-x^f zB6FXpuK@b!Q3z*rY?QK-1!hqRXYfzSs&2q;VigFUjFV)$JhL>L4rHIvu0QW^dk_Y> z12Bb2q4)Dc=`?DVix;nJy1%KkZ;o*6jR;ld!(}BZlhbE@I|FUM=iAasMEQiXBKpx zovShO-I)bsWHmidYE`6C)IB4bQKY<9dkQMCXcrGmR(?dI$_RK3<40dWR_Fq=_F^G* z5FN|-;aPBrbMCn@Pihy(>0D24!&Y%frwC~Zw|KRhfD}GApPd-{Eqt+XiFKRc%Y{%x;mAV`6bV#&T=6NTTpz?QF&AZ6IG$HufvL&?z=S7``-CbT2W;<6zm zD5|s2&~hx?e?;_U1Zl6EmyA49ac+*sXakFs7Fys z%Z9`E4=ss;G>Rcw3q%L^*o>9<80C5|Poj|6Qq_M>yY&- zXOfol2$)t1qx^df_TliFUDe+iMkImQ)IQn!P8u;YDE|>pR*}h@(5sY;*lx*Lyg1pB zQ2Urk%|c7Ar|v953%a9Es&A4-(MD?}OZUN;EURS$aR;S+o~w;N`3TQl8Q%qnA;T$T zz>+xth{>ZtfTr$y9vFpEX#3Xp_U$zYLYl(7<)MB7a0dS+9l^VQD(*fF3;Er<*6(^r zDy2u(9D3Y+mSE;w6$A08^ZmD7HIJd%W;^@-$?M@Zl;d;fwm+4$XBvXcIf+oKYe4jz zt#JHdA9zi`;Cx84{K&0xz>|h=&H}%pB7S{2`{pd5Pxs{n1)vyR4oDqen9COs3`)BV zPn+}Qq!Brt1`ODR^0r17VT>=NXVdRbaDaKTdk{!svzNu8&3%szWg8qhK&pj7qeZHfiIUDk@kp}~Nj8;)%54#}8hc9|>&5_jac)F=(NWL$*8W=`sEV(6 z0wF39w8D;PrH=&cY!y}=zp#@XXZcILkY%eH8dnn`3PF{tl>9h_EYgMu4V@f0M#|P7 z-iA29kd8$bJ1g3e+{2xKrHHCKA6ZnFA=3YX9vS!7JLHN$tYm1!^_0eI>r5)Ukquzn z-XNhR^3ey#&ZKO?B|>3{TOCx_qfzLlfn@_&FURSUHn zJY{-Uv;Uv=t~{Q~^z9?cQdvqWQ%H+M$(k)&F$x{BB|6hW)|wnrgluJ$k|9e%C1XFP zsANesN>Yw+6v8x0b)toXP7d$)rkeTH{P})9@B7EEKl&Uz&vWkQzLxu1zSnitT=W;^ zauA(*bHW=_#2iX<&%y7n&_-vkdZ{exFt260maR}f5~2}{P0VYI!a zZ)Qy3?b5r!eE_hzb4iPZg#ql2R%GdpGDEw=rWvz#r&@*FT@YaFb!=t#*lr_^B?cnh zVQ^BIY0}!hyWF{OgVni@hCa!e9EoJgXnH%kbCH&dV#??T`nhvMxEy@c+T-jV%oUZ` z-CvdW3}cFBIgJDACR#|JvF%nJIDVcJ^(M`+G4!aj?x5Xz>eJc1gR~Oujm7CfXFXT~_=mIET@A%H zKszxw!xisL9N85o&yKXu_^>t!xf`Ca#YE|yOLCXzV?mm?PNN7^Etp;hL~7!`*B7&Q zDHho5le2@9&~5H7nLs)zydY6-&e7zt_t^vf{fOr6qQl(t7~FzDdEcJY)DziHn{7#h z)q$f1qc``+c8^*A*_JuX7L8oJ^G@=Q}B)5{Dm}Ji41r6rl zNgE&8%qyOS>fFB^$9b&hww4F%-AWy9=}#A4Is3dv07W3+dMin6v&0g(cQu7>wAxa# zf3o0B30F-dQ=m@BHtnHrui?t;a4RjER_Bh(EI%~%S9Nb#gUP$yWirQEc540QT4l=ZODf%{P?pKh?J zh@WME_~JRXhK=nKM+Z8FE;cea1UG5IG5{jw96M;}%{-@Zv@V`m_}27x_SL(wZlzKp zqx%tzKDv#1F1P5^wm~(kC0tI5X|%^WFGsG0JOiDGB_W4`TKW!;*=#kPf|sGHSyx@y zsoaShMz`ebo(&27@HaYJzF`o8<>;rsr)6*YuGM{MhC@%T1goso+E3JZCa zE`(YjcBThj-$?4`^tW-=HVN7&%Zu6Le!D{R`S#;uNuiJW7OmiSJ!S1Hha-}JsV#Jm zxXHyG!>5%_iPlg4^SC}Hm`-Mp>vq&*C1#TNBG({aip9m)dT{UvWb7N5+f2V*zf zEnp1)AdgE!h~nrj}`g zppJAh0c`EJ&C0*)fP`CSHoxPAg6y8MRnXrg&y}lhRxU0G!4*(B;2{uhwB<_qw(D_< zN(x?R68#ZPLh@gE`mBz9!^q)LE}_RSOyEMrkk-?`%bN~XnRdEAP>WS+7LpHgMj;~jN>Fy7prG(U?#uS76X_L?hnk?oU*1T&Sca@D1t4S;}N?& z{8HwI3Tmq#<%QquPDPcbS_R425sG6rRhYJ%oY9uZW3?pc4I^`xg>2@M(q4BhK9UXO znYyabu-Pc3JxcYg)|pI;UUy5iK9yXTRg#O{{ZeC0Pk0EQJmIz5Q0kEDbt({PQFj5= z4Jcna5kH})ywb9)d^D(TNq@sEJk$8&)xqmIp2`xNXPuDUEs{3MX;g)?aW z7GA?;)fRF}#E*v)hd@nDnGAN&1v@xlq`!mlFfunw_ZtvxF*pyxMI{ETF)-M7E)xCQgqQ;Csj3fd1l*sHAnS$A7aYZUEF`Id1gwgY2F8PL4ERc zO#~awN*B`5#|VsIy<CE?woCc` zIS0PF%BP8|+=1i6bn|KwxJSs4RS|xvCEo$gaob}T<}-prcx8B4`@2wC z>ACKH?dd`N9!RfEYXaIx`n<;j_0sisG1%FyWb;dOcyebAY^@+Bxpocd3C=hGSoKg` z_W1Sc`q=$`{vQE+0b7&*D1lh(qcb!!{zcfzV`=K|rH*f=EVZya*1RWqg+o!nj(mX; zjo0`4l2aO*H@Z+-TWZy{JnNsXEBC|{yhP0r-@Ai$a>>~qE2&i-1{FvEJ()2LkXBa{ z4dAzzGFQQq(XJN|ne!N59p0Q^QCQ{xo!}5fCvtUjihlTX=^lZ?V$s6Hd#o5xZMVP=Aw1Z21Q#g-|6psUDdmS?BE96 zYvV~<1-U4PG;{<-56Q(v8QMgsT58)EW{XQII+<*cKY@@(QqAVgM1EDWA=?~40>pC6 zadT>OZztA3fC$&vf_+A^qG4GAu1lr_$bbh;fAPW;71@E4EG@h8U4 zSNNczE0mI*+l_%{*QId@ncgV1(3EW|+N-y^ zpin_tP||K-7|6E!N?usYw|aJT5JfYX<0l;lV~%FO_l(aKI4x09+vb{Brbc3L3jvzh z>}|CFB7(AG*u?j5<(;Cdbp)QxPauUDnzCm|-^3%Nb@E}03$+t&&ZH#x&iHa84=;+5 zUMwDoF$@=YHYec$dRI=}MY@vM`CvkrON0e0uIwCF=tcG`Y>hm&X0HEK%daY^4uGA4 z={l#(FkZ{(3CQYgID}~J6RSxNp{2;>%n4%%Y(&Q><{z@Jfrc|#e-@LG6qnS z3Cqi~^pkg9cu=M9^Ew6x?MdIU_*rOdrc)AvJcuzX^b#mu-yUFEN-&EPs1sc>H1fBqdaD599-fOi;^iHv$BWARHi8nb{_dl(YjIw#nr z;=n$4E^Og;4hMq{Z-xt;W8biC zl8GT3?Vtuaoq>R80thCcSuJtD(0co*sAyp*YM0CZ@Yo8>P^ovui;*W%8-Hav9f^`W5YFQ!}rD@e`eW$x`1xMwTvlw zzdd_ho+16J7m{;_KC58-c78^dgYwIu@Jrz%Yfv62q1YWVXE!IqS}0g46#$06s|>|f zg{2DDrG=ZGCd2Blov*EQ*Iw0Ikk{@kf6Ot`5C?X4VV+BsOq}yDIs72q8g)-&Q$WWE;`D?mq_9kzf?X{0}ARN z(9L~9bw50TkL^>!_=GloZ_dSPw$+N(moN+_?uUj`QZz3Dz9*4b1otW%F5`Yuh;Uu7 zOFqsp?3#eBAdRDX-SBd6a}# zNWoxKmFVu>HP3(PNvLlDl1#Ix6%flg7_x>Z_{za>Cny z3h}-)HU1d+xRT8J&b)efOms5Sz4D_1qI__@1Af zp{Kr{p>BUfi}$NgaR_~0-=x_8v&!?wK*|eSf)i8EJ5CO+Nr|0Q!~ZO%3G)@ITjcg9 zrt5rB(UYGhp;1za;(r^PhYj`kOSVUEP)v+5i*OgfGsV<+L#*yh0?e8TIxhADjmCc- zC8o#joXUqmaZvc3X!VCT4(W)4ou36le9izPM09#|ZI6(c3cu?iL99Nko^N!D%4gs# zD7H8Q<*tTbYcvdZ;E0rH_gDXcfCH-ycs3rcR|a^nk#983ViFzrZQv56LIfR%@6Q^X z-G|Sqgb^m;fzcPR)*phRH09#RrxQQfBI6Vz(JgaWvgN6s62wiey5XK4gCy|k>~Yk2 zm!F=N{||BcTg`$|9l@M6@z1XdCLju45c#+RFZ&-CkmA<5D_#eSH{kfr&+2@!ZLb!B8^n=2G-VY-GFD^obKTo*0FThR|&Dz51#JhW`&j&R&!6N-A^p2)# zp!Lw0C+q{sKR1Lt&lSsAT1`Eicj{-XL2=#Tj8Mr(je1M5Kg!sj$x7uq^3v0F6U~fe zEe_w{iOT(XI<7UswTX=Tdh#e0T!d?l2iHj9HAp3GM_@v)k!%D{C~qCl?d=fOAl{I% z#9*m`!!YJDc?_0`@b~!P*DD}5mtI>foJx2@Kw*M63=fLD8=zNwO8O0pzeNc z6I5dTNnjWLQ?|&DbtHOkp%Z`iU;ZYhKme911`kcYzfVkI7T!hm)sG92()#X8Gga`% zTcc}GYX1G-EJ#Q?Oa9<=Xz=PQT`Y-b!PpuHvSVHiA22~tED<|)A-)GqEhsa=h|d|v z^rrA_5qu|=sT7w3$#-2|-EiY`D09pg*Tkd#Egx1N9Gql4Rmy1yXjJ&?oWX~fQh zISEPaIH!YPkhwp|I0qfdSZn+>&sBB{f!>O;{FeB4C@ehJfs_-_K@%> z-kW-GP#(j7`D%zReD7UG(pjjaRXP}KdQ7}_BiVzRJLVweRKC&C6%kfF;?OTS*jy#W zRZ`GVkqCq=DY)&vlJBE1D17Mo8d2bBR)WG=!?Jz2V28*%+NH<0oRL(HC*ans&hv$W zvt=Q~!qtDFJmtp{9)od-r@`tAClqPJJqcNw2!4GWYH-K(m?MiQ$&eq#2?3BT!y-2B z>1y-^QDVUj8~>_zfzq7^HoNz7nbR+ZHzAlgKN+vY(IF&>w=>o~;fB-x9%O9`Ik@m^ zgn>62E^A!(J#X`bJtN0)SF-MRB*S_2kCK4z1u6#2Ykz2G|9>7?laGnx0W0wPNmvq; z`L_b4P=)t=A6qu%-}rA>pEe@}qa;0D=nTjd>Wq z0w&ERh)+_%*!faWI}lucx-(4|Tz&{#{;XZAWID?DAwF4Y&QPZki$USf)&29KvUkN6 z(>CM5UjvcmWmI0vwj~xc1b2r73+`?~f(Hq~U4y$jAq01q1b2eF1_>VAf`tIV-L31(x#!;V z?t5p9_ow@Hk7f)8JKz4Q_O7Z`Yt1#+tbBN-AccxdhztbZc!f+KyXK zyd4j+*v;-$_T}~kq0D_&WS(9X;zB+8A!Z@w#zMs?7%1w2Y^M)XvwFCGY>cUb5~p)u#&_wcVwZj+tQ*Rl zPn5U=%i#+BxuGjPZE|4;4*G(qCsH>HX~)oYB>SfAt;UO6ZJQBvwGB@4w-l5qPqX;X z9vFrn5(B)hKB5uv6pC4h2T=N|f6;Qs%E3-Xwy)=TPOYY786q5VInV^_ryO(~Vw@-V zb||jW;LcB|yWGp>C3bf&`uh(jM%k~gORbdf5eyr&zR_iiDPVTPqcdZF6~lVXm6nHv zrP}uG+KMhMZ#-*F2H}?8OE7x#S+(k|Z4gXkDA@!pE_I%mod0a`%Xm!KO89Q3s+Mlr zgyBL%3VzdXlJttNx@{lX2Z#`pmj3Jut;#b%qQ4o3nbU8ZHT@KVmD6>D@=h9K;99@H z?YsY_0n@CBe{zDrSE}ed3ABwayr3X@$G%LI?*0$-nP{6HCZDsyYN0k?MV zmS7CgG?cR6GG;0xau>vsjV5CEKBHUf4s&dbu^(_dG$AMaEQ+WP#Y?QAPxFH)IPj#$@8OmMRon&m5EpY-&Tk?^vbt_Y|>}q^G9mUycjeOWAi7P zm@ht1bR!z#kCAeB!P*H65b6f5*nLT97}Y0T-bGSG+6$5j+6j8nwTG15I8K&LjLJvu zn_P^mu{;A)@1LIO#$sxHQF z*knj0pHME`H7I`({o)uRp|o?blM~KqS-7;ug4G*~r41q$l89K0`BP|VDYh75@$j!PTVDloA8YuRzNjOg z`YM!*#Ot3fH6OF^^+hg5h}A6oLuI8p1MFaL?yOuX_Vd2t&zhwKLYpP(2-6YX z-5+>J%vMJ15TsW6dC;ZG8RMn zl>RxxF4~STGf$z5pE9OUvF4e_m+EANp=SzIOw=~y(-=`$3m>9Bv_8e`kN)}cn5HMT zAy-Tp$%4^>dkcRHxhU>bR99mgEQ-|vlW9m?2~6)WDKH}pyM)myp(x(iy{si`T(N$<(EDT(dssppdy z#aSx8h4(SVFPuMC6&V#dY-4Ue*|y)d-qs?FFnxiNZSoD-Le_k$Cawmd293}O6%REM zUe72|GEcI#7boIPsiXD|VSxIuV#^;s6>wsh*l+^NFGVXMQ{;jaBY*8^MAMHv31)z zy!pVRxK7pZq5yq>V0@FF!iH7M<++;nO>#}?NVH!9mOr;droWX)i>4X!>lo0f^o z(C(ubQJscA+3E2M@C7$|H!wCFP8c_=H@tD{aQkr`8MhhswUilB66X?27_=BXONUA& zwdCuIE1Qhu>r6FK%4XEFYJF^tEaTo*cjR|&G?w`@_SMbS^`2*v3^RSRiPcG1lIM!&Lq>7B4WCMe#)!lS=Lnk4sZO@e zMiWOv3!wogrBU}WhMAO?^O@7I>*9uoYwpO`uQ85NTv1+Y8!NqzB~Hy73XW)wS?f!` z?&rUZy;!LnTT7%_bgbEg8O0rC)?Lz7uF$MtvVDFacW}1EozgVKvthWwOBSH`d6XgR zix8Q2s~3v*#m)WJ=5Eqv{ouhB?SfK((t@PmdHvz$we8UXqCT84yw#Imh#Y7?;nf}q zAy+;Afnbg1(OYG~qw6eTr3zJPPYie4qM1?TC>DX#}kY zO$D6=c?K(@2$PD2goLV-+Q%7nt=bp!3dStOuv02ZDCJ=0OiKucx`ZT3az{x@Nk!`N zUh9TChx6o9;C4G0zUf_xs*Xs*cBRe4C&SYtc;GNF?m7*s{&3oj^r0DZ`Pm~HM){^R zt0V?R8|g7Qf;8ofV=0j=-Ebt~G#wE5Q2y(;Y!}kbUjl}v&Nhf%x9j*Fj ziPo(82@qNTse4S!F#N+B97`*Ch+Uo~xZb~xL@*^+cieoya{ z)S>rT?2utW*i+*Cg!mynfmf$X#GvuBRyS@LR_jQ_*Twi7A!aUHDP3Qmw>~=lnxdwn zaq3d(G(wzXPCw#5>|shtf56G|T@ zB&U$#r5}2ZeH1@k^h5ErV$<1iwgImlhn<$p)A3NBgIwWsd)4Fm$EiYhgzR4DB51te726c z$*v7+OFWikj&Dp%Nb=Sl((+o}j1hiw`@Q|Cr}TnyMIKQSM$(6*MRn2!uTA7VoSx^T z6~wJoJ~&3Nw~E`<6xF@kzUO#N5DpMtrACJS3hk{`vrHp>!92APQI}->Q1zlE z$H=T}L}hAeYN|Qhm9b3J@+YTGbKibj3yra&QJLdjBHvw8RjYo5*Fl(f%8ku#)cmr8 zpZmx2H#>SWWVvJvWm4M((-$0wM0|u3LPP5T({%QovzY}?yZoC*&1cRhnV^Db)llZ07(UY#}H9XONdl1PyT3s`x_U;f%R zT3Sm|G*%RH9r2a%)Vv?s)i`sk-+bTP)x^|}f0BKY_tQkrwV!O>x763{ZOw(Qd({;^;NCFSU31rQt)~ zmF47&=iQ^bJIwNU{zE#yPJdG~*h5nUs2V4zU!QGr!j;YgU;}7xW}xBjvDD7PV1GOk zVZ8N;ItbR|=7yeaMy=C}*26k4w&#F~^@7^o7vfN*9CUG2aZobI5_9{u{D6CIm74(N zr7^v{{AOzT!Q}a4-1Er3o&i!-3h`NXtH`U^*U^veBWQ0X4jgWeo?v6R`H>wwz=Sx) z;Gz0^6NBs{)I?p{R9+s69$X_r!9f#3JpxzI;8zHm=%3e;(6msnzdwh8f_iTO1^18d zD1g6^pJ?z4Y4gWl*w|nw1n@6R@ay&w=Fe}#jeLat^BQIY+=CKTc_A$g{;C)`n3&i& zn%g=t3g6*_50LC$YB)kc;XZ@>LQ5+@Jp}#FSg5Kysmsgp8QEGh85r9dnlQOp+d=w) z5^&=Km)0gu2IOwmRyK}&Zi1A*zrhEtAs;hSlK=jSlcgY~y8J8h7q$*2d{MVcRXehu88TwzQ;*WX${VAAeA!GsO ze=eF3vaAm&uq-HI3voqN@E39oexQL3K+*p37jm5>AG~C41_dPoB`q$h>IS`=fzW_0 zab6#VoD6H)6QVDogb{8jh9U8Y^%IQgD>=)KsWU8WxH}@H89!di=Y~*9EHp!*NEp~k z7)q=N;e~foMYXA!jqHvzXL9fBGdI~5r(RDLEU7Fy9z?0UMscG?LV`i}gMuT6f<_d9 z`VW_1$v;DNX~3f?{1^8iE%cGd(^=8~=4NytX#0fLSoA;k`Ohb2tMrJyw6urs8t38?5e++LO;7+{6F9C-ySaV z1WW_($LL3YGZR7Vy?-+hI0tkQo-I^$iDATG> zMCJbes&2uCe``E1%<6L|d`vY-AR&#PFOiuz5@NiJzW*a!+FImwifBHV%+0*=QKtqc zm=;A@Gs%cwSSxpU-ed*_7QmEb)+DyzFx(#QF4m{DY!fi3B)>|Lc^iA?W0>ya;d`$( zBj(jkYsz|D>@&V6^(d@)J4G*w9lL;Yo2I=yUVV*achG;W74Qf>gbDiIF1*|}+nC^B zg4P{Yt60MCLa+AaxPmIMW{DoGStS*iD8ch&ZM)7#*;(|7d+iVRRf9~;FK^EG@~ju? zrW5<)Xa;ko<9?1wktK}fNNHh|sT3$UTyGVG`NU?+LOn>%aH}UihS^6G^kG#97T%I@ zMmqFTO)?wTGHQ9c_+@LGY>ijvUO~kcV`TiN_4Qk%!a_E;Lw)!2IkP;=nR17245^nF z`->>!1nj2a-<>Ap*=)Y^%(*OBukTimN(wxGS?Ys#WP3O3xOs}cg12^hnVOlX;J2CM zWNJ~~=_Xx$5jW%7>v-%X6Ll{IwHh81O%KE8eUNMvSs7z0=y@J58VLWP+(b=oymqwa z_G=7PnK_r&rCq)?E^K+Q$)PQ(fn$j(7U)>DIctPl40MbkqCG<1;nv^LDLsV5l4-Or zGBhd+HZU_G)6@2spM+gwH==(h;VC&ZT!43lq z2rk_ZsffITxwpuUJ%b?*YE&v^MOicAC|0?Tre-00Gl(zI2C0i6ken0F$+c(Eby>%B zG?USF(fLiz1pA5qm_?eP$7wV!qvqGkqqR9}cCtdk&9!Ah*iDxrYDljx!Wp(fpjRrt zhICQCqgemIxMVDSfxhql$yrIjC+l)(za;cG_!2y4&YF?!-bdo9I@zk*>M=)`Mp)}I zKv$qTiJMUJ0_s8SJyn^=M|_xDL+!`U1^@&bmC_W{(`_eUUboLt;{a2TvxQe$oDTY2 ze{4ZP4R^leX**_KnP%Ij@q~Z(b7*PXdZO0P!!Ky{N|qOku6h#IN{xJA2Ap}uk>Nrq zz{~c$%9hgvW|2E}vVN9!q)f{FRw52+jW^Y&vv>eRm)9fFxP~J7OD*nItHDGAz|^0{ zQOn2HSudy=pZ)BQ%k#cDubS4h><61I*J2_cr&elx0C8n`;8KYQCJz&=p7IAJA$d4A zq4A%412+S1Md=ib;_tx|_}V&Rw=qR{N|+Qi!)T##)KWPuO?`cC_Uen(8xOls1`paF zY<$s2iUVqPB@b=qeib76S0NsT*=x)e#$&<=kkQ#YB{cQB6=A+i{Qw8u>-hA$9|i?* zttim+@{&}aOodL;jT;n#Cu68&IXtOIk7QBmN{%dRWb zCV{!E=Oeaw1k(fAa)z`@)Mb#l-M%`$9(JRPBC5@n_B$e$Z`Xgl4xAOe*-I2w3Kr*W z)j98rMU(U0Ifoe?S81@R2dlh>l3B-dq|hwJvfqW13F~sr>IYWMe^PcvT3Oyc&gDeJ zrUb)&_sfoP5Ap;d7?~a6wLA{p_((1=*)?~8&g)nrJkPU8IE`NFuXOmcIRSQLzP2O+ z*K)DRS-Le@$h#>*lZe|U{V_g^Nbq9Wr(Fp-A9&j0@JPPqLi2fJLzIu_o8~El$O_s*vo4XDQ_fQ>wziQ3xh{ z)64j46h8nP4W8&XveX~5=Tk;@4kV#`G(;Li-_2c&yGkQods6ahfCYyrQgwbG!pO8vkte&ii6KBETH!;uZGlD zF!bdR|IKw=3id|T!~G3#;JcK23-w802hHm$W(sIf%cnk#61px7Cl%D(7)~?mshUs} zCK+S+u?F)X`CT*_&lBu_Vo$tre{^8FL5SS`^6a{pzb3I=KOChSZql^uAete$vMdfN ziWHnJdhC?38FnLH{mO~562%$2zusZ?Y*`6F(X5nH1lyExYq7bW*yGnHxzQS@><6^_ zh_u#kZmhnba>s*NQfPA8>e^3{z=}{4y(MDw)t6_3Y*x10m&{!h9VZx1W^(HW z+@y=mCgzxX9r^AugLSFW^EwAlE6ri}NUAleP3vyKj%OnDmy~$|eVazCHy(El@gLX6 zoBLzbpa<&*@nODR*-3!_!31=Dfj44NO-j^$cDN=7`dT`Ohg}sY=c$X>57Uo1xn)?O zGjrhhlgeMqfv{m1RP+|OVR~1Zw(9WB?q;4^5^)~cvYjnAA()oKlNf19p?mpYvMmX3 zq(kDVBw^PmF6hiC^BbDC)2zDL8yLGqK~QUrm>!c6Y+x>EJrNeR#r*JmbUx4Q=3myE z7B(scfQHXz8=cAwfnUO*l6wsso}}8yf0SV&K(o|Sv`5~h@m41*J#r`D-nRYDWs1*f z#UEk5ZolPZq`0#}$KFU8vNLZuul6$D~Ck=oSw zyYq$7d)2Z|Y|bEGm7i8kghUWKBzj|DP6GOyg!Txx?RWc7dBV~P4g3M@Q{$%)UnWK@ z&0Q=7X6H;@vhvw2kwk^Q%6G&=;+|onSuI&GJGHqa51y>LKMr1auw+54xeK90#!7(E zXx!&=zb5ZMX-7t<`VMy=@!64ukqgc((r}IbaTzRF4?dBoBSr#dRu|HIZU6#o{DQ)R zfVYgDgMkAZ=ZcZ_Y0qt571a{JnzbHwW#5I~DCqZk?r=VGLCvR#pAf$J-?p z16jWJbO1sn^x$d@C8qK^=6l^=ZEA4dyB3>F>-l@Yd*e{alAEB`4w$WR>) zfSe4jruio?iLd_(?&RBR>tE1>E+Z#@>Hbcy1m3(@zyew?HyOQoW65|hxFaRJoIlu=AOgc>vs6#Az(dU^Gc%mjjnr-4efNZtL@fYwb|>NS7E)`LYbDGex6U`fFZHn$yCro>;cA& zwJ_AzXyvI~xBz5Xb#gIq_zzkf0|SI_4+9Ge^QEb%tEif%)s0Y7SPaBOSFJugON{2S zp1-vA%%rI{8%D*cJJg_ETdi1V3l7wFdpqN`e`pMyB)-O>^O3-`d&HJ(C zI?dN6Jm%@ZtFMLAvSx`)_Jtv*nIt77ywT;ose*X*X7weCI6Um>h;A(yd$1 zrn6zxU9PWpDy&uekZ97$+i#DJF{dm4HP_k&M&zSfe)w&f@|-C=;vq5oOZhsH!tqzef5sQ#x0&GFK3LhSV>ECbjbcSbJFdv(<+f};W_as+%%h0Vr~xVc}mXW8YyJ7_)- z0>RbWN<6jvd9EdLOeht2QDPdIyC@OuK3Uc-4IwhB^kJHu34JoR73k*jyfDe>5=65T z0(&@rTCn-U0`NXlD8v1@Mv==Umz>4FB{JC5*R%ruCgGsi@Le@9Z#}0#|C~s^fvf@5 zQoH==47VR2jkkdl*-0Yij{&Z3xCAe?KA+0ovA`nSZ92k1;L~J|Ba}K7CQ0?IuU#v1yinlh``B^3y?{4rbN0`sQJC0smz1tTC zIhNMQZRJ*q@XNbvVh%{gW-l3^tVmwI)?=kHlt^|lQIm=7MfZ+-9q9r+o`LPY3gb)ipX={m{PZ&BL9YM8-I0>ov8TI z1XtJ1;K6OLcII5}r;Vihg#y=3l>7edm4fi_NbeHBIT=a?$yn7!MHfeXWH-WEHjR0; zUl7fDRHL*B`kp!j8=%9{KlS5#MIlm569vV5`}2xpmg<%$x04lafU3h$m6W^e_48G^ z0~5-pyJY%Fx*kQD^Qtd-Zos+=RwvgN_@-}@`EQ#EQYT1)QF7>7Fq6z2VLn@xlb~KV zcX@U$r&`7h95%4Swi8mHO9R`xvzhvtfFFwhavpHnE`8s~a$R!IoYU@Mx0Laq=9^)U zG2&_mk%MbALXjqsv0)`(32+KKf=At0nJK;dDLO2IccYJ6+vlWtb0SN$1`4pCFh*oP zNopVlDT2-K)%%^o^tZVVvCIr>-)eE6e&-PdQEtH1l<^&V^-2Dn(f;LH)CKAjWHVXj z)*8H9BuQ=anqZB>`HDW#BY*}%%_OARv%VoRU1Qa&qOK0fxu0@umwp=!L=*az(~oCA zU8F)Y$TGvf)BbQ*(OS1%Tyniv_$zN{IQS82P&t|J?N}5oy9)0ep8^m21g~I(9ZEi+ zNoLP$Dhy{(LSB_k-r#tUF0!3ud>80 zjlM{dJu=7lBreIes;_t^gyA^phwm8LE`QvU`EIXGsc8)p@jB${B?bNfkO8~zgYp9v z`pl6scx2ZYcyid6to! zu`!ZiNi`iXF_~fKGR!$+j8~6gV>7?WJ}NM=;z37%XqNXan*r{vl;1G)P+BUSKRGmR z?c?#6ORV?unTlIVdjOv$jJC`ULw^^WG>u?q_5C1t11oELI9rkAm+YX{sA(uwOGC6}+hFyPfz~Ol*iuGUP(|_U=Ma@Jy9;;%U!q zf&a2aJ#WXr^9n1x*$d{MJS-DrCPjIPRc~w>R^hi6>MN2tM~rVTk9cLml7BI??xcLM z2|d2QIT*CAUo^!#uA3n7VXfQSHMa>4jLv#LO8&ww(>Wg%B)dT?M}w2Xxg-b@aUj7Ih&z3;(+@~DLmrtXHK)XN} z4z~R`MWOx>9(EO3qIh-{q4ke}Pu@l!u!ZnAKD`VQ=|-GXtX>;I521!W37LOj4b68} ze@zE{t3@Ze{erwLOhc;E6^xKw<@ug#ili2t3LS>oW~OJk^@|yy36jm1Js2yV`5-~Q zCi4fzSSX&p=y}k3E|`Z=H&LKy%Tk)@eXilTh-ijdZesS!b03g|XWu31@N}GF^t#LJ zO@r8lncbqF|6p)yoi(By34JEPi#VAGl?FSR$n6>i{X4G1*2J)2JV#E$s7$R!*Wv%rK?=t>xF`+a7R0uqm>E#aKrteBgX|q^<4ao+$%|_D0De?}O zaE4A(9wVF4CtK(OZdFhRjV<|97*#cBVrNFzv&g*??)DG%&q#`>An;-JjVzC&kX;s_*UvQTthSu@=BJtg zKoN{c{hAQal~O`&TiwFF>@~6Abu!Gp-OzOXWXR`%B3;8O;(iUEOEVe< zJ%j+-Wr{8kM+!F)mh-reL?&Tqgoem@;#}Vd^f_UB#ep5=P<2T(u6sKoW#BD>je>NCkhEG&Oh+qYb>(Qov;Uz-jkQ&2m|tLZquZicP= zz*iEfL}%5*x_MBD%_qF7BgrG%5V)ng1#B#7gh7Gx&^Ep0U76Ce4oB?Qqxx^KLD&-S zL^-PTKf)x_Gq6|#}dB}G?+$iyW#j6z4yEXd13TT!+|LO`Sz z3rg$ACtyZ$dK=o@?WRz9g0X|< zf!%^mMug9NbcB=01v2&TJXA8>kc3F!E_jLpbLAZw<#$Am2w$odIJpo(u!ghE(NBV2 z`fXB)Hy~N19>t0jS%toUihZdwIFIM znD4-(Eh8m3u4OOB=kblS>=6pe6nh}2qAM19mn{#977N1ac+3#0f!Wu<+8TtK$wb%h z2fc0|nh`360ntXB+i4#QN+iz!L6v6(0~|@5Wku_>WRgXE3N_9!hs+k>z7j3V8}oPT ziqw4pUwSXp5nzk~d9$myCDkM_$IkB5t(?8GjxZXxT_5EOZgSNNN_W) z zovPm2#arXe@KJnG0cuPuCfCWL`s2CME@QqC@n37pH6&L%r0rVSaTez5ZOIx23{*ko zpwJD5KUU69FFbBn=UR;p?0Xrc{1TzHXP{K&VM{P7ePQQu^WH8|ce$W$lW$MCC!x*z zW`2=P7J7A>Uu8@#=iAFMDnH+xO)Lf(Qoy4rKmZ^hh`dyzOb9g%TXH)Gx3;L(cm>-O zv=qTxZ5+j4yeTl-=4cz{S{Ops+h5mZNL=wSqtCX$CZvN@OPsv?pY>C(EM=71Ct3+l zG#`V`;` zx;P;LhBUEm>!C#mhBWS$*UGW>Y>$dufsa$d-RF!K8 zJ+*0NZ!H%i+{sv{dY$HVt3@>O{SizgR69ODrkB`93eZ)s+}D0ZKyPNDzWE^qwuf+* zfxSIg{LnS=qGYT>HMG;(T-oR@Nn7&Skfp6O$|y0vHgM7 zZjd6?g0Fg37TE59)|N)}mzilDc<+BUavW1+Ep%VTXuA&&{S4S$I4t5%;45OlLkThU zTR4ox6udx5V{M0@fmsd^_n_p_n=Vp2{m<6&pf#%5K8(K=gq4&Jazq>g?OC%PNkw3R z_5e*DQ2_Ehs^W7E2MAbsN7+u{h^n6deEBXgjjP9b&j0NcpelnAM4mQwxp1!s9cAQ zBHXB6M?u|}k;-ZYh&+Sz16XQ#KG09DXG7DSQHyF4P1#fuAdTB@8$I2CRdgB24my*s zOt%(qI`yw`h~bRBoC3S0q%Ss+{1pa$1g+`)b5pJ{L*}vjJrwCmE4Yh&U2nC#^J{>qK|CTTg{3(G00}m^^G%;y z5tyJ7Wc?&i2<*GJ)uY$-t=Jgh#_;Le{l& z1oe|;coO=@apOhs^qt^LqW8co8H_tP$b|f96~n^EPrD8UF-AxZZF#gUa9|S9?LhN! zcxQ@eCs6)>)*5zNfeh!Cr3OtYFCGbX(Eha%E)Y-=c4~l%n91sZMH3)@qvM<)m(22s zzPCY7zxb@{00;|+UBBB`0hF%R(^kV7;5)Hle@%n{voA=1pjC+hb4K~9Y?*4?MEIf% z6?&X{#BeWM0a8GmKO!%T*ExF!@(U{@lV&583U3PQK3#|Z+TS9K?Q zK#mUYU)au_ZKH=T^hq9~R>?zO5!66t91w$onFGdD^}#OF1IDY8+7s*r2s{@he+;AK zYb(Ck#^NQG1`%n}gMzC9BQ(KU6*z;tG?YI_2Q1b(b8DaoK3w+wQb?1x z!DSE^1rQ{~i}_s`)78+TKqDFh!ylyX;{6+4&tvEy_F(A0x%odSvU$D;H(Sm(ci;El z9-{j;(qT{PE`^_O+@78Xf%*K_`<~ zD`A{-KqWxE(0=*^#sEF!+LLPg{M{I# z@fFMejpD1kG^mBwGz$;^{UQF}xvl<>*xbJfwf@w#u>kLZ5C1ne|0hLxrA}D-&D+K| zWdE;ktN#-=NBlSXP>?PXeEB!MPG$5x2q!|ABzk{lopigE5d;4QbW8 z9xSmL^*(L{atLD}GaA*f`Swxt7^tH*TN0T;vF#w9b4FWbMDRk}9hB#faR$HYS>z=c z1Dy=l0igThM>6=$w9$UoSF1qHUV5Rl1m zhLmJDK1NHteapp?=ip|RZ)VJI^Uk!xf_1%# z_ModW)28+wsJ+`lv^iHl!kBYkyno`*gY)Jmz3EA$;JLaxs8Lw!frS3LWxe6A0$%@D zNN!Jo?+)~HUlF*YcoqkM;ZOx2VM@G(;*BbFTamd;gJ`tQ+ivy;waU##GkrWqxFfs) zYc^Hqdb#=@PsbIM0WYjC!NbGK>p|tfqoVaZRgZgF^x2)AbsNxR5xBI1!R#vsDpC=f z#t-*ct-ixM2lRGuc+o^YPGgB+mhOMA~t zRAiW6T^}m1R$ElK@Mdac93O&2fK()t0haJJa1U8f7-k!cG65pd9WW>i`ijFUnEzmY zuOib0J?lyOM2JtYGL|B7j)&CmGu<~*!Yw_jx{#m01ohvaAykpk5Urd7;PAHqYZ~a#`IxHXdRB`^m=nWlogPag@$gqvNJ#^C<5ou{#h7m_SH63}qC?n&{`_E7h?+4ZYWAJ2Vio zuvY3S6D&PMM(N7C=2%uc9h<^#7K*W83?u-@6&9`FF$*p>O9;{K<8zJnJfDcKfT z5fcjh!T=y0t z+}0_shB1caAP^j+)@!$+^qjxgdAL0p&BS;r7=g*ILc1pz@hz#sVpJj>LW#Q%CU51} zWhmuHUYXE;;CY@w-$9kA;Cfjf&% z0+_7D9I^+oMg7`I!~Dc!oG>7u?7p{s{c1)|qb5MI*=5fP7D58!2;yqni5@^o0(0h= zLDhh}UO2xgt6v(J>k~`KLo_q3hVjg*H0qnDjuJS2>FEQi)0-_4MqMcyLCaAxkV^g? zSN7BNA|;v(r`WwsP^>tBC4DRy1wXFK^Hi|A88_S6)~F{;tM7bt4?n7b$07_( z6#2gMxxYP8T&^+9e|rx~znKazrjDux=u6x@bl=je@uYEDCKkZthDs4y=Wl@$L+!=% zS9YAvsyM8X{q_eR#hvCX2>;98Z0|wB?^1Jy#!CsNnK z)AO>wDWWu-RQ@AJd3)bY3VfM20GRoso2U3NQ%^fKpdgL`5=i`S;hc`{0c)t?bG=0~ zh6cS?{7I{p%f{-ApSJQa9joewEMmw7-_(P-#$;Uda}Z; zyh8r9QKT|l79yPmJ6gGB6e1~V0d5}Djl#m&(*JrU;-fAAN81vaHxS?-f&^WCpLn`G zB@6U*lxuxa7iF33W|25Ns)yf52%QUh^@|hiOtUKpaWri}VIjTwE&SfFv)8P_`+gA2 zs7%N1wUbabaGs~9fVWpjW^O%=wE}L_CyhQgg)YoRN_jDbBRSo!4&c3T{WNsfc6;mz`GPmz8Xcdyq{E6luDE$=*NFi%K z4hcd&>`v~0=o}&Pb*H`g*F&d6;hB&+!ZA>L^cNLm(wlzq3%9HVRUDfrLznBJ2G>t< z`rRqW{<4g6=nC{qQIWM+pk(Sr@_B3Z4H zFNcA1t17453#65$zKcj^rf=kl%GUKRZ#s6z>AbH`V;=4=+PTI@WY`826oN0{@iB~blfSF`$g9d;rZy39V-wK;953}Lrkwh|zR zXsW)uZ)O;B!I*#B0Hts7nb-x>G;5%W)qczHLvAo-nY`PIhg?!5u3$*h42aBMCQ73in5BQI+1p~CZ zRTv^@f~T*Rgp!>*tWtoMfhR}qJJm=HyChGwH#^F-*=bpFTrcH_(}nCw&w+LAl$v|m z4M1+z1dJD;gnqLg`hkbt^4*oQ@YPQ&EpB=YGeYS4S;H`m8%XiJUC-ki(3z#RkMQld zIc|;&H}oR?l{R=QFY+Lj3I~k{hxm)E@cM-eH7LWj=*f3SdE*X6rY-^h9aEn@=RBUM z1vD}2L6J`p&G;U@?i&JOBdh-5iiv|deGh0k>#VDxMcSyX+UD%X6Nxy?qxH&JSBFY1 zHd1uNty^sto375X$E_YL(WFOZV4 zDk->V7Ny~8m>1_yTTkRiEPi5}t(H1BQ90Qt-B$WTElw3Z3k zp#dN?z+2co28@y1N6e{+CiIt-&?o3~4+%1c%fGN5*3@_LFM%(=wFm_;Rx@YnJLN?N zzw7y6QwaNo-1-eMze}l}6dGaJq{F+#?6a%+;Stw~`_M&Vkpm+*q8p)ah!tW=t|R#? zW}kc@hezs&!r@V-QAtNrIS1L>4;PM^j)x`EO)iuMw`#!|kwy0A01v`N966>R49&Xd z?jZ2rzFl`XCqg9tWeP9JP~+VQ92d+RLQ;TC+v?IK@3tfn&g3E$1+8hF4gePZ>QWBj z@d_US-sZO-jn@7xfuMzyMsK53qa}|ck8B~QX~@yJ{=v)5-hc9j?5hZj6~Q@tit-mw zFz*0Mij)1NRuvP1L~Qdc1+^fGc@6KgDcK_2o2n^&JWs^ezx;?C2iQaDw$CME z2)Mu^ZBcf3vgzFC*P|CgXs@?ed99%G8^(Fg#4rmy2fAV|lP-V=Y#yiP5Ip}AV1}AH z#=2R<@lP)hAZJ@l1{v!a%nu+D8jD&k8c1O!Ak}{@7l^=Ao@-9LZqAg5_hg9OWK06j z`dcqht}+76a^tWtz}qH3Z)N_9I=S*@zj?5J*~hZFyzTm}#Y}mIAOspX9V`i|bF8_& z!%>R{h)pk&Z=2R)s`zzp?aGI`Pu6ACBYfb5K*&m7OJCk+2z~wp`bkpr$De$#JiMgERq(h3qk-?LPJqo);jw{tjQC&TX3s5$J%8&E;7Jr2i0< z6&h+|+j^GB++~pl$%jgTvY5cS6r81FbKH2pkCES>iHEQ zf4VT{wgzpm$PT|gy_cEDm$$|!1B*QRmEt`yCB2meTxw>?=Z#pbkyH)~N|4qDMVu=i zF}~Rbhu8o{m#f(Yfr#QNsJf#4em>FOdMJbc&64{jN#lOgj+JVLWLqn!Poaue`rKXj zgY@D#2sHId)@B*biH}JIW;x@IG?@5tp0(xOHATfRj{qwFj`tbHN!+3B@?7%+`4I8)>d=g=W{e}{p?DH zT)s&A`wHQ&)>Akq1X`0)xZVyDg7V0@++^zT1gly0%I)onKSr4V752OX z&@d?-#$Ev1@`Pw4%_zqOumsHpYpmwja^wzkm0ddPPTg`MxT-EVei)zUO=d{_URPyq z{a-9#U3tN}Lh4ZgBBh8QpL7rC;A^3Pb)61yxZC&l-RWzq6+nDQy+t(u<9Qxaw6>jAd1LGArSbc-}Ae^6H!KFc|Fl0A^jV$zVd~UY>5R+ zF{@BAS=;2m_+9TSl)HPM5c2#f})a{ z+e&RJF=NSa8kcqeEW<+WtXxepJ_X}}s=SV?e1YSJOjG1M$XqB3GF(z2+IOEApz3=K zk!TWtIO?=BJzTzEnmzCnXfJRVFUgbXP4KJ+N$SBd!6lIB!%rA2ItxLnJAZ<}Xm}7L zEDqVcA+-v0SeT|)4}h1l#M^lQ$0KCw)bVD8!8wD{%lw^pIB_0l+s~-0218BhGc6pt zQf$OOb|r_ZzD?V&+lswfW@nw?*pq~%AKz)=q2~ouNl)p5>CP7AeUm`|vNMEvORk-} zJ*PL>g1qpv_uzYo@)$^i?&(P)LP@|gqr{3^a zW;?`A-$Z>_>w8l$Mg(+Pd=p-)9SuFnSOSM9ekQR7B$i!(qUUT&FBbw!yUFZvqmX=j zhMCLa>B4m2beYz|v1It*S8fbvF5xt#yj?b%;|nCq^!02p(tqJ>k$FUP`$XO0_)Gv( z&_l5O5y-q=0ad!<98j(q16jzkh5Rv7LT8AAdF*T?R2qTQ?PEA0rwsfPVsom|8U>ag zR)MHo(Z2U5SzOz)gJ-|SzDS1DnvZprdvD~+apBsVy5tXq@J!q!52JSP?$VheC<&g$ zNoHDu0|+h=+z}xt)YTXP-+?{4q;7!hk(w_-MI~=H;KHs?Je?QlDgI1Qpb>LZKmjbZ zRdE?&v*IBg;RNitKoaVL%8cc|uBbI+md#XLfpUXJ!lw%Iaznf^?ih7Lm+5#w_HC3T z#sG45XArkTm!>l?z@9-JMRj$g{WpFW%lVtje})7gaq_=S*ORyediNjfwg0V#G4;-YA%Xqo5Q zMM{(qKzf_^820C?&;bHSfw@#DON1JAFbUT*N-dl;i!KT+rX&f zx)Z6TBjwPj)XVfT*aRC5{zqv{Zt5mHad?6m_U!Oe^Fur~BnetTOU*}TJ#_*POA~b% z_|8h@3$eqMH%`*ojoW}D}8wNv0+iXT~i%wAkNvEBsi>x#tfphUfj6aI~QBt#NPIUuxtAfpS_e) z+%ab#Bge9mBv0R?71zR=<-E7MrLyD47w|}b=;TEtPW4P(grb-KYNFIaFKgq`j4R|Y zmR29uzZMjM*1q+nzIP6TjrN0*^cFGgV+t`3xpj)bAGcpSboAY06))OtNa=Dh$7bnS0d@b$BX2{iW4q^+?1BNq>5a2dAXCd5G1DI zp2p|gA98smyLQN!ouFry3}wOY^$PZG!t)T&ZW;eO?o!>M1ZxPW9RX0F{PvB3rGCD- zF_Eura1>i`-(s6_ck=KaR&>oKw;$GZ+~_GYR(w#d!|jX|v@VWD$%{e#^i?hUXHdZw z37#f34jt|2g~@O~d)CBPg1<5Kn11Wt#}uM|mc&Q)i(nMbsHzZH4qCrRTy09tV=KYq z97Ff+cepMqiIg%5eyo`9h@FM3OeY7G(yRM#J?3|o2igz8&$Ousy)z?)dvZ8+T4A=N z)VXS$aTjA1Ag#?YPO@vi_wGXRObDg$3&-->_Zgvf^TB5{AY+i+&7jTrVCXslI>=k? z$K9DJT=`0iM_+AkE=8;uM5%oy=k5~t*gauC)(Jk@#<#N>IQ0)9maj)Z;**$LK^*G+ zd*&f#sn1J;vK%0prQcuF>bD zv5sTl=Wh%QTNjOc6YrVEjvOGx4@MDjsN`}$Cwgx^2xxfhjjPgjw@E9%+W0S%Bkn;# z5I1LEjRDZwqrkAWR|=&o+f`11Pd8$u;Ghfs^;MQe$YWG9Kib0Y5o4o%Ig($q@)N+P zrD)p+w`$)KsXYC7YZOm^TkHDv8*?lUaG`2r-|7^GK`S6j92tfR!7d5`XX?cw8=RN*@~_$~H(AjO7)$oZ6p*ZkalK7`U`0@KbM) zwXBbG=V_nE*>lYz@uRb=?qzTNPnote_yGZ3QgXOD9q%dWaK{bOd#l&nCd$?N)X<5v zuP$EcVj$TS2hUU=O62W_etfUT@ABg0d5!oc($I?DZCn*3I7fg^{O1^ke6d>TvO_D> znXf){a!)ZswBbFare)Fl;`elN?W^^PT8+oj-CSWRL)F&eJ|#9?yidvsQKC!g!KClA zRvwx(6!sBGo$hzLi%`dK#wOaWWydlxT@g4c4zwNv%yq7;Pv)xD!V0K;gd=BUhmu3> zqnXB&9aeb_&4VA|vWS;ek?llYy6^k;Ca=PJ{hqeQ6SJNZL?Wcn|1@`|9}+|@QnxBs z|UqSqXhx8GL?HX%+Z$f>)G9Dg|fN1Pl9i!bsOEAcsUzb(b)DxRFdfN zjv8f@EW$jJ$J5Mt1l!`n)T%PN|Rd1 zg;r!(sU$wQu!Z~wpqwZ6!S}5F1TD4ZOH^}?UOOj|G6s^zYaATRiZ_oN>VS0B5C@sd zZvpd85iJs{($4!^WB1(o^lw7cH&wv?Gl_-={VY1ak=lB2 z30>rOW81E{E>>tcHn-AEgAS*+F__@=A+_}=Ikne;!PD7}C(UOOhw=+y@+zsn4P z{6amhc8!w{U>PZUll5kS-MgrBiLFPR8^yq*oEOyCn5^T~&tPzXAln9Gs%~@Z` z$J0`+acYvCeSsG{IOQ9I$X*$8ix+{MOOkoW%w}2L)pelt*ET=~)fp-Vq*1r(cYB1d z@LsKhP-oaewmo0kr1I|k#;Ck5Qw+E^GddiK^6xrXG%B{Y$|gZ#g};0W-0b#e#gIcB zd3ZGx9nXy7QJlUzY;iSnKHg@%df{M?Yvq|E9ed{(_*m>BV*CzENzUuP+Vc}x)XRSM z?*SCJQgjFf7ljROakmnWOR&(7*>C>w3F!b$F*-P(|L;2RW%C5j>6kLmgZO#&eZ2!n z@T>+~dB~KcY`#hvZ*S2{$Nebb=V1BP ztz!_W3{hw4`yL@&M>ydV>a%vMCjxdJt&Ns%tt`rXS)VlY&qyx1_}E?@MLjYetftB= zd{2?i&?#SM>c!3PxnfJB_f1pxhdR{l+!_Y^CgQk^;|uF>Obnggbq?i}A)#7hmcdOM z4T_(z5zbtAgKPvnK_F{gkZBV6HwKvzrC_*RB~bDz8O!L`CYo(9j;q?GSO53V=@zA5 z?U?7&llYwv_SXf^-{N-bC=9(jhH|Jr0j?8YJ_)pS;vC z4sHLbELxJFD27?`rZy_T@qy0KM$h^BxR!Kif{O$X8^0 zHAK+mRmS_rj_QCN{k3x|>W#>Uqp-LDDNxHXowX(aQ!$3d1ebahrt516T6)~uwiEHnABZLAvjqOm3}L-fjsg> zRs)fAB>vXM#X*zuM<+t$Y1YVJ{#f*Gw9;@NS2JfxM_}uBv>Xoy*`?nGwi0Nz+-uWP z)ZfR0QPT#p8eH*IzbiS(Fn(8HLJF;5$F?oIwqo46eSfX$91)&fP4=Bvmh)fwPy6;W zPNJT{mc@SpU9#FHz5hl6XmWDD2J5YkG#p5t!6ZION!UZZZ;CaoB|O_%PVk@@n}dhJ zEjE#UYqx7P&6s3v$Y4IvZpJS%$!-V6ukp;6(?)jKsdcjBmgh0(6D+-w$**-GVU&PCm6-ePvDX6pdbDUY zV`cyxu9+#t^tFVwXiaOshap@0rNTs+o<7r$6ZEf|Mlz$dmxp@oTE<^rP+tJ~u$>~i)nWP47dSZR zHl`RaKrNu8v4m-!_rcB;4 zROv2q4+jo`TKheB#c#`9EAaa~1TZhKnNy(0GzYe!K@=Ugy zR+4lEc8x9}q0W2Ex0YbDVqY{ip;}ZJ^Tw&xR9NzE`m|A70=Vd>Hq}w|B5TVpHjV*N zv8V&W2!d;~>G{{2H`6=*Y9VP63Gs`ap+~(WEFG?^H*97b3-UW6&r>AVVW934T3Nt?c@HKLmN5iD?0oDFsC zBhOypHT#O~_C>OpzV!3E91a+|P=`!{URFqrq~3k95UBj{SG%P1R?ujR1gI%&@}@`~ zCYxT=_R5g#qu<}pId`sVGD60$yw%W*(|DvLGscL;+1x&>Y>!P|SI?e$=f?G32`RuU zpW71>@ed)gEdFX1QiEC!T55WJw9O*fetzSnL&ylx%Q@dXS^&#ao}iVa7B0Oz{QOG0 zEnxBZS0@Zv5ophPIXBy%fOYNoZL#rIxz<|zh~?wA9t@Qe_VJxVhxsgqt`FGcv-IxE z{XEV%-#=t%lEXrF)GAWr7dvWNT2`orzWXwE79Z>9p(~c0F(C#c6%c)?(0v0?Jl&#x zGn*kQuflS$VCPiyef%&Wf&DRpyi>RbZ>mlRv`wtN)E2rD%?5OIPRsh~aH5{@-0{tx z31I3$aCSh_g1pRFYFA5CI6<7LTt}wm9dHk1I|`%Sw09o|-{#lv&yjbt=kn$t@>VJX z^GPSvF*yrqi|viM%cB)$cMK)ca$@wUiVI@&?fc83m1z`0rB>d7@8(5fDV-p#WqVCPDj%~#;dZGx<+?QGMnxv7L$#<8f(%*NyKI_IGsIG`I z%*#s7cI6dc=-PNcKGQu^Mmj|p9=P0n25Uhfya(hC}0xl!v$OjS!tsx0|Uu`=!h#>IE% zz?J<60;af)nBr;r0>FO#K`&GVq)DK2T$DXAmV+_R=HH4S71EdD1s_c_uO8Cnx_Xz= zIEFVM{f_|^g%=n8T2Pe71Y;nnpnZM*Re z197Yw0JaTCQ!NHt$PfUl1_b0_`*;gdXz<)ozGp*6iGY8v5eZ4_CMI2`a+FzIlH{knzo*P{}qXv19IuZQN& zg?gI7e$d3~-{QYbf*J*>_cI@^Tx0G87cOwVTNw1foIB#`Q~QYW+wIY5OV!_}OHJ&Y!`h$j~+AWDH2ckjK_ zysk?Ru*1I&GXN18S`G``g94Lx>SQvn*)a<3S>!A`A zKvPK-Dv|B>f#Ddyq+yZ@zPTE(quB4vtU(GQQ>Ah)8A9F^k4X$Oua+U6N9?ZW=+j zS%9-L4(|Z}N&<+mxKWIIdT=h(J4`jW&b=lx57}KCYX{cUiZh#aRN2f4o> zaLS0du|L`W#Pbw#RIq0VJ>`MK0#c{mrHcm_+M)HE(B#P0cX8!2%n5%|qrfHn0^v9l z9IW*>t-nyJT>FMq$w^C}pas#2^{8?7z`K zMA%$T=4}y-C6IS9)Lg*)v(Y4|!2*!7N0r9xo zvGo-MMwEcF&H%iJa5$>`3#T&efeBE$32e;tal>P$E02Ds1is|^?r}~ZIN)WWj=&cv zHqFibzL@1Z2@uF8Xli@~xcP;?7BIZ6Wk{5h7fD|*t+IeLR8;0>KVt#koLnDNl!=_b zcif-rJ7Vf%tecEEOzl*=UI=F9J%_6bYBHO;H%Rx`UF9ZG>m%fAnR#5;jJ zwxDtTePqFgsDRBf-ht|MK@3>Av^}}D*cgmpl!&{|$Pui-8C4NjRNRD<7OKW%pW|x> zxcwJczO10D5@8F9Se|E@qzdXb+cse@tLuu9OjryS#1h@zj>|$e-sfQQjUv$@J)~i7_-?6M&TwbMEGxJv98Xa=e zv>Q;i26uO)8??x3?+$lQe|j_Kvs#*s!_){AHi>+dcSoC~*zk`h7GXZ=nyg3YB~PICFq=xbqDtr&lSZ;LvGpA$o5= zH;RTHogUdBqMV}HQy2O!kiB?zUa(w}n~A|8u=H8{=q!nWrd1+@W8l!#&mNut5EGszMh^Fw*=Xd3bg(3*4vk} z(m?RH7!r|iMDc7P`d;y9s<=9YLI4#96A!oURHD+P~QMDKO@f93A&3k);;~! zAw3g?rb?>=p7+H4EdVMH?60#Gjb|M@0dAdOCcE+6o3vYG!_lOc;KPdsYYiYA*Msb- zGwS;$qQTYe8@d}8Td`QV1z6AU!>hduvAI6?b6BGnwz$!%JzHre)YD%g*1YvsZrnyRA?X!)Hb37p1Zx9FYRfhZn5 zbguMiW4$W4%k=;`J0mGAH1YCYM^M&%ot5j;%;_@K^c?7}ZDOD#ub>T#nzfy;7$7<0 zs4?apWatGc0+1qwu>eG8s3QOJH{T&P0`#v6>JiLz)rZ#61 zXDVTRUJM|5hy;IYHL8UnD9;=6-RpynU5D6h)Gq!6i>uojs>7lQ)muWzsMV5CUG>j9 zve}C=5TT-oYijt)RjAK#)kqy?h-as-19UxjrR{mD53zNN=-R;2hvjo5R$0e)?{CVc zlP8iNqdGrAEXmj%b^B7ju2w8V8oaW9TbA15;F4zs_(qa(PnKj`fV2MY zy6yxiR6&zDnP2i0`Nt92c{2zE)zy9hEshmLi~BaelRg&Jh(iZ`LT+*9^G|_SRAsX% zkl^RO&UqR@QC1-nvo~~NZMTVy=9?=W&Zxb!-2XUQsUr@OQj2k#OnFs5no|J24=ChU z5w|uo2zWl4Rol4PW7qZvcVA80sj)nUgY`wJvPVC8Wl-#$Ll7|&-?)Kz*Nxy;u}LHvOw3icUP4jYSi3wgDCnP zxpYK|mZJ1nZ+VYxUXhIaK>2D(c%cn~&h9tGuAAp!pxMNSm=vuzALQa^->*AZ&ff*a z!t9RHRs;L`vcw0s`MSgqH&cQZPCq8LHhQEK^Oqf{>bSVRnZx&G_jGb&1(9QS{TN0M zy{zpK23j*M?hC_kQoYCs@xKZfZ%38!qiHGJCG$wq#4LG}nplsMMLzUFYc-0@02f_D z936WS+uM)?nM=}ZOq9YTBdYHue7Gy4;`0k@ne%uuR!VZ;nY-R|sE?4PXI>(GR@0?& z&)7DyP#_EU_+l&{fNfJHLn3_)EJR5i*VL1Y-DS;=0gr<~`^lS%K;Y*s_PB1*{N z$(sQ^T8^R!u@)S2vi!Uqm*7^MYo8&h&kN_yT(gj2+#i2-E}`=4PS2BNo_=dtyu!Db@^(gs=MXq8tL7Q&dUruHa6&a6@%J>O z97qOoOPC~L%fGDJe%5S?C02PX$^Eb=qmRBkaaAeTGAiG_+M8oyRnSK$@Z{SqmC|t3 z{8CkJg=LnagiSeVN1?PvI<4X@P!w_9q#-t2#Woq!M2jm9pD7{PL zs25)2i=F>F`#u8AO5@**gKA_V3n??3K0!v>F#F8%gbg;DsU}wt!8=E=7`2Aln7ua0 zOo?BR1it9wYT&HSjmyMA^W7>Q&X7md6vm*8mPk!&=A%)cV)3v^IaB+Txi*dC%_@Ly zeurCGl1K4@_RHz>2qAFQJhmn)MzW?a3l+A+XUz2Qe7FXyS*$@6ftH3j5Yfw4;*lu( zqaM*lo*F1LD%#q;QA!lb>oph=Kg{lXl?d^&r4hbev{q|kKKT%M7!Gm2{4Aq9sTah- zbLYjQg6be(C_-3Vjr-j6F+@4)a+eowW1*)lGnKJ2692|Ic>)KvY<7Xfdh<5$M}PF$hGRrilI;Zk2*$-Z}2aiTu zHV4cD<8TY>Beg`cBlxV^R=pVwTU%GDL=#noc@$G+&liC|@bo<^gJ2#(jo&AT=18Rm z`WcJ(lsCaWtTZj307~g_3YYK$nDQ6aLDB zXhk&a$v~u24H4;@xwq3zshWnUyFYJh-jNi=*Amh3xuJDVIy~Mwn+!K6XEqEKdkfs| z`$YhSJ9zf{9CMGo;E09e(oAM(qfJ}2eS&@{+y5nBi~P{hw+SYs(1;hHFB_tAN2Y?M z(WgvD)pk9$vlxM13i2#fse%otJWALZCrn_2GfWf8jr%?~CKi%p)vB0Wd$y}2Z6E;X z;Rv`VtJ$c8rR^R_%awgxPk>sGarNh*72IV=e!}{MAOh5G(2Q6FU zE{6N>0xVT7J#Isw00t!H++WRs;78;wd~Cky#S1Y}3xWlqhEvXJz0?-Pl0H2cYFwo{ zsF{zId0#spXS(Wj!lZ57A48`VF5y#xh;SRnBWE7bOWQr6K)v3b4qN{u99mL>ZAc7_ zxM;OYj`jI+B^p*DJmCgk4x(|}P&X}8>6StyTTil7Kp=XOUaPdsDH=HWt-Rls$++Q1 z+KR2#-3eF4aDO>c#!N*VyTr=R3P=p~#{W6uz6Xr1m`9M>Wr2u}lHyJzj>3brvhWGz zQbOCOfkEjcv+{AtE+4RiCLh= zR9BSa7FnBW_NV3kr-31TEr*GKqTFiH1_&_f+8pArL?Bw~56%0bc8W?2mWr3mN8oX$ zP=RMoG~KST51fOFv@gI;V{bER9)ymwIIq5clY9z)AJyW9NJ26`+baTxi^)6ZXVwE# zlW$lYfxFhGPZ2Ry6)gB}YXOg{7*F4Uy7tiLZ?+IYY4mm`Mf?mdJa^ynyC5UyyKb)R zrleg}>M_o5FMvl-7;l+XYl;xzRVZwBkyynVDoICO=JFn>F3Y4cW##(r1J*r?haUvB zaH=hCCe6v~qStQnku z)Q6Yl{D+@N-3bL9tO>mth7O2F;KmQjL6Fpf74!@6RpO*Z+k@nF!+nS@M&lRxbZP?9 zaWqH7she!I?hz?Mx$d;i;7++xPcI+ej(qmbnv761_=aWjN=%d2!a1$Q9F{a~I=R+l z5?lIn?PJfLyD+SE!|qn4I$dI3E^}n8j7Fz6mX2#VutyDl7G=6-PD-{260LKN%4OG= z^cm!0GG0L%$OjEfo5f{@vci8wMj&<#g)o+wH=1UYBJjf)oYFr&lUbA{`>iu4zCw@y z2>QigUd80D!9UXwc~KC&7lL{HS3t)A)0X94U|pY4se=7>!D?Z@%$>6nPlc&Bff1UZ zg1Jowh7%hMXABPxhv^fKwLH_-no$QKp8WgrqwvUxzn1g*t8$u@KGj!!(x#3U>yeKz zkegLta1tfk?5YbJ9|4v2*^?@Sj9mD1YOHm$r{2GAO9j$ zQQtj9hbTg=d!-*zbQ zd|zb8R1gBx&Sv7Wuwedz^Y~7HER0?QjNS|=4_)yChlkSh-Q!X$G?zyQ3q^>~U;IgO z+9SOkUkuyRr73pBjq7|FU|m}7zhODdAd<`gh;4=d#!gB&Ov(uc=G>QY3=BpZx$ceg z-lf+r^VD4OjKDXNf)?jdrVMQHuOGm@1r+%IozR3cC*;Fmikf#Rso*>@z8K}-vo_Q6 zL$3i=hPNOj0zb+7d8l;nRh1_HXNy7mq6CJ_;Oz7@7y^D60z}OX)}|v?1XDt_ArnE5 zCfOfeA_#yqsr^+&5s8D3856Ipz}nrwT&7K^UVO_gm4$YosX`z(EeYxRrP@$+pRYXhmqv0S+qAgYXnXDjeaHX>t(g=Vzn z(|Rq~gxxR6Bgvyu0~ACN55e**lgVGpb$PoNrVH|Tj4xsMA$LAuaeu?c5)=eiiw&xd z|I2&7f=TiHPhYS~fq+&^w@MH9SzSiXFMQ~@&zVR%02~iM$~hll+W{&Iri%GtMHwU_ z`=ept)E<58Pz$!47ejWkH*(G~mz0rdV3-DiR2E%_HYoIZ#e zM4m|iGiEAv&nAlcA8w~E1QfUd%xj;c1o=+|{aMO4+aNC$i08o=kT*MU^#in ze!xc0;TRTDdUHJSLe>d0%zeB9ULkfcPX`J$sXARLLhkj06feURswRD0xa?WwoT26Fc#zdzQ;&s&w#GxM@XZG;KQ%5t!O47 zb`e5M9;rGhFa}Nlx%(^!e3EY5ubP8DA~&7D zM-f<5f2IZ4GlB4~0l9DZ@CG^L4R*{EUV`4p7rxwxfz6=wkJ1;sD-Bt7PLDoP!I-PV z)6(Xy6nF{}B6wAA{&p@EC=nEUyxVZp7G0C8-i0rifR36F1>QeK?KJ_49G08}JX;xV zrGR!Id(J;qBvuI)2f0`-#w{TEx~d;k#$*`Syy#k1(J&Agsj%1AYxAH|?H0gY=)tjpKRKtLj~PFXtQRJN+?V z9G}S%v(N<1!P%Nde!&aBfW(LpvVN{)c&8RPIzV@V63B!oi2mu0KP8GDx`Tss=S*-O zbVnZPPLoI~=b!Ev&)h`m5Og)ANPxuCKZk_-|K&LW3=zWID23KOGc1}Im}##>NvKEm z%tK5G4vG|ZA`X1KbLNdCw6PAl($cQ;6f~tck#PpS^WUediMUnD&re$2n*gq)dHdEV zBZCU(#`88#Dm|(7=q#l^Ut>e-#Lrca?lO`pHdiSby}eEITAXU=Gfhp36o-QNtMRN* zNycMWoyp^Z(y_yY`tF8;o%LhKzg9n#M3lo~%kAnXv zyR{3ukdja^!6$}G57up)IQ&&+R1+6t0Qr>Ym_1OAKMd)O@ku3611{j?NO_cZxBpH?rv>Xb}YV2D=O88ibf(hN{ zo7EN_uKa>$XVQ7_+~Lyg0yl@EtLh>}2#TVMRm?c`Cz~vWZ|Or%`CAaa`2GEf3}YH0 ztWfaa(aH%fahE;W31ED%Iis=P2qx%+G^Iv#h!LQ_cHqiCrm)4ZdPIVcPMzX!hDS%l zotY`<_vRJAr10RGz3m}L>dDhck?GUa3QDbAo0up&*x2-cf5MyTD*^o{x+<8UY_)03 zhuJ0qLr2%6c>!Vye(2%S4I4-Rrqm(xG}k!*^90IsT}|yb2!ts(q0v3;s5*Fn3VbXu z8a@kE!xCq#h>d2jz@K6ug9nVr*-&CBvB0N+tfbe%Dc}`#F|lTsVPOo&z^m3A&aA=M zF2m&J2jCyVP7{Jg;?REi1y9w3?-GXNV8D{_N7_Xbk^t?>c?vJ{muA6x^`BqBp@p%H zplF3*7N>@3R!8~5;Uz{2FaE|Z^bkMiA^fVXosq>$Z~)efve~jR4<=I-43(auS>SnrZ%8P3 z#wyB#1|I{HDFL3>bX96=lO9P>^3TD7_g3za0sql`H^lnENk= zi)r4(#SG)4{lw3E3a046EHd3M=PYQzoH_DRnu_6FH^qRhg$ax48s?14UD$A;y-nS@ zlLknU!;{;NG%H_8xy@g|6$9Bj>k|)5Z`t?LSOSzg2A8Rge@0ReaLGqV-cQJ401Pmp zkZkHq?db^d(

D3L53jYZ?tM)j-`IMY0Kqpi4by5%~qM`JJWg3X37q^Adz3J3(3H zk+C!ZR`dS(T&!u;9Y)w*iEgAvVK(K z02J#Lr(`1Ak4(Hz`CCo^+^ij(gnIz*Kl|XJ)ZW)J0-vKtv(G0!I{GP$5DOK@An}E( zQT)u)b;Ef|(oAwQ9_|o~X`VjX-_nZlEgdiQ;b~7K;({e|01X@nCVK*g4I8ml&lIqb zJiRYWGldH+?{BysBd0e&>eI#3?QDUO)8nbr>J=8Pn*Jbxlb6nbn@%iPxF3$~LTL09 zB14YP)JvedK^nf8yBn6z0~ChNRmj`sc&-3WTmFXJZTMh&DV#7GQ1ka%P>9L3Jh1mF zBS9pGWNC+A_P`2($lpDE-y}Y?_y`s;YHfo6UB0Eu3X7KqR`v8HUW8%X! zu0qv*AjrPPIAgi1c8+b zplppc`QU1e0}(Ex9%`G`uL&GYUwH@Z+s)TGZo|`fyUWu~({*I$yuGLkLP`$gT>Ums z(d8Cx%#fiUH!s;|$H{CBNlu%GyO0=C;y*?4V->;u0?tqfJ6`uZECGQ)O!H^H{c1qs z>0IZ>Z=tGep$a%$uXf*AxVB&BTcn8AV4)yjnVEKGKvhzArE!|L2lM17UU7z1)Pz`2 z^p_hjBXB8H0xb=JHyGw?fv}2<7y_)*piFv9dURbjYZ5KbI9Iu%jC;}O;~Ao_H~g*= ziB~3{Y-isGp1~gP@tb*6-;pNp->preF~fhv`xD=)0#*U{vXvdA;;WKO3DHZU3uHW& zDPu?7?k60y^PF|IK#XI@&fm74k429Ht2bu!2xg{=_cKigdnA$}1QDq?lkDZ3>p4P)Lo%G={^BPFOt65b3P6ynFerJp;J#IsiY6rJv zjX)b$F3Y>Nu&8C7C#=&{7^BH?k!R)fqawl+jr^@ZMijt`++Psitdl0r-}3Mk`SIct znV;g{bKg~O^Tfk7Ug0kq@c2;tLr7k|W>Ocq4Xbt)Zo>RdGjdh-VW_|8i~g%-P+x0h zN5jB{sfyfkdKdLonB{lz%U^nx{G*dOJT?y+4^oL#25RpykqtbM<=k*cAE7J93;?pk4 zhufp*7`JE1PlwHhuHg0d-}_{1We-SO&7{h5%MMUYd8&7S>QXb?aC*EEuOH$6nq7i9 zw+^;`ZfXvxdEQE1&XWNsl_cNOu76MSPKhRyGedcyPg9pf`(>)}fE1A*)70VJA>JQi3_r3B-K2jzPn=Pq0Br0oUX7YlPTENtQyz(>oIRyRIL znCtS7gRwh?v0H8pOox>QenbN8Jw|VEF0Ucp4t_%#`9rwAurEZ6Xh&V)N`^~YvF;~@ zro{+F8S!@%OWmoH8tU&AMe|XGn$_RxgZ+{FB={JA*t_fq*S8eXw5!(<3G_$(gHbR^&DTLCK4+-r#g*X8>8H|a>#_s;FD zLzLYtN}ZF6XJfU~W|)r;X0lu)r1En1l2m%6Sba-v zsj6L333H$Q)R6&!{kGg4M13d1Lx)A&{fAr=;`iUt5yAD8gHs!EhF0!Y12Z2faJG58 z)X>ws=f>5kNj32EGu+)%mA7m)N1Gs^Qr~1h>4-k<^&@`w`34c1P?=D=!6@n8j;3NS z%`Q?qGd1U=ygz9~`Z{;oUEXRpal@sR(?X++`DyWofvqu`FdA1Snq5wvQQ#pJ_gamT z6QR#txKwO4``Nhkcr&_@6T7-b;?a!M*5`dHaR&Qd-F)?HkJA)Ig_Q@WS)w=LR+Oi( zd=kD#EOsD8=c`V)O$g4P9#S%O(VRcVxODQ8rNVN`b+0;jX{&Xg&zD1%s!_FB&8;p6 z4`^z#;7|&11m##_oLr%3R(t7(O7My1pn5Hc_1AR@vwPE#EHK5X^o>NEZbzz3iDo1% z;G9vp1YTl4zC@(J$*3zGo7Aq*_H>)EG!2}=i~bn-bj7dH@uwzEt zRWDi1)9S#c)IG;yiO=G`->#z|d}Clsg3ZXfMHQ-xiUG=S21c~JRaT_5`|*lS_VS{%Z5QXS2cTPGx;k{Rc?2b9J?g>J zj(s%}_CE%g0vwOl)JSn8DvT5_b9pC}5E;y+>Ltka*EyA(Or)D;1BC>VnAgUWz?U)i zawJ~@zsV4(xzF8;_IG+{+~gu1QgmC&?J_%Fc>i)3p?-sr80fA{EM%$0^!ace zDdg)Md@Isv>`%j7me10*kGpva`cV$bG?-jEaiqE6&HD|)DrvJ`|?em97m8O4?G#E-)>k*ahN%Z37=-J@&IjB$&p`5JFi$z3bj#S-+VLwPUT43^{_?_-5tbsc=1nA|j#6+QldTpJ+O;^m9_;gPBiBu|8Mn z=tw6Q*KG$?pGV`@Un?0VRw9?L3uZl%CM>9LIc|h)+|(uj0{bAr z$mig6aELMCBQ3)XZ@`|e`G8l#a9z@x@M{`)71fln^08?F#s*SwbltJcx=2BpIsK}E zBy5UxI%85Af)5!JhKyYz;RNv*vJmJjWLd5WL`)AHJFsuX9ZlY6nzS7u@bth$0 zNQ_W?tHO9VQ`@zu6j95B(&r;xb0HTNrWDG81~CfbNO3=w&h+_!=7fwe@|xVWeg9I3 zFnOo5pVQ*dUQ&*r-bpz;-f-VgxnoDxxNilQpI22cqE zf{}sJ4zUt=m~4sQ=xK=BKt;BBqtk9Kw+4i+;|o$?R{swM!-)`^Qo4@yR|cgIT<2%^ z371xNnW;UR)uGk+cectx#Nw~|@@2k?hC_&MA>bd+<+N4Ul)Sp&d?UgQZ|BE1j0?-2 zSp%G<1CjzV!VI*<+f#6h)gs+WuteVWL6~%y(Q?aRoukc`O?u`~em5E~TKQ&J6kny& z-~ds;Yiko9#aj@B;YRCBadfug!&$9Qx|^%*qn;wTHO^S+mc;3Psi=h#kdw<#x?l?R z>A_@nz2*mJ;Td2kp0sZ*VUGC_zd=OA=OMQKRkUN$>t$n1cE8dxCKqM`@az3PSgQl- zB?@~$=pz|?vIX1G!y$GA58;FV=OcJT0>G6D!^e(R>MnL0##WVPHC_fAy#E9%)4)J8 zF(q;W2KC$j%*@-dlwo*(3*EuaK0Z2gI<8#}Op{#NWiU-x@Jz~W9#E(ky9OWAzCFGN zZz4b(jqtXd{L+=px4)L%Y>qU-hVKmr_tVE<|3RV$@Q zc!?wR>=|hFKQLkaCbm%~J(0ASBz6G6lLamWESN$7J|gC>O-m>NUZn{?vItV`sN#(@}kmkdhg;EpdY$#+NhgKuzrMoPy)#Ulv` z&(^=A0!jZ=5dqcS9(1vOI0%_s1RF}M*#??!AHkxWk7kEUB``s)F~NZPlRZq267*M& zj|)}>o~{RleY429%tU#NpSlxP>sfHzwLklF|WXt9A<m4l8u_yv%X#{B5 z;Uj~KgAqO=Z*sK%G(7mP*JlL7m1^+zB`os-_^8%fd=8ND2$=ewW&(Q@MhhWY$8A5; zZn^*K4!h}OUA1%qPw!>kA0ho*U8 zYF(__E^*p1i3fS>V~7av?VKzRzKh>k9q9wrj&ELXOh6xH5SqyC3`gcFRHB!IZw0RTt`7CS-1Wh4K zN;&f>yn}BkBmBiki2)$6=k}&*>{bxqfkV2?{qqfUhWZXLwQDCek*X@YjmZh5s)2jE zSGA|<1`>$jfp=t40_bLI5YG%&`~wGbllaLpfK>f>rGMyrxrqh@$Uj^cK(p#_H>B<% zaOG#}0qoQ~)`ha`7;{MKfZM7Nvd$52p~d<}ydZIne-)-t>ieDT5+u4`18`v%Se>;* z^oM&J8{Q+J#A}LZRK0`v5YKECZr!B*-V%!$-yGLG5{ph+qWew`)O9T=RL&w|R_?|v zfV-)OAvP}nJp&*#3{_9+9xzu|z`k>s6l#ujD!B#pmcbGONoQ~PA1Uj34YVrc#U+{(JmydVc&3v_fdDB89M7^xrY0NU~%W^EM2 zi2htHJhsph`)C}h&bXYSjN|F%-9g+%%k|bC6pf6fzPW9J5OV5)@>v0HLSEb*P%JF5 zUmN9Jng=)3%7nq7a1KVI6egm_4Zx(DmNRhiUFb?`fBrLQE{@Dy=~se6&^E#_tR}I` zf9%d1p_ou;YzLXVFvZe8F_xQyH<1p&sM1^UuCR?OplqZcOPkOX8dC|C2p=Y0wxq{E z9ac-0UAk_nSXcNE)F@1R@7;2NZ;zE3_pJiJWw({M|MfRiH)oLwKRWi|?2cVVwZv}Z z&ad?EGNtZ;zIK&kXAqDmouS&&B)_YH2@bmaFp)kftY*l95mUR|O_chR7N85YOcP4* z3BN<-BPjN_i9$7sVYl)(__L}%x^0Fh>tEW1k`?QcRRrF`z3}_F{ircL_YL=h$H3C9 z%O!~+(T&5EZe4!3Tpy(k;)Z&N!cv~w!HMQ0fz2<}gZZkgV-N(c0U5b%4u4q*0eOAs zW9A=IoCOgGVdz=xCMW*(7y>OkspG8 zec3md^0f3Aq)CX*XdHwi^dq2=sJvj+(5hBAQveBsU8EXfu{T}1zWo~_$gv78OAcjM zSsS`g6r!Es9AcWMwf64-pc1z4mw#fw@A5T_*4zr?Sh~`7NdXZrDj76*QUwDuJs{OYBRykcI86VZJ~o?oRyV(00U8t*cVh;QpG`1#?PNDDf*+3! z?w;gJFAQ7@aB?B4eo$#L4;#yD`9@Z1Fu?1Tzh+xCKp40{HlS|En_Iwd{A#eg()RI#^`AP;h~K;S)<-@z?z zMj)T{g%ws z=6b7GKh9^n7&7`fz70}8`5krNn6PO4U}1}oP!L1bi~?HY^%HTQLIg1)sk&&S`VZfP z-NcCF(b;kXInORAO*V~t*%)I0VUhU>+Z0&=NKUmpbPcpIo_+=AcHC$g+;_>RmNlIg zu?(r<{Gf+@5?IZUD<`IREd|1taib z0YNz#Pk!0kZ?~~%f}wGayp+HbNlAizu!GsE3+JUCwl`B(-969`ihB-LP?5W!Frq$) zFkS=$Ql%1zll>>?{KthpgJFz$e?aTwG^-v5OIIgaDnHcvTZPfDDwL zRn4&$3KdCcz_=>7%*TgSXNeG*h8LLE^KOti-M!?Qha8u2N_}!zN{?U`=N4j!g;GYb znpwIYT42V}e}uUR;e?)L7 zJ)%M0&lpQ_8B6Id^1%71sDW^gz`p;wnesxdM1LXbt4;m48e_RpIT zjM6>$&4)MFVfK)K#AM;qBoobtS6t&6)Ll)Tw?c&}z9L6AjdH@i3uVtkY;E`=4RBHR zUWQfUlR;Y%%7l4Bj;GZy&Eu)=-nQYAid@MM z6=fbWM489PJZ&=>Qxdi@vkXbe7?pXJu+3yHLW#%}+Z2*AmAMEZyyv3px_-~+z2E0^ z|MA?<=l!o?fA{{bwa&HHIUL7vihY3dW6Kf-3*KvtbaH)j+mI#?AS|N;U@}<@L^;5f zJq#m1-TMN8wN&8n;eQVz3WELMR_#2#jJSU2#~H}}f=`{#WzvM5!UM}%O;kb;p-{MV z7KHQ9DWNot@SO1B?=ZMjLU?0o^rzR^j}F1VV&RWi>0bzJw)cIl5asAwel{TmV><)i zJpIAh62cS|GJ8IE(fj*j44+{w_ZAEN>8=LjQFs|V>>~V~@8o-LhK7~^qiKKj;xr6F z488(iGT9C)*!I}4$= zB;66_F@WZl!Xr1V-lTG}B4cX&UyKQ^CLb$gJY+|9&95O?1gvTfesys0SvK(2uO1ct z5nOiDMR9^miEl69XBj=!2=gBxgBCOWJd-+%S{x?8U1%;>8`km-M&du05|V@OJPwsW zh?ov%-N`}v+9DI4YZ!l=| zIT^cwb;)PCk0T?)(*9TPYiD{ou^*pSEYW;vK)gS+M%9U^NM7~3G(%(5)|4Nfe{Q74 zdvT$bU%d4F`j)eYJG9RFdd7XxR|KC<#=;DjAY3~1g^Y4TVV!r zWBPDDN1Y;oZ2V388|N)8WvGRYJ5fLV3_l$FNIG&9$}dC*l(pdTHxx=sE-$a8_c#$v z06K+R@pUFwH*BzM4qa-O2vJ8q~(W1Z_4ndtf zyKkyJJ>Fgz`A^=>=@WAU7S+K*NsHc#% zNcppXOoY|CU)?uG*lzGwspj#;GlDRMPN?vWiC==9R}FcazN3OvuwHH`p+ae$2bi)y z2NM+>8k~2&0G9ZXBr?UO2h@OAI|ol!?BaL-eDLr0z6eGq44@x^U6=f_qt@{v;O? zntwj0i~PcUfjzJgtMI0!C-{vZdshy}WIrE$(Lb8N{?FkU9Ai^kPhfuK-j_gP_~B}! zVenmObHE@&pbW&yPNu(DJ{&H>B!zzJ}4kPyB$tx=3t)(gJi58G?#_4B(J*fwkx`X=ymb+$poG{z48G z2oF(TW8eUosI>w7Y=khks~D;OX_)`Xv+=I1G)SQS&4mzcNN{Ops}oJ6?b+~eDYKuT z2^|c7O&kBQ7C{5<(|-);gzXd<{>lGY!~Z|M5U^k)uwXNRiJ>qWF=T?o0)2n>HvG#G zupFvaHxWOM4yJU~TpFkE8kfuwc+c zUJ$`ab+Fw*(#ba^p9=R3c5yXig zB4z=V7Ll;?kB%P%ATl@FOSIHtXM&_7U>wfJ)v76u!r74VC0>pLqSqW9cETkMhIfmA zrrl2C^@2OR5&ruA$1iWiqcHf7Fg_ z(FC7sgGbKoLh%25VK9Vs+zt)VX`%#nms9>PoYVY@a0s>E=e7eOS2;wyAY?*>ojvFjOXxNeAice3!)-gu^2cB|{X(`y z=eL-%9}bs)2jo*?`9E*?|K&o6y|n?voy7*!59?3brd$8BE8$uS3*(-I!`2P5EBWHi zoN#aM)_hLgRBJ-D5Yz7&Af8@@`ZPjECtKOTze<3fyo*0X`8%}=Vf_-`m*4LK zzP`Xby9RK4ARmk*wcBs6)rZY3Z}uRa4Op+{=lR+2 z=HOpgGs7g)zIH+}&_%GBpG^VOaDU8$t=iJpvnyN9{_3 zB5ZFu;UJQIDfkTq_ihjx=QIY9yF}A*!ri-?1z~V}{<21`OFXxDih4<0 z@t&u$1ZW#IiHrMv!u9k>PCY2HeS7_9x#!&oC`;-`;gQB9!@ zu}*6Mwuu^rxh%3e@l=0f62|B>s~`=M6hOEvJpUQAqy}1{EM7E{dR(2TM@Ay4VKS=j zfKGDUJoGIU0-b7)=>SUgXQ0^hLd3YBIk8$Cnl03b3AAdyzfYJyQ~{W*VbFG(EW!$m zXepkdvNb-t*Av$U*;vXc4_Ig5sfegzuUdmM-DL^OER>PI8J=iU1s(SSS1v-XwG!4j zXmRVc(6HTjW5}oXAaA`2*}ipE_2?Ans)h3wf#A6uLpe$)dTpP(1rDK415vIqkD&7pWN9qw^&k;4E{ho zq`HMp-_jqdMxBV>Mb{oUCedYo{5t?PhCoHj7S#8hr_U|a;=W^jivWQbIKUc@dN8eB z!7D6fyEeusoX1WiHX35JZ-E|Im3U!p=9PBDa&|b98=!~fq>@6lOXKTm4Fx6u5sd@6 z72irI?gYg_`#o9qM3%#CS$g`0O?2LK7ucIvc3N36-XC3iF!|J0O%0`*@)7R$a}k~A0`%j2vwnlV2|H$cMd+Ysv95J=$+6pu2fJ2p*r6G?;!r3oC= zVg6NajHAa+72k528jbVrBN7jv)4@mUhMP0X@A_Wfyk$e+%&7%0at4#Up{)UTYHv`E z`2lh(s_vU`3tAuEv!{6?lN84k5SdfUD30y~O)pJ0y`Bf=>lxS>q$Pb6I?9c3o?=eO zz&=u;Da_PTQE3_j)oKAFWu_UZC%J@;Y5@9wvS%yOlxFKEK-GppB`Wji{Q z{~hD}@7(E!fTv7}GKS`Vfi)VlmC6bnNR(5ZX8IRYv`PWDuO8%3ojapPL0h<-^%IJ9 zfq0b$bsa&yR3Y6*ukd1KDl_%;zL)1|tg4GWtyC~Cj+5}V_r@XEBu?wL3SjkXw?ILm z5}J<<*2aB|n$W?^Y+XE5y@>QIpsU14Ofg5M3WW_4yeR}UwGc?63+JipNc!4flBYk7 zow>+LSuxJZ;f4H3#Zg)nxO8VA5Y~4b>y7h5mUdB&iZ)mzKR6K4H^spq!9%uH{5<1h z3%Y0{i57YKJxc;^r9((nAgDo!scDg6dU4=eAxObC)bRMKIL*<=?LYC1B&8Np(3+0p7OIvHvMiL0W zx~z@N#aX2qmvm zSB}^jE@fRCm9rdl1Q&}^JtQhLIl%?E40ppzh1lB_2@vbsE5!W@4=!tzl|I1mNOjLQ zFI`qTSJA7jeloX-Ml`EwDIb&g$rCiZUd&Cp>-XlI4}Y+v%=`*a3tbkC334%VQ5r_P zs>}n?Ab>{-J?_F*l4x5aVYk{UYy3or<8();5l9-pH zbv;ommm^xmh!$g;NyYvycC6bh&1WgZ_Sk_9J#5j z5*>1}aZSoh_Mm0=CAYY4qZMPS>y6k&_rL0`LTkBsoS7GgK6fX}_upPtO<_NS4n~XS z=qJYv>*+Ahved!|l$i&6Phe_#MeAvEi`IAg-Aob%j4&DR1SEBoXWwv;Heo@L5VtZd zF|6%`a~l$%>U~Jqdt4(#JDi%ON%d88ziNl4ZH6lZ9FE||+zLCpf@V>dW%*0afSgQxD7d0d5SYnHl-c_Aq0EqH-| z@5e+Fiv3!_{NXG;Oq)$(1%5l*Lhf@0m(zB%Ued+kvTRI|8~HliXPIw|*Ka*6>FH3m z2K|5^=7b(6+$z2?WL#JN@&n3_LvP~3eE)?)oJP;F`cr3bnIQ55~axipQIgMSlf{+u4!T=eNRsrx1R<%;~tHEqwO1H-Yz) zD6HiMfxut`OccN8@wM*rymf-l@E`Bj!?2%x8OVWkmJOq!0AdNjKDc@ zDo0>b1j2tu-oZ!9@YdovROK8WaQ zFXZ3kApb^vK{FHzs1&FgEBqdC>^? zK`ZKP;VF8=iH(s3ds#xS=V8E7#@B`;_YT{E#W;5x3~!c>x@Z$%#pFCp7J_hDDss05v#_p#-e4*50Es|>Q|Gb-hkDtG zSy|ulhv(pPxNA4GtZ2gBz)#zFs+Q4Tp8E>G;BQ=Jl7#dhec-$zu69}w->xyzd$b~y zzYxP#r$TX`BJd(sQf#Hq(dhGgE`^z5>Yu#oGf@WC82&Wm^$UyLY7*c24hjNaCga->c+|F*hU*3kKN!>1BGCxk9rRk`mslaLSCsT zH(+#3V=j-W*$!Z@6Q%tQ=HE%(tG2bU!5a&WE975i`Mjf1WTwjUHt(#MIUHEQ{@PIC z`n>0zj58LT@K+ftxU%v!ovzW)D>9~WOkx%UxG=FU=DaU)WH zfj{ZqDRyQvdq4+Cr3RO+@;}HAK-u&eN|cpYK`APFTbouQR34zn;p>n~j`45cfG-`{ zs}EF&`yAIqRuc2Xs3lOucE5T<9Axk{p3mw6_e>5L+0^^xLR(J2^JTOeP}sVz*v1H= z#=FUB{BMhieEXrJOAsG;q|g9=@hJU839~Z=vX14rgwaBsKI}chopr6CncSxb2`-Xy zH{l6bc)(z(;Dq9Q`b8!K^%JgnteAX5*uDHT0hVpAme8Sb$R74+sSHX?7My>ID*PxF z-)i?9{{$tl9X^v=4ThgN7bKmympW2ij=D4q>865RAAswDH9%@LQ5lhswg`^LQz*A; z@X8u$UR|6p*!6*XPY!nPET`=ltSyY#L&qA+*BhMij}PvZ1pY02 zw!^pFxESCH-+2~cvG+F&ClaCNmqIqZXK5w9%6DU!Ptbp5=m=NARn%y;5R_^KW8(VS z&7gChJ!Cxegm*xpw*tJ?r!&9%EOr0VH@rb*3@)YpM9bs8PyH;TIvSEU5D8b@16;`= zI5PF%)=6)WsHevaG(M?xzlD|eOrQm7nVD&@ueL5cy$_Ra596%(5h_-t zfIV~b)qZu?WPsghouH5P0AkiKf<;2M26wKko$t$fCEWvB;ZA7JV2OGYaG3)b?-_)= z{Wy&@2Z*gJeF3WE&^w4AN2vF_asIn@UuZs(+abRAQm<+)l&k0&8QBuTT92~X?M6sQU5B0y&!g@js=HJNAQWItdy(V?^Z8O( z%xRP(akR4r7)}=gk5oE^%@jeqxu#fS2T@}rum!5Q!FeAnCs8|JcgW(1oGJymSizbr zZ6Uxc{{WoxMTLZ;Gy*NC+l?{Y$8VTiw?IO+d=$v+IF4kP&y1}p;dN zU-IS1ti9={D5M z616!7WTo8=#szS0+*kT84;SBF;R4y`_pMX!+LU=ay?{myN^wdB##$l#ZdVa(wEaT! z2QY=jv4WLZAAEF|JhfpXYS{xzvOEX+lB;nvtAhXS!V~rTWkCqlcCm;6pRIyEaBr!v zcwVbojSkRhjyPkQb=lM{Cg~zlORw%0OWeq*c#X6npJ^ezKHBaqjuZ+#+k%q7&1}H} zAJNv|pi(*rX)-LlPFWlKboyfrL~8zMbeU#l1vGjGVOeJ6g8QH0mC}uR01a;U{qyTp zSet=P(XXcg82*NnI^C@v$xu_0U}#MMhQl>g=z57_205!85X`MMOKo>I$})ewxbcm7 z7?L3)N;wc`^i_P_m@W2@ShJ{h*K=G>kxr{Q^D5KaW{mXz#sx7**y@+EkPHn*i`> zlCQuuPZAD8`(w2hi}J~=ZD~V{5by0C%uoIDAf0-htFS!M_^WDzJYJF6MZjLGhoyqY z0CZ(LijV1iybHXy0iG8V6dp4GtQV+XNNDXkn}*L-nyU{=_yM(N7iG&m#M$YyBcSI29r%O`<+Hyw zvik}t6(AVpcC_~y=(A}|tI=Z~)&trVS3%%crDi~6`9_`}>K;QGoEY)q6#>cfZmZiV z62w}C_KKx9*$2mdy+((xbOqW+kxXW9;A%LQ8R&mnN0Hp>f`66)k$}ivR;63Y$LRem zc)HG|lR&>Oqa9p;ir!$g$!b07!A1e5wiX?HZQQtSxuJB; zo(aT(5H_-fmgKnH6k7Z8$5HyIeJ-tA7M&-Awi|*QU%`^ynA6K!YAllapH>6RaSo*! zkwz(W2-*35Y|Dr!XA4r0wYCG0qcmtMv~QhlirmP2tP56CLi;$I+|j6N9ulxllQx;3 zptJXwf@w&Y!XigVjhSkJKF((=dqPNiDvkp5z zp82MrVZ)LsS1?hm@D}_<0R}7^Dll`UU5SLd1^(aEVn$|~5uzeJ?b@XR$GDngpz zy<_Sdy+Gk~mMJT%o4`mXJBIfo=uUi;*+`tq(Zz=IW0s?zofST#&8OZV!`v9~ubZon(E$Cy@mKe_6GetskCWoVbCMps)Yg3(z!`aw4IN^0o?E3-ouaJoixLTn+ zZ{*r7pJXwhq)06DPGz2ulaA&Vt~)5wD>DJXAJ`9p*{y<$Q%W!EMRoyPC+0&&&0|pA z3&7}+Su^~-N4v_$GZomd<#D*)bI(8DrQ*tE=@b}I-BB5c*nsnM-lKjqC28RnGyocM zYCiL|bro`X-3ZKVxe`NZMWK=CY=FUAGb>Yt-^Ev3(oAM{+d8rH#{L`)cK7T}ynb=r z^o(GWmhDR_c8vW?5R3dM^^PvfC}D;!{7AGXYWeuFSFK6asc*=9yWV(mc}(pG{@d%= zNZhwn(C6R@TTD}^;HacXYr;Zj3#T8wXZZ^~aPI9V3WC_uaYs+6q#eT)>TWfSwLjG5 z;;F)y5T>LB8YU}>+j34_oIcf4rm%7Ujvrlo-Z}|o1h{51C@Qs*M+m7&*peK_$U4Bs zvq8|xQ-=A%5+KUq5Sk|UFtYzEp6DE=ILehHRw|}3p;(Tj-gcxE#h4eje_daDzc^%| zo{Z(cgXk`!bak`GYi;NO}N@Snuo!ZUPrZ z02_M3%81?$VAWXXLL=NZn?1#o=@1I0#Id}``XGi;RX=y7RwOEO=?6A4y@7UADkhuU zG)vjGO4-1sx~Sh1@sLLNS=MpWky+EAH0qKh>*=KLRV%r;@>}II>IseGsM(9igHhZm z78?!t5EeuNy15T1WMsjhY{c!HStPCe2&yRkm?XrmX;5Lgvkjd)t4-NX|_tP++xgV@kN$J6*5P{uPbOX-vS$*$}1>MzVy{-_S8i^DD0SC{UqA zGN<~oH}Rzbo^2l-F%D6~kf8dB!HjdOPc|ABs`=gmdHp(CH#YLTI@>Bdo+EUg%4%{W zdqcG;0e8(t3^$S0f0YX}w4$3a0M=^cjbT13IM$^h$WxfMwu@}}<42A2z#`s9i}=?O z4EV}JfO+cY@zEP85X=*Z(bPE%t)QK%s0X0^#yoCNkHTu`-FM)oIE8(*&6cyL_#8!j zXP{xOsYY@mH?bm^AG6Y9WY@R)^}YSry6aZ|okD1|%J|$EcqjTRv=SN_Dp) z|H$j;PMPzq91@Gi?wX2^NWKo{_sb*W{;=42aEx2xvJcrD>2%Ryovnr*L?#i>UM*kM zSlc}QK*V|dfT&J9puMF==2j4kd>yDI_&K8rcGMpEvch?${K0dtJF^yy&W{xg3N}gm z9>LjE+uC2tqrF$rVj}vw+U-8%qRYtU5)Fwg^cdqKhBG^_&R>Y*jn5mL@Ue9?nn_zY zDin2d@{o?8sh0mY*Tuf7t1M<^WA`}n7pl!OjtAGD_p`mw?>i6&hQGTN{x@NfR|cNcC~Skm;0&62Gk1eU)|Xc{%6^XZWGgLSK_d~ zZ>*SiJvj`RsyV1eCB4mQ`Va?>_p)suV%2%AE!Yo{Yt1@g6=l;v9)JU>DyfN|>F3jP zfnDShwtnIv1^i*SRef#QjjA44V5JUm8Yw!Py_T43y$!#a6(j6QGqRSfd;CF1tN)yP z@ne_Uib|%NYxO(x)RhA*IWhW1%GsQ_9EIeH`_sTsG2Of?6+Q=Tsn1{dp0QQ#lGiUb zJ5UVrSG5YsxqMAKsOy8!4ej>_>w2#0ANC1C4PUz1Dv+)?bGWpORDx6?dd0m1(mf+rhIjlFsOvrVa3GX=tR2-Ns6J%?Fa%jpvOhc@bWHl;`uI0bXrPRidv zOu3{$FSzssU+8bJaXR6gZLj5FQ$o`w49+LM4?IS{kjS?F0?LC)zibOH)N}Zsuu2qL zpEcX5E~oAIWP7IidW7g(uowqU({Z)0cON_AQjFO(QthDagWLNDsq~nhKGFG?(8^m? zY%u$`7HgxSE(-^sOF=!Bpqun+h5(U8n)LUy5Zam4tyF<4$DStRa!Yr!wQ`VtzkfCx zo5!u4PR5;R2I{OQFVmZ0e)N<#T(!`oRC~=6f9;cW&U4$>Qv~-R-76|l%y-wV@Cbx% zHyBSA>Jw!~)KC}HPcBsn=`Djx#6g(xxnb$9+dyJ@`Z>qDk6rFx4QhI(HmXr;lXvw+ zZa|(DIHw+Ke*w0l&&D{h)u2rmqo+&KbM>g`z}u^`fiet-lgjx|c7v|_lNvfAX^#$# zK5ok{dTf)egmPr6^-hlvCb`8z{u|IhkU;O-jEREkROfN9iO!gbN|u3(tleJ6>6u_OkWSi-pZ6JXD~5vrwl6U`;ZV z&i?%%m+?AE_ZYkiO$-y^y89ZTMNjQH?T_z2zMIGLWQ4xL7Ann7l%9~0um^{_6m`HQ z1-hBRK_h@AI=N&5Fb1G!!DYWr>Bzn%n)nj7g0lbOo><#ea%P3-@H@9JkWz0~T)6($ z;MtQB>{squPV9-Zg=M?STflpr?_B4*ci;1kY2L^CN2F{z%+li%)tNhO8sq`v{zPSO zczJ(O48lR#PX2wYU06chKX&5l^F?}un|DW!;BLY$l+dAeMW4jI$=Nm<=);s9OUjAU zag_GTQq8AbJ^=7$qEcb))Cps|3sqN3TZ!E#oU?rsjP~6E1-FZ~O#?>Dm$`ES0Xs}{ z^)ci>TL@u_*c#-RA>%JZb{1mKV3C7~#rJm8juog1A`|wKtc7SlZ((_kV=m~nTLLAn z6v@?5o4R(hNy;X7Glm;wdzsAATaa^= zW_8l%?f^|PI`4_ciXRDHWnrX~J6P_DaQX6iLdLKc@rph*<{ks!;|c$Mpq(5h=s;wj zf3N@~{wIl{de?-v^FK&^6L0tF*4J~xM;Zs!xP!||A0B3h;fNt(54n&YTsLm4nuQ|E z;WB@S7c0nJ6n2`qCw+z}I<(9jTL}Q6=hF$`x_LCkGIUx#y_BryBljF-g8l{N=7{?n zH6=JQ{CSfc#I>(g(m6&rqx8fec?1ICFADocs@=*^#koIHK!=~)ivQkyk^FI4RXvy) zLjvUx{`v!^g9=U*Xr@5Kwo99gFma#%AxPobK9cU~m;n+c^^yBBE5cda+0d|@r>0=# zTlbs!)afM9v)_&CCq`p^_B_O!-rKbaY(IU1j(meo2d{RXCG%@eiR+tX6z3>>fcJlE<{w828 zlS~g>nZTI0{g3NOg*FX>;oI+ONY3=NAz~Q0x_dbKGC+Qgf}{@tvm5j#MnCh z{u~^pZaso}tMM1E?;@{&3=l zCaOPy!7yD)7W5Vn+3z2?2V^f`q!$%C>L19^l_+A1iNB=;l2W^y0MTTgw`e<9?!A4N z5rtLWbx3uVAwM46nR%QY&$Cwqm^=koV}B9bB?H)22)B)(wr?)F10wzfyQXro9#^{r zLu0}&rD{+0!Ghxi0WpO&COI4us3=>t2{0D>7(M+9TVr^TMtar+>voJ>L3{cbStdj*O0QJ&oo$n;glB`9Bc6d)K(mUJrX;v2xO>U*Gv7_*I!eo8&HhoNfv`%$xYZX;J*f+M;7tjYYo^&sc%J z`2-YzY#^tbA9HdY4vAhkB(j7A1c18nAE3vs^AMICrL++{;@=2!K_RPwaI(Gdk6M&j z690LY-H}GjGCUpQ&|&{Ml$4hW{0Y&(FZ%tBryddzmTEBM<%$ooUnn~a5DK}-+u-5- zZD0W31ym#Ax%uyqL;dWYIOp*r=ii-#g?US$;KHaGD6JbNAIDMMk-f{@s1UXL2taEA zq-JuDFwsGV;Sv}V;TIh@v0Y;l3$e)7gZs8IG*;#b1x)epmDPpHsW$9;+g-BAs$2xp zqf`J=fo;Iq9z`M+RR}1#{cJPUE@im)dc04AJDs z&cZb5r;=E~%YTE_czRt~S(Ne71ig~zrFa8j(6~*{HYbu(cw*!<#Z?gW_s{O8^8bsU z{a+$!a>nKE_*-s@U@tlCs{8NTnGOv>)mWOLiqMA2 z3VT8a>uBac&msmz5{p2vRsQs+>0~Y3Fte7jif@Kx7;K#S;Xbv;9$IUZyYuTUVXQf$ zrXs;gYF>i-g_zaVz8~FBI8im}1{hwe`4>>gBx8i91I}}B0`!z?`k|J$k&@4_>_czH zN2CN55a}w$$E@$m?3cB~s$U8K1Orv*3X zwJ9vlR4uF4Hbb2{|)*h@qdeM$7F+^o;&-@Sw|o4|Iy*$Gi`buIV(=Y{2Khb16w|Q3%SjY zkiL)hMD7WKa9PMEh?L;6_wR31siQwImbCZ_U-OpXbCco6hV~w{ns^oOO$o(a794zysVa~Sn}RNuqx?-0Fv|*gsP|hd=rq0 z!N8#8rg(5Xl8PWWra&&wex9Yye*CQ|nY9JM-pVseiT?y$aV5mB6dRM~Vt8Z)PfIAq z)d@1+F~A10TxJh;ENW1;V$eMC=MmGpa;zwCVuUvCe$qJDOiD13Os|y~4H;n~RWgVA z)yIZ~96lcOS73Xz`)-0uWwN8NUtD24S$veyi%MD>o z`0yWok|S)NZpN%gF7nM*zpXCmee9aPH|_Wz8Mo1%3Fc2lTk&rmS{@ODzBW}v3$3gV zv!c-eK?a3Y&nMruy3re4vwRlvvmS#|vqM2%`a%m`zV@w@$BgUA2aMvVE{A8?G#buB zv+Gm3o0xMS;hx@I7oai14v|2K+rJqwy5!OYnVhE|3kUvEh$t^Q$kb``CR`*x{BqXE zy*FbmN=A?bO69L>GdFs0CWIk8ftQ-P$2tp!X2+C_vXF? zg>=Ehg}nNoA-}d8n8Fk%8wVfI*HtYF8FXW6=BmeR{0qcJ3p9Uad7lw8SiG)LwJ?Xx0bYMJz6fRd#+>1-p<#1Gqk2wUI{v5C1rNAk_x9LG>dPto6+@fta1}B{dCFj zOtpJiF0Z7O)Le~f6yJ(v|7KcAL|Nz5!YRK`HW5RN%(@CGh1b))4aN(I6JkgIC8247LnDugM$Ox_ z!92B@UcMKE)3oS~ZKLq;4}0Zg(RY2hQ5~g=X$8#@{sM^?N7zjKvM|rbm}a|bgE}_8 zlQ^*Gefj!BXntOJE@P9EJ$>ap42igKBS6w$!J-&MCD@A?$?=LNd1c z>)Nm*$YvjKs2i;fJb3W1)bDGTlQtUo)z7+v%9C&_CSydbL-@m?kFFYLdtBfD)VNq) zBJM!*mvT~s2v^{T)(c|=RMI9uOJjf{Rij~}Gzakatzqc>e-X1Uslh$^_1hty@vRKz z`Ktr>0~L+v?pLTik$0k26HcDb9Y_j36CCU&CmWr0B?iSUyWQt{N21Ye^+L zz2+pi3*OQ{7_pQT-pJoM>T$?6b9HkYd>b-D*OVyRl-rVN;+KD4f4eq% zFk?2gs)!3Zp7L%7Z9Q9a9C7Ws9rx)o&YSvc-;ZBq>ZGkTmBgcX2}8fMU5DX@cHaL=>zgcyu@v{-`B$f1zp{A@`eEBP~-|E*o zb%aIDs!)$WH~!~x(`zs6;->=Ra} z)#a?O!vUYo);cOn@TP*BS3KRGDecf)qx}8)xfE#`$9u^eP3DEkBD8+191g-5d!8p1 zAB|LRpTJ*HBoq(bwyd%o^gANN7VkPx7+hv)!I}0q<_n@f^p2Ne_~0E8e9xTb^w(76 z*D|uz4MsR7kEvPI8T-js1sbN6@i>cH2&GeT-;vth2{gX8WHIYI5yF4stK;5#e$a55 zql)gko^6_>yE2}3tn}Tn5gq;Hj-~93!mo>~Hj(w`ot{ESo~VCvr>EWR)7_O%(XAo; zn0(!llta9B2NVv4ZN+7ByuWxOivDGDbzw!S+fiJ>w(CZT(6kxznPW~^*mt9E9qQ9D zJ9CtchLq>e)nEFD82=Rf$*{z;lMpRs!CYs_d8R>2krkQ+$wpCmsnYpyQZZ#NS%X74!Xt;Od_vGF0FG<-Vsz1HmnUtHY9y01*`^{x_#ddda z&yArKbZzOd7_Cv7i$Xg$?=R7nCad|Nt0z?FbqyUF6Z)NJ{YLEjonF2Ao|?uh`ku|? zcDkFXi{$L4UtHi*dO^M#HbtPu6wg zzq`D;p_;V4p54#;;YC@ScH2;|@RbVsKzk{d6P>h+sV_Jhb{xL@b`7i87O$6<{Z3pJ zY{Y-6<8Dr^1Z3Dc_~UJgr>74R;!$OG>RVT33xs4$iivpyA&z+sS}xb0QdkMON(o(^&LXvT|S;uvq z)Y~7~q;6h*Mq({Rd$UKf-|6!l`TV!>Okk6*(mSg)H3wmIzU3Kh|L zp_A&#@`3O(S&tHLoS1w0LXzX`(DEBP@n|Y5-_E)gHWT;sC_R41Bdh%J-gL9Rzfbkw zxUct{V@+%;>ext8JbxYi>?hj;2P*WGne2X>sXH^j{}}RFv3Y-)5fGj{oZH1bRPNGGRh5=mwLns!aWSDD@Xk^ET~VtmyN1!6jp2Zx8keXG5g5vNO66OG72R z(lJmlZdYa;*A&Y)i_a3-yl&`w6eSTht5$?2hseK4Y+Jdq>Q?g3X3Cjyk zrrx07Lkh|a^X9CrbwcKb*zjU%i!{$IDhHJj?!!AT$Yo{Ij2R(IWc03cl888SH{vG` zybE+~0FR@rIp)K22#Ir$Ho+kSi;j!_<)JZiQ5eNzcxp4=cAUg==7HXHN2#QhPIekg z77ek~P`h#hr&yC@JP!_b*#whf2MT*FVZMK`z{LM8>Y*dgU~kTJ=4!H>*f zf_}e^O})f$4r~A$LO7p8;-A^1Vk$QHDu#v1EWq%9l&h=cGU47>{ereW!i%tkxDbHz+2`XBR~2eQo6QLCE6ZWFhksU23~We zgn=DFr1t{#|J~@GqtRVrr6O)b!;_yd`EUN72^f%&w*D}&+!?b?2bUx;A)(0xcGC>8 zp9|L|JqFvn=1Q~OpvqBacVw^0khNS0rP1?IZQ~sdrcvKm?Yl{0h_nzgLE?7=r4hLO%;>J+WQHO4qfQ7Ad>#aEC=+S@)Rv^**z-^Xugh0{J*Gxz^!6?W^1SF}sUEeUToOYQ zEEESRDQKdqgM~U%C{=4ADJ-|-rR03cv38eByCp1!P59X0LqJsMOiD;d{VWYjzc z>+Pk9i|9pJAGYHmf-Fazrf45ZFSB^w|F?n&KO+r~OCWup|436LXK|hm5nxksni9XW z_r!l_&PYFrhhK?51QQ~C8Xk7i%xeFWI)4ErlyfJpWl|nhU^q5b@LEQ}11*LC%G%w% zCkziDwZ99}SuH5ytjZo)8-v8jno4@;MHF$#&2I@Kt?-pU7BPD zVXzfH=-%icG#b4vG@GJf5R@vJ4(ZoSlx1V}e{u><@(_HiU*YCQNT=!|_NE{w-xT6> z9MfvQLjO{F_T^_Lp|;n4omUQnCWu5XM9D~|c2^355O^=v(6rXsKkU!T zYERC)g}QyiSNuAT1`I7Kt%wibnL)!UNBUDK`(8i}&l(9d>d9HOAlcv4r^;r#Cm-P6 z3%35^*u4%T;X!45wgZvCA^6Dk#)K=|AiMEE5qR08q%!H6`wk(rdD}gaahLGJP&|h$ zVQ`s3#6{mAXlRRxhU@Fht2r=bzTz9WYoy=5f%L+DGsL{iB=bG|*wX z_QAg&C9|j|@(%!)jDmDu-vDgEg1VQvN0FcMPt6AQ!hgs!StGo_<|%ENK$sN1b+y73vyyq|wwDGBJTt(msD(I3lvd zBm_taULZeF{^YdW9}F>=`#%4-D4F{}q(>F@d_6q29)3>pp0p8s8=iuA2qc_AW$LQB z8!J-(n&&!xB-epG9Dt59varVBUnJ4CX%We(A4Sp(@g!t%P=>_;H=M&muy%g}@s;)HSG-Cx9#?*KjHBooSip-Wc zeF8*aMoQUGf{X8G!}Aa|0!50N>&g+xSOR2;pYC2DrJtII%P1m;8N zXTriSp!6TBnaRta1Eg#jKn6c~tK&XZTv1|DV_s6so0l& zi3bWaEM2aY1bVQ+^@v05av7d<-$39IRIH~O<_GPQ4}t?G*8zBM+9Y9Ot@0TV))lB* zPKv~GJHI`sbsyoE03Ls3r`T-3qOsVcS1$qJYX=m42M%M-B21#dJC+u@TN95h=M6!( zJ0;$9SF}WR9B?Z%88S%>k5F3Irzz@FIb5-0c1siQpm$uAzSrx?RrHcUbZasUerWl_ zu&LhVfV&oWY)K*5##{dqsU z_5ccDBTrfl5K;9Gu%?G{C7h32C-L`ah%)t(CBMHqusTq`-4|7|oNMLg)6svMno&Pgb%oZvE7`{7>D#EY$4Q zBo)XpOg1corb-2(<9_iDDMhqQIWse50?XGW<4gHMq5{_$r$%z5XCBbB1+c>k#HgsfEXkhY8aV>o9(O3cS62-69>U1R z{N}<$>L3mIE$AXw2?h6oKKAYmza;z0gmW*yH)K`C(A>WU)ERF37{3adcf zsjf<#5rq~v>VqNkgJ}9UV?ne>LENU zdh?tMi)PzPao-KeGO#^3(W1y}wccx$fYBIf(x+AWQ#=n#U}R>`wZLx}N~dIj>&7+$o+O-}4iH(RU+$S0hFA zUyT&}j@?T@NjKc)e5f#lG=0=bV{5Dcbphxw?{?)2@9LeEdrk_Exc1~VC5$l|X6Z^G z{Jp?ad*}qwk@xsz4cLUK@s@m(x(`cxYZhuXg2+m@w@MJTIX!c;CvDpTiHJm5HLZ2R zY2^;h1p<>4lv=}2UTC=S*yN*q)kwpf*X`eyL}Nd}$^r)PUQlqgwXi-xxCBMe*faxt z6o^-!>m;+(BMjk2*Vx~_Ri7n6qy)%NJ9yUj_eDzQvQiaek!A{R$0>Uh1EZI`9NiD3 zPW(5aEC4{S(cD0qDK*ni{y!B{whDF?Qv@y%GlUZ@uYF6Di$7sRvz4i(snS~Pt{Mwe zPTsg)VPeiS0Y25IO3FG~70>H8aw>nhf*wVU{BPDo0qu5v1K^=0e~x;kW0rr)32DQN zznj{8;w|7j55TjV8E@|Rnq;TYqS z{bVhu*D|_1!-!c}VcMFsJ_EGsj2E{c(v1HXQZb6N4YWEpH*8%+Xy z>_jLH<{o0?$>u)oR-|#u)aCAsHFY->{(ss#^Kh!WwU1}pw50(>#Uq5wRK^IK44E^e zsE{&jq*R7v9vZMEvrSRh2+5qe0VkEAwv26wD5MOPnfG4Ob2_KzkN2GGd9HJv_j>!w zCBMDbZ>@W+b+7wg-_Jd}f4jEvhT`ceGM(^{Ps81CP6bk0%kSJ22h#B@xo)VqK(!aq z^PPtZsY$Orp{&DeZ=_8Eb_zi_wKft65LKJ>)nA=AewCfZe9*j9M=KQyC`Lm8#q@&% zgxq`R4s(%nT%=me3OS|u+84NidLSftv+ZzbTMk;$T{}^M0}dsx`wFHUeE2%csjMaT zJ(+3qB)04{!flH_f_4tsPnb~K3%ktcaQ^jms+FO232ha|zUc03T;6oq3x)-y;e@ia z{K`mgAk)cgGT{R14xxsa>e3Zh*zfoV!gYKVVqMprIfFo`_&(U}u3q+|H&N*YLdTlT zg!W8+BK{6Pr}vi^w{~^`GygFspZ8Yws&r0;m?Fn_==OhjXk8~#E2TwLFwM7o*j*#u z0Sp-j*paSPq}5IGx6kWGnQQ7gl43$6iI0RW+($sGdS=8uiz&A;-L*PeszC3yR%pLS zO_R<&Liw#XtOas4vA*YV2S0ti<~yI7C_I!$ z_(Dn!U%ml--H64PeGfU!=W1EkZ9^nRb@Q+YICaCSHmR+6l5w^obZO2|v`KqpvuvHQ zwUJipRFUX@f{~I7MQ|^tNR~xzaBOgp+%wf^dtPh-oHSxbr(evnGqma?@jQ1}tIJt9 zcPwjO;*x6)=??gZn^>NpV}V%k2Gd%-a-iktG8T&n^_Wlhd?uUe9bCby$rbri01=BvLa*>iV3Z22KGWGmJ72CGt5{P3q}asZk^)??2RNb zKt;JjxRYiOsmq>VBCb`ydU|~dRan~9$XGB27R;Jp-irB!NP)W4nxe<0X$Hl8AAvEq zRx>9{S5NZ`EWMAVs`3vf5#AfIb8vfs##II^sp6Z%i;Kx7C2eCbTQw!>LZC9s zP0PlHaD!EDX*J^PrC!4H z8D{fpS8rfa9=Io0sCI>IXzGoNxv>u8bH_Ewn|Guei@VnF#cD_j5i&GiWqf`nSep>% z1t=#D4*S5sK9kDoSPhG;b>>GcdEa>X@c4@jjHIPJ)Y*57j?| zO6{q#bve-6c9g&kN^;Lr1N)>YwVy#KdcGa_P2Qz!q_`aG3lu$BY33rmHQM7GwPqoK z`zYKI8(~9F!bUoqXu@@yE?}Da5pew=BICjhmP7NpG)wE~IF%n4ot1 zUH>y-+mYsnzJujE(tqofjM_ggY5W|g)Y)hJn*t|qllIjyrCcoEUZ7n$weKaaec5!` z{PNQb(5ZHv_L=0Me30NLdb|vnG3U1Ajh|;fN!pP`<>!9_YUcV-@$x5CVm;ow#9hcC zQVT{Ema|e&U;T_RhA2o8R9NsC*qjl^1OS&s$tkFGy>8AD_&nFhphBRUq|`a`5VDV$<0b%TaCpB%is53Q!W9$ zpXFuoti5{OH{N8@$eXxl+2NBxFNy_g~bo^L>Rs; zwtBcQR*km}eos5T@n*Zqy>ll##3Eev?$QMaRkA%1cNdFLeY;1o@FA{T_1+e5l_~nv zG}2KZ7MM5Eo>{(@|W+6ozHYqubp)umO9Nhsoycg9+?XvO(IVVJf&;~;cD63XOTSGcPtvm^bHSPtHD-J0KxYj+IOn3cI6=Zs^+ALnV!|#^a zqIClKV!(CJV5?Ko?gsgCjBZpXx4a3RFnZqBNAdxUj86wePFe2smaG(LTy3Z=8KaIg zWCu{e678TVK-tk?%@ulGzT+Ao_>;@ha^+5LX-|2>6F!N(0XKFrf@bs1;Y%7utk+a0 zrnEBYDENI_`x&pOCeb-2mi58ZIeDGW>Rz|)M*oc;e>0B|714iq{;6s{Q>5A&vKdaajZ zVe4tX8{;pSj}+S&p+UDI(WXP6C1HZT;+cp)pG_YmpX z(4p#i9?_JPC-ymz+2t;0*M7W>V^u+0*917LBsaNNRb2xn)*af&ra@J zg0z+gdc6UiAtpb+t$kq`P&xs0SF~!~ME8-l+%)w{VkACtoLImW4V3TJYxpi@S;(}_ zs)vxk)-=HRaIDRw*x_1NVx?lV*lAimS``P)jaN@E&o=hxm1pR^oj%CGg#4XFF0~W- zk#=?R-8g{H28+G`GR%3@XSKe@8ySg5{AumMZq^(dCQ7i}?*S=w-#1|gySCxXXcyT6e6GV6r58X!Q=-=`G$l`YETfs>j zK|%#8Ej-=^%S;Udk+cvF)ooN?@Ju?P5?M!%=+!cG9B%_w1;h{EJHb)2apKHk4s zV8jk~3ySQVJsX>T*ztGc+`|LhmW@Df?s$wKtPrQ^M2MF2dD|!m_z3LL`-ph~fUt~C z2pEh=jI$&1a@ZLmK*OJbmhZ(a&x#dO_rp6gs0lS8j>Hl00zLvpwfINK&Q+e0nO&0y!h}=BgVYC~|y%XZZVFD;80aI+{!ysX)$>rU;&1+2MBJeU!f%9LA zaMFP5IyVCHoTyo({zhj4DlV_ud#(#&5B+2%{bVJoDa1)oZA^#_-)m6xoa_sg=mHL{ z6=HpgLD|O=;F2I*(3S=hJ25yc)$T2vWlK33L4 zT7pf|Dg?sHYhCL1{dn8&ZLQU|0%V~ZNainx4ucst7WGy~vUS%$#K{Sksx{q!6k3g{ zx1I!q@S9O6R$UCvl|&^u&?eTb9)h#XF!AsnK2SN7JO`x*nQu}a{>1+f5&sggZ+$+* zc$yn3pm6u2HZwcUQWw9joJN}cKgd@B_*7)H4)2C!fA2pOMp5gHKM|h)_VHyDiFAmw z@XQj+uVryAPi@i)M*>Au75t26>SjAm1-GJsILn@a7kA^?vr4FL-JJr-Ue+kacLS7N zcUJ?>IQbORC6s}fwA`Ljhz5250PV(Wc{rLJ+s07-&1wCCFb9AjBGgSMFg`|y=fT@M zl}eN;nEPB)BggSKS6DByX;0qq!v{j{L3)}NKb5#~Syz(vb}DV2nY(%>R7f!Za(>^} zj5kz|s6hZ{0AfDaxJUbADZXtwQzJs?(ymCyn0z|!}ciCTTfj_uT^C|0l z#OrYyh&d#q$8|7SO7T2QBk`t3E}XCv46jNM=5B$4Rd(wZSG6n*!m5+DCu;8vp6UxB zJG}iN8J4zij(2o4+aNlVzYz<(J}(6TJy4Z_nyH6iJ|7KyXQM_TGVBI$x%F0$iutL8!>!!AbP+$pv&-PJbn~NKwQEjzLGEBD>^)Hhyw(`*@=lZV1dm ze`WEa(rN@}@%Hk^rx}&0PXag6ks^Pb!|9%Xw;Y~xZ0aZ=&`ZEpWrrxhi6)RrhSWPC zn)UJ1EBO=V_iLoUc-)=Eb|=?-@jTMw3+Gg4_eMlRB9bYYa;G#>vQp|&`@4)}rdm-E z5e|v7kd-Qip?}pBrf^o@PZ>YKX|*hSP{7rytTgGjUALY)F}Gxj5}Qb{8s)fu?qTJS z$d`r~uRArgki+@@LkNj>y}Jo`P(nN)alo_afD!KG%j>Z}fjOnYlrCp{jI=|zQx-^L z0uxG0^XC+XGgCi8q`3AwKLQXNoV;k zEh4pMkXdoULsX#@stdX#vZCm^{%tKXf%E}IPa$Y#9-kce2fN|C#1WWDm>mMAub$S& zXvuPewC20JJiJBtjyUPP`u0yHi3jTmTqwOr~~G;oggrLq&{PL zaa3A1xZr*`((OTg+tM9TD(U(B#^M+eM)X9zPM zIvn&S2(J(XHKwzjOB=_wug*t$!%MLNuzN4TkK=;>xP|01PeW9JdLTHqHF;$R^E-Zf>$d>`7pUef><$s?BPd6i$JG=!1C<|*PdDk9Hr@?)G{f`05`amr+Gn10fh7HeM`& zpnLbbhA1k@9#soi0AAQ~5&Cn8iOVrguAA#7>B|rCQJj+&X1^8!6OH$`0n?*BFkP1u zNecYeGu$HjK)4qgCy@?@uKViU)}MAWKm0)~pH0yduT4Tv3vSyRC_Zh8)?(O2+&f7C zzDFqvj4qB{Rz4gjer$e-q?t6Np_LB|wVooM1cR;yV`G)##P(9hUNP0ab{m0;(9T9ReLi8tg*#VwnyAN>I&@ciEgnnsbXPPxZD9qbV~m;1w^N0G^c; zWpkZXUnL8LzibZMZ^E8MN~P>GFSjnczRU0LpUGfcbm6Y{QE`*DrozMQ;1rPO2G%4J{JVgI4szw&;o&QIBV8Zd*y{a0)@FAFYD<4(ZGvEi81w$Q z_ZnT@?JV6n7TM}Ic^V7YZMtrN600yrEyPWgy8`wXgnPVp6~4C?E5l#M@+KM^V2kJ8 zx3VMJ2)0T~7qefuB$dq|l)%%n1D=&R-oFEUN2EFi#eDHD^u5*ny35kF)&6oTEe_?~ zK@ckBhbMJ(qU|+%zzv_+c6@fV|2z08@*0iU{;f2NtF+AU+nv{s6SoM~siZ*metA=S zHpgqEe>1DO%YPD`DHP5WT(s>;#>~c{E9Q}3vN$kAfJnA&aUsY3H2rol3&z;pp(n6y zbP5T^Gr*rRF!g{~@;bVi6dsNFaXJDMnZQ4|HYhTxbL)VhqD>?U|JoH!${UK;G_@Nk z&8S=12E5l@zgJmu4+=OeDQnN?IdLgdGF@r8_I{#tf^aIyTb!r)kF?ir+&R z%y5i#-tyo{VZEo!2eD-{2hnezsB(Uq05kL02*JeMVEb(c!5utcu{>iYjQ(pkem9-C zw=Ja|Ut-e|YM3u9>vB&rjO|GPh3sW~8;oPYHkB@}JqX106Gr?-qa+Y@!l04q3A&}T?e>1I?fg)(CQZ)x(3Ww%WBcuyuto;ox8*lW zrS7Yn=4CW20InEPGYMy@Hgh!?UaKbn9>jna~sOrEg7fpElN|B z$GHaP3~Jv+U^N(Y^EfEYvck^|whf*p*_Lu;U2uyUlkm^md_bV4=d}UfJuQ5{lQHSj z1QQKc+vT0%RJH08sV+M)C+NuS#G&^b98Tj_$pjMBCwj0`?@y<^92tfIkSaZKF!-e#=N{Itsqq)G?+ z-^9hmDbE?Wn^H-3Tys@bRa7$@()YiN=T32v;i0qR#`}H$#TJP%sKowka&mHC)6A2q zZD+X?;p5PMX8F~dJ++Sf@eM0~4_|d2UWH=mH@ETk5Bc#!8iw6YC{*ivlYW1|Zd(wE*xS&eO}msV9$a>v+8FWqlNU7{fBC_e@?8sFoCzIK&{9?!*Mi$NVl)^-e7|VO z7A}e2$K`VjPu$(Boi$VTS$R_9eX7HCKWagfLj2^-UKGTx@)t~TVN(ygwt4J0bpiXX zLcy|>w}oeCm9wt0rRI)re0V>+I{rqEy_t?5ep8EF==S=*f!%!a!AwwZ5LS-9~j+f(#Si*oq!-0)iJ%>Df^+V09UGoPB;M{(zI%jVU7{i-4Z6ZZk;0|FxE zZ7keO&^NcQ>{=PfpE}4t+pj*s?#Dp==wlciZsry_oddeeKmBUuu@<>WdSnhieG2+q zJ{T8<5TRejg$91#^UDB3=cqyVsqAts@l!rjrQrL&X#V&(Mk%Kv+0V=U{fHY@9=kjp zMxFpu9LJQ2%Zy?bb7Ww;0$*oXJM95;hiZVYu9$RHz;I$HBlTYDl0Fm>kHe}Pf|HCCbfPu2~6*(;plGPw2E!D8Bvm1<(yWW2ma{^Mg(O_je+IU3Br zIf^&LdYcb^ZnFGuFzUZ#sXOrpGvW{pQL_0TdwqFv!f#J;iA{;${_CWGQC#=ujpDEq zjqhDg_r*VVrT%M-`mb5)e%+#(?!(S>f&LFB1Pd&=*t$)BFe!+Vr~kxJv}2q;_o`wt z?*G>Q{V!NW{I6TYgM;|~=aRqdHB8(QIQt}iLGmB$=`f14f8Hp{A9;|R-E*Vn|JeTh zFD!L`(3f|?U%5&1>x5WwkJNvi6flY$|G`oG%U9ID+`IXgeboOS1hilNxi02^#US`E Zt)eXw>W!AQk6i`-wAA(0?y6b^{0|f;vH<`9 literal 0 HcmV?d00001 diff --git a/docs/design/images/2022-05-19-compact-table-via-sql-3.png b/docs/design/images/2022-05-19-compact-table-via-sql-3.png new file mode 100644 index 0000000000000000000000000000000000000000..779cbedb93021b28d7338e8613596a3b73324057 GIT binary patch literal 124588 zcmeFYWmHvB+cpXaQc4RV4bmyy9n#(1-Q6kDNJ}H#(t?15C=Jr3ba!{0xz#7$8s~e@ zpYP9MjP2g8wbzWh=Y3r}TtQCaDbfohC@83>lCMRTprBxfz{?BaF}PAY%EJx?^~Bmz zL_|SSM1)wu$==M;#uN(bb$F6GyoPcwc9wShyLUfe5MCp;k|{lUjralX-Mnqc%XbP! zG6*#1HN}gwVpz1FX>tuTE!}qea;I;Mgv2y)T&q7EQ@qUC`EuNP;_G~n&0>D9yf3rQ z2W8>6DE0EH7#j+1S=dt8gNcluFIdP6$-w}oZs~CC*aSmOM~5|l+2f(Pr5PIS+g)jG z&`$QrgR09V3@$2^30;?_5AH{(*1Kr?5a*)qe4k{jJ zv9P6R5UIb$S8Y$sJgoOfj-NPbDby6L!vw-E`#wJkPzpH?Gb!M+8c3)%ybIv(s`RmY zjn(xX?QQspajxohxsBp;c%w$`AJn%<*VHhH3f!u(E~U9JA2$jjIR!v* zmazuYD#4p|9i|~fbP%B-2H05W9=4Z5VF}%(kW?IRKdFH?i+rrM8d!ve6F~J2)xRHJ zE&OduK>(!!+4Wm@27KP<_Wbw}SN7lClqeVfDL1kI`%-Mpg(;X%ff-pI zO#7%aBrKJ&c+loQA6*+R1^%d|9IsJzq%G~d47{<&^|@eGaWhe8mLe2KA)UKxRQf9P z)j3Q|apzzsFN)2&cz*XoD$!kSTNyvme4bxj7q$_6RqZt5F0b2jYp5r?&nLs`jU=!< zEZq5Ur*(tpF&;1X2J`0nU*h@HL`e1NSnD+F2+(dI!6_p>gOBA?WLBl=L&s*;INT#P zaOmR2p$ttgOUHwXaF0Gd*s;BB=Ybuuzbj0?;DNgDzP`CZ^&Jx|EGGd=kHklG>yy1WlR!oKKuzxeu)+yTKd1i#Zn81X|!Ri?u`hZk7|a-BS+ zamDg=mDr=>l#L`NQC>?bBB=*NA>S0==^YbR*OFG{nO{fqD8N=00AYD$bt zoVGExQMMhoZMU^wM4O2q=bHXNvXr)%s7t7WuS0#|@)YN3%wv7yWbp#=*6+`v)zeI5 zt!0zfJqC>iodzGJrLZ>F_h_+e+0a)`%2Y^IP*kLAwa&Wi#m!dF!p%n3yIW>Y`BjRH zR*xV~EbRs-p}6HI>D;8y|81wsUncRyJ5QF`*{|@Aj1#4c-^F5Y~HziWBkK@!FQ(c;~ir{ zn+jVFz38r|PFP%a4mR$joL8LTnWY(a+{&4nLt;a0L#Vo^y1BaRpIIAQUKp~G8h0I| zn@jk(p1BOV|62EQ&mZdPiF1}h=Sy+hUv1u@)P;C`XKvW+}v+% z?51pd>OZ)mno|r?oD=6e|8%%TUfF3wM_jj&V&o+w@N3t=M$S;=22_j9lIx zNl;kGEy{7elkAZh{F!u>iq~JUd~W7gGiiQyINie5l5$IYPve);u3sg5NH-_oEp~o_ z|B#WyrQ0cJ*i@z6g zLqak<-#Gptepnm(a?WMQmoYKbvycG15@RVN~gpEEMNPEw|(Gq^dZ>{lrW~{nZ+NLJ2 z;rr!hp3m5ez86<%F%g>)-|N+^Q!4IStii_nS4!?UV2>7RUxLGZ6`hBc8;^oI58L9kYf8#Bhr#*Y~DGfJTX5p(H!MY zUm;|@%BI`gz5k_!(nQ|4!g(*5`|fj1tHDQ~gGk@h8@t`u*#)Nn&zy7h9sMbyd?LCE ziS44v3swX??iXWtbLph%T4{G_sDw^jbt_d@OMPTs63K)G_?iB74;n|+^DFTyzbKvL zcE}=W(!0&1c3R5HP#r$!Xy?7mA}2XNI=tC%GpDtET$yy+pJA<^HA@+fxF;_cnWyJ6k^VzP}#KjOJYb*Ov9B09}t6LS@>`VQ_sn(Aik!0Gk~1qCd@ew=V_oXJ~(|um}*Fx z$;v{}fX@g}u+T4{;J_zn@ZyKY`{!qIXey{jzpukULA|wvg8lm)Iq(kodk0>SXa2lD ziVuZ?2miwWFOM9Ue|#HuDCf~XKYM|DP(sQgl9J$E+1Sa{)Xv$$-X#V4Y6M(Bba<`l z3EvoP2@nKQoP;^JatVrFD!rU&1kclNY%G4!Cfb0+&U$Unvr zHFY+2vUG5XNs0=octG!F!cUKP&4zAJ5l?%6jW7^Wq+W)m zMWUjKfTIhGh-eM!KBk2+Vl3_sY7N9kM-B_xo35hG4GId>%L4*6x*Z;0Dgys5w z7!y_)fjHxhAPLfcyB#R7pws_hOoR|LK~CgAVXXfmgg=W!8`^yI--!s^MIZ@qz=nB4 z^&dp{udNYhjKKeA!T!CACJePi2aBZmKMX9$nTPs6X(j}!QxgFz_kS2Tz(JhoztJaT zT0sQpkXkesXPzIwqx@N&^Kk`uWwiUoNL<*R6SvJS{M4PpEXGX(^f;O zPuxglvrhheGQ=~{6Gt{7LF8*<(SyKp^M)GrIS-6&>)E6-JdsZo_vsH2)syu>Wx1Gj zk(_JTU0=<+v=VffB(nBMo3&|*5|b={$zuxJYFHiLd&f3S)T7jwFqQN6l1BI`4a6DW zzA48{U#%v{mr64cX@dc>hXV zbmwPNE%!p5|Pm2jck8C*MoUo0>8_pQi)hX&zeDFrm>)GW{4>h82SGk5}*cQ1H7s^aShkPvImQ}c<~tP;@&Ls5x_ z>IQ||;~vAHqb)`Xz=UBfJ%PTiWvL8%Lvos;(TGTf{BqID-|g2Im{$o;U_znpQe)o9 zf)J99u<-%*F$#Y1kidfv%jj+dZ6%ckYweqG4|aogD3;^h4Cf(5k+&%IDVHHour#j& z81Iof3pPu^^3(}D%W3o5x@LW!XN$F;-S2+()#T)QBd7;+y3yjHX&?^R^JPNQ3Bd%is(&1O5RBKw{x@E5ytASodXItEy~@Ovy(XD~!s zGqKv^ANodX3{e$t19t9aTyM5J`b`l`(12tX7ez5ajKdq3Hkht5G^=|rGtzRi-@GH6 z%_wv*-#VBmi(5b0=KtU$+4LA1K~xYd8-Z9UgavkUc&L#SI~0t&L=i>z?$5YLT{6rF z&tXo@xrNz5h_fNhm?EjVnD?f|rFXm8`&9N7g{OSz%!qWqaRp`Klf6ER)m{fY5gi3B zXzm7LARKfW$q@6;sZ z)KRh;$rbzUXGFfIacR7sHC7X)s+t;wa+#&G4Yp;I+V*^BLp%q?p-=IWRS8qUB)71a z%frKO%Z^9|{bN$vtZeD;3ML72?x_$P=d*P`F-qwxs2wBz$WAy|G13lmP@}6(EWlYR zSIiS%KbX?@kIp}nrmGIQ-utvLK-8)E@-;Oq3>V_zw`|O(I7Y7n?(s#6CVrS)OFvyw zc3p-2ZSFWtB2-KxKwjZ)_L9*eoV1AAUd$vJ1>IA1tf^o%@`d_ivozPvT73Fi&0CBG zj$|q6f1cbZRIlz+1=3XKQ|4(#8?x%=$7W+uWx=-Ve2uXBvK_}?j#Vd&1p3kPdOXV+Zw)(o(p|*}WDrB*N-z2OJ*L@1T z&p0Pk(7D-S{B}fIKlgXq2KVYAHZbiJuadVnnD*e&2K{q(oA)o63<9_}bArczxi9D@ zx3+p;sO5=AE2--^zS&-A^Vc+u!-YXB&W6&zJ!2si^FjELy|O}!Ls!t&xCmzx7V*bK za*rRaSidzdccwVl`V= zlE!2h1PP*2G;d`<0$l=vW6XF){fko2-)%vHLG!i0V=vyc62l+QPwLOU-2xhxOKC&4 zr0>)uNqn9658s?<^>$~`DhCl-OfeM%YutKvzgrL2A+C+$4SRGk16hU| z*Rlab$x@-<$GAu3n|-~YfJhVt-Wp#O*Q*GWDM7GOta*wfK@hA-j{e6)B!BcKYJmVT z{+ZwWEBcUA?vn6xX!K!*rd14<<8s5Vp{3@-S*6k`45c9Wm#UVjQS^_OC{L{RCQM+; z^M3dqg`0;>uVvCz!!pGAXoT(wjU!qyF)%G!gp}Mexqy2e*VXFEclQ|97luCAe*|Z` zw2B4e3H)n zsp~B;RQ@kc9ZJ`c$a?Le#Rz|frx8$);|BIUwzOfS74D$+gno4WCGprRJT@zn%7^ng zdnxf8bdU>YF&87-&iUJeHo=$kj8jA7HyJGtoxEl@f2WYsQ!K3)a9{&iSP80+J2h)= zHOD3o14^j9W;#e$-uAEQ)|73I7y zXGR-}UmH$q2!)rCJyPa^6-IQ>4>clTE(E6c%2g+Nps;NmfpEMpJZz7fcTfw5!4TzF|nadtFO^zRRwgZ2ld-E$;%)Rgyu~QRB0s!QH8e z--;bKc=5|UJHD;Bb8{q5R`KSb&0j^V^zM9a_*U%y+v19fU{8J-bUO;In$O0F$8jX?AMKC`)RHiFmg~mkYFf{2yaL_Lt zSVFXCxZ_$viD~paO$iHB(vkg>h&~n$=LmEUnLguqzEr*(Hl_@X`u_AXE&2Of z!L)<^1-~0v*;M8N3v)KRpBlE+->|pOru5bQZ;#%BESRL_)JrbQMdW{5ohIP#XOS9j zwi3;u21?o|t#)KM3_8Uo&dEB4S8FNigXtXVk&ItFDqogF%RBqaRgCppH16E3eMJ>e z(f2tL8=+6!_?jygnaCTfreO$IzZ8mhZQ64pxs3eOx}rWA5}mJ)`xz&8XKIsp{n=+N z@`P(8;`d96vXt5$?we=b=4|iGZC}DsmZSYPKgYJagV04k+T9KyB!=>5`4Nc$5&?pu zurA%FtFa(%HHr`qUaVzsIS;V(GE<$_f>8ZEP@1u|uJ_|Nj)f1D**;G5%)r%F>#e7T zc^0S6g^Lea&=63u<{SHZ!A) z+WyjIP0kyT^m65>C8&(bdl;bQy;pz7$>nkM4JXO(1FK^ux2qEHqr;EVzC=EUfdWWk z;%&c^t@p*gUm7b>ev?4a%t1lJud{n*heg4Ys2;m)-ba3SXX^DpG^G>j`g}JvpI76h z$3ccJlH=K9EqKGi=C-RLFj!^ybvyw2(SS2O-N)?1kB_KDvaCP-bc)=b`j|c184r@T zuv<+;)|`0-_7DyC?A~TxtQyBJ7iAlF8+S;xLUb|rV7DUm%U<92ENOeRAkyn(kV7OJ z$M^bFQ7cCslw6{lu2ZDC;P`MP^@i_R!YUP~E9~i0Y@FU6|@_Y!ap3 zhdrUsosd0LkIItUlFj61GhTg9;5Z3V`wyS%ZzE#@f~^#?Y45q!v~AfA((S+WzQ)6M zOfI|EtTpr0de36ITQ??$do|`G>@Qb)5ozzOa}$ZZJ=hBBn}>|(NoCr>ldZg1{=V17 z5*b?$P(5KE4_@|?4MFmi{dlor*1bH&SJ%VEj#GC0 z!8EoLyVf%```J3ntf=F}jrXk=p-*s?JU+0=;ZEJ_7~6zbW_h}TV$l_^`%&TNmXfD1 zZ_pOMluMT=ES1~I4VTGt)_R(F6=kA0p3j)iS)6EwIZz;;S+Jdu$;ZAv!KTxArzox^ zzd3Aq=3}`RW9?79`qqbm6v5m-A?~3I#K5;jQ;sadN^?YLL-%G_YjJCMrl}KDkvA1S zeeS zGXfCNdPv)SQ{hCjv1u}NT}H&RZVxuivwhcbl!_fFa8067mp&GWdFcD!)$5CY=$z5E zZ(VzsP6wq|w!k3#=?1HJ+hs#{eH+%Xh@Siv`*4x>eiP**8S)U8R`kH4q9|{01ZB16 zY_Of1oC6DLEvx)YvoeFtDr(QVNC3j!Jjl29;<;e~Y!*LqemuZdJ@gV=XY|&}%@SM3 z*?#)vW>yywrA18+4t|99EaT^JS5afdDF)59ctMT~1fW!AQ2S}WwqOrW=#kSj<#n-7 z5W&^BUe4RRjJ7D@6*`(af<8sG{)KQo|Ooc3cGj9bA(g(N!y5S zC-#cW)mHCnoCn)`nE%E`JWE*aG^Su4M8)L?NE_fgN=U8n=CkbY*X5TP^@AOE=WPXU zY>~x-9-MU+0b53zx`mX6w@NCoO#1nPX_u7Jm=wMsqiw(I#R(H zf&n)*m6&DZ8Ips{vvfD6%-Ig-;5iSk5qtSby!aILUfS-E2aux1Fe6CNrtX(VQ9aR9 zL-Kan>Bz)PPE&U?E~brRJ@M>Ys++KX&YUzKd^v$65KMm%roFtzu;eG7C)7`)a4te& zE|jX$Vw(F}zN!6&tvxUYQbuc0mRDZ_+fNg9!z+2D5`Fuc__*$M`t>oS0B~qfpyj2% z&|WHcs-S*8bt5Yt(U(@`^zio0(MosvjmC8tFXsP?@+Po`c3~cB2%4V?E!i}!C7%TO zcf{c`my&tENTw&N$Ufbu^ZHWf2Fy~FB%{~pC-XMl!U9<=?{e_qnaCGV&9B4TPuFe= zJY2;#HLRyuU$2F2lK-62b^paJoKkLUB384={hHnG>r;+XlMrpD9t!%&*yC)ZHQ10^ zr_G7%sq2#aM7x4TzneKC#d-UN@mPWTY2LT_{+k4ywsj}jj(fvH!O_)$2siM`!x~zJ&x?|(^_QlvPds4pEbPmjtii)y z@`t_5`ap>#LR@)A9zk^6uy@7Z9fiBHGC-cui`5&BNolsrV8bGt#@fsKQS|oH(or`N zha6ah8kxx$oDk~>F9QDvm3zu)Hl3dp+k;sH2f8N*55YDZBh0= zwO#dxjTY~7P^-@Cn|y`Z!aL^2)L}V4(E7z20YXNC8FM)Z8KFd(7I3J~DSrl_qxBGj zkP-Mb18il^D3X2vc}3Gaa}5szYv{|vy|=(Tq0iZb`a1lE-_4%+aRjPq+9PO^_xuX@ zfsOnRxtAnu*V|>@V}-K)H#iw;Nz$Ci7%9glpBBKG#j~5hJww2Yjw+*9OS~o~wJJri zwuUR9yqn$gc$J&%hs}%{r}uF$rLD?$b1;CcCo|u8Xbb_5*Np$oZf$HU{&lzmb_99N z>wS)Wj=5Bm*>#ZVR@=V1?9EQ33;2I&#g80rJm1i0k1mM=+HuW@Gy~chp`A{214Xy1 zw_trpt-jyYH*L2W)4|A<`YF8~Jf&=hb;@!08n<_sRl3b?9h>8XW7*0(*YPhvK)`t8 zfIYDeMH&)bJo(wx`az;shVNc!TW42VYVlL4o+c}R}5Hi_a!bA&!C+0W=dBL+8`Rs3$PgdCWl{o-geNAu}+U_RN~su;|V{y5rj8 z%V|6Zj@?L_(OV~DNJVCDI@Pj!czZIOzS9&^YvASS2kC8Q{eoJn>1Q}$Od-vjL>D#Y|CM&RBffwFN7oR+9cx#NC! z$jTm_eni;+B*Sb-#>=8S&X*9)*Y!}tvOlHougf9y#292V<1lV^3bKrE-&ObV`W}7v zB{DR;2Ir8;GtYvMS~z5E5Brv*Q1|6y)khXoSLix zJy}+3PQE$vudQBzR5_sS`0d#s>N4nAcsznfVfANTQ@nT10j;L2Fssfh(4Rdjl+WI$ z*J=Eb5RLd!uCN50e!gWzwI^l1nX9ujUQXw*&pNUZ<)kgw|8v^JF@;FG0KwdCL;Hp> z;Xt|2rkkXzUZ6YBR>xM9GqP+0zb+jIiUNu4xYylze^VcWhkw{>YH8v2XN%L zr)uC;2bq*Ctk)HN>;-5#z*d8rTs^nt)4W&zoK;zcOva$Gz}igmosI}6%k=hLH*-0M zVKhb~D0b)|Vvhj+H=IS_^>Ob0-5qWU`sw7p40lFBYP&%G(Pv{cVg+{SyEf#)C!bYj zsHmY};?NhrL_Hyg;}Qg|*#Jx<6PP>l7v@`vCfjQ%`f3ok!3(L-YNn>3#-<Rv?inW~_oFT|J=dI}FQgrpf*#?i;17i1%OsEGi5Ee)E1m^goW z%8`-T)niXVcZ}n9G5#5XIfEzJdc$Erc+;a-0&&3PFt;Yh*(2^Vw5C$V9j>J%r+ZeZ z{xq_0r9fD-?gOOm9el+UukzDtRt zHQY&CD}C8GpBiQh`on`si5>`o#71C;5>FDBtpHM^qkLOkmT`~4lVa#V_{ZV;gOjDv zibcRZc*$)tv>~MhI3#%DF$OW-9ndP6p|3s6=!;@aI2!_Ob43e4ipb%HG0dykVlQTU_)q zXVz5~{{ZR8_F-0C&{$`vX0frU{2wVnM~ya8cZyEkwA?*8^2rfeF+QZj-JKj?CJ%Pu zyY5&ScaLFyVCc>M$LdW_R`Fr3fW@7RNN%u&)Vf@^E~@p9%Ya1x;XOq922(r)r^)qk`=~m)aOUT7WwN;>ictKFuk* zIza+zWj0U};t(wVuaVGq34%>Pdo=IQhMeL^2w_G*S5?5PFj0dC*xzh*tnuJ!3a8i8 zbmzZF0YfENX%w)vD5=jYXhWlW4;dA)R=Tx1&#J&Sn88gg$Ts>DMy()CckO$}vdnu7 zj#xu`{=a(fPPRtt0o~CdicT)TI>|u^ES_R3Q|{15|0~z97!c`3s`K+_0cbTf5ARqL zaF0Q0Z)olNR}e({l_JCth%LXP8t_E&h{j12!d(ZqRgTqRcQV&VMR;LzNr1<*;!}V7 z6VlOeXg3)hkFeaZh5w+cZ(8KS3rs^b5Viqe9T)t=B z?KyEq(lFC2auqKiog59I9^7RT1%6;N{pPO`_d%p=UWw$?5cUrbPN}L^_zP_&vTP9( zgTQhcYy)&WBdX*@s52!a?%TWh(JC~@f>Jmw%7YAq#x)3@;1im-2%Mpo=Sx^HIna{_ zA|5jNpVT7Em4E^x9B?lq&_e;LP}!FS8*aL_T5>5*WVs_w2NVbPIIOJZml{pPiv?B4KUQHOcG582pwBK^v6uHol(9K7Xcwrl$t2k3#zQa z?C4;2p`mu zCs{*kyFNGEhw|MXe%=1OQ>hGS9_0d#(A2v)W-6%f?S6eOS&0?U+XHPSxN$iQ5d5M7 zS{c7qqQeI^VuWpbQ&jGC?{^{`-qJ>4mMn|4PizofXt!LWPhyL=Pux^6?MOZ&2Ov3> z<@YKG8d(BBYH_2cyUwU>Xv_lJIM#Q<5N6RTG1}P)qXe6%2p|hN#DE2F=mZHNQncSC z8*GfS79kR6aP4Jz;C2V|>fLv0fmLt=Hc76rq$GebTfpp*DQ#8k^RSHz%X%B0hAM|4H_ADHjaS*COH39%5sz{yUD1V2&EF0V}aAFA^Y! z|3HxX{q_Ir4=uVqbW`qQgq6-~0A`Ds>4v^x}G0x=obg<2qySHaYAlP8* zFL0T2p*lgd2c(z;G*gCQQ5pyZn?VLDI<6E0g&-hB^64hlK}#S-xn@ee??=S3KzrC= zOrZF-O${XsSYk9rAjRc1#`0(m=>1O~H>f)+(tv;R<%{!>6X7aEdq6WZKr`y(KU5w; zBXEM1Wjix;De1JbRd* z_DAPLlievG5cv3RWUTS_WCHC0)|x6PLO)97lsx{QO_DGw_`f9x7C!?Ps1;YF2hr%O zmj(|-ETM(Q%>SbN2?im0WTh-lj?`&k2y&K@`)-GVfzVihOkO!ZUl;xdXD%=iWC7{$ zJnJ7ap!^mrl7bBANN(_kMNejVyW`evW#QJR5$Uhx1bD-G_&W|$O4cGQ(LqiI5WXK^ z;ou}7b-LzQ-0lQmGC}{f!oXaF!+Rwx28om_Iv+!wh1 zVG9Pn#8`wdn_SCe5zOtb$R;Nb+av04mg`5tMB(mlmc!qkJV%V1eP~unNtJ zP<{*tmO;)bw|3(!NZaq4xvhe5#?WGG@d*v8?6J(JsXn}<kN-r!-hPv>q(T(*_*wfd750pY%Y*IulO+*ZI0IG99 zq!hF%0O3J3%^{+1Lnw>Xd~AFiW07CJVdu!c=1zKtSOF2Nibg1jnFLKP;N~gA5^Pun zD^QOVrkvsxZ;$e-O27gNeSQ?>#4RP{fB?{P1V5}YF8Bo{1;XNZVWR;br18#U{D!bcy&lxL%1<4T(e-^{4_Js&?Lho}lhF}smjDmgB$i6>@ zxb@30C`KDZC$rbehxY%*rF2MxI$#7Yfc~8%gS<1E6$YqA&=uj(NQq#mz7+0ZGJ^N) zAC{m%13j-VMK1A07;N^LRD{cK<>Yw0l60eN|FeoemiRIb3@UkAs%D*7G0h~jy z-4dvkb1$nV`fHmhSw7T1;!LSbz(oqpBY8tt1aoQ^Zdr&8f{FBGq{wk%eJ(i&{W^ae z9AvJ0iuRlS1v<{sCqQ<64h1TZLBe0QM@o|Z+U7udxTfO#2N9H*0+?ZVc%G3BMW<#| z=0@5UZ-z~7PZC8&?y={)qMjL6 zw))>j>Gi)cN+7XrYKh|PMEt{xx`rI3aS zfc~es;6VxSCzt=GA`>sc(j57vIhe41d<~Q-21CXLqH7>4W^q@fBxvQW-7QBlmfAMV ze9!oDf4u`C_ngPGWFwdlHNuGegUtLI5o{^D5SCz&?i~pKfjO@M{STP6lI(wmYNz%XLb(Z~*iNHSTT}9?r-700#)dmD&cE6l%gd`e;9v zYt>nNK;c?Ns|OsX>#bd(>dw?@F0WIIGtjjy<5=)fu4p}|Ir%N#8{I_cf7TC9=pbT)nmmsbq~vFkeZNCH3`~;8D{2+a8vv<~)0CAPQ3-$! za;f!H<+_Gjb>a#DGbb}RY^D{$-%;cyh5(zAg`g$~dKAJB##015K)Vy*(MkyQ)0w(4 zpsl=MGEV|kIax=?D4aYo%$@!8U_oCOv>(sMeL@kn8!8QE;x7UCtO%Na#a;khW$AmD z@JvlZ2r%n>#V?Wdp3n+-wEkR?y8NV*-h=wj)gMDvPef2DCIm7*QbZzjAQIYh0l${d zhn>i>y*RoqakGxy1kRIAfGw5ImJMNY)dOF9TZjj!t{BxH-a8EQT@(X(tLr*+z)`wHS$KXqf_-2K6&$=_P6vTB+aW#oCJ*6USl0gHv>LasZ8V1l7}Hy42G zd2S27?q`5YQ37xWRY|Ug3;zdmw>zv?*4L+7Qu~pB5g_#{RQtz;VP|p$tu|*uoGJ6D z4d5N9tLwTZm3W`;&X}F1XGB6#UDy=KWyf$U zW}GyT`lJ2A7r2cHoewb*%GF#TYRz~yJ*0V4q z7vEOY{{EO=#Z${+;Bu@pu1M^aynN*E5Sg&wg(*K!Tmmpn3ccBQg8P%~hxgW_Z9ziF zA>cyZ8<7J5={(Upugfv5+$l-NJ#xw|ittLesR?iL1LQeVohhTz@F``XI}hy??4euE zqqpM~9HNJH?YrG`k1X?argF=pnX?Vo%wJNS zA*MJmbcstqC6RF_moSFItGVbdLU=HwfrLv&e!PrLtMMNMDs7|RAefUH0bMh$$ori} zE}RypM|q^%0UB^2676P&+pJ|4VQa#L{DGUr;BioAsvd10U^5>u-viS04&U`=UKZh= z@$Qcu1y8St%xr<(oD8?=;~4&H3+3!JKs(r&m3HN?aG)^KX40dO9_@hWZk^O&fma=L1}lykX|}o+iE8ew``hXDpR#+$CMUc5hg;-M+}9#|LG6 zlj9Q`OXqaVy*L>bn0loxKf=1FZP&EMT&elWdd4*644Xmcy`!vLzP(hca*s}|Fvf*|1eVWR>tlQ0_iev`S z-38QsaVP>~QXB{lZEtR%8J2XyqGLehcA-<&$3QLL{2Y*#Yw5?D4KTE@7>uH*I0j8I-oS*X`kAKbB(QiR(QU z55=>y@N4h0Na<&29>DK+sna3Z?@X<;E~{u@-fLNj;c;SPG4}xgX|ZCS4EVv_me?8} z&$dkA`kObxAuQC=*KK;i{fwQ@mSmn_j7WUEGq9;!&YJff&FHuI?xU=F_xR3L;rISY z50fxXxlOX7wlO-oXUzrnTH^qk9Rt%}seRzIMc_KFIIkp4pQN}iMO(4jJ1$?6hqWa0 zu_8q=tDbGDD9nmoMT*lKz@onCtM4JFshS2 zg5oI%(mdNM3G)1}a^IF8yj14=8uSeQxWW-T`J+Nd$tc-SR|9}^7wRAwCkhLjz0QZd zcfzOpk%fdjURz;gF-#lGzASX(lo?xMmv@Sm4|fOtcceNPLtVwna$Mi)lnc$i0KdyB zr=+-}n^F;KBhiFUY8>uIA2Q)~ygA>K)fUAc!z1+oRYdE>(E|fLJCIvum7oFd?t&t2|CVKiuB5ZHt)7fIxj}0 z8S52svjKVx=veX9;;Ax*{r3rP#%!rfN(<@w7{(L|e67o?#WTfbZ8fj>k)I8n%0-#& zaiYhX)TM?{W_fgu(~}L0!uF%6abl8)9nZBvz&JcEr~D#!Ue+rRsvn}tf4jv{F=uyp z`GYA=-6yhQQBgW}m1TZL3C;w|RHKA)Rd(m~0q()-$XguRjIB%t*`^vtL5(Urj6~85 zNza}g-XjClJjo23$?6IYqld3hbu#)k@&@?peMpqC_mCj}0FOLlLiIdP2g0H+6= zKw2tDa0b@ojvgHfTi?gg{{HsL&vQ!0Ir=J&9!686IN8ChE%Dg*F^)1Sk~yg$s8s}3 z>=%45&TxuSesP+*C{TX@6(0me$Cu|^f^B;-6V)E`1*4jlM`MXjKi%}hh0RC>V8QF_ zo@>2xZWwGDGQH#6(pCyZkZclXTJK{#YPk2YjA&JhabS1ZR4^U`j0!BR3L7tCb{rN} zP4S;*Z}d>-h4l-r+meAoo5w z`R+Yv(+qPolbVm%?e}NZTzpgpdp}3P+%Hf_SwDfaJW6{PWPCpeg!sX6U%pgXwp}D zxAUv{6J_LfBr%z7yz5rb7itMk+T`Cm1qFw-%`Y#fE3w7C|`_R8tdc(pqs>ojQZX~kf5IEa4 z9vVLQ4MB(W2W24^3@p!1tU=PiIQ+l+h2B7zzF^*X@-?Rm()@^`wS0|%FukP@DSD9y z`;AV{rWvUgG08@_mwfBm1*`_lDF-s;L`M#HxXGUmzpNYyyh57^9(tzBI zHQ;s(kJs#t6~!9T8Y!LcXRC4c*OwL2$@nh@!l&KVG#6%;YQ=yb!fXh?55N4}M^pBk z#}r3(OPZPB6=zz>J13eWBc&Jza5Blh8Y2{Yo}|mTX8)GJ?eGPw#bWq#EBB(rvo3(E z+oh$p?rXTsnaZ+QmYpJ^_lXkotiBV8%AvRe^{6}Bl&LcEjO|!x6}n;5cKkhqYtJT| zrm=jXZQq^kjShgA2URSz8+<~Oh+6apUMaIieh>rAPDn(Dg4H2c2)|HPpNr6~hx}L$ zZ+{#qkQ>$n6KV#ko&VC=x6jT`Eu;EPmk?(^c7FNZu_caCifeYw9JWl#P9xgVZVy>( zc}fKn5xi_BjiM%k8Z08^M_564jrR+hFs!|^*V?q0<~*LnZ- z=riY5{3V&~I9Xg9?{eQ7v&>&>Dj}@s?W?ok_gU65{p4nv&0-2z9|n>svz`3=EcziV z(}UKFtf?APX=h!2+bTC$3AXi8>Fdwa7DaDnKZ|bSfgjmWrX!fVtvaq9!*UwJ8Wg4i zzef9O@0@5=2MUf=&+t|x7m17-_J2cU-(gR$wWVUi6uqQ!xB+uTq+$g52Q6Q;^cJ-R zc`CudEI#cm0(OzHwTSYM1WI2BtBW9{ zEmv^1xMYXx$AcV5&S$NYu*Jub8Cw?hwd`BZ66YrrESF>;1VuleLqWnkudmPraI9wo zEW=cMvyxjQE>f{7O4>HnZ$_0lgL|C`Q~J2f1O0V>G_#i)$8d>not?-CjO>_|NDz8< zowCtCE|E;2NOhZ1kI{MP7PvQzZ#ti-28ZpHJV1di0bEfUdb6vBhz?NIs(Km2*Mmbo z4ZTb%JJxaRPn=}|zPKnkA-Ljf*{pvBBDznOBzP+|tAUzs-gCa-AJaPWk8X0b)K;tw{;7XXzVA6`M z8`mChxg+?Ex{3IPlYv-Wv#FGi&j4=B1nzLxZa@i zP$|wKu{-bBORZQDW7q-j-9gj~$<}67-wghE0VsCrkSC*LO^qvTKE57#-lQ-54BkX% zethnNchhBIq7u-tra9d9;kNm-swgZ|k}2<-?rf1}>htE;1+-^OZOOB_Wp5;9%D7`x7bb>x#p$lj)A0d=AGq7``7Or0-CHOzq=e{UwdmsVHaxt z!q97|U=sV{2GvoSfwyc&s2RIh<%XOtcyC4y=RM_mzZ6lmB&8)&3kush?^J7_V!qtE z;HDVqPC}~x1ke%{&llr=laL&!#04!K7g$3un;=0?8oS}MntZw2wz(mGO`}<&)Wc)k zjfZqk{SG#zjgTdN!8HBl%u?)L1_od;0@+@~gsw$3CH<^6UI_MsW*#Dx7>fHz5=XEF9f9ccw*QZR<87(ux6(sy5~ z_@92MJMz*QL7~5W>UH?k6haI>4Ea>(CGIYECHl!$Znp72f%I?7^k#p+bt~mNN*~ZNnGsE07nE>^}qvWw){?JadX}aAv6jG(X6M;bTg#Z3O_Pj?nU5L zP~;R3S^IvzRN1QFa%gFqDCpOii|9$UewUpKM@bXT27V3W_DPni)k4}K;{X4L+WM5z z(xY1vk(Vz3Gs;A9YdOTBEkRqPc=WMTp-BCz(c`tD^K&j^)ThR>5q%7NljIttAh>G5 z@|ydjz$i9Bpdk25S-OOSQDGG9$LG7V6yLWXoIXd$Pl^(G4?lv*UJR_qw4%j|2J6RH z$48F@0LX=?Pn+_XUq2|VygPbxCVJ3N<7j?5kpJ9;Z*tn5#jrX*3}QKb&34&{Fmvf` zH%?q*ZDCLli@C@IvY*o12amGcDCTt^o0Dv^cFWg!BXpR79F)fq$Vx|_y&I%Nj36Ey zf31hVLD&opBFg~(6@x}vC;F)D(BXzr#|Ongz%}}fpsWaJr5=07#%^JljLFiVFes8y z*a$YmJm^fh@5zc}$pKO6Va6hRTEm>UISrSA`UEpItsO$)HZ zJ8RhFyg>;t0Q_}*MJ@HAb2*=(7&L^BDi07IZlco`%29Qbg~sN0SOv4Wz-P(W> z#^&*rH_eJY3jAT?T{!VUD3jByOeVgAXu)P2$l>1oCf_-KcuP7knKMC(HlyYU4MkwM zh^hj^h1~#!tYEF(haF0~y2?pH%E^&km(LdvdXu(yI<1gQc&18n*s|Yup}0_tDs>Tn z-KxTd2ptyT30upZ`NFK{U_O9zLYfAezuvhv{B&5ouB#t;egb4tm-{}UuS|dpacl8f zcB_a-87N^VK$iCPH;@mRheP;&|N8NxUrRBu(xrDgqn7s@qDLuI6W-B}4442p>l8rI z{dsS=-pdK!9Vy+Od!up8rM%){GsW!pF^`_L3cF)T9baGse#~}fn9sT9MoWo#9z3S- zoFFOREK;|bp6QV_d?0RW3be|U@HH65fvF%X8EH;I&3Q&OrZzTYtSF*Z(7(mF4)3&L4V-l5VO3H~0HNgMIlKFF=XBETyVLFR$TO2VMb4!haeXpkUqEnn z7n8k5IR}CHW==V^&uHbm3O3W*emt@TNw^V>FVk_*_;SRl!Rn=Kvnl-V)^8zYTbl^F zxdjUXJ8$ckYb>@CB@fCM=I2uly#ws$cWcWe}7lB>4WrjK?d5kKwj&O$5eoWVMTqOlLP3ndS>RzXAnRe^kc zV8hd;%1$4}ovo^Xakg_!+7+#{F3{O3z=KJGtk0L#e*LFv89n7zb?IWSqgk;8wR4H0 z7jQ3dM?36Ntupak_%K6`a#|7y6QOhtf9t-8L8Fh`cR2b92flXdKxv?oDfw8y&TsX@ zeUWRaCSPIq#jZpiHoFq{=F&<&FJ~IHnRGuZ`X)jVuPr}?ewf-_sMGHZ%1rbi0Ao!0 zuCy2W7zFX{&+Kn-k3hiwG@Rr0Y#093FDMa`; z#CNZ5LkU_?UNcQG2#t_@wW8n^8WxGO^A7hMp-M3Eu}y8i>@_&RMHm3bkE=`V6KdB$ zmaV1VV*--DlXYGlg!Ian1&jn`ufZcS*NZG5X!~cZvN&nQb{GO4U}yn@i(M$rvVUrz zc^%|LB~?8N=MllBR>eE}mHD;aq=^$}<{!ULV(dt=&gd4cu)J~WOi&2)Gz9A1gGc0z zN55WsSjSlFe0kLQ-Gvb9fl^FW-;nbtFggCJDXEBm-6}~w6rq$YMp|;b-;f!y0#agO z9~|1$V&ix_o2-nGZ`nh?uo2I+gL2DCinychyvCAt-?k-(&b{o++%6^)zPt#jz4OZ8 z)6;>sjM7&xW&tEtG2+>OO6gcNgwed=OO_`_Ww$qGRG5=HMRF4w^roe!g-VaM4BieL zll?bGD4-s!*DIENE*wwhYmBJs_!OT+L0m{K2W=-uKqw4cMym}I`Se9b&6r(ya|y^M zv|9Ztc;%Q|BeCP^r#Ft1jY0MwaJCdN((k}RVL+4bo~^pS<}>ShxSLBp=>>iLJjoMz zN|KHPX9rJB>wMM0ot)NLc$2}(Utd+&*MIWk0FBs+Gq?eQdrFU4NzOUP|+wD z0FZC)LoK=n+tjaq#vQ$Q?*T|*^O;)BVOqREjB{{~z~x*cq%Q!~ilBp^aYj_kNCo{2 zK-htta5hIHJ4&osTS9TXnA!^L_olL6YH|}cUF$weB^D}7IYVmzF-16C+7|T%0KaEY z2#Rs35WwiAe3f6;Bj54{N2_3H)rsRR|K5!$`xbOg5h}Gea23Fs=L-=r+d1OHF~>Ss ztic&4uP<`y0brD8!K+5Tu=UujjhqC@B->EfH2m{cSGCr_0-^{4tAGHqkC@giRcaHq zu1&#TcfE8VTat1*+0HLUKGKXE;P@B-h@QRw$@*XUbZ?HI(=chqP3W4;7w0TM*XNW| zCbZ-f99#44)V6X7AKC* z{@mwcksB(9;N9PEEL5LtW+9NfwK*i+iH$5%QP7Av zGXL&a6b+lxo8G+m7iCN~KSVLC+IDkHtHZ=jcsu$#?@#wMZ2Y8({A*f3Mfgj-=Jn)W zUfsq|hol0H`vmEtOa)bAYgU+EAtD)h`HlyA<4l%rLFkxOk>rU76))1={R#NfnQUk5 z%NISHpF^>4XPKH9MhSw#Je+2(#&vI>vWHO;7hznv{54~>#It;4?9?-hLd6}b20Wl_ zu}jMGzqE+-Lj#JDpb&7B{oh}olg1=Wy>+A$C%dl*{7eFsban?+`oAk$qiL-mqa_%Q0=9fPg>}-ixB<+VhrM`lPHje#5c+xqE%gDNaNjp zH0MZX!M=+6_4Liz4A+5zjH@ND08$(>rDRNyH|@7hEY1ODgj8YW&p>D83o3^o#FP{d z2s`DpH}CkEWC*_?6js*<>}HL^6404N457j{5wlxP@KO>vfb+|=m}G*!eIB4&_2;tAa?)^r4l61 zTA`pIH?EXxf#z*7L1s~iNvC-~!Ylg`?UuUB#6Z(U=!N9wRnzy5_eD)=%7)iGwy8Kj z`x&zCS6)Nm>&69~3%bNBQuELV`>{Ue0lh~9&yHS_^x--$S46~v66SJl!Ewt$vql#q zW-9K>Qhbzn9l)Uty9H{Bt!I?DlG6YkCt5F+i?+4N{I+xMYuK2MWNz=Y!%|*u!SSHq z>-+Y!JS97KFsK6^SNsj3lI-B0{Lo(Z0CaKf1$4nXnByNN=*zE}xI$~|RDciPLextI9r#JQTh;U9yJry)17QjfSz zqy;QvC)2x8R7eugy|dnNMTnU!Kj70L=NRkGt`7l=Wlxi$j1u2qg)OR!yj~N61>K}C z)-D~nYH_r`1^|GQXe472f*ja)Fh}LU8cey(6SgLgmOA7vzoHUf7`A>3G?=738%Wq( z4rQH2wVgdFd=vMeQg#-kXJWsTlNn3`c)+0vtLeUKIWL^Dz|2c9M;GqyL#AKY99 z5Cs=MzOv1Jh7#+zFi>n-9azFndyM3{Yv?bEf7k9p=!$_#(@^mx++^xkLaC}N6cmBYlvNUDi2OS(fdNjEWtf`_-EJ0-~6#C_&#ptHi7 ziQyW^l5vJtAu318fO|x8>37dTQ*#7aRGsG`?~+N`*&J?2dVA?5K%pf_bJ*kf+9%yh zAn(r|*heFGVI_jP?*+g*qe9z-5yr?))n~lb?UQ4a74OP zpaw{D!@easYGSDWWU}>&(%R>D=D>$)iCSW;Lkfm`FJCE)U9>cFnOC1CAMTAs6B-L4 ztV};Mx-zCH6gFxBZ~Of(%brd+jtcszRv$u}EcAhZ{Z>=;H6gC|>QRStCj7qZ=BN4? z_?#y7>^k}S@Vl_^P*}gc$L@%Hu4=wlyD_YY!k%F{0MLo zc9zvUqoE3Cz({Rw`)|MP$xpH9=vOvVuk|tXQ>c&@mAxrl=;b70UM{>qh z!y?q1GHKTO9BZos=Np9NDc)3QLoVaec+G(RbS!VtL(xp37O}KBqJ*z-lAY+4k}bcRj-wpeNM77`xW# zeN^^FR~y$boByhzQ?M75+rK_)`aIHHN@?A4InAjDgl;8L@* z{p1^^$0uukd8CS1$q7z>a>#yh&!_-M2`jdgSR^yXdR^THC|(YIvy_^% zU?jUxV)GlMwNKt;W-`vdu6Xo{1v&?q#}M{?MV~JR*Yd89SQX@F4waq%sJV-aHNsb%hwO~RGd1vteRJCxA?dh#<5IB(R!u5e3ULvC&QXl zQoequrL(9SsqGqg9fUj**_UaWF+}^rVIq?Xnu#=*gFplUVYt3b)D-x{h@-y9BU9sy6B z1-L}$X8o1}|C7W3-HNJ@!cX2_+Lo+;SzVhVjQ5Af4{$DRQ~M};?o;Qu>ws5JceXuRq%$T4M>Nndb@t zC;qxYIt938bA$8wc%i^h2;R*wvKw^h$3@Xs=SM->)lE7;!~u{F65)EH-vuL zb`;h>0TBP^$G7{?)OODbbwk|j+n33M`cQ9ui>t@lyU9{NYz`%-K=C|%hxBdJK0#u= zJaXEia3%bg@32w5E4W4~ab8`1qlPNT1dY0D<1^QecNwm`lCJjC?&3C}g57!jb*`J2 zI>GeDimyT^x%$UFzYcX0Itm5?Wa3$d_HHbNs7rJ(wVjRPjshf&c)#u=r*I-i^^`z>F4R8W)J$GYx zBRHHD;pY=>s90Z0lX}!P&ms<%qZ9{tc^x7CH6|!xzGD262ULZgu*w}j2ayi;>=Q(! zFtlxF*!6yR10T}myZ}jhcg8-}H?(JYf;NWPpGUeWJ z7t8e|ZOvss#eNOjX%*dTJ{;y|&olz9f5Fm+%7Fi9=sV>Kg^FHm3MOObDn&Si`s!Ai zY2er5>H$jYC#F(LsQs8+-2(lN_j#I$TiQTsFp5~Rb#zJX0VMrRlKj;%X~kPm=dUhy zda^s!j9gQ_khnG%d}gSiYWO3#4xA0Yj5?&>IHBJ80QAu}-R8n=Fa7iYYmYG}(TMD`L0V)ssRY7TWL zu;=zH`SbL12LKi3f>c#xUi}2*RXY~>q2k4h@fvL37UM&}Q)S?(A!Ac2d!3U-o$aum z;HUP-lV1|gOf3P$boG1Nvr9Kr_O*LJBDFaz_G7*EQY(+t^P0Cpc}OqMz?8&7+So@Xar9Pl zH)q*X_eDuA;n-&ONwFX#nV4Ve_M>0Bu8?2ITLNqX?RrRz5TslyQegHOW7Ys99i^)% zvHZdf0bx0T`nS((3tCSa$FTj(UtLv_1bXz(9vcrApahhF!PRUH0to{h^TcbPXz{fG zU`fuN+kQj&?t!_rcPoCeIQ`W@U&Gi4W5mkq^kBxmB3-EwRR?st->3vs?)e(w#p|En z3D(mW_8Fi3%#(kL)!_H$nz7GGzkD)f`u8>Nm-98siebS+KeiZY6h!J(jB4c)|dOZt9Am{H;}6lCQ#2>+@lm}rUekF!>ZnWEdtLG05Y^HP8CRjM)>h+ zpSGa9jP88A?`%3FA!&kq;K+d;h?+K3Sk}T}TVt@9sk`_LP;`FzXWf>g(A6G9u&C;r z4h#Lw1$R=f<-7R2B+DmLMQ8`Oo5eQHZx-J{bJu*H`jDTZCds$QnJkWd`*+@OjVEP} zI^E~4u*Qnq#6hr}6eaPoEj(S>qu96=OJwXH zI2E@(XxKbzqOb?nt0X1>Csy&6bEN=0E+WGt$+_4`x?xrpe*u=%b791nAo|MCeKktO zdlBb>_L9|CXT9k@a-iO4%_vtRc0D(jmY_;Y_eO*tKaajt5CeEaQ$;6Rwd0`Hyucl; z7H)Zx&InB(4P6h6Dkr*~`#m@S6sU419($f(7Fr3d0o-+;VI8nXVn1(tujj_+QDzP; zRW)*iebpy>$v&^2YdUG7wu*g9N9?YZ4|kTp4D=ApeX280GrH@m>Iim-n6x!t*sk`-a(fqy{Dt?UY04@I=CH%xoEyoLYaFa%y(v=K4AXd3xGOrim zmU4Bc-v@P$#NFql0MJVqyPPx71R-(-4g;AWa8@k_&X^kJOnVAbZ`C zoM(^ToD5n({E%w6gA=3p5oSbX5055om>X~Kh&roF5(geKVjnI2J0?E$lKH1-mR-;n z^}nhXr&!&n?5?)DU3=kCPu|=|tG9`7DKJ0gjaoI)Rn!|TIAvzQ z={)Z_uyu2apR3z8G=sX80_*dVglj|d7MeosMD@j?QcOXa7@hFp7L7h@=6j4(eY_V> z0(V)_2dN`-_rWUq{F(#*b64fT$$g>Wz~MVg942==*g=9B6q0BOsl%?ycAgW-pn&CU z8H`&KYR2(2@c{+#1eS0_T`Zy3W$k zlWO4axD|NqNbt0;8lvy2(k2w=UOc0IZ9eRKH>w&eY-V**@0qKAx)aC~07SLheRbX? zfOd(<;sQka<<(RZk3~9_*A^tq=NRs<_b8eYS%XTte*gJTCferZ4ndfA$J9F@ebgzUDbjQq?tXg6wTi1y&{as&%1ibA+NPo#2 zIb(K%bX;VhfBD1HN+QZFsi40rwG0+$k9A9-)9BYh4XCi|iSokAm7=`n^?CGhrrE~^ zKt8i*%>SUGMF-*1!%f56b{vDWB^?T@M0SljXxmiSqeXd9SZ^89)IV*6A{O|Ra0U`4 zk$)?&Q|~_qMbkAY-7qCK5(mGNz=`VTr~#qoF7B-~xc4PBZUWGb=s?(12kQ@gRFl?@ zuW~p}xhz^3kYU+wz=rE_=MNfm@6u=d5zr$eBy2(~JGef!7AjTUXfh#8%ZhdzJ{31Zo#6aQ=H`z)$C8QrdX>^>N3M_T&w^ll}RB6!c z$I%KKllwfI$@z-9rQYd)`;uL}8OIZ^PrXf|NSo6f5S`_oem6G|Z9IS$WPNr4z_OzY z&^;C4cX@P25AY=fO<+d5Ldq^GRj0WF?L+1_I(pVV^E^n`Iy~h&G=&}@q8KFe=d}Vp ztV%zcoEX&-tHknNe`YTK=J_Z$M%NwFZlBPqlAHogl(g>N!`1WBjCw#E1Yf8v4~g|c z!_MytH0SY)%}D~rCK*yL1He`(R6Tr0?1)F#$pv7?nm|q$Z1M0ykPILY$q_*g6lT)0 zpR4-^VE4g2od=}P1Plo~GYSSG4Izv30v*%1*bQ@G?UrQ48 zWHhaSf7;ZE;0P!A;$HpePpcEUF$D z5R?v`yG*@?Kq8>|z#P5S#-V;F0euy@q&mb^68gK=RRtP6cz@Ue9ifnv>UXuHm<0g+ z-kSnw>nq&7jG2X|UgSji^6e2_}- zO+5VK?ZYMt}1ZkBck|e}Db6^9$0=EbQ6q_&~ea!`;x;cUX4FDt9 zr7t3o!7<g%!^210hW<`fsQLk=1tJ3W~9BqglXwGaSCvPK$* z`HYKmz8~T>x*`gZu6o(Cg%W$)iIzL`-4>3V(xU?RLH3pH1NxUi#2~fp*+ji>xE>Qj zRvU{m=TaHi2i?wJP7){;`)JUNc($T0tOyvRtA;5Oqcy8E-Lt0h>PKb>gYw*Y@cGX}HKMitcZ zR|L8AOq9A^MMaMfBEl2=+ll~OD}j!)x+h?ldIFp}r7t!Y77(6X zbDYq#_eF+WT$XK;ymi`2Hp|kW6P@MVl@Tl|`pr%xk*nmWIl;fv}SMBvY4eKY4L9xSh1f`1+X2S-es zf^SLoa)yLMg5$*V+AM4J;E}a))lm8=jV;bI&X5xSG7Z^J-Sx9KN2NQ-7)@d)bDu$= zm5I~u7BuO14@WqWM3qm0ec}(htGfFgB?hX>*>Pii`H#b*!?QhE6|i1U@$|cfTujuJ zI!TiKmWP!g>DepHLmdH?%@LrT*cHrlNTPYvxEUUqEO0`IZs$VQ+z0PB-x$3~QHPW! zhkx{}-ro|;)=Gf?9;EBFamd`t7uec*WhvUxAn_=->>K+Dmr)7*olncvH)nLS6vpAG z#iW&Lx7o6~FVA@OVy~=#3ed?F;o$ZV%R6@97tyLvM+oa|0c`YH(}LVW`g`@;OVewY zmU#8H-x%}g;;pe={Z3cBee&W8HO#y(bpqdd&f#0D<<6Yqj>AGPXJ*#ZfqnANppjG! zG=mP9=kIk+>dSo#bq#0yt|dI$-t6LOP|@`3=iQRO+odmh zz~|8|V{_=>)E65U`Y8ThO)O+u_^}#Oz-zs7WlD%kznBj(QGPmLq|K~IsUp0dE)zNe zeJ3s96hGR)z!}mHr#S;iQ6_hZpgC|<9qjM92n$$M3KYO+EIS&pXspAhk@>5+LM0-R z4>~1I;W`de!PGA3n@&iW$K9A~w*;@!L-N0y9n}xF2d_xAt?J@<`I}9!A@KTrlITdXR%*&(aWL2$yyRcc})^go?h{XpWj31;L{8 zN`k`Yh*BU(891N~Fb6B8>!T(T=aVkI1dhblRMGxp;Lb@S-fdK1s#ODQ-9bPsibNW= zuH~yqDV80H(6@CgSaS4v#f;q;p78xDFCMIaxU=ZN*M^;m4HL>NEz0Zmn`OGr8u*d# zepwbvVq7n|qg}Fe-rPHR`8G4=%J+@|3ub2i+|E2*I$Ie>!2F_)g}#Oj{c57%BlCiN zku*yY;tpOpTsyX_xof#LMG7@59fGDi+rN{S(u>t(X2KtO_5^!N-&V5R%Shfg{@t{P z?#zX)jc@h~S*18nDB?>1Hn&4@jzojs<(-E-J*Z#!hd*s85bfcWX*Hz;@XfRad+UJD zq8WUtZoI|{l+=}V_jQg|pJ^$DX4-K~qP8jnYw2MqN@;kUVw35eF(cDyNe+ulY?XRA zYQXJ_G1bv9|v!KI9K4U_U!lfJKIlX5Z;P6 zXLkD`UHNC)NcW9-zSNEPz6Id1;^lT(QHM`9zc=lFB3k+dI&~u(hOu-KP9>VBD;d*u z){+HEDxt24^CTcRv5j=%xUSly5OYTjCpaXkeGN<`)LoCyDZ1 zY{uMERa0%_{KaW@XW4%ES`~=g)X?F!R#J}!%W3Q@n7%uN@nH7p42@9iXSU*5jBQum zIa6Q;(o5G7C$QTQMrVQ}epX@B{RCQGYw|h5J=qM;SKQI1iM+Y}mEak9cN~UC{|2`_ zP#?taGQ?Cm1UY}mF4RnS>Wj6VA(OYOBovS(D5(#&^NY()3*|P7>8T!J_@)zFiZkyz z+bUMVBS?B&Z#Q36`uq7R&+WjdzIe6ob{aakSw5}|?lyT*EyUR5WUWS-@~oUm3u zeMjjrP{su@MAnQS5Y=!uf0gr;4rA+6Rut()bzywF$-vTlcGKn(7)2Sb`}k*>ol9u_ z1peHOQ6J2tGvs}rj6@GuQ_&M_-}pcJ3EPr*yRDZY?zrInB=v zry%grHvC#x)x-5La%|;D5GqV;Q~Sob?t@}t`kT4~%{ZO<^LWviSU3dia% z)Fs5bRK7|jl{KvbN>;%>Mi$$o_)z8Nz8X`^_TP1~rAW-XtG2vZE-D66rkHr+{N~4im2rQn_d7HR8QgyWtLmyvjmmP_Z*hOKZ~Z z);-e-e?R@_M73R75VQD%*uGPKJiuzlWTE)i5hpGpFv>TYd~o}Vy-cYXFEYlb#$zuU zOGbi zAj?r-qNIqs6zV*E0%{-`=;L=h&`HLFqb%mZXW+4jago{ii(}@5YOF0DtK-|Ro8pE9 zGDHy87c*6B7N;M%t04X!XjBWVn8`ixOE6O$f%=_ejwUESqQwzG!bKz4;_`RvwtQ7u z>5oaq1AuJ1+~7)@xe&MZ*f!|Lqy>gExQ%m@v#X8e3f)qY?8rGK)uXJoNh2KLM=nj@ zg?nZ?i+S4sBwxps5A6lgmS+yFzb>GSN!V?$8CuFfS_FcO8BV zpejzE*`2pvWe&KOSTBMjJcqKA%*1>O$<(yYHQ)iz6;pBzl(r+QO6ofdbWfm*)@$gT z00;HZ2E)F44QBmsiPIC28rrmk>8Zl>^KeFo2h#=c+F_4-#Y7(}&i9N>-6SqhNKTY9 zG&JD%qK!S;AM;(O;O)5EOKSTw0q6?bfizM~qE=`nqJwwHz~bF+qi@>&Q1(`D^RX9G z(+{O%Mz6vk;%;I(wL8EpS**L4ESqA=6FqBff9R5PDz~2%z!~u`J|ASdPfN%7Gm zonEYmmAUhNT!$hD4p!-lur3T>MB(S&#AS(`JnSjYB?1r%DZs@L{VXDW9?*fkP7=PvhAR&9%gox_&T~FL zHT~w)aar>Yy&TShI}x=PnnU`T+XI8S(@>@;c50 zZY&u(iJhH}l$7+)wr!k{=KAG3bUa#d_$n;|GxzIX<2IVn;#Kk8iIG(gvVyX)0C5n(RzAmg)GUPu++=97!{Rssp!MQbKNI9E-)VU*BPcAim`$DUP_Y8NsZmxz` zV?vSzOH!eOI<(l@XB1)Va4ckZBjRv9j&w9_a<3eUMtFX-%*2s1{cXzUHfPxhTF35R zGof`ii;*^x9h1e6%f?#$MeZd-o%ZHBWrQv}I-Rg26=7rKr36%Y&rFX>HR{L5KE+ zLO^Ce<}avCom?WaDlmtYq~(&nV_q|t|{E%&+Gv$sj0Nd(}Y(%1<` z*QqL@ShFWTw`FyL)tM#pY<99VB`)&Nm92WRlIRLIhYwAK)8y=gyI7Ww3}~$3?PT>c z?BaXVWcI2uV0DeO-~&&2yII@VnD($1K1}-%NAHF&r39dm`y5OLDVyH!L5 z^g)c}O^X5K$dmf@KYmh1)zTT)Hy$cqZf^Y>Dk(xtYBBeyN?PGFp2fUPnaDjQEM@9> zg!s#*!g<;vuHEPRV3Fwg-vlrQBxmsT3f*_Zti5Jh`Gi(|>|Ug`esjRhpf8WpVo$0+ zrgTxpdlDZ{4OJq8i%B~qOyI>h&r+yo9)~HtE7prs%`VDMpBwv)>HFw$Oa=c@UE>TY zN`=T>>LA@gZnP4f$8hQ}=@R_l$>!05nWL&j=;DaNPYAi!6EB^?=7w5V2`@R>w%Rnk zAwkbB=A}dAK6Fx~Z?1XWL$CZC}EK#L7hT{nm2 zv6RIBh8!ja-zrtI8lIT~L1J0=9*@+~U}JbZ-pJ$WI&hSTn)EguR)xIE?PLpvJ^pv{ z*PNv;{`hF3X(3k8s#-MGUPb>UN3T-jbR((zP%qiW?)3%*w!RPBPvmz(A7+)dU>Awf zKlWw}-j2bZn0;C(u5G*Hhrtc|q&9VURMDv;V-ez2IfYZSSJ-zSmD48@KVDvWToNG7@nTNbW3E_|e#5)$cfy(qcz$;<}@y#aVq=4hHStyXD&=jRz1U8GJdn zMI?wOL%c~Yzlw4Ph~%-;EiHSq6A@LVE!(MCUvqXIOe@Hdmb)nZmjFfI~jnm1Hg#VfF{7J~w=)tq!s^-Uq!!ICV`G zn3amcTC*JRi%Rze<`!I2cE2r(doRfGTqiyFhv96%5A*L99`>W34u)bh@C%#c{!VrR z)&6_DD(oR3)84dEmr?4^p~?jO@||TSre! z!aO{?K6kM!N_TrOyCWjW-2Iv;0^)9zEs2D)G6%Q)Bp_TPtW?eT@+-<}6>tSgYrk&y z6KTQy6oFI2Z|>f@xQrsARL|UO+KQh4^Zhgf-{M>$b z`xBoem`TX@_7C!-K`MWg#D}zCZ5=CwsCLUDsSDy#N_cvcScG`UnZkD3vwI6o`xENO zpOc)y=L3dT`d@U?+MwY`l&(KA!!O%I)$t@ME0la8Au?!_^-J){nIDyGSp@b zl%&T<#@Ugj2MO5L(ug|g=RaNm|0irW|C=VuKd^W1CWZf%T>V#a^&e&5|3IerU&+4hmw%*mF^RNWN?RO(ip5yQJY#3RYfngXV~M?_sSg4AM9Lz3Om|Qrx5@vQ$^~Q zN(=7W!MGSP%*BNdaStyOFy_ph&cQ3scyg%ng{imd3 z?>&m(h2tFUrsn~ac+C0o+kVKE_aE^K48-h}K~>}TZvUxa4TzCfGoZ9hOjYq}41_O| z``w4+pIE|)Kb(7biV;nI`JYQU2N;W=qwuTWS49DmLl?FEMq?Ss4-JMYrS9G5v$5hD zKrjp(kb?U7(G$mQ97wk6dBSP_NyC334nPaCkc$QFJRFVh&tGGoOy~;B1s4@O!)vhx zK@P(GgQC$PN{)M_@4$Encq9j&*+sE#<9Jg;i2_{Pg95e1m|_?;h}_MNbQV>FN=M2( zmpDjLymv=NUNmB0lCpmnb31IGZb^sS&;gQv(hVU<41}uxbLP`+PB&0ZKbv|PPLo{r z`6p-joAv!Xa)4o2G@nql;%UJQObM5}M^sIpyT*eu!R?{79r7v9{+eMWXC|RF=LMX` z8;hk?F|4vToXfSK-}CGX_yW46THU~b1)fQ{lgXnvs9_mqOg_qR5lW8Kvkk%FyDD$T zC`fqr7O9|nc%I{5D+q-mbzn?S(8wfnU@X6=nTa`)bzA!TC@W0#3AgesfXwRN4782SM2t*M)AK1U4OnybN63&dPer{hSr@iUx#M9lZ_7$3I zF+X%EhmS8h(WO=nCHt5r0tB4yzUW#6Ds5uO@_U59|4YlKBaI8uwg1RMS7qY|y z9_u&QSvA}6RLG92Dfw-by~rJ$G%DyIlqE8O*kb^9LUc7et&*cOl|ma)bN8jqgR`Tg$sb^nkwFwcOafFA=hDN0UF#{9!&627un@b_XYc ziY%i*Ih%S97t1)QQfQey7e}9pfoETnpu^|U0o5vpz7|| zslWf5VOc8aN}KqDJ{|W*00!Fye6SAjcfJ7UFp#Lg(x+C7_)h!q%BS)$3A!>|gUgwklHPI;O&T{pXYKw;r z!j%P&*Y=sJ<4tA!>L18WmwJ`f(vx=R?2YFvo96gmu-!L6`SK&nU^dH$eK>v>Zl6CP z-sXTiqj#VK8>tc9?$gU(I%50BPAU5Yi9;Ty2DgU6LM-N3f4+Vj-gF8t7{Jtm`KWSs zA7X&B+v?%43Xg>F5RW`HS_ggyvpaB0_Bo>|ADQ2)vbOqF0HTG9vQ$NO`|X{3jBSR`ZpXu{YUou0xRd!{B;xyVW2q~z}oQqtR- zrTBYcmmp%2BN0=FG(~+qOo^{g$mH#PIRM&JvxcuCxOb$p?JzWZ7!A_*1>yHr+-X@# z1h%I~ptdxWQ1sI2pqxBh=v#0gnp`y9Q7FDY^Z_nevp(>$8VnbMrueiI2Z8gTh;E3| zUjQ%F(a=)(>eI+qcah=td)-rFagc&j!_9Yoq&(3X7PJXw9qboM$9=>g@zr#XIdqPCiY3vIekfsp6QR zmpP2JvU(UtFT>ugmtW@*`Rfpn6Z(_DWbyq4L?0xqM?^J(-sFpVEStREgTAv>21Uto zpeF7d1BI~ilPflW%H>kqYqH*!7Ie7KVJ6&y8R1m+XYW0>_etBUl3)J1q=(Acm`uBP z`yj(Axhl@?fm@J%(0W<>0suI~N=R;?oGxAA$Rh@o^j<`cD}Sl>2UfnK!roRT*C%8t z)h4dpr{nTeT7mlH+~ZR)iuF^LKkL{D$B!wq0IIV$iWKuQmcW?Z%|?PaP*-}Y^nP#& zPP3?(R9!EGE7oQTZ4-oDQ?5}>lE&TakY@S5-4q8JWTtgJM!tVm`833FKNc_5&S9Ar z%+E_k9V{FKEoR2!WI>sG5pC^xhPw)2y2{9`G2xdpkrdnAYY+XC_S*kS9ss)guVnOp zSuOCte?CzK8PQMr77a|@xqmI9iof~oZ*_>{J!!H^*%W($*z^-_01f#h6M{$mzSQjx z+qEr;9}Um{)0GYFeqK#aw!??d$}CwV7yE3-e(^u~s6O%Fv6P!~AF^y8;aCZ;I|>)R zdY(de8jh?V<-1z z?a*y}ZCDN8`pxmsY3-?55AFwObN&bYoMxl$KRvuZ6+2N798cm~Wce-PkyJ$0;oQ+M zPyjz8O4e4+4&Ivw3`<2oXz2r=aCz1ee9nz$@md1p5-PoVFSon{r?GJV)dy3M_hF9<4}_GAO)sfk9|kq+UU5Q%xbalMBibvPyPrK& zDeLyf9-J7pdJlQ^`yAE>MAKd88ox_1xu>Ka`N&QS3kf0JSo)_8Av-#965D zw><<`a4gqo0N5*cPZ*HzehR`~`8pm$@#obS_FtNS;7HM&wLSF8xl2S(s$(>U0@DtdlUB0=|k=XC;&t2DDSs*L;fK&n; z2YBzo7g=#P{xD#gy5<14s;+TX4Q`l%9>-2AzjyJg%)Bcj0F%cH|w| z*V_LA3$DO>hVD|m=C}~Kn?Q7#9V~`gj$v#AS<%041$!4_`op>cn{zc?x?qWP(#(= zYX>+}E-o4=iJU_b2_SSC-m8rBcCDeYt1w?_vBf9i$>G$|-5gr&eVGG0Tn8~SeU47v z=ZBxkkZS@$hcn>rPK{0Ba1tQ4tj+BR$&(l&9@}B$<*jiu#~q-e>`M^Kq;a=@zOUC0h4h<%ccK2{Pc;L_a{@QX&tl?A3r4&J zgLfBxZW>Y59DnT$$|PYgeM_)P*7^h4T!57BTXB4EN&zmSsxS2Z7{eKSvaG_uL6Z8t zaa=V0I<4@x3t^Wa9}l3DrH-=NH^MUH+#JnO&M z>};m~0%ggVb5-{&a_HMrpaa+1Y)vy=1(}hkdA^`U1^`u^h(c7@NpUKa{#os8dZ+Ni zMv|)_yK)CIYe47odO7zNv_=hBK8<#R4cS$&uUPt6raRFIKvBC={}ohH8pJT3gL>Si zDtI-zwyUT1xevKCTOK1B+>0B6`tRky7;EgH?9wEN0`H|HTev(P9D|?cUTm*bJnedU zX9xWj2%%%(@VZ2E1oV$rL26iWZwpR6`(k$PV%X^14OiX_XE85O_tiy9jTTuwDbMl+ zgSl~Fr!QhTMZgU83sA)uHs#CSBLLP~TK_FnQZ!tbUF2E6wKMourvCD3l=~1vspTP( z!9@HDOQ1>*##*K--J3WZys!ngZ;*foLt zNWJMzM2cg04+5rhxbqaE&`m`7 z!I2SLE-(ReT)%ncoU*N@gl!-NF3&+TUQ`r7d+TejIyeYXWcXR-Y_h#gERxXs@})&g zEg0q1fX1}vz2~#}0uRJa^gNi}{>EzJ5nm|L{sG^7u$+4L`-@k@_pcc{hMD>bKU*|c z&*?j~j*!ypOe*|Zs@iXfI5F;4;$Wn=pzwRuIN&NUmOa5oF+ke)Eh-G4$3 zxsn*8a32Z`p+Q3IeI#Wm&1ZO{k@$dg@?dFEtiop7*PgY{t^x0k z4J%l!&BK`hH`1H(s=;V^a~LH5?+`WZt(12NTNOF4tkC~OjNHmu#007iaqsE|fZ{3~ zb>vZ^eHM|hz5`j#Z;f#*Xooz&iNCxK>~Z~>2hV&y4{Ar{_|XR+27(T-ecJvO8&HEB zN%LM`{Ojs_7N^pwRGuNRfHid*9qEn9ka5tzen>s8C-<%8YRu_bH&)l@kKVy~CF6OC zdJDVyqOYu1Ynpb?-+8uw_5?gbEWmHv1Rd!r4aH43)N3sE)+(15z~4iJ z!G@os!!Krk%#8xi-zl6YjUYWkwS(zXUn%r{HOS_>G=(kK#7pvh{_z;(-y1J2v{+ik zBG6o0bVlS4W~;$zl8AU{f(SV^@jU&T2hMT zHBB$jF#8Db{!C43aRa9;ePO~vnBDIv@%9cFlh@j8Npb?a{&4t0g>n zpaQ<-VurKlh&eTJQz3Gce*1MF=&`3w4)C<#UNlSGyXx`VA95ngj8t^M4(b>4K zl!yGde2#IkHGaNo-KamI5&To$)q|u-#zfmE{R4!;nXmyk>u)Yruz@=za~UH_Q09<- zejz7}aYejYnfYEKic)!T`6bPds>X;$@?KmTv+y?Hp* zZP+$iNR*7FsLXRj8CsM%WhPTm5yfhmLWa!A5K+mzSf)&wtF>ebC5>3hm?={kLz!p$ zx_O@W-N)YV_a5K=Yya`R|2&7|$y$EHeP8!=o!5Dt=Zxm&1+5G3bsdbZs-V%tHGz$f z9Nq<~!mNi1NX?q&e>D1Y3vo6CjrQTDqOyKt~v-x;cU=whx_ZQ2WG@34 zhrObC@mQVdn2Jg;*VlvF=VP^NNk|e|szqCuSRAwQ=e}a=4FlC|R?hKXkK>GgfSaNt z92>Lc{MY$zljsa^Frj_hAEL+ls+V)NDcYJzeJnb94z0tBAbDTjlrs}`HF4b)Cc(?6 z{Wn)P$~L|+>wHeNaTbux6!f=U9)j8Tv0v>l%lx|@*C+E?C~Mkh0Z$d8Ntc7~w06(j8 zLax~PBhEoBqKa9ieSY1Iuo$O6U-wTc|03UV42epzPU#m(v6D`+p})A8KU0(0)_=L) zqUS7&&d(qIo4$(qx?1w8Dui9kuc3wjwD$d9;S1AZu=8$tUR-bl$>ESk4^)A2xK-Au zRO%Ong9^|MKfRL&6yP{r^Sg)>CqX!B5BdhP;6~o}ZU_~3UPj6_ob#=pI?TNrYl;tI zK|7WhuaX{N6CdpRmJ~E6sJI-an3Y9H{B>a6W141G`VrzuJ-HAyy| zF-%&|xo*~#$L7qIfY`0qpi<}S-dNULePdpOSSuwd!q>PX$tDrU9%rj*q#+fY z>02vxTW1Us88^kHw+ywyHJ!c*4;Q|wGA#SP8h_t;s`~G~Md-M7%uGDYjEXxD!sjU% zmc#cVXT5m}vH|BWV?I@w7KLj$$?l2tl}M*oLw#TD%TP&&an2~d^zUurm)lgCAQzEH zRh!rMZRU$z9boVutzN8}Uj7XRTn_Kt-1M9E>+Rlc3aJpXxVw|iAaEVZQ>c<~w14Dd z;`T4kAfhurw%`b=opHyh%>J2|7^Raxb49#n?&s&t#mx3ufJ9^;0Qu_8e6YQRTBrLI<*mvZ1(4qLq^82bw4QQh zd_*l&vXQSvV0C`b9$XFFktjip#nO3iPaIorR9u#=#dD7zo*$ch_MRMsmp`v%ltXaQ z?DG>lXqy z-la+_mGK+W6sL<8z9o=OwfV5hml&>Q1=rN9hDc*bY13pgOZ#JpV)vId|&D6upm4*Ut$dmTv zR%+URlPxFeXTpL;OM^JAkeD#tX5i+7V?|CNVqMX6F$iNf@3dob0NL}d=GfTca^r4< z?B|f-dcm*!jBgo9)GiCUYBHh|#(c*(Ax_-ZJuYhDTnmfhyJ*ppxeJS)gJFVeE$Wn% zjc21J44b3RzH1=ulllHeyovxTsuzaRQ#1)Ufy-5)bb3wu4dr(z>8%)C|A0vMg@N{+@zRq8QI zhMw_%KE^4pt#nU&47TUoQ|)@Ar0Ae-q|AL9r0LT~R}aIT`rg;w1$Rmf^{aG$zv@AR z>+O*Kpv40~sffcpCHL($@OwQoiJx>IFVn!FBHW|pdmCJE=k?rTF4^ib8$jNi-WQ#} zK?sVL+3YyCknTU>!!)Q}tG((E2?~ts^(yikYKfG@oa76uf-#pi90M_ubUvwzFFljb zGu!k^Vr-7c6V+nV~P&tR%M)fxaa;_eit8WSBq$->~O-nCi*M(FW%R3+Ys8+ z|7;t|)lPdPDAP01?AHO(jMO*v;`^9zspnFADcH72Gbi1nIINK)&N{OWj3N^2NwhT! z&Fmt%grGJ^6DR7`=Yl%TRAU>xt4#^=P$O;_n8^uW;N#BB%Fxh|ET7v!*{FodxXZ(K z$Me~|m@lo|%EsHw9G>WyR%<6%)XVdKn^TmV3u=0P%ItRa>s0VL;cvRS_^m;1@~r^V zcM**;TyFjhEaPc8`9nOJCX(b@&+T*h4L!hNhNr)Yo!4T*{t8tcvkSl+^9_B*MdhWk zjRQv)9hlcK3nM-|eAQ`)5xa}>mkY|Cc>rAVVZX2>@@;>eDSvJ2rX~-0g_F{DN5IML z(=9fA#JG|^lt>=BkC`BFUxd$aF&;zU+okKK?e0()vg4!L_BxxqysLkdW-e?S+0#oF zd)m!ImSYeYEVvzV1;IZ`1qf8S{?OXAI~a4!?uMp`ILg;m+_=69ZZ9MSw^yN#`8mo{ zPzhG6!755#1*vMz3lm-~`zLX%DebwZ$9K{WtUfy>-G3ZiJeR%+#lYg-`;;-#fOa1I zZ_&;@mdWNP(RGdF}!$8^V7g^SfpSB-XT-N=_gA|O<&mmNy zZ1@r zC3Hjt=A8=k34hJ$oLO(#^UfUIx#Cb!_)eAOKzaaSA-`+tlH_GJ!f)rV{ixHCK(c{%igGQc2 z8#yqc>;cd6Uyl?w?vw&Pf)$&Zy;c4zZVC=u50KBQjnq@<1qI0-5KtYcUZ1>G32s3r zI@Ya(Zvapq_x&3^O`Qtd_$40GAMQ|z9ZoW6qjNUNbJAy{nF96b33(irI^ zgC+tga^Wfm_SyD-0Zd%&Fq8ze)i~KG$2j%djq9-NDfAl-z!Zy%Rqa*9I7RL4$j| z2&VeQc`BT95InPwQ=vx;5Efml%uggJ<;xY<$MhVxGMk$9SKXjZlYApBYkys5vETf$ zQmCOx4l6+{x}(EzD!lTXNfA{cjQvnt{;b!7@Ffr~AAodLSN!An#2lQV-k(S zYCP+_{rJpGXp+?fVRrKDHjL{9&5$M1VvIu{le5j~joizRFJ(X`K_R)J3x==-y!}⁢z8iaDc1DD^FrUeU{0X@beF zQAvO#^aAENfQ)??13A~yOcbV*YSw;#03)2Sju&7x}K{@Sbm3?6k z;;&%OtS^03beC+92u#I$KyJ5E)yj9|9rHJUQ$Ahwo-CPySF$KMA@k$TP~ESm55g9% zMSo9ntQb3^^|zJTL`>EQ$$t^363k2Xzwj)UXrwbb!$$`Qp`C?5F=p>%OSKhXvmL$- zBy7SU0{E6L<)$=_fvlMaQrdbv0crtb@g_we?&S!8*2JA$o3f6C6W4pF58L;z;#WpM z@?b&_N%J{EqI(;;zC1q9v@X;WeQ~Ro^9j9{|90dtNajA@3%}LwzLg`O6vBlGatUGD9JCD;SGpq4`};%Y|5@>u$@)2AFc*4sGv@yg|mSz$CJuN^h_&1|W zD0d3kpCnzoN4qxC2jHjQ=iIYtalF(TxtG*^(5jnSn{8UG)!#MqfcRBG%&krIIBnia zru=kK&BjXCx*PJ6cSdTUEi>IKq&F0M@9cc$HKx4Fa}kqYlUMzuWFr%Ldu|1)5PiYTD>e_4vrP@aPVB8I0D{87=`!nLKv}vlz*}FQZ z^knmW!xUs)9EXm)rlDZA&F;6yn`d_~9 za+|7{IPP!nbgu7Bf3u4J6*mq-ue1cFr9SoW{<9@(f!=rag%%enVpyKLubo$1d7>y& z##iF54ObGeJQV0%>G{Lz$9T=#&Ko5!=O+&nEM*V_*DDU`3{DGHZ1#)3H1n%s>nZ5# zvjaetk^vMQ-+%}vMxyM(=9_*Nopt9}w*Ao}7?zWrQ z?G~Vv&P`ZscJ2GZpYsbp@Hx700n@@<^)}#fs%7TBU#iVBI$|tFdnB`KBP5Tu%qX{| zEAY!7QPNstHNEj<#onRsG1wDMdmlK7WBs1iI;%HvW;o&cnx)tYg6A;47qr@ju}+16 z)Ny+kbx82qK{LcHkooH_`YwVwagjtYaKrq0LuuF1G4t_T#`#t2I5U%6*WWKbRv?A) zbwI9N1EUJ#F5cSDBG%Yyr~@)(kGskdnBCORa=o!)rOAI|=(?PcLT_5yZ2c|OVTmD8 zAIC*Gp~|%eiC?snEzG?iIsJQ@MCq(cNG6h$(v{i{2F)(52uJ*=eC7i*o+5j5tIg;O z4h6AcZ9Tg1{)y{{1}{~P_9tzw)@t|YVKyAER#WP zD=lBKxl(;yEyjzNEO+@|q0j)-b$y6)^V#J3fiEOxtl11+W^C+%ncShM8}0Bt;yLo= ze8wt!PTqtTgG-lpLG=EjHqno+jZXf5zbaUBR{{F(*W%@H1!;^fzhjPE5&wOib;j?{ z+ud88E#zz3GSdKklhcHmVyNp#tG~qk$BG35viTDOiIvbN6YGc+50%s?0VGwLV7K7} zjiI)bJhrZQ*J)5U?fvNN6qnX#sMGPy{HED>K7U?UeUU6Km6%^tktaZ&b2X$TOKEt= zYCMt=48ybyfJW#-4#^`q;nZUNuuHqbZjNy*ABxrMh?k)Fth+*Ru)*Z@?NV0fxWh+x zaWnp-eX?@=Oc~YJeaFXUi7`f+4AhrET4Y_D`*WD=2!S|YRal2VIfjc(OwKm7^-8$A z(V~D|;T%bHXcZ-s4tDg#pc3!E+IHs`eiF)a0B*4)?bLUv7kuV+L6ijO0A z^LjX(O=8tA=+U^wZ=;HrQ=3JPIwTegTG`@LZDfRVr*DQtKe})24v)fS5ANNaMwh6x zL@3RYW$t!_tSPQ`Hm!*gZJe`p{=TE68n?Sb!n7E)4*yDunf2nC%69fT(~hq$mPoi|OJuhO~r5+Zsg3q$3gz&7!=M$FkYXx~}VZ z`{87I`YCBYln2G-+YYw?vN|bsTI`mHCLxyt!SvnW%eB^WvLxq{x>s|yO%yIwDlBJZ z|AJPMZC);6Ar57HYa2FDih29KdX$=nHzDA{ic%t>E0cJ&ozSWv#y~%4|7R!tP(y{? zaTbMRk3i$oV(I&z2f$Eyvh)_-Ee{PnDv7}DTEFjWQu&F&2JL63?SuvkB>iGSl>Vtt z#=MJV3_I-8Dd{n9ZpBzwoU|uZ18iQ792-(KnX#7dyu<>J_~cow^REfaBzne?j3mP` zEgZI%?U&!}GxIs*GnmmQYBAqlP`*-Cnizgl0~cu+s8{P6sOMu(jvBuWXv0&zTDyC+ z0)Ji>n=9opdxgG(tm=blUM_#JnaDDj9V}IKAvt$NL;ckx*v%{J&0f+&+V#dI1%?b) z%fLH^{g6lsv&G=ix=zTLC%W4rU%bDMoLR5BeC7|sGGW6mvD2HaUA?&jsduDJ0|RZ` zo91aT%o49Kf{|36>4_udd#LiZ8~$#T*< zIqx*#V15s;=XW!8)f25DT}WEKZd!*=p~!u>exHGJ3VL`P{<4#OZLf<>%JiD^`BLIA zwoT&lg)=Cjf;i=CVB)WBrQ>~FV=dVC>s&uYMb&wd$7l%o2K$aQHoTLcoVo*%tT*afSU{P!SA@?D53 zVD#T};kv^aaoThelp4!Ab0s7AxbMl(v+z^(|9q^G-LN^uh}pVtrM-0RWh1f-*~=kc zrSGe=ipPbeoDH*~LJhf%vW1!Ed8z9<^}hx1H<&wYKj4FsWX3wo?V_$iQe~#oysVkI zc3v;H^`u(kk7jdQGtDj$@TcsG*V5}ViVAAh78Vn3Rk)>^#Z!l$;+Azs8Nbftyp(36 zT#Fyp%r@_mhI%11Q46)D`7=VgHS~p32J`s1`OAgdA6P~@DHDWiIH*q;HA|QL zF1fl2kF~`Oylf0TAW8AYWmj{<(VCz2B9S|%aWxQgIFAF@#0$hJ9!{h%umO}8g-K3_ zB#(21&Is>e6JGL3-VEr!^KI4p_3g2s6*!Ug2|&^=eSOJSs3-@E*JoXSQ39aoHVgfs z#gBZMe71Zld_U%H=hdvwt^6aNaZ`D1 z54yxvVR8Q<0an{JqNV2?({HPr$dv8-md`L`-(i!}h3h4Ju14pyI7@y?@HL+ChUdO6 zD)Ko=N<;GgTnrg@PorY4qz?Y+>~A>vH*zDoJ{HDnXi2{}Xdu~_c=GzZw+&&lx!K=W zzhYe#efwVsO4a<&?9JrS zpLXY!>Mu^jC%vk#m!)yl><14;ckm}8Tn72d^&hu-_uquBmfS;zU%+#-LY)p#y&{?8 zJL@{jv@sx<>>RfvSd~#?o0V$w9Mnx|z|?>do6ffVNs5brHU_u=;`Ho@b7(nhC@~@9 z^0jV26Kj7OFg}xDNV06-v4hu1C&z%`~^D%*N z;sZ|EP@+V5f25b98`{x~0`d2hg14boksixjIjRWeoBU*oM?AX{0lIth1j^jBBM@6f z&L6QV+BGOTJj(iArp`1#qI^G<$UhTH1N2&L+}C2>=))frhG~+ut2AqO0A>54449Nl z9rv&q(0m^`1~{k^4$wqvDW}zLBRmJDo*NydI7gd#0J~F95UzJxrT@b*Kp5%Kk`jnkx;enU)iVb9Hb7|1C13d| z*5T(lB44hD5@XUnQ3rfN_lE?zjw*=Gn5)b}5XfUuN;8(f*UGuY=1lE+;Iw%esAB$f zA{QtTCVQ5j`>WeFLR#c;%n_}r2zV*QdPJ*q>Fzs!Y4=uf97I!BTnNQFTU`pR*{WYx zJ1e=#B8TNf=t0NiKSjX)RUVUK5wgu4mi6#&_ZfSnz)O~^J#Pt*7g$I`Ev!PA9)3W; zjt4xPgqZ%FN7I`t^6_?tVquZK9|)<66sGS@6D(z1etrtk;|JO9{ z543drOTN~4>aJ0T26lC{ZPUE-*C4N34YZiWPy0s|M$;{l3(s2-equR zRY&0}2IfHj8<{m2F#3KW_as#W3(cTtvBcgb=q*wqsY5(G|I|2HjuY+soLR3MRelwy zGX+wz1`SAVZsDQ#j>d|pzQF`QR#FD=NSCxCzvMC2I#Uy?pRn~;AQUcVGA@84 z7~J;%yo&#M75}pNk@W~t`ln~U(!zooc$Ls^=#>Ym?G%__xC6}`fBD7}wVtQp z$H!lFJDy$yN^qs(f$%>?9=&ey2%0c^We=u9u7Fv`U%nQws|SeSgTp{6jPtDb84Trp z%HpfZyAB zNaw1CAiq?lJ(P(q&~)*vulJd>v$&1gNtdC|b9s2+EN=kRDetd>o99M9NPZ6@;L9&KwQop0PDmtq}$NE+K1p-M**J2EHoq_P|_E3e0NdlgS-3T=` zbr1ZXErNU33)H1S76Fc6zEE)V*lf~7am^2nBMJaJS0{!N=e_~s4XdUL2crMhA$`7k zIGG76Gn7)^1{%4agmwf7Z6DHJ9kJpSqfbLjZP z^i3)vhLH#OipL;+AM-Kcv03O8VZt~jgbOjKanl2o*Y>*;K>2htkOGf*;wgv=3_$}W}v z3+gR;OqO)4m|o!YoqN)ks^$H3tZZQ{PY+?mEfIWKsFXznzRoG2<+&5da`~vT>IIy4 ziqzBrCQ@VAceuV1(97tCQrpoHQQ@uUn!u*A|5dO1ZntC|(r8Wk{Rdp17YL*|C*V1`gPqm= z?mI&dd*d!UjtL~Y8Z1Jec*Uy^Slm;{TBD{8pzIN-{uCC&E!JwqFBhab$MKPCeL+ro ztf+jL9T>>+A0yaUbY;bYW3*fRa21c^)Q-QXV@~<#Q9YY%R{iIkBn<6c7Cd?Chy>bAZ4c9|&hV)r%4%9v5BW^^2+yd5*PQKO^VCdpxK0J(3~-|S*Q%hgD;*Vb}w@$ieO#V#2fHja-3D2B5ptmbUA z&4XRk#|lYN2Ys0D*xDtsVbR=7ynXsUkqo@u^y}G2sYe%(18+CXA4Hf(!8UU%sR(^& z^At6g6&_P%#r2Db)vV1_KZ?5V)&CBr48|UJl$M$kmZJkVM``mDuU-;AEPYx}av>@z z$b#`jD3MR)GjKaW9ShZMFzopkSI)!e@4$%t`>`yNm503N?I$%ZeUO$$ddWXrFNf`? zJ;1L~yx8Zj*z$1VU$VROg$(ORaO2X{$+L$z;@6(&s4q@tz3O_wquZg>lPN3fe%G3L zJw>dM`AfUke?Q!gQi1KaxbD23!JlYm^Ien-4Wy%r-I~NQWJoVw%Hg+UY2gTa%023W zrr8&v7n!aQn_Aw%vf;VGXk#V(oY2n>;Jv7zuB!TQ`e5Fh$y$~dXW2Vg>3u`nZ;plDS3614y=xI~Gr zn;8#n^&;syzWx@Ie_^Ph_s|F8>lzFTyn-1lZ{eSMbQ>9+)|M8}!AkG*`cFZ{zd0n& zY8%gBK^w1GCP`s)66p_-id3*DR^8 z`ua`U2`Fc0n%XgYagU_{Joi+h9?aQpUBPZ^!B=Eb6bTV z&7!f$<{bBkr5uXS4S?PhCdHk-J;3-W+_Q+4m+WGh!ASLVd)911=TtRx4|))<{#LeM z_QUCSEmAG5O7(}qn93^EShDC|&|YilWA<^THnf~oH@~%DCf?)E#-24)?-JCO?1&+S ztuy%j33U3>l(&cYjYMZyzxfXtKF7CvyakBCgR{ZcnKhymG?~4-#3#w8r;p?i$Q|KC zAarzrry@@7xBGX?-Gbbwj6kZ0bnA+GRFKuh8hc{~wa8JUW&`6!b**S*@puYIG@fsT zzu{vLbw+VkpWy;c-zgxO8%UhF%N^74j`af|(TXzDoCjyPK*!v^IFxAr8Vxhg-B((w z`nri%xwj=XTcIpnYX^EebRs(Ul<2qj*O8p zSZ18Q2pY;M!Cy^(ZNt~rw1Hges^&C~kvG70u-&F$EYa-IA!zj7$9Y}{o^!NfN$Yb3 zX~i7_;5Xui=sLy4GF1#39qkFI8cp-|SsYZoAdY`9-iQ`v6vhVfWVtxz8uX zz8N)Z%ao|Obx<YT?EBkAIPL6emd}Ct$vubRzTb$w*adP6W z?33!5xN~XEO@0fK$fs>}aeiwyD-#kCt)>SANJKUv5#g*e-Lzs{g}mv-`LeVHZt@V}&i@7&v&M6Y@H*B*rYkaTs9$oqWI+jlyw1wI{!8ArxIj;_fxWevo}aV6Xj$Q8 zqN7sgo~`C80x|gB&-{Z}ujc_APWq`EGx0U8aVKquW5?4P=cQ-GNas&#cvoqtt%JdSYjG zaQ!?yJ`Y&^;%XGGd<2>a(=a61V#YMNnw`)VE-BwR7hkhxE%x|F>oi`#4>+#g?>v2# z2oD4K+5i&~(Y^tLx+`1gUDKz=UJhrQ<3E?#BQW(J9~v-*{zLZjsJc`;#NfXaSd$)y zgJiwjFK}5u5Ruci=vL4=&1BQ-lCU`Z(~jEkd!Tkb-@K~^?aN3~=3j_vvz!yD;3wpq zeHgc#mEy+4N4K-?A z<=?1nz=SD@)wKSwc@P}cl_{h7M;IqNT3S6{*tOx#{4mM}zN;RKh-93%W1$wo_JGl7 zda1&gm)Was0wcEZY_sn+djA@_eL(>JEI0C~y59{&jaU2HumP&VKSr?cMnx@u9lt~S0|1v6OP+a?Jp`!P-l}?U7te8a?^R`BB z<-Cek{rPu+jC@l3ug9aij;0vo4JGCZ%H~WHyqpDcVFpUP{owp`te)il*i*vX3l1$WwW>l)v8c%D29mw%i5l)|4v{qe0`9R}Mtb;|@68If z+!_rUBEeZnaw*1yhftXARg8@lijgofh>!|&1R~dh-m&@JH7i)4SN?wO( zTs`b!rXn1WwFxz=K#i$?Ri-Ujypu01+teOL;fQ;^Go(lAXHDy^jTtnwu3na$2+0eN za~Y@25kd1vD@!4GTsl2OgL~Tb3ex%d3_@QOJRUfF7Zi&bC^-bJ6f2C&0f+&WsJ^b$ zipf6(3Wm}w1Gjk~4)CzjtGqd~wW#ssjwzLI3+qg$_8NR;QD?@cjqQWqxP`e5iZ5xB zsK1lUl;LUu-&jx=F?^oDw(m@1h|)i;L_@*;>ss(naX01U0-(CGI3LV}Jvxvf0Sy~o z^jUe8dWUMByO@k)nebmN5XoT--R7^0t~g%3M9G8=PPZ>(1R0a_dWu|RuLns<;E^GV z+84uT)m{()YL|U$dMpG(WOeS^3yMoyG3#gbn{UpvV6}V9G1YhBuOMm~FTxPNUyt^Y z&)P~19A&_}^(ilW)%8WJq}2dltQ_D2-?UtIVGn@zVhFVV!9463D!HO00#|R8@CkSV z+knH<>2@hb=n5!9SAc@^b4FDL5h_2`??wswfRAKRg(r5DCJCqM+QP%=M2J%4ntZKImtd0a8@5;~i>&8ErKM z4(ygD#sWIZg)l)7D@);sw_X--y;3MwuMy z-j`4NEmj8i+z*e5ht~1y7wIT*Au(oQHcwkcVehBrBcqMWfX#rHDc-C zee{mMLiZZ*wB65hcxL{q=qFdvhlX9CFx%?^wB6N-2B_H_fdb-Y;qy1x8|vB~Bq@MR zMyA43VC+0TYQFzCa``GBQk{gywTT`V|3VcEEKG<^fBCV`*`)L91!tQ5AY=*(GA1_< z4q9F1(T~Bb!oX)Qnl=|D;;|{6$$+~!zCUoWpv`PV4>%}`C>r848jz%f-`Wso2YFWD zI4q#DK@GeAtr3-n-8SgYMDOp0fCK=i&Ip?IKLr_(VEc7wo4X=#Xm$8h%~%RN+`2PI z3KZZj=C7;o<+=v0Fn@D{%i?lDX7i+4Mx7g|)nG72`zYkMsT&iPyTkiU5LF^I%MD(M zoS5eo8@%(vUm|E^d;8>MOw_kdPaJQ1{UPMH^K9Tcq?R4kRZ&2`TkfLX-}m*kf`+*SJV>u;9zL2}|G;$p5SJ<1QZ@ssKgdXJ7khJXPpBk|Wj09QZtES{e{xg~QKmrb#2!eqM(JgL zGL`~fB_&ZlmYviCt9?4|k`=87?6{TN%7n3Z0G#$n5Vb#&PHZA`oO8p zwb@Fz!0!(gHc<((-%Sm0aEbSyzuN^P{$-IcT&5CsWfUoFXj`3lF zlSl2yl|<@*P;|NH_E&34fpAZl#C%OuC~@QCU3R53y6s43J?i)i$4WmQO?VM9R2Pmw!E9_&J#YHWAYk!}Z+_&(Kb3@|AJrIh> zh_?pKn{fYYNd z57HO`0(K>6LRUhSBB;YOb5<9k^On9X4|Ifm*fmB)TIlLbtyS9k9spCdKDAO6{Lr?< zn61Zre-(a&S&nHaP)^ib>?|OXyIwtomV!+nCJ81`ljF=}azHu;`0lH|4O7*tj;E~Z zLWh4l7FR8}zz~Gfd6)*z98*XtOb(lpdm%3@)RQD=HW&vP}mnJ-9vw2DQASyL$8PRM{c(N*QH&!VE-#E+`! zAC=yvji*QOec~P-eI7sXvQ2F6kNuk()|v`l4GJ-i-g|RUe4`?qy{$LJcWfx%`)(JM z!~3Z=M*KHEK7QA%_i^NL&4r5#x9K_bI~Hyh%u^xj>{8hMWngJvfhN5hp|ZG2uQ|Um zKQRI*2pejncs#2?qf~Z9E~hHTwH|uG)v#& z0N`o^^z&b?Uh@1E+Y3e5fYj9cVl^_}N@`~Fjw{fE_Sl##FYcxayLKep^ z#~)ticY4n&IU%={95Pp6=PgAIyfh^$(GNLys>`W1b`KPkzX1*IT2piRk7eBY78IwYM17i4IL0b2 zWSm>JuNWM8e0mCV%H^lJc*8?Z#UF()KCr(p^-N{l8AJ>sgb6dM1A%2p%F^Q;j=k12 z#xd-cJ`MKoauAbp+5sw2Y5^K-VFGS>Vit|B!QkP&tpP?f04Vv6H1`)pX+-EpGeW%? zj}^w_ZN6042208l;lY+cxa0Hy<6aA8x?= zea?yV4tbgt)3s>-A%e}ZE|kfQiQKw?IuTnFnE;Q56zwTC`;aSI3MSLJP7hDvn3XVE zEyb611%PKlgr4E7|zp=W;TjKdyTNKG9sF)3!Q5h9N*1=RTAnz~mNV zr?w>q_WvF#MzX;l9vLMrUkcWtmbohS_+T2W^D=MOH!gzO1Uof#KsNL9LkV>P1SAh& z(=<)u4aY5 z;sQ58ZG@ScDj<##u7+fWX&L_gMWSUaOe9?FRYD(?AIzxU`g|nK5O}KeJA))oSAQ^| z`3W4bC8?&3FGJs%$TOoQtPyMj1-rhEF;g(1dm?FKfIU6&`q~1t^Oo?>KkwH87nQ_BxQn70IzP{4dVSJffQ?h*-AH(m2XUe(Jo7GC)a-79MV*7y)@N zn@gae_qupD{gKgKL?q1Ly66fr9p8EHI@vUhT%rp&1AjJk_?w<5hh(j3N>xYQ3toaW zlgw3RM0=u)wxPvB?>_oq10szpQlcE8=01Mm#NHm>6m~jbT&YJwFuAAuo{xr4ovTiR zXw#_qRZjas0Q_G%^G|ONF_K;S`9f#{1kgj0GQ(8Ao7S1Qfhz z&9nBR!In;V+AqtUfSeZ!O_wF9;|!UK>_Nmi-zx|AGLgscJvqk}>5Z=I94!`2A;5K& z4bdx#uPjVXp!REy&ZhUm@3Fbf#PL4pASBD)f(&?F<)|^u{ObHfyy+%ndg86ZMaA!^ zUw%PtRq^}o?<8Tj`vr_-k`oq?ee6oH6wN~kqjkF?6!s@4;=74cpaS-#z#r0e z>??ne9>BIEV|&R7POs5){5(`5q=prUfqFLUX!sXDk6J?wE703EwOQv}A3cwscT2Q4 z>4c7~ZMnQI-mCSJm``};$A0f9vadS;! z%h?rwoOUy?YR=}5+L+t;W)h9hMq|M88y}&KN6*|9t|*L!4}P@sd+cN)slpB{zmtG} zuetO&vpsgAr7&hv!sPFX@BKo9b@zSzi&046+6jr=QYZIml3*`jFjM!OWT4}>(zvOI z8K8PBC()k2;R{a{?@d4wg!uU!ouoKZiyW7{m{*E!Hoi9vVk80?m$XVnqU0lQ=ALiBNhSe zKZ~mJZN|d!%F7^!X8p?IxauruVR_rEyRfC*f*K{=?{C5v>jX5WhW^lVGC@|7xDor< zZ)~!tgENTe^(6Bs;1UT+OOMWBj%C-bdclE^FW4B=t%A||UYF4;t${7E6ax9LK z9OW+lK{;(Ja=;GVSwX)m@@{s6@B!-}^)oGsVndm*#~E)PfD_#0Nsa}42|M}{+a>){ zcL;1Hl2YF&YVowEuJch<@WFe>k(^l5?1VcxH~#F-;lC*Yui>JyQp?+&`%$oH6u9fu zZa+8&Q=6_JW4uHQrT13wXqUx@u z9*8LBN?_SiP}54p>WQZzJW~wDeA3y0tqWMtmZ6GPK#MHkW&yxcKN{(jPDam&PO5Df zPLp|2cjYuW6va$vv2P_&$lAof5AWz%d~w0?M-1jRly6r2BAF^sdoj|?q6|{>`ZXBX_OzxixfnV>1 zIHfQtBg*lc6&|RgC2^6l7B8QA{RGv^?Js|6L(%cC*Th|=BPgsoN0v(NL5V@pXM-SO z;-9|~R8ib;x5^kTn-wdXf3JiV9nk`#AD-!`V8XlP|I*P9h;mp+Fw>9#7kEQ8P!KEm zs2Y~Xcl3Hgdj%sHDB9tT+-MwXEIl|-u-tIFlNYYN_d^=DzQ^{pSLpGw18m96gVeFP z7cKWtMWK;WX2*ywnjqq{UY4eQM)Kg_UAzHG$51-KNH&ul8UJqa?FsxW~rj#|^FkKT*Y7->M@xaVZ@i!WX7-#5W{lLuv8 zEPtDg4?{+mt*i#yS_CF6X!!KDemicB^g#)&9<=COAglICyzcSKWt0Y6?s|E4_d}<4 zpW!*0fJ58Q)QS)m0V>h|T*_J!#IXH+r{dv4ZhMZrIu%6hsdyBinpU)3&A272Y!#Q= zUp}r0Uv75su2efetT)&=%5W(OyFqPmd$}d7{`G^s{E3m=JZINisDjBq zZNMI+bbR|dTx5;eq#)MtZSCoxI7isid(e0fw@{uNdM>5kI=vVN8schRuu_0TGFlH` z8mWtr_V#mpSVSKzzB063hGvvjGGO)geRO-lI8bo>5RQtXe0w$2uan;IfPf!>q|c=o z1$E%Vy=cvyO8cvB4A-7lYh8M>7Js$xx-8X$?dy7g-7$@114)8N6-R!j0;KhYXkBp%T~P_ERcPeRT*vB84o_f~f~Haqu?}L);vvmg2~L0QX{C z_S@tfhB)CZdSB@TgbCVO)4z26^6-iS7SOek1m6h4XWHO1wk5lpr#S~ieK|Ycs{|1% zu4OXO{Dg)F>^afJ7FNoCxAXr_Av#C96;;qn5#fbUQ5Zp2GOh;k(wvXw7F6HPyRUOl ziJ(Idc8-&aKBU(FtP2w0=}V+t2|6BOPBvZZv4 z#z%X2csyt!iBPEW!_PSMAlYq+N4W`jH($39UV`z*@GeI6*zM=o)){dXjwX3@20z<< zstYb{{5%{5D=#v${?GX<{{#$W6fpLFfmk}gb4O{A2W1fM8VGdlTl)TTU2}9hX>K4RkMKPm z;~KCJQlwZM_YR0!j##l%<#BCqWUG7nkanFYDwL3IeTFeVJ z0eDY=Cw-N;{I>6XX@s8_^%K!2R3>h=NW$bMau^X6xsopkFV3r3yG#rgM{+oh@fjGG zq2vDnt%~t~?osHf4z&pI%Jdzr#V-r%v+W2Xe!*PWOOcE6Ixa$%LOvvn+ixS_4E(>; z;Z6(BBd2RkIjY+4o#2HwrrcE(E|u-ep%LU!g&ycXfo>C$6NUH7jr@>zU-?l!xo<$U zwCH+2<1^56Lvi$K4&%drBhik43Y50AAK-fEhMtbrSt9`MB!_oy%^AXCmK;|2_j?&b zL8oC!%KTQQ?8zXa)Vi5RKCFyKG1T0VTHDugMc0@bxMJH2p9w8|-lB&ew1bE@uP`kt zL8`LAkP)av+&-ETr7K6lUf>fi($M#uW4Rup^aWOfW@n}IcUYSDIiGSun43Fpb(itx zp6x3)%w|o7FPG&<`A|-g)ptg9B((myLWcbv!d29ID3vkc-HH+P6paAlL5#L*|NWE` zvYW?f&HenQ7r{m$46Z{$*3%y>0qs5F7wWg42K%itbf-$1@b(VV!JD~474;KhWnVC% z;(?_qmnlk!Cc19Ysba!E9B^iUgT--ELXPJDV(-nvscheW;gX7!v>%WM&mF;hY!W9E=q$}EyHRmd!v5+bw6Jnr*adcOO8-~HYD{q21m``CZ%-+#|j zmhN@m*L_{*`8huWKcFAz(drcjdBDL0AkAt;=v9e}rX(+vVKqcl()n8h3bo3(eho0O zxw8jIywZ2y%u85{Hd0tLnrI8QpX7LW#On?$?1)^;)Qg9QIB9ij#V8P9R5Q-d%DH=U zO~2$#Zma?3oUpw#(UGz4fB#>vHWND4!|?`B9s_jtv`}AI3p%&<3|o>~ncd^01za!C z2oJ%uD_~n)0hnqVn6T?2c>J!Vt?KPN9Tgnxt_v| z8x~n3G)ML%rxccK1^5E2mPUyrKH<@E?Rpv zJ%7w6n7+)C&%6RWdKOqJ2wax-cQqg!s$Tr)j&=Mt@W_Qb7V=781SVi2+Pb1&ei{Ema?rov@FgCm*A@oA z&@YV>u#TK`*X!O(v>|p|0j@bRbh2O4yIO_4k@3=b*7wQH&voveCcp?xuquRo8eiD|~pN+fb zXE+TZPM~fSj11X0w-5rn^%6}0khi{g;|y?7-d$d7l}T7$6M^AA!R;S$F*2spjet|B zYC<}SsPQnC^%#@LPJ&@VuOZBl2%hksS4=|P0njsae({^bZW7Fw96{F;ief0-Juwd# zZIJ^yNFS7wZJblnfxQWWF>MBL8=JqSoI?2%gL%nC|cyb1Q6 zJDJOmG)oef18&&WQ5xFS>qX(I)(O%M33k0LXHNVEHB32RJpQx!A5Wim zul4+j5ua%(hR1(mR_Rtbs3b+iW{`EF(N~Rjh9;z`@F2}1 z&S}@+*nA6VL2x&4ByhxgReFKaK@C}#Q;w| zxyI8-ofha6eKOXm@cJmGrN6$q8{i@pEsecB+u-I7#G_G1r)^-I|zVt z0!Vz_5<}*>5T3NUvi=8+67gkT)2Ago+=fouP7;prt?@PtxBwi5eZI@*thK`xI12eh z`p;8H3Y~xc;|Q_x?x|et(O~K0f|bZ(qnixz+%(bl;rPB!>1>Q_k+}PM86s~eSZQ9< zi6Z7mXwA~#B!mlWd@X%9ir5}2K(8FvNFL|^41O!UWEx0(RF$`m$uAnz$;VTjZZD9w zzW(a*>)(%O6-dauqYBa@E~TZc)0WDzU`ze1M0(P(0m*D+GiZ!=ruc;R^qtgtOvACO zr-lUQB>=^COTT{W^@js=&R%LniB>O}Myy~UcJmIIz&H&yJWKvlJ834z`s_F0>Tj+h zLnLn!VHleW!cfKNDvdLySGottDae|$X?Y^-Qd#+kijq5HhH*ZH5 zBq3G{xVO8v7 zWZqAGBk+Rm(&QR4~b*XH+c3&|Cf zdDuOK;Af_BzZCJ7%X6YT*Uk=t!QpJ*%PT@;*4Rqq*sDrol`LPh0f97A5|fLXT=Yi< z&y;7n0#o*N70*WT!5G>1>Nh>sMx~dN^32V1c!#CgS>pquh1hab=d_u$EEL$%E)shi z?VGD#l+Ho*Z37B{6s4Q(^oQQoU5I7y5Do5g#mRk)>X65T%2hed)MS{Enltm9%sP%$ zH80#$42lwb0&aY|Eh2i?^o$!GACjZF1)fHp6_5EywK1{MZ610J2AXexB8;SSG&XBr zW@Q9oql_)Ww{8zvL5PPv0g-?AU0zG0Ql06$LapZ0*AOzmq8k5gNO*vUs=4UsH#us< z7mb7<9n6bzjxG{hZ>PkG0MJ#uwbJ%K8xL0Z?*i+S=e0Bz`(gW<`D*B7Xyop-pUSK| zk6V$ya{-bD?~vja5^Aq;eK;a_klQREJ!)8k7J~ogq)p`8IRMEqg#E@b`c#XyBJ4LB z8ue3*@ia+$o*nht*{u?bJ=Up%j4;y3EVN0AT}dIsG}|mIrC>L2K%0;^aGo1;Be1Xv z#z~ZrHvmTqt~zFU>JW{kg>Ivi4?cbR0+%f(d?A!K5ESHFw0GZu6Wl6s0$+FxJDrb; z{(fKWSX>FL&W+J%sT1d^nwft1Z8Zl9PVn%D&Zlb(aZYKeV81FXwqsN%u?3&0xGFk} z@w8d9eskoNJX{Srnka4{+sspFEx8QNZfezyj{PNRa0%ue8m*zEnaqh8#YirBb55@x zeGC=?C4%~>ENp#>0QVSt+B?)-HUi-ah7yik52c=S>=wJ(>GCohmOX}_c*C8U>|}=( z&Dp;n_NXjTnJ<2;meK?cc_TSd+;2MN0`iZy!dzL+l`Hq4yxod2;; zo#WYL{~n^$s8P)G4fzWutx?rJ0xDG-&owZ_y(LQwIy1L*lEda7Qc$Q z?z;fwH0ovNj^f)x)8M-9fJ~VObyZesuMB4|r>7EnBS|2c)@Mw(UgCyv>F~nUB0%GU zzFsv}IST>kcTMnrcVI#zVxit%V)$>-SuQV?g8|kmk zy;@!T4A%*&W1c(LA$b7%*1Y9Xq!LeM=hk9|#L^t>b?#0u=-ODRtB#5CB}a#OchayA zWdTqtt}3xP4&&HpSAa7(t;W_>rd>3xHLPP-2}Rl4*QYd>iVbvDeAF{HFw!#YRBDb~ zBsiByl9S6y9m%93X_^Kww}+*0%SUq6{cshdKyH15DU7@D+A+ za-ST}e3%|6_)eFv0ZcnT?`#e3IEq-?0mQrGVzQ+V?iUS7OyOwY2$nc6~QmchvK8&4%7 zeHzStuRM=wj`*y@Df_U4uTJaWp79rIk;yXt>ySNgGaF?Oc*k4OL7h*%%=9}e)01mo z*LPbHvmdCST~%0tm$gIH(cX-0Qd94Knw*kx%@9f2$|?XDuxQrM;NnZ^#|1$b0GQTt zna^DjvLwAH;wR~_TUgsEL&8MELk=;@LI;Q5Z$>G4*Bdf7di|9%1T2zr?XgG?3;y^V z{dxX)7Ag4@!+DJ_p`?@rzYfutRJJja=oheEREROaggelt2S({CGIiV~<1Twn{h3KM zD=+Gvg&I#JA|N9T*S7mtltMs3S9oJ%- zRjBP;ZJ?xQVenEOg_-^eFmAGtoe={o4Vi4$z~Qd3#Ad|BHfgCD_x8RKc#b^>9r}{SYO;Aefv}duFHl zrnR4yvhb(BH_nhKk8dB_>HQ?$WGi;#=#a8^%<>KgRWqVXdn%?c-{{b2P-mJpbRP1X zDM*7X0;{m+FV4;@(`Yv|v|D}>(G>U!eI|9ReS>~SG7${k^@d`tIzAl2lmxDjAHRx? z1(}4)bNwM@*YJQ>fxgnD50lL#M0#k+xk$Q4&-LUS*$N7>!jOTZeFH{`bEt6N!K0^7 zaZ~a(7lp*C3e03P#5>A8BB1{ZN(`GG{)ZPf^{YRb0BA#%gF8z{U9ESA9Edh%e^0?6 zmK)Rq?|@|H5sv=aP3L9F(AEbE#uj2LK^(yC({PuPIS@vTpJE<~_v^C$& z(w53M*1!bscYjw1?^MyjIN$g7a^7)K<=J_@d^3g_K8(_&G`Jx?!Wfm|76OGj^Ga`+ z2v_S_mi@}pg@MSM$9rX-Gwv?TBQc8Uv|Oz5G2ZH_`Dkcp^44MOAe%9| zR_7NSfK_V`72KdfZDal^qPpEr*YQ1A8DIn?i{4@;3GkejbLndE>D$9C*8%R*{bu2n z7Tfgj3eXWOPxRFG$_XhoB|qEeWdl%*L@3Odh4q|s(Y*6PN}nzKZ*bnwQDosLq=klp z*=GUf(=N;GKLY>wzys6yW4?SrH$R$^IP4<)Q$k0}H0bvx9JpV5?CPpNqOX8A-|8go z^Ad@}4DnytPl+%vx}EXLC?@tk^J&rS<(YAMqA0*|?(}SY$)_L66^_30uXFLE_1Z7#+2BsDbzDXcMF_+S!Y*G# z4nFt_oR_|LR}y#z(nQ-v&@#noxr4L}$sI`>$9oJc;^@ zBYPE6v7`*~k#aS`x8eKN@jD;g3sM!-mEZeyISP%5(1uJQfjaojn+le`Bw))|X#5qx z&%<0tWEgaJfA~#oiprT>0a>tp-nyMf`0|L3W#R0z}7vYuqEB6a$;OYeC zx857P{*icBz-e$}^R zzHR=?R2fQ@tse&^#@}kR4Lm(}wH(CaDdURn-y05s>2e8bs0W~VF7FzL?5}YTNcdXE z@l;In3YxV(FPISpBmkvaF0Qmd<{dIxhnJM9d-E8YE+`&|wZB6LTFsY-H@l8oYn~kQXqCxY&LA zC~XlX8$!)?eg#OA<)~r6C2T>TaIXrSMJ#7|mJi%?U(zmxSRifDCU_59kwrgc_p73m z&GrnnGdu>WvHV%>A4NJhkhT-`K*)dh4Y*ol)okI7E0@0Bg+n=sdN1nZ@G-l_@MaJt zJa0~RA#N5k)Ks)_Uhwo#uDwpDFQBS~XB-^UhPZbqdZ-^Apl=Y|cQK);xEy-%3S<@% z#7JlLT1sWh4cN3c01mF`neIDq7!TaSH#Z=Dh|GF&8;atei0PWa9ItEW7`f3F;-OWR zG#tke|K;MxQ?rQg-d^iRrT@ttRO-_OH;XAJnd2)L+m&q6B6aLhSR5kG^#i}N%a{!e z5z_h~WCER>0}9yaX3{&V`f86W0 zyK1DC;?JA34dx;1m3a|MOVwmNWyrpUjIp;yF-E#83gqV1NP9)FfVeywq1g}ASA!DL znT=vNDlg}^7JNXNZP=-(3dUoTK#+CV+E{UdqA}$7QEanHf=HJ&#E^;~w}39@_sE;! z!1JPN?%SxNLOX!vbK*_y5K7eX9YiX3bp=X1Z*Ro4LX9bT+6J*b{km1eYr?5Eo(G^=Sbhp4G=ClseZxSVQ-*{oZSMn@L{r0Sdl|FQcXoxa?(Ac9UIm z>AaoBaFVkH-;Vm?HgfYt2t4ADah9Kt-@j3$;=CDg&P)v!%-@wRV^Z#&Bzrmu*)3)~APk$0`2!?-%qLQ!t} zdyGa2XUrqG)x!YUankgE)n*p|2BN6g>iy#kE!m-cX|NHh5Vb91$3Cek#S znJ3NOz&>-c43OgW;KkymYia$bar$HIAI@3|zo-==r2=v?uXH3sJP>Yc%=K^+q#=6# z`!Yj92gc|oeufC_M9ZgobXAFiyni4;^jgvlsp}e+gkVcGkA-mQ8cfU+n;YV6#>xVg z0j6vidgh3U_xrJTv$PtuTFlH}E<2!^#@k9lRuJUzT%Mky^yL%l(0=;c`OBs5Rg1c1 zq%oxDZFE@mF4$N6Q&%9%>`h)&oP~LI)EE^tS6!F!5h)?fE3v%9=ecORk(qv8IWIjk zg<@`}!t>$eCO+&L#m-bjlNsFac62XEEgFoLZ9W|~15(|h#Xc)(AITk5kn~!Un*DBYou6F$Hb# zN|lgrVM%eA#G;?C!&0oy1#mQYx4$!Ou-91DPgSCbS{#g5b?!_wN%qyGFRs1j`j(QK z%NooB$hIT`#5lTmyZ9OoMf_ew&fsY=3(=d|m37rC4<`2yDQ{~m-@?f$IiO^l4UP}u z*y7Kn81bM+WKXu0#mP70ly1lQz2rW}HJ9%VzRD1a*l>V% zjUHbE)x&S4O{T?6Gf!@;nfd($+34Hz8S!{3myY_m-qLA+8y$J7HVbyBRjr#a%?>Bu zgS*3-(>!m12ksZA{kF}%9jc^iTiRD?fx+BT*Y%;vxjbzw0M%A0ir$mi14iiW_hf)I zuzau&Ug5rhoCtGgd4FQF`zgf7xa_pkyWV^4tFhjp_2moD`XDjG5wS)wt?Ak@&cW4N zq2iquHDwV}3P*Wk3uAz-KDVC!clQ-TPa#=~X2rSg1-(DIMBl*-LZ<3YX;g_bSQ|lA zI0@V=vy95z!>}FjiZ`-i7gr@{lp9B(fRX$_gpJ zt}4_)`V8Y%)Txd|B2<7VidvZ3(>L{R$1ulFl#`!i<66Qr7cptjIAB7V+I@GZ6gS9{ zilgJA1T1{qNqM@W7&s>=9)Pg%-M|R!Mv(R~!pW^ytyiop;|d37N{dK11xKHwzWD^E zPT#MPbEDHU3!jdCrvHR}Uc>cuBvpb-_H^+TfB{`qXd)Vf4l4yKZTy14KVm*Tkz7?_ z`BUsDE?k7uQtku}|B`i*(IfbrJI`w1}}dRiNt{D7N{#W*bjLFYp?q1?yYi13&g~4k`E8 zHr$*BK~u-6LxXb0oWr?q0QoYXj-=vJ;-cJNQ%JIFW()fn6NSfdnAk6>VNxyP3{C*^ z^Hx1sRjhW4r{O($AR93#5?M>Q6)v`DtWD>4>J=4`ckJu1pP#5s9{7~uS;*9ES4 z<4tQ?F5L*&l@~~2{y8QMXWpc3ePNU?;2e%L9_BH{5fI276#`IlBK;LSp8TDaY*es9DEiEj|)GWJv~A{{RX- zqn_m`&30EIbr#XS<{c4br?@BlS^B6CrlIIe%yRE~&6({Vy}XgVV*OEaqafTGiB3iX z>6zI`kIZO%`n??EY;3|pQf+>Tl{mpAC|O~16C&nDYK>fj{bWeJ@#%99<*4*tbUr3e zDn2@Zh9@G0YDdwK$`eK50n3pUmw2oY1lCHq1Gj0z%J+MH}VmkiRgN;J5w!cRBL>> zeITK-n^z~YdAeh(JuS98o!4%-R?nKq@ba>C^=PhnNBWmofaAq>JiBuTjiM~>HS?}k z{L$|x@au&D@3{1Yi)$xt4-L+GC-Pc3i)6iC_v6gHd&z*>BjIux-dQY z1pp*Us}G$oV~#f9Eo8)60OITFv2Uxyy}*s0tE&?ug{5GRQ;PdTO-;|)TIK0EcyE5m zelBt#9h(BDp;KK;gk{cZdHu^#FOkEyV5;HNU_qvIGH8mjN;f?|Dw`uNCnoK?a-8sT z`@5elC1Ri7sug$YHa=i~c@9xv9g8MtfY^j8LRj`)NXIE=&eBg+tykS#$4{aK{6R$8 ztRqgg$RO@5NHOR$CfSa-R;j@dN`%Nz?5juU7(y{nCTJ{}#+QH!a3XsAiXWaTLFps~ zvHmW^^EC{@b2C6UR)M%VZY6qPC>mCDEOnUSxzG8cgGtLHmJa#A+c!%uf+9={ku@is zuT{7J^3%n~w?xpaz+jwZZ^Cw_rVKlU89xLSJHJ0dFVg7_C6Lp( zE04`G0UmQoZre_E2MV7EJPyPU?E>{)PGZbwRtn(u(5EuC{-veV@c$qJBuUtdA>Pke zK!cD7P=*o#YLP*|&}QfY8x(`fHuTmqLOJJ@W4$Gx>=uLeqQ*2R5rB=dndsq_LuScG z3G@YiM7l2OdHiz4A8+b@K+8;`TF>u!<^iwT-H&0CI-ec%A|{PE-t3)4Vzz6@eh9ro z?5kXA0u1q^pKoxTI$y0vHq>zmrJ)jFH8Ub!z?*>vYe-_)oW1tC18v884{+M!zb3c#3hXcNaERyMDBtL;q{QkxF>K=ofBTZfzG-qK{ z60?UW7CUEhnRo#ZDz{h(o>A9Y65QIdQOA`K{8#=?2V$?x-5ZjU7aekeW@{)!-#F6m^OrP4AKq0MNgBFKo1n5yk+jn? zYm73aiPO3~y#?^wp%G)aX@?QmM03Kx2I+hcc&WV9gStmFP>f<66pyHkT^T#1hKS(j z)L-JMc$-k1;yoXjS@`=L1tReabd#T!pY1Vbj0a}jI_*Qej!|OK0k5~atk_Y<5?&jU zZ_MghcpcRXWIri(v+2E_Arh;}^3^~S86l;%oFlJEhjRC-n`4NCcM;phAzADviVLdG z&kyCE|G@b}=5P4GZ8Sa!?m5H$1lW*ruLRz+z*S48DUkg9`%wLlM=~OFe*0>@G;h84 zAh{)k0n{RK0Bxa!#wlP62Hv!OphAK>@<1`1;uGWU*Cc1)72u%c&-&7r1xxSe|v)0+0U6R&0xG+k@~Xo zr^5qEfSKFWiKBvm#mgg7+&L**(~~4PQv<`eWYBuekC>CZ*xhlW8m8+2hJLP(wh&Az z@IqUG3Ov%GgD>f4t&YLY=^6t+mGX7wAur#*uVrHEGb9>9<84jReqhXk!vZ`9;>s9f zhWHpeN9%{63#bptU=01s@Z>0@Mt1)efKzKLJzj^EavojOjSoHcNhG+aN+Z7+7!$lt z;ULDq69sHs$ywTj7On(gxT^E`<4|Jwq7nnMX!_6#Z(83~OcRLLp_52_D9N!~Vq`Vl zoQ8Fs4U-lfu8WfZd%px4u(6e&7w{K|(zjqo;pP=U^XUP17<=qg{XG-#=TKcn=*TRB(U1v5(q(|Dg{7_bH-qAC z4@MgfpeS?>mz*lk-wT&|*V*ZdoX`}vaH5Z-{_Y|2C84H)yAL2(L^IYZ83Aa%&k)cL z8|c2VQ}|S|e^J(h{zX}*1jgdODC<{<#vo^U4(2)^o3rKY=77HD^p~FtFI zXkZ-5KKkvz0lmRP_8rdZdBml!p98RXb@CUk5Vv8D=JVlP*_EvGBzBcJo5+u*#9hq=bp51|@ExkgnU zey-pRVUOJ9Ho!`XteY_3#WtKM1$r-4Uft@Ilpe5zGB~LZ?MA<@k-PS)s97rmn&GlC zMun+>he{lo?tegNX!*SeEDJ*xszD+ahLriZ=p_t9)*aC^D5uB&|O zW=j0K8`UdHrF0xFzk$<$-*=FFQI}s0#k(Uim-FT2@2<9j;N{|}*5d>a!S%NAw_KoD z*HQLafipiB>|FglR-h#!h63Y?y2@)6-$LSqKn05N1G@#FAd8~lnYL}sSMS!dJ=b@( z=XUzRA8If;4dwBI8={5ZRJ4J8z{Z4{r~WEu^7J#4Cw9>0X0!yZ!x(LI+o1QjN_VlE zuAT76W4=R5Yvkgqw>+o7CCmvqXWRg{ih-3Xu!jSdjL6I%N8ssiSP9Kkq+6vSR{QKh ziB+lOH=Y22%;F?AVs>&po`|oTp{6zssK$CQ0)r8ryuU-+(vk7 zs%M;aNz4u4E-9X|(+_lpwRm5Zkm&%UrGCU(0WV`#WODNcLcJTsfI#ZekA{HA(8!Gr zNk)*p5i)}tfVlASK)&Vz-@;aS6t>gI@B_sUTn(nBw0VPwy$TROaC`FVNSD3*{B^Jh zAX}=Y!&0EgStpo~SH_7rW_Rp-d)mr;<5J#jS2fZ3+G+!eRDikL??#>!06@5G&h6Lb z@1-J=XCn~R_r0EU54^$jHg9Ceg=-UBCbPEx0K0AKO2}!$vX61gP_6a;`EeZDP*&8d z#3#x-X@MAZy(BM+F}{nEHCT$F^zX|YlLvgzbh-<-B$N5DNt8hvestEh4)_P;ht;17 zog0#sOX)*sHVS24Yf1AgVp7|*2q(zOEo){1XsOtN*@0AqP!_|{iGL~1}o z)=D;(d7d0Inh`MC)X&GWc*^H$ur~{73dHjEf|a)q`GEY}3W#0byk4WSrs{@%Aq0)L zh=I)=_p}7!rIj(X(>wq0ngx%tg<-wv*`Tq`$B30a@apk)6l@%u6{ZdLjY7O2R*W80| z+>RlaCm9X{{xW4CZz0o(EHn0OftBcBS|v7Q^fy>*BpuFMN8|%B4DZR@Fu(6uGSlEj z>VyBBz42PAh3)Q)(aw5w3%M&?o_pyo#|<$#tFPByi7|so?6M=1x&>)o0}PxDC9VYc zi~Jsf`E^CrO)J}c{FrUYRvT#!hA*h@3icPf$*Y_cm`M_4%l}5sndL6E)9$`}g?S(G zBtId_ag|>eM1-&Zy@+tENr{vlqou{kVzUde`YA3lfF)At@EncgP+MhjsKfn-xxVUh(#eebRJ9%p~hOFSFp6}85e>3Ef zBO+tF^flKdn3VJa7_{2m&FA z;Fy}y?P6jrIT|0tVD;n~Qs=C#xGvP;;Z)#+B2$(Ee+6^PGaYGi^9v*7$@C?VaLn65 zY!G`^-A5P>?dJbIB6u;GdcF`6@GU%1*}EqWk6>*lYE>x{XoaB=*?D(e=<@h;pSNs3ZHu<9-P8LvPAtGB zcsN6vs=K>ghV%|cgMRpP>cDkTP=sw>4N%3DblP01)-yXl@HtZtWM(nsC6g$nI&g~Jqj}=>t=ravODTxhWkwQ!|m-c&ux%Jq-_PoZL%9{$o04a z1h?7}zTH@VNEAEJ5}k5bIcOWM>g=Wrp)JB7wVCyE?bY4nd7|GQ#mSje&%Gk3_B_^944EI z-!EnBRm7Y06qRWt_`^5iwEe93xoSFDHtMQ-q{+B`7TMmgi{ZrvzM{@kkPjPAJ!`DC zkp{e%{F=MvMx608AQPGeN>4ks6h=@Yr1pl97C<1RqYjjn&Od*%BwOYflLPSCvw7P& z*pu8A0>Bn5np%k2!4#mFhpSC8^eGhYYy~ls+%Gv_@ zY%W)n_=_E)H4>rCzAw=41yd_V3J;#@or{>3PbU>P-~`Z^zvD9l+`tRBV*Mpm{|Zm} zgp-_yNi)*IlI8Dcm}=-e{iNe2^guPIvt5BJS!)fp&g-A1549+XvNvzLz)AXcTr&%Q z4=5QYc)xb0N1ZzWb2MZ9@7F=#^HYuM5+`@sK?X!qL#mJv-MH`mDJ8}J(NW?DuH{}A z3V%SoeEIzM7ZRdLOJMXC1%nCRDW)j%fb>A#o;yMHdH;kIbXDakutgk&kOJ7I`P#$nTPU(GO_(5TMO-D}aP1y>GS1fz27eohDIO+O zA`iOCz$hEH=>K2C3cM92!j!~#jw$shbBIh_+cFpL21MMoKZd zv_1mex??w9U%ef+>=MqvoPV`ZP8&(JZUpUT>C#M3EcmL$geN3UyK_~?&{8r?2}Er2@BhPAcC^3hRx=r0^4tI?1W1kxiNBm<#~zy%Gv zXS?+n@a8j0`reX>y9iaRpb4`xQYpQY!5*0SDKjHO*#ItEVtF`D?Y@llTCRBx!gHsV2AYb}-4Ev3|iP{$R>h$f))31Fb_DFRrMVBkC zlEv6zh>ABgT;o1da}+2DDE`oFvu{N{g0HI*e6IwUqF>d5iiGdKT*hERdQXWn-}n1^ z%5ss_3bIK zjxob%XfZu8-{+3?P*85;t)WoQNq~l_vpdHhuQQ9^gKaf$;Yx_j3SKDo<&VIBd*J&L zANHb_hg7MT!6?R|K{R|%riNR4W|Zh;tr|8r>oA=_)5;%E`nnG7h8LJ}h$s{}qwoUm z!X{qT3|*bG@$BI*uRn@lU>Sr1fZk8@VsY&v6mCn>&v`!=(NxvHID2s}Oo@BGQDkFr zLR09v)a|&FpT)7c6TV;YX5PMXq>thD^|pJFAStX7M~-+kX#%oMoKs7gLW6n@-7ej0 zIw`dk?Yy_%KWJ*IZ+Fp=I2W(CBzjMV!3Z_9YYXbzSA+0To@WH=h1k6k==Ek z`Py&J?xk@1S#5H&;@9-m2HB3i`|29E6#Kpb*&Oj$4@H9rNMTx|tK1X-->>CNMiErK#@qoem?mKUW(CeSYht2TnACgV#XF zJOzZrK$UG1AO1-aY@6)vq<(>Im*2S8UsC+iJ zq;!>4O0n^vQXK`UYJ-gktXeW9#w9Bd@_=b1E(dnd>o1s}>T6`x?2~RQ;{gKCy&kpR zK_wb3DT=U1FAlSVBnkNXm!W;?CouLG=!U1%_i`B@k=qHLtd%Fd(R-Ea1nFH&WOciW z;VT`X8z6kI{E>lUKB0I)>uE@Y)CvgA>yvIx@NEAIQ7w!x(|y~d*~nH*9aGCSGp}P1 zQOuHLw$BqwmJLVpZ4sM(AGGPubU0PwP_a1d5tB7fb)YGzM<|3q z6x+*aKqk&JSN~Fmfk4dlG7{tkg4pC^>Gw;IaI?lcR!E&*BrfpfYvo7czPmylKtoig zL+ix4*13Flj=-mXW~eE1czbhj`O3Bj3optad*<{pNkw6Jqcb;BrEIe>(S#Wk-0eA_nr{h^t5IQG1Bw1#%mUEmt!sJUYZF5F#sb8vK^&YhE+7f@=#} zWh`pE&hm(Ja3CTc)&K@%$UO{;q$lo&Ai>*fYc}^$3~= z-jac6X|b3r}4 z0Fl1|mft{kTh_A%Ejj_KXRgqHATs>S6OliNlel-~dvi%Up0iCAX*R(ODAOS0yY=<>bka|qosBN3;xnZ zA(W`o501Ruc{kr21@(ysqh?-NGqU&wA|6553s-o;6KqFU(gxY2$0O?!>u|R z9Lm=mfTB{@;N|oN3>4bujCJ&f06V(dV1*RqVm(E^|8Zw+isrL%zE8dfee|~aK$ap1ZH`-V2$Su261-s zR1PMzbfl~N$*_C}d#ZSxs-B(5V@V~cpu&;z4uO`RXk!!KtYu@o1-6i@o#icY*1(@B z4ER+f;aeCl^2Au?dGqe$c1)BF5Xi{o#Nw8wx?8@$u8e&d@%j?PjCx9t;0SshV}JTy zF|c7Ur%8`8LH0qj5T<6Pt9!=bf*1L@%nW?`yCP+++k@aeQW?4eGlU8-2x&^dSCi7$ zzxVr}+oY~_qdIo*TrTakEZU)Iu+ees_0P2(e7Ak4@%P|u$wh$4FV95k0EHEVGkJ;) zhy@V^niy$qdeh&3l6Lx0@WA$KZ;&l@2AN?xij)_D*@Q<88QIePa27aS__qSJDddBJ zz#D(bQpY0QSd9rz*0Xl9(=}j!IUoZzonLlkc$Urm4X!-Bk36fkmjCGKiBJ|6p@NeT z65Oj~1zx<%r8&o%f#?!koxX|EM2w{M`afc$4ab0)pHy_b`HAR)e8yG>2Rn$cB3qtm z4geU`_7cdMvnb((+5qo`)J%s|d}T-+q{mU(hgu(J|4$WrxT3JT><6S66i^AkL{F1{ z4Ls>Dv!*9A#@GEkQcr~Z&X8|dvCX^DtY6@Z)QGb}MkwMkN*m#R3$TIQXPkuO82`a> zBf9Mzt#k2N?^H+L0ve1oLenS~(QrX-Cg%%E$4`hs0$4v`l*x#{@Ymi59a^Ju&8GX3Y_663jk__7DO10c+a-e(6 zcVQ{81J}7Riii8WsQzB5S;6S*K3HKL=AXx)qnjh-q`e*mHKi32O{j6!zv!NB)K8QW)N6G*=UPV1;AWmnV#s+FS1%kUDT2TP z1d!B)bK?dqKffTph)wis2bLhgJqy7vOVd4k26-YQ%10GXpzNFYWgap)*83LhnW}$~ z6x=NV|0=c{%yi3;mJ64FCWG4-yzN_TQ8WYKa5nEB?7az@rsfg<(HA1XP>7H>0VYyC z)p}ChX~ac>-w-kPrm+b2?s#i*9{A{_!Ng0C)l3|8FXC~?C+N>@y;8anDOgQVj~*B5 z2a~uI0t4s3yQjbmqp@>%X$p0xV(2f%P&Cg{$gS>yCbT~*BXLsT+pT?05ZLqL;)mDZ zH1-I&a%#=qx&Rd;4SKe`Y-RlnP|zG9iqi+)c-H+Bt_-c`T1EVG zponv-1Z`)c=6#3)L!TzWVc+9{BKe2_k#ih8dGc^|%wE>+8d!oV*>Mq z4F&cZtIgGUkF!Wqw(TG{lOYEWOxQ(>hBCe-t)+0i$4s$*SU#y0ccHl(D!o5Pfcim zT1vbAvm2K$-yh9AdnkUR?eQh18I28jo}n2_;?~(9&f)!qAkGWz0ZSZ7u#cl?j`faF zg${`pU6G}1{fIVnNI95iU6Uj@9kGxB3NuuBw_6QO!=Cxuy9egSzlM&U5^5zgZQvzX zLztm0$+SJ42Yw-FbL%q(uyuC_AO_&8n9W$Cv=HAf`@;_#F}7w_@f%rqnv?1hHmw)KQ`wVV3s+e{^aF(N!ls&g~3u9J-eoTgcOtmEWuRG zHV|;h^9)G6h8|&nWbRu2`vib(x8!pyd^i1viBDj#3ohi3)pFj;bcsMOle15jOobs{ z?>Dc2G;)mm|2Y01eFu9r5>ft++g_RO>G%?Rsc{Dxobl2B0mTaSWTtfH;47o5rgbpu zcS%;YbEg*w)rQ=x{hq=_bcuT4BU6}N@j0AzV_By1cYi3O4DXZ#$n;{C{K@HK^>*O> zU+k8t7;=P}!RK74{(=9;4L<_f|?hq;52EaN_ZwG+IFS(w*<{1zf09YXJ)D9Gp4Y^fNJz>9A z!ITI7Wc;c`L+-1NJB1BkFcYI^B<-Nn13?abUApvZT$};i?xs*YEE6o@V^G(~wN(g= z1zgEa6JJY&&Qj9BBeW5E$mpm!R{~HZX?JasyEc$X60?LrsZ33S9mqq0zk}iq_qEDu z_;J(vKxJ7+XySl-#4jW{i9nB&i_C}a2olFrb)_i-5 zWk?wdb;JsdDGYU*928d1Os5I@@F>30+!>Vp@yC|D>EM{mypwG4pP>Z>qZ1N0|F}ZW zeXJecu}~?7&9Uf>Cyma&N)(y@&XiZ7(y!1pLu^7-V#|Trw-8lxw%eYtXE=9y<=IL`u7FnXuIUz(!tlsIs1UB_34{Ajxt=hU zHvM`=+|Ai^3&uI{30PxEdOP69|0Q=P$7qYA(BsYF0+?;Hp#Lb=R261M{~^?qrqSSu zMB?2_JK)X7Oh1{e`sVOEXmv!;-+usI<3;ozwTNwP=pHOr648i{ht9F;%w$%c;+w+_ z&|z>Cg42I2QjGCUMK|to2WvmFrJ#>m+%=0JO#hT?#O>n<)Dy_1a9dQeb*cn(zAK1n z21Yk^#MMsx)p{2xzjyyzs>}3ilW) zeFir;2_Y^i%@~--A0mn+Cw~rKh{8+&?PA9>Bw*`+{0Z2_hHCi+(C+MhFFjFKh;P*9 zv7e+K+Alz{wzqtFHS7VEs$V=jnnOhTJKI|_mk6|PSt(?ebm9jMF>KDrg?jIPTUrFI zNZr!3q#EM96mt70aXC`!!=B3NJccrkiF&`~n{1^)@|4S7YY5)40hgS^e_i@Dkb#2! zq4;Ll$H+m-J^8?w%n^}2aL=;@z@#7>mrq92h3XSXcp*!?4_OUcr_F%8@Up5C4EC!T ziJ*JH@8%H@6h0IefOz9((cZ?zPdk+B_xn#LjQkSc)?H_86++)mhYa^24xKwRW+wHI z=c9j;VUs2(yu9`ud}x8_zKCEAZ?h)4N9m_dd4WI1F(JmIGGI#S6Mga=F)cbTAo_;t z?PZTmXUH%(E!Jz)j%fydK+DqXz-6r=lo&wx9VE}t!G39^U;eP1P*Cl${G{4SbWC1YXDE5w#ED)%rjERdyG^4YY_#MEhMV zk2UMh`4~I~NQK(wb|{6J5IjLZPHHkJ>CxLs_;D{A^f2zxQ60S(1oc4a>mFBad|sCStSw0f$ao*=5cG z9XzQm-q)qVb?t*Nht55j{5?n~^ESo#ywagO-*E^ruR$HL1(Fi$~i=(jCAI8C=f1Gt%A zMictJgGcR@$N_SKr8M-SU53MU*z3tbd;USdE%I=3tF*yx?wR`0z zv;P>-?D29&k9cJ1O6)mMd33yTEIl-oJ1cKaLGymD1h2G_1d~g=UIOqmY+ZRsyfeZFFxIa~7V+rJYz>^?g^(`*I~Hli0w}U<_q}T$pp%Bgw#vHk7eYVt+`Bd9F!kjDfBsvz>(? z^#Qe5RMq{4M`&*(_?nBBFA{^O3)yfgvR%udLVIH?-y&yq3?t5jgIENWfUq~-k;aq3 z-l&H5#`_uj_7FWqPUyV;OhTs+i?Np^9H=x>kphgN61#8O<`>N?Fgf4{|E)p4v9>f6 zv4BBZFX;f0VpQ+@B_`vJH6IqH(`0NV-CaSf$Gva-^DHEvgKblN2GVH1xMSw-!Lcn`w25>Xy?uO%Gy7A zf|_J2djFgVDwRo3*+{)djUkBNlS!Xs{c|xt;gwnCcT#OwdjJmov+qrhoB?C+7ZV4G z>`>7JB0Mox8v1{JdkI>h)~;{YzCsRL(tU1MS)soCo!YCX|+x0oo*P(1~2;(6RDcX$Qzk(FP$sTXTYh3CZi2`cD=gBf(v( zU`_l0@X!d^6(TR4U0GGJeSMoCFd{Fb#ht(5J1_NPr|0s~3mJjjhtEtfs7MmKe(CKpIACwnd5Nqsu_jBAZIz0?Qq+q&xcCQ`4@aR|unkNnBj3%pKfW4%3?DGt%IuEqp(rI4Q_hVf=Ee+fV4_?Nk~g~Dj^C= z!$wdMP&x!uRHRc-Y7>Hhs3@(%rc^*F>ALF`&-dMXXKu{>>;Cc0FlXkR;XS_#6KW#db_dlLs;h31*`kYii@lJw28|%W~Du*F;#oMsm zyNBw&bND(lHLr)MND`0y`+1^|yv}yA`6*gLy1p~{-wmuuT`7g7o@oVH3pu7sx_>aX7YQvaJjTlVF&bCxK!s?yg{m?oC_@CzQ*-jQXafiFUSBKW9% zmV=oK)&!&L4P*qPO-OOa;nnP@Wl%I|g2M-t%a`Hb*dspB_$)6fyD>%`Jw>2FvUIcS zBUYr-{i2MZ#xV8^J+?~OM($n?s#2q+BZ>rDweX9rR+W$oBS(08-#Yf2fS)>-!7yr5 z&Ukp<4E^|-k14u{E>g@$h78B)gXY2z9*!5*Zps%8pOVjw%;AUA#K+VL@6o^AiBYdO=cGY{Lzfb0kg)hl4gUrr;Ha(qzAZvf6!(X*8L>tC?30Z}_}?s0^d7 zk1xiU@}nG|_%KAOO<*+HQHSe=4EJUV15>H2gRHWAYjVoFC>HLPMGg(dCYEPs38}3X_%i$Y-tbn7dOH_hF3gR8lk0g2TH9SLZCKIH zfE0Nbpj{&?D_2E&nNXR`RWv`hdXUJDy|8k!%1ze_{w`Bo3G^3%T!6v!N8-DHJ<=i z;s>w6$GgBj+U$cP3?HBE>ucU330ERXT8zjex%v@w1)(<0L=9ySFZo8-3Ul4rcUg=t zSqEO2JODVQ3(>=;^sJpQXJG{;_zEDgH?5THcIs-AmX26iyrT{Rc#ptj?yTR{WS7R{Djct|pm@5O9%05idlqEp!S_aS|AN#lwSYcX1E;GQZ`2mOV)^ z4fY_Z>%Ycs=Kjh8^v{Y&G8i-;B}8VMUXcb;U>%@jZLTJ4+hn!pujHj9zk7A-6ZrU_ zLg+RZ07h+XHSV$SBLxGN^#KdRhQ#E@*_T!&!Ph(p0LU3|flqUdVZ|hg+dk03(j) z3s7NTW3fX_Kzq4iS$zxIb~vw0zcs4yFaa|_?j zOz@{DCuw%ue*rH!`BH|Arq3(vydU$x zrVnnl(@U}0ZAm$_qql&yD^R<`(BF9TI^r50urLwv2YzR6gL>Es#8a7wvj2{?pi|3t z=5!7@Xi|^(kBhJ(6gHIFkJ)u<0xsDZf_P`o-qvuL3ho4s-6Opar-j{M}1F z`%7DAI(?W|Q%H$L)zbzb#R(N`q+s)25s*+~A=fUkthv829%id178|BSLVoW0@9avS zSw^8f23L$=0#NYu^ikVN2TKYuLBrFwo>{ldtT?c%2QXOfQpFfS7;po{QJ2(FTZlLA%Pafi`UzFVbTIawylOiscOh#B=7^{RUJ zIg;w5TuDJjhznMHXQb)0MWrexXf6L1>wREpFIL&(K|+ul)Y>fEGqeD4&!Y0Rt_pgH zNxUGwGpN<9%yEM8;y%^3D??cQe>^`|n!lcqzARMUr2GnQkn>{vDM?!$Y*t0l;r+!H z;w7YF#153jNr@NEFRKQ@np$hLm5E9j1GC~6@*nqiSM>uzn{TXn2;bVqytdmz9s7f{ zcb}bNEOk@f)uQWTr{q}SEmi2tIcL^Uf?PuLxGPx!SH~-zbt2rN|A5Q4=IZRbHM!EP zN48w$5eCsvNE4`E9cg!~DASFKfS%os!-I!r_Dad3(VB0}4a~MzYl=>Jc_h;;VLqv8 zsOIYT_i5nQrJQ=&`b93sZ>4FKR(jpn#LuNcCFMR`o?mn$m;_t1mSPij;eEcA3Oqdj1Uu&~^ z{&5a$m4VRFV@iZU5JlsnTP?o5%=!G>P$lHPJr`KQhuh@n1{fo@pS;I$-QDVm)ob>R z12vvzd$G@xpggdDBNV>?tjbDp+R}$GB8-_Pv`3aGi7Lu2U#Uz|;2^}`c6sC@b#vWI zDg0>G@5T+AnkQ}FEo(f2hN$gFh$Bc3ejBh3(Q%X`y{{h<Za4A|z`5Vr&JD?WE8F$|~aYk5S;3_72LGAUR z)dIzsa*;FQ%)d~Z`~-K835NTlr4!DNloEeYukPC>953#^=2ke73xeyW@{&9iS{+i9X7SM(N$orY9TyaKD#g2xMcX9N;Ijv&OGMt zN9Tzm^DlfiC9aQ;KU>XJ*#9hKM6w+{Z?y944#UvYgI_-xn=wHvu>UAn=!%nl;;Bi@ z=1I4dxtaxll-M1f%eLi1l zhEDI1D33zP+MM$qyy%Sp>ZwL>+-nT3jxV}Z7fK>1%+>5OSaM1F{dmT>R`ZZzOO`RV zD(`t_d2!Sa=a4wueF}eEjn4GeDXY=Nd*K}D;sHB7m&Z1(3M_-T+}pQrqxOFm^JVDD ztPaV7wXexlRn}p}X(!IViB2pl!H)Es{r4$9-X)56OB0Z~ZrMZ-{>*K)0C)UbT731?zhNEXXP@h;A``{Ul)bQntpWp6O^CRj?Ty$j&0syBXx62YHrtUg2pRN9@30X- z-OL>u!xHP)wS8WTx*cv_i8mF*(nIHtZL zIr1X~Q)s6a^BJz+AP`Hfg;d%u)yKp>)|{YfcCoYn1X0(_R-np5`aaQp(7c)4iI@U; zTFK%v+0YH@A4lFU4(W++-Z@r~DN;U1V4Y*7XVT?1#g#zriY0 z&Q52c@#ojr%QRr_=Lh7@omH)lZVUEILfsLvG7E;z-EwSBbv$uD?qj;VlruoG`J)r2 z@n!^}A9r^CDXV7HKRQ{kHl~QU!_}Cw$C@FA?k~K^D;)VeXDQ9)U~lWO{x-G>7j=xR zU5RUxKC0tHa484Uk-)ECi}&YFt@qfAw&36;GS@nqQ79G{jI2UEeLi`-t8}{ZcS=5;E4siO1^uoTrNLn(0myytXr0i6cgQ5+f92QOon73jBc1!JrYV zL;54j0&D2}5eID+{}TL^tUm?`Kdxwos#^fM4i5T7Kr}QLFE2DAQz>NNxuRtJdVgoYTVZgfUj?B$gx{A#u`%)Fp$) zX|Eqr7QuK-t|08HFG%wKZ}G^i%cGdhRk+nVbXiWa?5IdonK3Q1gmz(Ss6#977qH6R z=mJ4WL+1#lj~%U4y=prUjXi$9=HcpDi7OZsff)k_5AHsh1I;6QPW%mS)ia+FagQ-q zlnl1AZq%Ycn_d`k7q5pTVJ+l?h?Nc&H*Q^$?5ExxJk5p{Vqu)ZT3N1QsdY0z=srg@<^uLuq~`OXfjeXtqQA?ls~dX%c+J& zQ;BCyJf`rE%!@`Xlbc$Vcee^g^K+RyBK=6VS)1)@h}C=2L^f7LrBSBg>=H~FjH zlDck^+f!ze2|04c*%&X;;1JOuwZ&!fki()x783LT%q51m&7}S0U{a$f4G8 zQiq)0WjDJ}58s>3`B^YCb`m{*Mh4$$(%4Tkr&MkUDdk|q z2gi~c?V?MA-{V?Gs@*?T^IPRaZxSh?%`~EDgz7OZM^a5HMCq$XLeb)A2Q<0;16!G@ zp!8LcKBe%Zw2Qs9{A<@XGnxJ-9_n{%U=}O&kaU};d!2>^z&)PXi#4k1>>#OPgHB45m*V_UnewMlwc(Md_MLBbwlRM6qtWa17B=l zc5jL&VgcYWz~h8o-|XrbdtZ`v`ZGAl+|E~r{v34 zb37?a2FcVRn3M6Fc~C>53#18_N)m1#6feK}$)l|G;7D-hh+7V5G)UHW_})D=&;Fm` z&QW6ixZ&0qx3h58RUfx!CB;^WmQi-96VCj@@K(lb7XVlL5Af^H1i#{#5E_v|3m^ub?DR z{r9~e$2zmA|3jb@ul@~3`{wB1AkgWXvoj&dbbTvLD3&86(UFH=nLm7k1@+$pM@UdG z=)ug;olEu)eLCLvZ8mI1cf z5-V#8P-6Y%8F>#tyzXV6ysyZ2_#q74CFZ}QEbILWpplG<%ezc&Vaw`7L~Qc&cpnt_ zZGq&_2^6;}8mvKF?)7};kZ8=u%S1 z7tix$W3PKFO9I%OZ%IiRgIWQw8o^q>c)tXhv<;7LIy**=U2&Ro57Kl$IM83j20-gV z=g&l{x!{qiJQfIP)C@oi7G;fwkJaCTUYQr5HB zFA=BubAN&nFD(}Su0@DdMn=5S2z%}VjQHzoGiUu^#BXQiBq_T6!~9?W3LLh-^;WQ~ zffMSa?~zj1{~$@VeU~Mjc9c>^k^Qbyrd^6e@TSiQ;91U~dfxUJ%a*%YyU?%-+pLcT zX_9FLO&iw@ooq{du7NY74%8s%Wb?MSd%yfz+kVD5Bpr_3kX$gXSOGC;Hy{Aqw7eIL zeR~z24iuRcb?rco(6jYDR|F{nIH-8AXubP-Vp`e1p9|vvyPkA;Qq{?gu8*@avHl@3 zw#tguWrxe)_`k&b>o*})yw}G^Y6uurIP)1~wZZ^hWFJGD$u8Feq=g7=wIHx-bTtfd z#8zFpT(1W3P~uB>7af^P<#rJWq`3lquXFUh;(aowE?>C*#rKvXSM(Z7^e4!VI^?b| zN5SQ5W#!tJfoj!-;k38Oal0GeY9TQw_Pq)&CI9w?a1`mkee(0lJ80=(UbeUgLTDpE z2f;kq`Uwa@J>s#0p$B`{L-s*$)&*sMPu?T_66?Xls|~sh1`tmaEL6*!1MwVT4E;s@jDoM2m>G%Vv)H%*8o)(9?Mr3Dmt}c z@eQ200uVzb^zI}kxPj5y|8-m51!ORm3A~5J6f02*N_hV#Dfaw0n#JL_H^amT8!J1YX+E4en zb*8b5HXj14t6bu3N*2DvE1eu$*3}nd0Go$@f;ZUs?Vxc2RK6e-&^pCk11BN5fn+>(n>keCYy6A}{xsh&jf&E#q~KPh_2Ah$Dwj3t+7% zs20#mKQ{Ub(5$rpu&wpglLv+BT$lXv3---;HU*nZ3HQmATioeui^SBFtyF_Vg*P{T zuK+TyQ=pylXZ2uzLjh`jJW=;{eNTS8oor5KrII6k<)P&OZj`dHxHThZT!l)Gt9&w2 zaYlbBXan$u8+%hD3J1HV5S)_OW+}~kw7QTqynJ0HFS2r5X7y#9RQY>E6^J{Y#I7yB z_W9y{H}@Z9og0Ty_{?9zYwO~*294D)oea%9V?x2OS3;@pa*fV8A%v9yjcC{z$4y^; z;R8Q)p)3VM7pS@$@94!#y0m8v17e9{_a=BFBgH64Cb2Ru`A4*`rAeOf_$hbf=O zrFg^P_-pDG3AO{uBGVG)`SIqkUbIyObPAAiY1a&KQI#tz(R#q+{;=Z5os1%Erme9t zo%o=&nB&ia_jeViQ>5h^9479p$EW0_U5Vs$`-ZennZd_b9RCzd`$H)xDyWjpQEzr& z?frZ6^bQc=_Z)w-258d{d8LClw|fN>q~BOHuM_n>;MKnejEIm+@qxn|QvFrKPG~wE z(7kOB37J^2;AT`l_01Y8(JWbi7mF-sxkFBn1=F&j5Sp$DU47hqUG}kJDHp;STnoe< zNN9W_!tWetBrv!ZE_%tB-0}}lO+oqtPYKvS{mvOE>A|h(-i#N^NDw#o&4@~{JdVq6 z(5q{T^NiFSbm+Fs=AdG5Jszb%M0Bi$z8U82t8H=nMqLa^vsv8R*z?Rm)I((RgC&CM z#Sfr_NT|X*L;lj8;*-~gmBwmVCbC%vZ`NIin>dPAwxcaoxb|z*q|9Q{lDi+EP+kBz zA0jmu4ysa@jBP7;>N4Y=KTWf_QMlnX_qosmckE61t4Lev;zDZq4)oO@#5OvE(8;E0}7kHH{ik@*kCNRx-<_0!3d*wK_GL;5D&BRz!n;oO2-7P#Bx{1$>58@F_GZ#sDZ)&bh z*J;4$FbW}gvDq2?v)-r8``bD>gKfTh@Rj)^81`kyz4VbvxX)8)FUTk%^cHTd7igJo z`Tik5oHP)t32!Pw=2ExTmzSWS^JKRLp8S!e`y+fT^{IyEnb$&L57$z=K zyj2(Ppf}m9)X`n?Uk96~z}Ae4@K2tlaWj$fZDvgEAKOtJl#ZKlvaEfCX?oU6yW#90 z7s~S&p5e14T))W_wg_vE5hC)LNDtH)(6ua+i@j4gk z5*j5k%WAj$^UIhWbdvVFl(%b<3=^`&b9T-y!hMNp@zLQ#nD>bj;vy$?I=ue8mpY8I zwE=hCs=f;1Xg$CnDz(I0y$xmVEKT}(a6ui$UkyV4F#A)!;;TqCa(rx@XK1;Xa_hGU zEG?(=sOL3UI5O?G+5*jGac9VfxbR1spw88A(89Ga#p00<@v}{-@bGJ!<7HMPfTzTH zMq|<@l3A6h+6AnF+6OvyAJC^x+--7w|Iqgx;3Y4|Lusu7s`)i#qCSOK<*FSy%B9NK zVPX+$^oB9R#+96=&wwFv@I$`JvZ+r9%`dK}x`5bi5s|o}uN}$94BA4Oj@hK>pHYEz zy;`g|Dp)ZO#f35{H?$o(aYX#^dyyD+)%}~@xlu8!xCW#AS5Oq_q#f^hpeXS#r&wBw3kz&Di(ZURVNv4P1aySmnJzh~ff+fXrJnO!RR%`FBVi z9tmt`8St)@pC^0uBIBM9F`LKWoN#Fr7n^RoMnGyp=aMke2IoapAc|8_kuXb_PFsvx zu0u8Gvu6lqt2-0mC%ga;s!TgYpfD<|`AD2E4u1ep%;4)6ae`&#c?cZRu6iDdr-Iak ziSXm(p?$mi{oOs24>finM<=R}-{!`??$UzJB#Lq+A%SL;R$mDFq+M%8BYnxZ&7MbgpZi!v-{ud$ELC zCN*``#B{MIYQcB13#K}-aA>ehd-*yXReLc z=GbjWNrlysPZjHU{8(w0a&OcC({2vp?)SRKSgp5U-C3kcJLl9OtyL1c zX2c0|M~V`(D^^?-H!#poGumD=bbpp{dW!s)2gR@G-}+e76P{?7%QE}c z3Z-l;(9$$=dF(Uf81>K!Pm;}wnuN|jDxH|xe}8xDejo*Xfc>&G@ZvS!q7v%5Qv`3y z_j!#KG$Q<^54J|Am1%br(FN?yjD(g;m)9-@43*g`kh(JH$~L{)yq?dpTqy{jPKP89 zM4CePsR37E!uvlvgB8C%m?_R3asQBoG3~#H4vm=)i*=C&KT6>vxwZhZMF@)FI2VAD zHWTZ(f+ISx#9TFa2ASrKnGb{5nGs0B25ys1>Ismfs>Jr!513wZf6`|>=k&T4%ux#r zned$@=V?nlBDvG?qaKe=1s9t@?sZi7N4I}=;X08Rru8IB%j%}{lOE%@Gf%T2_4HG@ zUvmElNpcG8hmU- z$9lC&{|VIC=iU&oOtdA(q5PBpOg#Nuy$a> zT|R(#qav#elQubWSib2VYd_zS~+|5N8;4O;qwR-1SNqGW9?qN?umk>W4Uz&X;FF6UlI^xLPJdEx(V?3NR$D!iZ( zd?J`_q3eA3Y|#IkXW!QPm^N;?5H%c?xeB=mBl^jyzb}oIYmvRL$bpdk93`gq$fw$u zUaC*X1clit*IQf|?ZcIa^%L|nfEfYHe{9vr+NeuSnvJl4YmJgMim1w86;|uL7WEPfQdh%EhJCkC-T&~o%3t(@z&m!aMO?*-d)f~|qvtxM zi-)=dMM93hFzOQ>>JfX0>=yc6t~)i8K$fK*BH2Sx}VCmeNFsDc>#nD@8;3U17HSUK-cGYBt`GOBEi3pJt8e|OPBB->?sS2|Brw5F0-UI zb`J;}`_*9sGBysQONCXyJQj2J{_&!?hdU@rVm!fB3~}fhztUurRoR z)G_5`mFFU$L6aaGi$iI#SClVXw(4_+UNS;?m}j9LDIbpT^97r}C>T&{h|l}E->ks% zTgb|iV9U1t{h&3YrzrQ$7yj%5f;(zj?5&5%%blf|+0PL0YI7;#i&f7DgDws8R|BdN z`7OiVK*rytP&|tdTh+T+Aw{AJIU^!XHXLcpJ_eih6|C0+ox3wg00}JQs=K_*>o2@m z@ia4q_inrzY%=Q|&N*N;P zy`wixEa~A%@u>2qEESB`RfH8mVsAJKqy8R+KG*8?NnKBug);j?-byUQzzv7rAx%iSmj!Lt16Cu$~ zi>30yCOL+T_twOMUoKsrmCc^S>;=3Kg}IP6A@x5_K3ZDh8EjtzPPiPoC-J~9L?j{; z%S`j9k36b1LtH0kL>0E@n}ixI0L%Ys!S8hco;eTW-H+T+)(%1-1qJOQqKw9k%*gX+ z*Eejt_vV19Pq}$Aic|M+V$oOJF2utjtqK#X-QIQ!UX%k-Y)f$c$YCNzsE)H$aYiyd zciB82C>kH$%nT`CdxGGwc%lR2{i@ykA&fU0GTyUMRXrg_GX6nppVH4^t0e3y_()XK z4=0xT+lOps7_T_wAd0<31rw_bi)+rSG9~CWE(|SMW z5%5iW0)lrxsO|5%aPC)fwbw8!DBvs~-ux}Q1$@9X=+BbC?Oq!s-TXOc14zdVRAgC! z!n9(bybY%%Aj;oAp-{W&j&SVyk;@3YgGi}c8}5ljpze{Hn=@nLKL=w1Mi6omaf9f8 zf#B}o0#A#Gi+Mnzzl?(R-n=4>f9|_{1EZh*#~M^+EkKK~A3J$9aijhTG!kVCnx>Wo zr!5@;hTE=SiS(e<`gJ!XPtD%~LE@{xBzL1)!sD}Bqie6ZMtJMUS%zQ@xT9^5JEQG2 z^G6mY;FU`~2VK?hLi3s<>?CwthcDNCy-4}T(j6C!2MAF_EUMj-pnn9Fk+JcUqpda# zAQT(eE)Lc(PjAE-EdX}=8K(el=6-X9&u-9yeiV9NIWAfVF+2!boqd}r>DkcWG4SFo zbe?nq-ga%Jb^U2wojGDqwFMo?8}kPWFKRHHw)zord91bsh|hBl%|i{xf~SzV&Yw5| zCq0I?e0*v%t3#Er9#ls{jK-&id9YRJ(*n;_37!85bC{&GnTi^qZ-%d_Q=R-Km|AFo z?!LNXJnX<4EYpr}At0@IbmDmEjvf?PFbWgHT7$-B+CVFxwwRHa|Tbm2OzGs z{BGX^laWvfE_0&^!FtHM5}*~3`{{o8-i{JpD7l9=P6l6gMkz`TW!T$DHziG`NFlBJ=5q_BrDPeaw!BJFb6EPKQ@u1Jr>%k{l zvyr8US+R3qa1K!a#O|x7#Uj!MTs<0$!Mvl@8H~{e}_> zeegJ8^nMJ&Edd6k@p2_NUfdp^6w;KbI(hxa2cNM6`mj^s(u5CYo^*Q-%X1o9KniH5h3hGG(al}!K=n;}h@@QJF z*vp77e=tZHw`G?ZgCFA#75o~QboM3=jvp-mEe}=Iy#h7GfFA{+v4iqWaNZp7G8eJ1 zrySDp94ak|G;zp5B>XYtOq0%S?sCC*EyOx+E{$iwZt4bm4*WaXk@D*KZ;h{SN5#z? zQt`lQ7x57mmSvLCeDE)1M48>{nmWIA`Pm{Gz3ycAY|s`gBR_7hxv zX(KR_i*d0S0gel2h-mYC7<#ew5L@E^CjVA~n7xmXgzq!EI>$gB7?~8GsEGTQga_9O ztFmrpgNnF@J@3hbS66>>Vxkh+?<=VmU3yvC0FcxQltsA97gw?y;9gO@70Jen7xyAv zu^TDRf+)f%h$K}TV=X%PpW6S0dBvkmtoS98AuWz0IL_WdoH*Lv^usIIhSNx?hX7xK zuZmDqnY7?(`^%W;UXk?7(VSdXYS^V1SqC90+$8CRT@DGHQSTYIq0$`EV9b)}kg-tT zgdjO*R?gA67C&z&pz=702)ut?1M3~bB=V-V1@ji-R5x(=`E&-u=S##$651uI9(xPD zxdeO3qbaN@6tBycX&YNkIV4Z? zT#YuG)ATt`Qq%mTh2ab!f5t_U)r*XM$UV0Ou8tB2DH9K=tH#NIWXntV?Wr&I{w80x;gYNV26V!_Y-y*7I$B2)>8f*`o_=A3-(=h95dM?dKy)JD|fYU&t=XH+y3!zozEbz-5-=uEDK^Nfi1!NN z>E(co7X*y&pW!cI-4bG3Evkp4J*yu3q)H}x#iB?v4-=M|D(5SWYo@pqO)hg`93g+s zRoJmNBdLYby--?X#KkW$6|^LVr(EA<7A)?yj(}X~$)iMh(dJ#VZPw6Dd)Z&rpZ-v1 ze{bvVQP13k-8T>^Xebj!p9XAb0ZEj#!x^A@)v)K(s_Sv$A zzGF}1z3B*NZ9&LvlA zGiv5I%`DOEakAns7>|6x18P3wXC(4BEbv{3o~~BLBrsTV5tI z?#QiPBfcOm)AaJA#6>28Qt2i*v@Gcu%uJ)J0_^E)q50Bqf9ttAFvS?ngfi!pgpygp*@J;=c99M}87=$gK6EQ{d z4KJQ(_8~U>Kx|jOM(t$}3b$>sk73!uN5fpfn*D>EAKZ*%UTf~Rn$ja7aU`24ZzcKXS^E(*gimD>Uc&~RWY0FF z^1ZFN+XplRZ-Z1^&4f$iJRINMzNvMMI%{f5q->s9m#u4(6Dr!clvG;@X7dH|NHtDa=!x>VK9bXsk%$h^*yT2l+7l?RG{`8!awzl18|W(6#Om9z;t zEyVufV>b6RDFHs^N|W4q@W>}M=G8@6_!!bAgeED5Sv&#m1e&h^xc)eS<*25zeI zV~=BDtNU+<29%K1F!=ZhKKFX1~~(C0fxcF8YdiF%ivd zw5;-kN^8IeP!_yfy>bjk_6UZnZEXaYK+PeR07O2A)LAZ>(vG&(SU1y;b|sxE099IT zY-wD)=Iz>#R2M~Iz;IUSpx(utCR|^9x|eH!XFjPxO9V;Iqw?D`qQ^l0cqGF*QIX)k zm{w}G6-!nCq9EV>>Io}W2a-~kRzvyXS6(Y*oaM$*OHgY=GI^ZkN81XS18FJK=uyeh zpP*Jt?SYPOLNZ5$e%OsfKhN+>*mlna*`y~Ju^1nH9v4n!>E?&svVXPM6<`#FiBs|W~8Sz z(*#kiz3_457jWAjbeHW?^y*b#G z7r!+Qbh2!hQ=hdriP>j=wOz6^p+RZtVz&+s!~%~eCmkdTCWYD+p4!3CMAnOfPMX~c z$yj4Ni5@91gRz3`RB>-&tA(j(B{dCS;31NjvRI34LCv$rs&E&uKie!+Ud zOG|iCJ6&V1mS;O^UjWq5Pbyd$mI>?MR8wUp#Ft2Ve`sD zkw>qi?L)S2k27o3%`Y9RCw zOhL2S(Y}!bBkwhPm(_`JG6rK)Ok^yo*}LvqoSA39HBhT|flZ%lQK3pOioS*3G;8S! zogoj07cL=F4>QXyFjAD&rEexP29}w|ODfiAEQUO(T)jLGX*DKcS?IFAF%KD?*|`6{ z=fm>#NRcyEJ=g^4u*d=CK~$2Vy>Kk-#+E+-GYs-&>ZA&N{9e8gC3y+z7xzWuF4(74$#Fo^d&5Bq5O{3hb=ko|X2_;WuxCjg7WdaStbE-VV)N--Kf z^TS0kbLF)%CUPnStzTy}ThBahdkuVhP+BI zt93vt*49V$N=IJ1yxm*u5#sV>Wr=rAGc@yfQfNA`D{=XY`9$%!h!J<{tzi^3g{c>W zWbUTeQ&wpmGVGR*yv+wwq!_G9RHH9`{_58w91v3CoO$v!T=9xAp1I`$s^Ns2^=+e; z3huS{4(RXl=szs>JfcK9qr;rmTspr9Ky=elnJ^kYa%A$!9GxGDYs0r$^knye805lo zr7sQN@xyPU?_KZRQG7t+8pLKELB*~qBe=v*-@62WPOp(~9@m5fax7mE31sEb2Ya>; zWEDCPCD(bdh?6)0J&Mo?BjXx|-bt+%mra2lyqPV;So>IU?^0Zk~2JjBz7dHV2IL77E-8sz3adIcs>zhL0N1aUzyLJnnKnJrz@ z$M@FIi%bOrxJ1oTl%&}H4=2?;2O4z~Qbi`g&3b zL28g+cvSl+n~FsB0g{)xl{y6?U`>N3?TaL!We5CxM7a{B|1A{n*A3?NB>_t{6_v1pGunia4dj%7z=VrEZUzY8(18B*dN7cPPLY3x*dEs!N zf1&@z1*D~^JGv+ZiS{foa2H{9dXh(!T{>0E{s0Rm2)mB^Vt)8L1DABq~J z%B2^+de@GqLpUHk0!0D^kibXGkaDa-Qk(`m(+05v`3cgJ46xdg>i+D@BtXx|*aKd( zyhy)E-CTKVurH94od`eL@8J&W=l-X9Ckkzwn-F&h_knkFzpG{ajZb(`pjf#rWGO3Li%!0q_>bA&}sqo0YF~im72iiSM}|;O}lTKsrL{|dT7_EQ0dq-Me}CV;_v&^&J}4@ZV%-? z(Y`ZiQWD@#Qx`cJ?5!5LN&8K?l|@CZo!1T04?KV3MN z;@&&r_E`61T0YI-eCnmkhGgf=WmCBgWdzA2sYzX(r`{l*I5U5fvMm~yD_|RO%!m`f zLmQc~&8k9&UOFR(v_<99B(6%u{Mkf?{5K!|J^S8n=$K=x5a)>B^}JgHNPuj~-ZGJS zsyHLPRqu%M-_iwWEE@d>{8M$Kz$}UfG#uSA@imYGcK>DF{nQtNuLET^PFP3)Y%!Q- z*7{+E!hUNN8aS#HUYqtMJy+{cFLZO)e#+lVyzxTa+$xFDpSQ=}7|ppB)(zAe6|50# zJDTx4r`YitkhVR3r}lfn=KG(OO7Aj_+{bIp)gNGiUqs1(Eg4tZ>aPu}RU^ouEatFa?X-Ji$*) zZBuH;wO%Y#6zXQ z>DU~6!7jP|kk<9*OOHujBd6lt7pxyHltoUizQ_$V2(8N`@oY934-po$2~-!57&U$o za86r5`m5fHYWHVD(!M3M46Z3zHOVP#P2P3EFMgSg23;=<4Wt^$Gxfg0qIAV4Yw49Z=)D5S%7lJkUd(P_yG=il|*5_wHuCNEOd zw5L6hR%-Ogx?ts@@RJqRcg|6x0S$hU+X-KI>6yKR1`^KN+?Ef#)KXemQCakD3dO(u zL(W2S=2DI)HnZl>$cx1$u@iN}qN5?+p$hx5$mVm_?k{sxJ1q26 zdaRgLDy=cQoE%NBfBCu5bayqOyHR;BeyE6H9q8D%W?cZSgnj<6G|46`nysQKWJen; zH-w7$SRP{yw9DJ9vcnPPOu?xvaJ&uiI?-MK zdfm3rtn6X4r}Hj6(HTDv|4fUcWK8D=2Pa^aVyriqn+OEt;nn~!eaZYLEj zNwT)xjDA{2<<>=yTO2|LyBl|hiyLb{z#|I6Xd zl~*p_C;!xMtb2}U4b1?spE4w^rdHAM(q^$np(x1@9RTdsfU3h>^S zK7IGf+o-6T({m}wspR!4J;F^>F?9|)wRieC&ROJ+>~_4gD$0#dFPNFl zz1CnhRB3s&&1Abb)!J|Tm)O#$@50yrbiY5h96OxZac6Q{XSdvCq%yryjC@GQvhMft z3lp*8qoYJX z9x}JK-I+)Sh9TlPx`UnTUf<$0h}U2KctwzD`)5X#PPndmw}+|1YoCrKoQBv7b?Pe> z(uK@xDqebU11O_@vjy5y!|iv=sfSuYxH>5({pezEEX6n1sZIfx?7Ger*c9`P-+w%Y z!m{mp^>}76S~Pp<0uiHC-g_v}Nz2{D1BWx@2St=lUrwE}?6^De`2F2*#wvf@Jh966H zeW|-_BJ^Koue#A+lKW*=)||o?mKXUhyVSek{3W*5w+m639jVRR!Q_$eGYfPab#en$ zccR>;^f3<%LigWlbeis+kz}}3#Z!A@)>bU`LRaoh^GlDDi)pDomen>NTo#Kpow#^@ zeth`;>nnr7=LPj-1h;3&gwHKTx9zIL3CrwomBl;jQ(oF2pG^LF<-+&9hV%MMTl%P0 zW_rDCwPR*|wk@JpK4rHf)6a`_ZO873FcTHuQ3dp2V>ee|RajnvO zU9~>KU%LdThv1tAmbQnAE3?k+=VQZ)LbGYawNG`Yp(TE+>v57Uo5o2I67i@8A=@kx1h2j zUP&gZYev5i?Fv3WcbGa?=HtDgpVe97SPq^RMX%+_j*6L{v`)lD>^Mrk*B9){Y_6Ni zM76jGwK^dF4qk;@Hj<`hVpG?3dmdAw=pB6-RCbxc=GhIME}g`KL9I-JJQ3tTWj%N%n`bINhH$58|cRf8>8fibe0L<&kQ<5fDWL z;GXgi|F`zOJF2PfX%|B1MT#_$E}(QoAfgndC{2)8=_Pa!5D>&d4Mh|Xun<5%iYUEE zEHovcpmeDMN|oNEL#TJ+TfX~yy?=gd-Lfu!okdP^_MVwNd!Lzko^cRJ;Z*SvT~GDt z(CU9$o|l_eSoVbBE#HXvz%Dt4!}|(sWX&eYNhZ27{*@~hZ3gyenl(g(;d3*ZOh#Y&l66Ka#%mHpYAjHsv}2|M$0aK^m7QR;Tz z>8>?%Y@c>dzjOCvj}uMUjklQFIS1O0$Vik=_ViX+4cO&p>qq4m?-B|k-IQFXFP*;e z#_`pVC}imi!kO5;m(`KDks z-^Sp7K<0o$0^R!tzpHgJr>?p(MIf-e;;-DEbQXoTaw8H{Fe*Jj|Fxx$>Rh6?P`I)E zJ=yqeX!`kHw}5cgLo*Vn{Yz3q*aOV16BMs(4hZkb8^r_}e@ne#kgBS?Vu%vHkL2Rw ztM^~)+r9))l4{zSxKbc@$RVSZ4+0u6|GOA~Ei9=R^3%5|L&G?!A&cotOY{PnDoJ9= zGAz`FK|2JjUmOIgn6Vf4kYXoRZ%t)H-0tghCd=gOm#g8g^8J$wFD;3LLSCn^6P=OJ zctBk(nTX+Zp9Mep%5>{bV@d^MqH`r>VxN5oK=hFC^5e;yIq53E5IQyfV^|B%iP+q$ zgROlmlH`VQ(Pzy9;4)pR?fcG?YPS-jbYjt|{dL04eBL8qIu=VS=I(vkw5{{c1U@$E z_w1dEmw?QOQQ%?>7SsL23a7gnEAfrZ%01vN`@Kwxr{?_G&Z8om9Omn*p6-52^ZT7` z_o}fW|5m(IDqEL*iSC-t8M+HiNzX`erP*H z!_2lek-sDT9neLp@tOl9ikT%`)Fd&(ZF^laQU5#<91U4{D`#FeW)=?H1y(VmXunGj zr^Y9p#71aD>pSx^2j1?iF~F*xZWY1-LId-oTZ9w1anm;o~;c_Rji&iCA6C4%5F0KxLaK~`F&{U4u{Ab zt(3A(q;jD3tVfBB-jCrxkL&f;&-MIr%lg=?Dy@R*kNNE?++W!av-fw;4ZZbk=%INr z3#ug|bT87}E!CjkGf~CZDB$EpGMcXPeJ_kRf!rxAhhcG6d39`SoLw@meBL}~W$*se zn_llG_vpOG4cM$Zt*@+}{6qfqhOMdm4!`N`A@|bu9GTKlLbi_um(p5ZjC+lXQuk6& z=#BSr^jEsRIL`IDT6*IstW+IcF3P!DE~&7TWn>3&)F?#9HKaWLi)Vm|E-2*3Ejxlxy{J|*tKPn#jPck< z3-g2CyN1z1v3vMF<%dk}V|n>S9UJusofs70Rb+#XAStBJrX0TZncuePP9r33@(Jq+ z^bDRrB1Jp6&C-8X8i86ocSP^UTV3c^XYK;fagKaxnZ3@Bn==oI&El+E{5*E9xRet! zF=Q0IiuJrn979G{C#!BjNlz9qvmQc{JmA}(VoIU^0#oit+S!zqTKzuBo2&R(UrXKAt7X0xN32!A7U4x!yUeJ zEY?`mu?qS$Lb|%ZwG5cXTF{=s2ul=0GV-t>x-EM%xj`Lql0QyNBW7MRibSGyRHC(j zH?PB}&&-cdSm(xb_$}r0aI)6RvxhB$)Jn&9-3tOr6yJ21I7(Elo?crckK);t!8(-j zxD@ue?b`2-YzfZv?aeu%n$y0$bMHtCPbj?po<2&U%i43qMX_Klkkb`)TEA|z;O3wO zy)?CKV4_HSGxS9ji!hrtM}m2fRDS;?vCA;INK-+%ZxS`FE-04 zTzdX(>wvZR1y+km9Q9tcF7j;4C#!ylt9OgiewCcR+@S6%vKbvRgd=w9c>|?Qx3Akw zcURu{iApo<87;^fCMhv2HKwx>3bX$d$w#6YEw%)Jce77s7W`DIwr9T}ntwvUhpzuY z9Ger7UUY-lZR<<^lbldAN!b-5{CbwvVDp;Jr%`cdiQy(%_wBElNAh1^>c0s)^sAPS zB(%PU#1#2tV3h2(_`p|tHlph*MSVaS4p?g%bQAbgHMIObQoWeIAk{SvbD9|k(x71E zo?vjb*PpI$c%hNC@9t}$dkzE6x7eipk}$Q~RI>6F!bO(^o{OdQIX_v9$7-Mbc#g`> zyT6#VjCq#N);36?`Dzp%XilDJSAMrTbh$;E z+KEw5CL&WxdF9S@kX^vLi>PM+6`n!$u1|N0tS8?)GOZ7d1M`)+fMiAG)Phn zDaKICiBt+|ytI1OJn7965j9(O0C<%5zf^57o~r4B3Z^GwMfYDUhpMv31CV(7nht=M zlo!o9_Db7N@Gn<9KAKYgsy{`H`k*A~&byj_9VbSZTd_(66SoImbg`e~?@_n$+2U=f zk6Dhm_2r1XQr;;e7ot*{@*Cg&u_{v^ytl|=@AYbDjeRX?=AHl#3(;?J<|z{I-G}>6Qfxv|8Q)HY4G)0R*jgFAnD;7%hiC*eZ507J zy7p!EEvgbCLfvO+dvsvHp6Bm4VD*3snaKarBWJAWez`58kJvKv^frZ&mNQp7OCj05 z8?}@C(OTdea;;pkV$wS3P4}#h)4#^X4U#Kgrk)L^@B4W}vSrpAxLsEaieIHcSkq6B zf+){FcWU{PI*8EE+;7^aILlK&ZF7(Px%!B>c78*7oT?9XZ}qYERDLI{2P9Phv`gNh z5{~nvcJlmwfFJRdjiqzhSaTL9bYmzpwu7}ba@Kn3`>qyEc*1x?NWiRApb(d&t_~brL-Ra1{5E=1u88NU4m)@3%>K zBoDVhDW4MUaga}VN##O?SkY-^IxczLx&#O{+-!3rJS@7BDP5#B8b@RoESBs)^VC`w zcOZg}GqVj?WbARaeax+BrHDY5;!XbHU{;n7M^i|Vn>KTMa%##5sY``sm(w8ks$Lhi zvU4erSZ{jbIl$<363hX{y`dwz-L#n;#mVcZm<=JZZyD3GnHo{6Tf!#zCz9HL%X)0| z$eSN1BJdkhe-d$U9LDd^+KHx;gp%2-pj7K5-afc36>WYXP_%|LGJ98aCFW<|b=Os8 zJ<*?iwR9yWsq0d_yN}O%y>vUV?2ud5;%nuJvELP^xn zsXAGw&&y6c3dKQ(oVGq`6^c>ZvNO(J5#fmFuP69+#|$82mh74mI3xD-qh#9RV!|-Y#ny^l9F^(?QP=TtSr+EdPL+j6rhr((|4@O;beKXy!z_jVk{emmaqyy*VA9p`}n1o1W6U}|f$QM#JGRiFygzIc7i2Lc3kT|7wy%bXWU#sn>)Nn9zE)sBO8-)b83-~nx|FXb@Vg`q+h|K#+AAbm!vIutETPl7L zz?tUQp9L~%tG{7xp&*Y6E~gPC@&bIveH(PlXg(&hcF zS*cVh1Upo((oo!PGD&R!S1kpE;*X#QvdIQF;v~A5m_d)mi$HkI;QMX% zElLW?qpPfAn_75!#Jc|#bruETW>c&R#^TsDHw81Gl3|mKlASAJ^@!~j!(4fQ%+J~n zobWk`_g;;?Hf@%Y!N4s3tYAFp>pH)LUDHz@hI7)cw1mJ9hIFKKZl+bO5W6%weIw@0 zqr5s!Xz1Qj*WX$zCb_zX-bpF67_rz#adaRr-1x{QMMhG5kOXW7->%dtpQnGl-ve9P5%BDW@ zDvNnHGEZvf5Wz>O!J@JxK53*!kayn7Xfxj%H`96P_U=w#tqa|ktp=KF!XDfU+uR*o zq(|RC@#WM$tGozLK;7l)Pdt9Q0a5c1XMEy>JnnH&Gc1xAsPwvSz>Xm&R?Zh69yFoN z>=187GdXN4Ts-CfHKvBSEoO+y@3uobRkmW9q-8~Krb8fo zDN4=m*Lg*aYkE%VvBE_|r6$=*l|%Aw9F@TG_0F~GD0{qm6fURW2q3`Ph0)cq&y;jp zCDVTK0L23j5Z@e|)62^mrDw)PpX(p76|!x@HqP)?8;eQn7LHDsYd=rD@rXX}QcI(* zofww3pg3}W)3Pv2RKfKa7Z6)jx!Vmm_(yWm;;OScE>uz#e6M|ou375m*T;pt#HJQKMA1LxbJHBeize;O-e+sig(VZ7{ z$#mP}+DM;>R{O`;=Y#pY9J9EVWtq9?EsLe@=I9v_qt-WdN#z~c*o*ge!`4=yD%`#! zOY1mPANCR32S6QnG?%D2Hky$is4=e=i(6Hh#2v`zIbtHqdFFz|-BD4eX|gQ>WMsNC z>3iHt;+n1>7f8B>xqJzz7P5>A&8!kiu98Vbe?UqvE`M01=m`SKZg!UIh7k@SsGRV0 zYGekRU-=Mps`ZH z{Aa(|$FJoOvZ$+k<&0puoN>@q8N2wG zQ<1b;Q_nWCG}B}qK6p#D4&h~|PPzd@f>V-nFoIYA{NhV;*MdVrrlF3~K}JFtdnAL( zjweP+e$x^5=h-epMhCqxEsTmh)USV4e-1qh8lIp7KNAa72?(#7kQUbFoATE_aU|{^ z;+${|l)1ii{$=wktBRNNHWW4FbFLQMk%6*`vB#>Vp$!jDQ8X-PN4O5L&U7cxtR4D1 zGZfgK&e~nUE%dtRy;j|;6b8w2_aih=ey}VrOK*=o()W2m?iAxdXdMt1j=b2O&D$9H)hcdOB|E>ZM!S+DNVjs~rF`QH zLhQ_0Jpx;+mH(2)yNA?i7thxpdr#wdcj1#df*`yI{DF;-IN{?BSbZ?O3uvq(5Harh zoON$+c3H`2Jxv*MY{WP=l%On&#I5?9?Wx(sG7&MdnY79CF5^0=XK3tvyuIIF*ee2a zt0y^wa&vvGwVx2NZ>e$3TrI6svm29XGLaY55a7(>8V2G3&y6$w1+S&~UegMVhsvfL zlCIsNbvoGGSgKv1hA1bp$!ArCU~qBPJ$SoXT<=U0Wv(m3{6oqTf+(MJUAXH|vaAopoWjw7OF*bAW) zLNu`^Hfj9nK)1as7{n2S<1k^Nm61PS!k&yJlUa#g+_nD`G|I6k9Db7N`-v38BSidk z!)s!Ha}(C)!CRBu1G4m7@~cQ36lZ8&8(qraLH+b6T(=HUFdp4jtv5A0ip@OZVxcyX}2bl@XL^uJ&Pv5R@D5% zWim1ZQo^^<1n~A!CH}|LuET4cE1DoyR3=ce3-}_*lAZK3p@^mf9MXIxJHUp+X|e|i zXKHa`>tk}P3SsOKSyL>Bj6lf_FV@i2avB0EtE+Yn$ouKsPTiHG=~uJ06~bEU zg!%Ve8x9-}TnsvwRmrP)L3G);ZByy5a!o!A31ZzNu=HnHuxPUY<`oQ5hc=-y{8&m| zaX|0hho?VGE0qt@e7*L0JP@!z`<4b+{h#~Wg>*cpO$t1XfU2{6Dl=*fieK6%=0*+_ zZ<x3+iQavhlm-s36bs#tOW>_@XJu0MK8XO zCp&<|MhNjghs^aEQbF(hl zsV@lks_z`&eQ#y@Sj(|y11fweu=HK;>OD%L?6Soi4wr)^b_Oa|~0U&u05Yv~=zIWftQT zUJ?ti*L`_sh;!6CGp2jjxy%vplCZo9!m7M~G2kDG18#)2>|9Tqq&YxWVwJEDt{yY4 zO$yf}ZFx8>_Gb&NM?j+H`<&?@3F44=b{v`su|%g*Ru7>);%^bFlB~!2q@eUqk?6}_ ze28$61=NkW_nix`;#^0X&t?VxqUNZD8p2R5@I;jzW!k<#DokCJ&VRYsYJlYnBP#e8 z|8iq8oClVLTlZiAH2z`=Fh#w*&J{i{ZLW_?KaXQhSM5-=tafzI? zMKip>>~)?cRu_|F4t^InYGK4lT7`c@AzI0>w3z&Y&5sigR?Xb4eH)7z8=tbYjse!d&4i%_rXVf z8k2M-tuZp>9U4?(^bep%T5-WZ_$a$mO-f2ipY~Pq{(*id|Egyl)k0D+@~H-sVDO06 zY|IDKtP@c^t7+KS$eG5NGTsJX$1gYV0B-08(!0=N7~X<_V=ZjZtiZaZHAnF zKthTj{siNBJmG`^M-y2|s{u>9(n=7STjRM2y<5IcMJO1e&{i_4K$|;d$4WsKG zBmR(~t*Skp^w+ngFX9P40)Y?CnWwN)-&Ri+r0fYL?=P;l+}UfJjB=8)igxanrFzWD zL3rwPmS=hyoLeVies_CABr|zGY~@odFT0fW`)2hQMhGMu6UWJYHvUcuQUoP!q8epr z#Q+&;6pr$UWQv%y9eP0WOW@XzwkYept?lm}0kI^Q`v}>=mHW^2c6;d(ehyWqu{i@s z;oZ%Rg@LX;r4$lO1wwYh+R{Chh3^ZqbcC5`5%)2)mv5Hil3&YVlds16R;Md>%5Nh2 zMZQ?0C;SGhD_7E_JI7~cW=1E56ju&_d>pBoQ-Wl-)^ribQR+msrQ@cB0#Ueb=G5L; zj2zR@Gio~K#iwsG%3oPp2&tk2NLM^|qpzRms|Y@MP$^?)%`|1aO-Pw() z8BMJ5JLW2O+{||+(F_{;M}q>`oCvaAq%gcY)Tl(dXX|{w6iKuNdawnqIDh+u;JQ9Z zWou#8U{wh)1hZSS*uT|n3F^LmmN1J$HKk=RfB{JB;%n?VUQ;p=t%+z8Zj7GF1Hog0 zl}Pbc-zCCv_}>+HP*(rBGlXSzpso%m!M1cUI09HqN|Jw_EB8oV-1 zafBCG@V(wdwIxt9@)_vhYqE?6D}?@fjyk)i^8KddvGimnrbVHKtcXPgoqR`Te) z=2BUv9gN>XObl&t&O(@w)cffFf$GR114E!=@s#G&oImrI)=3du$P|4AKEh{c?Txea zf3FLAHv_%DUha_~8V>j)F#Q#m|K}BDlgL0HVHI zlL%xcc(4E7)$!k=4mEC25IQk9v_bmH0fE03A!9G!uzv+-^-m8yi`$f@w0@S_gM%wC#}#Z6Q~5sp)a^QopTQ_gLwOsAalc;{Z9)Z`jAgl zo8D70F$>EE_|dtHdty#B-}f;6ctw^kU#S?+i=p?P-@h<|q?$V@z3+?IQY6|=7<&nd zmiOXQ1b51^{c}SEGKN)pKW5EG0fAhmN({PHYIuxzTZ#yH@TbWha>&CehmJ}z@fn=O z?}#zsgsB;w;Wqk%2}CPI9|Hq@KAQ3^I<(UwNMA%1{WO@M&^GvpnlTZc-m;fjTj0Lv`mN3FbIr!a6BOP8%$|6sGL(>?`Uo3;)wc61QHw z9effyy~+98C#t6K{L*&Ul)t^XD_dott9iM>_ezd@C^^?Lvv7pLe7dJKIXMXhF_-CznB-LT?huj`BF~nubsDmFB zH{!|7_wv-{P8+3W?1kIr@hf1ohOiOk{`liGaSm>bC8%%U_`iX0-<%{Igl!50GV72h zh(Nx9t8Xd(k;HK?)0m`;w@6xyGh7%ExX4I~2t_tmpZUTAjiphfD^eCmXo%K@2_+YA zWuPeZKSi0{I6a%H>Q1^+?7LSXVql8j$xnGdNZb)0Y85`7hid2ekyJ=93B;Yu zBe1uDI@J=D!iWzPh6n0>+;Ao;?eAUc*9E2wRN#fX_`3@5>LBQQn>>2I360ShgmG0~ zRPY}${(X^n6zwHX+|~c{AuxbZ7(iW%ro+FVPl{lHNeYh^uR8~E6RzBwi6Frgf@?QA zmL{l*mV@)aYxOlX;t)3AzJ;jGwLS#H1*cAe04bsY_O>xn(KHea4z6(1{=ZIS52;L2 zgs4T;k{B*<8X5|cB9uV3Jb8fQQKbOcs%Zn$8pu`&=IyuYzY!B0+R4oTy^+%6f(gb) z-3pocJkQ~GDnE>{6USPT4yF^5(R!t!#syn7$;>Bhj^9a64W4A-lh%PNp`Yc%+oZMe zqi&CVFq0@0E0t*!lIjGENH_{E4l+vb(}WdhfaDLt3KiSqQr-St2<{;3Uz_m11y={N zj@+mCqD(Ze|M*bV95nyyDD>|Syx_-1t4Uw!|E9SwY$YgH;`aIP#baOrKep)+Eck8X z>jU20X)7d^J+t)w@#oLsSQ!H*71QM-MI6Ch&@h`!e3slvlo0)eK5#*7;DT5x;4q~H zQ_iY6mQ4I@d0LYc_ZQ1G*{~3vGB0~0kVfE*jP#7GkyNLJ(0fWkW0S;T$610Y`GsI+ zZg|ohykkhk$RUc}qaRQ{&;VUk1>GF-utYWbFMk(`pw@WP(e=e>Lub!XlQ6xr5=eXqM<>W z(al~kb3x#Qo+vJ$kf6R4ee22v4 zI0za=%ttuyFN9B8`q73f=*eI+?GC<(YbwM-}&*{gshuMT^7rs9*ec*DJhE?!|sM{r*=NP5mHs`hs z@Rc<+IlZASXVo>;=y+d|IVHX;DcfgC4hI-SxM&6}4Qd`lg=pVr(56>h@lC1F#9mI? z*}WI@F@BiG{1W{^&0`>vLr+YHa5u!FE2P;~_L8D;u8Rt$1;k>yatb|M zWttCaKr`VJC9Nm0JH%twb7o?U%dWqPHzw+i({Pu(zWpfl-h%as+QSn~t4=)Sc`iyb zD(ZOJ6#m^amX5O+Uys9gxFq+pADD{yQhTX>(QqY5yAgNE7Ji?ZUipc6fN;P;YZa!q zqTg14L584NdsLAw(p#vZz{Bd%jfO@%|G;hiRHdUlizm0S^eQyE@1#DE!*95V$9|*v z0f7>CVg>=hv)b+>i#v%Ky(u$~u}+vg1j85TN}ioq`(cE>rs!iNq|bOD<1?J|=nXz* z(ZvR~;+h7==#FeXDt@DG2_|`k2I~v9z9OXLd9zKgi!*d_n2vifMxNIW8@&%8Nc((z z`K8qL)}!ZHF5^B2x@^OSK5@|k%{1W|;<)plZ~6H#**7I$ZfFT)O2&=+JPnyse@opG z_H<;1(D%gVKZ;^MN8=?` zdrsYaA+>z{mDPbXFTuqMkxO>oX!mnDUob0T88vJsV88xEfrsO5VWPEJpNDos6d6mE zzqNd&7|ST=qVlXy7Tzsy#&BG(HZ0{p|C$VM1_hcUe`hvQf!o$Xq^}RX3Mu+M`#!#- zSI>yRbu%!zrjGwGkg5S&?{*J4&u2^4$uQ2UfYWVrv@5H(2LsCVByYHwI13UDYrUAnzxci7g+Qv;0f|TP>&LB{=9<-- z!psYoE%x^*SL`b#smKxlMz>ErwT#xrrobubo4-HdZW8>iHo(*2v7n zeSWmlrlaFyTv(qtIp*^e9)VNYDF-JRw9*0e`$G*;(m`+dG;OKsy>Mh*dawB0`3P~k zOTjaJ;Y0)v`$7aIA+3+YU2+L_zhliy^e>j4Ut>f!d`_f#b^+N{&X1H9t(h?E1pv0mqxy%c=C)SNSjuQ&~7K$v$X$ z&$~!ldOInMO_aHt$$>{bELns-b*Sgs4&7)Fx(r!gQ#*%oa9~l9Dhp;?WBRa6-VNra zoR8{xL_!O>s#t>|u!g|Ums1V9FPTc4 z43tKdVwK_&J6yeWHT0s6evCwh1fua~h)RNithsE=yi12(hg}C|LM&%>S+hE~x&>>& zpiKVbeER%Eb;PK{TEuA4=!MasGH272AX9*H>*>=e3Q5)@k1gi zgqt{tdzrj|f86@Mwad~b^2^5Uw8m?VIE{V5aAG*|enMP=U*aQ~!GyP*OWXrISD_tT zT83`@rES4hgwN@r8ir2os}~-IebSq~$8>oUu_ zh9XOR%t%Zwiw28(UVEN|hAcd%s7hb9+(`X${*Y=)nWweBd6Zd6edee6%6uQzrt;zP z#=T^+4z_NqNUi7zF85TyKA|qp7f3^_{!e|Piwa9NE!Q1)>~8Q{*;qJJ-#@$`n3VI@ zitlNXTBmp?S0}F4j#jGHd=+O!4Y4j4wSL3abz?~n$6bdG=ZSeY=k(6z<_LR9?l6y; z`RPXcT!-p;IeT3Dl)1^p)6p+I52uTIW?~p7>`NCgx(K`2wI{U|3)KtRteH1tHg+d@ z;;Y*4&+E|#mzB18d0cwC03n-h2TqVffyOT|~dV_D+5H5MDs6+4w^U#Pw){Q~!5o$+z8@y9U#=TzUQ zGN>%y^Sy6*zw&^>Pt9-6Z@_QY&;6DBWnpsBfPmMk~0iG~fhQ<4@6DU|^N4O0*|VnS;N*uSy@}kk-2HH=W|@xh=)-uDw*2pV zMz+O+#=D!tHC#2ZCzPj5p0V{hA0KS8j0wAo?`@Nwy^ZG8`YfVb`B9^RFrNU?8PYuQ z=2(cG+geiF%hSwL%ST<*NHj`S@-2f9SC7N@H-T&Tl2U=``jH#y15s}2PkPs&V%&0r zs@a!?mo?^`wo+zsgD_q*1qsQ>#d|z|y?5iso52s)p5Bs$DU&Lnv~hY6TdTJ@Y%keo z3cvl7P-I9Cy=QLJ#?q#?ZS2^2Ed^nJ%}{Rg?bLi#rxwS4q3eh3goVWR6v49}^5299 z-k7|3ByGjEsqxX2s#c-&qh-W#YFSWi%$79!o64BzSeSOZhR3&s2w|0z@j6;}sWHXE z43gOEv4OD@s^k?O3zybzeq|;vBt;Z^-qcrG%BfS9SB2G%r+M@dw-O&FguY&U-B_k< z9-EJ>k*ipDGqf?BaGTi;8mybDpRzI3?`|BGrB7_-hK;g}dYDTe#2hgt=4|CRA>W`Ky|P)Yrd%YESFYw(|T_o%W!{rNchF6#hTR0k_Rhn;>_rW#uqsM>DEDR22a8Mr)pnk*oL(5ZO`Gu zh{GSNS^e@|NAzCwuz~}^ z?D=B)G9C}1Lrw{l_ku9LUl3tE@eJE|rNhI6K3sjZTqj(IU@ym(6D`sMZFyaY^BHxU zlk-!%C%P#QT)L;u2=^?~qtQIn2B)S}2ByvonXeG;g*JU{C4WXGHhj+_^e|E>{K9Dn z<8j}H-N}z@H?F&QQ*4}J1lUL1y6OWX1-p;ehN@CVva)DQ;2IkZf=-Nf0bHSjZy|J& zKi3lIjA)oY?_;2$`J19ae!n9Jexts^!8htNzkXvzzCyzSf8m2~mvjB z89se$OEz5tYdu3Y7fTydJ7@wfeBjd3&_S2d#nQscp3g;)`sW*b;2L$CotpCJD-Py@ z)T*)yln<@#3@N$U?y}vb7P>@9Nhx4wV8o{=Ch_~@;4eXH69)$yK6Z9zXJ(@99T}=P#$;$rsw7>+}QBT-8*zU6bX&Zc00Cks7!PLdjLQTxn637hnA;fhb zD)96De?0lC$G?42^{-EIK%ty}|LETy{r5+e><#T6T3doX9fbbU*Y6Mi{o(Hq3b3Q5 z{x?zllJn2IK+r;$1la#5P3TgTjJONXBdMvF{4?+ySQ+XIod*11{Pi1ktuRFj{ZNdC zCW0m5E#9kT={@OFXmR4 z%)kvLU*cq{M#%u;#74Y_*F(Z@@V^&*Z8f9+(RJ zbL`$|IMAbi>EBzVAA`0z^_p;qQh(_?V8eu0h+|1#p$X%C382Ymyg7}f_W^5z>GI~G%E-(7eAfB}9rLEJ z++W7`(+WTb6%+v;@2wQk?pqdM>?~;Vz9e>JFLmjDjT-!E)D6tQ ziYJHrw&*W!zN~6NpWdmw)SY5Kccm73B)v%$n&Y#xko{R!|EGh%K&^Zm8&mb73kMIh zRu60(M<9z`BQ<*LvR;7*EWZKIBYV{!>0Cb*^Wzd&h&+3xH7yuG%yww^VgD2J|HS-1 zYySVd(+D`cyn9Q>LpM_{DS=l+&~4`~@_3&oO!%}SUf8Qv&3VGHLobNcXeFM@JjQjq zgNJ$WgW;g2y=))CegF0GL}R7>Lc&6_dndwjyQ&@97B`@*(ZNzN?Q6OBwU%oBl+=B7 z#nh|kxQ)56MsL#dc&%Nwugl+cd!7g>usl+dr3Tv_fRP;?rjMCx)ov$k#0kNU;=E3` zQ|9Mlb-0fXOB#vI``ujF=ICV;3~Eodq)V1E6OXz^HJPsEMW*gA)K{2Z`xRxb5>Sc` z-c@IkcvmITu9L0fg>>AW*zkn&DZE4=k9O~!9W)8IM@qAqa>i#!mG2JeOvMO%51?tU zTu9dVmR-ohTC@4t2(deO@1g%SE{0^6)E)zk!>z@%ump>B$va^{xxIG@oC* zMyguOKF`CkzYsU0Z9Nt=sUs0eYRXo(M)S2^3BP&-5s&nX@V^sBUO7=$0kE^0+Ijn1byowieFQ3W` z%ulqc`XC$0mPAPcH3(2MdH?c+BOyGIL;n?pQfgrFN@3mP6$+v8*g#i0pi}$t6-)P? z;~@IXoqMSsJzd^i!e`?wVERTcZz?lRS<3ORac+MPVpo<8Us_Pl&~IbzqfMkg)qfOD zW>b$o50t>c@Ycpx_D!_L4J54R*uDo@A@nP;dQC0jo5vV2&6i35!r553eB9kn!M8a+ zuXh{7hiYe+3Y?v6&*v7_7Q^|gBA#18MjA=1EjW9WYHNb5)i_nfSeC+eY;{t;$TL{Bc4sPYmq-{bH~{hg*Yplc)R9)bBd{q{Rq zZ@)W@G$d4iEk;1%32P7M6FWlr*G(kjmQFzN;ie z2-xo4m^FM`s_My?s#wSfk3=5TuVi0j2qeQRZuXr4-ZxDKC;~#R=;Y%R7wI<D}@Jcl#f}FXI;mECW!!afoI(2MGw~H#Q>J1VI4{MayAR% zW~oPy9pBN~h;a%R->57F?K@oIp04gK124C0Pz2Y`poTTq{-k>$lH$sXi&vr58@1Iv zLwP}y2?aRCTjC4;0_znstvt6*VYzz?pTF4o{L1|t%Dg?|^+eHp=&_sYRsv6pJZLBh z)HMhkBd%*stuksvxi(tx#-~qhM5M<~(8E=aEZ6dzEhA3_BHwI(C3l^@UWXVxWUtwL zpdsh!LBEALKtZv63CTOZXl(7CyLZc#F#zeTV^w4U?=gOB`-+t&YA5zhMEaWvHn?hx z#l~VsMy$43Sd`R}^y^kllNLe^3*G$>Sppe65n>@;?C=PNErqB^ecDga&6v_2@#2JP z4TXhO%Q=C;A6f5g2$>ster(9E9$v~7pt|om?Su6=^ut_;02Qa}?tlt%N`l1FpztBS zK^dub1++8W=2qi{bG6c=dnOrwPrZwAhy#+gd_xtw=ACvZ^z$ht|M6-G^5rw>@QxH= z;fNME0kSAZ_!5_ARp(Y28ZtgPT>PEVv2+B!L4u5;q-&Kmb(&5TaJ_UJb!VP-`xDL# z<^`3R6R~o=u+2|cQeDY{YkM5}jkH^vkz(_9G6pv+scAl~?l}%=4O!V2IY<>;mQRw|@se0PoC|EAYpT#Msi_j9z$s=id-F3j1qpQxNT}ynAH%Tw z8M||Y@yso>wZ2wCMd{_MN`=?2@y_CTA|MYA@)R++8AgO2OXW9>3X|uGq z&OKKtMO+M`3+lUAdW}X#Oq0<##Z(Vp4O2J+F^APPI{3svkYQn0coM_q=roawfnY3mH;3+!}1 z+}nN}bz%kLj{7P%;;W+YN^2~LqNCSMmc|;jd{ehbtPPmB_xoT-WaH`lw zi{wK;MTM`~Jqa2qudo^VB^}+DH&?~>Jtbe>thDT5DeG5s>Z-7vX$Jw_*x(w}ej&+O znb9&hcpeI!itF(Yt=Z|7o^NF?EG1Zy_HA#rjNOZq7dmXnE9vBOf_m)^WgS|feDSHV zPmfb&T)Ovtl8a z&F3^Y!e3~SbSyQZzc%AT3!d}nqYozzP|HB2u3P^;{aY>yuQFvJ_h`yPPL9*Co9_82vE}6SQ z;w4N6^VC{^>zc@{kz4fY7R4GlZ0_xGplMF_Iz8Mmsw5h}Y;H`Zl;*-f4E3 zra5QnBks>ojSi`L?0wx}Yc1r0j}dmonFjk=G0)7-O95k*5X!zD9S}W(xVo?FLhC}c zHcTLD#B0TEZ1-p+Lpg)ksVnqyS=n5KD_ z`%xNAS*A?9p2M3sT10T@XLYbdWX?{JAu{RKl0mE`?a+C|{-n7*8`*9~TF*N00rpd{ zeInPKbk;PQ&-C)i7Dqcv`cu`{4f--=@0sTnNhG>Cq>q8vuMPE;vBI%@Du-`5pCybm zOhUO(2(T-hgx_p1wt!O9a;cPBH9&@K+9S6?{CXRoJiEEPnKt-i-HV?A9l3EeSj-`@^OM4r`>FLxm}XQhKA%G3mNN+2beIev6p+PCYF$q(8$7l7n5kjN8B@yS!6S z@F;w#;Rx#X_0*;2`)lUM1(k!4k(0yiwftz}hw)8W>7lAq-6{GZ9*+;a~};D>4PxEeN`9p zS0WmL!lY_2yHDnS`fFGkzJXoPldah(d3t|xHZ0jCzhH z$)Y036MO^v`F5!oEhn|a?&YCE9jM!m$#E8(Y55Ftn?p}5wdJ0p(B80q6iB?N;`yDe zQgN}nE#2G?hl}(H$eu9EZ^?2GKgX=rn!n3@XhAEgtWgIfuBoxY`102{1yGPmAQvlZ z9BP~wX$v9oC60v?)z0!qduxdx0~M&;9n>HoV|`*O#n3!nv^1oRaHBiId}b})OYxRv z3j2PRb@|6vBFCtx@j8<7sam`y2#hDBLgb_x{!5+MlnN&3$$EFD%vAAAUU>&UF3yUU4={S0 zY)>8j7vM8XphtAmD20#Leb0ghE|*X?q&;?DL-2)A{ua2r7dd+KN1>udUerU5t5boiR(NDxQ@cNbdG#l00Jq$CL9`#Ktop2 zQ0--1SY$W7I;f%LeLB8;efH=VntK4?QzAihfOLT*L2C1DITe7pUVXf%aR*5^+$}M3 z_sIT@ZdT9F6^jnQn#*^yIX`eg4jxPp+kR&<2bj^&yx50A(DsgBIKX+R#G}f8VMZ(f zGn!xQ+snKJTK;hIR=^lwVx8}NR3yD{1nMKW!cIxm>P^{D(h~Uy6zF@E^(u1}P+$-_ zRRMbtz{G+G4VTz=eE!$$|Ag>AOZeY%_cD$ zq;M%YJh?1T&-pV0qjUif{3rjoi-bzv6anHPSB3_N&^;-gB-oSn4^M&^206;!bk)VB z%4+;{s1I%c^jq(lDFFty3;O=Uw-jv2em5a_0b;18F=X%kuAx!h07nvsg`l9n5`x zLTd z<|O!?3fWx;n}*9Y(f?M=D$M}}2IUaap^%?p_dkDx2Z4v0w?9CBH}vdwIJlUSVgQ{)plYT?mq5|UeAblbx2*eVz_7`=Lcz#l zu7f&Lo8Rlp;%Gi^FC3h-5xrUdEgMP`U<9K6v5jEM4h94&mH#IL`bX{4=Q&XO1%k)K zzW=F+pF&g!1KM!wzkl&xv_W|rWzl|-ul`NPe~tW~pTw2`BE0)Po$7b~{vYbv#SW*; zl?>6x6M1I{sd8K~D&&%ijHxtYy8#+%eh4HeaO1--_z@kV2ds=xI^C23BjVB~CP#05 zp9KW(uE%K)&bit6i~^Gxuo9O7st`q~;(2$40cc1ZH1szsqee9(_0E(Yr2*Mni(Ce;6kT}HNJrx>!iM2-`fJVnoCRJWkk`-k8Gu@jzilGL z>#qXsJ0~EuGL0wr|CEaer3iS2eleQ)$3i)dQ|8e`L@F9%Ag6rS)(*E1J_)7x+ z)IA!i;&}U6g5N*eq*SN|a~aKN1W7zAV|Ts=kNbdBWa;#PeG4kpLV}48ST{xd)*)UB zc=%$G93TS$jN|_ub>4? z{M$2Kuz$PEUt4i`mWg758M4uQ# ze~ai(-P>pL0uZBTuqhP=_A^vuD8BqF$bhDO9)tNYefW449pa1fFu#^%_!F=^f}h{` zO`!OTNg#57aUya>xUm!u9dcgrKA(gL9T;5nv*A~07@&BL$q}c}iw9O3BT%ly-Ph-? z@eenH(gQ5VR}}i{0caS}z@g=?epWs=-*=I~njSM*{F~4NP~L;{edhb~e$OoerMqb0 z%ajy#NzV69e{(G&oM5J@I9ykMgM_ag9CXA({^pSVu~qbU~o~WqS(r zU?f-k;NYzo@Ya{t4EezOkD>%L@jCLnso&HW2rCG{@4iUS0MmF4w6A0{{A()=*W&~hya5FsqGj)(P1bc!Z{@)XiE)Ec4o=Yz=5TX`XK)-zFFH!v> z_K0+&I^(P11dBTd5KL4q_-mc3R3skA5{fqohNOymi#fRzL|wC6U`Rj1&TsMj|AC5x zt^QN~FsVq$gdDLy${11X7`D&ue-=0l7{J=tM=W#*1|D!bXrk$0T`0r?8+I(R#rtiY zVnF^ZKZCj+$~W0lBSA%n2&rKdZwQ1r(;s0S0Glz7(RN#MYWp}t;zJWA&^vC| z#_IG109G9Z)$7a7LYwuNMw`>0PGM!yUkNqZHQkmnjKT!>vX%3`9c#O~RgMela=Bh*58~*3T&R4n65@NMoC647pc#@Ri-;(-vd|jB#Oq z3G>b=sL4e)q)CR6&x3NpJ=pQy?p&3fVQ;#W1c9f#FtSvT6{onMXJNb0{4MY0-%9nP z0!j_O-+KwD?cE&=HdozmJcWKMQSw3@pqO1fVqS=7M==Wa$(+Y+ z=Nky*S?&xi@_@VLu3*mX{uG|7c1-vNbfo9#JqQGhfJ<160r8PW1#xwy>8&<)9^tdY zF67*l`v$@Yl+brJCRzio(^!0$nJO%9WpW~Cg3A-=F(f%e_UJpl>B~y-G+XFeo~o;x zDA+d1hy}IVuEhXaEr}3fLmg0tW3mVJuCXWP8rF!i!+}CiLkIE#`%d1baoZ-c`FP`G zj@K)EeWMOy(NorPR4M^g zp9I+c#1Ko>V(R?iLJE8c25Pg3vD$7eC9;)bS zy-U~uO8A{?#f7n2PVtt7M<#ZPQAutEX`y_rFwI`3Z*4(9?j6U z#6E)dol4>N$3{Tn*}q&mkHR3th~ssi>9_ydgNbNiXjl}zly_@h%xrgTe#GomL;q#A z{IeMSvAhA)xlT)hite6q=Z<3*fk8rcL}e$wE1C#3-1$gf6GJk2G7g1QtJ$^CA%$2s z@Ew*Q@%$3x)ZxvnsdP#6lzC<|a^`uO=0^FeykQdUaxhnq{ZVWAlATP+O89~53EMY4*fpT*DiCf%*;*pn#x zo|xN7XHM)d=N8iQCA5-lr%@oA8O&$*Wb3r{+w!CL;()ILWRN$_9@ zSyK1l&G`YI#{}{M-?$m<_|v9pL0#r(%}zRFs&?Aq%wt%70*!a&;$c&93vm9 z>GJJmJR5t`?NJbrMoDu6NdOZdi}MHxiq-61hthRgDhA0MUhQc>S0a{t(ahOcD!kG>Zzm_Mr#C2+)_!i*G;Ob3PS&MT7%|YdCJdM_YisH2h}ula zZBvBXWkWY%oZ?dMmGk@s4I`f#q^?TTUmZaTqlK~V@2-Vqui%fdbX~cL)LzDauJ3|D zu}6w35xE7ZtkN;yRTFzJzMO&+r>Cy> zeS|Y)+BXX6V2J3!-TMfozO~~)cPTRYj?F*uxCkghA_63tv=(&3bHoQ;q?vR_{0bzTFUpveigJDjj=1(HN^-p#Vz6BB`0%SlX!|AR2h8 z7~>nL2MY1_YgN|@f#6oYjzd6yur_90hY;3rXXa~4$dx28#3X>_k_`j#tfRS zR|QGi7y6d`i4XZ8VaVh9-k=1`DhXu9c?1QGUnV`aJ0fwuIX;V{5ASUQDU}ZN^P`2b zUUA8&%7ewVkQ9$CsiV8c{St655cT(tiK8j|>?c69Jr$_hqB#3OaZ2n6b<^50jajeS zxf9Q3n|0hP?=Hj8jEXg0!JWr*Ip*Q9td1nxpIwR6e)|l%PI8uD5g`S-?Wm7mXe}S679e?*pPiQ zT_(A8mZ=S4_;WZxT@3iYAjs=BU~uIaDME)5kqbIw_daG2`s_e)>4rN}pxAp1aO%tla}TSs~( z>6`B?_5@BQy8=?)v3cK$bJXb+7`LZ;7`Y%m0``mo!=_7cpB9z89q&j*SoMsN^vQ?z zGl(unaj^p?B|fICT%}RklyVuQ?Tk{^6<`VIG7v0+FU>P z9wMJER4whc++qZcBvN+|E$5i4*Q;_X*Q-`e=21z`$&$QR@dh2QvFCpE)FEsOBC!px zLw3{-RT-|cgGh(YrQ>;+0b_?N6B8i~4LGCK%=61uxru%l>d@iLh1oKLl&2s<11E$w zBse%5x>YJeBmc3q+R&4oriDGNiM8%zOM-rD=(pp`^ay&UU#=3t&w)n&mGNC(2(&^(^+MuYWYIb$r@L>xp5K%o7AIG zc&v_RWyEZ%me=82YzQrLh_;Ss;AQMNQ7 z(UEm<+kS7LMHPN z2Zg$NeI0c)h@stI4Jel79!a0Do5NZO;3xZ(>BU@7ks3i8qXzb3Z9#|o_6wb5X#=xC zFPA2&;ea)StQAn6l}Z}a>Pn-TGYBnxO;bN!2{w6jH1O=v;z-Jpv@HjRWQ|c0(fPgY zKX@iUN3R1g_9h3DCaC%Jqwpwt`b$dJP1r@1=kT1OaTYCe0S=?^Hyozh4uBZyYzCTl zF|h9gXD{bFrybh_LTj@l1t_u6M&e^Wc)T%)~J$jOTyCB(9 z+={JM<`8!`J=vQI07UuSBbpBe06CdO@G8IovTS+a;$T*!0GzVLt`N%*n)dly3RDF~ zvMjU;3L6*z`((DUso(@?J?d_zzxU`1D;7u&huu7;xqWbmdbDUMs!$ zCkqjA1cDfV4TIh81yEj5o}RWb1yH9SU#FP)91x4IR6zF3L zUIW~Qb2Q6m83-fi_~R{f$R$8S$gBT!r#>SD(1B|J4P@SZ>NvS_IVbm$jkuOOPFAv8 zP9;dR%=wnU?$O$+axNyJ)rghTX;t-}=ZVX#lrq|i&JmBButa!X974v0z^QiaQZ+R|W0z=b!-_8rXBNZ4G@n9ZGnseKFx%Bw3UO~R+TN4qNvQ%xVY zwo}f=H9ZgKxVUz!r;r6G0;oAtVp%sTvjlt6g<*pUdC#q=JpZvRT%m(^vY|_fs2!D2 zS_pTJY8MK<0MsNiLe=N!8tYT1Yg0@y`ZRyr$jpJ!wTiod(jzOXdS&~5RM(%ZJs&=4 zwqCsl4+9wXkDl)~6et0{8}Q>ppy(#kW^R0M+2eiK)$GuRH0bw+ASJ ztNpO9f4qH{AP*wZ@sC{sH4yGQj8kAnY=|Aj z62K51n+*$94KlUGDF)e2$H}rvotb*y_TzBG+3B%S2iH(3ykj{(3X9Z6L@KdUU|S}w zMTG1s>L_MBNX?54gssmP16UT>C}20?+A&aIHxN;=|3ZW$c@ca90l11rA2MQ~Q)j39 z9RSlY9MpEV)EH@xzqfV@V2i;;%fKq@0VNeKNeWo0nqDBC1(EEt#e8?S+{Dn%-s^%W z?)->Blx#a-?5(hyot_-dr$@;Zw=j|QaOr-A*w00)j9xblVl9tDyxc~S-%*5o0CK_F z>wPdgHotvA{@5)zIqdS{c3&;>9|6>CFf&bk2TxW+CqPSvPd5F9^Wf@U;{Z_6mAtzK z*w^!8Q>W!*rp|y^I#SumS+r=iR)gY?qX>Q|REl5G3gi@dE{R!}3Fr~y%`a$O)4Gzt*P~j0+ zUC%Y9;yCmp!Ksr{(r4N+Uk&CnpB<2fSgHH)X_{Fia zw@6gWBS9*t1#%glDH&Q0)BOpXFIPb(P_fD_m3L@amRGUY!kW_pV)%MtlU8iIl!~2a z{?lXs#*t{-LXhJq15>FUUfZhoR+pR7jR%{R!9h5%>Eq%^ys|OcJInB*zg2@p2+^Z5 z?ivEG=tGS&8&BF-^*|f79b?!Ms}_sm1eA7P>zrVZ$DE?&AQFsOCWQCQwuO0}=qK8Q zKT+621Y;K;8%8=-BjfZ#f3%RuhBg}nOV7rL zRBIKgNtn|??b_C>L8=0Cjn`*>qLZ&9v3Iz3cDiw91#ihu_e?CPfW*^F3{;P%1Yg^3 z${b^$X^aGTWUsmq@WJINi-(5K;@AtBp{BKni?VX4#2k<~J@le7wjfWcwZdAMMY+uf z^9+7`y#=>b{!(``$G%5u#_hhh8oBcQ=W)*8=zbZ>H7UKe0|+uIqL=xL&3`YJg2*aH zv!i@4K|G6agvQ_r8yO;9>T0E9mv!IG%8ZUsWxEyz_){xg8MmE9-LrA-Sjs*(hZ`z~ zse!j97@~HLwE#*qCS-mp&2U^AiNb6KKj84gP4uKBjh8A>Xbrq_T_skew@1koOP0-y zTA?cqr(q|xg$f)#kUZ*GNb#zja6%zQmR92WsdIC8Qmvw3N!*qR6o#2C_bP7=#`Hn5 z=<9ofq{B@G$&jXsZL9^bgbHSP#^r2izOIx2R~-UXoXtM&Da4d#ln-o0t*&+*&DjpV-TRxi&G3-X20cL;TkxzFxa7fmh85C zM<=CVXJx3X{HdgF#;w>LB0wBwbtE-Jef0#;Ha{(0shM97kR` z=d@_TiI8LU0zg!C$Ei|s=i*9lgdK4o7P(9PhBGspD3C%tMZt1PyVv|RY0YD%B#n9A zl7+!1URDu@Gk|7aR@(brf~E@K6GFLMwft+PPcxV}+4` zUZX&-F3-hAj6^D2{Dwusd48Bg_&n0{aIYCG%X}2exU5N5f{&>3W><4IjeEFUH#-fy zJAh?TPu!3+$Z#V09f zS&gxL(3z4+;v10DE*zr5DF=LitL~O{M?3fUrZ&|1FFOB~r&sAXOlUdu3%QVYeIk9i zY9gM<&I2vpg(kE%3#igbEa?&iWMcvXB9}?s7T-(Ezq=PndCxA$N(z?bnudEVTr4U7JHQONSMuT7 z#yOX_R4emG%cYO6oR=MbvJaBvik_gO_CK`f65)m~hQ_9L$_0`*wH9n5&jf~rlM-bX zN*_;=4^3i25ifgWu?q=WR~lCNM2~K&XzDINb2)uw76fxkpAm~Hf6&Yyqm7>#n;lhN zDnmrbl_fWYe2io;DCb3iX@#N`M&a7y*ifnP!wwWLx)_AVWAm6SS9;VDxm37H>r>&i zy~k-4r5O9u=$0u0IQ>KM+JIiZoRnUtc{`c=bmUg>&L#Z7;Bg9nl}h zE{;q}a@htLnJd4mn<0bQrG15=voDrZtbdB-TLjM`@gD95pl&>P<2ghPY1{Wi`)Q^O z2Pla6NASK6roG+HyYNtgRKTu<+jVPpD|M}Q=*h#aaP%~qg4w`?`7gM_rDZ~Gi3;M0D zCqfs+caPBEg$Y)ra-cvlaHf+*f70s=Rx%qYP3(&2t!5al^tjd038I#-VuTSwsJ8i{La)-XC0SFYF;7S?RNE>uYo zP}_)X!Y&3UTZcQzGVN_YJUuK16C5#Tr4ZfgIdbY6xMkAWK4TKQx9@G_u3Nx|KXsj1F~d!Vv5d^9VpY&nnEV#xWJY9G>) zALt~<=K4J|iDVVjS8^1#R%H2%qdT?e2)-V>w$7qsNJw%m1+cp^=~XvXk+}v_9s_7$ zt(iSOduox?z|K4~cd~PVPCl?S?WlEu^64>&RmG>(n(fX#CCTum>@t9#lDHCQf}sAU zFe_Igyt1<1K`gQH8M)YW;Wv;bcL}@q2xalW3Bd>n39k61 zzN|651dFO4jU;RN-T)4Ab@xA71Bbdwp|ZXc;53|*ool>(`(!~5E^{{)rAZ#=4^D?g z1%9bBfKvufZdHI1->3#NSMqXB5PkA1Tnejvu2*a$^I#|yz5Y;#Qj;XNYO}IErK+!r zE782G!NK24S~%87mUH1n%d;nZAL`^%Hvr2>OEUjhDtv%RocFIdoB$amQEzktH{LB` zjO)d-K7ej9M(`$%j!$MreW@Os(hLEIccek~%&ViEtZ3@o;$av_i#TygdZRN>CyfGa z4aO^MJ-5z)NexZ%F`iKs!y#2}Z92I8t9NyV(6LArYO6E%-AGcCvqslk3aFDBp0bOP~oPL2DDf;5#2yDwh zPD2?#bPvH7%YGDoLD2KXRqjW$KTMpfHCPwUM*PQ*v&J zJ>WmeuHh#8#$`V*lNPnHztq>X5n~YNx?0q<%Hh_QDu=&8NG1hxK^A_d9QXZoEBgTz zGMt@4FJv2vmLQQFtX5``=0pwxq*LTRssI(jC;ruKmr%Ow**(NWsmBAikO6>UuHm1( zzIoFit=-mmhw3t)n}W59;rA0?$g8%$XlRQH044Dky8+VnRFGCOn;EFDR2k4lj#!j* zv>V218R=-OELW=k(Jt7d0)8b#jr?`L4;qFnU_A}q*qfHiv3CTeWir7`T%zZq?;mRilt_M6b&iyvL!h9GHf#x7kfMcx6zzLMYILEPHNWf>{b;M@x6vT>O^{f{ zgZ;{Q^EhnJ=X!|Dn-&juLNWSbMxiSzfCKgm3FcG-BZ$JRUjitPC&++z$7}&tiBqzk zu6cZfB%YB*bpDO&AC2$G63jnFLOQo@t_E@t&+kWdWP+Q z!iV%}9)s8+ih}Vt?`jePmLs!*xo^8tcf1%84m7Ko`5Y$cE&?J74PgXwzJPK?-Sswr zPuyJ~G6;#!&At~1Rlzzx+yj^fC^0(B#B6nCMj7#q9O(w5k!(?Xe2c`M?nE z=X~Gy-l|h|f86t9S5e#5-D^GT33HA)#+YU=?b=el3QjBmj|vzs<~gKY{eOtwSOg^F z)tdn-XJTOw}h7#t}RlI)5?S9Z%Fa`mZ~&f^pLqykjT?M04cJrD#@G zo%4|F4<*l)WpF0T^U*DQn3C@qT~+?y5rz*7kkfm7<;H>;Wxra(NUi|HXDRBFu~%69 z4onP_ljxIjIyE2I|4gm8PWTZC^i^tj#xrnzbMRz2JWQ4>FgR8ZnJ$!{_%}75x2!Z= zy=0$+oL#;!PN(7=uTveIdLyd@V+L1$|Ie;=-h-q3;>%J5AB=s3JnX0xyS9Ht@mdPIRHaB=E*|GX|* zZ2)6+hWsmFG!Tm(>izq}fIi6pbD~H67XKgmB`wS-g(FUp0kXU>H1f|+%)?8~fIo9z zX#C51>ODeGE z_2B!>nG^rzv88SV&+W{)3gD;@GMO)2Fl2-IfS|6IAEy5Gdw94JiGoQRMwl`lkY4(e z!TCVT&PBEXe#M9XdiDQ%a2Ltqo-S!#{5=h?ApGT;xK1-Fd4z0v{RCzYC9N5Z!rL1G zOaCen{qLb&WGOR@Cp-N9#b?N__e3t`3+)pqNB@RU>k%SZ`jPQJ+tvRH?2dN89ug1Z z#@3)b{`(gX#=~9GK2{B3q*Z_~<$ayRLCY0%1OC)G-|(;UWH*e^4U!40A11^L$>(ux z@^Nq3%kY0i--I>%{zWc%xJyQw=u;Bjl&JXPgXHuy7^i%%h zUk90)57W^!J-ZnD;>AUQiTddXM~OOb3Z|8Fv#GS<%pU_T?ot2nuI`B*qr!(?;|iit zL4`;~S!YP;-HDObycQc*XJ=y8(~2lkYMR~Q-sWti1|!OQ&USFp+?J>^BpF&Dy>saB zpS)lDvMMZdhEX@#T+z@&r*B5NjC(7~wqL56`1cZfkMM)q$Vj8_36uE%ffPY3u2BXwzmY-ri?SIpiH-zp$c&i@N^5zHrUU2OKqqfoyO9DM5?P z_jhmY=9?n)GyVecd{b7{mX!t6Li;f&L4`OUco4SY$WL=H}wncu>@^4D5Pe83mZ1SCB;D9rr6OjyJymcGZI1bjM72+MQRs=S`{=+v z$p}xX5N8j}yDmTYznNPWA1`4nPAHs@!SaQz76QU z=;f#T&ch3=#Eeu{^g$3Yq&BD^Hee$n0;+dcOQ>~-*ZLoLCGhsa2x3Bn7|i4W$P@*c zRZ-tEb3IZWJbOa>X39Jy+2ROm^WnyP5!xSKs#F+g#DXf}`PV&U>0u&Yg>}F^dI91E zt6MO=tLWK}a}9u3P!p6@#w*|7gYw!M?a77bG-ilt6K0^$HcyoQen)*crV|%|@Ispa z@cmW7XU_5=?v16gmhrWU40C)Pq@KEJbRpGeceTU$_da-M&8&cwG5?$YSTp=00C zyg^+?CW(q_}vYf;&`9R97W*R&f?6MBzyM4|K9!p$0Y){TYg> z^?~rzF6+K;VJ^tSQ$ImOEi_`fLVonCMRD}=Lw#|cg zDh<1j-TF5G>?Rhjeem6<6!_QCLS0hk>$_|bBEbF&hkUbVC`_DxC1!}4XZvU3gZ#IV zc-A7mO#^UX8)8z)6iVhxXcv4w21{z9=MHQmlTBdPVuh+x)1q*$f|6fapH+Hh6RgO^ zG(y)U4>5!XEl$*u8$v45rB2h|D9fO1*%_);X8sF+;L_AMw$LQ!*5-L=)}XKqPj%V`~U$Ry+TB?cZ;vp zJ`0s7hW_mV;%ATh>0Uem+{^ss+(B{_l6ztipzi%OpHv~WeGO|`A+#tsv)D9m|LPs_ zb=d0Koj|VY^YPU0pX2_94={S1k$eXo*r&ni6RkScl6PGF_HI7XuqVa-z>5>%r_70s zz@Mj-Txtl-;ZdPvAM_iaXR|~%40tzkXqC-l(e82u`Xu3_%w`W#y1qgXsLl5%U_ zlG$}o`lL`ch2hOlMY`rX0IPn5UF)91(1hn2@Rw>OHVdKzW=UQkl34g_@vYxm-8A56F9|aORNXgPeEbVm}<52MW@Bi-z3< zCM_GQPC4gO=w$a>rTdY-Z79YHv2PWV!I8)8fyT!DT}7>Dxj7NjzK4_3c@|2?#^;MD z41Nr|FZvOu(7}c4()v)`-CcG}=U2qaeD~cIIF4M1tg{P?&3HMM!p_A!ANAXufkCdu z4(l*|YnGZTe!g{iVeY6vDe2tPJpxJ&-A_*|7Nm)bD|xNb)~n@7du)C&zgHy>6vh`y z4$)v2M#{H_1`byA@!Re?Hsy$UTxijF9|)lx%$9503P}TEcqDzw=T&dHui~781StjF zJAZfAPE^emISF0cEnJ2V-ss2_DsZ`?c3Y{kSg453WKN5H=zJONg9q0*&+<;?CVEp! zmhxv?8&M-%5dmwZb}EQ}1)s*y9P`~rMt)yJCkvFSd@>M=w4ON}X6G=&J=8}2JRV{NUPk?EeGh(1! zXQ8tDfJ?s4ZKX8oB}-f78B3)XC9}PIWT@6vdJAxrl2-gT!JBW;lW1Zni}N-IGpv1c zQzf^7dC~IO%ahrT|3QJ)FG@*D!E9@{l2Um7D3r`K0}syWQWvSh>cA;Sa$5Bg@EnTL zjaOZgBIwNx)-yygSH81yKCPoRE5&o1XDNp*a6#m@l1`~-kCZ^RwXMJjt~|9sCsMq3 z|2;&-F#r|zT~+C4vRq1&(g}h4orr7FxmuD28D3@cg^rVb?ID71$GAW+dK$vG7%5%F zT|S2(=($ey-3b<*9&Dx3>Bo<||@LN0Hz9 z>XW_XHi^@i0QKmpRl`R=>+o}6zIae@VmS7G&EAD@4}J3xQ??tp^?9el`&GW2SPPA> zC`?Dv{VFe0)@9d8-eQdHOB%Cbk%ewm3WoHhQDs9}iQ60{w z-#ORZ)q=g}AbxQ%wqQQ?5&gVF8e7`#%v}F#X?;lT`mIiRE<+tM{D_;SlR7jNxaZfrfs`-6P|~e_vW*W95t_fe-CKkms3$s z#y8oqCqiB;GJoL_zwmQ>(|S;9!!)x3Hgd`ofbIp}otl+7&6c_fUYbvDoN!+8Q8%nk zz0S-%qi;{4XGe=$^6ZLI6jT&Pk7-M3j3n6-J(Ain_Oa$=cHw1Djfw=yoitq+8Ciqs zsO4Rn#az+UpEGSUsm&1I!Sr<9tXt?b&6~J36;BtB{8OK=lgA+-b|S8~c22b?8f0o} zS>D+4-o4+l;0!fn{2UZ4(?wp%y%2MWQQu6e65GJ)K>R&Td8Q-w-pvJt^@6EV1|1dU zmyc$P8!&SB1?BMyJ@YFPMa6EO^kbx6KL1bnZu2>(%ce>+C7gn3_+}dVPpUrjY60%K zaWR3ez?&&e5&IBh>s9`*_}k zEj|K_`vxDg+T5Io5f1;@{`5S4q@b;m{iGq8w(8gvf2OG~ z&)jZa?yAPenwt%zYMmCb|8_-pkYK9xrS9x-?7(E@t2tVU#C3D6Y%=`XZM#jd%VQ0BT= zB~brxQfV(H^PDkKS*Yh@-kya-;Baq>%F{%37Cp1qw}CH5hid)-l`w}5N2TG9*qI!; zrvmA93!=^GC9%=2!xt-$zpiL!G#kH}wEaQ=|Ez0^WPrUB9Km2gx$*sTn{4MHgi3ZQ z@!hwDwZ+J#5zL8L#r<{ImgQv8npT-hvYkKjPdXcdcF(Y^K=tO}1mWWgb~%gK{0P?< z9&Rb=i%^rbacFwQl}DjhEQ2v1(+Z>K0|xSsMQ)KTB~nI$ON&sxXBIpQSfcT%{#Wx0 z3!uFH@~1T_VVg8En7*mMu9DyLw0I2kKiOq=gS8Uh8U_w_gtl@_v*^C{fB-dcm)4n1 z`^hxe0%%}c73R{Uh5$8bCqP{j`n0CBH5_t+Ygw(XlXB2LswZekE&36pAiD+2GsUK^ zIeY%xoO%f5B1igJYsQDR`s&8x#BmvMruKwy5xy8Jfb0}@2CttnlB!ktXHYNt#Bcpa*BQ=6%eY{}m7!;qTTY$IlIkEfcBDqsA zasJ0~O~c#Y2ckFhp3&C_7Z$|q7ACW6$fU`jn$IsN9q3a(TD4`|d+lW9HMSt6xLII| zgzY~Y-odFKdWefF{mmI7K%sSTitwv8A~|7epCLtzQ`Fnt=rkax!R*I zl3I|-MiCJs{C>4}+V^2|`}rXfMA9tBl-YtX+L1Gsg6Fia#yjlZv)_quea12+uylE> zhpTf^;ee_-Ig}P2soU%AtUXl(=iVbmGJ=~wHW#s+KDhSh@e*$dXINNs{(usnq*R-{CcsrqtDnCaZr=h+@Hzcv2l83hH@XrX=XJC|*_> z?Cu-8Kn8{DG8{`Wg(-Yt5;%o+gMl>0plK6It5$^VMtFxLj~tR)zs71!?MZ(f3wSuT z(Tm?ZiL?L2WFH?IO{ps~ z)?3HMZjq;oixt+#6q`a#UnFm3*56dSk9Gg$eQy#HgvAEFrd1OX2>cEXLJ@L7$;bJ; zL5Fvfj0U(7X9-R?yaVg|8wZhaPdlE%kP_l^ULC+kZeDBw1XiCFIo9MnGNWJvqo$4z=x!-<;UveybReR z(VCTK&Wa}~?9RNLw2Ft6?E@r5EW9_{I*${RK7h1gh@+ErW-5dQ@(r6JW3{*WFG+i3 z$SWb~_do$`=LMO&FYt3uAGzp|L>~wKujh-5O>P}G5Tz83XgOEAZ$U}TQTsuBK^D*j zF0k&i>Q8I!9c!Nt*7=th;5;o08I&heBVAB=}q@PZ)(hARTB${#t zws%Z7BNMgHU#@tz4jW^$%KaB?q-k{do7>!J{Y>EbM@f6)Fc+8dgaQTvFis=@t6VX5 zS%xk3^B%KG8%M-p?|>ScxSiqG+Ca;Pf^FL%O!bFjU9d81mw>)bVnho>L8q)k`cn2j zl?xY2CpMx=uOv1h$(gP{eoQ>m8CUnq<01CM(g4U5Ywx#kE`oykcI-D+%ZhyYLMtjW zEB|Ab=SCha`;kTm(yN6_)yo?SomwT16=6`XJ-Cw%4zv(BJmz~c4eo>N;&U|H3Hz>u z#iatZ2q#j)*Bi0XO2;0a_|a)5%1pQ?edPtaMyMHsu(5Hxt>7sx<(Y-WOnshu)J)^6 z{73L{scU&zt9+68SBQ*R7uQb_p@`?p?ylv4Q=hAh(zXk=A#(&0d~ez2y(;Xahk65- zv`DJvnXek4>g02M@8FMbi{_bbq#OOA1(hiNyhPtYbAg{;wzgmI*xbH7*MH{|{+fl3 zp}j&LfN6yfqFn5qO;cVUWv4pIo_$pE#hI+~=)k)+-gj)yjn8P8Oj*=kG`);JMaVS? z%i+0Y=uu*uttH+Jyg4k#7Ymypi#aXfW~d0WWD1Kc&QFOaZbSh-W33f-)+oRuAwI|2 zb-W-a*}X;JufF52Af0P`*em+&U1GGuOa@Qg(lE=IWo*Fo`p3XsPDCPD99{w-0G~Bi zA75_$##-sTt>8+3T`8-+SN!-KuZjpzFcGnGB=|`(x#_2mJ9&w{5t}K4biJ~BDVZtz zLs1UTaKJ5Rng0O)Xslv$XEUnC%URfgGo@*)qLmwIVd%-__|Rw<%$&BSsnXifvSGuJ zvCUwWI8BNO58fvwK?=)-ov4L)HZ#C$^0fs+}c{cS%|e^VV;YJ@nJqBPRBg}qWhkX|}HP?WV@@Sc#WmBLnLVR%E)F|%rG{S!XS zVs70{p&&E_qEI*ZrS80m>u`o!AZPevO%C|c0OUt?|NN-_;sPkS&jHX{@est~*dESK zZHWygO;gtC(|-vZuIf3qQI9X16yK2Dofxa%>j@q6SQ8dt=f?$Gw|Dze+g^!d0RYP? zKFpGblJ+Cr!JxX_Y%mIAlSQz`kiGRw=-!>qBQW-U^aq2OckG(#!{H`ANj#Z=h0+Zl zUrPNyd!dymmph2Fnv<@D46dt;f8zdq5W;rF)piqx1=~F`C`jpi5+;N0(3fVo4N4z< z`g*1S1~ybZVxW%zeip#ULEkU(V?N0x(DYpMPvuZZR+eS41`2uaDPUCDrcOix!-1Lc zeW>>twt$DUl)^$E{*VF8uE8iMSI8N_?(iBOnfojq%m9wPW>Q=)=|5ePp@<0kS4BGa z1Y}*w!*PNTeO-l*F|zu>91Wz(WeX}`3V7)GAAPSNm%xc5%bQeYw|xKb;p>Cphu^wA z2Fp!ZZn)tqhD?w)OB|}dF8lUBz4odkm%yq2@O!Tca0#3|lFs6FkgB_siSVb&B7p>3 zi7nrC4qBq%E)-|&7eIPbz6*w+-s>b+qqxa|-v#6Rv$C>1r3mq3Cl)(8eV{fz@NRWf#sn@skgWO5+)eAHvNCTmcIUy?QyxI6cL1Pf9~SHd3z7P zz4aBQ6wdniAIys7F8iJD9sQra*XktK^7r?$VZ;i0r0&m+ za0Ts!SE1sx8w-ci9ax7iG{*jddA-L+pP_!{LyY?$ebzeU{^_lr2j{3j35^=$DqCA!Y@pLJyvFUY3Q081!gSjl_jt@ZA9&7?c02g#($qh!w+YZ7- z4DHQ>Kmz&*FRm^w8!_eE`AuFqc&AjCRDtW_1fh8cadHNhMGVYjw^hSGGUypCxJY^< zv`jwkINSX0% z0vCC#3b^{oUWQu{y&~=y0xnHFu+$&d{3)5r#!kyB9un@~b^R%v3m%-r{9+ zp;oa1kI|7g+3@=+^;o!w*q1fPOv!K@7RAA#CNvB#vV-rVEBkw-;LRO^PhGDb1DAoT zy6`JTUI?T)y%(PKZfodol0uv`4C73maftvGn>-BL@zRJW&?Fy$dp*MRj~~=Yc=%B7 zrjpqke!-7@6(_rq)F9m;yZFm~Uqa?qtkJ%NiZJBf$`b>=+W>v`Te`9EU&>+tXb z6tA@4*^S^=oX90`g6N~q*tk}1-yryUAt=AVV8Xk>v(KZf1+wvWP8dPlzuOQxQ8&sCTuT2sCN`e z=pO88zXTguxa`H`p62M~N+HLQr&QQsVC=))fX{>p?v2`dIcdEErL3 zSI0b9_sAAnzV~qd@tSWS z6Lv$MYO!6OGM8}6y%`j0r?<_@QlMp#1%$g|Nc$#4jTrH8BEq!sWZy%`hN9vx`$r-! zMm7M?UsDwj5#`G4R9&5mc7KUBw4x&&C+ z?fu~Ckq16rejl`R^$pHSLDG}ASh-OuFC1~sME)hI9T8E3oqzEXyG^zaLJiihl*Jw! z0a472xd*F1z zq@uz2oCCE7j@Sp*QAG5{>imv-m?yWwph3YsP;>5c71(D4%F&t;<%e1gBd<6;e|>OT z80t*&)5nMoHQs>PMhAg6y zyoFexAw|_TL}_nt^4d4!PEoP94e-0|lKM+$@zybb)!j+ATfM{?ga#bL?oAhX=`?`{ z2de=T?yJP1-dMHQw>mVF;H~n30Hr%S1k`W@QEAEjj>jrdV$JU!e0x3tBl{`>gNq1Q zzYFm8s{cbQE##|WEsLO^Jr5EAlkZo@N7i>315SyTqRPI>?cp%PmoHrxAs9MFfGI%) zBHrcg75F@{JNM4E=L(c*r$)xu@0FXdKndOf^;`lHp(Z$*%xRJ(U=)xLf|yZq2q=ik za8_2y1XG#ORUr+MHm}dc0JWjr2IkxZG%aw}T7*J%Os|9AoQeq7z=JU?)^{H|O8fo6 zA|u4K3xNM~{2KtXN4YW7cf~@Jt(0~r&5Bn8-;?goK_cwJv-zz7wO*OA$4o0lRh0fr zwztLs zA9QYX*qw_2acGS)xM+}d++(%xa|YO)S%Bh%Zq^H6z`@dM>zY|Y-w^>h`+em00H7~+UyH~?1~CTcv4 z#7hj=GCuIn8u^xj0dn*ETyDGwEdvHdfJoJziFJR&o2z zLi40NP_BlHv|j3rWve`esYU1lIL8Lh9=*HHr1z;lr6fdKK`c^Sz;s61uLn$*a!9&3 z@#G`@MhM1qNt;e`M;l1ccELG5Me$=DnxpGIUuGE*B>PccLQa?Q)#+=sdc}|#$nbyc z@{@36jMLi%HA44YFOXJ0ZWQ*vDHMS8L(T-dQn`o!Uf~asiDLt!5Gs`akco|8p zF_gSmws-d`*9LiWkxV=;wL7z*BFJ0CIe&$8nIco(mAs*I^AkX0M-OLTPV0l?!zYp{ ze0}L-+&dZ3=dy{A)g7mP`id;bJERg3*El(159E5qLd&Sq$%gC|{m?e9RYA zO5FFV7iPm;_Gt0Y}x z^qtzg`&BY{T}f_!eRG8ocS%XM+`ie~YC=ILO*S5(pDlX6+G4j}{Z6_6%{u9FSTV@S z45Q8h>3|Y{l@S(wp={%}P(pv^s2dVgQ2IDLHBEcD!_#2i`8)lX*mhk-vir3qY7#VQ zb22sqFljOSx+Hhgd|sj(rKH=LZ0j&;7bkR_(AUyR535_ZV*`)8ZG{r^vzThtAt!Uf zO;hYwaJ!IQMJZ@-a#pseo^OQDH3Ua5`ISh;qZJsT#k=JSF|8{!T30(f3%o z!}IlpUyZld!^#RbVR*Y2M zm)_&qJYke&7u;_vJ*Qrc-(|x7jZ&Bx!m;*UWhEjgdRs-{nG`gS^95#?NH$NKFXDm) zweT0Fy)D2q?DtUB*7-06EWPgwi(oglkEt;2EOnUYGDQU5IyUXdlw(|LN{vd{vvL8M z#xr?T2nE1`s8^6k9`uxqZ1!*=u4VP~=*!}6YmyRv7{rR(O8+XDeN9a#p*(b*yb(V; zpz)+78>*AZV*bmpL?~gKW0p+@b-;Us*5&-J1bBUBNHSTa_9#5a>-(bAwaTwnPyA=P zXxVM)YUa2Xd=wXYF*aVh>s79vA0uiGR*ln`_1{aW6ip?Bbx-jd3E7!_*?aok*TBBh zchhdXfwhZPaws->on>f# z_#VVL3*o+;Rm_eaUUq}&itVY}o0C_v=$ z^pw#&bIEdq_iNXANwUabrba;jT|yx}Nt84J{Sfy-mz+LdF9(^}5@I1sP^wqA*v6CI zAKHH1(tnkMQC|(Yu99zj(Z32OuRu(fekonQ{P_W7(kn2^6470-iVi34E-HsYGvYm~ws_B!u_$^ceB5Clf4 z8H2SCt0rm85c;8Y7_NP79&AwQnC|f9cK{zD)S)VJhGpo{5mQ3?i-r9o0|fb5 z=LrV{Y$mY%_Tbw?1C2iSBQ0}W(c2~Ms4Hc51|gUXn69<>;vg+_?(CNm>rd2XCC9@X z21Gv@J}kpWc`}G&-)fdQAMott<(psb)FXRV%s86ImQMvD3hsrV#lzSOF9Y3HCi{q= z4Ig69u0SbBg~BXe#>L};S41xXSP}P<8N=F}Nzb50yhu zw2wCXY883*BW&RXeAV3Kg1&nQ@%TFH?bln6Y8}pYWcAw~F`v`EWLnEq^tFg1;tt{| zQdI9g_-)GNMO2Q~RL+l2Wa`j8cm9)dQGtyV#gF?@yoVe~rPH4;)=<1@?qp zep@AnhUpybDN-VJcR?!JW!)gjY0Drwow|9RRv78-S)AfL^TkvDfzeBU>%3VFz@XcE zF^?t?%TzI{^A&DHVHc_hgA60{>m|nQXCj~~Y}j4g{FLQWV3z}l4B__L(C94Eb1eU6 zIYa$m=Vj-(@L7LmmS2kW!vlQJ7M}aMeM>0>j$r>j=vY>v)=zV^p+i-`cIhj%ZUp(2 zvE?w0*}{kY{fk*9yY)I#)Hsd1D*733eo{?$qAx{!pGMEKXB=SLoHUof-0halr@#Gexn~1s!3>L zlD7pJcY||P+BU)jAq)VYfoMI9E9mHonWD|TgOAnC8Ttva?;eQm&3^#R<`c($(i$JW zeb0pxwvUfJh}-zlVluw%?g(8POUK~5zTsl5Dp1{|QGZUc)CQ>MtC5{HL_M^|QFQp^ zR$L&)2xaR@6@hd4WUGGw?QDhA_5-+djCL&+QMVH;KIx|Rh;eC zeWzcPx=aMRynH(f2K-4pZA%wcoccfRa3Fjkj-HOy>G zPWhFpDw)uD<7@1#)atZ0!E3TOzinqJYKjt{dWi1_b3DoG1#Z)Wd~YR2tG3BmlEyEy ztnD8OTHp(cHjza?>*>rO^-!I}Vl>udY>?&l%R5jB0&_fPW@l9u$k3N(RzD|X>5Hgz zs&?;3%XPoaX8_H7vZV2WUC#zKI|Um#f-n>w1qRjzn0WF|b@auYx@z!X_C~=X&+S?4lLmH>%D(O5QD;>3*tJ-fQK@i?@Xnl)0m|{$GF%XI~)*_B1_Gr0uxJDTn~7k&yLTzkLeSaP>)Kzvq~%a+*MRT-L_-i3SZ^E@q7d_IsB3dBFV}ZQd3b22{zccc0Vx^yBGqqa@R?OsUdv2#6 zhwPiKlT7SEsg;NVwA+O6&LMFOg>JnE6s(d3g7IZ~QsZgTF{HhnEPdVVO*mU8*0=QA zbp>Vb5gs}atzS$+RKX-QH!SeJ7p(n^BBplJ1Q*J6vUdB^3+AiGIg)YJwg>@`I4do{ z7{mbt5WFWE+cY%S|8SufsO&7q_)l>ylWr4k($8WRunT$zGI&=$<1Z5oZ`gS`Qbgd6 zH)@l5I|{*1yUd~8|Hjm!k76{lM0XCXmr7fgXnDeZbi4&XAaj5LnP;h4GJ@a>lp;D@ zOm&bxF2E#OD$_O&CU$mARDI>&?5OA6Hf6nhK$WRHS)1#9@YtdP zLn|bmskseXoa4qk_bmD>Cuuras(WmS*&}7sy#QOWzA@Mp*b9~j7n{L&@JvcJ$9Q1*nNjY9@>=h4|*pffA z8d1y|a(+F(Izj{iE>Iv8TFUA)5y=G(#`E$dAEW9aqeSJfvv%kr5Nv!Lie10muE6U2 zIUg5>BPbRn(a=3H%P0+$nBgTQwuZ^eZp2aTFa&&M{LYpZ_R!@Wu%<2x$)?n0p6%@b7=#T< zcN^8~%#0i`s@>jbA{%Jb6ZxPf7|Oas$=F&q+}XYL0iv;#Pz6w$ZHJN}>#^1il#FfP z*+xZUW_@R9z>q0PM6@FY%?K0Vwb|At=rCwjzhFmKM7ho zJgSHcegI=u>YR+w30%F91Xo~W&})8Y|<$w@D0< z`tMtPJsOhA-qx^VZ|A9f*|(|P2OW~1h9M#u!xBxFX;SF436-=Zk%ZG7a$R`;`3|M9 z-#fRfep!+d*ISC>HM8u2PQa5UNJnKbfV!h;64@}2qEY&-j>c?jD>2J&i`>TcR6^|l z{`SsH&DRK-lhG7|szcla6%M;mo?#o4>YkF@qhyX`ImuLZ4}Owzbt*Ra=Nt5m*+Tn- zsups)e^nK_Tu;i056tm-k>lf&qZW&Dd7_#yum!UdYZTSk+phd6(9U79AA|Yy^z^5| zfb%ZeKt2zQ?83zASlS%9JBRvmYD0jmd~rd%Ur};*u2ZSd@ql@K{gENu*rHvkz&guy<97T{ti1?Reu4r{I&cR2R`bf6Jk= zNXrjv)Y(|y_q6g;;Po>GL>C4Tg6EK~sEj}K;LgfP{1y~b=c|nJ~-+@Xn^RUoGx2ID&D_CV=c0xFY@NorfE1H2jL1*4)^p2C@#tj(+X-VYs6cAKUj#F5;z@sM!Tdp zR@6)h+>Sh%h<|(@@V?Db`=0kFX!yBEB(r02?nZD326#xM<{Y3nR|Y<*8nOwmufycD zOK{_}2{=%XKxT-`N+c+#%3?Kbm^AxDrwwB*wRMxlvBCLjsiNRPYajday74P$`p8uR z_v0Ya0>c#m2)ua`k0lL&s4$3UV&8e689AZsK38gI$Chlg(crf`Qhpu5oguSrX#$mo z-Auek?%lVMhS~;!mzJ)Y61W^Ett93EQn@>KnKHmAstJ9(+w!EJh0PbLN3s*Gn)NX3 z#@p5!S#0f=u?G#hO0KE70dU%cD4*uE?3w{yc)Q{RqFmWf1O=mNHis#{3Ga4XJXsOa zLk1YfO;2tpA%E#VhQ=T|8!V1{28n*>YbeX=-G<$)^{n0F=iAvHAkvN;H3m^Q-YX5f znxt=k=-qSfNZD1H(VOu>7svBg!+Y$O2M}MH6)k}QXlp#*Gc*9lp;hvi{KGy^VeVMh z+fCva-yeVF_0b_f;F@?xi2KnFXRz#!7nCFnd{7~o9X4Atp0I9

  • <+S;?+~FGi{JgM1dP5^>60hnJe1e-|~eC!u6`liWPSt*#5B<=h=TxDV)B z2j3R}B-@KLW&jp*GxS!#^netM%GKbe1{65n{ce~Ft(xFNKhGl8oorCbOZQPtcn-6c z(MNx8eZq%ue=e@&Lx21iWZ%M@$?alCn$R7Br#y-D6sPTaE!(p@xA=C`^5LL|dvE=b zODn_o>di3pL#^gHeC&|D#u4x%y~>qNp{M*IeDd^v`@LybM-ear^iG#7K!uoom8=cH z&QD4`fFt*Q+KfK<-~Li)<`2a24>7-42Bk@9N_#WBYYPBi&LPH$7pLMLHvwDsW}&sH zn}X@*Kv6JzQ_I^o-(MhNRO)nosMJ{mP<4OA9X3iZYJ9hTxkt^Dy*R*k5nILv2Q87# z!VD_2Yrf}SDalTpgrxiLGUpj2u1FQPq=)f81cl>|Hyli`ERZdcn*d`SurY$>)#*ni zoZcg?6`}-deQ=bBAtYkzQ&9>$Y9T$ov$;j#4oW>`K$pX z*9?jl=7lJL_M85I@%N42j$8t#a7Btyfvoaxk=JB(9^6Z24hlUK!0hRo9NJaIfShJK zL&k-a`!o?HzN`hZ=n2daRM`mgt8{Mdp#Qb|M2Z!WRty$EdlZI5G-1O7LN62)KjNn$ zIkR%@cx{s(q^Xj=@hNH!Yx)8rM#o7Cab&;J@RFA?e zRM-%`ETZ0>ZE5!cm8Qrtq8K9MXWzl&PQd<&;!zB^Z=IUT5GkD&B;=|^+nr@Bvkb+y zNGMl)HbTlrsP#VBnuk`Z2OUvHKLD_`SB(_DFcu&ZiS(qItK3+yKm@C>-tr!TUd*VB z(|zvzGQ8n3egMs+^13vY5!hi6EgldEq^9~!XK|oLE`uRViLt1{GBCD(`Ul|7@uNE3 zEJ_1$BKXpov=-BXZXNo@HE}k^2ar8P!HLzFNsr4m`9RI;L`rj+epCfqZJEt8^-~~P zzK=&3d7$np({rCMFdCVW+ux2p=K0aYNrpmCJ8_(Zp*E|$ksNR$y^BeCMC{K#^Y4t7 zdW5IoTsjKc;rlOBHqtRDJ=CM64PqMg=4o`~cNe}VaA2ZG`@!Ap?k&jNN$v|?`7RMX ztus)RdvH}QSYFIle{(z18vlh|&bemq>EhRxboCD6-LBBerFaO|vCENajQ$j)rzT7) z<96HYgsP_Guv=^kTTp8lV!KWE=^J$5h-$tBJ%2_$(gGA)Y(a8)mMTERt)kIWrifU-9RH?pidGnVwv)C1su+{dA7lvfthxS~18Ixj6RDTS z3L$xDyaXN4E&9yEoc$pJA)E`4T&m}3MtVrd=T4O%wWkuGqeopTmm@7ItzCkGDZW|4 z-}?+dVQC|Ox9~{mogIpjSkUIgu3h{ah;8BLKP;jDg~WzWpciQR1-8SCF=@~arjw07Tib&bS%6s3@zR`%p>UCP`G@xMHPLiT7jn93Z;ci8`$ ze5b&wb35Vdm8hCbJ}umdGy%3HONsCFLgfc(gd;0^^X|5uyM(kk8stS8aj^wu=6762 z`Y$>9gy8mBF}k5AJ8{d*R^5>eMrk1MKz3e56BBAE|u5$%3r zinf&&*SJzAb16+k%Q8>V$}OI3v(d-rhY&ex6J&I_+6m}6F9rc7-aSiTD1|uXR0Sw;sQ@c^(9)lh>D~n0 z=AsOhnFh9IC#Xt`SF<}d%NB8e3cymS~i&Pzw|@a_3F!B>#nz6 zxrse%9g(IbkbB)wAk>vYHFl`fIPHq{T@1%rTeei3?6sF&>E){eS3B8^Jg*q$m-ldJ z9mJs%+n5H|(1daz2n=S?g)!Yu9X>%3ktMJ|1fOnut-r|Uv5L!`SF<425NB)R)T&N- zeY@YuSXkEpWrbEdLEKhIgc^FGIWK3Q z7crhk8S0s?@~JOH_5AF(in7&L@q~KtlK&ttpG>Jr=V?(}A-14qXbht`o2F1CKK!6I zba{j62;X2kZkO6@1oo)Nsl`hiuV~Dm*k4rwLW_cd9q-N$0NnwbV77D(Z4x(^(1|AA zr=*VTN7ZTbG|}~WZmr!Ff$C5(mzIykU||D)dhzQgZs9)Cd-@Y7B5Gu|xv0D;HHVV&r775%WQ zOUV{`etFfr*57)BJ@+DTVjYCgM#Wk%KfrClpQlYb1;@DVt&&)Q5gU6%RJ+WaWWJZun=fe0=w>$E%-ZXg$ZPglel%wGFITLS-pQXw zXLjLUV@g>Ij1F-B=oIpZV|MkJ*W!9F-S$2UT)?Xo)lych1IJ?$SNF4vHO&Xon3c#S z)A63`q?NHnU<-ityYaWKqsvrJ^yBzU{SiVi8))85taV^uqFrv+x*=$slDgVxT$ zi=L*~?basiR0cAa8op<2*OJ97U%O+?mPv_iSFLmS9(1gzqL;f^80r&9Yz)ELsyvG* zI&_q8hI5bJ=L(^dTw2r)gnkpx&O5*KnT+MW7Tb>v+GV;$@Wuih;OFCkl$| z8p{y$`lVPJi4pZsT%93TUS0V1EIVNM3IP=|0{#jk--|tub2aktSt@}Ha|_4xhVRM< zNNG0D{&TGV8_Aam2%0(p`u3B&y%(S`KJ+E>7UE1>>No6R3qiTF@dMFFF+W*DYcUEz)rMrfg7FWft9Lq%j_T?>MJ@z^&XWUs+Gx*XBRjc? zQMQE<0tsJX#;?7;3BeTYq6}LQ-w93H_n8Q&Rw11E9EVo(9fU-G$X$wAJ3a*9H(csF z1ehPl+P=bh{~-8kn&+R5jLDuBuPXKg1YzhgZN7}_i!@4UnGUp1EDmiof1c5Yl)Lgy!?NG$T)I!9Xut=6QCPXrhJdkc_s;j9UGbWd3a_adDFitP%0kM*0kBZ!40 zbe?e~K{;w`M)57*xpv$^75YBw6Jhd)(Fax%^R?BGXr#mef?vUoEJl0%!^&!@UB1M~ zb$FhsD#|n4-xd7~>4NgrD-=IrPDJ=h+qR!}w@OQh$xdao>@OVgbaw?kkh_E8(lT;82aM%~X**n7BJE9F?DsG zhJ8D2&lBaIzQX;eB%}Floi7%PAklMD$YVtT&5-yC?k`p zMnDI9nJ%xE(e>9yQ-76D0c~**9|vx3Q{5-R+4j*P;cAQMo`TCCqOEFB5>sQpcyQgG8EUtnB)As;(t8s)nXOHdboXUsf zwS#<�jBTXb*`I`k^3>;=hy0=dcV82{r(A8V7^YF}t5yMt_$vMF@`41yE%)z4o(? zek6e|T}Ihzsnu%7jVT!yxY*@m&I zD4E2}B>M7`&tz9q+O=<59tkzL6(3w9jJ-bFl^XKXI&A!{z=SK>@rERmYMLGPjdiAA zq;Kl!7c+xkVgHes#fh01I0z^46HF;iaqo;Ag}%S4$1LIe^5hTwW3Y~wE@%Q~@b^3Y z3eaIW6?jLgJHn(DC77p*7MFORDWK{8;x|T_D^XI=G(tn6H&OPC=FdTa1>B+grhhx| z|9%;Gd(YjYA1*ipb3a!tM`>^Ui%|RjuPZSbRcl$pPt9Eqi`R7JMGSePXVJeXaAS`j+L8B%8oOsDWxnekTXtz$hrBx zwRH-{ZC5qWVvHV>2JXq^O+;GvsE%2eC&P0L`=B^~!)E_AFEzdB*tqAP*J%p_lke_5 z3*bOI+<`);JvV?&#qg4`u3XK3j1`*@%muG(3AIipet4GmJC?x1Fac&Vl}b2&PmZfF zENAAAeO-rP&4+JjcX|hE9#N2>=!|h(1tQAda0D{q5wnNbpqIKILY>%g&V4TqH{v#{ zvG51Y>j1)@a@n3@Y4}M&_y!cmX&sHP0NscnmR1N)0&y*VS-AwJKqk=9CBzlsm^5oD z3nC^h%z4m7Fai!cv8;WPGr)Ao*jj%h`I}GwZ!!q=-+~5xQ6Hp3ry&3W6R`1}NBn!* z2}4M?kjn?8y1uQ^HMYoXKT5RD?Jj~R^B@ViBJFyS$t_uO+Yq#pz#N83mL^MwGy5`qxg^R z;E#gf#$jGpSr^#i!!Pf@c%qPuBW?)pdR!K(qU=Snq=gi(AqVy7*=^~)4tS(n7hZ2hWL0Y>Zu zLIKq}B=s4pbk-BmuD9~QU^)^j)?fb)xO{f~2{OYtG`l18eu>z<|3dZR4j&kSVwxd( zs_}5^%rp>Zt>(+-C$O18D1^`~4Kk|07R_R&D^+neIHx-nhKEe#bA>s%^HTp&x5WLe zQ(>Fx0-EBdGdUj*Spb5!(-*mlyz2NN+&4*TwYT;WnMkmx4?1pvR37=IxwL|CyU-MU z(2PPz+}6G=P7Y^WH-X*GP0G^1#1U@Wi|{pG0Nq@mC9}On;@8*5&4>v< zP%17t0=R-IAtU`aGT=2?AiZ&r(!}RSvN&UW<@(Gip1S-j1aL*LoN?!2#0UX#V1-u8 zm}N-$#E<3>;fr=msr;;gOSRGnMOL{R%=trrt>W#rnHqZ(gHk)7hB;)}FbRE`?}!cF zul2=Wj_(I(Ha&AIq2Tw18-zqsb0kJVSSAPx8R=u)bLIAyJHkO0y@<LGYi5sV-x*3SiV>$%%6KU@S^d;bY8t?VRkKG#NudZ-A` z^c;1EPAO<;PzVhWv{WzF0pxhmB<`(lTr~$|QEZ|(OsM{8G|!ktBnC&BC7YvMV%h*rJzkYpAi&ja3tQ*>pjZYmfwIFw^>p}C_ZgI|M{=ii9hd=X>$?pHj_ud zziR^jK&;}+aXT1-S0zyFjG{6_m?cp?0$RN{AY}Jv;GG+Q6dD^7^F`+$+11~_`HyPp zx80CUQ0HXC-DBdx*E_jgVen45l7X-viF6~E8eM{wzyC2RQ%>jqY46JasZRHJ#H6Dp zqfpe18jU4OI+l_;rYO^Ph^7#YBy^}Dr%B~lW}2>2W+IrpFiVRlygp)-;en~=DN7Scl-{#Y>F0aQ7Q|?eE-~8}oZax8oIqqzgRh5m|#$8{347Jnp}xY(Af8sd($`m`T%AFaJ*#Klag$X}Lj%(p7Y zS&hD!qzmU>>rr=bB{7RFGYeP*HXK;S5I-zKqgq zZHrj#m6(~1Y(srVt#fQpF=lqtBDb}+y}a8C>2ytg>byeWD>)LYUN^kH4Vgl52(j#8 zgNh9=5qQ!e_Thc(>MX;!xkfB&q=+767e>j zMTR`wF?O47?aKiuqz;+;5CM_p?KbjKl7esX)4Yk{<-8^EB4ka~kUXT5M&NoFJc2$) z=3UcatnB3JwrI9)wS_d67>3tTAl`{cNm@l;l7w_-h^5K~BTMAw1cmM{_DiSvcH@Qm z?w5cYxSQuyR6&}v4MaqB)=CKE%a?HstmL(sxxoaR_#vG|MvF-1Uq0Ep`{+mVS=DB~ z2xD5TDqf-`x^xq+Ku|^93o0yQNx)X+dA7idR$CIxX6Sjm-lQLDv0c4_i%iD7z3u3j zNG=?3SB=@>+l~M<`(ZU11l^egd!aes5VY3#DyHJJDH< zT%H?}C9D9WZ{I0v)CA3rCxfB9kPE<4ETM~##UH!Hz<ZBON6wI z/Yr#05Gijz~C&eUh13MSAf^n*Zg0nLo0w9mM|UMA(d#m{4g|1-rDB zUJX!K!RD!6e=Itv)toUSDvE8^KYH=eo6$vmYr2k|9_iRLaBTkC-t@OE%DG=Dsz(IV z1aFPW$E9%1n(nezG-x@z>Gt=@cDizVj{Ndoq=saMNR?H2E{9Xl9*x zCZls2qj0wK!tCKY>?_(HuC2r!BGn6~A${z(j@e|&u|pAN&G7h4)ORWlukJ=I(>7K= z(zI*1C!St5+H=f4nUf6)I}{y4164{Ba#Fq3-6*g=r?KwH+tTW8GRctD546lR+$_`2 zTt9U5CIfZotOf_>F+c2(`-W;nD;C(~WPEWz&i1e|tH0SattBU|pFLPLea@QFj;F&< zF@&U>m$=1qX*Xawp5qDx^<^Rl;83Kmf2|8GQ*IV^Ze>PveTVk|06JW7a#d*M^B1QJ z(N3tr9cebT^+75nr723GtRJXzKQ_g>xw=1oOF#|rf+am48#w1RIE|Y*_Xahq5dP#T zn;B6yKZ)4&dL)vx2aAu@c@nv`mEIQdk94-4cOW|One5)#f=|RT zzcU*)v(o0g$2tjmFe6@6XZwx3ewh%LFGKW#x|o|17H`dU#>>JWxLRMtH!{0rB{_$e5 zsMm7lE#Km~uCq18_m8NaU0GtVZ+?hZdv%!{AHv`U`O3-rXNJJIOVCLzP=J>~`Wc4N zOiM$>4{Ts*#!2!zvp#G~PlErisZzdrKQggbwDO70Ws(Auz%mQU^wCK}X8?OmdJw}% zB2FgwSSIdS?JaYnhtreX>bS2QMu?gtO4ffZ^nGTYAj;scx#a{ZS0++esFF<#_ zYbM^ON)&cjqjGb68tq~1;e-@6b6cEVZ*C(cW|jMt_z2GP7AW9U!KWCJoY80Z>~Et- zeT+LH#Ht4V;|;81vv{9>c)#W~KP;*wGBRQ%5#~yD%xglLTT^W^RK=vaL{Aw1MXFsG zu`|0qCYO0%8l*})EMT*~DQiT8r7&xnz0}GE6S@>^z?Tmia%m4OGz3fXp?&wT)%DT7 zZ0Dhp|u&<0rAc!EcpzB~nY?Cuj^s z?oA1lHWSc~|0fNVzUr1_%HWt$r4CL0^di^`7wdUj1^v~+&xC>`)VKJ=>su{g)HO3h zqsF`YC!R^ESmGg5l{eK(KRI^{X}@-p67!KG15g0808)K0`qk+v@U^7p_Q=IYw2m;nX%jQn1N<@fEj2Hr)v82>sVmk5!p@k1M7IK<+ zhRK{bN+6?g9wlbN&jyH3ekJGiHsPhH{~?fIFo*{2nCe;UMzGCi;8#O9m-GGL+1$%E zmU_H!zZ|YCGZ*JiYRG9Sp@=?FZ&w&hu6e}*I?j=(hVv(Sc&?z%PfB6r-!UQ)K-7@z z5>MH`_#A|0mj|fs2TF0c8;Rg_BJT-muK5QLvZoSh z%OA)+4m)XpVy_-`EIQC((>Y6U?|DGv+3B;x6`$#|W`c$5qm6Omm_%|0HGMg1QBqc% z^#1d>{;5PG9-1q(xu9mMXa5}yq^Qf^<~Jma@#Op+f!6rsnJe+j^9_TW#|MfCy*>ny zuAPyK4@~t;D46LNCXU!mBTdY>)WaCWgcL8s`B%3w-Knsn8VPTjlaVrhX+)o~=Nvu> zERfB(=`R2B+(Yrk1iZ0s*R2H-i#XxUUNstc6HOFa$?A|g3aBR#HqVMb-9*t1a+6R$ z-4^lLXVNd<*;{wp!oTGo+m!^sWRC~wAqe{rEL)D=*C!0 z@RgkI2c8mmMZ(08uq1&X>BNe0<0?c52OlBO71Rmyxw@#%EQ;`x0>lnG}gWU6h|vWD-&UrQ*O{RsNyH*M`os2V46D2|%jSHBbN2ax-ssk7L$tr(=x zO^ALXtO8{EUwhkR<_G~%OeX(lAiqwvDjnBnl5hy2`b`#%1<<2Kd#{&%Ir(?wQi3;6 zBh{zUqmE2SLO}G_7bdL-iA)4s%+9ga8ZGmRcKOqHg$`H@&D>GT1dVs%SFU?=QWX zvU^7RM%hFji)p_w^V^u@4dzOw6eDdbm$mkkkrNYfNo^wMPw%(sX_HrLw`+F19XFG> zEY8$6Q0BK#CQ-hmPTza;(W-=Eacm&JAu`0$b;uo4i4`h_*qDNWPXlv zzxtt{W29RKs>PkH?8uFU5pOKJ0?Y_L^*uH|)MnlU!h$yd}-quLSk_b;!femS}FT=rV~u~CNG zgx{et+X&R}-CLnnnm1{Zcne>Mo;_o7Y)QP**!GGk5ik79H}rX(cBpf zPobLNX{)4IvnHxy@nwWl_C(!irn@uS80c6L_N3kI2b7AWKpe{m!cV4cMBRERrF0_z zb|}Yx9iv?AvV$+=UJkcEvnr-}LCW|p`* z1QM@qH)5IG>Z9QMg7#Qch{VAE+vA3q@*X3K={0O6?Dc2T&sLw|ep$y(srX8nLWUzi z1%H=Ks69Q5TILs@=*F>uH+q{ZFO(F;Vgl{h_`6?k5yL>C`V;2tFNc07_QYNX+>cz$ zG&vtqNa0eCBg$19#2Xx6N~)}GuBHa@*kn$vy^o1a-D$lbdY?TaJ?=mda@KNua)RgG zFG+!Bl*A`=rZDZ~A`dAZM7cNI7$FxJDWLb{c0C+h!R_l+zdHr!E*zA6-!7es;bUD2 zqavjBbK;+u-$#j3 zs=-!~57_*E@zP^q_;S>b-md;TTb;#&!!Y~swl zOip|{p^0MbDZ_oZyL4lLDDvd}EuD8Qf?j?4q{V{P(fnaV{^Jeimh1wZkHjL2Ia-)Q z!QPFp?vY!3>wb(W{jKdDUedFkMxxo(9|>;4K9}oWi#B@x$VVsn{O+5~54y)#R?o=3 znBB*IA77C0_|0RI#5Bb(g4AJ|O2u>@4Mp!1JLwc@*yv$YL)SvDkG~3iRYObA_GV6I zo1rPZ{KI`!Y%5kPzU5oXm$M>v#TE3sZKJsej>e)q)sgy8V_Y54j>X_LS zvhzoEGAPlp)7jJTMX-knwt3&Md(GOE6#k$st@HgOW$QZ`cMcfs2L}gP2fPPn2PIdE z2AKyAvXj){nP*|y63)qmS*BSID+DXJD^FJJR&+^%%_Xj+KzlD+$yp8-M-*cgSw z5aEPi7@9^&rAgH^-wf7_HB+!rh+1&#GU;;YLW_;&t}1EO;nlHa%^Q->eV9w18>dq< z=CmF*_G#?WSYU~(Rno9ep2XKreb|HFS38GCvmI3z?~gWyUy@D;4H(KzCo)2y@3!X zhuj4^;_3e#LmUIuU)5j0e-$I?DsjO30QZ0n9MY?sq~E1*rM9FZf_X_2cvmR$1Sjn8 z*}EW(K+%MAOX#RmP=+_sf1P9@}qiVi}>EVc6m>VR7Tl zVQ5?uxmBnS`<$rOuu}8lFct1HWkK2tt6?EN{~7vDg4^yWt0?O zsFbvHhynkxL6CFMz4Y6JjSeQ7%~PR8!ErZS853_&5*ZSoa~qp|*?m^@YPS*lRTaTB z-6aNAg~~YFXci@yY@a-FoNB_h2oJ5Zq_ykSL;|6myx^_1M-Av=$M%dh`M%($XJw~;XSdQEq5>ty7K2s^L6v;o}5 z+DG3{N8DUILQ6WHL4>EzX(sa3I)Sw8s}H8(n;!-vJU*y=UFUqjt2m^Uc|~+Zciv?? zX$~(C^)*wVh`eHqm(lC}8(Wb>S-5IM3f>xI8mArHUL^K99ZoyTj_IQDpJP8k={dP< z%sW^*w0A6=yK$3h9C4wFQ}NRamEHPx5Axl!c48OfI+KLYx0Jq%4n|r=%E-ake&`lh z-L6$GF0czbNht}ejoOxDkF1D#8|`h-sq6K9F-%nRbfS*dQ+8Z6KaDi{TJ)>vNiB+U zuf@yjH=CF#^2urneQuhnFK5?jC~0}uPNaJEleCi@#fH3Idfi;2VH2H;s8%fB@PIys zPI}Dl1rF6s*H1r&n)Wu2DbUBY^Lmf5jCt9}9Y!59#bs~jHs(s|Cc)%JUG|9<*aydh zOQY@13ni*kO)b83s|`*K4ps%ZvgV50%<oObP$`P&)T?L}9!^BC+`OiYMmBbMIQGc)7`=m|RpJFd^{ zy<1)93%8%Bs-^`b1YkYQT3>?t%N%1+uG)X=q!at!_v4n0^I z7WF@~U6);5_G0y(JU8FcZ735w+dj-m_Jxdqw}b4*CF6J)8tJ~c>X|0&v2#*<+b^t0<4 z`+;aPm&904eL^<_4Egv_Myha14c{1E-_L%+4GH&xtZayIKcen%aaD6rF;2Sg);oPp zxNrO6EyPQEXnI<6aQYm|e3fuNq@}5y;?eB~BV4v2N8##kE}aE4p7d`zoNnRXxaQ_d zxp|KI(lLw($IqJ#R3ERQTC(N}3J@l6jRiqRA%R>1S19011cmg^wG;{?1nuYNs1V2t zD+v0<8b$CE`R@(*Lf-T1Ct7#_1QYy50KVLkQGc&SZ%;=1eT|w4{ss|OlaQ4KKh;bf zpir2jrM;5^S-=qZ0Q<3wwj%^WNQeAFkyWMr0iHi>^+?M}OF>@1)ZUKG*v#Gp%I0SG z82KECkedLww1YYsQ@PpM!W;$MgsFe75CGT6kJ+iIelBsc5vJBsP^OZwcYspyvT?9+ zP>WoqqM{OVFf$iWeIRvlJNQkQ+S1ACu>d=}tE(%UD;Jx+g9ZCtetv#-4o-GXPFApj z)zKa1WbDQYbENt8kl)XF0ChBVuzKudWe=l5KG)d9-q}f*ni~0{KmUHw33apj=S?uj zi*11ovLpY(ewU4d{m-+(twP981(dDaptjl%tn2{Kz&k{E?{NtIT>l?``R9#)-Kq7@ zop(7odH=rkuRs0!t?G_Y2MK#S@J=U@f8=#>^WT5GxKW55x%IzT@r%ztKLv~yxh%x~ zM`$9KSzE?JfgH)K9w zv}|2}Hqyjq#rg?d^4VPeKsdUJwat2@s*R0d&P)A=>jR3c;;bd;@Avfw3FWQ@Cp4aygpwZ#r)8PUUl1cswY)MTm|s20{7vZ>U5| z;>v_>c*)0DRK5_@fBUB4n;aDKpC5x&zF7E?eK`N^iTL;^PdO`5{{2xupG+;LEXDU< zsK6H!L&s`g!uYq>|N8YMM4Wi?KPgAP`x+HvD(Qcr8;gnx6&>n?_V2s>N$&%&DcJuu z1ytdGlKQP67wT6mn=BnlVJE-TTRiW!<3QH^_D?RqHlpF&O}2|mwz1O5BN`?;(yyrqFk3@{beRVSMjKN z=}`_O!|;aetRaeVIS>TDrs5`^6nv}pm47xBfqkJ0ygui z$Lhh@y-B2wSDe|81!ZYi>i_H`FCj-w2tUFG$ z?T?F+)*t5nr25NED#j~i>sjt`mky047cjoEDHH;|` z={xX!A>!JQVELPYcj_@$O-iq54!(mRD(;pkAEeYh8{767iaT4gs8VG{vfc0TC9L)@ zcAnN#<|rasWfds*yz>AZdi*t_w-8My^tjnicNddm7pm2c>|Y`9*#;6$ma2QVr-eVFJLtcqL}nQR{xLH z|I_OKj~od8Hs)%6-Pu?Wxgf7&!|uvp$JJXL^+J0CY714HwOk$Sb=5J@DDAZ;Qy#0V zJ7*`Gn!@Q(+8)bk3O$L!IWSk|4EY!n1FtQ~1DmE*J;yF+l$HrL7h<bd}3u-#3i^D{*MMxtEB$wpO=*U5UB`QD&r==sosgHaEz>h9SZG2fv0&$K0Z z8F*X;h6Qx2M?_dh{(FFa8=u1lN{ts;MJ{c0(cI_9!*!i&MLjU?fv*L2O*b`*J=e;n zeHJ>odOlVh1aWm+YG@f>;<6p#aO}L>atmj&J-UU$v-5P0LG+HH{&v+yMhpL9yklpw zxc|e_=cjUDcXKZG#T#IERcnQT6?5TooSK{2IeDC$H3y0OOUeG7^l`Q(u-c=A!&?~% zF0uSS>d|JWt2`XnKDESNqmgd_I3JqudXvRuX2B@ z+-fFe{!(eC%1=0xedgJB;@F2M7x|SYyrvgZ)+i?Y?(y^*2(wVNPzvEO+sfs17ykfrf!OzB z+h66_V>au~r=~ELr2;ChL38pdv-|3Eyp9fML)Vu=Y7VBxTHIJ9Jfwke9m`~R`*JQ zu*f^(xKeOQU@y>wOPAG7S!11~@^brx$T6L3Fwamy8H~c`@cROw*EP(wE(-~EQAmi@yGcpgnR*iGTb#|+#g*pW2(0;Ss(xGu!RaZCPvedOz zji-fHQq>Z_(ssXnZ2ZWQo{O{E7~R zpPx*3O}b9o*;7ti=j58io1;So`f4!_h7I90eVF)5SghU<-w>BU6&;U2%cGQeC+>+W z?A1n2yKy3q#VEs5;1V4Y=W}FGkB%kQvI!HQEfgn|Bp7%Uwxv@ZHmUjAL^V)-|H5s_ z#ZaN`R+vZl9ZX%}$c*lzVBs7k2PmpmjC8 z)qbhxcA3w9rK>eO?8S=__=&?RwCiD#L;Ic5Ao&GDRjDs`stjN?EZWn9>_qxbVUtSq zJM4Fu9MdFia!aBn(-XHJP0OIh9lBQ32!+2$id>@+*QCYcebv#)-7mj&(nP4s5`L|W zZhmrFCq|UTVN-v6IY&t4p2v^Rm~#8FJApzf+#W!QW|P!4w3`fGphK-(*!74lRLif- zKK+%@iY4DhsZHZ+Y*xf_G6ad%+ko85?Yo_vs$YEW3Ib0%dnJn~()(c2HBA01`Es-d z!Xa}#V^jN#Sqz@eV0LY3;7s*Hvb+SHb7TgJG$#q~@j{$^N?QbjG+=~9@$Of-al5vH zjR+-?R~`o&cEh1${G8!0;vaAho6@FQ(-0Gz53|>uZo4Gd$PtAo@J%}Pt>olW06p*y zn}AWYsWH(YLJ2&4+@C&a6M;>$zirLPe6cMFP*6YNp+RKjkS^3MXu(7M?UAG)>R#5#(rIjP2hcH5@$%Go$Z0?8=o9JZ15?Ya30oo zNtp7;g0Hqm-%(2+xn|Wi-lnhYa(1*BO;koRGGP~wC%miNw9mqf)m~_+LU;iz#In|# zoo&m$z2xVD#c^bx<}ucGI*H!TDIF8Wp0_WiO}{ft?o_v!=-GV=A}OB(X}GYIX2cQv zbgM;5O7T{Oe*fuSKD^{)!ecepya9)=cvIqx@-JhTU&y~cPwCVvE(P=CCOB)k2je`y zM5NcN_>$5-b6W~#!3`qhi%EyACZaff*U7Nw6>f9x#_3ejpKW^)A6FF|b7A??k^9En2K7 z;$BE71a)wmLm^dC?oytPcg|E2}E72ZL5zr_gs2BuF$lvEEnKh zNKS{f{w3$pjh3fkg6I?QnrQg$`aSR4J)_yzpaLYefvlwk7y|d@7p6aLR*|SgJ*WaX zMJHR;nu&R$ySbo)1bvkv40l5q!Cy|lsqtK}q;BqRiu_Yf8b5XGI%}HtxqX>bnzBT< z8SK>5X4+Fu0VbPQbWt2=7*{zszt@rr^A77d#>hs{tH;Yp`zR7`1)ItfL*j*nVKHHFr3?}94-rbz8o4F^jy$;Nsl~{$o?6pQ@M12>?A$V`DPF|LZ zaqw-Pp?Sw`NHDf>a zA(T51LEwHgA04gdIhMQz^24GsJ!{f-AyK*(tUVCyGFhj!Y5-Gj&@l>vbZKq6z_=km z-OhJ5RCEfaH04nN(!mkkyFDT&YuP7j6^n^3Nplgt*zMV>>L1qz&h%8SXyrQtDCweh@aC*d7V z7FYuxi8b;QBk@y2oM=WS%fniV&!;^0R&IKOpg}ix%qNS+$)kt?C3&-4`Bu|_|EoJm zL}0)LBQJlZiDLCgG=fh=-98_n5zL4kto8wQ=^#3I ze2M3zkv~of*we=u)NpZa2l&C%dVan~$9$$rDpSa{mCF{_tr1zSynY}Fk;HDF0voUd zp}X|+P`Df$aqz9XU(S|TgN`ZDcqg6uWhA(rMP^>oq(mF~@W*mOAAZ0i=ymoOKYGQG zcxxF;*L~Z*`0GRF(LK%Rzs}K0I!UfX4bd#+C-S zVhs=?;~roh6iRR=LqA!D61}rFSyhHp?^G3F!PR{)b<$_jd3ZT{35YQegY<58Dzo&QyzMcjK@7>0Mgtsc`Us za~#WwD6ph9uBkCGD8^gPMLjH9E9@Y#$g$|n(5>C+7L-C2p`qzqlZ9^k?11Dl1~|H| z)>iKR3?<5wIYVOgs2<5nZZ|Bt&yz4m4j&OC-9^0|fOy&C-ad4G3WO&3{%v`4F@SE0 zRU_3P?FCUD>+5%uc}v^Hon3#T;3>AlU{*D$;$j`cYmh5>RtkeQc)< zIHcY5g-r>LU4`(@5k`!b5vGa!wAS}m@FSm3p=H9y(E;kk>!CP-j&kgHU0GO!7SS}@ zj8kB}y9gmoV4^`nQ*Nwc$_vgrsuk2%05sK_+fV|%nJj8?^GcNvf^Kd9Aa{S)Vp=Ei z_@@OvV*@yr2VU}--E~kyj4$y&xej)uAhvh-W|t_Gl&?O!FA2@oDLV*h9p6eI;xWTr za`UfQzz(U)aEGJReb3_6S=P39qYK8T%f|YXylqrg{E>_CD?vW9B;JX!cw`4hB{BnbX`sf_lY zyfXm2`yZqK$LRlQ^#7IXD90O;td+{c@%sq+4$#Tx+bE!mz+Nwfm)w4exRM-}Mg7wh z@qsCdb923ANr4#N2pnP3Q*%YOAM`(S>@RK_-dZQqVLdt_(g{>E{OL^%fj9NC(_VQF zLDi`DzP)(mDmf|&`u*$QHVrLV6}>M~Bq}@r3<%xe$3Vrpq2bH^`neqm+f@|wSQgOm zpht6byGUezsQM;%c9^HB0?&kgn6gCcCJFKDML+!OedzeD5}+4>Cm8}zpyxM1Kg2jL z;EDvqkNyhiiL}*|QCy^()i+m^M19#WLcl+jlYbX9i#802}_qX0Hz(VrrXOI3{OYtvG{KrehT7YMB_PVZm zLES|NzJNG{Kz^{TJa-F=N?FT~LfB)(CNL$lV7T@9Z4e{6KqttM>k&UaQii};AA%^b z;CV$&6de`WGx|pEyF!Cs0D+&Q!o0FWklq}u!U3=2k#7mXqUyy)^-@`TrZGcuO)Qxh z^%P+fz|`{Kg8pfwi0zG6UWk1ItFBg~DSM-%5`y>D2P+N!&Vh}AfCH9CuA!g{hywv9 z^gh(1LB-;h@U4S)DbK&V;Lo=z5N|eFmzs_0@6zXQ+4uzjLP^q1r7yJU@4ulW0q%KOE37>Im&g4QL2m(& zbc55{uY~bm7krBV6#tWI{L3@ZNeHPJZQg~D1EyU^_t;%BX6JgHqFAn%2iSn$ssQ$0 zq2fSwK@OG-(A(tZ^lYqWjP*oK___%KG`_yT219+268lMt84=J&SThBXh&WOr$;~sW zKy5XF+RnGPeZEi#D$rIf6eEeXrpC_(Mk-n<2Om5Q_b=wAUziA851B$SvS!GBpmSbCA159`HH!KSFM?^{eoP7 zllkj4-`)e(ya;~y_Fwe$LU#*+ZbA7>@&21E@Bnc_wvV!({p<67(*8ytkW@wN`DZ7; zDFJKU7=iGOQ&s8y%R~LvfYFV1ji3PqltK1b_pMVISCVrU0RJF<8%XDH5LEV?@ zpy`<3m30?Y9Q%jl-Cs@7H%J>o^)LJVMzM!r?~%$kfOF}-0gT9^lvES&d6Eb$JHX*b zy^z^629R(W#HpF0KP3RGl)Dt5KxF^Y$^{fe0`l&>XAFR#Cfot6?zZL2BSAs0WCsp# zgVX{2w?LG@?yGq-nz7L7z^Y%(NwNDtYS>q&{@BAEz(3XdD!_sM2j}bULE>CD9zBM1 z3cok<`|1O5iE`MIcmMM?2udAz<|)k3`o-_Xe+l3XaLX)oH=6$AZNNdC0u$J&^3Edv zkGFxe?r#Pz^`*Y3`N8;Fpa@lu2Q2PQ28u@V%#`o<)^)&3mKHL$u@2emi6qa*-;q%J zia7(#S9djL{rz-hz@U$bxa0U@$+O6?w51>eDwSk$u#5+*1$MDx{9<6xyF`8z@_ET`7K;gkRUQJi7kKo)NCry^?Cf27ddly`f3tWG z_@%!AHo0ifB~SvIT;PSzc=$&tno7f&wOfS&Gh>37hT-@FAPMw ztW|tTm6qY>VCN82Z4hdsB?EsA2!I>v;Xo^vuz%JGhBs})_R_5f89E(-bz7BEU&+_>L+_9Fo{ z!egWbY|IDihh$%<%LhU%D&P?6;J*y}*TSDu4#@h#OJ%pxJ&Oz#Yv3BNNOCgXKRV9? z7+_sr01WK+v;SOCu>fid#^T2QD^o!_kb}}-{hzl%{0hJ`zgXK_|C~ekH{((UDal8S zt2r0J@Gn`Qpno9%t|q4@>yHZmb??8h4k-*~0?9DSn<-%1NaHy>OM8}b>$mE^0mR7} z{QyAd>Gf}cDR8@o!C(*}0a(#j{tFBN>{8}3ek2`_8^8jmfM@Hg0tasZ4^;sVe|eu@ zQlkZ%A-Gj72|+nWip(%hl?L3(07T}b82NMJ0~zSTfjne!ra+W(MKU{2HVYpW>%$Ge z?1zl6|1cXHF!blpO1&tM?eTE>02T`RS8R}5CR|VZ&40fSz}8g#4}L4n-354<`n?)KJyCSl>TS=pR0e^@E>o){|HDaP*7_6Z_@gE z%fcSJ1k=^ zWhn;0&3vT0N?br}cd!Sab@B}Btye(0frSRt!JSZlD;yQbkF;I5_CdhFMFx{q5}TRE zR|}wd%oi$pR!%JHQ+)&g!A|Nx7849yGRq-7M};B)V7*(;NUC)vn%jy=^xBp!nhg;g z`e+Q=)Lj7dhec_d6Yq2wo_%Gh+5O7XQR>iOm62dRiL!)lO z8t%RN3DDtoX#${^W$A=Nhuf?_o`_|@Rbqm3gUv3MqD~XG$|dTe_0sVn=Am7?&FY;H zZR@PGHJOI_DBbSrA78F0h*~T$02M<54L{KjU1UMZx{YXh@>zTIm zpnBKAG@LR?UYd&)+-p8j{$v4kHv@+cHm9d8kN6kf(HOMEt)L1J$fL>Vv(+A&6ahS{ zHz&6=7NIYJZ11#8 zxhy9Nc{E;=eg`Czx8iIalV5w>Zdq{sIGmR;&NzU$lW*Ef?aN*hk>NXJhtswWvA+O@ z07}P0gtldnE&0eun|<~UwyKXY9L7(NcKem&6eX24zIn zh`E}D&QEvDk$^iUrB^|p&65t~2!hMBx9dqx;^Ig{&seJ_DRML6$>5t?V5vECuveisFkMmmHWG=kpL>^)A8G3Nwr zf_I~^s`XAq!F4laXJ7}k#79?a&(BV}a`J1f4+i+BoVXLachYaGfo?}ha%Y0;;g9bN zK%S*_E4gt1d(0g-Gi0xrBhWS6>wBXrp(H#kj!AB|s-~*t(-&r!DizF$M1+aCE#Hi~ z%?5W-c&-`e>kB(oET_lK$j+7cuw@pJo*#%_xW?a(?8ZYeky{t~(fg(zkVx2eo;8{T z+CZNc?$XM8N=ih5KAsxHsq~l`Kh%9;h+Cf`#A7gcIf-;=@h-odEA4TkB${jOr$`Ov9b=63-XEQ{ z)0w*D+?L$APJ>7Ka&D+E+}mk<8tnd%M3H|!^3YAkAYV3LSL#L6D$mP@Y`}7Ssuf_Y zCBrO^0ZpsBWHeGiFLz`2CmeeS{6Sx1)R}Xpn}Sn8IbB@8aEkLXYi;~>#e26!@;gm$ zwzU>55!D?^$}VNrRm}&G!d`Z)I`6NK^L;Ii#=I2yao7$?Mt=Ub(bIAk5Yhtv3Tcu^ z3skjv%!;(Y7uYQ2-=01>e6rofU{QH&HjD1!ngu^lvwKRSh+YFc%! z!xK5IhxMCQZj&Uptrxn~gb3&wKfgI+rxTFOvM`79Coqd)f-vrEl=`F%xqB6KGsWC{ zwEV!qbLwkwQjLKr4CehoNXv7s?ME$co}7&1@N>lM-m%oe?ZATy;P+?kIF&TGK_jJU z+>C;wDp!c{q|^SKBm+rzhHx1(8-$Udi)@z4_q zhfGCfigGS$(<= z4`Nh8zlPo&0K-s$ViKd5b1nR8)m+;fo&zt?a$KW576>~_IzP40RZI*ae!&cybmmR9 z9;>M~qTYkA&YfjRBPW5UqHB?#ipoNH$JUlCPsyZQD<6OF(;Ue}?#pfNuJApN)l8AA zMquhIWSGIf5QlGq!yqaCQQfIm+jKt$R40F1vLPF!1ymv#A?c38J}X%n&8vgz5_vkJ zBr9###b&n!>U57%X!^oS(V_GA*H0!9CsxxpADjOcM)D3wRR!|9%1uB`f!xMbEu0&| zpwBjgFWEp%Wd4@&p?$8C-CeU*{3`7v+U23Q)AzG3C3>u6xm8WgEW>0wX&4;Qw#pT>E8&6NpVG@DR(cxn zM9J(|6qVC4`xt^NZH9-Dp^CzDcd2jdIGAtho`_*Cp8#f@U% zZl(C3#wL2-;F86k2MT>fD!M zSh@N&!@3fpR7RJC8*{ZC8Eo7r6Noy6eZ>wcK;W3QlX!tUsO}<*0kwXs7ann>2`0Lr zJk^fFS@|Gk$q;TVe=<9>CvY%6*A@}+B19^f1vxhhv1l2xhuIfAJQG8b zN%$SiiuZ<}51#8c&=bA%22JTckc^Pn_jAZEBEB-*r*Qk`NB6yRy=&FhD@urJ`JcyyDY2GJIh{=w7Ru8u{S6@sp?4Ch}IY$U& zWpT)<0~Rnn`Gg<)Lnjzcg3(&O9)4Rl>d?X31*REE^QS}^XlnDKMc=lB3YTs?!*#X#P>x-Waz}E zh6uEi7tT0?L6L>`FDN)2;<6@PK>wS>-2^JjPrPM5*A^=U>*sd3k~5h;ybA{EA|oB^ zmYiidtl*&1kPChmw2wfJveFiFdw7>biJonRO409>#WBdgGn)I1BPErjej~ciS4xVo zd6kZl%UZRzl{HCfp|~h#PitH3`8M@wVv+B&cl$5B20DnN8ZIhM-)U5m$WO>jJc04e zq9vy=q1*vF$U}BDM2_HH;gsZ0KG1~k@^)DVhD7Rs+T!7UpQ%j!jRddlHo7w2l{5Ds z#{O|UXW4F*^wjYUpiqoQv`RI&*xNJwt!s+gqF|iNaG=b7`Y_hH(5uEyHEAG*OxdIu-*i zna!(YwagOXXYM;-_F%Nn6j=a-L2^3+2N82tkx~mooRy&=hYqQqDkmyFqqbdx5=0VL z1b?Mi2D0E5Sb6yLHciB-rf_A*K=_-U5Ds0tl0oYz=4ox|Z_6)v5!#mTp9ekDHt*(N zWML;f7lF3l5cE(GF?uKBeSE88l4~leh19(;!+CgTsg*xQB6sB_hie|l;|E)Fb4ov= zLtP>hLxvpXa<4&Qg83I)K-T9sw1KBXcBuidrVqe@N+DbM+se$0a-Wzb&lr-+iU=b6 z{wa^qy}%i(oiw^EOjHyk5HPaDuNjm4@gx}?I>|SO_0Mygz&d6{-N$qQ&rkt~8qKXH zW)K^gWH|EyXfEH3`_Ix#0F-2YJp0%R^hyi?F|~NQRfN46JRx9YgD@WWyDrMJA20G; zQy#qX?Ur3gOwS_CE1$VMn^>x)`o{22<&KWO1OfEpTGFxsKtCHG+80n{GAdia0H%1s zKN#r;;Qr(mAKXF5XG{hqJPPS^x%&{mFW`*L&Gb9v_!kahi3<5RYZkVfDCiFXwDwT= z5IG(7?_kCM+lQrzMHhowCKj34vD6;UEVMJ{v-G|#Sx{BH_Z{64Ov|PTjG|s)F+xsn zKY|t!3V=MP3yj2cf_V{}Asw6Ck5%K_g6qW*#C))LVoo!eXeIhd`#MA?FmOCBqk4!V z7~SC1ex=q$K{%RbxnO9k%hXmJ@z6n6xLk#;s>&@EvW!LMooS1h3M&fci;fk z-gfC&o+A$UEaIO8;H>-X7bQU_C!^p??_b!;q}QDyHsO58#B6XD8EDN%3XILcXm!#2 z+2;9KzBRwbUx1)8GFC@jT?KueA0UhBvPn_k9c=_~2I~Sa#FEbvT;1F*Yg$EMo+kzb zjs{Eo@Wn!q8oUETNcFp5v|m1!yNwK#VNwlsZwQGZ$-z9Kd9=PumhN7nuxG3f;u!pw z6idxuvQzjVuWG~kCo%yF;5ddvw=V(4Z8pbXq|h$&1Uc>v%Ab~aP&c%7RsYE)0Pgn| zXc-gzWng00fFi{XGbNBkDj-L{i+k@cfU^tS$cf!i;}_(%SRZIc?(66&yC4VCC2;va z*s1q0GsX$I7a|Wctgjb)$jyWD*g!QyO-Xprk})%N7I~_H>twAc+II2pM%!1Q&S@$s zSAmHmI|80XLdQ#~`!3Z;p8zllvIS6XhK+(ErmL`n%?58D1hlbrBtq0Te$#o}B#_k{ zSLD!$YuvVIl#@hjiPhWVN_@!B*J~o6a<9||&Z!erXyk?iV8GTQM0j6|RA(70zb{of zjybRVONeqD@-#+n0+E5+du_Mx&$~hdH!Lx5NX&T0EK}XirKV58EbLtXpX81L2&!vP z)6mZ8jJ2j)n!>#id62F6yH8U4LEXKoJT_>KrNVuYaCgs|(2(S?| zpt8-fU19I@gn<7>h*yuZT%%n##3Oa2!Aie%zll_ z?#PSqWi^Oi2+m2Gs3c)Uo{`xrUk&G9eb}AfGH~{OeXYHG7-$0h7*MD+5BCDG*nU@S zbAKG(f)*gL$(cfepfP8HkmgmlpHoY}5Mux#riUerU;=hAJ{3#69YfKvIFGD-XqD;% zxj!ODBWJVQWin;_3ndA)8}s^&q$J)Y0f>{=?Tw1czAN!HHD9;aD_Q^P@?@X2T@o{=E zhpT9X1-ZShJv+AXdn%Ci8U%nZNB&*SNS-`3Z*xJmBi$R&)C1 zhbhHrsIhyx(<2SC_>%EC-p0dwN+L%e^ciUOsVmfb^M-Ar`gFsB<_gDQAjd7Q?)-c+ zE1$wp>bPkYezub|?(7g~RjoK(o-B{1x|erAliD=n9kCa2&8&~BN63^Fnd6Tu?VfTz zWMRv>!o0r3W$iS zh+Qg%_{UF$(=l9OeZI2NHp>;T^6waBmESdqXqPQ>KKKU&kzlR3bIZuHkq|6GE z05@te?wk?t@UXhOrjNI8dwW2oti3jbl#h+VIsJC*1hq|vRd1HKmID}>r^v{m#3`GY zS;;^#a4*PA?cCK8$nIcedl`AuKwYB9=26qHBw91satW%lS!@??+cNh=NpPLpQyvF* zo>2Y7V1ZSR@OuqbxeF#}NB_3g8sM=#m>+_s=4X($eC2y7^!clg)sTuZfE=^2#M(n2 zfpK}Nj$k&q4H~7~Vj3^*8&ANXrzbcuHL-G7qM@8#=9G?b2m6?vU5=xR*84i7CJm({ zB057J5o#O6sc~&#!fV(y+Bn4%J$>Bv$@xQE03m0eSG56ogdkNm#?XeuT! z$P?0sv*tquH|~`Q_`J>FZ*KV-<~WuPass33$r`|d0837|-XvuVW!E}k#j(0zS*O2w zPi`t?l6t3O)Y+rMs-!7c(FtH0QgY+9NNlP^VoXdCbzc?fnzM)H;g#Xch~7@l7c2uEX;0%Ps%cEM~CT|gUQsA>q`f#!})VY zdx;)*P~P>mok3~C`r6L-q#4wr&n-AXl1r>^2B65Wpdiko>-in)@pLW2hRq@5HVhTR zt8bEXO+1Hg3f!qxe9FG#6$8qcY%^Ey%PHv3q)8;%xS|A^Q#bj@_3w4PrO%h=g zhxvVUYxN}@0i(phohlZzj9d<@u(qwroj?&IF;qd%m&lU~9}B)*p5%{@K51VcDm56h z(E2AMRlPh%Sn&ioQj**RDufP z1`LVA*b~mwdmt&8=yukydl1tds9~4IoAOboqIi{d;MMBfipF}$P-{l<;uD&@xmU3_ zt6KXd)_ImB@WKOsO`IPC*!dM`Hgz4w&MznRfwOVs%XZP*k;+xIW0`EizMf)!LrRHc zB$Ql7&B2T|PMqh~d;LfAyYT|C9jq{l!i+?Z_FN`+;kAN>H4)eY?Y3sv>Go$#%0`#% z;tm*D+qAy^`9WF0z@L$ZRzom(yL~4d#5Vyoa2Z?{7jo<|^NOMK-Cfh5PGTsnB?mIk z3(Mdc^Vc9%aT(TkjRDT7r&S2*f*u(n6(PGi8~El&*HSA_Oio9hBTYLkV^M1P%+qsE zHJb;cAoLg*6fbNFyq2MmGIBzX67uW!I$q9jDUOq$F#Ob$TQ+4a&D&=8Qc60Br?UK- zCIM}XUFm5090%+{3_4V5;qY~tI)G4yZDT1ou!$6GD=0M8 zzPO8b;;{r5^6UXPb`%Ey^~qK=v`&C`=&0sZ(X?j|=7UN8+Y$k{D)##0orkntcD33h zTiJC)u~lb5IY*)ql+8Tdg2-DRse{80Y}!^|Q2JQTVJEvJ>>IkjwrO+mxf=jRl>amk zlTrLmi}*s2lT5`@fN#qQ&fAXU&uPmL>T7fKQo~`!Fd+F(1rGKyo=TIg9K~$*b0RYa zaGG_J3<{1ZEXV76#21SSFewmu$EI&Y#0z%lzwq=1aN*4CN(2+B{`M`EedGa=iYBK# zHfJV7WGdyEJ5SRta=b)yGN+oGBN8?cOwu1_HNTTkVb?{WQo5Q~HD-foV)s!o!0}X& zWBK%K5_!nt)?I4Yi@Y)I6XV?H*Ptb4>A9}8{X;*%V8DgCCo_YI>b49S0H!R-gUdAg zGSCm{X{G(K02*L?$08L=r&JGWBr47{0?*-n}eC=b&vc64dG zF0ejfT!&b<&xt(UWJ>6NxL6vFJwNN-IVX-LCC$H##o_+iq|19UKuGFm+8u%(fsF|6 z2IoX)1XlTm0&u>hLGeDjWBBN}*4}%J<79L@W}M^z98z6Lac!MD?SF<9u*?XStXql4kwalAmmjY}OQXWVWWqX^ zA3Yh5Q>5O61v*NTfg{M=(_TAG(D|$=mxxXaLfONc`NM4|(5b>D#KCkZDA#5VZ=)dF zsx@iKXLxN>OiLlUXO*CD*w``#&Lej4WgP3=`T+ypt zYPjWe{#HZhPB`Q%%%#j=<(OL()Dxdh26GHc07S|loFOz>I43m5H@BP|R#j3~9XcA} zJPR-1D@KvHt*S`qQ*Z0_tLv5@wo{j1=7S=q ze~A_Wj-?0l$uXKEM+>S#r`j_k+Nva_qma3vOXMRkWiYu?nJ>V(Bzk@n3JwcP1_{X) z!A8yPgaAsvX7=L+g@wam(m{~T8qOH*iCxmZNy0`nj$2Y!vJkO7JjQ7v|V~@eu zUgvqPm}{;z=W)zql}bs{{R+Ev9X`_TRdZUks{KtA_?1Z$A%~FNmBHZ()Pn@WBRbIuECktZy0Zfb}AUzCy_~N4Q_&5C+|KWaNBW2uW zm|qB96Zi&8POeEbW{(YzwB##SDSx&@do9 z#i;4~ln*kC*&-UE2?8SINIQo96;A%AcD%*u{^DJahrC+ie}467dM{a-fOmob=D%eG z^$=RKjW`y}VV=m0=tM8$F0;9O9TW8G&lWUvvz_1cN0Ib8B;B&RumtTChtZ(>=3ov_ zy?uGP`&4LuK)2McK~%_5i2UL|W6iGysTviXuvMU+MLkm*wKAkoMHY7s_Pq1zf|{a7 z)T%RX^uLj3(|sX2;eWUJ&Iw%%=J;~cMI|qEenm8LMWv?)QB;6EKzfDim4c%{FDrej#+vtU7= zL4Ru$*G%E};QgJ^g$gMHFjsdAnH6A2{VP2DYonJ20&Q%6Advo*L}1R*xUu7a8O#jn zUMnn4`oE|I=e*Csw4ot&OD|dBd(BleyD-Z|@TXP>!#^qXxBtIs*jUJ0Rr@7G{q0pm zkOrUU=2w&a<0i0W4YVWO0I^nwPZt?Q|C3|;42lBTj+szCfUg85_A>N6Eey9TbVK`n zugveD4r%bOI;}s!)g-Vo%Lu`mF9aJ)M-Yxl1p8LAL-9DXcY=~F@oJeGO^F%4qKKIb$!2dDBwo^z$|Nr=heO?$4Y@Ed- zaVdg^TEq^{}sHB zJap;dANNEjE?g5d^aP2NUr8xl`woJbi&*Y`zyvfz3ZqrAPmXpuHT0ZgaMDK&ygVua zZ;ns0YvvniE@rs*_&xg6XzjF1Qo=opNTCI%A^9h<+Lm`JArF!W<;@8{sJi@l7ot^q zfm&jC74lk@U;Rl@&rOW4LagBku~k>s@GOu(&kpwIlYp5x5^{_0bIDN~#i=2%P9wj3 z88}7&fLrcAT3u$9vm_H}V>L5T_6Q9p5z%k3y}w#ixi*_9aZ_(N8IpvEtW9Yh1u2+> zCYVHr<+T!^w{d{SGW{cy0cuca!GljEgC-5BUE`oN9OupcczGTLk$thCbnDnH;0T~L zyC`@7$;m8!FcWtSNKO)@lxapD#{s6b4e~$nh_(*G_;jg?-ALR2ES*wq0}zVyHUw)) z@ZT)ZWI4=*Z>X{q((Z@-blJcW>0!(Gf>C9efKmg9J;dt2j=o)7xIClDA-K z+IfV)Oi2c(TX9@HJoUG&QXqY;LrT>Q(4LFaP=pk@g;}zF=fO_6&|Z8~KVTIFX2A-}{vxpueH?X0D^njFWne{a}Rq^a(sd_21_QdjB@%dSR z{FTZn7{MT(BL-oGR|5R@N3ah5BIiGc(~J7VTXfSHI;}Ap-wM0t!_Ho|wkulKB3>L8 zdG{d{Vs%&)^;wu)*28oCkZ^Yq*WI|e-g>^Wo~N?g!k43IWUH+0Drdkghu1h!NG{$H zV)P_3m+{+iJ&s)l*5w@`nB~Aw4hla8J4-k{IZlE(`ve0+Y&Pq$A6mOiN@kA`k0;oa zrFbh~nU=$}1WgO;LRHQ%GB3Vubm-TAK!r_Fg=-RVw8cSYmc*}oDM}y6d}|*f5TGLF zCY2Q;prNDRlDC@2bYj#9UrB<*XHDi$0I4kPt^q(b)(cUpl^yZ$6z1-zWQvMx91edBC||4TFyKjfle`{L*;y&XjNi?V6xe%!G#g&)+@vLdo>(n z$~6GkO{_M-a$mCvMZmk_D9kIC&xhoBI>rijM-lapTeU% z?FwBPtQ*~NA0fm!uc2j1@WLvm|9*CD7+oAwn1R|TT_@7n-a^GNK_s(KrYXV(l>K~}Vr zev_qaqSXofd#?sM`Z;odjS)Gn>fO&%S<@|gwbK~yj_mxDO%+~gpkp&p?l1i!9FyH) z+fBeyPQ6bS_=g1bOOZEd8p4Op(Tm~NUT@h8d`@!FjorgwOhOcu@M7fqbg@9Kp1M^+V80s z6)Tet&p)f7{Zi8)`=|L?-)hSXtRXA%M?OEKkF(Kjxq8Z=+A!$0(%FzWdr3qga#SwB zo+pl6E=ThD-U>H9aMBDJAC|piJ=tA}6}y0T1Q!;M#P6wpOtp==Hjt-O+`2!jaE!BGiGdI^qibe|ox z(|~F|T{1TC@B#*LvB}tK`R+afI@`w4yH4%`A{jZz;?fjyB?uwuH|R?;Wgj`DUg_RB zNP{fT2!+N)p}NbD3ouMAflqK*0zTnDla*_IVnWVHYG{5Cb)~|@Pvxy>Lhpjf>abdh zO~&1`EELC4_o&X4sHB|)hXeDY@RHb~Fb;>_R4MZ7H!RZL)a6kTOAwrZ&|%qXHBKRh zDM^(zVAK&IIl(!gr0Ai0Hu6U0f@WPcYvc`!h~7ifa9TxIrn;X@V=I%=pBT5RB{<%H zswu>ble%T^C)7OI^MRAJe{|J~_GZLE$19@|AXAivPzD1vmU!x-$=ht3C&iVR&-9or zd^lV`zby{H9I-w8sOV`lcCs1njJTN7;<{E7Q1;*)i$Aq&Ps`{R(+DSW`VkO=QYX8B|c}G zcuezpbE5RsFH5%?w!F^f!a(Zk*#s9tJ7#{-eJ7VDcw!$R_pfog)|-ChCNs_HiktfK zZI|TS@)99VDxG{He(5tilh0la;h4O7%s@AHReuqq^I+~l-Sr%8_+E-Sx~gSXS9DVcSux$EdQ$ewt|RZO69mN7yTGybksH{yReR?N zgf*|^QSn47rM@Jxb9fvXR`OYCn3)87Z*5|T{5#~go)xYa3WV2W=~U~qQmZn@rkeW0Vs)_8poJH`)Hw{iZGY^D2ITb_Y9Mkfp9(01f#rwY)%oc-g#L)Y zhZ=Y3Wr@4Bayn7849C!_^*C*u8wJyMvsaz)!%{VTi8b`Sp4Ad52soNZxrLH&-6faE z&T8#$BVjJ34l(g8E-=U|d?g8#<{ak2+BG#D6C?Wuy_SJz)ik~w2~X~Rewi^DF38Lw z9R176+XAkI_tf$5SIC7rMMt z_Q1s1sa2^uUFBNAZ+D&bK(eQw1ip-xrW)y+)XK@*N2 z8ud5xE9(WqEk(7e4W&^|m5amY>I$<9)7PM`s-jC9fCAa8*%=A`Pgdx+b7QPmH&|h7 ztrRL*RfN6hNnF6Nj#c$fINg0V)q-kGkrx>8y#}Y+z=RZeg3XrT^-p;+Agd4C8tJ#k zK2FdYvhUJe7+cp?pualyvD=x$rEa}n;k~}=tC6t}#jhn%TO1CgnGq3Uds6fUV-jeX z)U}-*0_rbtGlb>8!`ZT2(Y1*CYXkaoNy(Ox;s60rYLHUf8I*G2aK~~8*(eSELTGMs zFF8V_3&cf*H-7A?(YM6u41|19xAYnd{As$uR0+J?$-wWXCJHi#md-49-fZ2(nqL#& z;%{yAL^*C;U^i_KEYev9~q2-IFTsoH>H}q1V&G&f zZFD#^n|ikM_2T4JhX&J+F6+KWG!Ij(4Q$V|OpHz&=u!-vGG!?=2qyQoJfWDUy#1C~ z$jvEk;SO*gZ1BzP6uKsm6l5m`k@Dy}vKN5(Ke^17SyXW{KUYOotXyY0naD)lUD9&u z_M#Eaqh^!myJzZ@AMnS;Gu}7(@TTI?h?fWOt*Kn^nhM@Eg%fPpC~)GW?e2du4i{QG z+KS*Ekvx=6f84Y3_Q35}qc{yJ4JwXa(TvhbMK7=?AOdb_yhl~F`~@<7H0go!G7;t8 zGbQf;Jc|o)D^OpwkScJ`r9O6|QLY|zE2#=Qa0HZkYTN1K%DZntjFYRQuI&ZPw*5tj ztlGz91PvBm-_!^DjI{ZMer_HWjjaGy$ecDpn8GR#r)8T|+m9>yc2ID2w4&?mrrk~hH6N&RvLX~#agzs5~8;YCoY!mysrAYaZGmAhS`q&Jt*r`>Rl zOayp7dOEa#VB=?nKT4zeRx|OwTe;$#?{2%1;;vhZfU+MSsHX6A3#~u8P_w}TN|KRd z`QFKsaL#@lA*`kfN4em7S}Vgjv|JOBg|Ha+k?rNWdus!z5_WWMhOJe(j|^?4*j;Tz7j^b^O;%07%&F$O6h&u->6f1`?Wvu&9pO86jb&>*1)QUtxF2>)R1Q63XQ!0=eAZjwBIF_Zn?Co*hDLa;iw)aV+c6TS*~>M4;imcSunwn#^o% z1t!|9<>ZHY%*B_wck=z*gd9j8cPuP%7w^M)o7Ap{Bzl>F3xp9{GyDm(H;^OdV)qI| zZS>?xVV`o&i3CMidY7Ff(lI@QJ$vy4QsVez1Lh6EvlnxcZ0vcf@8kJy3Ib3OSuw-bN_HYN;>UxZdAx>v41^t-IB?Mt%;he5}o7i3H}wo z1KWthxHY`9stwl0A3i!Z2D{7dZj3p$Lm!74w6$+UFF&}raI?Bc#->j+w@ZYUA!e(a8hf}S$B;>j^ zzXVZE`F#>yo(dCyP4LXmelFhUap2zr#tR|#@Y%h=_m{syivQoU%}8SWuFj(tHkkE} zR}3VY%gKh-p1+D9i4t^$Jhe0B7|4Igd@5QZqLu4)JAv{16tOy;;S>}B>iljJbnM|y z_j#_x$vyoZB+0KBq5mD5vFLgVQ1c=%uN>Wz@!umC zPc!~Waj6s>DC#AD@M(?^2%_}s&la3bB6NT!9>n|BacU9bN$J$1o6GK&7P0E+rfy_M zfBIq&YEqU2H0a~7MB#N?O}IIau8$Cs>Cq^bRc&|F8PglYX_+M~K>%G?V-?+@Odp_O z;CeT^I2f~R^4SKEJwHqf7uGJ-cYt$|)HF@V>Lp`cN6gUF`I)oT2+#*GP?Gw>+r6A& zZ&8JU&Ouh7e_^(bh_Dmexey{a zE!qwXt#1y+MkyztfOpKPD6|``<+>0vv@vSRSDfZIBspaZEQO^C_h;DB=WI>TlLs}5 zuOS1a1Ew;*-Zlcb1sSBY_<5~C`k)l?!b!CWqFM_Rnt1gudzHl`> z8V&+E5jt|AAxE4X6na3x`A9cVlrBe3{unZ7#wO^7HJ2^Zt3s|9umTxMb0npdb?$_$=N+fX%-((4{lTbJfa&?pfI zSaC2Q)5@d1vhcA|64kp}A&97?oWKct-R?pO_-7IX6#OM}0y-*4hu@?ZefHz(BNzu> zM+o#;F|+o=icQTHB^tG3TmsoMa1QQz+=5mcY6o6UH>2wk5m9LrWGWn5xOe!SS5&DT zyQ@u{#-Ka^RB*Ydx4&~a6|cczz8K;qt0ulD5Ktw9t8?&~XZvNAkT8>%R8qHipNPU< za)0i1JT==3lI6Dg9=mE;+fmERE?dKo;JAqwxzJwf7}a!Vd#g@@+}&^|7cUknUWfhj z1=h-ZEl|47DTb|FR0A^5xs+jTT^QeJ$kxU@{Q+Z+4sucI6XQs7uIM+A_%ER#Y(U4< z-}wK@0b{cOkH^ga-~C|?Y9Z~q(4{HsDB2$bCLz)`}!EZZS|8?u2)=}QoayzJ$=@bAcR5Q0#)tA9=W_Zowj z0ca99-76R-CB&DylhJ=e`$`BzPgR3G2Gg7mbNh&08B(Ky_zZ9)qN8-|1CRt!aQmyl zRAA@?IC5K;E2U^<^*h57F=IzH6*6umBf&jl;4|DRL#t5ddVnuc%ulfR_w>VC{uZASVSp*msJgccmgxW= zafq;J!Hbr!{PVYz(SJg3%r%rycPCRKO#AA0KkJeH9u43Gz!=nLbv1*1BWQBXmZJk~ zR9PYoz8o66l4zVh9JyU!p1)w||J21G+-SZ@re7S9x!-91f1H!c-M8!${|HI%wb=Yk z6Oce)VWYyQAkXvWA;4b*c&|Wp=Ye-7rfFEczM-TND*At%l0jb%DSc)(D4ga0%~kl1 zqpae?A(eKe7t8|t_dxt7vskK1@8$BwG>f8>=YUeQ7@dyagF&gp;C*)S+_|$ezfbs6 zzwaztOSu@&Br)QDnf8wKBT*6Bhi_6hR}S8c8XARUfOHR{CtQaCT@%VZYzy)85)nZ{x0DrI3o=*I^x@%Awh0el9(u zJ8vk@oxikm+uMTYzHZ?NC+mHK;`Koybjowywl{c>JL4>E+Xws<)nmqgO7;8*qj zvHrKudHdsuFjyU+KPLbEG7s5v8AW!vSKHu0KaLXkfT1Q9Wi-xeFQ+Z?F}U#vy! z>+*GdF>po?V6}Onuw`Kpx`E>e?Z?ZocH&O<{(F%cT7w5L=FCKgQCEULaZ-qSePHb? zq$^~|b=uZ_yP{WHH$^4~w_B8729Fhmcr3&T{9y>5f^_ZB)ag2rIXrklsOid8@D%Vp zrfU~7m_GU4`t6?z;FpE{Atc~}9$cb~<}#sTyMT5*{wX}=IuHNR&HwGlD3wy6;m&5m z;2B87py9Xiyoupce(-`GYAJ>u&f6D%JE|4%6$PAGn9y!e9W%vAxYP?z(IvwR$oa&y zQnC1c4~HuKOC|LbVlMD*{@|>vK8PE_Q*4NP-1=~%drR)a9l9SNyA=SOYsy|wu>Trr z=Wc9&+u^nnwKr4~-fl#3m11%B4xEwZ4!?etLph)c9}G<)xdJ}FlL{O=;qCqh=g@o~ zLf7wCn)+2Bw<$b^Lt3{+O){YDjE#`azsgutCU)>aN*GsvzkZA$wlLE5Jr=zmz`dMn z5ZXNkF`I%%)pHe4QDx72@Tafw31Cq75YE5N6hG7=w!w`G>za~t_d$MtGWPpo)cv1d zueH@S?-oA=`~V}+^r7AJHq!S;tECi+OM2hkH11!sOSeH3s@hx^Q{uvuOZUf(!*`tb z0%xoME!WfpChmI>bEwSi+;taBqkz1;7cXG=ZHS0G0;=0jh?>GKa*R0fPx0h1dMt<& z$JI=H1+$}w^i=}-YE@E4R|G8o!)|6iC?RnNQJc%}p*(oNA!o9u)MMT1D`3}`XW`ze zHV_Y*&^)yBI@URWyaz?BzT2{DdcmmW@n=w@aT^EXfzfUDI!K>Sh8e@b7$;8O&4Q@~ zs0yzjanmJS*I-G5wuw$hX8oZwi`h7H=HJ_ueeYhL9qGYBPWB#rGGpNoKP6 z9$f-65zXYEqBh+rBU-flIg@&@w|ZoG3Ful7 zfaMFrHum$D+6}lbXamZCj7q$yuZ$2P_N1JO+@ z1JvzR+R54z<+o0hLpk!74Ng`x+V-3Go7f=yzu5N)SC;8(E}z*Y_cddR;Gdqmg$A## zx&L_~fpgkxMQj4JNr;++rB>}hoaQP}QHh!`+5P2 zR5`!SQJg(LfIK@{-24e#;l#kCL01_kHW47alR+u^pyv)P3w|~Q+;8?60Prh!UqCg3 zpD021y);;Pj|VtZgjgSbqls99`3fZWk}&lbDqR+vfcuyzpx}0fa&sHRey+k0JLA8d z@6eKR$S4~66b_6YnnozjxRwTmy=~#1J6Zg4nK)p~vWPLG$C`mN+&~UG!p3eeiHS0Z zF|#^jnObU*Q*6k%0BtA$sPq=%0otiZ%Hd)MQ)Jxd$fynEVu<2ggHC`zgxi%z>q~MB1+j#SI;rjtgYM*7$(;b1=gXk+DqLR?xJ~=G zJMKnM#=$8};OlR>g7ZPK_}#$|?dxozobU(0c3s(>L3N8`mDdnvzv52nu!D%Jm4*9? z#@H!vA;{-;@>4$}>puMuxf{;Ny|%b!@~-~=mgdyl0gNe;O^55KXe~E*2yHsg7w94-m#hvIe;$7qJ5g(QF^qP#d_QD z9=P&obh_CeMvoSauTxHHtc!=cd=$^q>_pFV>G;N{qfOW|Sy5b7w5}WH{`IoK>%$c* z;7&bEOZqcL5agOHw1moOJvJKy+1O{>BdZ#xtjc>n^V12wU6j8kBlld^z-uRqLLOXI zcGa4>qTIl1XE|vZYreG0GI>c7hk=7)X^%hBgy5=Q-e;@L>HZ9dveBGvyS+gJJK-- zVNbb(P?yB(q&-k|cXiWwPl(%c0hhk>_%MS7s_Vzihn_WWr{D@bLWdD2@ZOuHb2BA4 zfg{Cr2GMkd2sxXGUx)Y0-ux0A)m)c8+Wk2dn%y0`0a^j%cILGcW=T~t+t-6#Q43M= zJX%KOeYo2-GEUSbr#5j-?Jg>EJ+;`TsIO7uk^P!3F>2~Eh$d4z+Q+Uf9w>?H<+)bf zF8Ig@?uDU+?M;sZe14Z>3zYB=)kwYgCYIpJK#Sb}MAd3CVH(#*P|K@OF28;BkSX5F zedcboGr#`!yo@!Lu2F*p9U&gmAg``Zf1$$|v^Jg5^ zffkcp0<(wx?cdaCTMql|aCHiJv}5kI6yUJ>e&%e;>cZKd^g8dB~@)Ypc}jWEF2)Fc_`4L~Z*t_z*QsSPF%Ycrl&0 z-84#;Ot`JiO?T?fS#NgZVx`W>&z<~KIzy=u5hGjN99O^*_~V<1Kfb{t566*d2suaf znyap!zjQASPDP%!x~d=2T)A?)LrVmOhjRpNqiJ4imFF_X-Wr1UdJMnpZ50gtJao>?LZ`xt ztr_MB3M$?$m^74KF*?Sj(~;l}_KJ10(E|k}3ylqBvz`Yk-cs}nLQ6BxzyK7r^%Wb_ zipm)5_(JoSNR++!pUA!*a?xc_X?_uwO}`%Kr;t)d~NOyB@CliM!QS6Cu(bLZzRDcCFc zV6RBzys3$H{*o$eZR$6OI$+!IzK!g152tBX2U*nW2A#thvE&~%hbZ^_8T1+4*Yx-q z5IWOROQ@okF}1&mSw%jJxGZy#%t>i=`OB($#u{8~817=>nvx50AWRYQw3`p}r5Q*+ zJ+DS*Nz#ojf37~Uwo}(J6U1%yszTvNk@VI=;2JO7IT{7!euA!3Q!|9eJFAjCc+4eJ zuNR8dY@QAlO0PxB5ZbZqY0*Si2R*)P$Ns^{HTXCu&LVO@dTo?Q@T1j3*XG`lZRcTU zQvt5d?Q5XdDY)QBb*-xC=G(E#1C#Q`MjRnEf^Hh|SP#vgyx((v@Wv{kisro&)0k&3lT~n!;vgr(ZxQMD+Y&UmIP#l!`mUkNQaK3RC-p&xg4Mg?}uflnD{p%C2dl zYP@u`D#j1*B0CDPYv$Rz%)y66rZ%INPN@oc0nlNR)>B;lwC0_!@{D?br&gxp0rLoB+; zj=pQwt~Sp>tfmd1ZIof^Ls8Om__R8fBt5Q%FG`K=2Yn{37??k>E3xp+vgd5_rRiBZaf&|@&b`547@cTmy~2U!b&EWZe?1tc~wsw9Y(Bo#h!KU z*K+G>dLAxR<~~Oj{VNe)aRE6b&{GOLM{XMPR2eckM3iXHqZjChnsuaDLw_b{I2Jtj z8&`-e7h>*F;fSDY<`$)5A2b>cV0BeZxl3+}FiXC9STfa6>uwkCN^&1RtU5=nrg0_oXo)F*co=capvl-6LRYKMx-3!s;Nf1UFs ztJei9^iX>=-(f4r>Y2?-^0%Bjby5kj?u4`*vMNd1sO9`1sf4lo)$#r;YdA!P)O63d zgpcCj6ijlbehRlXsJUQKRXK9ZtmBUut{qm@zT)>{n;jSYRP&q{>-?oFY{>b2 z{s_n6w6~DY)Xs^ZR<_|ZUw->X5YB;Z9BxT>4xDYdFDZH9qHDj3fC7K2Q~?Ooy4LEQ zm$UCC!@Zd0%|2C(ubX{R8M)(fsWOU6HVdw;tLBxP8mGI?@>@osrF_CRP1#Aa8A?k7 zoT{$oM0YPQ-bg#_u0Yj2wfq7cQtL9&qS8>bpvdq3TD3|!BF7av<)!#sCbyMT6(=!@(*Nu1B0FM^)eaglGV+*As1jc`kgvBGf7hy-#EHSnzkR zDWi8|?>8Y!loCSLtIt&-j5|t1HijdoxqPVxRg|QLPav~Eu#L?4k9l&bEzCO)>qTcl zA}Dh|I*pYvdf9CD8AUR;O`?Wk_+;xN1SuA86PWb(NfVL_VF`xeaZEnE)x>-H}fZr*a( z3%lWyB2v9+rQ?l|EtW|7X!>BgXtxq%SJ+y6K3pNpY!!vGmA3@H4{YZ&k81$uveWKO0u3LyLmvf=^Dd0BO%0m4pw)3#c ztWTwB^Z-W}zHXH~eZ9MRE2gx4uiB)7OUt_Hxh+To%P2ZiMIRC|Vd_lwn=9Tkv$0m* zdu+cVX>}JBP8V(AS`lvrrIav1g(^a>Jm41==% z@c;~TMdXY+50ImD82?J1NyaO}phLnC5Al5ou?dJA-yo8M<&7A_in>l^B0t#e;Ls*t zrLu6%^M|hqsDlJjMb6H9{ds{x+Vf=9AbcrnS7Pw9Yz*IA2&d68338_{ccylPQVyDP z$e2JqP%3Z@9CB!baGCHg$O$}Vf#80yrbua2=4?h|br9_vJ>injUik;A;P!Ys*k`S8 zrJ)SHF$iBT;&03L*jDD>y@2(=sHg^nt-}unc4^rMxK&>-*58hnITwA9*-RkY%k3-; zK_i7k#=~zZyo^^^xR=%1;nwp+Sxrs=>jj(qm1UXc^t1@p)$RB)qEiN955) z2-=#nfRqcw=hpPR-SqU#yXNIp@pSPY!j)FdS_Ag3S# zIfVuKM?9@=RkEI0tEQDwff}+bbtWOQ2t71(vKQQDf?KTX2?UM;+E;T;1g_DsymoESE_jY;sJ>HSEUtQVYZkeXwkLeA%dWjL2|KUINTY5swmr=n z){sg9f9I;lG*qJpHYU1bebbrQihZZM3&LuF#+B0qP3CSzpS`s6{|pq*L{wY%=G;#h zSOAt$qdE?HL_k_KuQcYf{*d@M@-=lh_mmpvCT$gRaG&f$LYAIT`uG+eNK5EG-7R0v zv#)g%b)kxuSsOMZoN*bo>bF@MEth79dX{M&!N#i|c`;j@E2egu(Rfqg6$gdap61^6 z1NN(Giln-Lzj&i?;}voyl48&ifBpTET*Kj4ZBV#9JhZe4??hq{r{F&(c$!tMhIC4J z9~7_eX0kV6g-t88Yv}0QC6@bwMOTE5chzTIow(OH_j4Ht@opNHH~6@6Vr;tvIos4} zv)!PL&y{0iEi@#Trk<>t#_WK z=yptlI~SezSZnAKF}807f9bo-IBpx$+KBFWV#di|F1l@n_K@v13wJdLI|hJ08P&LL zmi*gQ~gg*!we2NXEqV0}w$mHr6`Bo{TnN=qOD+@8x7^ zekMO$QrtI4^7(iw*8JkOdFWXB`XhdABNmq7p&2-}d{2choM)&rtzq=9OLl`Q>%jg9pw^IR4Ulm4J*CaiyRdLtYG zqxzAC^@k-w8^htx-REdai(nrjvBs#IrWaX(3qTzaR3s-~&gfXnMo zE3&c#Z`nvqgsrF-J3&^^|C-0rgI1Rg9E3|cvJwr*7hOESXXqDl1;EIKVGwuA-_pXL zEr?%OybSsDw-$WbD=v^TMrT$Y-i9(Ysvu0l5TTpl#TGSj)l%$RP%ib^upz`BP8Rf? z^p&ARJ~qSGnd0wLX=%M-UcgV1E1wo@+D`d?#cfCS2vU0Fb6Yl0{&ix+;UZ?32Lpk# zyTkeX(hd)d!)faMB4`vpq|RfX1i*h{SAQdtBk)VLCz41y4QK-0Bpf&5&!4{}haiN4 z4)7nI1F|NOjD8-1^zn0AA_f3^?&zDqKdykQdk-&Ih@g&MNO*JMuOGraKG5Tl1Z98) z%rC)<{N>Qi;3+(Xbc0vG*=bJ<+ccX^`4SwbMbPALb_bzz zq+$9&o&SyB=mZ@=tybTP4DsOb9Rz2rJ97Ads$jkmQF&G2f|j6oK@6(#Fa2%^9*hW= zy?PmBxfmqiGx&7a(2xt#=@w#I#NNHR=QDa`p8EZb*rW#{fhjNJIQYY8Kp)hx@m_(K zJWn%#pHUPff*N`WvmL7RE?f>G{EYW;hRIOJsdYAtf)5!|2{)BCI z3Cg*se=k0H8W@FktyVf%efTh%kK!4?46aamd*XGwW7EWg#3vUb_mKQsqmTai{~!?V;getpf6u(2sYXSQLqh$oCHyr zNDOEKcKREPzm5MKx+Q!|{epNRn5!ECr|^7P1~_F&2pTi<={(`z>yMHx9v;MW)F&oD z_ea*dk{qWmtO;yzkOU&PFaN#r&R-e^azhd3>w92jJO0olyLUs3unt~8W17+~Fd>Tq z@lS?qh{r=Ua@@Ui1p_{n5pRYGQ6N6_jq#-z;qSg)&+~?N;9R#rgC^l1O?sNE0fWB} zjd8nwQS)zC2XH8?c+y^1Q`PH4*zldeVcPmUZcl>sD9wi$-e9GS_k(dU23DUQ7*jm45Oe7s{%rxZ^ zd5Z;2;z63!mZA?+HiH}tPHq{V|J@B=fHBp>OB$D}K+| zOA#D+hviFYJZO>~{BGA@pb4g|3ATaJW4=Goes=>i3MRm?pE1kl4<%%Y23eTBfk)0m z-_RI9zwfsjB!PE~oQt3X%bS7Ut>Qd$gQ(LDz{8?_$CnttyYXMc`>)~s2g6g!RdP9% zx%g^ZQ8;hkx8S1>jf|ybpxJMWC6WaDZ{uAJAx67^>^~xOk&nS)Ac0kA*p>e>9sa*e zheIzkq~@RvDzQs{)YLXH_p&`Wv(s&D+`H+mq3=qI+_3f4x%7pe7janibs{6glL;Qe zoxLZ>S#I{p^<(fZ3NX40A7gHGdu}Jq`u)QA$-Y0A;svyKJM@ZcKU8lJBBINW@Co&w ze|-vyq*CWC5kMb)p!g2HAUl%sS`2}2l2;9We!KY?ioMHw#(-w=xV(e#a>GWw)i|V` zuCrJszy9vNDDXA+{)q(=#k>a@0wv>~+iMU6e+8Pn?9}ioAXU`OrvNMyLBZ6~PV#-% zbt?l>V2^K>L9vqme$#i4$NAZ@TPkD}ARE{46cU7f$gM-S+?FzIrY0c%ZG#B=Cd9pU zPvNVtp!}N+QE7M#7j#+bghzo};{-3pWlUa%YypVJyf}~tpuiP$@OD4%#op-dzkl|# z3`px;Nn`L9+F9u~M^K-5`4p<*J4x-PUyh|9cUF{vu0^z^%W`(yRYW}bJ*c=5&-_I3 zNZB!ulnOX=nDwnA(fTRWLI)s0K63BFV?ZjgksR!sP?<#mCDKmT&-DfZ`IanVK!cD_4K9LrxFovIe+8 ztdd}q8sbdzt1tklUW`~+a0FlyYUq;ou&)}zXf5PAK(H6?v1a@gxiRWsq|}qZu6-N7 zGcJmt`xFfDBG#2GS}et_$OLX1ZCe@{B|!~X{)-rTCbku7SFZ9_wEy&6Cezrh9z z$nxR9Rbs}IXCSVIx84XGk@wtF)_554J2Z-q(A4T=of@HhtqqdQr)BN9vTxUiH``w! z5*+M^SSod*Q-++_IDq!isV`6oPu?T2H&jYiJcU|CDSM}1EQsW%V%j-T^l68^Pu}0j zO~ze+L!Z=MX5Fpt)v#Wpa{%D~O+~BaQgQP@cZ4xfgZ&AX0w!|K|MRUOq|lVYz%=+2 zqLFicIQ;3UfN+ZnP$sV`fXH%34rW9xDX+<0traz@V~6wIZLDF^4d%aLd5t3A0ia~Juytg%rDnDLN{SBMtjE(tVR5nzfjV5>C=>A2~)ZtzaTfeUR>rZXGHN0pm*% z1Kj7QIRTOU6dqXTv%n@ZXxI+@Z$|M`_?{bVHlv4xBB8Y)TqdNy9u&&^gq$*vnKNWp z?#e^C_@Gbn?GKxGvROkJ3U31K`e82~V{8IkCV+?cg(Q4j2R|F9W|37r)vwW#=;Dol2=e*inelNL3Bx7{mnhdn$=eBoXyvclL6GGov& zw$+0UEpL6n6BU&V!_q%LVr!wkMoTkP@+#QN8He%_-aVj3p*#q>ie1^h z7q<(J;cgN!sx6qeE*Rgh5f#t0dLO$$i;{?G+%2{XJxpw)%8;WfY{%cukxhl{)Q7H8 zgTQ25L*`CxBXWztj-Np`6D;A>y`?Z$5l7X}Z30~nI$UF(9s&=Ox`uTRNHstyr_R39O)-pTxygv!FR(LbzhXU) zbZYEHZ3-JA>L3`)8Q#5_D1vbAgTW2fUy93t^|FD`;#mx|A>V~KN~Zffo_NXkw=EUa zGJ0dxJCi2W71^qn$nS2UN4?CUVHa_E_e`Pp=PZR?k56z{G9*;#Vo&;?HAgWMrolaD zGn-%F?85ymxv_V+nb}cxoA%sCNBQ?dK&~5d_cChu-ocOiwDX}>Mz1R&nT>_{WOOi_ zgjj+N<{P}cc1as;rR^#luT1+X+vq^%C)-7D&3l(WDyK?cil}{5xW4u6QH#-6N0hwk zf_cxr(Q(Zc9_-ASu&^3&p4~}6{|Y~=u+V*Mc$k?%ALz5?6xjr~S^HV8JE|k< zY1GvQPBssFhxv3k*7Q_S8^$o0G?+aXSd!*VbA%TXxYP?CFsM*$;3RsSg^WCF*xn)^ zA(~e+?~$K#4AvPfdFU`8da=WSKN12u6teYgd@&@3TmmMX1lZ5nlR!cY;l>{3kKh!N zuRkKlSOcxA@O2q+25q`*LD#Q&iddZb`lTL$k4Fhz-2lDP%D)-Y*s9$@5)&5?(J^+n zF@Xg))LNrSQ(boNFG9t0paImf@(WQR2xO-h=oH^^AGeLL5c*}UQxR;;i;DAFbdp=` zH^SA=ck~>VoA-L!7G*Uy(g+sAkSuR$?G^_Jjv>HOE{xmKd!>khd`Vimf;2#A)vlE4ZgojEB9iEX(h-pPIn6OV9rQyEmdI(HG zmP_5+m0>io+LLQz!+=l4y19mpHph+?`~;o@ul5ZjJ&$>vGQP#*5%8qEPicS_ahl9i z1O*VCiZ2aMUHzReZfiVs!P9(?Fd>O3@6Q^)UR3Kx_a`dSZ5_9n(7W%Oko9RWV2j0t~Sx3Jk@ljv*cujgCoCXTU@Q`9a2%-SQqRL>MYcBUQr>}GS21~ z?4B3OJDtZU;B|4%M)*fHg*7&{J?u$bq8zb28P2^G_(%>;D!s|e?fY8h6C~SXjHI>j zZeV~7pZ^8oXin#?0QKoccL^}t$5VasZ_;+eZ|U9r{%$TzytQtm#ie?i@_51xNXfzH z4^HUxy!X}gsYD8S9pHZBc1KYCds-9t{+)93R5p6eW=?OcnA}eAex2>tHQVd3SWxGUW6v zjd)_8F1`NJ+O(Vd<_M$yY|gP4~qvPi|W)O1d?$<@O-Y#P6AqI=L!?4Xe@ zM!^Yzxh{090}l6G{i*qo|8|NJIkv-_UQ4J!s>wo-TpaKDM`U2qI=`H66hj8kIV2cV z1U9&k+hI_N%=KTLBT$7v5`Vo#7=CBNCocsDgyfdnjo#2s-gqR7A z4P-tMgz^b#{b*l&4hPoo9)Zh2;+x$GA03tQ?@HO=ryo|vJC&F0Q`?EvYR#?rE2)o+ zD)@0wJlNCTW+x<@)+u3#_lmeUEzZ<5Y(_kJU;bZ@&z4rjBYTLF&6%j#Ha!@2F zX@O*Mq2!EYP;yR-B!UW}3nZzafQSSMB3Xh0l7kW@N=A?@C^?IgGp|n!XW#8T_ncR+ z{(rxIon2+s-rKq67^9Cc`{=#3)}Ny5-zUS$Fs?_g67-~*Lh_p4e4o8nARRaNLTDQ1 zl1w^oyU>?_@2))(6mmjnBV@K80rrH4L~h+FnH45KaJ_kW`keyrJ6YHC!ae@pQRmi5os4EKgzYi@$;W+VuSFQsJ}3~;w=RU zaEN9Qhjr3r)d`7c$2r16iq1`B0e|%~vPTvt1WJ&D5ptRsIU%m{tp%GTM5FWi2S$A( zuPu7G3FSQ(Q-9)4pxc|XV2|Wx<*K=I2M%49Xli6HixgINk}8LCXf^1Mfvh!>m)qRA zxm!=Gf|!FPd;9u267O*Nh>FqMv(9W=?I9#oBzUY)yAp$rD`_a8$!(!xzDj8#YXnR%dP6<(29-dFf~o~twX5c>Dq!W=ZED{vr9n_G@nvq290#} zg-9A+-B`_DZb;-1DtF-u(!fbJI}!s9i2R3fax#D&$-ka$mnCL`vG?L8hhSGsCHK7b zTe!&sTh}tsttDG`@(bk5k&yLUnJ=?6#;@|HIJKY0cdfX5h`;*4x^je%Y9mscf883S0@7=IqU2^R4x2EX!W54$DHJV%>ssz>)xfb9B zOd6-}a%K`zR&n`pQy$>#3GV-ToR^8SSj`!##NhS6Ig6Cv{?0o%)VSF0tP?P3OQ~;P z(wv{cJuitp-uE^4-SBumXQMo3l!{(f=Dy6JSP2-m}=A2xf{*eLeY`eYqb=PkM$lJYnDgZKnnwO!9G7 z_*MV|GX3fMv`mr)Bh0(Azp{n@l>b>u+HfastE_^6F*{6PLH9C6zD196 zTOv;JFu=D6&*JoPU@5Cbf@3=Qp148Qs_OW!FHtQEJc?#ifU^#Pq;|FWz5{3u+@2UP~BiXkYBwn21I5WnLHBG>XR##r`{mlyJmwvNUx;6I>|yU{iH zqxAllpu#@|xdD<=?(zTYNXkLgYN%yMH@K86;EiP1O9k)<8F-|Teuo4gy;0%y0g?l>@l8p2x3?lS;MgV-&GY}Xw!q0Ran6b5w9ETCV1=s(FGv-AAeiYg?RnEz& z_PEvk4izEJPtN^Uk!1c5(7oMsoO>}<#e>VQvp}ip0o;b{rLjg%DqkS$h7XDOxzwYG z48CcQ3?%^qbYELa5u**$G4oY#M6Xu^Q??RP^Hh+LEc=$9>6THSjIKo^&TlA51C-qr z^Nx1zV>9l(0Xgp1-=J!&5R&H|Vx(Wc$SIJETnrI3LKP5*Xb~hrrTa`v2~-%Uf|`;i zqYRxs$W%cZu(QA)h~#2-Kw9iEhJOcW1+TBJyNj*^L7V5!jQ;1tw+3!%Kha5ELO34j zfG)WHc**0@M|fs(y5RaE6`H9Vl9@W7wzEh;}{vL5gX z#0al-EvDTS)C4xPb*{ZTSOul9$v`n7ErER${KUUNV}Gul+RIXd<^Y6tYg6vAwh7a< z^cC<%I90M|B;5)3yY=p|tiD2)fl_q&rD_uMYY1~^flNYDX1_>~2^N@U4?x^D!7#fv zdJ{=oQtI1f^IYxA%YtxrT~ti%@m}$J$oyZDMHPenDv|JY=BMCX&{2N?dIt;~09D|$ zdQ}?wEp2rf2$b93dRyJmT00Qzq!is^gdIQYsoqk!8DVl)rnlEXzwO6J=(SDOvF zULQs7QvR^+iDsxF(OY&;R3wd_49S2_?k@!r$bCexVhG|RM-f!RZJ@yqr!u{`3Oe6z zf|XyR+$#1Tx(|srv{47yc4EYzVhxC;p_^3*ZT|N5hzm05(0dK;3?^!`v) zU}p%zG(->_mm{C(@0>UZZln+#X{F|>t>EIEhb0_9W>Pb~f#E+BEFTdxQ3LdIvYn0Y zav!*d@)o4hoOL4zy>;Bi4##~lIT$GljNnn)6*44y5enauK`_%0B!I2jE*(>apE|CA zNW)3Mx|p`wE~yI;ksfw{NF)U4b=RjPQH{B;PKVmO8GzJK6gpuw$Y@3n;hiVmPQZ+j z1~?fyKl8hvgTzu?US|MKc?dutU-PEaXWpoxx0gEPGG8K0vpegJ##|hktxpO|#yBNn zSr@WSXv8bw^uVqxlI)c(1?jQS`ZT4={QG5Os=^;0AXsMW6Z<#o z5C8!~TR3h^`BeOi7CB?d2!_O~HY#SjSY)x&{j?dJEFsDu~!f-X}=cMxE)l zsYQ?j(vF(9a*`72?Q)yLAiokuKzj4mxvV+mvZ0z6Ep-f@3p1$$I!HyCx|tT~EOmbo zFqE(FzU=3m^)&egkz_0t)y_ct>#O@>guX$_crQ$tlRj* zsD^24$=+1V=K6h3*MO0Z-JB1>Sdv=40uKAU(^9d!NWPnoFW#KW&K?S$hX z#?E87^1?lDNrDc~uQ^dLN1{6v0rBX5$W<9XKbdDeXYK~*|D~H=xZhMz2#$I*oD_nqb8^h3a>m5TV!1z z);L0sHMFh|bHfn8A?(qng9)E@4W0;atwyl~3ty_%ysl5aDyR5~zi^?cp5^;BqG~%) zI*jg#x!4i)u!+kxiAvq0uuy{>Mm;VpG&!mA)iAZYvP3PFb-Jp|J7C$SO4H73XGpx& z7il2>tghItA~^M8?Lsp3{S`*3o@-S{JI3c~#U^uA6!|d|2=0nLC#9*D?W=kL5+Q-8 zs}P`}JRZ)w)tt3c-sz1`ieJE?Yfem`y}i7S9E51D{o!*+>B^<{4n71^a?Y7MQ^ZYn za?z<%Q~M}H>{OPrNN8%<$m_Z^3U<3(cJyNP&8|bq*+(bjE+0Wya#>T!QMIj{nKO0l z=*dOpM&TNxQw037oX0=*EQ`r4a;l0lV{KO==xPFey78p@+hEKBIi&*QU=J zet*5RJBk4Bgu}NJd*a`^D?K^NvQgV=TJvdfrnhn0BTBT2;;iCsIKSv}+7;v72O4rj zNpPCa^P;DaX!=MoTUpp^*X_*ZJw1+-jfr+8^HVv_f!&Ep+mnK7!ooF*)oJM**aiuc zy9E4lxmBcyAJvKfaY{M4;9?Cf9YlTC{8P7G{p2=tj11at%;9ln(*&%o4a~cThZXPc zY?5u7i6ziFsLB-wT~soWZ=CVnbxB`zJ`f%u{xr>De@+MzHVe-bq|7{XwKdEq88d2V znl0Jhtt%wD`ldDM=F%wSO5|sK;)9!AG^V2eCVyA?kVsX_HeRVlNVt7rtvgr%Vbg>-kRymJt zS)=&jqY)FZGNQgD>ZE`3W69dCx|^j{>$>B1;Pcec&jf^DIRa0mrc9$O~N5#oHjw z6yB5(>r<0Og{Xzo7GmRT-i~WN(NW6H`wZYFRzUcPhFDxhfTB^>$0Jw=vV`ad+WNT% ze5G*~rR5iMN16{~j9Q(=n|=MqNrntZ9~T5k^r0nnkF};Ahp;HW`_b{%l4qA?P>v*n z!sZ=#O;Q|2nQxPuqd2&fyC%ZQZ!H;pZ>7})C5^oF%Q)F~Ppw}wPg7nLEpRZ3+r9Yd7Di zeXshym`y@h|D?yo>kIars`utwP8-{Nd(6pg+G7h5#nKrx3t|C? z&OtrHv<>=n=cRj*Mz_pu%KE!qmgaFsNE-YGJ2JK(E%$lzZF_W*esm`4mls%QcP%@& z$7`Ngij)QqprLc~7W#F?Z555|+L_ltY+00t_OW?^mH3!Q$ zDHNhsxoZSj0}qwQXEVo5WsV9K0?(4wFDc2Q@ltz%Q%jgZ?Uq)&k^s-w$nT&K#X@TR z39omgK1@Gp-2>RvB0|e$YP17KT-uuh3mHio%J(&jmYa5AYbw8Mpk*Vc>n}IczwL!! z@B($#*sDRiVS!t(Qh29iHRT1+I|mIXMmdMs&h*ufG=~V|jL{}M7_Y=ydGife_XhK2 z23us(EGI_!C)`)llE%ym10{~p&pFM~-KT z?&gTyNi_~M@0}J&cchb28Z3(eYWWN$?U-iD`r?eVoSVITR;X2)bWN8=rw@X&&UPaR z*%52(&-;e^vO8=yF5dC0dUC7r?sKiO9l1BF1K_16rt@RPbWPO@`|M-?i5Zv=Ok3lO{+$A>=NvHXpZ-|xFtvL%=>zCo5$Y(>5Q9+v`=m1NpM|4ZwQ7 z*Ieb^63jHV7FiEB8?*8_*S$U)Oy|EmqB*SW<4(KEok@o()I@b0l|;*%EC`au3cq~r zp0v8Kk#8u_r%Uw44m2oI6H_=?XSeABm^i&n?Wh7m-M&Hi`t<6Fl*yvbRP$M~ zfU;naaq%8Lgrwoqh=OEDFt_ws ztL96XyXaKkhrpv7*|i?h>ZPyHMU$FL&aXuL1*kS*E9=$pJT0UK8J8Njjz)d}GTDLZ zi;WEw2P`SOX}CU-?QVBz;X=?My)MG`{*GOq{P5+R*X*J44>>li4HM>_*FKb!J(5FR zaZ`2e*L=(sTHoI?oSg5r); zXpUrLiG*IruswEHP~qbO(=NFS_(kqLPX^&2G8*~v~<3q?8NJW2*Bh#D`$-^@%pDmKnCy>E2X5EvCL z0Ca*|j6C^e{TKt`js?-A)A4q~BTp+rl^4*R)Z}S7sCq}miCjrt)@`bMD07w&B-$;y z_Kqm|5qQb1c?y~|cWlgNc9|X8b~7mmFr?S@LdKO0Mwi2tcRV`KVh?+{TDwp}u z2Y}=?X#wX?hXz!rZybc}^VB%;=k_sJYj4gEc6fHGRl8<`W^PE4PBX<%&9YdWWiX{S z;k^IG*2f=GGioeC!V#sz`fN77NgLnBf&htUIHKvXBf4A7DjFaD<^?7&hxJV8&32F1 zrj2bx`=NNIwVBWcWwgrTovAcjW>t`wA)*F}nHA@))bl}t#!R6Uobos0XCXGU%rcw} zr%8@Ve%dZsqG5Qc$eTZ!v5qowgLsHJ5)60A8*@V?dWlae4&Sb5kdDw&dVJ)}y1=%NyE@`1cQhT1|@V0)`QeU94 z&R68wJuO*-1$TwZJz&pWfhP_HmhT!~yX9q2so4tvp$@$LC+>-E{!+8)$eGK&Gy}RD z;oYHqT2Gv}c^K3Uv)>5A>b)7D?Y1^?2aWe^H6szt_QXv$K0Rb7_v*$XB%TGj3t}7T zmXC9`3iS?&o!7{J=fUxP*Q0l3`O~ecbbV9^Yr$++;DsHJSNZc@-Cxzalz(X16j?c@ zMep>LlQ~VRYc0jB6r`P;A#D&&mqDYiPza%br|pCYm)%-#7la6e40r{G_xs!P!j`5K zi|9y|mlUnhHrgC>f>W5hs<1lYpj(j>S9%j0EJL`jsz=U#eYlgPvmWgd3f;Hmrm@p6 zfN%o(Wk8wG^WmJLfj-TSgThaD5dpB6#={sn+M62BxHHp%B#GqHMZ!yuHyy@g?0=d| zbBY`rT2DDZ{w7+`g?I;n3N<^7vBs+wOvev=2_zcm;*9L5H- zJm9qXeC89xFvH=p0XJOZ-Z%Hv!lW*-ABg7&tj}ykw3iz2F)p9&bKrWr@ic3wUgWVX znbY!MKg60qa;KZ^OsG85w}C?`%2DH;9f{@Lg3G9oC#Okdcr)*uM9}*E+@uX9pKjNs zsgMgOE;2e~%uXcG#6^4F6y5CY0WB1xmDJq>wyqo`^20A0Y#x~JmSo!Q-P!w5e(K5e z(Ra>zz4hh`=r(_nzvIz$!3}+;AjW-i;pl<4oS{=54{wLLJzBo^*n@Ra8bvcF!imV7;lL?~r_;Wc$*^^G4RjbbcIk#fo6-VT^)^^f z)i2(B4w5~6kL))Pg@&b)YMvV54@c96zZG@+OxLGXgm|!Nxu>;-1gLk}PXUa4>!wsZ z&DqY*W~nac&nLz8%UsLs?U}YF@uZA{^Y7QRVs7C)rMhdhd4conr6b}&ZRw-O&Sy8!}r<@7uI?=@7aGZ&dO z+H+pa%Ho@GWeNu9e)k4V*n+&red-HQx8{xC#hzP5v}uEvSvs}?Df^Gbb0nm1fx2;E zQ)U%1gP_c}B)U3qe9SPkl+6QftL>{^bz)0l%RAp&4`SPHv%}x-1ZDJMzHIE5*q&N! zJLv8pMVDF@`u1|X($#eqCO`Qk_q;o;mxI)_x^I^f9bD@!(yxnarCQZ%L8PMQ-7tZq zzIh#48(mg89%Yt|7poWOIdf=8H1Z`HPt<6APt)iQxnDSIU-e5Ih}X8>;!682Lx3?| zWsD$6!<{2CigTrgVzy0w{kkhdrc@>FGw{8)_#DE1HfA+%dSp<)9 zi0|`FttiDk<&{5T#xn3NL90vW z`cb8VN3JD6E95V;%W~i>UU*St((9}rt60%sZ}1IL3L@H~XJTHK*%u7-p9V1acyYd4 zDJAQwxMOjS-2EJmfQ42-Eox>|;h5E}4^opGM8q-0E&P6p*1ihp|TXp*f z45;Sf!sl|Wr;=K%O7*ls3dingrO6JvTHWo^M9{Wl+)i&Nm6@w#o7bGX|MDm)|H>~# zB6YE6Ysc;{tY|hBBDAi@s*VU3uzH8$Wcj&k?wWXIFa7&273~*saj+S55lQy=TiqnNs299C)vkk=SCBke)An7kz!{kF~EliZ0Tkc%Mk zkYCLaY*>^`p8z?UJdsSsOn%>edXi&s;Mw%1jw=b68_8`KxNKn!#Q^`{YYUSr%w7I< z!OE!ldVEu(Wguy0n8A(uBHcQu(7kM%6lR6s2&Z+&mzNpuqR)H>X{4lI(nxfx-y!Ft zbdc9UJEmme{0FNL=Mv2wB*xxMMJGqr@X^QBKHuGv0vn1Q0dcAR_8bT7xph)$&Z({& znxFY)V;VAED(q(5%*|vb#Yo|vHwQOuurwzVrI{@eTl|*t&V;F(b3>xCQNvE;)OEVf z@CWqNsyrTrx>1xxU!Kv%)(%V`?DbVk-hJ6i|0KuA(O#{hfk%!T^$N*uy z65XQ>(fDS1WWvr8D6!_wR^0~|@9$bi4o_%u$K2nA9Xd#OgI0-}z}fM*5-y>$6wi2; zvW4IBY%*uhW21NXqfN|uBbI9~SCoB{qv`Y^z`P67t{I>09x=IYb9bN`mvF(j#~bT$ zkZk=fXU5?)$Q`AQ>7D5h?l8eIyn`t3ysJ(($#&55Ps*J0cg&)p+V-SqB0JR+VBun4 zqS|zixzAK!H~Z=$9(4z0@yjCxgu*X~ z7v~vbXvr8j@+h^+-v$4<+lb;a|5CYDfx-3JRr?HD=XhSSMDsMV@4!wWWci9qrge9a zsvyc}izA(ngRK4@LF+qV#k&ME*{`-uh?S&$=S3EC4v%~BC&>3(FqRc6RnP~0o};?? zNZ?AM@4~wZ?uk;b$RB z8rl3%QhnuWaVjra`8|Sa3Aq6O|JFNC(4kgF9cW&|CkH^)m7qQh%&byO{De+?~rg&OvFlhop?)idPo9 zJNe+$V)fwyjcWj;%wydHQWoHk*V8u5WR{Ygi#|8shH8+{ipDp6&5~<_ha&~bGIpuS zQST&8vJl){*Bxg^eh_Pl_eZ|l7(D>LU50GcS({5OvNTu8xj<{$B%ADB4Xc05^_*5` z!lMN*#9UDIin??~+s{TZC?X6t#U_10GE6%|*k>8b##G z+^u^xDF3Q9Qm6hc<{nF&cV^)=Nl69rQYSsi1056QZpAlkKJ9t?*?n`p@p0f`=J+fU zva#NL$b#baQ=@v0b<*}L$-vh8L-oFAzg-v3$p;RQ!n$tZCsmobzO1wW<-W;B*j}FN z*X86Ac$I?DJMItuY78;_k?*)+Clz5r^9WIEa;!-;03P1Gx%LSEx-S0y#=b;qETV3o zWhh$(lG|h>AvQ0g=L}B#7qzAj!fdrK>9#}3XEaVv17&!illta%HLm`tHVtrvm7|Z( zLfott0Y9*uJ`5h%0VJ*>mJJ^N|E+=wf5mNrT+{#axXnwCE08pJi{MQ9HK@yd=qll7 zn~hm3F&fyb1$!=oUD1$FBH9sP3_evvV!|ANT+eP*C%lo6NwDOv7Dn;@7|j&q;pcmvL2-Ca1Mnt>LNBWjK&`Q=)Vb%khkZ>EGC8n@H!Go3{5jhT zDoX-h$%GggoJcace*qPZ|EcG8#ClbkqM?t#LX_fPVLCsW!%`55@!UUG_EReg=U<^$ zcjUx{=9FWEFMI-+%g|Ix>@mHDHH5j$0s3K#)Y}tcyS}_-h@YM{;M|XtUf_d|E%Oj$ z4jqxKpE+&k6=e{`H8_t{rGUEtjS;ME1b^TS5p984NWXKG+IiQ^Tqm>kQ`3@RSd?J= z7pT`KBu{SVgRYx5$P9=f)i!(s5ljj~nIA%g|9MMKe*7huP-nP~L|J?e96QgzBtM*T z&$&xz>P1K;(C#9}MK&fAO(FMA2r_n#&r(gY12|Db#fv`f`ux7(#8j*HHTPW~Tp5EnN`e1uYD3B9FDIq4$h|NBlW*F5cnf zg`DoDKF4Kko?ULYDMj^AnyY3=s#cB&eRn-j)|zL*l{qv(jv)%Qplj-yuPV<&+-ns&zOszp5=#Rn5%mN|ZKx?TOPNeXK4njd+?lz`xeWR94 z1%=xe@LPGu5Z%8ah<5spY0mqdd5w>dp8<=*TnT~_bVi$f-LG_%5X6R#?aQQu_v4b0 zVHiX|!N{U%MAdyEq^Drs<$|U6E(M4uDZUbGJQW%s>VS_)%pE%NOM%E7B#CIENSGmD z<%2LsC#(6J(vjjQDA8W1AJB^M3?FByBy>XXj26Mn03eCzeqrg{)eo1I6uGUQ*(86P z_t*80=EqH4_>$qjaDWs~#nY?v@@^xkMI``G$&oU$pGYHvZ2%OEPPDr&Y-z_OI zSkp5PICppY6)iR;Kbae<4-A3gsB&Xr4EJWANQ26Su~1On7kZ>OS52n+=q@cF5gy_V zO9Or9L60v2`d#`TK)*JrgabDkIZdHFMPPRA;Lb{6A?7+@{2voHjOHgLg=DL0^Ja|Z^I9!KARpEbLLheL z3)cd3awL19o+AZlh=EJ4qtCUa#td+>vkt0YxV|3sM%m;=9b@2BfW;%P3&O&QwC_ zT+(qkq^^N_XU)<5B_W_9a4vv#oVPiT@iQDslQOl!hz6uYqbx~7t@JsKgU`4^*Dc5i z5Mj3mzcKHB3cLA~wLxr+3K6d_N*j|C+xWoH$~|X|g$aLvfb9@T-Jw8_E%PM?vUDH` zP9vcYQ-Fx5GxcM_4et(zZNmW&D6s{O#?&>ycqG$d*gwllqGs;*ed(H_if^)FIDCz>G%9mqNF?l7It~Y|#k9QVW zAMR|_ARaUcaP&i*@9Y2%XbL!-Wscv?h{vhJemZfEmKyXQHBPmGI17%iaQMaMQ`bP~ zG?dsy$WbwS zS2%n43FzR9{O;nnU1+|D6n!J;-3-4woNvReZQFuq2NkNvEt>z zquB&Y3MeTC8drN1sM`G{DEfnTQU$67cDE;Df~4Z+36%2Ge8js}!;c*Btf)Y=*&<$A zF#LY$BY0^Q3$7!)bODzIdFdH3)=-eqV*!#?Z}qigs$V_zfeb_Mx8@6sKJc2!0(dE! zC;adMuZjxfBU8kN5&!+tzi2s5nJUY&Vv^asYY3 zWaiBNc)QUF`pmtbeA(?QH2+lO#enZ=#&pP!e1qulmP+ZRRyFAZIB4SNp0Y6&BHa%| zM+RE5PUv_Zjo7yzCl{9-lgkt>Es&{(WI=AB2W0bP<8F8)@c=dR0opvlg2 zcoG+%nBiMEj$cms{RJyr{857EnxO%#nh{3{VsY_b5TF{`NwbN5Ja&y@zf7-Y@X|?K zd(<^&b@W9FOQ}lM{Wxs!>rkdy3D3RP87PaZJLgM_5P8uMq_a2Ry{xpmxW*(Nrzfti z745l>v?uOap?Q!TqpjTKNK)igF`Z!dguThRIm zQi-D})64L)SLu{$p&66{fj@d*s^S_tVJ!Fcbdo0vN?jV!&cCNg?>elg!Za)9|Y> zMwkaLQRtSaZg1#5uY1txeZyz=;}nvpg$}UT5-I$ca}JmhPCS9f`@FF5EH+H#w0K-N zw5)AZz`Qq;xEnBp4GYX=9Nc5@#DRyIxcHxBVWC8&ilBa&BuhtJy@2}Dc^uqv7{a)& zpzl97$dBdm)9&zd6>_!E=!Y}bKWFZT{r+p;f7<;=e_>o-q``K+MUPp2^lxuf{@w0t zL|#|>%iQzlH$ArcRN;{}?TYPg0ih_eA04;}LlJZhAt2Ua_#$(Ce+~&QzVzXu-P7jT z_d`lI1#n2nn&EMyE>;uZeGI2Mff z#1F}@?aI`evHchggMqj@CEb%{eB#?m@PxU$yE3RL?O!$iJZq=&V>lndVAHe8EaE0! zo>Vml-^&s+5VL*R@qbiv>4%NPi#!uJgc)W?{%$z@bePDD>uB$PnCz#a{?Y8;f71n1 z*>yfj^0z5}{mVZ_ySHxsV(}|5$Scp4PyJzQeAwaMhb%;JaOuc;3g%`!-9FO`z0ZV& z*QNxf=HQw6xk^dE)DNjSBs?p0ag4}YgKu)}$C<+UTE29DR_;gpOoveoq?fbF_TzNm zmY`JjwpugpCy!>FTzD3-1M#Uil>ftbq`3IcnPE0KbSFp|o^XBHQB-~J{?kqgU?*M* zArjczU~7JbLnQwv9r?!s{c-5sfF*b1#(^K(>M!QOC51ic`?AOPk6AEz1AFqq8#uK8 zZc<+`NVUeS^e^N8Hf?#1xN2-Qw<$WO1S1rWy9Y~B zO2L`w$4lB_wrqTR9_;ga3ARqTA^2#Ymn_&?)>QZDj~+gRRZ}BvN`iyiLi{b^w$H1x z1jLgJUv_4m1k?UJH29%0z0)hF#7$nontb{E7^8Sx5HzE%qxJJhFoR}z)phRg^EwL? z785_Mgo7)HU$iUu!b0xn(uV^!EbO5hoNw%~!Prx(@QJONzU*8f57YeF3^g>PL{9nV z%1VVJL327v`49UL!?Eh`TY36VlRgELD!AqTIX%C7`qw!_W+d(seAHe#p#T2=J;fiv zLSi}W|MywL#n*sk%D&wE=dtq-oBnOCy%b=KNoBo%c9<7B9D(Bab0?~?#i7zZ3Pk;} zo*3ZFr6lu~`CAjve|A`cPd6-oo@{?TBM|7(J3vBAYCdYEnNb4@_l>n zv(LTbjB)ONcieFpW3k>f-*?9Q%sHRDsw|6z@e~6A0RcCm$i3kDZr-Y4!H^*~U(a@K^7|sx++8@8yk1X)C4Gno- zb9vvjw6-AP^xl-fLw0hG@3h=6ke*;8Sg`cy`9G;d__>Pkea8pSh6+I{Y}wT{S1Q;_ zvWz(lQKD5o*e%Js2+4}^@CRZ>J~CfVDxzwhYtsZ70?+2@3V;O7y*FkBi0nCZ0D?B0pLyBKVK@LK)N(L0L* zQQP6fYSWw7Vm%f9j;{!M`f$P{j?F)6U6t9Zk)WD28jLf3lu*X&d5FVK@I?Y&i!Y-9 zA78V5{K}p&qhKOuRUY*ZlfUTuOhCIE}egqL4q% zVarHn*m`OoZCSI`(fErJD8^C<`e+$fd!pQ$;#~&4e_K*Mt$K+DK@cR zJW1>j9_^A$zpFEoCGfWP5`8jj6uOM}@aI6N$j6`yQvaGr`9VWF!xlqv=1mL~4Gcom zI8ik&Ev7+SA}-zEdz2989uZ>tw;81wqzI8-F-HeG4u3uiBY$nW$ch~=XT24~97$1i-5%*XTz^_SpGiroH4_X$e~$?_u$4~CLNf3Dy!np%>qIJTF} z<4o@S`f=GW*gwvEeSAVYABCv+Y_flt(<(Zmx>}dz!BAiRtYR4fbAL&de%TYT%~D;| z>6n0?2!Ur--$$KMWxo#yVCTFU>mgp<`JLr08i>&mDcP|grjGxlQcLUG4TmWY(<(n_qjRL3#*M>Yh=_}5qrn?^zxL{g zzBi#UUqT(-hSi3Di)0I*S9(Fpo<8($*n@u8;Jyx4UcZ~O7K%eKR|!P6M1B<4rUaSR&;>#5qrTGU$Xr|ww9 zSg{XH%u{3vWZL=&V|3Cjlm{u&8FQ+Td&~KY} z-;1BGo=2IFs`IkRnF*|roT&baKJ{Z~cxJA|O?{K|S)RefoI7~{c>w7RPn$;SC%e+2 zuOm~elS_FEg$sE_d7HVZANi##)eqhohl>?D<&AwEn=vWrFu8ksTVzVtt=o;zebaqFlGpdR54~?Py5VWV)AMv#`kRbbiqq+tJX?HI{8(HgyhfJ3-)e`V9f=@x zTn3h&!@DSwah+ys+)N}zB%&LA8@L;;$E+I;8v#W1L<2-_tlKR62I?$nDRU{MECwuo zWy57M21*Sj)y?Kg^-z7x@)_Nnx~=fcj7rt)Cc{`%SazO(FSBW&Z22}a3F zynY`=C&k7BLvJimzjaQ&{H40(Jb>$Q;z}Us=xpyrEpRCikyVoED5R04H!3~KJBn>| zV)W5yqnW3%^{FW@wRz7Gu9d96$Eo{>*V2ZsSN`aiFY#`&d~yD(8{hlfO5IyFl-;o1 za@LoB-OhJ^mHb})bv1>4$*p!1X^d!$-FVqpy;8rD&58Ly@!)isKdpIKV8d)fkRn8- zYK$ePTZ|&0%^x%1{Q7olb2oLfVd&tJVL>fKZ9zu#tl{_OmDAw?8U)$mq5b1uXgt_! z54BOmFlrwCM0LRS?W?g`uknN%qqIC!2*rNbVRc<&RTUQwq545pK;@7tlsk~SBSG;- z@6Gy~sW+!@{NAcyij%*5_byVG+$GVx>xWB;plJMZJU6w9v|1is-n6u6r02U78U8pK zS=m@)!7Jlvk7$8HDxw}&Go8NWICxA3ffqwI2?epqlRF+$i>{M5@Q9Nh^oSO`6LFrJfv-9=ik*nu_2y}nULO!d6FNU6utMP&0M0tjC@jX z+Rc7omUPv=U+IZgVC&b5KZmk(*2>?N1>VbjQ5;!JzDy$>D*t%~b*Y)QI{iJ{%G;Xy zhw_#wF!i%ZmBeqB1#v&=vtzQm%w$2Mt{0|FRR%po<@jx*F<+LFuEp5-oMerI0&N40 zg7sfQUnc6xX3~rCes%wu6tRaVD;JS(o^X&qmFSzVHnGPg#iu;2`w3Hg+hE=EC}#~j z3MrB)N=#8X%^wnZMsS!kUHn*sSSdi8O#9CekN;CA{UP_`Ew@7P%+B;`OFAxYJLnM0 zklwMC$LQmnHn+!?%FCH68_lCeoadE3#mDKJ8N)fEcZVuJ#HW(1lU^w}vi&xwvY~2M zt*vs1zy4Sk)t+*sz@F5UlAIb~JZ#|qV>4b{=g(pXji20tdSwA=DsE~->XI&bqyHwx z9$_ytc_mp}O(3DU)>cV}wu){*`(mE|Z?I1-#MGxc z+|@|7=9yb{jcQCSPffK%d$E?kv|HmfYU$r^Z>6_TF)w%9OA)$hu4#i*`X59Eq+L7i z#?7y|zV`WWrn6%*Ly=FxQZBn)G=0v4Mk@4ll5`=1IzvDGCLNpHRj_uo>hj0nb6?pM z@&dA~pxQg#!|LVLgw-W_S7i~$v~kySi}?Y&qMs^tS+PO^`--z^iB@`6aaqUWUKgzg zmxY+am{`PZY|{jvGQ9j2%u>TiJPy&zcq&#Y%2|C4Xk7uXt-H>K5a+_wy9C zv&Wy{wAd_o2JL6PE#GW7cIs%?a4$O8-}QUrS8!gppDNA~sCwFRbKvpJ_?ayETM_$! zq>Eqs=F6+8Di$hYUZX+se)_k=yLzW?4V&RDUCnGAB*!0*3)U=6yap)dgUW&|&$nm9 zLvQW(Hicss0@TH7y)VY2Nj>E0rQxb(nj_&ozH&!& zW}lyo;IB8mvZ6D!a%ai>i0CY~zju&alS*oq+dlR(LF+xrZ4ATp!%2atj2Bhpe=4yvStzyz+*5Reg{BA|dPMDP(qB>mU53?c)y` z5Rm`zj57GX|9cNU_jUgMevt4M0Tukh10U}XNdJ5qdHBPFe_kVvgL?=sH6-Qaz_*6E ztEHu*o3)dBI6I3KC_s09rRRo#Kty~0L6lRc`3>5ivC-6Z*Huy!GIw%dGqrFsvt;vj zaK3K`LBv}KTsl~~n^Jl^*gLujd5cp2^@I?(zAt8{ru^#>cRNvPT_sgYNheoJN|S18Y+l@KPOet$oPvUa>>OO|TwJW+ z305~BM|V?iR!6tze>d{acBCxb%w27q-EEv4Dev1gHFNTC7p11Y@91BDe~;7B+vY!d za&-I0Yk?PJzrVxI$;QF{ueL!|k^531RU2hsa;g{~ve$qsM=% zsrw%_Ik~t5{=MqI-TL3FYPnguN;)}!KHbIsW4``T`QLB;qoN4={j2|*DE^l7U!_3M zVi+Rq|C%&0j631z4ls{oHc~2@;2W&6`#(f$@Q>l|@B3?bGy?Peb1x9&q+V)zBkpFQ zFKSI*_3QLKqJqDB^@T;kNHU@L9ao~yyGCdNbEM$_A@vi9L6IbeYS6f2#A(yNGBc$lS_E|HVCIP!$DIh5n!W`&V6L z?$Q6^4%+|J{A-^6M?(K!t%Trabm)wcSJt6O9PaBEWoWgIOYCOzh7)=#2l2~v!MIemSBzf#jUaZidq5D^9Eyu8JD%GdKb8xGS6jp;2 z?RSe2bk3LSsTsF7*YwTUBspS8VQKhJJP%6SWfZ z*w*x%x?28p5J!X76Y)9v3HPwjP8mLZVpRbK@-`a`i$ZL%*81y545K`GGy)aKbeSIX z+b1NhkkS$|F2Sb^);Xy@2hLb@` zsz^J_FjR|c?2*u*mU#;PFOlIr*rp{yuOTTVKk~WnR1i9&HK<~c!8-|*G+n=eq-0>p zK7EG}qPO=DRP`*gDdqqzbwy94Kj>sBf1WWMNCl&HTdWci2gQH<1=1Bk%L8N$uH`Zu z%&BDS>e2|HT6Y9QTX?!w69yVGm9ZHKhNMi#D?vMOh7$xGb+d9naiAEP5*=v6BG#Zu z`#CIGA=b5+`ZYwm<%bkU=`%ZhCPmN?M_%p53l4>yFLZE|`)>HlM!|?##3^Fw5~*3av!XF+F54knxsGoL^qE`59UdCQr#3Z6!xTm7d1prUzRWJM&iC3Q;$GU ze0k~2lLp!Va$nbvfoQV{4i-e-Pw!j&|Np0#Rdf6sm66wyN1;haAoSw*59s&L!5&qw z>8O4a2j|6!%|}1w9TfTVySvb!!7=C=gxM9TUBHH|`4{!^m*;V?5&jjCqpM;Zqyn5- z>%5p@_?Va-bS&FJzusO?UhRl3^YePDiR0DzfzyGkEV+-xN8h|VojCDZZ#llw==}}_ z_O{xaprS4l|HClG`UAVlwtB(SZyF<&9k;d}9jw(oc+ityUuox5l{XIBuSa*AHnR(j zT0Eh{FwS+&_;+04WUlX?2z!?KY@}I5v#+H(z)7%3I6fP?4$yA=?#5P%=Q~~-~Lv-+jr8YcFw-k;89M)2o#u2Z6w@dCE%PFpH7j38#>If zQwCRd3Hy?4h_ue+ket$THg8A!&8HorZKIVG{-PK8H%BRn=~u$gRIkXjc6Z7fmTP=~g;*bc*LplEES-ArH7}g?eC6(TCPgpsa-I8B z(`^-q4Zc_2w2?k#U%2nFYP#W*IpZOY^_(98n|pNUi%JQJ=DTpcM-Ow~(tS5gF)YU) zdQS5?T<;ABre;{U9^wnK3PwbcVcSSy?HJm*?%lTYW$qRKV5|bG@Q9T815u ziFX{La36P~Qt%9D#9-$H-8P*@YIGQbaoh6XMhP8qaKAkkA}TV=m-%km=ksg}oiK6m zH%HXKi*JL`Pb1xw(!09L&>xF^O9RrWBSipX{7Zj)?*2<}y{ZYV(*I?NW$M7#(uOi)-7P@VYLKBtvsX|b?3=?XIvEljAno^!Jc`eDtnB>p`?mVeeH1IB>^>e(d zrYTBq#!m02TXbutVNd3m!>#&rml;g@hbf=kG>cOhq4t*^sSMzMa4+0SJXqeT>{zz= zEE`LT@%mwQrdz031eQ^S`P?H?nfglWY&c#fb!=cFzSRe%OP%{N)8 z#~l`7A7o4u-fzq!tcn%h7Z@{{o48($6K_p5_8nNTOf}lQU2gNUdT97FO{$qT)o{Y^ zXn?+LCGdJ=QgaM=4io$5qLW-cS(MZ+Bj>Tlf2NK^BHRu-?%FqRb1s;@p-1!{z`**i zZL_jC8;8d!RQur$cgeXJhr7uQ8YAC}kjIl6CfyyckEeWZVITY=cb@o;t14_;tH}yR zIYmu+=Ziahi#NPhPWzH+M&VYwl zVo2duQj>A`Gf!Qpg&R8bwj^sATl~-afla;ePWD z+rgOl-7wkEvU5n9$V5%A!1}!|cQCreNS~9&KYaq^)yPChhxss?mMszAUA0TvJ9pEE;aEa8|I-AM91)#)Rfu8jVe1Au06^}oh zViI^scfEwF6W1{wypa6G3mN{~v{Xv>d+TE>{h|Bu)c#+MCrX<`7-+CE20BE=Z1ZQr zo+npldpm-oFFO^BA_`TP-6xfnJr^9TiZn|ynfCefe$}&*GcA*z7gxtWG`DUPWvb-w! z%@?BQF2S+s@#DB$6?ieu5{s2!)r>tl>x&1=;*pGWwQ$!@*PhR{Z#fac3YLt4oDthq z+;r%Ev%~BxGhW+SVFN96b;;3ZSx8rNkBPwNK8t~nWCR(mmM7!$;Nhd+U^PcUFI7ZO zH8V`^n3HgZ)2Rkq9^Cuv2k7dnp+MZ9cNk$V9gisz)z7$S;RN~vW26e5yNN1dhn`Cw zGYQ6#ez{YLn)_d6DMq5BQ`p9|QrpXH(2t*RLoJT1oERvrbuX7Z7e<6{FIF8|u>#VV zc>{oo%y!h(#|XH++-zXhut6y{Zu9jyU-B|3o|@tbBH2uTs`{ZCSixSu;}L$$HCtbj z&N9Y0+rUA3n34C7u-qAgS>9h~0asf^_0fKfuJt)CeNS>d_JF<5Cf_Eo27yM0=Uhgc z+aAC45m2;Yu8!%YVmY(P>FpdP8nDGqdR=T91S{xv%Ul%8lS9lnqYX@cZc=ba-bK^uyF%3kF73UVmPRqG0u4MB*LmU zX{OuD86&t9(zKq!tZO{^D1-yP4(X8);OyejU4PAn*J;5^J_hz0nuF|hK66&x6pWg8rq?B@zn zjhh*Q6|!+i?W=-WpJVg;q&MZ8ulC2C6X{!1`Tfbc`%@%LPcaWE2VbRj7iah< z7xl>A7kXIR^?u7pg@aLEv)E}>QozyI=)eLU2Mg2!_k8EK3!^7{ zh+cL6L_B*+h#p%lb4BCF8wn4v0XGLH>^p!GHu9W<_AkZ)FQFf(?ms?^1t#(zVBvIZ-90dn5+b&2lfXiT$2-5KoLY zr&6rgm352JV0WoNgw5|T2oc$ww}UJzY|HSghq%o|KHW1`84iu$(JbKwH`!&mPJ({B zyUWd-hU}+!Z)na$!O~x*2>OG|BMdKRRH4uDugR+M{z4V*O>F4=a`ak?)5pz4JG9sM zI$ff8C-7)(V0!dE;OS$WAl}A(F2i&9c+{Py|E+MG``CbT$;=WqbR<1FpmD$o8BQng zbo)Vwo+Osnu5c8D=$fkQGZzT^lM!R6s{t$k3L?LK)Xscuut@Ph1F-i0IYXgoY)nEwLaZ)=9>}R2PbQokB*{e)UAXoCE4QKKD@U)DI#AgPjx}Az<<)bZx+Pb+Bf-cQx z>+!kCt2_bUjGpfY^gr{m-vF8L&+Aym=jj!nW(hkr?UxNR6`IbBo;8bw?lreFWUEz2 zWtVl_UZwO7r*Vfa4cPBwfAS60IN?TpK#B%MB*ChWfJ{RkkRR_;!+QN+b5ybb*V587 zPjwi$44{CT54)2EL6g~J8&To#y`d}iNTeTd2gEInF=R?@P8eZ;t^&+}f z&(hS&iNLrlpzXZA9%xIK zX}HwK4@mh=^>a|qj@E5MCR!&LWEyUzVo=mN%hRjn&~~+5YP`}Ouy<>ZI5b6Ns#IM} z9(Z9&S|r6n@Xkolsr?daW&HKvh)aEL_Y6bp#rU1Xn~%ce{GGeNlEF8lk?EAGJsFi} z+M`zMfxXkK$7jVO=j#m%3uArO{)1+nZP^%q^iCFv zEO!b?4u$gzx^gcxs2nNhF6-OZ*DGve2l5F&b$0HWp(F1pRh{l54f5dm`Zx&D_W+=E z7c#1~x7`Lrd3NZsb_!5s(4~6md?IY7K0yL}-zOI(_q;K@)6D$DZ9iKDRbuq6*iLMS zP>5V83~(%9Y%sx2#7HRLO8a}zj=bt9n_9xY?;rQksV({HdXQUmGP+ef?kyO~4rJt1 zJw*vqu(HYpzN^HW?I01ykhC89EGJ(S6j-C-wZ6SnNFy>w#6~Aguu4?&7f)gz>q>#3}F})`W3S7N1Kqq{6vJ115##E(xjV7NEuPJ1`rlAfY`nYSV z_EZ9(lQ_92d{JGlh7gHmAr_K!6yyZ!A)#kD-A2YUH*vs=7vCAO| zo5|XNm=$GPu|wA*J`UCfa@htYD#pgXsw1iL13Nuz^sr)V+oGVm+gv39s-yfwO##rJCj2~yu!(cVP(<}^>DSZao5nwHN_u$ITqAH<_unDr` zZ?^q7YdL3czo3Rc&H3q=oY8){L3(hu;B@|2=ojUVWp22aPjCRRjP>%p=o4bS7hlD5 zs%EI8Sj>(a!viB^mB$ljWM(Ri_6QitC6f1Wb?)a)u+ts@IrV!?gVerBuFU@73|0z$n)G zs+lPKE8d$4r55cR9u&j=z8KQYd#|~|@2oAURe2W!16p8wUGdl_o=Di~FasSwW z)yeYNJ%)F>3ft~T-MJW8yrZ_ujg0!t){fer=Itll*y7&4>k1ggmEkgNGzg3~FA)P$ z=%bOmHvLdUCoL8}ybNsxIEa`=(YBt)lvW#vAUG ztLe?#38j+cY;#~YXS`|_>sR7vSx+(8y)3pF=maw8@H$wY7DNAeS>acnxs_1JV|)54 z;zF5chL@=lzH1YC->8l%2lMH?m&cbpUY2Ag*nox(bMW?Z=}flY{#F36_NH2!i{X$I ze5Gw{09psW!=D09x)c_19}xhn%U-EkC=h_vA%&&W+eZz%5tl$2_YBb-$IOF;Z`}Ky zeM@RS5z3bd26absRyiZ7BMTs_QzmMa?Nc~HufjJ~Ng{rP!ARc*@*MZ>T^sNd=9(=O zDw5d2g3W|%5Q)J!OeL&6=(D{)IvfD-a~Ny~N|@z)o?)3qWH<d%;z;%hl9ZHSb|5USFWo#C-+43oXvN*XswZ&E7j6$4RsuVrTnmv0boHr*u5!?vQUJe%=9}{Uq=Cu3 zn($ygpaCimOzvF>CR^1^Tp`(8+9hn75}N`&+H#tLF}yi=qISWzx=);{O#im&pJQ|? zMm_w*&RkX{VQtvI+rD`f4M~YoygCP7kPYoTP!qf-b^YXi-jV#DP<(5<>~xz!Oe;B5 zG8%Yu#?!-$P%ji28q>aq$0y0Q2h?R{42t~@IO*XAo3 zSLBI}+~cu|C{+&g>8mxZ(U$m!{5BRf|lf_1Q3Ml-ExO(EOr>VQMZr>%4 zzv?OpZv}0N6v@6J|H^I!Y3FfJC*Q%F&@Nq?X5W^Kx z+h-(9@1Q@2;Xpgp{iUiI1K-;lLDMYljb1LCjAyW#hLd-qeJT! z91#(h*pu!JLd6G&_IBOD_B#mlao4LLFl<8)+@q7KqRpeWq5105(gk?JLUGfy}7f# zn`6I&HY>p-d=)M1pI7_O*%!?;3{e37-O+yjHo%J2(|&|3hkX7CzBNgwqKiLpyt0Pv_UdNNQgqEChDU;O!-$UdTMpWx z&-cmGc>_3GT8}3t#Bl<0C%Wx8c0pmSPIvv^BRBSUCid=zH%XEm|LcK$VVGEvrCS|#shH;MRWag^- z9>_4ym7uPGeDF+$h)<0dFe!f98E4pJ0vv81t;6J&4r*V#6Hz}2I~x@~sI4A43DYsA zzXw-GO#XIExV3)nKkGHTu85UK;l4o|FVSLN82cU|XfK?~ANSxbWaAajvR@rC`~kMe zxZlm`Ohzlgss^(;2u^d)(x*H!_E`7ZUsHH|QyUp%suamOX?tnup@K%|)7)aNZ+ZnX zNf_j342m(^Yz>=(9Tu;wF!+PFBSm8H0&N>v7H(S(QM54l-M%X z|B_>9szIh@<~eL#2!HkfjYgisO{QqwsAKWh+hT;pqd(2S9=lcWq%B$bDE%_M+sJS_ zGxpo7+wE2o+iBd^9;UdF-)mJ!x462)E?C22u56S$`j!YQ3{srBu)?2qEVw?2YQKG% za*U&9vk;@*uGz^F<3HGTrllNm#5-fSx}{(Wv&vkhjnfP`Y?(hdPDZ!`5YbvIxx&EC z_E?|_Pld`Yw=X!7$8k38j{@vO)^b)ZpAom>rZ^Kv4hVa*k$ZP3xN6lKTlY`jvmJQ&3|nowcx?B-=+)F<=?CEQ zLgE*OIDzt#3}%~7x-9G(4a;>-oe~JqiQ!k0EkYeIKUbC9ROOp>6%R(vNJPA=_$|KR zkUo@|SqY?79B^Nmaero2^c`F0ic?$%Y@Wj2s702L-OcC3?K0=cMj6dAaALbj(O1_E zbI=T^kvc3Bzz4cxzCPPbeR?orzxM0&mN6?#XpKR2wukt)05RCflFC@*)k`Vkz(}W? zt)iTV@L7kYRZZiuaCzUtcpZ2)p@N(tewa;@OAko@mGZ2CRDU&ce;HAc(DYafU%b#RgYdboy)xkQ z6M7G(hDl{%Ufhwz6GqW- zL3Y8@u?T}29St?3R=H@t6T+PX3?;&;XK5x(ZcYSRnV*~k_GZ?t5U6qY)`UmO8Tl>VlJ-|$U z4mnH;J#nW{Z)-x@Wp*k@;jUxE=9=>W{GJ_iI>xtoXFFowE1X6y&KXkl|>Q>e3eA zC$(f+VrMP&nwsLW zHzKKvUDCcEN}){aRPz2bM4})M!PmY6ka$=`=hH{Ed*m#HFAQCDX=4Yvhm;ButfH@4 z{st3#a&4r|!@9=DaQ5^#ay!(4z<_U<15h?}E@zJ)+uqAR^snWxVe z!F3X1d2ugp0PYx7U_zgD|&r{W!cW@E8H@XIkhpDkP4Vv?a_C2=%s zON9}x_o_uW!kuBnnUF&7hUOwhwtA7#kN_9H4L_CDHn9@BE*H1ecaV*D`Pa`nS(a86 z4k7_fMw{jnS`8IU8X=yE`Z<8uo(%SAC8=;|SR;y3GL9CL$A%2_e))RH(W!_nj@K5# zuC(Ru;-;yblf|&M?P>CvuuNaWZE*+W@@q73X_<=4jMIklVhCzE<){xOnl>_C$RM`Q@}$$d7UPDs*%fZ zqFNfJ9#RY#AJc`0tNo~63w!qv1Gz#XNRKSp!xWA<6uNv^4+4y=B3V-co;8wm3K>U3 zrZcDS&a4M(j(rD-`PG_ss9-jO^;aa(O(nf?wd}v?)DHx zT+Yc&YH7(tJ#I=mXy4dB#v3MhoTre=_M|1mp@i++{-d9jr}`hgx>5>sL)VbBGrKF_ zCPs(zD_cYS`*l4iI+URGxRXV1HLOmH`ui;-Zg-jd>jzWR_l#IQFQ7y`C_Y(e29c2o1xb{(7?T0VGty?J5KSvQU(LxsFJ!EY6)W~C{`b2cdD++w~-MF9AqrAcx*SX#8Htc{2j@Mc@(A)nNfJ; zYyDU`*tLJ@fhWv?O{LnVWu~<~cDqUn7kTnNlnekum~^ z!HD8ljJ6tXh8JMj`Ak~U)9>529`;QuNMkQ|rbfx;ZzXo-hg>#@CuXtolSJ;fjzan;yq%3JX4ah35l9Oe{@XeFYdyDq@%U4V@23 zXYC}YZ$ME0jETppK@YoAk*`tLLcdL0?i?{f;pO3gk4;mU{-US=17FEXGj>_cAX9yI zm3WDPPo{SQSatKNiqgqn6|QS0Q%wSG;y_-%W*~sw8{5z#UfQ0V-(eb<^1*T%7bSwi z&AzWOK0ol*UU30&(ZrG0%gTTRK35V7vjhk8yLXRx?>DvNET6>IOIh$Ksh(=s(LZLyq`Z6&{f!DqqUD8EG;qFD8 zAYn%H)xG+2JhpfE;WKdxyS>R@G1Nw0a{clAsciLiSAw?QHy@KSNFj=Vve@NXH1QPh z)@*d@uu6sPP1=yTmoUGn02nS$XCYnxiDLS8d87M1F>9%9LY{ff->KuqlP=?DQgPY- zWm7^1`ToxY!A_Rl`@LJDVeG^PO)tDcO*Hr}JvrsJbiH`1V-he$2Ew@r0O7F@^Ws%|XqqS>E(i{q-x~4XS_nrW*qc$KN>b-K3Sp2MX$jrje9G~`j3v)p zSD}4!A=^_NUdU$zJ3EdIJuzXzKl%Y0kGOr3k#)wCX3}C&P`EcurvbrY+?J43$#R0` zuid&0cI#+2i(lBQ8Mg)w8Q}I6qBsA+OB~c&T3k9`bJ zh`IK9D5GP_j{5n#lc(9Nd`;fgktW0`JSdB-NXH<5HOJD62Hh7sxW(QICt$VxnUH}| zLXvEn-OFb3TeLVtukPSf%@{wn;W{YhzhfuX$Tho=x#guL*Swv|?Q zvjpq*M>U}~!i@3ULNuPhcSg>ei~%+-*6aTbBM~g|x;;0Twb#5&(VV%yy|iHndFgiQ zG0zm|k{vt|e>rrLa;c3WCp_&~CNaygB(Q$7GQ+jl&j=`zd|8TsTH+IHD#e~=o^&dB~dEE*^_70=&m^<*|>0}RGxz&e}H?{=iZT4IjG z9nKujFJ4?P^aVVk8Ko?pAX@7%l)2lr*dsiOrP0etqzs z+*sQrz+-1*y(h^P1ax%_!r8HQmB2z_hhR8YhKr#q9u2CG`}LA1Fsx}$>;ewgTsCzb z5fRTreTGia`%tfGd2Jw3wA^sM-oc>7ggQ;ZmqC4HUF~D_gst}xxgMa*wR!H%Ted0z zZuoaO9F2+YShK8REYaB=Q$0)^hO@MvPi=zIk-1OjvaB*^KB`?8_u0bdH|wJ8K3ykO zWEeO$_4W-Vgcnz!X3c~lbKeNuqZ59jjAuseGk^<}=NMM#ivb;#RKD4psWkZ%uw8V# zrrvHJuNHRCR2&YFVAf}HOx=I3|6SVBJqW#G)BvaoK4pO)$OftVqT+Xd;GT1Sb;oE) z7hVAQ@#SB14lSZU`L=Hs#kvu2@>Q0v@|H%6BXLizkVP7orzq3CzUvb>C-*!_k z0QGHJEc2CaJWfF)bo7{7K)PSa3sX5BJ@g|X_i0t04#zkgsIOQ^p{GTq*(t-@)Y6r< zolYGI4OqA|EXMSDn+i4%d(V*bO>b)dWrQ97R9tlTVd9kl;6XvJDzy&Rj0$HZFp&0R zQ9{M-M3y9r?IfAqnzgn;X57UXbqsce2Cey!zFAkea}S8PJ^R&~^1dvLZB40o49Jss za>xk(a@lD7ve4r0yf_h(HVn*TcwH?I3srLV%Uh2I)2Rx^A1hQw^xX^?SvzuTRVCv&NwEZaGz4`&%9YJ^E$r+G`Lr^ zZ40d5c<+9e`WyxIa<|Vr`KPL{-bv^lpV{Cby+jHmLs-4rcKV~sa8EYwl(d&s{7g(A zBOdVOLPK&GME>M1dzz@h6KEHRK`|5xJr|Sp(H;JNHemx<)<}WBtyjf0=da^C^!1k8 zFI^-)Kby0XVdump zP4y4%yeOS`B3_YpD6O`Ndf^#vd$#X~oL+6-|5SWA{V?oL&BCd)#%jMOcY9Ihp^ntB z0rsYcv51zq?%wfy&NkHFbogSDJ7!<6na(!r4-5hY=TG{Gzbh6gXAL?UuaKVJsGU0) zF`vF_Zn&CUh+%Ie6&FiCY}=3SP`6Z4{Mx2+&LCQUZ5+~fE>$YmQs+p+IP>OU-O8U` zaupMLCi01we!i45*w-dl{pzN|H$7A-@0ywc*O6fdF1UaQ|-NWwV+iz z&{}n0He*^0@A3)Xe{r;;(?2BbPNALfS?yrD)ITm#y zXq9_8#tc0~FE85N{1hED1oexy?X>Z3H1gKBvd5+KgS!2Ris2Yd`#mv4g&X(Z5F zeiJ<8cR&_yFE|;%xA1apQVs$qI87jT{MF11c28!20`_p93Cd{Sof_GP3Zs^EkmxCR zf)!BuAXvW-Bo8XO5%{eWZ6ZylKS;i6kVM1jW9aauey|Jp&gX&C}kb z!xGZosCD&~077P|&`-q&XwWAF_C2rf>4Fb)mFYl|0bOL|EyxLB_I8cF0Z%DhAeaR* z0QXdtrFba@oQCp;)pRaFY}}0&j*vs>LXL z+6E6v3H=ur32;z-KqgNKq{!0A*EI5x{aq&RPn;QE$Ez3AxT1@IlNK8wYT^alNIq7B zdX|Q#gS7Aed0=C#jUm`PRVFd$YKxxRb5|FjJ7Xj!U)WosKw^j!FipR=vSm=BNNJ$&eI0GWX}S zU_YyKy+>(Qqa*n5F3eP;?2xW28oM+en*==&UV?Lcop-p6KO>okvG^AuAj5AD z|0+WoTp)o24qu3qFa_U7f;(81Y|Q=$d~t2&|l-Z(4RdKz;26G!?8mtRUZd;hN6=GFXG-hs>h=3v?62ev_ln?=fl9tX*tDw>;0@5M15d}#_5eY$5LP`+7 zxj3HltMfeH`#gWV?--1sW7mD(Yt1#+oY%bOwL)we25zi=H9yG|=Zl-8*u7t(eK#}+ z>ZHl_f1eu`cVDrh!;EV}RvYoaz)zO~&(pvugfRgAiLjc#)>&+nC1hokhZJ;rP>UGf zej0An!REJ%w(m2f!ud|09@y<`6y6{4n#?x~8qTywf~}5XdeLXoK8}2<1rGLiactU8 zptnp{_U6rk1V%DQBs0#V-^eXoI2nDUXw<0m8b zOXl7;4%tH<{z6It%q{R3$QAr;Pc$8*@61_^QvcDtjk>9E&EQ?_*9=KW{;t291OgZe zyz+hrmA^W#|G8gV;{W8r(G6cxoT@v{LEFL9M68~aqStp#OIi_DGCKaC24KPxA)a)4 z9?5NRYv(+=*NarspCy`0n_U*bkL+roD~643MuTr6xa{oZDl9jPv~&XnD!HX2^<}RkNHQ zYs3!~Tj7Cx_z-9x%VPfS9CQM{%KkeEz_d=uF?^H^Shu>Vbi6XQ09G-BQYlyzJB}EH zLv47S7!6Sh9M%9vGkQF$GzqpY=qIvN&w}?jTeWQW90yzaup`_uVV<<*CPf?~f9PCg z^e4e^^1$o~oD>YBgvUwRjlrv)fR7rao*uw%U*;uTM%_;(`%f)^|LG-fp`IZWhNrsq zYCVKCMEPyc6ijz3!zt`I&_bH{MHa-mgbf`;7nQ*Xg5!M1wuEGU*gZ__NOKWD7U_@gf%3>Sr+C370%fb{^&kDLi!u_UI7~<6f+aE25wE=ydy+{Aqj{Z^? z@GV8$%z*IYe{YO-Fc#2Inr-p-Z`;Ds6ryGe_;;0pKWw)bjT`C;`TMs!QLhMd#Tw&& zX{XexF+XY|6wPhDmS>sRo1w{>?y;cn#|Zfs3g2IUW^@WZkib|U|95j0Euj*}o?`uXb2Tzz&E|Od4?Fsw+Rp#d zT>qz+DUkhymbdI&aWnD#*2>d6t2>uCNPMj+`LBNdKGWCjIh}Sf0oo(2!=h=COUuGA zKGRuX1jg;9G5}Il(W1;ek3l^~Z1r={ARr0M?c7Axa%x!aZ-JbPhpl$%&fb&T-I3cX z{{P5Wq^Qt}_)l335TZd1xe2%b{_Qx_DlP25==Yy`8t`+PLSPH1sp{|F<`0A4HW?Z> z`~=Mxq8HPyB~*R7AEpBa^6|pH1XxICc&-AiECp2o6xJtV@z6>4_%AypEqE;t1xDA& z__N85Pe-NBO0B41^)c#q_gR(iUj{hO5m94Rha*bp+SSHTj{&QqS2;n%%5Me`l0t{M zAQB82xE(T)n93aQPdj$eluOqkw^~9yw40P7P8mnG{?B&$*q3wLPWk@BPKB~|QndIq zj5Ydq7h5$G0cz3x_OAAVB|R@L(}m1`=N_@U^6v0ujfVHa82REQDbOB3E8%C-w2N+u;}mK< z7i-ktjr1<%)zkM0p|Kc*=fxck#wPxvL86e#$d!&Z2 zA8H$KQs6R;$;Z?QXQ z@`Lg6;kfNf>Tt*Op*<)C!JoxPS}V7 zu}vIwEBYY!=tKrCL9%UrV*hu{?|)g!b!b1`hxZw^IxEVOy56+1$MChzFc2WEuEF-kZ)3$lc{&KRPOYi3NQeQnhYq z!m|T>mv;QU`iX?jUep}g{XqNu-*3Ywkc5Mn7}Z}P!M|8@76DdmvPNRvf8kk?b_CS@ z{?jF+pVJgavl03AX#J)&>XsJ&wYii6&3N*EtWvDX7;Vq{*Fp+Eudg^PL8lDCh`K_| zmH3KNA$H}3XppEU&GeCWdNd9Jl=F&NB3v%qkv?^1pWjYxTw8oj6Xv@2>hv-hwXo$> zvVGxOR#`Gq3L6M#Y)^5!Uide|>zKnBf#G#iY%b1DKi^o6#BXSiimrwhbf<<9&t4W? zgg5EZI6<&&#w14-SADxb@C};BjyVn%X-A%y0{Dh*!+dBat z+V+~Va7U9Q%}x-t7$MImd0^jOdKi~roPlmWyw`eJMue9GeMjt66;V~BOD*sxohB-) z+C;JssATHHXCyl{rHzi!6ZhNKuw_4`z=w))lx?l zKFUd^C0PRT8FSu;hNL^lKeyv1#g>3zDgn9GUD%tRnRrgdyV_njhQUUOz||<(M2G=6 ze+G}QRnvFRqneE!(BJ(HGT+#7iQM|(wgk1r6vv9;=lR7=RCb>a{j@tp)}yqMPY}Hj zIPCQ7iw^J++2Y7X195h6{SzP6V8BSoBssu1_Yxtq_?PGvi}n(JX7}STX^)S>WB*uu zbnRb0ky!#RQgUUj?*@$fL-<8&IgzA)b<`3V_+sEbyc|egafibX_Nwo`ft=u9E5TO+ zrJie-po8$~6!eZtA#$)?f9{#48~*EbK0d&;n9`hwfV73s0PSiQzMBGdmIww(chRt9 z51HCt05|qCvF=sH_1yJxcL1Rug7?kJd@>n$aJA&8g+LhsdyY{(*)&Dik!C+`=n3&7f z5G@HCHT#NE!r&)F`F7g!LNlP)ToA4l#+2vAX7M4iaM<_r_0_9Oen1D6TlYF z>P){#(*-N~JEH`qY4~}lzLl7J7AT9p{1Qj+0VNtDs>fBk%~+rH(KT89bP}u=LbC3Z zD|L2|X@Fz8`{M2*o~0G`sQt>Q?u_wB&OH5>A3)Yk_qol+!n#|A#%om9V;C zHw;S288;V0)}dcnCy3@CaXjp3`usafB@A{9Zcdh=2By`ScOGy&0*IsxI1ACbQj6kZ?D}WNZKdN43#`Uc+$9oV1+VP3N0IP z#z|JO-${}<-~)X{7WIpQ_ZTr)I(c}W5;lh25H}yE*v%@|viqRiYZM+pionOc>lrpm z;Xh3D{I24CWE}r)qO39g<}kF#={ayxZUB+Ln6&vE1>^6I3`ojnA(zZAT(he=0XLP4 zNx1y^_X`U5(16HYec=EsRln;LStwkuejfYouMjJI6jX_xANp0ka_EnAG#CABvYY)W zX+M)a*ggzq76&g>y!-{dX(^A=y*g_)7rXF4eU^r9opckRU38Dw7dZmwJz`{k?AQPh zM6jPY-l!r@$XkvwwS5g*(agpdcWgC@WgB&j0q^tSTlPUxPv;fi?D zATkng=t@f($N*Hc9=l!h#jEIiEfEshn8vF!*@?-}g`=<}JNUft+1Ja3mE%+hKAi3} zc=Gdsdbk*>f*1VCE24P1ke29R##vP`n#4R`!| zXfSXBn!8dNfNS^_$;CiF%Wvg@kvzU_SgL z8=;1wyAlgw9d*D8H2PWc3~s96I>FTRY9o0!O0XvMd0)*TfO_LweZ423BLy6$0mgas zv3JcS3->k71#?A+z^q^1NodJ|B6M5f+$MBxkNiZ!D@>+hgxunV!;olo+U@POYw zVP~421^u?-h3;~dce|j4LlWA|?t6l*IhcoD=e?2#5bnz}R_|eVba-HQ7xe*bqa)C^ zZ}(7#U_H@Vu{>aw0&w#HYLZreX9{F%pY^%2o;eAPc{CmUbO6YIpN|`)u(%)DhfwOAT4>FpIQ~ z$*DhqWY&;O-d%+f*H#@B5myOMNPzzgT-#y4Kr%LH;H}Y`t@fXX_JCx9xy+}$# zYv8_G7(Ve^Sszsea&88(G}!tL_mwcv1}8_?N6hpO+_LGnav!m)ObUz z8QLYXMU$Yp?-*BPo&&8^$T5Zp2q}-G)B=TY3OLA8NMXwSF0N12U#suKL?h3Frj{S- z*1O0}gXin_{1`4hqBJvv;#}CCFK{OW@K73pP*ql1liaNz-|;3kk6J3NG2+AA^6{WL zWd-da2cI1#H8@Iny)nx<8s~xZMcgfb>-G>DC8xjz^vv}IDv%aIIpLXJN$1wqYJG}@ z_s@O>NVNzzkzQ5w#S{=^*h$UyPgaQXzEYogdbCX)fSusXb;gOwYpBCr{RF{DV8H=) z$%^xqg84bO8QyU>XbKyglL*285au`udNeEkzchRoC)at79!V_s%bJU#xDM%K_2?9! z<|T)WMm}J~9ko>ro|URK5o#XCO++t4DQD>8VsVT|Rt;pAiP0Amjds+6>jdNJQCw)* z>u&bW@nn>#Q`IC90z5qT-72P|JjFSCJOewZPm*rL1OL}M*TT>Ilt}E;QH1?1&GfKX z+_X9bN(HXqB20Wwp0^msZrBW}vpL$diiO0W4j3k?qN=I=LYH$kfv;>!j%#JNhx}2! z{q%-45)ip)RDb$5Uy_}Ck{=K|{cMH;|6xCujyL?`NsO~36TkT4`(|cGI(vg?V{L;Q zY^x~^Jw-RCx^Le$YAt$S|MQA_|IHQ3wLbF6=QbA59`Eu>=LI+b9a#%udxICN5uFE? zT62k_7LPmURv+n){H(he_Y@ix@hclb*YJxQhC<>SJu)Bf61PD7Nv6yjGGM~bEyj~Q zEO}yW={Fa(dRV?HSZFTZydwTB)1RhuV)>D;7xm%tsP4S$Jd@_ID!wRrk362%drzEL zZ7tBLt$Ug_>J3ro#g+BrX1qmG&+oFv%1*5X)?S}|-LbTz^HIbDM4H=!WW9Kd9 zAohpyIkak)XZ%`-^tQRfbZpD$Rs6<6x!B~*Zw0)QH;?8JMZG7-i6y|sY@_d*TU9I; zTmNZnL-@(ipNeH$hvc%m~G%5HvIJsLnTE@ z%dI##*TQ@D(R2~nIu@a%FU8F??_mGAV!^;N8D7yTm}fJx zU&PYKQRQt*SGY#qDpJ039Zz2T2?>_&-0P#$g$jNqdnL|nz-}m6e)^mSY)#kh==Gg} zcoMykigH6r_H+LlY;%r$+4ZxBLg+T#Ej>e~vewJD4M#uP})wp&Q?=TFKpeYkbTId6cgSM*fuP7-Z41Dyo%Wjn$9+VK>40C!#bJx-oB zc*D@X*)OW-X>3RJaBs-O;pw6YyV{PoO47^v)O%7PHzm|CJUZ9=S~t;RVU+RNT(ria z1tViC@@L-BYRnxTAAsdb8Js(qof&jCDTr|A#9&0Fob!v2BXSR&*PzmvpO{+TiRb+V z@Ydy&DkvR_Z-8QqNtg2dGGjh>*T(Op&RtiOA&1rv1& zgQ~~S67v*lNKz$@!(5FgX1+Xq|DuK5@HVA&*wMPm!(2ULlNk`7D7In@tAA~6qy*~@ z--_WM??wFSEXP*%8{l@ne1I+?u`29Vsd7_lOiicXd6&!YTTKm>mIqt!8WPK371oFB zDg32;gaXg(*#GO2KIJXiS8=usnBL>K^rvz~l9Cx1-*P$trMyAR>rcN4`AS66P87q$ zAwfAe{A2>+7Sw_^&wT<3*EgPnT_+bqdr!AQjQmuKHADd^T$Tw0z{EA$WxQy107RwY z(3OL7Ztdkq>2-uo?2J{Cch>QX!HDa!DNaPGh)wzlUB28e%$F-mdg6Y(-}1#r&^{yc z(e6>r(c14Z-3E6pD?oMf;z;5yf_BEO6#^frxLJPe&?<1@svTB-!ZKK3ywW|Kvx*|m zXWx6Z#Wz6k3;S`$Uy+v&E(BxBU;j%j7PFo1igM0(* zOa!u=qocu_6LnEGn*vD)lBasozj|B>R3Q8oSV)RJZ;A>QvDUQ_78KhZYP*is*Bt&x z?SIdZIZC#pMod4b`99d*bhdal>G5t2;${@Bc02v{7mk47lhe?6Ky>C z;IGeP<6$!}kj;0a0+>?{KAaV4`F({8L}^1o5SJbSE`R3jp=gyKcr-M%xZ`k&_pgoj zc*sGdK`S$|AZ_1HunI(YkT@S61q8Kue)JQJ*JyOt`|Y7lEU}fFtAAGS7TsfNUx>fo zt>v+&fUeg*T|GW!>>dn|GMrTO9BP*b@E{eYt&;d@2PPL8*=%s$n}*+Y`0hKDdl9cE z_sza101QZ?+Yx^qVy9Ws%2=f1+&%zf(lTVOt@5*##9_D8li@;h`7$tq!P50W2m61c zfb8k;Ir%8^-|^7CL1nMkQGaFin`*a_S|wWyX<8sI2L|c{4|CZj1zwbmj_8Z~6JLb2 zeFjqAGfYS02a%j%T0@`1pD#Q@0AJHC=Kn`c~lmyDmQo|0~Bxo%(OIecq32Ar|aEbfepb5pH@Y_3>SxUo1Cvge z;_RC~Glw8n)PV z=f`s=|qT{t!@WWB_cxdX|uIlD*r*g3 z3MZ)p+teOu81`!_SDLJ^2s}?3_t(-iGzE-rPNEYys02i(Q)4kNpV)_IXmlU4+Qn9{ zW1WgAQK6A-L88qSqISL9Wx%u;A}8nWwCkJ#S$BM7_(A!B18y&s-QQ5+Au4}HfbNfV zpcUmSB483B0o{POID%5FYKnB?c8{$q$h5wJ)c;vNLK>OKC!b`?IGMiAe9ib=-}yO8 zo9ljU-9VT_63TQ33T~fF61jJOA0D~;Vc-mG^N@6zb}eW8u?PKO1->p2AH=I1R<{K$ zQ`T)YY1sSU0o8ZEXz6Z4fX`3B2hJQXDlN4-9%&}3Yu$siq?s*lI-`<=nX>jF!$tKQ z0HHZFL$=|nzf%)y*vZuFK(I|HX@jTfAa+KhLdEjFz7m9{9=h&FP-;I}I|bT@_7>pg z!fRebe}HWt2qOVK(N66p_=Xn_0DcU7bj-<45CgRAfh??}f;G3QH|SVfwmuRKgFem6x2F0SN*wA z{R20OsOT}97EiP&dB={~cx;K>*gCp|&MbhXFpRlYTuI65 z_-()CZ3z;j9v$oMSyJp&u*h26?9Wb)tAha7@w|md*~YD+BO4~GlgEod+wXP40{e3X z3msMI!(_$Ot3xL_(sGCk5e5lLzY;?ksp1o8y9Yj4V(!e1Uo7h2;tT+V( zb>o}^Kkc$AYLYhYhhGQvX)gpwOdln&T;0eEma>mM_=-*j<$WI^?^8GAc!Fxmo9t9` zo{FA}6Z?$)Dd~a(eZFZq_1GU1J}$vzK#kdtl(@H^e&&7z4eLq1J9dra2><-CtnG5P z7+`%0x`t+9k_XRIhST+bs2MG6BQzHuYVXwTU7Y(e@}%q#KKU|2buhHkZ+gc@~>yX3$2n%Kr(n^5qQhD$xF}9)5G(OEN$7V2O4Y^`&?V+o{bk=`!dr1+!Pkp zMzUwpEr5Rq%$%SyA-(vxeWhsxQWD3vQmvAP=>`T;8m+3a8s#Ie83!zT<6goNR1$}i zLL)+hU@Zr+wu%~>FIpR-Mn1;_6(FJ2<<$e`>C}iHj_~nOz(Doj^36u#$*19&Cvtso zqm(j5))pe-P)sU(rnry9m;Wzf=X3h_U2@=h`TcM^g8g#7avs9x96B#I*nlLn-;Z6q zp>7UEm=zY2V=~^4^|wxI^mrx(;Q5QiiOo7{?{2M{e*-sO`tIUZy{o+nsQA@@ZzDQw z_z?<*J(Jd{86@my0P9}boFSw;obFr^S$cF}xcc1Bs(0+rI)lH)G7{Zg0+Hi85DD66 zO0?K)82O+;JT4wzg$(J*wT*ELg5)V-cHp}&zWQKn0KEwbucMp4-h|sjY2uoNH*?_81 zO}~*@!Q;gG+3bR&#O67uX;e7=5K@w|?_=?vD{-JL0|H*5$vbG?ep9m*n!I9M5j2bn zosnJkx}sgBvE#|3pn;aI(0@XA_`r?tpRC?2p8e%u6t6I|`As-|Nkl0A!LdDk^0SX0 zD70}8`P6qlDtMDo%KEn3T@^E7_3#^q)yMt%dr$B1P2LOAi!VNl^o%C6srb_c2m2@x z-Oj%UJ`R1qrX-!_F__m-eWs2b5LIVBMH=C&SJXtP#Sz;7uXAm1z3RFI#>r>=gb%xKt^7 znIds>Y;y|8f+E^SsSW`3nkdzJJ(eZ4=EJ$(&{QOZZ#`|)AsqjpCY7-ndhSA>O~w3@e6gaHH@Z$+ z=WcuQ`rYe)CJ1Qbo`7gDA(EzTJC6Ay8s6x}Q!AF*z+ovypQ3%TNTy9A%$2|Yl!=j= zudDaN4;zX}wk{%KWHMP6wkDZ#X2c9;-|>d2G46ii!#h8O+>_CM^%knv?#UnG1Pt>< z9x5=T-@GFBjKcTZ!iCK>dxFoeS&bx1UiudRD?FxUAo=NWaSFE2iWPLoz(55A2ss{F z0x2I6vduy>f1zx6ra=BndF+DX_q#1eLR`!6-JQVgmYldaz-YAmVbH?+3E32qQTUwT z!&^~zJ*5hgZS8`D&3_8n_8`uP@2-qO*l`w699Cb_!R(dF1BI$mA=w{>EME$t!>WNw z0deitp_T; zNF)Uqbc#o=2*#hC1*Va?nn|9WASert{DPWkGHAi{gc|+G=l-BWW?CiG65t#c^bSVS z0rIJ>diBkEhL1GIHm{b!=MoPm<%m&+A}n8@SZdijOE8sng}t5nD;`yY|H76%A|X;+<29E7TYZE51%O5!0Yr1DnFMAlF-JVl+nL&)8bT!>A}Thw3VA-Q z%Cfr);bgiw#u9`&-JX5fK!KKG;bxLO5BY53^#p|{+znc(pGThLbre3+Qn9_g2KBOn z$Jtx%00Ey|7cR@ix1N6b11mM_uX@be_7#48i1e=<6vCk2ol7zh2K&k(6b;JrL~To; zSbIk9qV4R{Hq_8s=F7}^4Duake(q`f$2noY`5na$u9DjYsFa!Ch+$u zE!t)Mp^`=pi)Ys*P+3_@emz%n3_pxh+BHG2u~zeg&#hNpG$25inm2=(nZna*() zWl(XPq9ubcq9`r5P)!vYNSD2ma4B@bsXpnf%X2`_+qK_Y?OXfjN5?6$kY;6*#VCoS}kcBME@rBIC-49j)9Op@p zvWCk2DFj&b67afw9-iln0T-7et+lZKy%aw^ka3aX2s z<08H5fLGQ`$>>R@!DnMN&xK{KgoS>-_YWIAB?J@jgi}`@t;rKuXzy{Kxh0)6bF}`{ z@h+PH5=^h^7}PZ-k)pl6IaR3bt^-$nUipz&Je|9wwDq;;e&=*o`GA;f@{`0gE(_*u z5NP`@|KdgMMhoW`eeem9lpC!B_!nwZJA^RUmzfxi9?<4nZp$FOU$-SNDG(a|t^4*`bqe~L-X-f75 zL?(U<{U=j^8c0HRsO51*gq@7;Dtv$Y(~#8Mp~NfSbAXDMbQ`;Ojq@SfVMmT6+r1xL zx#*Ihm7)qJvs+)|pe=+)wd$Y5ZaIM6wY{rq(3UOm-qdGQn@l+UX*|Y2P@tNR#@*qL zjCoeQ{4WA6(%$Ns6@l#?-c4yW+PxKO8YZZb{wGV$Dtnf+Px-Zg)ns3U&k z$i}UU@2*chRm!NnIF+_s_~hNU{VQ*-r!VJ2M%o1>Q*J~e2i0;;+QOtnO)@4voRyJt zpot^tn=0Caa;Kq~%?XVl-Lu9+vXoIiH!%Ffw--7RG_e_!cc{xws}gM0t?*j&q8Sv7 zOiWtN&LwdP2`8+qtdz_FY7L^?%_gOD;>3xp{CpiMDk_(?6P|Y5V~0qo#8-{+**sKb z`IZ+SagWKu3qAB@I~dxj$v!mgW=ZA0FT9^Bz(j(+?!_)LM*O-+*ZTEv_*Z(mW;}Fn zrbwPlbAAQRD&frLTq;I^3uiUOYAU5e=Bd^|B_1p`=QUj8xfEBJ(Im`sEYfcgOZtLZ)B?6kESGKq$z6`FW6YTT83NsU-57?%+WW151Nx)9M>wVhu%s&3vQ`#V{nolu9tCHnH&QQZr@W9O!-c zg^nCa=SOWtCPFq`BgHxx*F28ZOi*bpEuQXgtXc8XV+x{UI zcK{~BxnW)eCP*EwY0D@rzeEXko1udV`XC7ZIqO_nqQHT(VxA9qC_=r4LC+Tizi~iB zK?f#MAHL7p+Wq3@ai1n4-`+;6$`kOPGpP>3dKBmp?h6p%%)kRmlr88tcI~D`gAxbd zS9dNG&!Ec`S?W-|y04HSQ^b{tkF9Gj{2XbXKnTUct>2%+`kv!~@8*ejedZ)(ppXCq#p$u{(j${qf$v;=7{hX$44FH! zcushe(YHhnS+0#A{5~W%|WR~2THfqs1B_vdhARWN|UXD=oUs*!QiU4jexrYq5; zV_X8^J1=y2j^6pbOf<1-@TLISlG(C=;luVy=Z_OWBS{p0A=7-N`J)Hic`rBI9Is5e zIvMxapHt~Whawvr=xlQVu?{Wh(I*qTOcRUg%**DLzOq8Mh4BS6yh6((>9yUmi%oz-(m9)7TevTCltNzfdfqC$sbFCfE7w^C@H1H+=U$ft6Gb=c1O$_?9X5Lo0{&!2z10XgVa0khJ zm9UP7_wU@`Aw5f}0OuVn0tQ0eVglrP^Z!3XoceP5IB{;|i8dVdl2-hHoSL~EGMmHD z7ZZ1DWth+Er<)Ln!7GDYCl6xQrFx8qP^Nf}hw#}z!5|KjCZ7B*s^7Lm#|;CloP0PU z9vtZv7~RS4(@V#-8U|r3$Mt#;NDrrip}E$ESoz~w;psGR+U=mJdgPLjK58|Z00L5i zr2RlUVD-t5y++`zt&QFaN3$v*AJ_nJLHWk+_2cL{h#3}v^2w9WR&*Rf;|ZYqg|4pv z*m(p{@W}N;v|vccY5OF?rE9XfD!_O&vAS@3c77l2;C9i{42#^`l>#Rr1IHT}U@_P_L?f-8SU56#tm#)|n@=!JSieX9!hFQy<6SBN@q#60H5hSKKwfA1_d zXa{1>EYn8oTvHX+BRxl*9xP%?GAGJ1`Jx%l{*}&_;L$G^mZrK(-DY}6_osn(2jlK2 zx9iI)EdZ}#}^UUj+{z1bP;IX@$8 z+5k{--xPC%y#rcX_p!h`t82=~Z@=~iX4wBTIY+I+L8C*OE{x)(G7;5+q` zT~;fQH)=!rBJoOzvp~w}EkAIk`)V#})_DVBX!p~&X}4$n%Rv}0{wJ6FV@|{i&a7XX zeXU;=uhYBr(sIj#K`3Y#8q|5)#dk&r`qfD}j&MVVaa3y0i{AQxEI-V@xBdDmq*35+) zXJNRa^-#B+_TE6Tl}hbY05!{H#{KWio?Jc?yyMO=RB#IA4LI*14b`K;`<~4_OgTMd zxvKNspW!eOi!J#;f*AybVL@Q*z48L}lE73zp#{_7$Xc^_vmfd)_12VS_wREGz#b35 zwH{Be@~(s2Fpvf&PhBQYgsYcn{o>e&5sT3$8aQy2$=i(z=(I zj0}Tj3XICgCl5|oL}8e>4ES!^0^!56#XvTv z;G_@;2Ptis6Lh~q=hlyOCVZ5=@qit5)-d4DM?oB?|C5Ajp&E4=sMo7+$N@AanM_O% zYePNpE`VNkf=0n4Pg0#LY1i$joJ6ft15z=|edKw_X0#@m-=e+~8!m5#uhu%d|?v5ogIXiv3# zD7bYA&}X1PacyP~woQ+$3T^w6aKz!~U=n}ErqT{rac8$yp4N8&AjJtJ)Vxsa z>j05v2AzpLVMpln=?v)9gJDYW&?ou&pd7vVPCMrFaNL3T%*%WJRFbNN*#`FmPyp|? z&ivyU+}(BiT=nq%`^f_-#3`B)9q6fC3C&x;H2qvl_D(|tbib>msdZ?m~6GvsT&3 z%+O9w2B#3#s23nP>>t0f&Qw=u*xn9)ji%V0q)ik5d=DuXMhk{6d8LD>%w6r=~d&b|2)4X+Ik{{ehf; zMq8GnEJAT*X#OdKQY16}(j$nusyRsATXrA)v`gORKCNn!mBuOV&w3AWe)p8_FJPCU znTFMQ1kO~-hCM!6h345^q<}wEOVvg~B97BiRX$DsZHfF($Nt~lcuUi6KZlo@%AIx4 z6e$KBjGj<{#L!z-SSAud<7}qOxM`04*IE!FZdX))O%2(^2cJrjzg~DKGD1X!9r;^e z9CW41x*ne%?*LU*&R~TU;9T?eM|y}ihQaXbQh~H!mgfLqm2C0XD=6nGgKW=7`5Glz zx%%5saZX1~@8O&S-d^t#6l3tNwF4uL-K@Bbp%@x#|En0=`Kcos-EYG zV{&Ad54{a1CY8de^xYu?naVqcAy*L`5$=IN1!Zk;02?@w@E7cSoUcAx{Q7aW@u|z& zRCEBunbIiqeXg)5sEPdqzKMZayuN1vJg&f+)u{S1HS4)!+B{vt0_sV3O%x6l+_;mI)`Ha7?J?i0Fwzm&SU5GlP22~ zrr(?~NSb_ecgbrMyxP>xR^bX|s|2;Rxnzt2Byv+6=cIJua*)r8Q`W#fM2)DzI^((l+EeckVGMw_4Lx5aw$;wJPHBokAl^&qb~ ztP3goBg%9YvN|r28!HZkW8QVxZ_AKCe0jFujN=~DfI+yO$`la-_x=sWa`LB!4~b?2 z3rrrQgA2+NSa8Vb4@(U%m6D{Hqx|BZvU6V@n2ZN-9+zu{Rr5|n`cN-LsnU#8^U|MR zwMSnKOg{*?LKN;MGT-6(gJ4HQz@xfqL{gM)Rc}H-He2`qSxgV*95M#{Ai$>ixH@dsZD zj7yG#3+t+$y;ty1oDLmX?vpj(^eGTCWzfbtnHDZ^C& zd_B;FyFgwhR7Zw>Lf3A!D@#rx_1b*po-^8K&6Z;mRC0hAw41$pz8EFF?U6w{03yP?QlFL7m^P+^qk=( zob;yt8L>U&odHHXcoT4?iI7iUdMBz`1l_e=pu58fhXOf#+984`?pyD&*y5DG{xuGA z#16P`TWDE!0kti=+eHvn7}^d3iK*a=OavDYxt}6s85c--14k&m(b-R0;OlLH9=Ql% zMHfn!P=A@j+e-a_0pQ$m0O_SHRN=;<1kZs&b2A7Xa{TtRL5-XPrTd5NfY13JumvGiCIkH!t8-=* z4$)|rLWc&!xs3-nA*7Ug=TdtrsVX9F&kr9kQCePK6^cWoOS=d>4H%fK$t1nB3!Fr< zn@k4)0K-Z6I%*Hfnz0lV&i6#HKUJ0P9^BqsBz&b4zFmD-S6|z7Oxx&-PPJbB>3qXS z5f>&5-hPm_^)k79aY(cKjN*fq~I)A*12C}tTE&{2+>n3F7C!vG> z`x?leFz1Hxp%SX=OXXc^2f~)ef$-bEr0bUGU!nk6M?}c^zFgRO`d9CYgvs=)-kNdd zA|7wk%?de`2L}>=y4|aZYYRT2=rp1K_-j)D_42;2iq7dFGJUIeWn5`w#z@+NkF_}% zXSO+*mtW8py!8_rIS1>G)U8>!1#7lZsJFGtw^3**-bM^Gyt#Kv>eznWJA}`UWy%cg z%%ogjmR%h=&C4V)E81kfTn!XuW3n%T>BXCb$#z*p;a<>L-uaFBk?H!-<4d@RbiM&R zzQF^~aYC2_(#kzgC}=)w-3-Gy!I6lHf!Sfkfgu#da2i0+EXT<1X|9;v+MJF@Mr>lY z-cX23tN~^7&i#h@{o1^SAtH!*HUa4v3)hdU)y%40R#yA34WtH^!j_(2OGOFXO&iv4+*8fTDO%E7FQI}S@$b~zE zfF=zFO+ON@KM;R9RG6UBe0km=iQMtI%y|aQ2v{dBsm5X8B+bA{mISaqerngJ&a=-Y z81d11Yj0c^)wiSH>z5rGrDQA@k{!ATYTo3>H6qdegdXUah9029EWPDcb>_kuGs+vh(tpD9!>9j<<*Eyh7)Shno}nncmm z%${$%v`sjK04P?aD!VFj^CTH1?C(4zEl;(uXtUjS%Fm7Bz0o)?C8bq#L;T-tZ{yTu zVE#s~QRu|V@1i_Czbenyxm6t6@EQGVWKxXH- zD$By1VekxRX4R%e{98a0E%tT3iTM44WfP~#54ptOGmgFnhk><23dl8^EWwfdXio}l z<}EYn1QG4mgDB2JYq?dp>vcCRr<~YPL>%a5nm$7#PGox?nd-8KP3rE~^dc>66lVT? z7xcPjvYAL+dWnr1Y*3B`u?B$qlVvuzQ1HZ!9-B1e3~Y`He01DafoU8ZCT|5bJt8mh zYCc@-OqR5FlUAEMC2Kz*tgCb7LB@P5uluzY`?<@9iY!?=0BtvZzhaQ|fxT-V1fI;b z$|O7Y>LppN>dsBEo;RS!hy3t!^Y=i^4i#-PM+<-vpE=oF1(KQvvlV`0oNIgU!`+4& zSl74p>3-Og*Puv1(Blhi=JZwIqQh7LS|o#{@sK80^~VQhhNsg-{8gJzhIgk(4}wq9 zn;RbjV6xjJa5Yk9oIS@o?f~Xa z=B+ndNXw*AOrxG!yjZ2~DS(%zNTM#NC=%^X^fc|hzngNU9u7iIi-LAS(F;xa3itf^rG*7r#MCJyDI|oysnC=G zei^8w5aoo|&`~D4Rrb;Llgu4fmkh_Zm7;FK1A9}x9~+1vg4kRKfB}5LUMfzKj1FQx z)8N&XYDN4#S`0MHK0ch1_rI#D+`_*208YY-ZPE#uXZa(5_^%<>`3E9l%(g3j;sQi$ zhUt^b+45>zAK8CJLCHX+*mVl+Pn!9oV1c@Ek-?>aVCBA5ZZQSap7&K7O)P}2g8o=< zFQ8f-g;SS5?0a{VpZ_IK+CAy!^f*3ZvBDFO$`edO!ds&bmO!el{>ZIrZe?}Pa+r(i zhdND}Y8SWu;1ti2mlp32FJO||H29G2cMeLu$`9di=Q6<2uP7vnT$1cAZbIn;B^#>{ zpO9_QH*|Lcxijm-7RX;vuk~MDgEtevI(Qm2z;qoL=3_;|_&S9%bg0C?SGmm?KnXZf zJ!V*Rj9#`0#XBXedlNKPzP-0FDms5KH*}N_WYCJ5v`tJ1Sp6+P*AKdC-7hdp7g0%r zhGi|lFb{=yHho5s56D>LepC^=0hV!hop3Q-pL88OsAjz288?9`kg<}yOdEXz2<>vY zRp(*T7hY?mc{C&t7l3i`CE4l@_Lg6CO`|UBuy1DsXQC6m@;C#imi7!LZ<9Fq$p0g0 zSW`~2vikr8qvP5~KZG~NK*TFCqEX`A7#P3RXo^T}$p3*ryW>=W?@j!i7f3$(W!a3ppWkU_!W z_SFci6LTDq(0d;AscwASI{Z=?b8@;{ySf&32Zrt^l@_f)gb*jyFP!AH9!a6|?;oOL@XnS@M;)TujlT3Pvi*Az)1;bESqoS4_bYEamy zBxMVB3eU1RRVtg7U$U|=TBEP3kFq%=TW>6;HwJanIOrs&loDy^9RzZFJIY2tkmps} zF$iAGbIhMr@a=dYvV;+0>lTA45Y{7kMhkUh@eI8;g&7qcg^u#$5sSKIeXqKv@Z+3I zhtFxp-jq6@bK3D}PPs+D$CV`+Ks|fVy*eD1Il$0vNCt`7&;oFSocDnH7h9nj0>ggwForpmW|?BCZDo*uIm(CE0bzg6L#mvs20_eP91W z-+im{Nz;S3Wh-^17i;%3IR)*3_}>O0l6aXi78v$LN*jYDD{g?u4efS%R>1Tx?qXT~ z%5?S|2h^9szvSv3_UW*(>pe`J&mdth9g zPJJ&1W2gO4YR51SAcr`+9!-!snWt()Q4{f`FVjV!QpA8<=5gc<-^1>erE3YfNU77P z94d#QP5g^H0Oo5U4X|Z`4u;WvO;{c4lQ}Ht4B*G}df* z3hVWzyyUNg`vM3(e8J6Jq>(Tuo|uGg7|kqIEPs=5c9^v-gC-8bGE!{aK}~pO-&83Y zxh|9tDv*X$KN8Va4gMC3b3!HaZqLax(YTlJ8{5$vf>WxLqaD}6Vbb2??|h_$1V}ui zhv9Fp2ABA6*($?%s#Xb#D|3XuYWmW!j8Cl!vS{Ao@&n29Y8xW#phvj_JWJq%Q2L#YX@ z?l>65>^ouc2uR!ha|wpw?BR=rRl2w3L$r?uJdOfc7al%ffS5WAM~q-e{XtM@=Rns) z8{XfT58T6vN@H0K3aqSL%odldTQ)Jd7v~Z&rP}bqv&~_7%BTi!FwuqBsK|daJ9hf{WY?&Mf3Z63uA-szPIdZefWG2A?Z)ilH6RePk7DYbs^np~ihae89nnslR`Bo;N{{#xvjHu5Fbd zEHJ?3`xd=mw$zcw;hPD@M?Fh)e)}i>J6Hr?-;@(SkeGp1kf%Fw2*y`Q`M@Dc4^-3P zDt1T!P>2VvCh);}@&JOsxb}8*1p~^(?CXD26Dt63X<2oi2aeWU$;BS-C|!5~6;fc7 zf3q3ZQN1?BFBmwV@i0H-S~|?sw@1M{xP5EjmlIg}h+%g* zi=mzjNk^@O!_cYA1Np4WM=WA+B9uct*T7q3K@!|pLlKdT5d)x3xVgdbOGK0Kng5Tn z?|_PGZL%fX1O+53l5=dLfD%POl4Qx)ghmvJf`ABupk&EG5d;K52@)hrXfiY?AQ_Py zj3fn=@ao+E&dmSb%s+2tE!VnUp}S9ir%r`kyLQ$! zLrAVER!8+z6O+pM{_6vhKn?hw&UCoBc$F;V*&L2saY7KRzy6@WfRsrP%iuWih~%Fj z^!Srtw8Awse;Qh&&gpo@Cw}zC5AaY>522|k>C01iK3yUpUEtiqICVg6>Qw_sNOhKKXf&kxr z_b)B;CaMr1MshV#P93pqtb70i=a=HJv6ZoP=7J-(`!#fQDGjNBGoU*i$YYP-v8|y)^3cryqX3560gG5umH# zd;N`?WsrK}2PSG7kaitvcKkRh;Y20-p1fT`0%`U;?tkrh{jKCbg--w(lNWk4i3o_P zmQASc$Z7u-(6LlV`HiM3aK(^Mc)>>7i&fN@c6#`;yMeNdfNNeb{?peC6NV8Aw$1mZ z8(@-q39_pMWRkeSX9xOuJqTdXogE$_pZXQSP1)}JVzq`xN-6u?e|Ntm!!yp|s1gxr zq?D0YS15Bk4T{lU2J;>^f?BGCvV+#*48+i)D`iz!KK)PP&(k+2@Rpj{K50FAiL2yx|>7538 z1lPLFPa$#zy-0LefQ;ub!rH=$2XcU znhEpZcQ-~B1U^HqA#8Dv2wqAOsB((8BHta`{8SD;hex#qvmq1mP57+BKhV4%uYxvz z`gnPO??@o27PYxDcl3SZDDI51Tcm4x84~C?NQvAm?Fd> zPQFtfhKe&mU|4>6^b-i7#5tU7`Lac)FV_lg^=_d&w+}1#==NJcRpdo2t2|;1hR2Yu z-Tj@Lm-0({?DB6iV;n9&`g1Myw1S3_Jl|L)(QbKbBJ+ z>o@q0LCH!(&9LIW_k+Q;%1W?3nuf>2r<$`A~fRvNcn zUTSeiT&*m3!=qz3y!HvYz2^+NN}^p~D+;^xcuZEGBY07OE7aYb@}v^j+}iEho`e*$ zkB;FijtrfXf8(co~jDDJi_>@ian>qnOBY#WgF?e zZ*qfx186%1SHs>&m7*L#I==)xpA$Qb42jhGeE5Z{NOyHSdVU& zY4vq3dfL6ti|n)|YKYvgBGnNuzjg8^OC6U+(3+J7+ge38IEHROjD%ifdPRxl zSl8bwN{>L7AiGAoVW6w9@0cBEmrOlZ;vz%@nF<^9A8QFBq&8$W%Dz2ke@Q%;d_*SU z=3@s+9jo$U+PXT>e-z@wxz;>o|K_Dmhv7p2E?IltgUdfHJrBmE3A|s!^lo@37QT76 zd_O5;K-+oMyCQk&5}_t#!xcK=eZb^O?gBpOt-5^KYe6B$mdX3HW6{xdu1uE>66{JE zc{7&Ty8`sgg`%B``RWIK5f}xaEQC2{cli;LMEvKX0g@J8Yl|Y+aF)eL3vDKe2ZslL z{$h(F5}1G^3T1PI>e3skD>j1ml++>!#toh~ra9gHDMi;NBF8ao0%V771&#~Sp6nKJ zh+%Gn!6_jcTbk>04T6kJOgG%nD5XW&3;Hr&>#tFI1*vYI|Gkc)CKU#{N8oJg%#uhBX`EjbI~b-n?OjgiiuNvYUiC^)81q%Ovf z+KaBiA$L|Gp{-xUdt|K0L!!@9=EXE{d0sfYCp+AABE?(`V|g6v2TZ}14l)7}BXfe{M1A5s%NKMp~=8{ z-HNnmdU^T`2W~)QEar~3KP@)mc>wTspEsf^P0AP|e}h9eBpx^}E{mheboiDS86oW& zbMK=NY=$X#|A4T_Lmo7km%uECVkvQdI=7I3eagqo5Q9zq_W1$PU%gD1VZQp)O#=jk z5#Fv*p1}IkLWJ1gb2o5=%pyhA9_znO4pUNkX|VB5@z}wRMUo`o4)j8xD^Yt62%x0v zIJyq7v)orvv67+h&6@)2c&wNp%V*?Lz91WHYHdB87?DJUs_ z+9ZwJ<|T79xymEF9_RN6ug7iiB-an!8R^!cOClI`5@z3tm@B<>*i>VGoXE(Hj(yBS z{TO#hsA$`?dGH4QgSi*&H}9`ehoQ8Etk+p^nMcqdYxtHG!CP$JYYAVUlr8u0J-f~_ z`E2t+V&ch^f%tdGEsq2b%8N}NIR#ir$Gd?`9RJiD4nz8V_veOR#8Y=;7;ngb+*`>O zN2c;AxFT|a&h(i&O#fDhNvM|OrS&B!FtzH%+KnZ_{cyoL4ijr-7ewH*7(B@~gXmdxnorU+yB zr<~|)nGzhB7Xb3l*O5Kv^O=Wl?JZ)&b0a3#LoWa0EE07Wf2v8JNo}+DFA(iPNv*5S zNQrc*pO8y7sWm5$;7tO)X_d{Fh3un!XD1^#3t0_Rk9N0eDjx23-+Av}%t#YlDf&r@(2p1=hZn*vF2FY1joauJhjMb*a;nutOAx7%xlV~SD{x}k5g_g`z^@6u_og{>6xS@TP0e(=Po~A zw`bPh>N_ctK!UG%6E~sSD^aJ+OrgY0EEzp|a;^Bi03_DWgV{G~2EXR6rblp<-0;o3rE8K@ z=`n|^`h1Ed|J$icj0&Gb-hObv-Vb>%K{v5H>n`0X>(oo%CBlA7H#eZGTe3@wzN@?5 zz^3uh9SZI&RyzOuuEC}WQKPQwzLQiC@Ek06AP2quK6}sp9gFm}`09zKyLY-4-G}S_ zq)$^^`w0A;35h4HatI;x4EN9E>$0{VK%mb=?tAW9=7CVq7uzQ}G!1&;&pQ0>A`IqOHwHFFS70B5C)L@@fHl1|cb509 zChxhT!Pl-l-SvRVCRDK^msBI z5=S5YJ_7Q}1$z-|-5erEj)9w_#TgF1)gQcd9qYU&oYF2Nb4PY8TczH_q?tmQ^s@1O1 zuY9?^<%DkaRcC?4J(X6og>HU(O0U@TYhrrK(SfW2gHvr&^=`_iB(JymXv9-= zLbfZ6xeS4PtHhwW#IcYMq~=DofbL4B$vCdYkb*%*4~_y?Iy!!-8E zfFkP~(AV=Bz@>#u`U-8Pi!fOer*4*7*BQlEJcr^i;Ia%<9JM-(vNzRMn!-IY;X?n|ie}MtQ;TFH?!bAr>^TJLm($ish;DFuK@6SyI zEoji%qxKkFRZA&(9$cJ}+7d08)gY7Sizg-+>SxqqSzg-8|A+Rsx;XHRuj zpC$%|BmeJoa4EyXFp$!`&bd5M?&tLNBNm+;yaYz zwg{xplXg!BCWG9$lVHERn0yGuz0QA!<;AH87YsH z=jL$uqr4?lANRdGe2e$n?oxkm*f&_`b%a##J1_KvJzCwK)Ve!HUT%4=I{4JFyiE|Q zjeE)fRqZ*j&Buk}M8qD4xu^36gr12rc&&GqtH+*~MW#;b*6Euc+T4t^(xZ6Zj5DAT zMa@DM{pZ4&wx67Be576;e=gphMiy(K+U^2lF-(C|EN5ED_J}=aqZtFckMD(Zwra0)sMR|#fi&iRv4!$q(Ie265fl)J zfVj=R1zGI%KfB>iYdG>>|Dh*2!?jOC_;3Sb4LF#w1(~2f3?LN>1j0%bpDDl0M`xaL zOwkxVJk*`HWHoDw&w}tU>O>Uqe!l{TZ;A_Qm;J)0{50KK;5TDw*P>oz>2lu~Fgi0C zPehSy0{JE5=tU+Np?7jGN>5DUAeVM0z+CLfH~Sxv3*QngW_FR;nB(}kCAqOult5Wf zQ?qk;#*&IB^h6R!DgccXx$`yYa|SZ)DsZ^B)9X#j;zdCJpj@lR8xxUMD z448Y}zc=Az-Tb<}H{6wJ`VQBzQ>mu&O_C|0cdk!2aNZ=-u_qeD;j>i#abS|r`V`s1 z8-E5CE3Y+eO2Usm4dvr3k~}=`xP&_VLU?`xM<}z2m<78OQxauuWnLX&PikC?zQ88& z#F&uv6NND5-7YJu-<1q8!wvW3;D|0YOjBn3uc6QG;MU zzhI+`JsrBisu$);p2L<*`h;@YjlyFj^W;)frWZR{%!;pGATKs3ZzZU{iL_Gx#y;KS zjMDR!8O%RFT=Uf|Qs*svj&iLwU4-A>V({003N`bZ0 zo?T}w&z^rt(TC=|WnfP(b(*Ds;c%B!o}@f1bXB_Qc-`5nEd(}}E?c3h>KK3Crjbe_ zioFrE2E95+$M6b4MY4GOLg`uh2yt~BCNIpXFLg~q#OW;z6#i6HufDj#J*;im{JM?F zt3fp&hr@_!WioUrH#|K2w`3~1twP^A@8j3GWMloo?h2D;hBpUJU*K+9El!9-sw4-g zg~zGDEOC;r+62>dlXv0*>wImUf1!9zVtpOgqNEx^iS72e`za90g+2JANEW9zWDqBJ z6B;Pb8Rl{rQCc~2*9$gYb>Xc()5TqF-l?CYr9BC-LkYqv)LUi`8URS6s6lIf93Xrd zyR<1B73Z$+#EWmIE2u(Iqz&e3wVc-}{o7oFq+R9iH9B|aJlJ=EiH9^T^|GAVw?GCm z%ud@8W`PPaO;^3fBFH>-51jJ#v*UYods8$^;StS#q^GH$xp%81uSY00o@ma9V8o8! z^@e7Cy)>yVjZ?%E+lC^d0c!0WgfHQl$rC>X(ah^kkiSnk7(>T0E^p=}s<}(AfM0mv zwIuOgC>;YurFz(|68Tpc^mDz<*QB6^Nj<#9ful&!7B(w>uYquo$4liK!!)BgSp?Nm z5*qLt5g2(++bfWk-m-+gm0IKqOePB3=;a}7cN~%=lnsI z_&ZOLx*u@0$MM?V@ETc#}kH^PhViG0xdtysTOf-;oM_}xNsIV zyabc1-D$6!i9&@E*4K70J^TcHB8|i=8JWq8KFgMnmK_;0K49qBSj@_CPcL*m{7h=e z#GS{iwIa5m@5pC$JOW#@ub0nLopw&SB7pDBYunM{kPSIHMnG_M^d*wqmw-Tp0GXxv z`joseX5^@=`S$lF_im$0IoWS`q6$T5Be|*~)YS~TPvsj4aSGkwoQO(%ns`@7UF}Fc zC)s1?ZbseFmg1OHKmKW*l=lyjcT!vOWohT6-g#$LHoiMJYG~dGe78eFAE_qrlGngP zHhf!9eu>S_AYXvuC8P0o->T6T=mr(SUd~R;&gYS6WIB|BoID2*z+q?@sO$$55fT!@ zh$-LoPfc%&``)crT9Z*Rs{oNY_x*bV;2JQ3I-;3v8R+1gIp-TgL`+;{`pT@zq3eYI z0Zg6^iY6i0GHh{@JJJW}t6fOR#%(^ZShzy*cyzgi~g(k=#D9$bZ1YY7Wd{QBd z8&!GCshxtoAC0zkUQP<$%Ty({Zy)<9S;J;DU64qlUHcgz{{rU8dcMrf%_Tp5D*EZ{ z&B^4T-MM3XSzR))f>0q~Tj1x~@IwUe@$~fEhFL2EbMy4#;X*@hf4djMgTOV5__HMG zJivMiO2I(o>mLLHAlF&Ev!odm)a2Naogw@v(hjClw2(1^;?9MjosYK#B_$J;_SY}W zZ+Kuv0f^2AT|z_9<6nZQzn49Gz^Aqv@Q3o&tA7{R=29*IK2+F3vh8woc(8dF)Dzc` zdHElD!631kQF}%-xsmS3$grtt`oi&2+RGuHb4zZ=j^86d?ed2_eerYq%FCSTq)!d= zqK_4Q?CYjfi8%;};7P5R_jILl6lq@a}4@Obkk7KZAifv-I;F0?J3HF*VFD)c-n^#CQ-4m`HhX|1+e2e@OibvXp7&2gCLko4soFz-0Iha2 zYmK_}g@<7ni;n8|`xeP&<|Y;B5w#HXFir>)!!i4Z#F`iX*+e8)A*J_-_udb>E~*ig zSy0O=fK3zGjN-N;N4`dF>1eV~{Od5(Gz<&SzUiCnoz6O%VVAHc*+}cz@7Sp}4*1Rg zcHhop+DJ71^OX>EWdlv&ENmden>68Ru%#Byz@n@*xNMpNCyG_$Y#+lgK(Z?KZ2LoK z0Y61lDHZ(IpG}k?EV3w`I5J~x|F8)F$X8*pce0)!Emj2SweffD1B%&mXP>B11**8z{n^TPW((IuWwmyuHcwG*}Aq#?X6TP%6O6&$qG+MSF{mZmqo2NoX6q z%=)xrYRg~) z5y!LZMAcy#iiQQc6gXYZkSjDEX9!VJW97GWbZD?{tuH4vO30-~PK@dhOZDpZ)v<6`rmASfW$N$BJaN@s;9}mm8t5G1sf!B4JRYcvpR|$rh z$@}0p;vfuUiK5Vzy30SV-Jrq7C8{aXcx1vCT_|oeqQl{Z$KQ?$t}reHI_>ul!JzWz zH=c!c4g*DYKLQlpOBxOyAhcorwt*XR5u=0fmV}qE$o5N56$wP?fp}gG!ZFW0v8d z%cRef-B}j@>U;&@aOwIVPUc~ zL}Z8xq9O4xx8da#`Ni@r0j6LirL9)Ar3MauZ#2MO7VN!A2X4#lF-EB^HaM*x<+lr+ zQBn`lb#G6GW!#`iOIrZ0fewfg@Xz>u{sQJXc7=J1nD8c;z;LGmCi$rlQ<6-XUifB%dS5&rD_0(|r;pj=Hh2P?Bsk<|mEXANdy@&Grk z!&^xz!{q(sYYg#S*roOX^bZh57Lcw}4xb=pgSoABGPfeQ7FOxhH&6?)T?Qap(C4 z<~l3BLfDEsVOaT-125fhi&+k}i&%HIrM@m17|ug@{Px!8?XTN%gN?{+$N5lUmYMmT zZ&>ubO-W1R-@0F}wQ!P&NwcV^Xd8wW-UCTokdrebzGF zmQYo|&+7<-e$gk@D^1+4Aeu?M_YkIX96pbgWe1j_DuD(y1PR=wva!C6eMs`?nVGd< zQ)2i`@a%t$>pJwY ztY&dWl$4#dDW069N|OMmd*{XjT5ue@h(|cGzKx%Ji#+XpHPa;*(o;_7jF1&avMr%D zfNqkGsFOhGe8vv4ZxkhPg%>pP1TP(VdwZ8u-|$c1?yMc}#h3wD?w=iQ35fFIAJ{9| z8zIrO1GM~}8&T*5>ht=COsuRTU1QMrnHPAI9ekJpdOR0K!&tIWNP$)l50>NeDVZ_r z3oIwVrs1 z2JMYJn4hPkEQ6YGuvl?Ms1DL|!3Tc2QRo|ErDtwVRCE-7hrQa*RZfGZ7$^{HnuS{c zg*dVX!=@CN?x*v@NKP%9`=eje%De2_3YL9ItnxX*z!h73e**@Nxckj>H+-*PdtO9{ z0ShAI@vbwsrVa zODqFVJH5UEzwXv-3t<3gMn~qosHmuLG)dvYn@5y0ONnN>mw=O#96Qyg*oUfwoPtFZ zMo4WYVcIKFpctlMwU=)?(5Q~9tsaQw}Uw3x{XufAp8%idKoG&8cTb+hJ&qSm$V z&1Cd!9-@aocwt_1%pzGqps$NWxe?e<3em9|6Y!O~Z*=a5xTJ%#?q#CJ;T0(z1p^pG zE!APPEyara#pP_<++oPfj9p!eb;4ORbw2`Y#HbxnI?<^0oDZJy?SA31IjW8u-( zyOQQIS13IY>{}*heH9W%-mnSCH7~>L`3rn&Yn_}cgOX^KeMeNao7LOzE^%hZIK{kf zvAUnT3EN@Mdwwue6h*Ff=sKpDW3~ag+S@lJq^966u9dB4ej-8s$K+MVNjTzy%{TY| zK2lKBF@o``Rp_rK@Y_HB)w|cf8>XvY_>LAEn8@45t4ZM=ve-skd@!~&4~XZ63`m{ytSD+T^68CyX;Mdn1qh`;_n5+`q$h4R(XEvc&IhrrY1R=Jk)4-~_ zy*-k|ibyX|%gl~$ z{%*3tqZwDbAiub>q7wqN4IS;vm(xBGZ0*XceTUl1=lcHjQYuWr@<{8rN(-o#rtiJ8 zbSQwVV3H4v?%U6{)sai^&^-NO&&m*%K^v~~$(&AcnlL-Q=hBgUf>K&{x7K{vCd^v$ z6oi;DXng%B1Yi}3l7|A6(Nc`pGNnpHI)+&-1mL+Fu)tdS3G-!V zvaqqa1y~IK09X4qkoi1nmkM#U7TUTb5dXW7_`gf!5MYaBuMA(~Zc|{E<%F+{I9~Dr zSDCSxorhDqaKUDz>inLhxcDQ#8~H}i`n!mT48{DOUzN@G_%5ws=SkCzr}yNYLGicMJ9g?+va_Bt&2bUw`B-=>kUIkpo+>9 zu#T!aH)a_qP>vuC-)ffWiurgM(zSz_z7gUui@NnW3_m$Y&qeQMbBEU?rpie6C;U2i zdm=2O`H1Z5A;&XV#ESiFek=x;N$_HD^;2{6-o-#7LS=BV%PY+U=}lH}!|F(1eK{2l zQ%^coj@bn6{)@qJ^l@%?paFHJC`!SJg8*YjcFhktl_j_!>p1%3_4is}0^KRjLMb_N z*s8Yvb^O9DCSa8@egt8*Qa845A@~0A#=@t5dLvQyh!i~09udZmW0nv8-;F35RvD9I zB2PQLn8`01LeO9$ql$kE-SIToMg85#Qx=v7!zR>|B^H%rgWGJD#nrSx`Q1b;}XC}lpy?v4piejqPlzA$gZcfw{Q8>WA z9Hk}P(Spqs#D)Af7jy%syEe)2*J~sHm#+;O3d@iT+l-JxNriLSzEKD3*Lf9QejO3B{2OavcZiNY&!X*pr6zQaK;8XH2(d;@ScDFx%UV4 zq15Kd@AKzk6qL;Zd$IljFX_*yq~vmh_O;QxtB-^4`o%n#m&O1NK#<4e_h%sxnQL~w z_Nh$y<*ig^MzD6?5L`5i`tME6%cOi^uYj55)wD|mAcH=Ub1A*yb^X1gV8fJjf|sW# z%byLHj?NUqsHk|2u7cDiMBZVLWxWQ-Vb#J}MH?;Q;6=x_mmwdXy&TBEJQG9J#=_@M zz0bucBmQZ#<8}&~@+JL6)e(CWB`Ydw5)~EBd#ryUF$5^TBm%E=|0Jc~)fZbbSg{4U zx-QgzfeQbka%^>?S@#4}RBCETXG2{JgHO_6?0xrOEdpK1*GsSa7o_p~B={LSOFjHW zQFxp_RGgIU1R#T$3KncWLXLTZv~ycUg&Te*U11O;iysQJaLLteCBYQLx4aTZ+(7nU619t@f$Vlyb*V|Dk)Lr>E;+YV@!?r%ohbOg5=m_DM#A-o zm+&p;GQu@pw;(^ga1UKcNS3DhAMTFh8LWfFp0W^7K3!0R_SQ%iOy|JU!=iIu_~)jo z$LZ0QaMy~QcwJ+Nad9CNiKd#< zb^3lBSBzg>e-?#Uzn>>sy8kchANucJe&Zi!PtkVk??;P;gcb2NG<-iyL(yAocebgB zW{{Aq_j~m;a?imQ{`q+6;p_e+0tnu6#hlAiwN#Sy5sh{N~U>yZ5Ok%tq%@78l* zkE6i`mE+T&Bh{lnb8Y2E)pU6ss-LPYXOZ9RRIq2{c;*A|Y-y(&^XU|P=^G{oA?U+!;&w3W2ri#~va8vI;XBsur>4CoQ=a02=<=*OK{%;fG@sF2dV4oarb@FIXMY&_^IgGyn_qyE|~w}yAS}C8LNMQdscA)uALRP_*NV4p2MRo)fOaa zXuQOJ7s`DB`0M}m?85OtFx<0gdG*gA^Z&%G!LT=?zt3G#``_6$SoYz$;?52DZvQL8 zzKQW&#S?%@hj~?-(_!Ir5beQ1z;(2cq5ev}$zpdQeC-fq zZ~eu$0vQh+N?i7FGN%WigX_) zl*E2RVWvYRarSJQkm*bHH#f!#KrNgu%z|9qR~)|6#l?>lV8%<2W>j0O5Hr5Wb^y<0 zI=FkO{E(d{Fkk!?vy)qBB*pW^bh6(5K79#5L~)R?^0@{0FjW5%^Ggw811!F)?>T!1 z1`0q4#0Ws!{1_u45UmCZc#N#;I7i+X%*gA4LFK`Ck%6P*^T06f$!fQcLPwrrZ=ja% z4j!|;Q!@0t{|YbeE9}3@1CfcyCXfjE4Wdo`oo&=*&`pJz4{ZdXD|hI!WcW)J<0`F+ps*4^ipjZrrWom}M022SUP62m*QMMifiPJw~|mE}kQ3>>W6rue7+k zqHr|A+2{oskgQZHmz%#Pl4@u@kfo^e==^3xV<-79AB8i#{QPwM@0~e6JikO|@y;l{ zJP=f_*g-PSz`!7WzbnC&b)$nTdTQ|ttZI^icAUjB=fVSIeh)z=U57`hFQ#vVkd`%W zM!5p~^p_ef1OBz z5gtmx;{Yjo3JD1%CIqmSrzIs_KoArA_%9V`hobH&E0BX%8?*L~U2NH@m78?-y!qoO z483)DDp`RDRcW*_%ZJ)ULS43vr7jhTH27&<!KM=aa;-{wto1vy6O~|$>o)}xA<}Su zVn6R^ppmoj7795X+sdC8Xt99YBsn;ck3rcjFSf*zY8H5qb?1A|Uwy{X42nwV^!Ig( zH@%aeHg|P$bkthwTwqfUI**8FRLEaU?`Us-KihhLiIXVs)HQqi+~T&OxFuH23ez&% zr@y{`)IlVgGLLDFmdh>9>t=5fGVDWc)XJ4COM?TW#-{$yi?ry{BPw^~90 zc`^-+oL;5v^aq^82+;=wJ2xDsXH`^x5PVn=Kl)98LmKaPP+^9Co^RCJWJ7O$Kt}O$ z>b;wk|C6zkq-2JxcaLt$SN@4l69!<*E*+pXH4PQLrwS=hedz=JITz6j-N&-(VJ&BZ zkM5q5b!G0)QhWmN-YU%Q=;$Qf-1O~4(9o(&^WWk{7R-IaA3l8YQR?6sN-FNuLWCTn zKhRa;L$wn;d&e!d-@rg+7`l>Cuj{Ty6-%`23}3+EzCQ&Om9$2>A_&t}fnv&nf%$7s zN6T;uhE?F}NY?At?SUl9FtXQz_Zb&HbIsVLYpM+VgDv4X2$mCsVN)EldGH_v-oi+T z(!57pd8{b&gD*7jm#Osa;HNJ_FB}utO`Q^Wt)Vx?pi`GLU2?_{!^vAbVWwR&Z?)#| z&}gRSw4mpf(vWGPV@;5SFn!i}4Z*h!Mt+ZM>1D3xe#9o8tH~uOeb;@+j4nO67 z%MKZgb;6oPO=de%LkP8j2`~{rp@buc=KhSwV{q$y~HmYwQ^<5 z*(AL;A8VxZ^73Bl>A>CG`TmifhUS^Hs9iSf7p?o4?8G`|$c0A)TkBz;M~sjBCL%e! z$zw1u0SLzp{uS{B(N`k)%x}Y0qV7!((OZ0XkNtjt_9i&_^4>jaa2#8$7dw(wRPMtO zO4nXvuCRe8sG4@^rlrmr-2UCcU&-lh-)og5w!Pa_3CVqF z_oz`KS}9XYPr01#$rYTc%6b#-5AS@SO<+Cu%E!l&P85Q5og!Br)Np&Tp*6!<+-({W zQD(IR2)R8;^70$XePT*_-Vk5)DW%xx_-9tM=LdedjHWXwLvcad#BFmS zU0#jcN2O&4DHCQM+iP$sC3y#@_B(lYwO$OqXVS`cagAA_qR5O!=^7? zM^hzmu0Ly%8@>w*?QU=|M1Heq^*Lo=UZdIXl`=7%D0tZA`#sgi#BbACsJB0VR-?wa z+h!+GjB~J9TvM4jF-gGv3m?Txe<{*WW`Qqr0?k|_`#$|V!@zn@%|4!fdiMlHT^=d# z?YCvtR+5xK`yoch&TEtpgZ53IS=@p;*xtl5kN}fYi&|p$oCi0fW#Y`ET{M-x)gO@cjSwH`X zdrSm`H?%2ud3d^?A2d28Z6#-phvU6Z4hC z!;H$ikE@sZcaVzdhsr0Yq*T-xXx3HX^p$E0(@&zV#Y6r^#&rBbwN8r+t~kQ7`G6*z zbmE)7DFfA|w8+p40-nP&Cl|EOtcTHPsy%05><-%@oDWv#FaE^KYdZ7GUEuEa)_ed5 zscCZpm0%Cg)X5U{%jvopoBGpJ4j8lKrx#Vr_08BI4eK6Ejw zys_l8lj!$X0^Cp4EUR*H#?A3!eARe)yw_N1Zp>Jh0q%;G;QH^=?VP9puBUa$uUQ@Z zm7#H{m+o0w0!eU&d3tzQ7>y3cz*I+g4OR(sMlMTpp)e)3E1~cM4*3V1oA}&l8dZ zm{HK_vv?HzTIH2RCEs5;-{#^a|BtEYg*I}zO)*IL2QuZkNm-R!bob~D2Xe3h+67W~ zd*e*kiE+B5(FT(=9C+=#`*;4SRYLO~L1RQq7&|`;z#}6yXz@2zH0Wk(C|*<&QCh)? z_q$hJ?a24Wov(|%#)FsC?5$ao`1uXG{OxaINk94Brn$>Ds2Q=py_+cwP43&zWL;8y z^Mxidq8z{GZZyE8v$&QakyOz0n>Q>YVpsca{V|42ISq+2y`1+ao;tk^ZTnymu5G=G!~CROH+f>ZD&ZFu!{O+)eKOZByWLi^2h$x5I76N-;6v&0tg95_#` zXc=y%INg;_sZf5ZLwou7^TwN2TEU0KuFAXB1@!W_k`r>cZz@>ATzM1vmFX)+nd=|A zBa$x{BT{2dLkCcmSk`*vOyjNr^}Vc2(|pLMH;Uw5-7wK0Ft zvRwCf*#{Q%RRV;D6|}f-FH&C%rHRO@Id((VBsH9scL$+xSv!g}oBNiFEZXa&Z@dLs zV9&Q__*CjkvpE0ZI5tHg>eI6*>6yfyjtr$GVxe6^OhM>Hft#qm30CV!>}qPLPG><~ zLvVpqGfD@aWcbTZl+?1bUOIylHX@>o(@V#3cj0mdM*psvdOmrEDW)ufjPiKW4n2mk z=4$@<=w5c_RTtmaD>a7~{W4#KohP_@Gi4)lJDc?9 zKH_UCGEA#BTaD5^Nvwy6MVHd)M%~Wp;Io6TQs*lkB^#y-?sXSSTg@BIryfY26uTQE zcVMhuDv>m+p(n6j|Jj6tYnkE^d;Y!G9#K~77t6Io2Gu5zQiutgddniwQ*s{)n3K>e zu|kPbW}|zi1eEs{=@vwT;_{g&#Ro8*9gooq#^?qhJuJ~_-m_`+Gvr(O0eO3}xAuLJ zl3kj6C{IaY%~u=0F2)_sTRV?NiDF>>RaS7a_RK&VBMXZldkr6cw-e@4H4zDQHNCZm z9W0d611}1ahoF~&3F)4)kgtf%+fL{1(~J{X2tF^CFE{zZ=+pl=6I%((U@_q&pt>TK zbeP>-xW1B?%&VW}xMzZS_s)pK)25Es*fmFY|JSpLynW7Zdrir~S8M1sGP(PW(w$(T zafI5Rm|Xe#Vy7FGE0piD5(b@fZl3GymZ^N=3u?Yn1FupQW<4|VM6WLj^JX=9?TAWJ z#Bk%ynXygEbx5x}_dzW80Bss)bpYex;pvObo7EZ2eHPUQTFqydMn8m!%Ua#Y#g9S1 zl(JItf{gzrN7Zd$w<4TM}7C9lsP1X(_IhDOz>2!(s+Z3sRN zMBQ_8lgN(~wzFv3P$dy*FrSujpJ~J&c(vQscKde^iSYAlQZyjc0e34I(bBKORW`R9 zAELdFnKD(AW0uv@uThXlBRTzr=*wKwu@UB&)10DnK_m$<4)G0n{_E?$47({PoHBpubM>Js`nU{ zF}>3*9S^=Ch)-s9sMasn{T`(3dE%^-PQd{iYgxi{CHLC9%-k@abAqq6WxCj_jDgo+!t`GJOFa+6R47i$chnnnf4gn)OlzP0mV>DFX{CFcJ3quY z@p7N>ldVK7bee|Wf0qfk%}9D$It?e{TvvxTpWW+b>NL12^M&?&=8c$apMqo>?3bqJ zF=HCWy9;*(ukZHpKGyzx>+D)??nf@X-<=yXx1R`nUtriw9z6LpZG#9F4_UAT`}F=a zhsD*49{EhWfzYPpZN*s6l`3;nmra;b#|7g;p zNUEU5)Qx`iKw=6h#WT@;nF^6!sp4GvAJ_wqg8gIDD^H3#)$t}J9bOs#N#{xb*yYmq zp~u@y19a>spS?1v3q$t>{y5%iu_1UdmihABBSoj*h10(+JhorQ^imXiCQT^bVTVMd z_3}f(H+GlnY$Ssw6e_zce%ZK;q=%}rvqIh+TA;L*ZgIwn-8>Ye1mx`~ILB*?I6Tx_HEdzEhl9s#s( zQ(s;wE43b4gzj{28qg&FWkbpWX+uDVL0s?Iy5^s5hNVF3y3@@C;~CioT3dHwUWr`p z`8uo@RLzIyI(9rSBIq5d-!bv>KeR{yfcX;5K4kR>BF7!OLB4* zdTVQI2G|q zl#TN+1yB-xECn5qZZz(a=wzjpPX09EIKk6>cq{9Ci^`Q8`-~dLP2YFc<=>JnlOO+a z$g@^jq=rx3s_`VTn$FDEAS+l9d3Oh~gw6fKS&M%liH7~AD5((TBts_FvCGX znYS%*p-4b6cAlxM;933g=jh>%PNQpG4+XyD6`3R_h_O>f#O0*dU=6e?Q?;dds~tCE zxoC@UQ!3tLftX@V)mHjLkIPY4%Xz)h<*kazrQGh<1Y`A?_^(FQd6Aocoou zaO>ZPapF1f$)BVe|86Y)f14A<^cdft zHLdZf+FYEpYeHs{he50Fj#PhF2GguiFBFR8d$cDX;^$$I|ZcdigT2GS%U6 zxDLjyN!}mAvF83V)^$9272O8vs%OU%3@y!Gru;eC*A%*-=~QLCe)Rdt%^VT;g$Ez# zhq4b%)u)yy++?CJcTk}O+$+tp$D+_y^|u6p0GYAV`&qr$BjK`3OjIaZm{{Sw{X@uE z&Zx{Z0r>ZgLFI^J_6#I8Gjnma(3Zgi`j@fQ!v6_6@>sj{qPzJ>r9=_Odh1$xV~R9X zawkScym;SGR`Tkkz6VThwL5`Zk|hvo8Jh>Ead;vf?w=^#obBv|LdeoGGR~cnQpD3NBHo?k&>jSunaY#vVUiK{p(KrC<-nZaGm{^ z3NpHHP*MYivM_&K_j70zG1G)rpEpP=aVp9@2C`ftlv3!uY{1gDlEOQsRaFqW0@9SR z&43D7H9cjGNpp$X-zr`H|4{bUaancIx+ow>DIF5h-6`GO9WN;jQqrA*Al)rUi^P>urV#}?3FJgJ9S=n+81OT@gNz!|4p$?WD~^gk_$Ku(FKI;HP- z?94Wmr876AVHx0~K76QYHv&gmwY-wgpBV&$9{n@tzrCY)4FjrqnV4F^UlsIyQy2dR z^VA5VkQf7;|2ObrSmQo-vW0kLOLnF4M-$a z1kQiVW6k@#JUla2j`@^WuVG(;iolTH{}b=xpV6KNk+MVdMPe7|GO<1%5vZtmY74B$ zOF(x>0QP=V!SCjQ!EHtAjRda;_DgQO8iY;AW}D`pY%%@?zQfHz|Ao>or*|YW<_N3< zZ<$wcr!Px5wBQ7p#sMs5?yVyPR4cU_>wf}H!C&9BHG$15I^Lza7aB@SYsc8@PwlLG zjL?V&OL>KYZ+rIYgD=Mzi*7_b^@M}AbeZngp2?gOiQkG%XpHNPFcv9rqqXdrOl}HJ z&Mx6imXm9!C)89Hv>OdW9vP8zt0nR){|xQ<81f}Pn!EfBT@%xIB;y?K8rM=E%Vs?9 zjKkAE5L`POpJ~XcUKyqxHtxy_b?!lif*o$WFnM6H4-z?bi3(@A#eUd*gWq z=)jRxN5ZC;*4qa|#-Ld*F3cS29OZeYJwIL)ZmWF|ADB#U^XJbO0D&g-l>Y&SrOZG4 ze?nGA8#$O0r&>26n$!hrKZf9I!M9mq!c>;)(W z`MmfrdW?55g@_hZk6A2whlo%5n9H;;Xx;cQosgugaK&5v=HGbBJ7G4}ALKXp zN+|*XwOCqa(!L@!M$=%WaUrk%^v%ASUCVen$1$8fUPJMy3U9HF4H)*OKQsL#KO3h;`4D#?SVQ+@pctfTr4Ed%z#T>@W7a1khKqze(pf9^m{7afu9klw zsE|phUrdaD@18L=&>{a`iIB0`jJk`!8Yk)6j1i{iT>nka&(O6~G9_;AyoQ{^oXo1_j`$`rK1%K(ZtRvEy=VA+r zCHHYV>igxBCpseLDBL>im`{-r0;RwKz&Y(tH67r{HFJz)3B9Y`3ZkGM;xABUB=t#c zvU?uBxRe?CW>$kO-CHLNq7SwRw-;BVhe9M~{XH>O5brDcn1gNTV|0?xl2ZQFq8#VZ zSlTQ;Fp|bfW+iPUXtY&qi<)KbX(zk#t6i~c>6OWDY{TOPMR(cJ(11ko6H|wjdCt_< z8==nP`t+v~8I0mpowfYgfIkLmQGRH3$DYDPLevK98`$b1nr$v1oCL{pFOsr%P1g8x zfrFJfol0Rs+r0kb>E48IqFyL;ZGyI_uLIA2??9GJ;oy))>J)#?#@^Yz<^=CyS17c6 zFaCYwue|Bm-lljW=h#x=lL#dZUxSmiRm1M%2tUpBPb+v>FW20YbG?CVa8o9c!Qn^* z7{E)El};FnC%`t!KcM&Ls|^E%u%wgLGi-CFX4L>HF5Sp&m4z3meKFzKH8n=BYyI;2hsPvF(9-d)=p~#cf&`Q%-YvQDv z9=gXX9+o-xW$g)yZ8U6=n?>&#JwY0t?+l-k6ScX7+cH>=l`}!8by<2AV=8~9JCGn`I4sYu8fR*t5e8Y^4B(p0urto? zQTB(71idrZTcOzVg8$N!@ln4)4srt&=zC_mV8BSYk4N4 zIZsCgO>^r}_GIe}8x^b1L&MlFZWo!Chu+yGZc+uCmyRY)>NY;rCot=GQ{8#^>S1Q> zA`W+DuazO7(GzF$%dewhxI#QRqDdg|Q7{9bw6V)cY5c56Zbs&ca;f?}Hp z(+IuiJ)<>9tCrb0?#W6&pt~CPtJRj8n$xAzxkpMlIVnogOA>fD3n*)jqN3Sn+<3XS zGV3jZN;ypwpMQ>%p~t$4G$@ zf+(>-3_Nl_)?AFv%Ccp>JSyL-^d$sX;A;wSX zaM?3=8b;}}--MEi`DIp}>?Ii%u6@OT^R?(1J633PlgiWCE}1ym^>KoNzXtuS?c=Ti z(gpgW`3y(%&wGb#edcYy>X(9JZ(@}>twq42ERWmp-YGY0jQ#-{=%(YMn{lF$ z=ud3*c1>vXeL~wK^Kb@#6#s{y{h8G+yD=uC2lX!;BrK{0!p-qGT|zXN@ank}^Yv*{ z^w0DligUze&*d7N3nL2>R%Y3IsGyEH7NE~E=;TI*ffxNdMh-vZ*q1m+>$9x5=o2< zkA7X4&HinM_hRlrPv<&G-tzjD2D@6psFx{o4L*--40FBI)~cvVxXK-J@p*m^f`!oi z`*6s)y;jWZ)Ig*~OOq2%;Hv;!06#^2?D)-4&%Y#ndu_(mI$lrKh1p&u*cWt*UsJ#8 zd~|InJkmpXAi?XLwsNe9!Cl_758Q}JnSYeC_i+rb@}boqL1Q6CEyh0ePm>V-y}xv>E}r&YkSzQc;lZ03b1{7i zOz&lTQiZfGB^n2OTBk+VwZx70K~PGJ|h58HTE;j zZ7~SXEMw5R_A|^Vejb!|>gCKV`n;|u$W(E+FMY7R?YsMY5tsG!EbyrYl?|CX^^X}i z+#AL}^HqGwApIwPDZ9aO!Foq}MjwfOH;uNZ0=8}8SEqSwg4W5laznDs82+;2|6F6 z8f|<@Kyb8cmw|-jstn;?N}V|SK8Z^g?c$Ms0Ai~tpv|k8Aka8IM2(I0+N`U6Lc{MQ zFOYGmj)j^3vXI|}BCMcqNUq&8=Tjaucr~m{;=(!gX0o}KYIOabIf?k&Lb`%L561E9 zZ6`L;K7ua!de*~vCIM#A;?b+cgr@J(r_k;6rFxy8_Oq1#@Az6#DRIZ1m?o1=7RES+ z_t}FgjB>S@LL)Vc^YyoLx^0 zR8=*|C*xY^LNLnPH{pIhwu_Wa_$>Q+{vIuSar(Xa;)DFFa#^$fCucM!7zsPSjV;WUIF*=M#_34ZuYN6->BeHMxy$(~|`MveHs)IV)YC zl0XI~Nv&&Z8_22Xjb`4u=zl8=SU09c{3w<&%=YNO)Q+R;l>d#GZ!(qOdE{(@PanIf zvogwvz}7j~qXvtAj6C&|IiAiGA)nlWM3HhOWv3c)X|;M$NEt-kJG4G%nsD57hjIk; zQq)-S7(SX2bXmqOsM>4F;5!39bv$kZ+<{;>kw8*J-h;DBfcqZlvs=*Wba zxs(3Ay_~9G={05RQ)a1#TLS>8Aaj*=$OteK46!TRxGk$RFmA?{nG;Iuq2bei1s3}x zLqJ46ZE%m-OE(hw5=*;{txwrsZak!GFgCj`th}YeL8!PiXzKaK;nsg}J*yTDKEY!h zBmHyOmOZ)i}{L%wB|Bf@_sA!;B*L4z5>s3eX7o7^WzyGv#oI{_FsQuaQeV?(8JgMOJt^ zcWHQ#L$%SFg=+Q7ae7|aa_@w;_pKh|RI@W%&NMTiYowgq>JJ;YX_`cTJmCyq@Ti8Z zh)}{E^o=d?M6DPlpKv1du8bWwnZX;!S`peK#2^&M>28*zrxS)Nl`(qnZ%zNcQjqLK ziWXxDb#`RQZ8MZ^``}Nsnx&~6ZV1pWSNjrIwVQV_VsdK!={?8Kx)FX1D&jpX02*4^A~yPazNbPzg%xX3p*+eZTRH`DdIhT8 z?3>;=0Ez}$AI7uqsUxc+V^LJI&M$AQN?p5ztJ{S_&+c{$3ZpSMaDoc$YVfSA@k-^e zr((KTw%Ef3*g54^d*)2VYo@8VeUISWJeiLypi$~- zV7~hPZ-2YNsTpNy)q5ea;YuWoxJNxj3E^^ za&p^X&5?m*(!M#>o!b7OC7oMCcbZEzdd+`^CKR9blkczf&_Pz<+E7qrao2qTXC9GT zbS@2I*`QSEF9fraU#Z>A;@`Spl#yBAyAg*D+R+?rt6_uBG8v3(-Q}q8iLK#o&)iIw zw@;ii7qB#J=DYIG6W{f*#jL)hu;%-F4(Q4N?5QvXz89^VQ@}xK6t{bK_rlFi<~gCl zEqTzF-goT}TypnoIE!ZrJ_Y7~+|)E4ul&{<03I-=3dd#CDXs7_Ys4?dg5@OpiRNLm zL57T|SQ1sPukPh{v1fK=G-@0;*c$CB+%9WmAUu)HDnQ*I4sJ^heebu-=2LE?N4?F! z*xkqh{p5)u;;1p?yutCCSX#W3%wh17hxTZO1=>y(6DYn%x5*o z49d5MbPr8%O^NYK)17#oYG^k}QaEl;b3r*u=}>Va{C04dhuME~({)ju!VhRI6zEXt zxfG<*8wa8Gj9w^DQh_&@-Ftz9LJiXJ!!7!729K)luMfEXtLaso^Hl@$1j*!YsNIZR zXvanR4N+33eXfb=Nn%NbLzjk=+RS*Z+6{83nXem#dKIFSMp#Fg4uu$#=d+!L*2AuA zrZ<`XYA!ai0ycrbO(yQ4i%$ngs`WyYz}S30*hMfI-0ShR)o#`ndtI%hcs;3jgysp!icsM*K+KP`14+D;d6g91Q>~WFSpcs zz+-p*?)!6|8wg>#`1=`4CalsFSJVpQh|s-g$4Ybmn+S}^qWfy7*QXHvs+@RMJ%Bi; zw+u*=2^7;@?aTWn@}x)a*eTgfpoe&W(x-4$ZDMfQ_Y&7{KFxRoBBZt3cwW3fxyFat zb_1ub%nJH&BJzq}WFi-)=M8uS0w0zhKBcYL`jYx*`S*vVGnnS=Dy*ur6d#+J`BUh< zPJJ^>+tekQ(ejd0@L1{}li&_4GoidgUdlYP%;OWb)~H!^>7@{^D0%3t&#(!P-e9`e zx(H+bCVM*Sl7K0K*gz?#D;^R3!&Wk(fSeTAyY^S6_Qi}E8!IY;o<1{X(wux94@Yso55PSJ#>)r_XqOMy_POa|X z!hU_r5|qilaO%m`-d03unURT|HIt%t=U*dNF6eq1Z{Hb?f#d_>R-zJA<}q%;mSZ?A zL((hGmpH#1k~LXO2b=4+@|Zf=9{Hagaf?^AwZFZHU}IZkz*njEy905(O?q$-AQXWE z4AMQ>H5I%X8h5zqWA4k84)+j#PX9dDp?tlYCXAr`?P1N&e6lxfwU8jE=&lSlepmUD z7pPz6#WyK0c`#A9)o|s49>rlEoN^FYa*co=m1AJ{9XZP4GyA-MWU>DJRD&di4%aWv z*uf-1R-EdmOTSXQ+vq;*9b#GM#E%#}vVFXkjdDwH>0}kH%8IYt>`8>Qe)%bV8btY3 zx+q2XK*Pvphk)_XMd1MY`5AIL`E|rljy^*lJMObyWUn5<{7rFxAJxIP|>TOwqt*fA*lTu)#sj4TOkFx1Iok)UtPeS|O1PYdBbB z62eDRE-B5ftF+NPiw465lP}EZHotqMKEX`!yh)`V>}eBWW49W^k#v@*`N3Ujahuy@ zOXpuRbe_bt2tp{#i8xTOj{ShD211#ali}I{+Mh5U#jUr+q17Lv_g_SDpFFUTUd?`* z!3N~ssM1~)DMc=;&Ks_cld@>Z{*1IA7{&~{eVno&=^DLj_(@}rvI{AXGcpeWrBzSc z70yIV>O8}ZZ%g%vcG8bhxf&roYmRQQ?^maw*Yd?xq4U)9 z*FwW;QAzpeb&8tk!E`Q%7EYEcH9RK}Phe>b)7nIP1CQISq2=`!LJ$j^q^}GH4(g5H z1@5LAeYMLYS5sk;Bk{DMPx1YW?=rJyJNZjXxqy3mm8BLZO5v+7G=a_6YS?kJXpfv#?4`kbQz z0DWcit8K?`-7+yRT?eHK7S*x#p}-!MM6XZB`$R1loC$d?H6Zh(t6CNFG;3xKU+h^a z*c-lyaU)}xG_|fHG>>mesC|uAHLFhdov+P#;|aR-rg};$trbz8xn23!3TB*(=rA61 zyol%OuTL744KH4{R?RBq7ekY0^U;ugezr@APn&SVKw0!;if1OAr+HFttS7mjiN?Kg zuD+NwT>83p*@9??8426Evo+uwdfwOfDy-f11_( ze{L3FZp^8J&BX-pQTbSVBOr<|ivhid)_fCMqWolSR-$h0lExcECxbyMOF3r1lzxJS z@Cx`<=cM0}P^epjN3CSHGLb0(wmCQ5al8|kQyq<-_j{<+tKc{NCoyQms9t|*;M1Q~ z7Et)-v)|H`pR<#_$3TEZhK)uL;A(N=`~U&MPqVw6>BD)tL+;i-2;|fODaPh_1`e*3BC6m zXTomhCpBFBL70te_`ZObgH)D8{h~yaS$t;DBJs`31F{GqhrwkRDUDq{pl2 zP^|#8ia?7YGz_DmrKaepDNUtcGe`+lio9s8Wv`H=WmDVZnTtV(nej25K%kB{tjQ7L zg(UGXAuu?(Z~hyXeGiMm98Nck2Ibj~FckgryQD1F^c+5Y%Hk8!p4BZb$RQo}wigE6 zb1kFrjSRK~c~b^4hjz-#Hh?WGdhwW3-pl);I!_jGrs0AYdAxs;w_x!!25p(00hXIFLOBnvpt=>;*i< z!Z2WkJ7+T52G-~_KHoD~ih_x^vh0tr@RyCfN-HyhOLfdBGH9A1s?FF(iMcbePe1&Q z3NwbdKZL$hI#2%t)mS~qTIsmen2EMfTDXcp&&2``pg;=`Lz}}{Q}zBL7rdzo+P=WA zKgi#e6}eMbNXHNJRrxN|mgkNE*Yd(A`%>>*H6N8WQjd5Um5ja-{*ER?R#g<-r_c zJm}o4wZ~~=kQ9>GKK;W@T@mg?#DWhYAiO%8*zC~}RT50LiQzxqqHlbtAt)|UAN*U% zShQm-Eg_N8wI?(z)~vR!I#ROyK9*s>+etda+;ID6Qj-rWK2a1yg_I3MZSz4M@5a#_+pYtag%fqZSN1|65f+*1FVJJ}PC{r23grv*I4DithONk=Ac?Q3 z9CoG6IAku#PAP+^$_R!`B?ELp(Woo>8PyWxaChQEQ^eEgn=QwA<(s9WLV)ZsH%Fc= zZ&clB9*auQ=p+98B4uXmmsBN5Yw>a;c4aAu;i!#O+xyUKNVI6~uQp&vA;|?MJaflq z7MP*`uvECfx@Sq&Ua(+<=H%XX|1eEU!(@!#6?L-i*fMNbLAf=I-cl$4Y*(qn#b_Q^ zU?>N7!8Z~RPRI_Mn^gYp2d}`SaFU9C-vNuDiawrfm;iUQZl-RdIh4qSQcgSn%5yq* zsO_+0QZq3Xd`_oRzb*_I{MGU8%dF;~V;e?*YvKuc#Az`@Qu5>gIVjm@0->je5yoou zzI;~0gK9UIWxjY5j36m#m7NnGdsZuhK_&sID;eIq7f%0L6|X67`(lH|R2mOQG9{0_ zI>K{XlG9l^_2@>;uuuwF!w)Hh{g#O_RaUz`+NmUZg>K=FN!g|CF?H={d4ya zQ4-TBdhwg=RWeTeZ+<)-7fKTd;6e%u|KPIw;m+I)|LOz{t^Lw>ZDAjk`(bX0@g~s| z&)v;9%Ct4@+?;Lb@4S}ukxkJ$z&cR6%Wgy=08aL`q7guYYU%2t{MjBWUAbKwM6z9m#SPYY-SJ(rTb|a2N*S2 zwh|vw0l{-PU>`8WWeLe(M$Kh?`v)fMAnBh7jGDf14PK27KTcFJA)!pviCxsnUq1y@ z0`G_CMbWW!KJnBEwTpQUY)lRw)-E62vaJI6!+Qk9Bq!Os7z?}gwXHe@a$Wf<=wx1< zSVRG$7rpv=f(G87PF#FaQF*}LOrjJ}7%u;8w#HEJC`Z|y^r0!Zc3I(mGdMvL-*SRw z^qdMokuaTtW?YhpQx~AnncRzMle%Wcy_Y{x`{woBWZVJ88dtjhZ=I&W32L5ZOb3^y zw5DUyypc~EbNarTXhFr?wAIqR@EP;<4TDt-x$12fv?T#??QqXVf5%#YQXdPy%UnIW%LoME!Fl!n)?$Yd=L2PIcL4TBqNvvzc4n=CMI? zg|ey%5lYATxc;orZ%&E}^?Hh`a(hE@4j|_5pF|A6=EEZU3Y#Z0|J(Kpgeb4d+62Oa z+c(gG&9}y^#FLbg9ex9ntqp$Ko?YVhoB7o->P4>@f(!P-iGz;{bvBB3-{$F(1$ojZ zE%7}`8RMR5kzpljD;#7eQZm0|+81UjOv4Mo6FGd2t#O&5nLH(jLAN+X#E3y0>AeeC zyHKV)aZU3#k8xTh0bfCB{o(l$gqRhuwhTNJbsd*tIV_>)6Q~K~u@e{yH)+&=9_dkM zgkeIH=JDvL0>Q8o?7Udry6!;mG8PeMzadL%Qk^TKRp%KKqXioX1c4ANK6E9s3ZO+d zeH-?7IU3V#y;a`Rk)=@?3EYT8b;qQ2IiAsiPxh5WbO6c#*PI*&&$$ zDGap-+p}DdEM(Jeb$bFA#=(*x3h>SXJDSoDgw`x^mjI#*u1CsnaLyTiJD6Uxz`HvxlXXir>N|U zB-qYeCNCKa|Ev=DQ=Ua&sy#>bCt<7kns(Na{3w2T1odBP*V(}nh>8BF_o z+_x8ADBsokWB$XggCXV;U`~+COqR9*JJ*uU@#hLgR&`dF%j^7y(!pE-;{HY%Go$d7 z7TT+&&Akk-__v6b(1Y&nqKHMSW>Q(McfKkQP9USMJK|4#V2#Fp@?Nv(+6Dr={EX`} zl2HLcqT%(nR(RZ`v2?{hej>x-LAI4B!<{%^S>Rb$cWUL_n=Y7{S<81y;x|HVD@xmk zO!y;e0r-JJV zFNQl}dtZ$`#q09+aGA+n)^Is;Y2lr!&tGULXIQ)7q@^0IxX8S0$0{-xva+MqzhCSf zdsNmnQ^0;(4`ZE3)>4z?ou{C3u+{EFF5VQy!9#pvS*w zk^fA1wLJq{R6u?f%F)MY(Yh47@Y2BilmYUh<)^U>PAiJp%#3Pw{>x4+9f@aAj0KDZ zM-);1PSPK?rcCgLw)@JGSgiEN84S6Y_XTCa9=ZQ^9IdV(hNtCV>i4&wd|(%#aTyAs z`tzn(cq7q@Ebj}E_7MT&k5H#7>Vnoc3W>egy0T`CcC5u}FWAB$rj*Z28y$Y_yJdy} zzE+FG6?)xW7~XyKH5{DQuLZA_p*+UotM8iTwWG?1jIRAIfBz0#H$sMT=?gHo#@;Gt zJP$CCZa7Fny<=5_(;?odthL5=rn(7w%zw~+`ARRv zw4mlvVQ~rO9_7U38g8-tgw3lARRyOYD}^EY(o@tg{*rGVtByu7^wi1ZgY2+eQgS!a zl)c40{x|q{mLJuBpM!V8VhpHm>-I}HC=nzP1&1pPd_9Nt-8yIu5h{3@&7#DRY<%4 zltdrhWbTTGODMCN!q*cMPP9d4u~Li&Kz)Xr#LH6}CD8{B;E?rEq>)R{91?z)JJlBbZAZmwZ-?oxUATZwFU zIg(>!sZFZNTBB{`0!9hiPPx9+UCJ&Dj~4~VM)cfa=`IRs-->tv-hFNyH=d5s8r)VbfP$A z1=#_W#-_=#mUPY04?Ex~)|Tcp^bYgHQyD;OcvmNs5X-JToJbk3SO(`FzlAKFRlzGwy2HGQdtVG#{7fsO!yb{8{c;K*-|h zy1Zz2VmJg4gz&J4>9)%0%;?mfA1s=Tr>Z@9-crh%h?N6EO481_gz}TsUf|Dj!B~c+ z>7qFaWLbWfVuUy}t(u+ty$7y&p1%3=EZ-L10ttMAUIY56qb65 zwU8#3i2_rfHvNLkzm&rdZ;-51KT4Cjl;9cn*{`Fl|qVg zt+h%AUU$5U)fbfhWLobj`jl?BDJ`4tolhFRQJ^{gj4PmTr^RGW;2Nbth$|f6q678P zi7oMG2{qa3jp?JvL?*qtXi;Td_qM7Wss2>`XpYrz#I_cO|6Ux_CmitsrR81US<7uQ zkxVqM#?6dc@v-k2@#fJ2Q|9UI$xspeBqx_3i&HLEViPa+)X6hibZi?(CAub^l#i_Q zL&j&}Q#=~#94BeNB5h1LgywkcFWngjKY#tEpT1!s-32Bo6XDCs(wQcIVB2UyCty|( z>~=Pv?`}{3g;_zB7YBZ@WKMja3)M_vshm#nb+b7D3sz40bU_w=1$&p8zLC`xk8lK= zfG&ipRfUj5Z@Mm`R9wtYHyz=$=hSPyd*p2TT^{`q=7!h1S&}; za7P{Ql;QHDZ1wiD76;Q2%^8Xw`GIp@QPEcf5a?t8911dA$hxpk>Mxzg zQnKpoTa3{g_dy$RvljKn{5YNCKCd;b@UF}E!6mAVTVp5s_(zp!((ir00-hUEMg10; zO|rV1oWAFx6rqJDgA~ot-wF5?rwkG=0--9h$h%D1c!nV|#?cGz_c?_Z6&E01TP#`? zpyB`5E>I0-4-u`-SL}d2%%zKaybpnSy!0Tfz~i-~It-}|=BL+}hHhgIJ&D88+g5 zFnA|9OD(x7cF!m#;a|)WfP^4N$1wWB^3x9_3eDy>!FWh(iO5TVrOJZNKB4(;b*Ei_ zK)nT1Hy`ou(|@g`jRB&CH+9Z76;0|aFhZyak^m@@eqFxBZf*j^gsW4TJuu91FmGfG z?i)K8b+qvpdZ7o*EoqUXj*gfE3+;Qx%06?E223F{)o>btT*^s(F@s+Ld3e3U1`^tz zBai`1#6F|WwQP|7l^)sq*ZUId{?YqTQm|j|^;U=mi)wxIn-|>wE5`Vh0t?U}3*;5T&1Q9l{|A8Hy@x=r| z>40W%^m^8X_Ft_C<@kOChtix2dcK$0S})^;&K=2>&PGW51U?-mA0kB_kg8oBmSS@w zI6_Y6-`v2dU`jv)mRB^ezQQ)HPhT_`d8Tu`W6&Bwvmm8-4GTH(U~5JLS>7QqW!Z>b z{Qo-oFu@%rQyV>WZY2!@?Iy}_k8S1d*sK={+f0u^ep~Qc(hUzMQb!0(uULM66~>l^ z_xb$kwc%TQZU5n@4lx#e-7x$gI&WpLuo(+NnL~Muv1R&9)bq^Ym+G$KD43=wp?-%; zsF^e)PPNIFbT^9=HVS|WZ`f%gf3vaR*Ob0H??58jsP@V!<#}Gg#Bf4$aZs`VPk>A%p32_YS z{#aEv>R6Q3Opmi=lQ~@9<0fA08DH+IG^Op%A8pq{med|?W+BnS8UgbMMYoM(R4Tu0 z-rdULS8DBf;!igoyc@AS4Bu>%Z;&-?sSu=lY3N>7V~<{@+Z8fWQIP@x|GP7>ey9~> zbKAbcuRR$!GJYlUm6N+BnuXXf*2TT7`mw zS|~ig{$Nr24NsZPqgo(SzQv4QcWcwFqK7W0#|b%t0F@uJPxVm4Ex!*y zJu=Tf-FgbiUsN`^%=0-UJkJZ*2XZ79;!S^J$m=pEWj|&=nC@aC2d<5z(< zEn&Z^Tr$gfov3&{9S&=ELSi#N7+X&MeiKs|-9b1if(`k?$6ZZ1FWvDzP zI(o6r_eeIKsVgr7YH=IH+PV!@c<3U6yn1k2R13VPoU+WDhwg1c{VgYF5d$0}*<4Je zaD2o#tIci=2q41&VmIUMc@;6f;7a~2DDbn|nU^tZ9T4joMV-NWzrtPC+5JFVTcx1| zndZ5CD4O&lSKzI4vay`&*uZf82A}HxTw(p8xBY6z@3kdrCB$M;O*EcoZtFuw-ab(s zS;=G23^aX`_V8}VdGld?i~oq;`lHH23^wGPdywu7(yN2xD^}*XsCMaTT!nn=TUzax zC?jV0lOwODV5Hx)^ ze!Nj`^*FcDJPQO#bgppEh)W(KSvETI+B-dMZah2g+^p9vh5%9+Pdc?6(N>K4ZbZhuX z&lHb3iD%pEBeHg>|z@)(p4~t<8vq);4ZdD&;P5pU1t0w(RA0J?c6N`;Sm>%Gs&rx#0(ruul) zGZntCbMARBF`Asm45s#Dp)HP^wi|ljND~Los#o^mT$~{H|#T7I=;=tjb)#jU3ko!T0#?C*eqIIsv$L`G)oaeE?kh`sTX0_QdU~L%nWnMf0fEu%|Qbbz}SN zq^$lem%}W*hnk=Y@$*0XdpY+PpWj)8W!>7CvOIqbh+R8{#&0V9n8LVgcHI2DqQfUI za(0UYIUiZBwGfg8ZI|VXO=$I@Z`o&>y1%SGgpFP2o@m*_#5mjCEtDk%KRsusc1L1* zO0IY_@bJ$4Ae4S1evdu=#E0dsV0#u34)u1dSoBUezO zOn9ZEsWGWt?Sx+1v(%s5ipHLfhId%Tk1yVaR|I(Gju^RhJsJH?23tpn7@`L?+t#qB zyhk4r>F(~Eru};F;R-l^=u4+t4LoqmKAVAAhmk1^S(e3Zf>~lC&xRSXfrr-0f_IA+ zlQKvDmV>V;ZqA>sc3Gkmb`fSDm z%l){OuxB}j>t^JM54qrFZ@mxfyQw{FAT-`IK=w+Y!!3c0qT!NB?MfGU^S(#xOJ|e9 zfiTCo*Php(3X2!3oocq{$4f_`s$HD>d%`=uechTCm4^P1o9dvE!&#%9=HGJoP8S{Y ztvv0QvSnTdrsL{K0 z&g&{LcW&oo#nwPC`{Vn`us(1Vh3dGM1uYhrn6|Zy-okrdP}d-6z6+e~ zzrSsM{K#j~X=H2l96h4f@?(YH1`_1%-e1o!OQOL1wL9eQ+z4{jU(V*%@ZM12dpR;} zZd#AKL%M!HnCaDV5gWPc#8d~`MUD4PIh5T7ut%Q!WZ#-3(&B|q{)yDF>AkKZLaGF{jSM}GHregQ6&_w27xv|j=Lcv@SY6EMQ;N^Y+H_U~1tnsalerjIBR{Vu z!%IGCc3K&!y(DT9&+m+N;<=LKy+U4Xm>5gXCBx}H?&mVh|y&tx~ z{+uq=H}hL_E*}djNf0p(xcgY_xtKXwRqM3)j*j}!FTLZoinZ8tHu?K2IgUH~(E~c6 z)7h`ac{6Y3O?Gp?&dmyxNou$gSR8zhOA$1VcQbl~PFg1P;^Jt3tNeTNHoe+&`s^nAI3Ll9LjVs`vddKgCx2#Dn_1@2U3*VsKk~vb*Ih_ zSPp-=uHPKeH~X^koKLC2puyPVa|Su9YZUfhlIJlGwe6cjT+|L7hpi^uRKzAk%!9V3 z4HJnnb>DVPUohWRW^r+DqF10XU7(qVTUn8OR49X)7sA-gezw{f(et-uZVvJPOm7wI zXZb2@ejf|tlK(`I1N+y3uNE`or*SkiSa4Ob*ft4r=51H7jtRyRygV1g-!G^u=OIxA zHiZyBypVfUTlxp|`=m^ecjS1O$<=Oj?PUuzDE{xhDX`IT0WXd2MsYdCQfzf!f44U` zHPv}*3d%WUm-9@V4`e+}zw`pu(3GUGPQZS@F=9sSuHc1fz*y{f#+Q~L<}$B062E7$ zM-9e1jo(z?vjWpUa ze-Y8LkRlU5xY7%VNb|uDn2y zpbmabP#-zDM1dvFfi9UMd{`L8e;XpG$v@3rFbN?e<5Tj$3KmB|)c?)`uiq>Qhrmd^ zjAE_h_~~aH@A|jRLvpe&2K2L1a0pnz->eZ(tg|7<&k#(o#W~KREaCPf_|aeg*mfNU zFCWQN=T=9#&Vmcgto|;wMFB9@FjR_+aE)U`gk0G3&#bJOd{;lvTi%}UxPp??@};VW zJLstg*gX&{g%#wGbMTN7xxGOsU{%luzt$Bc8MkB)$>g?Cgg`RSkCsgTGm)ES^-(FkJCU$n}o}rBVftL!G%Ix7n+%VgSUsi>@qqK358V2 z=DL(7fg4OaCHaTiDAuY?Yp6CCv2S(h$1$SEd4++f7lGK$-cvV-!y)K^anj`tGKD{v%nDcjF0Ks&L&b8FER|xO|-s#1&*AH4&-=E;iQ` zZh^KFfCi7OE_3mqAVz|}3J#Ugql62|0bB93=@pw}Mk?@w>Z>+7Jc{@yCLr4{lJWN7 zW!2zwJ~BzZu+J5NVSv23(e!fy12{6)%>4#M{M#0I)}M@`6*+{I61;*s#2S{|rv_6xjLP%}y#t~_$%o7_;K zg@26wK_$|S&xLWPkfUu3M2Z9!AZE)UQ1z5znHKyxyPc+AmIeZ63s0 zm=^yEPl4BgwUU-%WmO1hUMD*0%V7lY10Itq6!Ptf{1v4-26(yax7Y9C5T_KuU{*HU zjnczJii6Uchz0J2yagR=qy<#lffb;6JYW!c2Zx9Q8(b_l z;3f;fgH3#F{Vx~1@W$sspah!hv1!hV2_S zXi{!|5eNYIga*F-1uk9*re;NnpzXwF)7t;bB8gMTg9k$KSZ~on_G!UKJ8G;ffjW4C zuZ~oCa}ocm=SYY^p$Nf15@KLbz=b@6-N*9I3J2uD1PahC#E=z?5JfO4$`NUd{?WZ` z@Uhkp(_}!^yrs^>D?D@n+gMyd<9))`+n+{WT z*qA4q+5BLH>Ad;3vZ_E($cfAM+!mwR zcvds0TX2jE(|4J7o#GT;U|MWA?YsG`gR&==1KYK=TtgER6IOUJBH}=B2zV^5#4muf zLr`_@uI(D3-|ZVyM<2S7>z%semS|>h|9@uL+EAF+J%pl^`Mz)PY$e zG&mPx1O*^FC4{UzHVKY1EuL+3#CDj%zEpYxEPfLGGIIAQ_OR&fHnmU!tl~0AzL)d+PkGY(KwDhdH|IFa5$hZxsVt63K=-?3Xu2OqKMLuUsASW@YQB+r{x(B3 zi4_9#Sr2qLF!tF)sJIjMVA$*T>S=~o8RT+87X8nBK=I0Tkd`} zSTDZ7T7xA6v=j{`0_!*!=q)l<83L^Jv#{3FDF%Snb0N0ljV}71iV_L97WspseT~fX z4t22(T@RO2RmMV@!alCbS8T1@RFHbx#E~izQc&%17#vnW-OUk@^>nB9YI{-&6x7}a zhxti4a#oPJ`0bnr^7sgQ*MimblLJHd9gvCf{9u08>ep@k;&wsX|I^-^$5XYwal;XX zR3v03Mdo3fXBi@*%w!B(#thq-A&Lq~=4~t)5+U0xB9YiKlPQ&XEJKLoy_Qbrch32p z_j#Uo`aJ)<=by8)*Iw&h_qx_Ke6QilH#I$7gCEjfyoR|X21EHP+kOC9Rck1%`3#%` zVfKDzQszEr;q!^O3g+blw3&pmi&`jYJfXn*3fS^5b2Y54Mfitnj3Fy1x zfQuWV5m$u83?^`ztpm~N+gNQm@Lq#Uu<}85ZFL~XZ7@k2S?bH(ool|pc4)Q^vzgm~ zCJkb_GiT031IOEYTluLZ>j~h61JFj{%L5pOIA}ZY-t_un8K46&B2*oQ`b?M1%lg42 z>l!DR6CMNn;5BgRoo@qez>~00vir61s5>}AupUAGP1o6>vi!E6pKi0X4%%e^>Ji+` z;b$E(V1E5V;fs~l6JzI?eU;bWTOUmf`0&D4*rtU7f`RnFE&D;g`DbS9x0(IaRPQtI z{VvDv+Z_LPl92U8!$B6zAicN?FAS1)9bEf3m2xcsIJ8G>$DAuhZD2$_taV0}T;S!8 zlC}>(#EccOkAbKYNGA(<&0oB?{nDKsF#@D~aM&8K^40CEUryY~iq(n%5_M!7fu*w* znAgqLfy@4l%PSiDkz%Wxn1%O305e*6_UXGf)7e!nAWUD3K|hw^EF+kn3qtGK@;f3^ ze&1y0lP`5+I*x3Wp3nANrFXB7TJkZ-JIf!Y5KVgPOfZL_%SdJPcZ%b}EdGSDN2HDt zkW`Wu7_aDF{g(M9|3}5A3jL*UNl+ZD(ffjFO(~p#U9s0ISE{hVXn~njCWz*&duVH7QI-r8M zx7;Jg@SLl81Canz3-%MQH|ecd3yUc8ulU<_q%#c+_*X?Oi=Nbg?|w9G(hIRSS9mQB z2?aU`p1crw4QMUKx7KHSIGYDZSlj~ceWV^Aes;N{vJSXt?=8#jto4j<&DmX@{@~7h zkCmRAi|cttQHJjn_NymwQruf+lEPfwe6czKwBM1-J=I^TrkUTI{;}CaYw`MO<&zty&67W17m25@2_SR7YjBI z+?jZ1KQQh)oklwjTpHWe{;B)f72aIqmkawn3t*F;ZB%m6PvnYz?%#@`za5OB5Lr8(ujx^c|ZC(~)if5=!5Ie8ymlF)pcyCnXT>FTQXHT?x(fK%JFD<_H(iEG)gxU$L6G&i?{fMz zP#;$cwO<g_JGSC4I{x&R)AfojB>rAe6t1LhvPt}d ztU>y<7n>ZPQ`G#amhaJ8*iwCAj+nk|4*%j4sH<9|&%~z*Th{MOseNJ`>g($pg*`{2 zV0j2sY56`*l3egYC{AyY1iuE0d-`1@52*bwq&|GmBppff&TG`ua2AN?Ox>EOqYmAt zXgNWU#!t_eWLtm|At$(xSPG&T2Y1A~u5Y}D*uU-ok~k$WIE;cx;7XJpQ;WBpA}9QW zXKQ$z0f&F*qF-y`0kv=bAS>Mzye*oqSK0r~Wa_{QZYoNEMVdK?n7uoM0bh|QZa_It zfEG?fEYpgRqKxXRdV5PvL@N=>(O1LkC;_*{cnk?+u&1K=8}M}OufR=vqj{`TG#wwZ z>)t?2+d^=VG5T#`rx=k8yP)@yX^lqS(vpMN`Q-w0?hpkr#QUF`5f%f(T?{89WRQ|N z5?pXH+NA*>eA{&@`Xhu{h=tl0tcmzy$9;zoU#yrI7J2zbBNzP4nMn9_lH~qK&eT!= z)%Na+YZh-pVapNwi)e;pa3oj5d0?#BNAwizYk|X$N_ud9!W43dQFg~O0=M!;+}im+CdR?{5+UwX~IA%{0jy;QE>GCxF@GD6a0 zXMiq*l3lgM`o1j38|LdyA%^2ef=9qrr1@Hz7#9b_2_MBJB%D`NRIERzv;m&-Z>L&; zH5RsBSUsa0rJ@iT`uXz}E*_rPA3uI*dfxP6`^Xhcu>XJ&n-5r>n8d_s`%%3E!A{^r z(|m3DSNDqo-yNRP?0*(~8P4E`4C%0ItH(LxCrQ9txp5GJk&gsf+Z1Qu>c*0mg7DJY zBObZIh{zNbY?d4;(dgGW5WQT9Jl=DAqSuK;9;~Cc6(nJqtpqpeAl&AGla+qX5y}8R zYE_+eCM*}elW9`7m4xf^ehfB;A%46@#R^(dv#0F)p6dS(ZuukV)i^_QUllFsMGS%BQ5@L!z zvSI!8TnR`{slh4~Eh`bsMC~A~6>~A?FvKY*;SQ-OKg;}*G zTv6Byblx8yKiUR=4ip6_1{hNttq#k;W@!&&Pd?0e_6_9)^vYS_2w~?AES{~`B^((@ z|D08V#LYBc75H6F1f2fAG#G>jJTCOBq~M%q5QVt-zy{%UQ}q(qkq*i^6y7NcOrtJ*_m>;^SB3ZgzK{a#ayqYRmV6lzFJ8B*^G+KDy#2yDUZLrn(wi3)5S z_sQ43zg6Pi=9jA`xPcl9hLDqH=4+QPiOpea`FSoj3cT zf}BDb)}a_l9}uRn4wvSU&jC-n>c!VrhgLd*e(J7|1?{ddRk)(Z0yg5o1&HX^za+U> z+GB%oT7Vbd5+%K4^x%5;^zLfLt{SjLPeOzc3;s{dD?h}U-lkiF816}Dh0JZ>OVM3(>1c)X1lYyA1dz8vv?WEo$|AQPK;=`!_s?p7dPo zL>`Jv$lqdYimzRr0ig8Z9r|U&T28@juFPtuN%aM;>s(TA%C-q-A;t;RZJrGyr#lI6 zY;0Y?wt)n%01;}Z%eyh`unuthCaohZ;Sd8jN6Pkv9pbN4(31E>1hKicr*7}j;8Kbc zf|fVPS>#NCik=g^4SV}oRaOcNY#G6Q)_=!p3j9X~xs&4MKX2MSjFniDd)fcCNARZ6>dzondfB&GaSd&>8+KcQw;x zjLieT+YVN`J!W)oVRP?Np{?rY=DBu$3W(!ufR=6rDAC%U>gP{@990amSDT&_o`m4? z0Rj(0d^K^0J&MGTk}Py$`3*}V#Cb!W1J#(>ljm3hN*937Og}j$&lCrR zBG97^c8`f*XyYHg&c7O*A|f1Wcz}}HG{m5M5Y507sIqanDHCXQMhoS04t)^dRiUP{4sDXBR)KD(ui;PT zuY1|XUt_fE6M`+=&UUU{eJ9-wH0xtc}?*p4P^H$KEZtN36NW7fC;!A*sxp2*1cFSXAsjUKDsdwD&!>L3vg|2%iH z7tE!`VMB*@mpu^?_%{)$=V@4Y*`XPiMwnn_U~1}w$_s=Lwi4f#5;M!&QWP=-akxuS z!2GicWnAal2=F1$7HSjYkeGN$E$`44h^>i;bwSYXPhqbGqnhhE<&RIB2 zA=qiPJi$hgU17$3$w!gbPa(|NBlnw9|L3NNMHkf?fDNXoWB`XG1aO77^V8#1iWYi8 zb5is(uL8D%Vnxhn__6D#$5{Ebb42hW(77o@#lkKl5>=a@BiOa| zH)Tu&d&ngqrxVbBn1Jz-1`om%GV$K}{^3LDcYZe1VpkY@4N;%`HkO=G1Eqm!TGgNp z1|zG0fv#t1Vgsy*IuP`5>t^T5C70!H+(7SKVvRmb1f0kxSpwD`q%jxnZkz!X-h`L| z^sR%4BU=idhK95|J@~Aj5rNPYj8J|;XUR=ATlyC0;Sr#Qm&dR$4&wGUGR3+45Uf^1 zuV-Us1lc8SwLHr#tW6ilDK&RF0Lm- zAAov>TDQ;Ova>C&UZhczxY4n&ildn`{S)YAidqBj&p+U0o&W zS&EJT(IaAtg7v`T(l~842T(I>H1ZTrHMr4%Vda94uZdA9CWl zj9kF%ls0e&*wi|k5f1FTHu!T zdguG{6X#3rF^HcBL}ZcPy}E4%bodvk4uZWEE3o!~RO(!iX!fho&hK`=>`Du}*QI~~ z=-SUDMvZ1!0>NPMd_e#peg-c*o}Q5NuN)oKX;L^R*wcQPTUW$GYrt2R%?Uic@_~Uo zT~7Zz+#>}T89%ONd{{bX=C0PWzI(L5#2%AL^Mrm?tVE&B$ED(@9t%xa4W<(gw_YpJ110;NaMW4?p9(uZ_US$ zLlZ#Qdfe=$CT*Fam0GUw)2}PwFTRD453N!NsOON%uBX&gvZ&((xH&kl_8c983oqZ# zZCPtI8nmr9x%f1(1%HUF|Xi#8eRix|Y3|Z#PL)+4)(=BsqT;@bBj#_nd zl{xg4d`=*{TpebdiG1NXa|8=vd>Kq2al{WtFSCi`txx-lq{I2mUlVY@(w2vbIQqbKqm7Pdv>`#B9deRY$e%%r6rZgqN{3J?d3X+qc%2-wJ$tkgJ zBatL;(s$W81Ph+AunpHbZAV|Pz&2mea^-sk}J z@?Ay&bChr|^xg!^@F*uJo(u5Kgml^K2F;@L0TlQWx!0273PiZnLz4T~e77csC)b{$QFr%^+X86L&~<<7_!a=j_Mo zo@ZL$`^%`f{>OGrNif3WvL%8KgLa%Cf_qIRLEQwS7XmmC?^VjjFDY8IU`t5QgT8hs z?q31Y|Hx4OKRMxq<59D(yNIEL;s2`1;@6hu>$)WY5?Fx^FMMbiLRPfs?C|l_a^|7NCcdD@OsUP;b+p|kv$mZ{r^IU4qlM&Zj-axncJt*vo(7Bj{1%!@t;HT9<%+= zQjk>9{9NNVHl(=Zlm>E6byhl66IfF=b;ZU#wj|$!ix<^XM9Q`I7932<;(dmf$rZ0`S{wh~7XOL6j@<9b%VBB;yC9y`i(u z%xg^`*}}KIfV&Tzy*)#a+&G67whd?Wop=CM&<9(Tj>?|8R`)#zAyhmfy&#Chq^}0% zV=ttL-RT4UxWeTHCS~W3$p3Jyk$b>zR5S?( z#v!R6B(FMD0W9y5GY@Yf4r1B#`t_6n_xemKfhR5ts2Yk zQYcq@eIr43{XKh0{(4oE%A0lYQ8XYQKUp~*6}!jy>pMxEQY+^3)v{kO;!zk|3v6%* zqwH2X7YGXWl-cve~`tW6}FF>I=2+bvUy<-2NYhx z@{8?)HXZ8lLSp(Ek=MYVtyEYwMPCKJN?&_IQQg*gEy>B_JbI$`-O6xIegt7)@_Mhx z3=GRPXX?VuOfm{BXvjjdTt=%i2X%oSJ5l%E{3UM-Ko)kKGxUmeDK)pBeydyS)&M!A z<?MS8D3IvFkt#Ck9uB%})8JlL3_RWZ2H;5#nQn4zKLle8-qgzoP>O`! z?IgNv7c^`K1U-mQ?SY6j839;b?6Q_hsSOdzxh7Lc8k&*oAOIkXyzE>S4`o69E>6XP zJAPGjPxBX^qx(i7#b;7$_U1wpikd=nRSUJDCs+f#Qm;}-&P-cJNj7h-9{r_~`9UR* z+>>Po$jTplBmA3&M~)ONY#OoXCt)4PfZ*E-iXo5aluWe1P@3m6UnyxxnBk#t(SMHUVDx-vJ&)RU}$ zI#6J(0QEBI^@C0pwjV(mv6V=0P|(VY@y8PMb4Ovvu*7Tun$SRLn89N1^e>xyPvSA|ToW<`D&A zdE~g($U#c{vE_T5Nwd|q59Z?~buR(|q&&~9Sxd}T4*Ic5M6jP$=y_^hN_^Y_7?``K zHOU zLCCuK&XDI;Hcgtt6I~+F>#fF47PN~&5RNWEdN4SK`rJ~03zxa96S&HI*H!M){y>Y4If$b)^G>1tfkqN|+= zvTa@|7fyScrz@r}*mI8&6aM7qUd)Zzf7DF; z(!gTR;c+D5K24IOXM6qzY0^<+Hma$M_V)HS&M6K+n(iv0W5SQ)Cbcu$+`G=3&ia@> zMkG)0MC7kY?4RBYJfmn^rXY9)Iz0!>XoYpD7wy>Cx%Jn(&wt1AK93$;EHND`G^-i% zYA^KQXTiNoDob&CUc3W1P>C2jNvQKXD)#k35AbNqb0EEO+Q!Qtqjj5wQ#;*hM@wc| z*|7$2=*8#8MZ62+GD0aNx6=LQ@}emR8FjBhx`T|4fY0Cb168f6{N-eF+zHynl=N*# zhckI-%@25}3gV`Z(Rm2hsR|z2XmH}T92N^Pem^B6RAU(ysW!zJ1{r{7MIYq5nlyvT zrXwwBq`>AW0fToJKcf<4`ZPK6YhO;cr_jC^>U1N?O4&z;o=TH#$>Ka=l7B*(-CV!?*lgn{DFx$)7 zzC?1ii-}BhUO8f2*6*nSMRRnx+?G(ICaO*~iqU=-LJLb=A^zail+f&XU0;3fRzXxq ze411hV3b0ivCanGIla?ToyrJUlMQ=&>C!be%_%W?1CISeR=1frRAmlX4MkJMm83KV2mT2$7!z)HJ(nQ^=tfOR1i@q8+$WhmPbr^_#oJBsdTlJTbrh&p+Pt;#Mz_5BX z=UAu+`_0cBt^;P?p=-4EIFj=1TarUi*I_N8iW^0zIG;st)FnCYTs=bOPYW0FkavO92m!!-p4D?S2|!jrcpJ z<>t7nK`jOHG+Ti0orQSXsD<~{wMD-VTp%)H?P1*or2yGVW?}7GRd)u8EWT8R;hXpM zR{)!rO`rtH@Z$_8u13v(%jHh?^HnpM5ocBP$PldW5lFRE>99+UaSd!)f1w!z^G$Tu z1+4ICn|`fBv-9!IT3=2J<1@dIMd5-$ng%IUnxkC^w=^hD*MkNjWR$5s63ho+*Dzr! zkv5h*F8b{$oJ9zuVlKGTY_Z=bN)`c`c?5M4{0&36;Zd70W6+VJ;dEDRiBOZy!W(C$ zMq4|YY^E!TAc|@4TDfV77H&1w6?hyfJiY5#O^nLM^ywQ8ULTolgp4zRacn(8dF<90D684*FD1scDWbbuW+8mn$l74GUgp*X3=h5Gtj>%dJ-&XcE z@5zjikOva#%m`hEF3rejc8*m|xu2WX*Xsbl(sHXf2s5?{k_C3J5TS(VeE3gRA9{#8 zQLSYwGEX$PDy6c7)MnKDoIP&Dmz(bQ6rUv2D7VojFIBbxeoZ2Hvj&)_mu-YXut`p_ zYP#>_Iws;Y*&Xg^W%SG@KF}3tI}}EFYvFj)$~#VnqO0d~bThQ3`|R%kU{u}{U7P%2 z2Ylmj%3NiZG65uPU00@x*W-O@X!8$ezfD~aONtZ~BD@rztKld+RH#*Z zWNw#s)JW00Uxdy*As1!aekW|b4rT27Do%6YS+n8|Qi%<5UE%$V%GK#!V8rduEjiz; z4vbt{c|6}FVPWvXId}+hQ<2Pa*w&YqyDl#!=HA(xFbAw%s;M!xS**^%VykT~RJ+8J zDOJ~DbvZPXSkY2(W*bhB+{%dS-Y7V?xJ<6YqJ0HwX)m0TLUk%fD=|22>EK> z-7k#qhcbm4xv0RrS}3(DFtRHtRUT067O>T-T5e{T?9rAlv-r4nj?(OA3-1A!raY)V zi+C&Z+`@|~XnGmIQYy0^we3ecU*11|ywf9aog?Cv8}2^0tNkdfw(gnXOVgA*o4xY- zSdI(I1A@Z3_}9GV{2UrhQM%q)J!K#CEPIb}CCy&Hv}R%p8K}$|{ty$*_4J+3*N)=l zxdq7?#PB6u0;ARFq^Ju6-a0N1CXv81#id#u<1JuzJx}TO5ih$L@a0s0F5eHjWkxlL zE+K8Brl!EZGCEBKM%tD;CeO%#?h$_W85tVA>^)uFdY_xy**+1^Ev`UJ$M*1ZZQ+h% z$^J#hk{;69fwgaw`9*alMG^UiMdd{SKt$zc>T?USoipC50WbSEzPEG;1T!B`yR8t8 z1b8O|`}Pv+NSa)@_ezoPw?4y3Bq_v+X>e>mh3cug!kBv@1+Bs;>$dWi^Kw^nfrm9J zTF)lJ5W-VVbO1jORM~!P@d2kHhN zT6iGk=G0FxLW?b#SjN@qjy>nYsb8hmGkDHNrAfhzVuZ3(Pp`8)6V<$Cf@@2oUNH~a z@k|M&HOs)guN>kzhR2Xg(eR6+?4bJ1&KP;n~SlUH;0CA;nK_TTWdR+W2P2Fj+&LBE}Ho-|z5--raJQ?^?m5 zD+vzlK z8XQM-`$U?IZ%JBU_B!sJl*XEoMwVsHn`GG>i`OD44ZJU&aa_i1#h^X%)N-*RCL+2H zZ(~lcSF!(?2@bdC-pq^6euYJ=*k5|S)Gj_&I<4=cBXluh3;Qin8FjMx9)lgKueWdj zUbZgan_HBzmS5$M({R#pdu=w|C99v!X+v|$xk zizwFIvvKR{-mYF=OQ?uP!-4kulP~0ErqPVrYok8uow^Rl=d%l_c8rsW9&TjEAZL4t+t0;p823z7FDpLw*3N7b3Tja-SPVVN-pnj#E49 zt#w_YsVVE($D@l2m#aoclPQ{rpxPJ6?K0+RuTn=~9BvlsXHDzf3O|k?DtG2VDp083 zpIl6)&L)g2RLbg6*4OK?mzSc}%iV8(_qa>xE;z&$USTL{d>=3g22XynEipbx{8 zt<)fl=}@_rbYG^YeVWP1CVtr{#)?9gtR4|h?mR7)OXl1UuWFW#K>tX>XHS6+PGOYhJip`T^{w>mx3u1sAg#^r*{Ay(owE4V8IiCtmf3cWtFW{B$$>A4UQZk=qLYitj7#i?mO zpNi$4YUBIxjRbupeogD`$jIE3-Cdhhwj}4NX4Eg75vtLU%*Gr!%H(FCP`7}6b<49= zV$R1*nQl-vfK)=ITb9!4+3nV<)rMc_$<;2>YXblIQ#-*s*$vaKytD@@#7ym1x-$=T z&x@q1HE8CF&Xp?iai5ipm{+yaw{K2D>)Jym4wRnhyk>w%%3L^4VP~$6sVVRNgGWj` zL}N9Hr8u`W?{U{jItEFCXg$x_>CH5@sSfK%IB!dgqW+R9V zO%-siwk%!EuPu_+oCdY@tD7W;EDd9lg+689rO9XJD(833D$Qgt=K_OtQ;ESd6NCD0 zay~P84Go~}oK(S$Tia9tB(umLN=_fR|F+&E=HC0f2)e+f+9Xaz~ zXMn=ES}wVcb|xuk^V31e@kms`C?#G5rGcs{ojJfZF3{CM-|kwaG1?^2eIKRub#+(Sw5O=@&@9`TWM#h4oMOGYw> zJqrz`Lk7`Oft2`EW@mK?B|(of%SQTyTkLnj*SD%4XUY;24k{>kZ}i(TXQJ>pE2l^T zC@G}KBuc*VJ@dG*__-G7!(cv`2B66ZQ9ti@u2R3Q4gH_PUx&H9nLos`<0Wj)dMah#E3sIZjx0X^ zD0Gpdkr&z}IZzV*??@EaM;_f~rZ2~uF`E5ECtux6HNH5Ojyik4jPALBO^$xpPJ1=l z%7Q(~xMr}D)#>`j;DfK~&}L%8w74kk78~B%i>hj+Kf#SA8&2g!@07WksS0}FKb>P= z3hhWaI@9k$vvG7d>#CPdx%Tf!cFy5LM zO4;|`keQ>O zgAUP$w`ZeNnM|BJ8q375gue7M`mGm9g=5L&S#EcGA;$vPuTOk7C9MVRPP`PtJ$#Kn zx6B`@$e+qDei#I}mj)6uz42@p9k{g^Rhj)W$`UOmPJ;c`+k2d!niM^ixZs)=busU( z4b&PuGgFa>U8E7#HBdq?mKFOvA)JJ47PYW(;HUWu9LZ3V(A=Kv_Q~TAEr02ha+Rc~ z`9V(XW3qop|y^(hoXp3x5@77Ln^^QKg9IQIZ!=;kz@fY(nx>IxhEk%*^>14 z-RYqqIfDR>`@P>uPTwM?IVTt%*ejq0ZNp`FKcm8VMz;$^`oY{fma9wrFB8k)(GAbU8jwE3wGEWv<`qOT)SqCIr-*^_B%%QE%FDS;Su=YZ%nOeQjgXG zVpgaDwNsLjOrKP9jh?d7O6~vvy5$vq#ikLO2Rv=F?ZuLe^Lf{xQYLJB6si%5i?bk+ zMQ3v4cDeI#SRI_9<*8C~Z5af#q4TqS`G%YRDj|cZk{+)HoNK@Jv*BEWHcfC0Ir-*` zH_-{Ry_cw4Q&+$TzYSg~Rkx7j4u-kL`-@AivsB$(caK63=Ct{;Kw&&1_sT)W~TF7)!r!GIHMfWY5j{g?va5b(EV6S2i!x-v_;s;=lRn_LcraRO|_Z zX1wEOk39t}>PD8{1MX}sL}$J{a;zMiS8%0db0GuE?BMMuHi&kc7kY*D2kiDr$=GX~ zuDiyGT1X-`@tW|6DYYv_;*Bwyo%_S;krZkK1acO+&;*N4-M8V{tfX*|5Xtgc5t`} z7{KNa(i=BJ-8NCO0=PrBgm*{yN#x%)5xTr}%b(#U!iAH#G4CO^VA+sr>i<-DW@!j2 zomm}o29=ZX{#z@C-U$kPehesPtdqFpk&;8JYo!j)p4Sf7`iV4n?VfOcep{e+Ek*_^ zln`GBDJ(5jvCx9Dlm@nFhX(RTyb>FPvY`qba8tH1a?W<)^-%7}4uy^{kW8A3T1;MG zCqL1PQlLZTg^PASJL4XS0VJYHr>c$CZs$0 zG2XJ=>UOp8)=xKK_X#!QI{&JfY9qTe&jdhPq;>KQ_y%S;v{KJN8mYK#9a4F%)P|mc|HJW65Y(9iV|IA>`r2?1>6V7pr5BJ1L6UxT;A=h_4 zOC12PlSo?xbWA}+9KY9Cbqz8m_;guhl1hGI^?9VCG0`lyTsv9hEi}5N)xr5-+Q~U4 z&$vJOs!}oC>MM&CW7P{#h5bigRcDolopF_Wc3a?5)pRSIV5-FVeX*hxX67R-xX;jn z52|j%+UhW~I+6e%Li1ZClnq!1Uh4OJsU^R&c47>&VV5zKJ1r}KKR}74~4)Zu(-nEXus!TOCaN{@@%lWq?2gf6>dI)xDrUNMwcOiBh&WA+M%4Z)%w~Z88$V-AJWmb-)?V@B>)S#!$ z<;Ak9I-3anHB?FlQUiRmH* ze0{g_+ ztS)N|JsR!8DWZ-E&MMv(VD5Rcp{V?mfn)Cz%Ggoc4@^jGYQ-|GezFcaH!=K&Xj0Y%5pXJWX=l&>UkkLkTO{H|aAL9a>tfuAn* znDX2+2Y>c2#;c8K?WMiEolstXF8#A45uwIM1fKhXJu16)aU^oI{zzohvfZXiTbi{nH{57vHVqz4PhHU2tc z4@R&=f&0?V>jOnG3iaYd-AKEW>c1R!g9|&dfA5;=tfL{={RBuK7^vkxtP*nV=r(vd zhkVu`_a%U`dlv^Tg@u&Y!inG2>d{;cczgMDS22g)GqX@CJ^pI%_eehwXg5>-ngwC| z;c`A@)@W)X1P}Fv_SGyX@6N0_uZxtOVsh622X0-8`2cm=PA868-%x0Uo6Kf(b6_~b z#r5FTtFv5ue71=-y_q>)(7LhLgjM|D)gZus+-Xvt?pn*p%_|@!@rX*5^q5kEPDD1s z-~!f*dU~gtnwktK-a%`6xR4OZUTGQ92nEH3u~1!h12!!lTHXyv)dIbszuW^e!+JntifS78$P&QJ>@{}~UG7zf2_UHJK*dTK(!s>f? zVY~krg}_Jw@<2d(d{h(p?rf!N*p0o)9fnT;=Eg$P)*UiCsKGh=&n~6<4x**HZ5KLe zuYyQAlq%H0PfZtgi<9Uj*Uxl4Mqod8usCfcNqGDA zy3fhrn5UiNCV!LF|4cD~85Rnxr(v&jLgLT=!?zWWvKQHs^ z9Z?2fJtB*;e%fOS|A0$3r$yh;?6g@mhV&!Jj4&wl|4dtWNtq<(oU94`C`hvr6o)<; zt3DUR8ht!CPdB#}HqPc_eQ-B)S7gn7JV~;7uO6u*KnndBuD!T~WaVFa-v6mk|7%!5 zLaX=FUs=fPU!ys=gV@pmzxyL%z`zL8&6N6RHYJ;x&PSQ@AHOXOQt?x?kP*V<1O!|G zj3y={BZJ!d3I#BXNa*&leD2~gk=KH19Yl65*@{$56t<25krJa_-ZGsBy_n8ckWMy1?nimDX8`G zsGxl^xgKf3kLA=ZpO<9XwR!)kz5lCjRHz4O{59I)SB>5vouMjt(|=KmL^nZ(1V1Ya zKt`PrDdMswu&ex=NZ%+z#GacGa{v+hUBH#bwf=e$L?24RUSO@Y+mIi&A|Wt8z1CkY zf_zkygw@8eYdP<)?sQwnS8e57RiE>*G{XQwUi9bc#Tu>0>*VqSI+d?F!`r}{^G z(BHJdTTh|}UP%{IVuF`Q75nvDGb}`VrMQ(x|5R>lNp43KSlwWg?xI%vsDH^;@C`eV zP#Gn46^N|J5Vb+UuW$Ej5>1cmpMT(kZM$)s=D+(r>!1;NPr?4~X!eKk6L`Z<4^ae2 z%VD%a{6IT2uxc{ z>Z5TF>Y?9@7n)*^QJ;V`PvSi#8%;?y1==4$Ft5V4^Pyrl#NNn}Y|#?1wAWZSWZFw` z9NfHyZ?j!me>E-`f8-i~Rc>3j9z-)Ruh}1&m=JbpKe*zF=fq_3yNd9YN-- z#Kpz6APrmKs;+t4%_7r>=z^hvNHg7ciq02e1{=uqe?vN~5g8{5hE^91aL3q@X)%nD zS-EQ(ESb9|sNK2|Xpstr1Q{?R z<-L^JlD&Br+V_z+bB#NJnuaFah063#A$?dv1al{BD-P^JcFP9rQo(}lCSpyXmG%#P zDQx$SaTd_uy)IB;mH*ovqoXz(aC6Q6)gRQwSpDI$Lnm`X$OgqOhV&L$Ll zl{prOIQAEmBz`&wf&wkJ9@FpQIn#a4BA7H%&?)mQrIj_Bi?G^j5a41iWE2flxk>4c zJ~V?{{0Q|X@!wacHC_hpOx)OiG z7%nH%z4=)GC63Y0Uf>cwIGs_quDRL+%UDtjyLkbe9 zLx@c08O6FlBEJMe(%e!$ZcwGH7ny@$?2kIkaYH)Qp;w{Rqwz?v0%hf7!eyJ+cPLjN z$=e2ns4P$FdW=cp&RK#Dc88=SV&o^FKAOMdFih^DK1}}{#~t(9hA*H z1RX#~M|{iIPmQ*}dg2Ygk>wE(O2Qy9-j9k<5#_UQR_;He(O_Bt7?vq6E$;u*uI3#T z@0E$S-7kZJpxadAmlqh^daTp_BEYtBcd0mAp@#V4VB1${Rzo_U(n`bvwlGytR5{V9 zboASvG~|%TtaU1R4Gle1yqOiSz5ctITQ^7Z0n#D|Jp*TMGrHgTPY3oNO~C&jJ^gWT zn*U8K{{nWw&HPq>e|}U4HK;+xK>O#Agufa6y$1I`Jcs}OErlB5SL(d41Y7)11&ZkB z%kY}pI@-|D{>ybiHgpgT@v0fD+MlcW%g4X8>i=y?f81mYGRy3^82=yb2LHJAzaoPF z_zJmL^l>mYS;cOY|Gu!l3?2Bl9sS?_qW}Mo`Y#gmzx7eq?-IlvmcaEFN*~w<|Ea5J LDHoo#4Ea9*@@fBd literal 0 HcmV?d00001 diff --git a/docs/design/images/2022-05-19-compact-table-via-sql-7.png b/docs/design/images/2022-05-19-compact-table-via-sql-7.png new file mode 100644 index 0000000000000000000000000000000000000000..64f8025c72596a88f7d6d30450b763166d0cbf56 GIT binary patch literal 122437 zcmeEuWmH_xwkLETSONipB*B6Q4Fq?W;1ZERYjvH|bxOK+?b`C&`w*rmFZmq(H97(U!gFaUF=YeFepT^)=01_Ik6YZVZIQH3^){vyQ1LA-m1TO())t zhnXKt?^O#AtREZUsA+4n`LVh^ zG&D9KzUa6sstnl8JbC!!bP0NeggCN=N&>;1}MDSz%AK_V*;e`mdutY}(#xir#A548Bm?U?;P9OP)-b$$R#| zFz}EV;B^JTdc##9VlEay?yK=z%N;l8<#%-Z8ZLTjH6_b0f?qCs>!0{32OWPg%H_A{ zk1PLt=O@r!>SZJKvc2;~XxNEi_NVI-D4yKE78oc5EV7TG>Ph=`kDp9+YLsrG)HlJ-ToMpz5XeTs*k`;q^VEdjhtP55MpyF z!;Sm2RtVj}4}q(gEs$Or#iadD3Tk*O=?gSJD>I!xze*5Z3g0EYg&yxbt3WY{c&fJM zpZ|iuk2VI&w+}@vEVMD#k4BN=I@Fbkh>y@#fGGURw~TbmXY?n88l3wL%frYF6O1Do z_O>0>fN+?Ev+aqUAm3}xkN*+kEI$b7#S z;cG5Vf@=KJGTh!DU`@aMP#XOP(RA*~_2*@OcqPqP#V33EqP9!_8{1c27i_97M(V5| zNhC1IXYXp2e+&P1{35Qjd$^nPmEE#nVJ|D0^scg{Sb%gP#|PGqZ-DZ%atdva&xOzu z;n^PH#Fr`q$(L?suKcg2bOIM}p04x+@@4y8zVWIE|JbW-sokJ0NWY1Wtb&e*63wr~ z`iZU=8=qC<&pw&HT{|BEO-O2S>KlY`*U00;UF+Llyhy{gcX_E7ya@MQ*Ecs<-lO6q zPxLc6`R?Twom^xQVB?5flkIUb5%E0Q_HTdrqRF|9JoA6|6UpU0Dd!UMtScdNhFwqZnV*<1Gn`tX%UU}Uuis_@R!r4pF%tLyMrJSJ*D~e$_`um>13-;ttU^b z4E)K0zW|c36zcOVijXg_3?jM6ZBTL1ghalXQ6fr9vVJ2H``Z0&yPNOFGfn>z(Q2~s zZh;?Y-2Q2jGv8*rMSozp2YQLH#E0)>Qx`nv0U2jAu%XL|bY*dGQNjq*qnU*1;dD-% z;OGn?=IqH~%u}kF2t--p(XM_L)30IWS*uBokm2cxE|5l&?qw@G&`CY!BN}hs=+JE|TMQI>!!4Des&um#y3OpS$0O zggKp%n$Db$H2bV`t?`&UM8VmR^)*!=sX}IVe8JK8U4rs*|m=QO{P} z`1&(DxncV;wyC6-^O@6t>->g?>yN?i?r)Bg9MN8@8%v#z#ZC{&*)J7x65yH-&$w!Wc6h2ijW3r2CWB;2b~3Z zhA3bNk_dnK60SjFA7|LMY+uC9|83#h2XX~*r5v1`332{#moJGDoY4}Jl2N+c*ScSw zzjEch#cy{oQ14ubu8d55=}MbHNJ^mh>VfUEQQK)yW!PyuT37?lA{8dmTE zm)t8?pT^I*(3Nu&`-%zEvp-Xf?2SpcWcPGFNx$@d ziu_@i74#H8KOuTZOW@XN6Z%~DQ>z^xirX|8**zbBBf!jIE2-=2W8tIY4;D5Nj?<7# zqY+>qc3O!K+sBcV4*Ovkd-!8K&f|yD$UdtWhx~*_0fyj?*1F4a<{DN6D4Z@rKvq84 zOF#VlHR}gUlRkz%%@b4SLCnl1 zM@(b+g|x+u`avC*i!%4Zla$TW{!IRdBZXzb@p!X%DH$84KUzP{-!?14ep-LK$*ziM zPCS-jj;~8hNb=V0*YaB4{3fV=JJ&+#DLt!Pmis0NJ1Hz_UW268YZHARuY;bXjHs!? z2hZ@+c2SF(f`)hVT#nc1>)zK_DN*5D;hj}#mPydNM)}$U4`Vyyd5_hzh>4cPUyF9e zhVafAIqKA24(}O;881tj%fxHC)S_c(J5*dN(?(|6<(y!Hd3-jqI?4K>LbNf*(6nt( zWqe_Lyy2@WBUIROja{dq>!7)j#z?^s>bRfCb5~!{q+jNB7~!3KW3v}Mv*_UG4mnrf z)te;!LCOG?+{vG~U_*Vw^Lq5nY$|ywIOQ${i^PE&w)*pGxtGF2GLa;gDBTzKpm9{b zuo}BMPvaoZXOj$fxGuL_5p=z5JiN-o=*M_Y(8M%R z<55j=J8)7HXsv5~mek11u6tfLHz!bZx0UdVo~ATJo8K|gaew9P`|dtxtnFMq_qjL5w*y^rEq!`-1Xu`aPBNeG{ncl_nnf#Jexl7f+f zfa{>|M^Es5|DNWVW6fr0Lt8yl3*kxjN$#4lo@)>3jBkmr@x{)hVBo#gzVz<47o+$5 zgUONB?;76wFS4VsKV+upp)V|c-rI!W|xmT8Blb(0T zcXv3Y@w|WD`L+6+m_GSqf`R~ZLfHCgoAXuaJm5(H?ad@2(mk%)dBl?yWFf{|pXkF7 zJx)%A1qaq+7zD7U>?ht{G0OFf}-b*0TB0Tx49)y4pYL0;P_crpt_3+Hl&PX0L2KmY#rJB{7U|7#{2$GDmCb2HTKo~jovlE}!AmXTKs$nV+?GgHMPvRr3aR`_y_4pMliA|J3o-;mR3QY= z3qJ&;f4lYHKmvQ;4b&_cM7Y@a#M}P227mA7M+M*6%=A5)sj)@D{2xaG8j@l7tF_#n zr%o1qDrU&%Li(SlL_~!h-y%HykAv_cBEj~wxzPVZZy5x{NdNz|8a|{K@Z(?0FaLd! z{~YNBobbtiSYfv)0wN<8k-YH#K%~e1h>bF*K= z_i6nl5?ErX;jsC`q@Hh+rhNyVR$|Q5D^QMPGzW#m*Y_AqwfY-`yf0{zezEp37?#%W z6ggDB^^Qb4-JhLwI+&~Ltb$y!VPBrl+746>31}(D+0-sSbzF_)Ou;1PiI2|^d(kbS zf3=>(nWXDsUaf*0ZCx|(VghtTUd2WPb>P4&Bbnc13gEU7vkO=W(XvWoGsBL*1(9 z+xTwS56==sjI{&xG1jDA-QOJIE}$2Y(twD0U0@G2GKHg}So}uSGZyssdI_LtHchmx zhw;Mv42zIwuVT-!Wod`b5^ zUyT-&m=$p-mH|nUdc*4ev|(Vwd-uF)Ady7eBSdaX{>B>;#~w6)w(~PnZ076VLUN7B zq=ZH{f$#Ft5oQvMXjb*xc^lB01v)LEII0<4&;3t*d5SLRsErTzHzv6WYT;r+soahx z=wIMLU|g>ADZ_!Q&CF;&wcRMlZ9R>jpW1~eSvtZCc=onB63mP%wvP{lYPS6v4JdYo zz%@@1p%eTh0b{wD> z&#ba}4gy0x{f+~H$^f>h=qG@XrRnNI-FB3e^e(mP7)%B?BtSo}aG*0+bQ6Nr;%t&##kE>K#yP#)FT@_&REh zH5yn$y#xp-t}7>?!a~K)Sc*O|KxNF+8+)KVeyA=4^M#O*;jG+$W64<&QvZN|d-zL$zXHb?Lj(x&)juAnjmR^^hG#EJF`WQI=2}t$ zMX{je{A(l__yS=a1=uKL^Pi+BKzsWwnSpB;)~_| znnjKeCjIZduTS5-b-s-qi}2g%lq~1}xmVGbV%u^zaeKKMZBjL^_5lj6s*qmgp#Idt zx#0HOX@6$iX?FrTwb8+$%a3NJG)Likxr}mh)J00LH$;`F(Z*1>7TcLnw^LM{G6B^h zkX8Q}1e>S8hJSp8{KH=e8P$yiZPvD>sRk~B(eZ;rm0}wyif4y*s71SRNZ^KHTdvUj zF4JS@U7!70_cwMZN^pMQuSdDjHOkUO6c_b$+@1aw#nblpbIz~)!Ne|8tx>#p3bd!& z79kr3t zuJdUQz3-BFsNFZ4E*2{9oOTNH6X%}Id-AyKX*gV4P<>q?aUbHpc*$O>AaG?gTSDwQ zH{5xl2U0CbKB$-O<5zA;chZB~z5^kVDe{7#sxV)c3BF5C+eaxq!)e7{_6$#o50L?o z4kAGD7dz|w1IH)kB?e5(rmh3riL{ONhNJ)XbW}mY_H?$|I+c4JgUci*l7saL3MQN9 zVN;E{l@)B$036hxo1{DQwcGdpg!F^W9M>ouuJ84>e7cVRVs7*+Axj(I z!|h2YL7{9YCh^`?ZDfyWruW37F1rM!aIb#-kW%$FOmC+L#2D_m1`x{9%Mg`aM;{B=BI!&TI0e-6^v z02E-JFJDk!6rewD!6}TT5$1d~Yn$>)SGOiCKXDk3{AcljFoats#YzDFu}UcQgJ~2^ z1jZs_V-;KMkorSX=3O%&4BUiC`R|7%Y43{t&07$QsMHBct%gV*L-u3aVRqYn2?Kni|7L<&@X#O z`I$aEqpg4ds@^Pnx&xdyM~;m4nv)^H#m*D&8sn9^(BF#>_ZJJArs?Zf^h?vhj$RKU z9Hobow78sTyr^cmXlA)l(!N!)LweZ54F~nRe7?wXLwbc14Y$XA`%_AKwRVf%7t@9~ z9^ShjJT+Eg#aMGE+g_o|tA5kZE9VB}n1afWD%60(8sPVETQ~l=6&C8_ADBOL=0LQb z)Nvi~@E!9$Nwsb6aq_Kxm!=f`xc%60*59GTsAG=?gWQpdTP~o4#YO+k;1B_5FxW;v zJCS*9&dZhRx54}A;?_KUfP@TBb0zrrIRDQyOz`2hvI^-qztbIOP&;PH{mB@rS2W`hNJyo6I4ToqfI z-1Voh_qo_{ATmy!;QSVNxOKEFuLO>=roBj_AXB%pBvz*H&FmtdoOuerXEpD*)h-F9 zx1QWUltL=UK;GmNi&(%eGY5fR8Y%}&YS}g!2Wx8zG66a@Ig3I@B2tq0L>;RtF)BIE zpO^-6RUGm(YH-$BMM%$czv^nRK3G!B(IYvC2?VpQ%}!hW))$S=01ouXJQ&SWg^{=e-Dm z8cdPzDu2cxg0Tb{&fIbv`)+}ssMms#u=#QYBh_Wry5>1a)SbXz%P2HdUKLaEz3X6Q zHI(xiU|R?Pr96#lq4m3HSiEDxwe7%DKgAThwc~w0{hWaYl5W?AcA#kD`KJ|S7Iu7? z6c3o^DSm2$bpPbCSceNphB;uiC!Uj(j-I~-tnZ`$&3vo4Fa;PDfJG`nB4xTwjid6s z18?{}U>AeDhf`3o3W(o^!)@^3FdxhoiuGkQ9$OG6e&$Sf!{TC zl!dZp)q3V^XJN!0f!aMIYmh8ZK^PY;;M*a42$1K`EuP*cE?kvHu4OD65zxJ-X@TyH zA3M{U;?8XZG! ziex;epeZ2rxy*_z<7`HE@JL~7T3N^Vn8}zPbtN;rDU)(B2S|a#$k*(>_wOg7@$V?Y zwNO8N)w)9;M?r+X$XXc$e$LuYp2;(8fzo#p)1C2YW}&OiN1bmbvh15*?;x_I$uFJ> zy@tWNlO!+GZf(sx{L`GVh7i9=n+XD?tvp4+)eeBJ@V0qu`|;tAY|ADO0z400DehRM69&vg38W*o-S@(eik5=2v&uCd!jq`E0VhY+7wE=Fa(_qDHmE|V1a*#(? z#tXXjR*il-csHw%!NFr!?NuK}F*?oslaGI(ws_&`Wna{F-8&X;IOd&H`oK{?0Q3C* zY9$zp3=$khdBY0=pWkC(n0Y8cJR^{@_2FZE&pGGd8IGWt_AuOi*ZI}9&v9}b<(qRK zzZ_(w;P0X%!FJHh8uqJGiDg>H_1G6kJht9?^3g)9>(&!9?#t@!H#>6~CfG{T4c#st z&=3c(kNJ>b59O$b!QPLYmbXt5=7&!r?iDIz{m5r=IrF#*eC5-U%|fRN4Ov-P)Ak*n zCj*>KfHOaKjb$I@n(kK(FpQQj*6|f&Nbvp$tIT$>`8={TajN-z#xnIR02h85Q?_VCz5GT+XiyI2& z3VQz=JRu#G?0LHNc^Xd(3khZ*y&j#UAk=WRk-FJ%&z3G~s0mW|Q=hQ9Y-QMsW|oq- zRw{FMIe|s*>TP~j`m-Y+7k)nPy4dgR-b0l%E*(5NSWxi^`Kh-XTu_bAbw00@l>x?o3s2lMg1(7Ma4}!Y(|b_0sosX{1VfD=EavLzr^Pexb5itC3P^SW|FPSV8HsH%ueAx68#b!QnnuTV1^zvsR-FSQp}nRy@qhp-Y+>%*~hC={wh(f)CRb z!&p@YPe9=EAIDvNG!SE%ZKQH5fe*(WLng)?hMmuBfB7L+AG%H(Cz%tK7ts3W>2CXr7%H3<)=&B zi8+zjB*}@j0Py*a5Fh?hX7Wpe;EvB|v$yBXvFt^H^Q6w?LSPzIG|~snB|6?>H{piTUAX@nONr%lU9&-t)MJW4ISk3Z74 z^`uC)K@69UN2!HkrDQ<(9DJtG=*UaaG8-JG9neC>*c%zEX{WpLdkMMDknzXw7v{Gp4?+DSuuWj zEcq-d-O142aZzEWZ`$iUtrhKOz(}2Vwly8PN#P~1dTJCblD$VTP4~?J>&;=3$kE!3 z)CGrTZcyOm4#`^?LCw{kx3*l}%1eMPS{a%zo4lgqIKaLD0BkniEqD|j&DT4{iI&w# z1P`~Hg8Ow=^gru7$ROM}+f&qjF1>Dzp}rC)e1kc_kmk?6PC?GIr81z{uu+>r=CNrUq|%H-nlbMv6skSCYG~`*iPoH*r)u-1`0X5og&aZjHbXUK+t(OYg1<;s{ z>tPF%Rz+DpS8HT%$hJ=|>8FEx$gbXck7;DkuD&8^K@(0ZySU`hoMiKNr=dQAwS4KL z$|lrWz&YHn^U9_Ifd%S&ctiaX8wDa9BeCHtbG&EjoJg?i-ILlHLPW*`ezCiqV8HN$ zJKrGC_Ps-7_W9ASQ_JtO-;+l*zIHR+d6oSL0mS>R%{ozI+G%!N~KpUXwh;>+-_ zj9>A(lnm&386D--)9RVxZIT>2YkzH*lga5bO+Z& ziap90Z3EC9Y@PGY?;iAV`eCf!^CY*gyN#zej)x~S<)vRPcwfvzZL|N(Ug~@86L{+M z&d4|?*&QrQ>`K-huve5gPRf=a%~j#Oggbc`e@%Cpc~|MM2q5mM$@tW}cmz&5yY;GC z&I-e)L{;P5G)cz7&5&J<`>hK5adOh4(A)WSbU-e5Q=V{vz^pfnrBxC_5tc(^rcE#4 zJC%#8a~1$Tdx7a(OamZ<{tGUNy3zdS-}!ft*!W*=Tw4Ju7#Yf;jqPzc&U)V3XhcO$ zs#mcWgQxvo>Y1W;0yZWXL3oo5PizA0L7Vt~y=3#Hp2!R$3Xy&=n>RB@#!3h>%=d0r zubNLz6az)>m5NT9c%)}{(zu3ON;tkl%nP;d5i0GcAn_=rRQZb)=%Yn$t%`fjg=_#mRsq#hQBir1aoCDNdJ1#M z4RQ`PSgo_{12m}1yFL{(gZkz`VUeU#cE+fsfQHiFyMZ}Y7_e+e=$o|Xt#@|{75ce4Yq~36u zRy4CdhRUz5^UjmT0yz?q6>3c287-{HivHN4QiH(TkKGnxQNsx}xCln`>%D5agRGrb z4Cot&Br6I$+mxPe>j~=BSKXnhUM017Qi0|$JuAKo$Uei`V`w3@#gT}lUwyQ7?CIuB zH~$(b5D;iJb9n%;x@vo4#<5sGzX=@ha7SdIAGM&+YrVUhiy^jtG&7voz{$K)dSWdA zqOpFhnS=MnscE-<1oRT8YW}5h0@U(T08B)tRL0f%5w4tI-37uBgW&YPkxYlV@lSxY zshx4NK4Sd>RVnY=S)EBBVxsvz<)VW?Z^~=!Pc4=3WVss7ONs=%^JLsX3`9m*%IQUH zcJm>MFc^3fW7w<^BCMX~sX}#9jM;Rx2E=0!6Kzmd7@e$D7mfN^G*)n+cWyJV_ZdaRN@@0OA+HFVS@k z<%Rk-B4nTdbTWN5*IQ6>S!OY z)0RHt)?5=gA#-E2A3k;L-XW@4&p}J=I0+)74Zrf@84!0kypJ~A1ti)O{a7Ih0L9z+=+7cCnt_M@AjjQsM}qH%&D+^4{aL3`IqL-g zc&t9Fqi(TW9=beQ)xme6XOYQXka%NlB+gkS?UUY?rTEBDB;g&#S;6=g)SY#F=?~PJ z8??2dw<_}79pXJ!0C+)*Q;v&*D5sY2x~qcC@lmz9-5fNO$tBqiqeIOKS&|jt;|7Fu zi&y&e>-F(VKi&;KI(&0&X!RLw>-gcDrHnKX?ykFfNh6G7xq^C2V{K$tolx~n>oC8( zn>bxMT0PB@Z4wPDAtCxiRo0ca5UQNaSY;U`*pE|H?=f6NsK7P1G!*c8lx>wJ*>Ht~ zZfp;E1a%KuRYN1N;cPm}Sh3HaKX>m6yN&C4O!#G8v&D2l3Qv=z2Rh{apN9PPwVXLOtWW$g6-2@=m0|J z8GwFL-JqE;9eq@hN!SEbxw!(snmSqs0%v=bLo;5dORq3-R@t6^ib$$mD&18^W|Mn< z2%z+gAND2(6293+_$5Zm9}fbAC>vVQHhdJm@^|CU5ux|jd1G z3yI_ANoAopt~D>D44>=mfh9Z~IEmX5B5#vjc?W)~t$mcfL`B|+2nqPTkYHjZvMIYe zU|!6QJ{nsLf?>Uj&|llfN$Ulr(-cu$Z6tIY84cBQ-eMQD^ELPy`unf)V;i=7&S6T$ z-tGPR$KutBey`JEN&Y)u-pw?}MgZXJ>s!x%pBCM|W@Ws|ledTg?of-!hP9ho)Xssu zlUl?t9LkiPzeE6xXi@4D-vM>3#NL1i3??w`xw58Bcr%;ITI_Am33|;9IJxaYYxcf7 zt4<;xB)*HyvWhTQKk zrS^BmM3a3@c!-SQ*b*c!XfuU$(9A4L%0tpaatk%CyIJlX`WQ_~QQ5tZfpC$n#2A`c z#gd)*oElcPhu!$^A7U_(hxW4^DXi`AGKZ?2ypqX-#*;x4f!N1jxrs9z>tpUKKDVDV z!ov9aU`>^-HBLF<2S31-TsFrBtrwHl9Tx*Jh5Tu^!B)5OqxgJJxCl7zrjtlE%0dUAe-q?JX z&bL%qRl--Us2@ACyO?(+*h7%>RpkaER3h zK~HUW78^Vc>io@{b#Tpc5S zal_z@Q>t~fdFT8*LDhf>OR)zX8DwBJCb2~c%?$imeNw+Vy3`nYP=svOmi@gn$I9GE z7Jm^4-B(VEey{0lxfi69=W1fmJyAf3xh(NQ)hQk6Jo15bmWEk0U7VTQ?0H1kHE}!7XX1c3x<@kusHErh)-Rg~E z*FS0Z3Uq7NnM*r(lTakvDw@5gZ?(Luc$)4r0hV4k_boH=ByydzKWUmm&xeNi<<)Q3 zgwK#KLYK)V@p$rYAZinsC|Bm=SBOuZ1}P-m&PP7bmo-+b@%2h;^m3IZdD4Tx+!PV8 z7hj?t`R1Ai_tp&joyfy-T;PI(>B|I~Su2~3Uh(aAXjUF$2&qJ%JJQ8j^&BJq(y_f( zn5P3O38sS)lYMK)s2T>+R%XjWZ7o@AKV!g)uk;vfO@)qGQ*H;63ApfhT{Tcj(d&M_ zhp}}^{%*2GK~~cNmHucFbyI6iF*2zfkDGBEc!1%@8ySZ3AR2pFL6ShjzN$Givt=ib z?%nCi{2Bm9Uh^{CJ+{+YFGB{U*;eRh)K9F*$Q7$T6)m649oa%ruoG8UXKod^U4?yj zJD5R7H=`3Rj^6RXFKJNg7)#yOx0`hSZeA_`KSm=iwW|3k{ZqRp+HXVZhvAzlKk5is zO-B?#kI$3@sIT?FN9<;It4M4*SgmXhJwF#bcBk1YM4k(L|2Sqp>F?#K3#6J%@Mr~0 z$#KrFZTpt@0@i!eu}kc6JRJYlY?&l?{Gn&be!s7+U8m)Y98wqRd-aWhx?N6aG$L-P zcBJ`Q*&w)Gy=iT@ci_?5RJ&?*+Br;jU6>wx55Kf4&Tnv1ij=+!c^1vvB&L*HFA=cG z)O4z>)!2aF_JFHBzf#YrCN$bt0|YP@Sm6v)=-IXU48K&H6-!!KM1c*r{D1(H4AmO}&h<}7UO*X#{ zxfkwH7R9{$ja4^Y^aDlVPH}V$_UCtF1T#h@fu7g!Xm5~V=hyOL!+OV7>++)j^SNdc zUBHXbQ5c<~mi@kD-YY`%TS=1G*jM!>uQhp;^%v1}FddL?Fqk_bj8B8}y=o`lr@$h` z7=8$@_KK)uN{~MyabLquJ&aQjNIn^GEkCLlH}3rUs(P@(X&A#F0*_(zVT>yDzAi5# z^$%LZ{Q?&1W0#XQ^i24qR&FRMl7t~IG+K*}66sqL^M!#%rxr+Ou@=CwqznoTY&WVo zF3l6Jp_KaC$JxBQ{8phY`&Rzr&tVMIBdH?44K@5Bj;E&}F>8$aDLc>%L^!a5s+_#& zyVNhVL2$v^cPHkMfOTa0Bp1Z}01T#|ca(@KE#7!*%o@xdW4;cVG4!f3z?Qm=chouA zoqa1POlG2o!ea5PjF*WqJ(Tib9ek2OqZ`+{mWYg6P3V!U9$7$9)6Laopd|6^pBC1&hkMbQ1@+I$c z-=fz0f>MX0rJbw(4?`MJty~pME$fL|RwN$wiM#75BU@<}2BkW7t75-*0elK=Bgwi2&V`9?X5RE4H|W*B=JALKAj01R>|k<558BI((cI8~{_6x>zBBeis*6(`(lgZqdDtH@73Ki&TX6n`@ost{hZIQaD~cJ1(c3#CE8G z)Y^Q^Y(89NmU#J+5ytGB7fx{7t*2cMO40r#MK~S}cfOT~|Qfh%|8 z+KPsHg~W#TzE{TQT$@%xXq3$;*D=8SuU!FL$lsw@ww)5r@mO$3?GJlObwsdirzQa) z?V#0HV81@O&n|{??m&J)&Gd?PI0Lq<0FSfoRqavn_5{vMpzunpmaNp#+4e$H0}Z|4 zK7j*DcEXmrj`O&FVzT#@nkvaOg9ct97>H}J)Im|Ts6T3}-ce@Aj6JKFjO46yw-;X3 zCXd*CQWv4fBgdKeU_fv@uQzvzg>H7dNT<8Q%-FahQht%;W!SZzV5?G3TnA&2pllzcbJf}nh^xeOuM0+U0e zzs9Y9C#g5qS$f<-=a}{6UWa@CIaThffFdzkH6fLBO6~4gQI%b`eQ;Y)o?+b*i5+{e zOn`sMSeo0G+EGGB_{w&cW}*;SO3WR_`(01ARM(n}v^ONwS+nZwV1Aaw6(hL-0O|QB zNmK;i>1p1Z@igG>fa|~bF)XOM06R2!am-0&Nyu$P%I)w&=T6(%D$I}ip;F##EE|D| zOPFHvr92uI4l8A1?RHS9Zo=!h;2;zlhPt04^ETI5Ubq70qWB zdpwz+i}U~qbC))N7kFcxbL$vvfo4Xx$k=`M<8Y-FS(Cm<&&j*bLVbUneJ9`qx!AJ= zqXJ#E5deUCnvvl4bvnaMB3)SdpTXS=TEFLA`NLE zg^}jXHRO=>H&khXLT7-W`xMPi_30ZR`PhEWN6Ye?y&zL$dJNa2b<%q?Ar8oujXZIB zzvVv#)rE%g5Tqfeka(R!)&(3UnBK}`iF<5e$`h!7X&}->BTt%cwpvG_EA&deVK(6R zQtrt4{`EX!<$!9$t$`Rv_u@YZdp@Y1;Q1?oDy*bFpcT#d^AT_Y5+oIaVvtku)zC`G zOB&MxeNBzC9jNxUgxR$@P1EC55#`7u#Yn>tmJ7Xbez8#psPV}QZ2?(TCJgl~g~;S- zxE`7+PZ=-M{HSGwIq|C!s8WxWxv!Db=}J$(Gf?sY>KtVKp6e<2XD>2~;+=6F;j@ag zu_LS1-&I3sRXD9M#nAS8J9sup>-HDXP|T{UUP+DjX6X5qA2IZvw7S)uuQT^E)>ADr zsTbt= z)z{v;%ro$Szg|yTEQw0{J#0-N-f|D=q_0NruLHDyp@szW$_a9OQSg|@xKp0@gYGel z?CN~CYgHkGAd}Ej4l2U|=>Tw{-;hR*T+LQ9Xrz;hpqrXMiL}DZ8vZf+8UpIj+ zy@Cl0?}UcqzI@+yM!)G<4gzzSE=5J)qRl@kZU3bogOJ5x#@=crj8$dPg8Mws-pD(z zFw|pejBcg33k~%^x6C|h8B$mho+?L%*7e*cIs!T6gen&gO5PEUX%B#9(*L_ zueMkH>CItkY6o9!#%fSRe3Gkk5;pk8^-&f*w6jyfvy=)DPE>N!pSW@2*0dP5R2Z}+ z*pbT$8-Kp~^GA-4+k(Oz+G;aAY#OE*{yM8N@$j6~1_yJK*fA!~>wd zZ3Ad6$K{027@M%xU-P!|o$M35?vfXhxq0-<7HOA>kcxmbPN+x8yTb6t98~{(lA_1l zCex&P)a&c#U>@y>km>OE}U|#2dCS+ zBB9vn(cxj#in_r-=60VfYGg#~KsRgsB&>6fLhAJsnu?1w_ryj6K5{C`Pbq@{|7~}f z<9J<*SJ1Jt97=i)q)+qG6V;v!;=lo*@@byq=VD>?mS6&1u5!=}kl$=Fi=Sq*J(J6l zQy4mp1Ou{#*wW?kWd}pkJL3UUD~@odGkdn4YuWlATI6#CmF5QvJ|uA!nt&i==m6*9 zT<7y>@e7?RvvC=M_!_-kWFKP+;sCjCxccfhl#4Yz(OglEB z8lXsAxx?NZlXv3w>3(^&JGj431>5#Tgh{lDdmBFl><51Q2$U&_x9UUOKsDtr|KWyUT<7nc*W|n%( zm%Pvef{BTJC@B#HbM@tFg)Evsjkk;@VPEra=Ok{>mT5SGPfUlog{AkuWp zLccr|_gsw9kx6AlIe~yvKc-zq?y(^}BgUMHiYxQp1c|QVw~(HsZ#&fI(*Y$LM#R}< zyZjvqSZtfHh1@zIXLgU922$tQ&q2Q9M8)lUY9q1HC~vz8bMc+RMXRkD z@!*tmU!ZCG4T|A@YTGQD&k^^5#JzHc3c1LMQ$0-??r*blP0tk2;-%<^`Dr0_@6vFb z+CBsRb+s`*4m{1R{?Yvlrf9obyV12q90;qh$zAz*R=7i_WF3sB_7mgBmH;(NKe8zv z@Z$8Sd*p{J)wtkP5f)HN1E<##@Jd^&%4kfwv8Bv`C>=8zfDX^776X2FiUckqsOpg& zZv-Gu@g6h4pQ~OdcDx)kbDQvFdd!AL1WbKXVh7+*a9aKe#1b%L6sF{ZsS3sY%5q8x zUl)-sZ^}k7(<(t77#4FnvuZ z+b5Z$HmtFfQkq2zo=uWiDwkdnNoy=-uBH1->Q6#(&MKRO>H8qGK~@DnN;(_ZutcIL zFikmrOC;yQRK-N5G7?*E@!I#25WkI*SFXT|4($1w@PH>Pr-02fy_$kwd}yPDNHC3- zEDUH3>+m=D0>;EN!jhXnL>Fgp?PL#da_zrygs51#m+`rB^oPk@}+4;t{9|nDiC>I8VRn%|Z7bd(iwx(X*1@dyw@6T&mG?(Fq>43GAa!eiKK*n?=KH*ex9y>&frtgqa21&S~M4~Gt@ze9UGA=7D?BS`L zJmn&ud{RZ!Nnbn8VO&nFB$lXk{O*U3Oa&HOm7GY{OmTHjQv3|vQU8&7q2ERxSMTfY zU;w(!EgO5JbKbSg4?I1c!-}9-p*(RuC)t%Sj1-At!wNBCPMfrRWHfocBQaY5Wv>9` zx07B~j!Y$t$EQtMUgr|s)9axLSF}0YB`OHTK`4baFW$)!44m~)#j9%x;ZoG_6nI4= z6BO9prom}L9F9pbqss7c9|Mh5Y;nG#n$cn9TIH3wpp5(k>emM$^V@f6oi$?xN{qA| zj7S1^rz)#jF9{zcAx9nYFt%cw+2yj=rl&_xIS1B1#4c}qJs?N<@M*65-(^Rn@sYMV zpKnoG)3%H3+w_!%FR^+$YeGIW6B>JAV619seahLA>iv^%@cKb`>T!t~cb>nsnNSx+t`#zTG zmHOt=uY(?q*bb+v(;_x%>MDC(iC_6Y#R>+brnFBeH-Feqd-$BDIaY!H1Wu@^(Rsba3atOT!aw5a z*Zhy)SO+*WE$SSJwx{Y8zw}dF(YYkjZkQa{G>~J7kCMk;By?z>OioN-dEb-QJ`>xZ zkUGArJv#)rE7wM7SlWHq)tg5r@#Rn694UId4g13r_e*KMLK2 z7L#9^Et|+!;W@~Am+OX=u%ON&FY(G^l?1rD%ss8!qXFyZstO98X3P2S1UO3R&ZC`# z!|U3k#`l%b8C}Maq#syx8G#|dp}F~>xrYq!=Th3?)>c>}_=~9$;Lp=Wql4Z6ZoTY@ zU`{52TGHAWRmNMfo= z|3)^0@M~<++C*O+HAyG&t?f)dwdXVL$)%3$0r@V z*ri&?0FKf#SpB0Fx<1UuOtT#Bc|88%xD!kMq+@Sy(+q)w@MnfHqs#ENwX&A=5`-o7 zUHtFH^nb-Xvpj*KyU&~oA5gdo$=?5cZr%UasczO>`TPwNdfzq9pmIOV)B(ln1x1WL ztC07a!M7Rvyi5V}W}FAgC~eHoMW&-G)+F1>vCef(?Ni>qgTlQT1?MBE(}S;`Qsa6Z|gO@7iRrts+Ka!IaV z8RBccQK;OWh0SEB$zySs(Il8=u?8o<(=o0eSWDf&@)CDXF(<&}XQ8d@i0l!{4z%KR zhZ@h;R&eVVX8=;*#rM_m-umlv=Mr5@jfzTu`ht|s$16=AJ5gmedwy>Fan%dDahZUm z6Y_SAsZ|uU9~P@>+sQKanX%owq>r9HBj=ZaupA;Yl%qtkz)Q+PGQVNNSe3b}LBG83 z<9A)S}6;N_9++hZ+>}e zH=HBQL43X_P2(Ja4V#e1f-*vsx^t4=j*gzbPUSrqcOVcxWpmBipAyk8E9k)t?ZUGoysNAV~_%A;i2-)IAW;<(So0t_=Gw-gV)s-1X& z(L3b{K)oIp?o}A> zmGY*5R_H&p5B`t^;6l0$dnfwgo&OX^fHyGrfWOdt^F-|Azy3Kt0G?=BlpX(H)eNwd zfSk~{^5OQ*jQt8i#xAdH%yOJ8GwgEG@n%WTMj`wX@7paCdK#Gk<&nJ?_rQ=e7^99Z z*37h)9TDYy)MTx`s*c8T5h6Se0p*78V;a9HS}!X4iD9JQ%@nh{u!rEFoshv+eW$_i z^R5$H48iqauH5X6kD~<>E^7F=1j=fbh)vp$!(8fxi@I)&;x0CJl)X|l0p-|Bp*z^JQyfPMIPBG`IMExV*&VWQblrH zjUWYiZWw|xQkY{REXQmT1RJTb@nJBGCy{sI_(`k0=p8$DO4c$rU`u`ck$H<}Eh|5o znUoCsP`@M#dd0I00h>?lpwk@7b-5OcgC4_J0$W2AA!|MX?D_Z~2ItnkS1i5Rd29$O zMb~}a`hEy|d}pl$+v)q^p8fN9MrQ;_eIQd(aNhA4T2C8-{bfeYZ_NKH8Sw%b$VO)r z^v?gQ)Wm}vu*VlN87u#;M3I8j%Vcjqxbv^ZtR@ZP5azwPUe$5ESS#DE@r|d(c(eC< z*ktaS67xOuZG;^Qb+l~6+aV0Q|Cn5^@5<>#-`p98`*9a9>aY?ht6T1WUtIw-!eisW z?Mba4pXcrmX0kxsRNo|hJ@?ulob3ZDvOCizr3Vt_(rrZh-kP?+t= zj=snW!%HqfZV1lGB)l`pwIBX;QT2inY#_(sM<<-N99E$97hEtQFt`Knv29A?g;|F= zJK2_zD*@4JiQxDcIl8|7JNu_#OdxQ{HLfVIfdM`%y>1S_mB(B!P9)_*oTB=)ngI(e z0xFb7BL}Q@-qVbP@NjgADsdEhG z!Ru4Qt1NJ!CI7p4I(gtYBI*YsNWB&K;97rqy7_=_aZf%q&JN}s9oW|ab)%#%v7QJ8 zgV+hZ`s<)tAz23qw{%2VA~LBH;ju#UDp2?S4t*Awj%ud3uVOws3c`U0zCaK`_>tWx zM-DP}GT2tMU;-u|Fe(}&jAJb=ExVnKnPO-mkS(cEgClr?=G4lvVNxEJ`(ftw|2G3< z5Vec-1q9#dSeo`F(qatl;;cCfL-@GC>Li#`AVuQryu4*?HqPMpk;9Ip3Ex- za1`opMB6&JScKIN(3mk7z)slx+J2>@7~`NcM%ZMZJ+7lLSMU5vV0EbA5Qg~$lIOLC zI-dDg@f{2zNAY6ug44h6lqtwg>5D%OQX&816|q}LfYN(+;@`Gt927Sc3)8Ae+JV0? zq#@X5Gc5npg_=z5%KW9d@8r7(dmMIGm*2jy%A(`DL`oX?R(?)*;Ot~T0y%WzjBHbn zqdE?@lrRA7!>&7$%&s#S*Dmy=MC3XDtx!x1;Tu@LEAS~|_V>K0V+iUq03eEkb%MJt zm-s3&KriK<@95+tQfEIR0VNDBQOP9m?w3e2AkAL|kd8FaW`_V~#-x(Ago{h?APn>6 zZQD?ewAbKLf3p4rJVTwnO{`C{v_W=c<5Xinuh_3^|n%&73A13wURAeNuH|DN<+VL8rNBaKfKN&XI${ zi8%7EV;`3(qk@e{pWdCR9MI;^i2tPwR-F^xJZaXhOYO8``Q0#l{OL~CSdR0h=!Jxc^tAB?>9eW0UOPF^niu5ln1P`Y@xUs1-C_n zvv6Y)T!kT{G#*%`E$o!M$FA}xvF1;PoU>K>Eo6fWPMrv@(JM_X)^Ut&o}@!h+6(=JnZ%9RnIvKeC?QwBKJLE#<(1y$#>WwV#4kkBL!eEsG`L zV21EBHD#m|X})`$kF-=AH`f)BjX4EcW?}G+8V+PMDAH0>xDD^wZ83)bnJWZC3Pp;z zH7;cRkR?Q-{v$~CJ3#y`zoYlMMvNFm8kq)fsh92lN4kMYLIU82k4t%BWJKX>#Lw$I z{f9Gjf~d1{#eB4uC%(9 zN*te#=@iOy`VrL}(6d~805f#UAlhU?91Ge{vu|$B2d)787IAE)&eURO;4T$lFn51k zs-s)Gpz~+AcVv^5_?{TegLJxT1Ns@kM~$}Jbc2L}LDY9^sa_({ha)oB=$(s$gV3`( zCq?7p;t0#64Yhc&NKxz3ebr2e1(7_uxFNG1MnaA7_k^*7|2Y@CR*VE{PR&VoF!^O* z5M|NXT||iG;6#oaBCR6tB}%a16h-**6So@FpbSsEmO_Yo4P<=qU{P_yH5Ut1vkLyKT;cexP;_J6Ue0Sm!@ChM46o4R3pNQAbSn&-3O$l*O2*qvnT9w4rOverf9f4 zEZFG7vEY^S%vSJukcvPnqbO7O+8txZqEyXtqe^lk|1x6i5&#D@uFqfcBA7_(1j4Qj zq;hjDH>+&Lx#&@8!2us{Bby9tK4O5}4Ix51X$RY9XWhi5g8b72!q7^*>PS$(n-GKn z*#G2jLh?V&dgSUl` zU}%PrH0gvK9vH~q4XPB$B(iQQ4)C}yC0@t^?*HVuVAuKfzyWd!dNAd`{5`grvl{rV zS+H0pW0@4%A4!ky-Xh%{2|B!nP-eVVGVQaDkw`J0NiC%I*0S7u_jD!A^tv#@D>t^p z;;m)j#YNzt7j(dL90pEQ-1#$vQ!VH6hV*qtx*ZlyDv~fsH!hvPic$PMX~@b}pCwoN zXVUOs%0P7J|By7S6pu;3fZ+X}xTrsphHN@yczl+@!0n-x$zL}pGLZ3kuH{0uZ?xz5 zK$1%VK!sr6l9D0O^k~RF&LARB8@DV{PSB&~7Ng z{&g9*AIz}o<}`JfZLBo53a8TSxcv)JJ{w|<;Y;E-QHhs3*nGA!eeG_;b}LAa@aU;# zi9i&f@5b4|4~&*C&*~dsBwgh$`ZFh7hua`rxA-GD*E$LFxz*wcvCT^LGZN9<54J-t zh=AL*-gt4SOPEZ3bCl=m5JXB6(h$WDLFLaiJ4**cTnpQ-_#pI+T*}_le1pug@h}+_ zTvY1DF7jr=&`DmQ+Np!JIDiU2FOJNV=y%E-ByJ!PVt_2wS{-gEZ7!)r&r?|zJkCMh zvTFBf0t3Wv`d(2u+`Yyd`wLRiDfLa@W!r?)?#U*8Nk(X_)YXWOMugZ}yT~(|6M`Ed zfiNi-6xTv6AvuWY#ZhwCA;WbqoLfrU+c9Yg z0zPvl=v@rOo)&!`Z3rD|NsxHBRTnV?3+HJL@N86hUjxR$zW8$0)fHH5bu%E2QDN~| zv*&UD8w3SG4_fHrHg*1(@m$`ZlNJzUvN1X7nOq+_3ZqhHq=-Rar={(eoED z7?5)9U}21alR(ou6#Xy3puHF*8VOae!8ogi3jWQX{Hc*q90(qV9ahqdXP`@T_!25F z_(o<)+bI=HA&R~13tVb_t3Yd?Q`lQd7S*Twgzd}&d6nVlE9Y*2##RPAac^#_LPP%a zc$p!u7l>V%SXkBB4ZT>(?;&Pq?icw8w)PlHIDHC_QSTs$QK)v2WdlBru2^lQSM? z#9k==sm6Y>x$&sW(7&uV&)r*Jo%_s$Zs z3b|LyQ{Y^3&!@I4y<(Kz58;KItn@Sej=k}C&*p;+g^{X}9Z{`Naz2N~6X@&?@zr#b zPvK{wtDtwAiExab%Y4iatw_z~TV*HZIl(0VX6pP;NNQ9M6DS*J?C9AvWQ+-;A1di$ z>hKSLUI;981P;6+&&J<2p9i_YN6&H-m{INXL?e$;YPxX`dC+5u@6zv<_}3kNbL`KX zN$ya?k0dvFFO&7E40mUdSXbr4JDqQX55Dm?h^!uPXJjrS1Vp*pkgeLs&%t`kY$~s) zjh6dt8UV6Q>i%zN;zT?GpZ1^mR&v2E-W*t&ncqQ)r=Q`63a9@;LByG@fSSQ?CF*ty zd+kp}z6Ll;!X;fB!a*iwKOhjYTQ~!Do9p&;WiQP$^IE%MM%=m}Yl5@{tS-&mc#B)> zrZ0Wlx36*zfi+$q-;G}v?X6#oG2dx&o3p?BGls-wi)1J2#rMh&8`2gS=d=hAI1Aiq z-D0QY6!PD3(P2WN7ciI~iofE&B4rq0B zOD;_sAc0UD{V(p2(>2edbbFRK$<#tFNwC;w+NRZfU{~e_yP4nOOKx)mCMF(U5wtH9 z^z(izIGrSNAU_Cs`uKL;*FX#F%x8e@$^Bs3PRnB(z}85XJcJ z2KAe^r%XkCcc=V!GJY=VY}bFN^|%a}-wS0;pCAETR-b0aFq5DIi1qygThj48N-4Zc z-B9hOI5M2Vq@`axFzYM{l(1aXS05ftQT&mO>LYdI)<-u`+5+Vq^GBIC5$fCBigwC? z4#%Af+s*5bWGoQjyhgdRUoQgEo?DOxkwKv!x%uV`i7$}lgRsGxW}kL%KX*9;mG(lp z&pH-^1cu}fQhwC*o^f78b$kbSgtF;D(4v-wu0{;8vbaOX~Qe7>!_o_$Hk}6(MbtxNHq^%g*3fON3F-@+{L8cK?k%D74-}5@ES^D7)IkJ318* zL)D`4?E8zTgfDOA&j=V|zFp3&Uk#;pP$X_-3OaZRzP{GRw`laSo5jz-hec^yi1L&F zpat{OGk|}4bVyzo@;+YX^CpO+TK6d3Yy~9C7-Qh~mzg5>96s`WJD`#@pE7>j?$)Gf zGXKT6PKumwk_26Ecje+?$nR=`5t8EL@@#(nOqt5r5U>pIf7znJ?~9a%HqAYag?^gMFor}d#sJ9T6qx_=`C^4hdS*MAubicZ<{XvK%mvEoD{`clKU&FHl7 zy?=CZ;fWdNtS(lqWIn7qh*xjbpLTzC@{>o;OC|h$fdOfIs*=?lBBWq?wq|tlvcBFd zh$^bDSB*I?uHOAt$H`>3^@{T)Z&M@&@W%@U1O6M+H4=h*Zr?7GHS`*$72iGQq3zLo zmnn~1|47HoZ*p$W$je(9Gmy{Ye8DJ*{^#K)0cqqhETW&WbMrxmh#N!Mc}8YSiv-87 zS;lpCipOBP!fZfD+tVIX$_$+nM5OfP_pXmG@5^Zg-sp;({ain;{T7-?vKB10v147w z1afCzTpZNeHYn7?t4AOVX{YNk&Q4BSOS|)_9ArAwHv}9k#-)}!Ej*UyVOnkbi@un+ zbXK~TtmZ>$(UVi~Ub@haZVH{IjSU|V+C2SU!=oM4;X+WaL4+Z_|8o41c?N7yugWi1 z4N5PcT{$JY860E%n4IZoZzRZW;h|hD&Kp?i8^U0uO}l?fuXR75O&G9Oxrk7i$VtcZgtyz zVoq9%6Jm!bg6EHu2Ix#(CXIrkOP1G6TQVW6d=5gEm*@%q7EzG~2MdGPEDA~ORFEA2 zr5AGFhGLZbO-^lDe6SIuRYVEzqD-0{q$@8GV0)VV8_)BYpB8r!#Vb7jzkjeh ztYn6$FqdCcZ#Gg2kWNv6{vGl$hB88IH++$X=H*m_m;Rrh^yYXCOG|tLttQGn%9lwz zhWC|0vqTsn!HqjXocf+_l2xw9iKnu7VkL_xCpT>@iJf$lv4K7){OaOQ6|uwUF7;9o zZ%t{ars%zWDs~FR4e0~aju@eu`jycv8~Wb#LADKlG_Rh(T_w%loM1pTu)nJ+b6T3$ zmOlb%OmS=!#o4SocaDp?Obd-qPhVMI)0IaL0%L8k^wRyo!A#N0`AZ2%`6W^mRxa?m z(?GY`7Td)_V1R`|p<>oYqxBHo;P87{?>K=0-t`RaqXM*ik82fsH1B(deIuy+zQ5uqgc=rdZ_QnMJHzE=H*DnlJj|wDU;Zwt6H_mq_K%4+l zOv_LcoBx7zVjvrtp*aRHN0njC`JkgfB>gkK@T|7m)p2(zS!Ym*nBV5y&ChQNAJCY& zW2F(rZeT@PU9KERz4fNa1LQ*+g-wR^XG_mFNNHsVJ#7UxqH%dpA4|l$chbD4NuU<^ zmk~}N>ePN8ZC@iL9WB^KT*8b+Zvo4^5+iQiO)&o0ZN9?eMl}QQLu~+ZB96~^BjL%` z3tq3Z#81kA)&ENh+zW&yGoiU=Gf-A>ZSvg%B2dq5Ux8Bch{-DiGlE2~#v>l2|M_CD zl7A*DtWU3aa&8OcQxQL6(B`=YEde2z7I7emTy?3iwa$O-W?I2TbFJ|d{DTX+}V_>|4 z)gt)C(&re-os2J4gNWd9EA%D(p={J3C*d?vs@ZDolM6!1ZjeI^EIo#B>0_GbxgH)k$i-aeK@dxn0Wu`(!ip==XVZDDOgGM~8 z$Jt=KEQ4+|p7hA|Kl(g&MOo-=dP6OFkmltXk7&Pg?Q%#mUOF||X87?sM6+)F5%hM4 zcNa1fsa_RlUv~oC-21V3S%MoG$irC{9d4y}T+yhjK2;Bi=NmpKK1~U}=14cQe#4VZ z?^>Ve3+eq3z?v#kvq0oFT+*5QRi&w9cHZZnvb?pU#~r_h!uUs1I;iCm#IptWjn}33 ztY-l*IEeI9ulg=iEyjf6t$$%P2u-f|I(60G^&4cwlRKeCa@#!$18-@5x%is}PE+e( zu2}+V$Zl(?zvnC~Cz?n3AN4e&Qp(zJ%vbb5y3x+j61Y?2(&Ae#-C+PaQDOc}#I)_K z{W~3@Mzse|9iZo-Ef!4`?5XxnL(DB`d;`avx+?LxBi79aM zlqbeDzVu=%5eF+px@jRh?VlhB1nhI5+x`|w*n1xc=KmG{SYlTc`q{Zb?HVbRI_khwOdJS7FIk{zO#8vV5t>4Py`P&0eh&~|N=LkkM zz-plN$6Nm?rmUm9+1u&!;`Z1Tou6V5kJ=PV^?X&~Sk(C-2EwIV0n`q)#Rx*8QcyVP zwH@a@ACL%@b)GDul8HGv`R+SZ1&lkqz|PCRmSo$N`HX8xX149T9(a@nGRvcb9zM}* zaFZT6k%kE1)G$J{h*AdtrVGYo6cDz`fhT(XLovn)M2c9=l<#0{z)i89pW(Y-E%e}F z3jOt_0;<{&vJyaz(3~E(&$|PzxoH7O$p|e{4V8cb3t*8e*c0 z$wq45Hi;@(ROOEM$a@2v>-W%N%N&Z@`mA+$Q9Ph8(dWanUgPBc!lXKA6IP`#F@Jr+ zJ(HhMz+8@g{`C*g=X-Z8q$+}z312~4M))KpU_;Z855<_LFIA29B|;DdokY+)bUSbq z-(&LvZxtB}dy@@Uwk;*~o%*rjiZ%ds8|8Q=+9o!FQkgy9=)r23&-}fShV2qP6NRf* zxSt}~;;ZF@54Wp=j0br3_VuKo1co%lSHQdg!57;>iYca@CwpuR+G&HqxW@(o8YGh! z+&CmLdnM1Y>;sg!veYlHX^Tts<7Wj7s~->_-@X7XzE2k)Lc~LuYu!PBLSSo&kulp> z$*LpuJV4d+$ArF)I20frZ_5~eMRDVx@Cc9s@N~UH)+ZUQ48BS6{B}q{MV* z8k>bJ(J(&18}He5v?zeK^x{dVyqqEWiO~-@%ImlZM`kp!`>e+pT3eGi4Np7Z`UutfB^hQ^jJ=`MK4Pjgk89Gc45)dbij>$^-#n?iCou!5C|h|xk{s-F z$?U!hjjIqjP{G@nc!?A=uQ$+@8kHqeWp5#3?V1R{6Uv;&8J%PR9ZZi^glu{u+{Jgx2agmh@5bKn z&o)8|&gn11sLG-SX^^*(wK%Bp4FJECD`@Qt8+3n4K`Lk9ywh9gD!$(vr~2e!93xFE zHaVVGn;A8K|H7$}A0BBBRi{x}XAe|JWG$oy9$Yfr-(LUVi1#w>j+4jOS~hHbPDURf ze06(~!p&bP@|ERpcbUKgqdr;Wa`8T3iqa6TTGag_rpizPkvnq!I#GUEu!R8QD^0hk z^GdAtM6!5Rh|!1dJ$(KF_q?ZS(rnHNlp<7xH0iq5XJNe(v?R&+#K=l3V& zTfEopXn46e&1*b(o~}YdCuXvJW2(aVISr9$Yjh>W*T07~391ko60RFc&rCd<1}VAG zji&-uL;28AW&qF9?QOil8vr1>3<$AWEaea-)H+Zmp^Uu{wncy(Qt-XBbvl z<$y($PUVbfc5d;gIT0v>5?Qu`No#R!1kH&kX8+n?09-WL(^o!!h!+IOHMEBi_RMfw za?$leO1ANd37fzp@OUg5CrBWjNkytFgk{_JN;-704Vt{litT;%=eP@M(Mw)mn2vTM zjIEw)z330qy|%rzUeOw6N5rnitArzkDUMW>=OvU z-ym6dTZGc~c?>b`dtP)I0(ZQa&wU&v&Pqx>Y9{`IY7BBU^+pdEgWu<*{CO}8WPT8_ z1XKSwLo}%~9N9qhCeuDQ>i!TZeY;YA5;%Sy_4HO2U4Imizb@D5h1Y|6eKvwRW4kLm zX4^xJ5@!=~eJ$n%4I+r!XPyhN>()i<3ZV}*ED^5en0T!#M#g2=Zfj+oz39mV>|lN8^9{?5o_~`aud~;wo6Y^1nQQt> zRA-~BzV2g4CD8@;)28Ym zXtI|}{{olGpUa7%hs&F`2|%dt+6nODf3~o<*7sqH&4iF^mchLf7ircKmdj0sV?2?O zfBdCX8qEU@$jYP}{W4A{q>6eP0CB6ckE@J9R`&jI?g8_tHR|(>s*(wm*-UeC(TKk( zLoUgx=SE-%e291RVWf({s322MDVX`6vI=$@${{gYmvzZt=N^WK0Ws8xlc)xy2`aOb zf~<`{wwwQ#K97yd8LDMIEVz}_Qvhrx@MI^3)kq75)1%8uf?h*TR{FK=n`C+b!EH98 zE(lt9@^&%&jcT6yrT!NQ0K^CV+_&{^^YR>(g{NWRnfx|bdwdcGhpeou!Htz~v8f?^AScDwLTpY@_M(1~nC_zFL>MfZ6 z!t@^?3LA9(;%%-<7!9|1&IVg(DtwY&JBi(gYtpK3=gIokLi15)5NYfk=CyqbzFBzX z=cdM&wrJ)nxGmPWDm@{VxRks*KK%JyJ8uQ71@3(Zs6YRaGQvwAr|ay=nX$BF`IwlOHXfzy2^Ub5 z4!&VSkJN(EfBpktD5V2gm;Eca@GA=jOP%**K~0@G56v~YrJfy_h(nn0$3M&sc&YH( zNUnM?+FBXzUNyO=API8@xqC8+7FJwxxIg3{mH=K-Ay7zugvg4Y7I#VkjL20u9x_dx zLN)@e#(WIE@iby@iXGeF-dEASTui z(eznyr-{gQTlY6`pDIf5^>!42FStwkK(L&L_wu8qAQ9ng84}62@QDj@K@4()OVKb? zkNRro%t$k&JPP4}JL5Yfj68rf8C0QhNk)&c;uQSsW4qh8wLDN{RB7UP4-RmF@k?-n z+uP&3`t*=%l!qXpBpGuDK2rqUJFb@B2~S3~i>8@gzLZZj9Y)=`48M1>Z;`9zM`l*xo~0zMM%jCZ&%8h>}umAs_(it+jR9bo{5=#{30iIfgPr)&ex*a63b`aYygQM zb1IKv^vp+TGxA75xNLWr@q#w$3A`;$quE7f7Hw2*i!<*|7D5 zn>VxT6ThPWGi7<^$vA}F5io86E75Em*Z}f;%)nXK9hJYV3Im5Cfi1<$`E!CTlEGKe zYEegAMEN3LbzyoV`Y)>rMzbO}$r9kBg<>aVf)V|i$s3=7-cpB|%d4NE`OB(s2$e6u zOXr|W1(u--BW*2a%)mpqln8fm#;T6@FRS{G?TfDNgFIF`8JnsDYI<$5+Zc^aZDm7(4xB33ftWxO2oth*u@sVhv6!?6sHlJV&nnw-Z8G_^Mca;!!-b+vb z2flo-H#PuhjD;tS0|Hw+SmU3heAzZ@3U7lgCBmVFFyTc4; z9ZT+Hf#++|2}TY5l3pNCXC?yUx}22muQTEk9a?7x+rh&tOizg8x)G{37mkN|fD!$ICr8CYSEc46szh`s+OTLmXZSG$|?Y=wtkt(8F$=n+koyn&+ctV zJyNcNlVZ?rVagtl6DYOag$9oBXvYw0YNw4Y7^oij3~^XuZ=}(>vg#p>uGgkS1WxLu#1J zsWnEx!zbC}{n}^qDrmJDUiTgG2Nm}suvb4QJu+R!io^MOPXIe4mQ(?RK@-MATLc(| z5F={+ww>bMtpRN-ciq(unigP-^r>8n0DmA3Q zc-cQ~VtxUL*9)uj9UZRPRro*R`AGv5QJ)|bO*Z$LQK=rJu)fRj6FsbO6oA#e2ZXSR z3S-fqjMa$iAn)VE{l-L2{CkhUNTZEL%(Je1?a?`XRFSizAuYG*3Okk*`IUnY``!9^ zm5)Y8x@^}MZ>=WY-Gg$eR>nwo6o8@xF2Trzk#R-=5$IzL962Lw8vxKetwOz`i-c^$ zV=K`ougiiRKaMix>L%og~13?*dY&hD1?Ar|u0CG;0 zhAs2j{V-Qz?;p@G2#Z=!&nV?>osb7t=lN+|xofpX2z8PQIm<+DWC=6=Z8O4{u%K77 zKIOr*1?NZ4yE|3ubaJ{zi+b1%lc`=vqoWC;$lBxftRv07aDNkIqY$UD`2L>*2;- zh<aQ(>a}L;%!U`hwP;pYI5car|ZKgwzawo1x`ILMse~8mJ*)f4ej*1 zFBG9;qOTgaHNQ4d9DT##(vySe3Pog#mgWTCc&*bV7p;~rgoHECWuYYaXD}od;BS=A~%(f0py({p^*b&EIj)Ibcfjy zlFZ7ry55S4u5}T1^K1Zs3OZUgr@D#~47-TV1zd-jnZ3jGH8ido*PV=B;T%OjVu z(S``_<}ligKw>FBwEh4P#68NdI3g^)@)Z-j*Cx1q10ZMPkz#qDKtZzC&-}J)C+fi6 zZsuVuo%qG@%e7YD(AMN$c@Ht68)5@a%jH$=Nl=h}Ju8+MfrL6J@^Xk)y6?b->QcOx zhh3Nz4ycP>D%(v<3J<@R;DLjK!BJOMG=R860k*HLXPIpYm^UyeDw5ZX@)hHm3tyO$ z8it{NOglh1=Qz)8kOE$;EF6)XwmgQ=cF`y^v=inEoW@ z-1|Dl=hmLZ9P`Ap)a<8hbbJZFXizxVW!_g^N~Oly^+xtJjj`5L@8Le0SyI>Mj*261 z&AVKQ;dQ0auB+Ea^lzU47qgpBZt^5;XHl!X!tT3c!UN(QYQ;NilNyrjeFpg_&&Xb3 zma1gX)IO0H#tKhNM8OLy1X_i7CYqctNGJ+^+?C%ieV2VMnXT*03L9+U@yHfVE0lc} zn}Nie*B2@<#7UfX$vAV22vVI++-B=mRDT^uD({ioX<8pUmsgMOF#u4l`I&TKOxFa= zef2kvFvhfBR_fP`@fMZpsS4kZDm=fwb&G@TaaUvbJz=tNTM72jqRK{{jkTzc^`OCL zcE3FYO{bhbvxynBJ60s{zWaI_ssA+DAt2(ahw$aC8E8+@p4WY?my4Mb3O$+e?V$TZLRm<1UReBu!G<%n^H&(>F*6 zGTfHMPM7U5CxlMTu*1_jqMM>HjyAV)(9zjAl4OzB4?l6D00?gWf>{%tdfQ0kN%i+8 zV}a5tE%c>}ZLm7^?q(m@J5CbUMdA2Viglpa4_dU4z^K`(6CNaoVSpuqAxX$u5IaQT zk`&0^@tDKK6ex&OjBJ3A!uXfT(|<^t$~yls~G2r%~V zSEd_vXO`s0y;!Q5QyTYFO zx6ndN;n%*i`|Q@x%1OfHo0>t$iG*^;U)sk+5lvhJYQkf=2X`St+K_ch3*gjzTZ?Q$ z$cgs#I+6MT65X;^!!NI@py_Px&n&S>pOII;3+u(N^-4BctSfRo$>Ghbo=X14N@(brh8{>KfDfYN_ ztK%&wk+Y5G&b{S}Q-kb4a_?iaQlM_WApw&PCpLrw*6G}Y3Dz!+f^+RvRo;Otj#Ze0 zfSV$_K@2s2lK19OzHxkp(7=A{B)QcZ9-&y>f$&Qx;G{Haj_rl@TcnNIKfXRkZWp0d zX`jZs*2HoXhCWU9)dopcseVa(v>-3;fb&3Ztl;y(<7l3^jiYTl$FgV<(+_|`zRY4X ze?j?lH+A>D%lX*2nMwU#Hy@tKah4_^LA+K5p^W5Lww}Elnz_~Jw{^Dgdz9by8b?pQ z+|^vndAZ070maWg;H6XtvihA&bl+*-bV&2!1gt?vEFK-}JLY#TV)scRgC3|r)Xi^J zp^0LDCxyF$5j|>o{*XD5+lN-E)?h==AhsuUQ%6FNs;@CTMy*rPj`V0w77586bPh`=gTqU$@FDCSM%p+rilqUi86dkK`_@hOkPY%g8j6#}(!H%u zp0r*jYs~$TGnUJClo54@oUP}^{bRBG<4M}4s;&LN0uh^0@XYn~>Xc@YAzr5TG#cgzWEhFwY4L_p09n*QvCIEJ^jZy%{+%iILLQQ zK5?96@wq+mpueN^=!-R)CnLvC&@$WkR?ya5HZp9#Oa$Y@i+AD_Qu|2)Gvq6&Y{Hjw ze}@s)KvAL^^&$-3KwK;S9>5Q|Zh4VKfYPdsJyooB=0xU-bpgi9!$^vgJTe!)>4w)kDu)DR zzUySceImTaZiF6p2_rRp7>RlIftn)oY}2dgws#u3bt&SmCny~p*%k>QdLV)5)a#a~ zRDBY9P)`A&#p2E1^4XZaG$;~D`F;yF^?q23g=e!kPFQdGvC}_&oIibLjxTid+BEfB zc53S6R!L)gCXzF^cdY^g0pL#5Xe_Bg@9pfr!DX9v+~GUZ+PiG?WYhDyBTJ&{jkr+} zZt8&KtZeaTDMOBEn)1kQWoxFtQd{e5?OE6duLv|rYy?jhY0%8!l|%0I)@cijkq~{l z-Ad9|v)*-s;f`?TQe!h4^%89%#U)a2yEMCK_3rW!S2UgcXqGK!f1}oIv8N=4Mmg_I zo$Dx7=X*0q`|YK#{oqfwDC!o^KNCi~#&V9-?CVJdALeJDukF5b^qkbuvC2{RS!K&Y3|d(JSvVf?%O!e{R%4j+PfD$FaUG zLgirh@KHulIh~?|0*!5;yRZ{!Ud>nO9e~Qw)KN-*@8cf*qm+ZCh2m7qQ+AKYBeqTW z#z-nbh$^cP!s(I!x1+qw0}kt?F=o_AT|O&-KE&fbMmLk;Oo%Pzq38Z;$pA>a7J5p`H{E?7k4`(d z0y$2!hyCMfS#n1YY0 z-MPQen?S&e^dH|#oBw~OVngv(kI&dwvzzEsgR;} zAG0`j9>uP!23sDHqeRR*lSfSpL~%E|idaofyfj&+Qg&hCIho`HWq`?}O^;zH-#lU} z-DH@LRocCUN0%m@3jo}##bb%)jTj!Q)lzkn3TxtC3*hF(eed$(9o=Lgvy3!5e@GF@ zN*NHWEI_h!R#e8vk)8fIzI4R3kKe#JCj$f|fTv@S@RLMZnR(LW@pb-PqgNZ%MXs!6 zB+{m&9`^S?Lx#ktc$?PRqJR%0C5%WDc=~d_xkp zKqhi>CM7LayHsD!sA{CtJ}6Lnj)d zrLUw=9n{VL$_bGJ*Kri=u2qt;Fhq6ryJ|2-1TaR{5;3HLk<3ad+7jfo&o{P~Arqan zO{JA2Z)V*uZ}o%{g|lbCL}*_Ktr9k%%Tt$;8rx|{DX{NQ)JSzqyyryF0qA z49w46@<{z6ubjV4N~wHY-Y%`LkeTk}s|a*Y5uup!%6mUVwNkFv-_@_PMuBUTA)AyW z0?h#Hb)`?)owk7B-I0C3YWRqFk!vAo8H*9mIg}!5NQq11!KHlTivdu3;zC1=Ae`L; z?LC$7(3xUc_lSoX=Gw2HsWeHkR_QVqRJ3xJ7r;TEm+tq4D95Lo_eC6MS&qYGLlsIO zMtSxM(K#si%)l{iVubSXRX5o%B4uPp7WPT{);!lDnz$DF66e~ZXs=EgYtfnFS%%=Hnh zxy$x<7Fn?l$eJ6-Yb&{clP69p{rT5|U>o(7mMerQ9HV^6%!#U4yVXso8t0QjzSQGF z9b=yF`^{ESjfHi{BH`2ta-yhTEH%|G6kW#&H_60T_a>VL0U`;d&WPl( zi?rAk*xEUANH5d!z5pm78x)W#%>usm@Zcr>7*g%n`>Fi75vm=85?UBsQ=vvYX`Sks z5DPQwn-hRiHjq>b7CIkvC2QReHp9R>Qu5@HNPZ~xX2#hL3|?gMd7YjYD4#sz9fSDz ze$Nl6TQ49DH|;vx=O!DdlVlo^kxEb59-L&q%MWG4+ad+ro>9L57Dcwv9!4OjeCZeX z@a7W~+co-}TRAL0Xwaw0r|o%0nO{_d)si^?hKbDWvzXgqBLJOBxz^#t3h4F{4hYqV zQNFw~ol1d+Iyt}le%6|D#9DOE6rJZ`5p5ejK>;E%DC= zPU^jt_m*jR$jWG9AmEXMkF+tF=V9Eov|3vaH4A^HS_p78P={UY3P{y&Fj>`@ zD|4O3l!^U3`^D(t$)4RfD8888uXQJkv{AD5S5B~E!%>!Fk)*RZef4~;RfIsF7*uyu zYHz!61?O0h>TzA6_5)vN%Qu=3#^iec3iggLv-Jj)-zSQaK(Zr3wNlY&2ujR9YkkM7 z11Ja760;vb&5h^PDD?_`2(jw|nYX+7^bo~MYoK)+t=<_r&itSib@It;C{Bgh=}D0N zFZSL%n##U?AFkcrWwwbFVw-0wLloJD%!FhfG80OajAmL457i$ zfJ_xirV{BrKc4$}zR$bX_x}E#-@D%5AMaZ4TKBqHWM9|yxjw^r9_Mi!=b~Ef=gb*; zck2qD1@rfEE0CPLgvN?Sc4>y9pTmk&Rnr}JufMLaRT*(e(v`Zx_AJ7&JU-Dhk1Bn) z+-GJNvc3wIlY5Bpt|IG|MFnpSNRmIO^vT~PdOHcr;u$@o(I9ue)l2<+y2&MYe1UF|#=I2mo1Use&wfE#oI z;M5S#y;3SE4WX6y0Pn{5gyt$M)f|{&aoBspH21847KMoFb1}0j-t;t@u{9+bu4A-S zG`LL0vy!_4ev@}pTT}-umnsP(V|TBClPoDB{e%|6w?n&CV{Q-g-Snj!IP=)YgZY|o zjFm=Z0dsq18Xd!>{cL#~74mzB+#wwpdA?E0#wK@GWe*EAF5`N2%i#p#3bu-?|54q2 z;oiC55*f#Sm3*Y2N@)g2Qpy99_nzVQ3uTNgWmp%<>21q%y3!uvm&7om#EGOg!*xge z3y9qmY^R{o-HA$!?8GwtBqo>Ji(HSs+>q>xPqX&d%+H$F#7hA{eEn>P_P0SjjbiBI zWLDu_P&eMnld7%kb=%=l*xtvkbgWVuZz~vFKGBc8vAv=3B#tcADM~&hul}@3Z_2%O z(6I9^Se%<)*^*zVdASmlIZMk;#kc@a@S9I^4N-F{m_L9atMUwbx&z(v5&LWYS;e{1 zYOLSnAsIHgeM}@L@29qw!@N6M7?$YRaD-#@lXW=1S%eH(QeFSC$VHiORd7Fxk3^YG z)03~q&f!>jZNTQiLy{E72u<9%`RjF!foUn<`;2d`(cq{NdLv_%`8mp$UIS>8}S*DdyPb1G#l%)nkE7wNo zz<-W-FU{Z>@#%0diM=&45v)JHBx?NG_%PI+D)w?N{ATOdJTg1AaM7Ohh#y>4!iI;g z9;!_UEL^3mEsC7Ony@`}!kM3ZHr`TygXnF$Pj)w>CuGhU0p2YPfI2zk5X~@R8d)O@ z`nhI||2inkd%1BLZjy;>z5w|y?53K+Qq{bQDm7}1el>jW-< z1WO{2j8kQ+bq)Xfj}h$OQc@$O`p-eVb!l9LQIHA7D*Q-N104YARTREB^Zgf{-sTaj zdH)s0SZ1E2k{WX2UHHxpjU+?D)`z)PK#-%ov8gP#^KlmU9}5_OX#jX`{C2!Z={$co zp5?!)NK)!pR21V;k4=8>^Hvue3Fg>WTb!s9c3@d$?zYSxzYtr!RGxwLEMtp{tv|Bhakp zB?nLqMmv2`>9%$IquBrcqhRz)Bu_t_%p;1ttDjCuARY!#&)egT5KIk**hnmt%~>xX ziU9$}NXf{z=L~!;@6(xxy^bmKoJ}=R!?Z)Oy7z(EU1Il_8?X@7-rTp50JcZxxyRGe zwdY<77aJBv8$qqL;htUtC$oTn_=zckV83#RoirnqFrXqUGjX`d5~}2<&eBAlBJFF{ zoQ#KUOADUcSU7|HA(u#vyNc?gV2b!0OeBn)eghJ&7hi1)hD(lbsQtaSHrEaB>K$|i zi`4$NjvTGSv5H&1^+0-lUOEJsomt)aTY->oIKJs#=?(N;id_JVcaV11`KI4K-zKJ( z`Lt6zL5g^1=rKarA#;%cg_7=*Z%ccrq|%%t?RiL_5Tx6Yt#J3$j+{tTh(HMe(6{c4 z?RY^TNkODjNO|Km+;kRz<1?cxP}9svXr()@b8e_y3=Hrb(pM!RRW>Jc90#)W6uLxp zT<+0Usv4BUK(Sr~s(e)E?zD7x^>!U6s{UGlg6h)mA7fzQvH3AbJ0}22g`29ElVE2C5c3~#N$TTj zC>lSn9mbiP-c0vW-au-eKxoKt1bpR?QGSifKwcQpqS7wh9mMLa9!Pqt1@qc*Anznh z_Q!~7;|RhNVb#{yqtX6<)$PYmKuz_>ESU!aTXHOd1?ER2;i7AB1CBYa?dAnU>h^EI zWlcEu`d$}mj!@d<``ym(b77hToH11pyLLj)X;Dy~T^!YxWl%zxf7f)HjW`*99Ln8} zVC%*zmU3u$m+cAwMcvS%oDZt>gM4^$?mf$TFE3FU%mRq5l7S3u3DP+>*Rdy!S^i&& zg|zpDa`*RlK;k9=3hT;J>qhhZTK^wkZzY_4&LNoatf#cV68(9_mjuKxpw@=%Qx z%&JtDbD-d<^(=->00NRBzFYJse&$NK59(Q&{&~5ECZ)dd#;^V zU>Vd)vanOa&JqAS%e&0IK&{arc*81AukP+mhvW3&ul9Zr+UEPUAq(aan5Pe7{z*3Q z7_BiNtwE`kFWkcHR<4*TX&ve`$x3&)Bt~H&6ds{jAhOy@h8Ak4@lWHiqo7$7L&yg* zPZ@m=;+!*C9u4203BSZw&ak*3NmlmqVk`X|KlXKr$?#9vhqdwtumK8hFsu@A3}WP7oTWIEi}(w=4z+(Vqx5oiJeg8H zKNL!LOas;MSJ=i6@W%7Q@gha4JO&T`a(heVpsLZA{K%|gY8bU?ZtCR+m1y1<>PAe7 zuO}HE>y7w#Q5DK7JrNhBU!|_e3EnbGBd>jasa}|+@ILwxXk@j)k6`5xmh~BV;bS9C z^Y_A$(+%I4UGSb{uhy^0GF5%q0fU+XU2wBy1D7DutBvdq>X8;uedVX^itzvq;=n=O z%K6te%~X%WQ|t2YL9xH-4m6XyxTcQzjw)Aiejf$@a9w-BN6zI04=@^b*?3b8!Ytgo zSM1G$P}KU1p%wWh8Y9NUe}BJn>|u&KAO!w&MDM~bP;xv(JEm}deCY{|ao@Q~ImWm4 z>Y7c&>Q&@Ni`*%3Ei~Hd3G}oKZdXY@EtCSC5c;~Eda3SyFc@lcb6z^`T<}EkPF9aD zF{n0Qo_ixUEgv|3P{F@OZA7UeT&#Y^2u_CN+sXX>V6rMgzOzdrm=xCf_15?&FUYL? zd}L+*wV!zZf*T)nekLib28pjOqU zsDUxJ`ucD(Hu-#|+zzj-5r<#aqzHTbRdR+KumPU?t+gxsc$d#`Ztb#PT>Z?{w*$4pq&`>-)NZ>YuoD zwV%3%)G6FVR1udH^+Wxm9&oY+T?Sv-=iqp0qMXYmcOvguLwhaaoqT$>!mq>T%eic0D?`btm79Ri=#Z`{~o2K zAcEs1nLvS~XePC$ltFD>RRxk;)~|&>o>SBej`)7~9(@}$tZHCsQ`CKQ46tVhG2S9| zy(3f1-D-3-aH!c-sLVp*EB?T3=*`uNOM5dZ2)}xp!m5GADg3Qs*IhBwt5FwkpO0ac zHca)jOWMWzszVL;kpF@ls4*`K=vziAe!E$h{l~9dZoR`WgV>rl^dxBGUSI>*h9r%WKj9agiy$BS%tLk zoLYk&g+P|?$EYPB6;tIsg(9PDisi1|H1UYU14ejfa+Q3_EbFV@t#LikCsZS%LZ?MU z;cZj?Xufn4%ZuSzelzJu+}?=RLs|QB)F;>gisQ(~T6bejQy&5$NB30#oPN)tKeQm2 zHO1G4ACw0wkmW)v8!2KJIq&gQqPA2EZBXu6^*&H38I;%0ur{7h*uNA9!3$lX6lwRq zPO%?=EDEbEvI9TY(r-G&3#cuh%I|#eu?*BB&3r8N)ZxXWnE2D|H6B#gOM|C-?A@9G zVRiwcGzB@;D}=(zw`tn1yz&N6Wl``~QeFzA5LAuk`eyn_uD_q6fZ1nNHM@L$d3H`b z5hV{_ODPR}0I};vBuug%YVdvO?x}e}N|Fzh=UuR@UU!r}aQzqpnpv5mPyAI<#5*_0 zo8RQXn?CmdsCGsAq2%Qm`f88l)tmHsAGV;JT@=b!V`_LD=n)FIZbjr>)f+TpL-obu zWy&nXH9go4rGKsvSyqT^>d+Y^&B;y+Ax1lVf98hG+KW42XnJ}+KB}_4HcKGdWGDZihjVc{e|&$->u`?2>Ozf9(=^)lJL=amUmnfVjkza5xani^Y6Sy zhZu-|-sn3Ob5YfZl0@G*Rai>v)C1fueNJ4*>Wxqi{CfR{X(%=tu{aO+`Cl#fgOj~W zVsYsuPXHHb9=3<>6Q^qdJ^hOTzUPMyX;LPY1x? z&Jk#9l~XNB9oOMm8fLEN0})9OR+0Hg;L;y)ocUIIwK^k(5==HMuQ3;i8?I*@mXv*0 zJAzov_U)`BH(AQvGK^fK<4rcqopxOs)gQX!IjBnvj%6m--=#j? zyLZBA=3%UgP}%6f9v1k^zyv2nG%|32rqVzEyrM)(r4xI@M{0l_x|{8sk1D1TUBBdW z^y{-Qb=SC$Ft}=`iKhS%*9p`cJ}8P*>wZtbdci3r=Ikt?D`dlOKMSfqlcrTX{7oL%^@ZBvI?NZ~*47{H^HRmj-SR2#W`Uo8<|i9STMJUH1=J!FtO9aKAP>k7}L#nU5LZ5K?IEb*?={C8Zq_CytV$?Nf|Eaz`(6Uzhs>6;i5L zqI)E^N(*kTM0R99B@+!yfRJ>pS@AZB{^xt0*D!+pqLe2d5h1w^Hyhs%KxGxGN?Y(& zzox^P*J@N9Wb71%zbXCbV9yIWJE#Xa3Ez5*7wNCiymquB-Rhp%=ry73n>IidxR7HT zkRf$4!=G3t^wwBW)#!A6muKu0f*)jkbe!bHTfm@I-a0nBhmI5xgPot*F&!ltpv2^o zisiI|DLK<3>PZ97IRbMK?tbK4B)XdsI!cEWSU1KOs}3^??uU=SNv8B!bwX!M5nwQu zhcW*@d=%;#u7kV2wd*3y>(y2(%zo%Ej%Jc>Zr+ z*EV^i;Kh^QFoK(^KQkbo=!0)9GZp@B>0U?sMDof=9c-r=ITSXi@|})O?`n0J7&koS zd2`_de0g9W!NLot&TFwZKvbo~6B}!=!7lHeaD$vp7FD7>*3gyyJD@{U3x?8Q(0S}C ze7(vYh=dT%@NIc72STG-zQljUI4hYx6AfT8AqdOw z!`3*_X{1z-#C43=vm}#iDla04ZdF&G zNLe=Zn;n~bQy3;v&b;Sak1U5Q}iauXbh!IkNr1`uaN zUgn1o*M4F^5&g9R$iRBfo|P0i#Cbh3k98sihRTbgbAsL2!8DH7hV+D0;!vC`ger{ z^hD(z#(}GcqHZ>VgQ4F_tQ2_J`-#33z-zsAua3!%o!c+(>v{HcoZ{9)1hf4509E2b zKsiXOmM4A!Zu1z+QoN11jxMX%VRuAChd3|`>xlj3LIznS8voN)oU z>x5JWLgM9^?FTkGi2LgJ7pk}S<3jo4uT=ipsK@UJkP{FLHCSn{-gpYE+Fv$5x+tds z%U$O`+E`S@pZv6|2TjE^Xk6oIkG|7q5VLt=dFND4x!o?zJlt^#(ptvde>a58I!dh` z8w0XZaCu-~UUr}CBuzbtA-WZs8mJ_ldg9A!N3}{oV!aK;;j+EJ>Et=Y>9@a4K!%qM z!=P~UP*doYh0!z6G(!SK;)u)rWG%;rMe3WDlr`Y`1BqH752PveYQ27uD+mKR8=|@n zaNC)JcHP`YxV{&-u68!~Ef+u@>hBj$=xnvN=e4u0 zx5X4U7elQDQsuWm@v+peLf-|9dA{QdHA%)#2U`Fyak(l^mQ!sbX#=IskUQu3T*n&A z3b`=C)bP5GK?Uz>lt6a;#mQ$~%qEt%OG}w7A6+WGaqze?A@&fxA)v01_7#cL z;MOT9qa4c-_WWkKFGpd&(+7|DOB^}S<);`VTI?6w7`pDEQf-tp==_;$QNI>7O(!Tw zCB%YPZ1yIE`WZjOMhu#h5O~#0e?+bv0Eh?a|Iz7_BGKPWGhl&QD9yZUahv*Z3_N%* z*R!S4a|=_o>}r`q+99wm+40&ad4w+u>0Qotv8ysK<#1hL!Rjy1^j_ASg`$@rEt^a- zq)a6`t%Is$35t@zXl57WsjQPNZ*~Xx7eIE^D~=6x2KSunG*Cwap7InEY4=4R0KAGV zN%+=!4MF+?SKgR8yB^1%H^umaC2ThU8+Ya2;|sAnUGkd&M=rd9pLhfQB+TJJZ_yu! zP-|I9I^yzzx`whoT>dgXTo4Ql!1IJCGtmeNQ*&g{tkg!xqu?SDn&SVsfp15K_g5$- z>xf?oC%y%=#SyR_=(n{u#ckZ=A2@2V*3p$o78l#ugR>Dr@iF8b8-NDX97`Ryd$KK! z>AxNN!8yK~%x(c#u9_T-UiXoU!Q8A2j5VNWHw0{pQKe=%%F?Z9Z-?^#7_k0(1k^&g zjgaKG#=AE{w2%5FcY{<#2%6_r@aNyRZhWML7fn$yTub0l7e3c?jnce@(qMObM1zaQ z8eLRq6@+}}=wvzx6gjzexX~07>*4`>&{dSGwJ8BN#`$EYJXu2H1ce+-xvQ0P+y$I@ zukaJ$-*3l&`GX7P*!=jGi1Qk+2Tb89)7oQi9p*|m8R8`$K zgC8&xjE^H5hl+;lv_8Xbty3TWHhPwQ{NQ6PD9L+Pff>`sCcUV+@xTAo)6 zWeo-4Q8aXM3S)oB8nU7)s@RnGhjv~T<-AV}b=YMuTKB-QJ(`(yWaz$)q92Ir5JWth z0rKavJ8`n~-nJV{eVnTNKO@qAB&&7%PB6Qeworq{{@W*%%tSz4+7 z^Szcw3j}^O2ZhwDUt-I)9}!NS;l77w_gDu%flD1`-)1mtjSuDj^Mk|_L%Z||u^Xmi z;-R}FV7fmqN%}o$z@vK3(KPDUg?hzvV>T zM)T{M`%>yD#otEv-%{07{Pumjc%nmE=^U3GjrK_oN=6so3%wcA+ahzQA92N#Odmz@ zCB)|fWNT$o`S@PlS#>Q{$B9P)t6x%%1e1SJ9sl-0rrZB3D#qa}ItBI&OpBe}QI2;R z=GynzG1UY*OS0VJoQe+ZIVmn}0f1y7H@@4Rp>P;H#)7D|jlRFj`Sv24B@#*^Lrziu z(P{RCs6rKB-aJjB{#q8G4WLx zF?YoeG7T@dlH!S+A3HPaL^MDF!xciU?Q!_(6vJDnZyH5&oysuDy><21XD=YZ`92=h zrHX$`=Y7NSy!FnEzJ|L}=Fg~MBGxnjYHsv%WIA)r$o>3|+f<~~vXBNpp!GI}6qX2f z9*8WG);QI3g&h!9w`#TjB;^vZ$yMS#>o*M|Dq--n%%2$=g@!@WLQOW!KiBXfko<`baroZJZmvaJOcQtEOUuKd&=hXKY^+vGp=^8a2Pq$w(hU zk7IB(weyE~rhr0U|~X3rO$xx~1=NU2j!(0FKpXN;Qnr$DloETr8g4 zDY1tXG1q39YSO=3hPGLa@*zuwQg|I9=b);D>4=pHLp{Z_yIEdZMJE~->X))Gy(#Gx zlezz(5a^-A;@mOW{fwT#6w8o2{gkw+h)@eEarljhMXv2SC-)vtSDzz#UxxzM9?-iBfFayX0&; z!m$*pBszRc>?z(r*>qo&U=H%hue1};QZ2G|WiMceHfjE}b9oOhjgL~>l13-3=vx(D zwgTL!a$7&t(aW^gwli3B%RvIVfxuZAsJkGS^03kqb)Srt%c$fGRFwqdb z@7l`}`RRD4?z_`)Mz5S38v z6JitEZ4mKEoGhVw7f8gXgu3VZBg8=Qvqb-q`Xd!%zj(k3wNwp>b_djRb+zJR7zha{ z*;}{+CAFhmbrnpW5U2qO?J~2WZw&iIyekUIYZ>lvmhE5Mqso%kMo1$JxIgt$;r*qq zy3glk*bbFGSy77zA(RZE8hvqv_e{B|E6KE!g#L=-Lfrj}90g1>^N7OE`8G0m3syR0 zbm~pS&`-y~J_c7m?@ls0>A;;@shnGAQL(bhTV24Cd3HF2EIDNvD!3xv)c!jF($~*@ z_hf{7v-hTL9|F<;$ssT!kodt(l|mOn;51*KFRwK7Gh1qQ`0aig?gD(p;?S^`i@Y;+ zk9Czfxcn&+?2Mm>yDHu~_Gts0e%@HRl0PV)uf9b6jFz-}J@Vioq-Xd@-;MPs)=-4< zrRAlwB#omNNMWi^B)e(kTJQ>WUHqA!;>xHKunf~OR~CNxizRl;c)Y!t18PhozDbBVUFkc%A^=|KKMJV;y}E3Q zG3AIZOa#9MEw%ayOHqY3Mk2Y?omq9lR$W*0Y1P!K`D9uM2ZQg_&n);e5Gd%MuggR8eF4sf8CG%=AKB){6hudFf z@)=W0xYOPNSNdk$SFbmKH}&;94IjdpM~wTj{?ksO>M|o)1oG#HeU=lknTYBn6KkSI z+Sg&`?cBo|xJXh9=Q2}FeEF?mxsSDd>iug;i~N&Yhc)EfA6+}BW>%%Xqx)p#X@R9F zoQH8jW46~ABhLKmfxAN>tJ~IzEg{VCRPG2ZW!nwFta|uMDI&B<#@HnGeC>k2?0$%t zX@D3-SPZg7KyrCN_Aw916ELfmi*{m+f-(T`w98C>wUNMOYY{3qcJTl)fHC(@^PD+&+oR0)sBETZ|i5v5i@{mJ`)aiqn)P= zNYu1~h*H#bs9zRb{rwA5JP0iCnmL2q6ylvdOMLx<=q8}No(Nf9$CJ|jNx(Kluq&F@8yCeliQ zh?fLtSC!Ya)(Gq`1<-q`%Nt-d-k&5{x8aK0o4|8l0!^Vd$TC(sjA>5CVR{F(f_fqW zi!Q!91?b;kz0QI{(|yFagMLacF5I1uf%3q8ctK0i$5ow~T(}68RFqSwth!xoB=3SH zE8AmuF{UyPQ+7TjrRv8W%DcB4fg@p9?=^h{XAV5XmEYhntlbj$j3+SDY3dy?;s_G&HbQ-Rk3hbfo0d>wY2A5jyyw_?s_9rm8p}R8+DUMrb za;crI;1;ux>vwP3CeQ|MBiM)krvbXGvDKU|;lP81H7CrV%VU`Z#x*qnUKmy~4= zRG}0VkTyA*-~8vra2W>;T+ziC|2HmHwgTaOB|uihG*7z&pMbo$4gNyOm~VDl1p$4> z`0HOux+_rC@U|oih*L-%Cp01Xk^|!X{)bC@Lp^t)CMM_d!5upedm%4?mwHt|Bjuk( zls%M~Mk&PV;=nk=!m2Oo)-jK#YP%3Jbzbxu?e3O0a53y@PAf(i9|z-tAlQjRHbTOY z3)g?RKmDPQnjfqOpMOoLwKxKqd4#ExMN=g-xm?kqmMUni&IG*AA}fG8}+WyJ^`Gbc<8_k>XnnpvsBjfb!;>uIaKyO+!)d(t;G@Nz%W=RLrN2pEh?`J+aJA%>nv~)IJvAIwRY{%QY$_QYrI5rK=;1bGU<7>G5bJ>vK6A(^>F+~I{duS`uFXG2W0a`l8wR`K z6ZmV(%;WzbEtXFFx(eKq0+3q`f`_1|K-eNxo%7k) z5UBj&yeAA81$|KTb9K+dE5DgszV=21t3rF?S|FL9K{i`|EO0+EFlFO+l3Bn5OBYpO zKLeBP|J4d?@TV6!iZ(O7nF8S5mk14wwSd9x08vNPfEt@7nDkA9ziATK^FD_z8vL-O z>FSNZ*?{QNe6bQ1wK`5OvcoHJ#4~@hs9KN*%%LGg_Oj4ocr0S_Za5f?_?cg0< zHdKdkgh~I4b@Kc`UigACCv*@LljWRr|IkpM9eNn3Xohg4r5O%|vjCm72h#yX>3A_1 zq7m-&G9L)4PbMz>my;n6DFlTkhfbRvVo@gXFClUu0?)uM#ee})Tt)n$2n zm@gg42mCMi@3*E1!0Ov(fbquDqQ}X-*((E&qnYIK&tftk8#gHgVlHF-mu7gD)AJY) zs1hP0wiE|wcXJn1?(*nV7fuK5TX&iS@*PBc=`&BHUHXabZ@@)j3O0{4#MRG#D=gfP zKUoVMyEMQgURseAI|>#O1qq&DLiaMj8$2yg$n_aq!lLCVYN3VAbD$(k3&wPOk z+^}o2Z*v`{PyqGcQd5$#(N?)JQ2t4C;sBN>(pOCx1p=8?I1}ytlp((mFcs++#F^v9zZ)@jB1h48m5$y2pfOy`Zfs!>sHB$C5YQ+BQn{34 zNW$I)hyzarPzv;|!<<0sf_@j73;EDyZ(uqFg6X0PVhLzDRuvVKAAI|j_7v)TP2|(Qr0L+t;G*$K@#^MJ2RbR$ASF9nN@xA2Ggwm}vr$I9 z-~bmML69W%BR2nFxJfQlTy&vX2hV5lXNZcyq0X@HjbH?DoHnqp#uMKl&cH!e99sdg zgh0#&cTHFVschx$DcFxdX%+h_Y3e(9#UK}a!S(QE;qN7W51dZ#xpu`h|4(2;q{AQb z=+eG<%curNFa>$+@bbL)A*Y`G?|YY_n%uAY=Yve+3*duot$FJ4V`z_uDklRC@q87O zn*$!dDJ&qL%?IW|3kz^5c;hd_JT_A|f4W=fFMHDSaL!Ceh*D-ho2A- z;GH!*<2dKdT?>s)7N_y`$M2%;`VC$@F7Iy*u7w{w!c|{NY^~}6H$h4nR%p8$YTPP* zbJ!i3XLVKT!Fjw0W^EfT%E50ryb^7zLm)Qq0zUiQZu!=l$Yn_e7jj4d@U#m7aSY+h zuZ0d75=#J$c7ANyvsX=cC+(aQSmA z*bJMjGxv-%uaraF-g-V2oc8j7F`xk*sZCJPpGsQe5R*v1G3;1~F#xQ@Q)y@IZkV2z zyYNB&X@E~DD6}6cTR%VnO8zIp3x1uOxrU3t>Q93+;j^6WXzTtX3@)25}efwtr107tA>Jmfj*)s+t=W-+BLy@rXYKxBoq>PMPC! zGNw%GqLZJ6kfV+BFH>Wvu>h%+^NbAINFNuORp0q)9mQ^EoI4Q%p_<+t*Ak63;V7s! ztn8u-vp-ZS-<;Ub;ouHq z(zkFI4;xU_EVZTM=28sTO!<*9O-EmN|&+d5DX?2kHaor{djocR9@!xK?aaw*tVzRGER}4HfrJ^oP+>zEog|d zGMQ<81kYCoIw9Ujx3p4j*7MM!Sc~JzG$k%WMDpoHSceXw5YWj8I#ttB)wmBjSRiF0 ztOk{?2#wraSO(0mF70b;pHj&!6zLN+ZI~R7ZZrs(7^l+~4R}+nIl< zMhbINc>qry5o1VrY$%-m+pPM|<^M89A&_*?R-+|uG7HSbXwkQPBq|6%OYkS)edEA2 zy6o&-nOc@qdeUy$&Q<8DW77qP2&($sRyB{~P;h{5Rh_IBnY%>YXsd3QBI?yKLm8Bg zpqWiEvByW3w2vIVN0V?4hR)!{jtzl-O1;|U?*&pyb?@(5}`Q1Hr#A=rovJ`M9OME$m z`lFr8{R(;YX||(G$G?v-qY{Ckh|eK;w>LKf;vI)4v3d7jKQ+ihH+n||aZ5KzP+wXeGHy^;b= zq^G>gjGEh~&h;`Q|3e5rgw1f=nIHpB{YalZ&vjzoP8eqHn-138luTfd44jqBzrXeI zdq66r+=%_}Rc|Hx6|;3QatILOsc=ul+M$DHX0PeB*1JTJX3d!Ie1Kv4F3Da3tbh@Q zSBnt!6JAi;-sZ2G1Y`<)h=qfXWCzHn@%)u*0Yj%qGas^;mdSul`K|da@ap?+zmwB{ z|4!gD{(qVzyhKHYC&=(xa(gNQ$n@WH%Kv}v>i%rTt0=csBKh=%z}U`lPoYTP^}$3CxW5Dr~tt7^rJ@O z&gGW%M;56nrS1c~MEdsq>dLF_l%1cW+i zTfaBD!9pxi;3!ByQBvy%Nm!xl%R60@?JHAIpiP9cUK=_}7C->!{~4H-k=Jzfi_IV0 zbAR;t&LDK1tDjD_K?UP4 z1t8$|3f_-IPKU7bF2jEFFEaJuV+I7mLO-Qg-kv5`B}cH{N`=_73v{RlR(zB=OCV|@ zk}bjM&rMp`?*uxmzw|*ZWJ2-2q?2EXPKsDlr24(euz?Dge6&SlU zYnW@0H60*Bov#RH`8PY1>#4vzKV8#$4|oW}pw;_u{#!~_vv{=SUA_028^2+oFlJ%A zvv^UI+94_*l5~3@^>PxV-^T-6afa3MqzHLq@{BU+8Vd5JqeaU5)!738OPVf7(rbCC z(GAK|K2*_xO3v{r_mLw9sd45`rw1E@&bV6NH(YXvuI8mW(e;-_wy>vkutyut#vQQMsjAp*~_oU$9>v17wvS zbvNO73lPB+;-x~g$ZNJon*iSS|8O^Z4%KWPXbPDM+NsV2bq*mABqv87d)PZZk3 z!yTT6w1czk!rb_1LNy&D{rVTwWO5y>yw8xWq1YZmA#34@x_-LZe1TwqDz;yul$b9| z9dm=Z8-iuy-Rw_NuE47IQ}3gLLXH=*%8&GvJO@1(c--ZquwDT8h{;bN6a_GrGB3)3 z)14F3VzM$wVpj7~?|2u~*Na|&HSzPQM70Y+@=YMOGqRgl2ajnkV1;pK-`o7Il?Cg# z3+dSETcG>G?aPaM9QuJHo3G8u>HmpK4$9f9jbGnCP9uk?IJ|?FnS3tP2Hx?$z!p`h z65GIj{00>2GY7)$k)`lE$h+1Srg+*uK(Ue=brNO=x1_GgwXT+M>O!>`)-HAzsEW_3 zjNS$ieAlrpkjqg$qv!f0q0W9WjA)%_l?*zLA92AerfyqfjBsN&lqoyQi*Pq?vV|y+j7po~~J-^D42pmWKufY9Fwtx8TVopFe zm>-n3yn|lZ_HkmBY{FhAz;5J?g)RtL8>YrwH+H99I(6&V>nE?&*@2keYZyE64oAJO zfL(?2XWj00s+xfN$CTcr2Mf3to0mPxy8PRIt?jNZr=#}Se|Te zni2H+j7cH;Te!t(Wa=R&1GWIakG2Qk|J)gs*fD=`(;`Gdv0?hAee8e)e?!OOH|adf zMqFMn@I#h42CZCx%G#Z5kCcwVNez1{KR<)jKc&D6=R^}C&rc7-#v=h*|HVHyf7^8$ zw+#kt1V<@ENC+(QMb-6WNNeYl8#@-;>A0^q;%tn6z5r%QkoF2m;@qY^(!6SK{s|#p z*tD4u`&q|f^Ujz02_XZVz<$JALiY3PLi`WH6V-IEmHZLEH8u!{RNDYac|QODEOQ^eezgMQPw~`ED=cds1{7YHUC_PFcgOz}wY@qadf$I3C`nPoHY_FEnQ=xy$0%j% zrp=?({iBkMnsdN7ne4K6;t6R3%5K^rhI3u^MQ(Zhj591-VF~!INj0WV>oY99Q0}}Uw3cW3`siFq{<(cKAmj-f?PB86Lp>oOE8H- zP0Vbseja=T=#*&-IUy{;in6-o4u@nSxZv6hGD6O)TkD$on7?Px`1fH0d&*^MhZfbw z&l^A!)ZTxwFet;OCUG|RKCsZ>b2N21B<*zO(gh!ct+$)UW!hB1jpl|iV55NlU>2AD z-rxbpR&914vUlhVNI=9BRez5FSxEVP1M)KoR8!^(x=#vYay;XCJtDbQq zKNj+TAD=FB&y&@DwE>%RuWsaKJY$)vAqfXu$uGJ%VcEE;`vTU2Gzik$zvSTvTgAtx6(I~3GK zpE$~RywDneQ|(tfz5fhDuvNmi3T^vL^EhTkZ9Gr?eB0VMDes==sU@*Gybf#Gj13rHeqNAeL^SN+Tu)DO2ln)IaSx z@b4qrwh*il8w_*SpJre;YR{-02s2F~)&SOgtr2p+6*)K+S$C1Nu)I8yvNA4J%3k4wGCEDN=g zuhiMwtyf9&YF{_2c5#Dt7~v?FoPo|t>ga!OoXvDfdX$hO5xQA7h!z*Kg!N86z zsIn!=b}*uMPrRf^(KsuiJ6?iv-N)K@0{2uppc~wJ#GO9Jb5wyp-T8+4x_I?H2ImCh zT|1YjAcGlUc1aaGnZft8;}S%$g(}xCqgRdIwmdxjV(HT9B#fA1)(ACa0g^GwAJRB1 z8m#)1hkKyG5qJ{&lBjjAJPI2=?AO>f6*@4Tct)pc0zw(-1MSG3Gq~-165hd-|0X!; z<3si@iBGJkRe+^rf{p1Ha*?P853j{D#vZ5)e|oluyrvv>z~?n|eeaFsJkqUpVAb(Es%|LnK-CQkr-N;TEUX#eh z8V|e30gyrPXNCo`K_d5LH+?YX0u(u*j=z)16;!jWz#rSzw*U2{2n+ix*}ZT8aO~;s zH<|Xo$F!m>`d>FH|=~( zsv!C}&1ZB6aQ>y+9zd7=*1LHAh@`kcmN&)U(Sd7z7$CUn7*8B z`aCM+yKVg>Xacx9Pw(dP1uq}hCf_lzF}2@C&on=?dp+V~9I^HAd|RZImI1yok}u!b zi2Eab{fCMhdza`K9Eaa~g7B&T;2`0!N49MB&QLlmk|sf45HBt0Q_q!`4@}@Y-3--2{S~~7 zIOWdy#NroRFDtT`l-*wZvHzxXHeYm9cB|&3p3*xESa6k zMM{wWY@T3@pc3{eQ(N9iqCwGcr$NQ{BsTgMApp3v4xbShqAII3Z`$hkws5ri?ckNS z5ELzJMM^#1m~W6=-g3IVGUV*!|08wqH2v$ruE8XQ*NlelmuTwuC{awTL6R%n6;W7( zw-xk_ofj8*TQ;w;-4r^u&DgB?gRrq zaDGAM!rBpVyLr2$B9HfaqqC1Ihk#@G6uznuO3A0-<2#<(BLdg7`ndBYYMC`W>RU6S)0zs7jB zXcsBGFeTny_GYuN>DB193isE-hz*i{32!4f9-7o4&ey*pdSpH_LdJmbv?6Tp#&a`V zw=Ro>b!k;c>Hamx%tN*zLtF0gsl~;`iShBd_slW0%h$f!(p`^^e&uB$%G1uZyHjsj zl*y%!O6q!(_~2rCO%+qwkM;eiT-4(IL1nc(;r6Z6RDK)|cP^8p-$)+|hWUhT!kI_5 zQ2)Ds6iqQt_qhxYb7=qY#Ml=xcM3VgO}x@@usoB>7s?k`6elV{!6BNz;yi$3y*hVr z@80|)brp*Hk90g2>E!Qz@xo;CX|^phvqRzZ)RbV<8#pESrk-!a9IJXvpt-2!1Uj{n zRf{X0$~m%POx>~x6UNz7EjJDJER!yN)Q=b*Hhr)s;3J#!YS)rmK67Nzd7ID8@kwe4HHwV~hO4ZcuW(y{5-7Ua0{ zb|xFs>#;2s=f;bI$StBrC@Y(csl=u~HN1?OrmVSFXIx#r!sH@7_Z8!P^gP~$R4-Y} zdJhIn`SMJQ_dD31w@~LH4z$>xa=*A*q71(ie&vfSQz~GN;jRas{B4+fMm6FhBZctJyI9tC zv+=v@kn<&g!A()DjQjBT557E7Q)u(*wN^^KoD*Wuo8sCyfg<%eOyx6uWa%srZqJBb zSTk0@$QH9jeh!3wR9|v)zdU$WvK>KGzyn~qhkk&MV4(9e5sl`Z7}7o$CTBVw?#Beu z4zw@tFp@Dt9EH~{9oPRligHmiOXfVvK{oIj#Jo7KIX;NR@bwG!iTB?6mU!(gLgRuW zS}p{_s2rVbu>Z$-y$!3ij$D<1LX=r0Kc%cj^B5-PW6#qrdlOzf5k7KfSofV0Al#N| z4f`@|x24)aX#~RhGDB556VPVt2Wsuk0PH4TzkEBiGyClTeM}hH*h+Q}f^loq_3M(m zQ*v@7K;kTIXJU0ZrRLo0=VU-hw7k5%V@zR2;cWcnE`AJyCSiv3n#c$e_8%F$j@W}q zVA6Qt@C{n~SKXj(Dag$w(rdzxQOQ!x9UMDR+o=PUAg9`Y9|Xo1Mjl_e&ro9ZxVEjM zgRQcqMHw+aVs18#w70cU$h;ovhP;v&=_WasxOIa|+kDj=v~5>a zRZX_CFf`;2Seq9Lh!OA^u|k*}DDVypWh(|-$Fj@WwLRZ>XZxaNvLlXdb-wEH)~~tg z47c*b<-L7Ez#XSR zq9d2m-4fHx7^wW2L~OXZLOL+^dL=4NU&|I7`R|(m&Xu#W0C1-`~@iN z%Yap=Z!`l%HEgg|{)l^%=I{s<+Kp}F(EG8Wz#P<-4v0qQG0_EeW#pxxMl+u#X5rLk zXlUpFzBg*_UtSfX9hFDMZ^cngXcE9T_%j!h*G<8mEMtenBlDvV zy2(oB4_xi3(m#E2c~IYEU0GD5LGUZZFzjKt!$LvSP;4a1D@%Pw0{W_;4Ic&G^z`&E zlK1v-1%QM;NP%~F$1ji&AF=}_NESd=B5L!HETsVL=0N4ES4Ib5n__4ki&|N^#FLtm z!Us~1I0krz%=vfqFUbc_1P*HxkGj3-(cZ5T_Bn(_xfTFnHH-LbaFD6dv{VD8r&j7# zhfS}u%VcDC0OeZai6i#F{$gx5to6Z@Uivb1z601lkKBV`L2CaKziu$Khg#U z22T&t1OQ8mK+=LeHwE89|0@0zKue-* z%5dsnmrhE=8-StOfXbK1p8#MEk-5CPPd|AN7z~^(XJ3W3Ei+li?_GFXcl3+W2|uSN z+ntJPR;mKmy_D86tqGIJcVB^*nnj$6Ox;NQ`Ps}UUKzR>_LmghIC@Iy@=VVGKZsjz z_{X1w?w-ngl5!+|UQq=&8a zr~MCrhilrkmMD+xk!uL;g%I*nnwH-@L3@?+Rr~?UsC#Ju0Q+_+P>ch;_{(c-p2yF{ z2{oPIf#TBwDLbz$jw>nj%<|$7p}kI}J7O1c$Jp3+ZTY}BfCsJrFY3NLoa(i0J1b;} zmaz=6Ov#)fWy-=*=AucWkYOpxlxQ@EWJ*L5qBJ8Zr9vumNRd=3Ly{7ul=#lu-p_vb zGrZsXet&+?aqMUBBQ0yK`*+{heO>2uUgtS$uk$gI?DZJp$5Yz|Pp3g><$m(zq|nox zKP+=L-ET7$)wrG{)SvNS+tWTm=6z4d@h{7{pC+%(iTcnOHEu zajX6L!fV(;PcKc(jk*nQ)u>Hx7m$lVn3FJ3BPref)9CL4@;0f3 zg^PvM6clXzDh^^4y8dnYcFTLQlau>HhA!W51E07Jj(^v?1;2Y;Py9|K_=sTY!3@&H zvAcKhfY4SbEhvL%slBs%F>%BE`tkW%!sY)kz4oW(Lk!x=68&32At7@T_Yv$rIGiy$ zyp^cLj(>bE12MQhDGKj3&pkLhcd-S=PHl3&ipQqBwakXHo=D)B`c}lVR%g+{XV1TR z)1FsCe?h-&@Gl!a-`jL}h?fz?0||5sHbGIBFokH3nzId*z)}XoEiEmm0 zeVi_boqJtF!!b1Y&Z}L_JGD26O|<4YgjMY282M8;6 z-@|(#$>9d!g6$RdD`Q>HskHxsxbX4(hNh5!jyIO zz18+@q5=YIpJhK;OOd*dd1>%##+LFlcGg5>Ls&iJ$LSvPWqY7WqxCxD z3(+Ko%+5istVL8sAUkuOGdn*@Jl{-&4U{3p0ukhkC0COaqUK&gi@|*R_6ta53isAl zTC853j%(oJ_u*&P;2pQC6HLRxsaM7yY92nkFt-ha&f5CpN>77#mG=oNC@ZhqO{*CH z{Q9c)(5h(ByGvBvjz0I{K0VLv^&8#&yWwo|ZQ|__wR*HxQzOL?b}?*SR{vu5FwrLl zTla$Zbs!2B5OO1WD|WwM0mc5{%;qQ-ZyY8(_rpgx4Cjy#c)>z-4W!A~W(Ypp3BgBr z#TF}y3$cawKDUCoOhS<|Oh%a--cv4(4Lr64JGhy0f3p=K@hsW(LC0o@rstHU^wMCG#HF;hhaE?D1Jj^xzyTDIX zG`RxSLWA~dNV>_3^OUmA@Lt=ya8m$%NCiNhTrjJ1SO@_ z2Hi#$@$H8AKIlX)AewfvJ#%o>4~D+)ScHoKj?~Ga_sBy$BKtxrY=pElw~P%(-{uht zi}qHkmA~Vhcnz8f7Y|exeP1xXQV)sm7E^=6%pr{bIy5tHWG@V3_llM~)@j(-wXgBR ze!zL*-YUu=rE$lHqdG{%`P~26zMvPz+6&XRxF%kMJdJ(&J_FaSrS)mPZ>4uwJyXc4 z)v`1*ORA}1l;boX4T31ij=ILibrn*b;kFp0GCFktL2KRd%0n!(W=&7a8p)0Q!-?_Z zsW~SEA3Temrz2k{t-~EB%Vre5rOw2Kw zw?qGVvl4AnT)+87Nqypdo>W<wLY3hhsK z^pDM2?q^KwfZ4bNlhBB-hX}S%3%jI6nz90JzGR%~sP3JJZiFtLq2V4^ig;pvy6`4^ z@Z5-nHq_BO-l46ShU4-h(Hc(1L{zjcRpD4d0aC)LZMoy$h;0+papW~Un^TppRD zIC7FpNOAF1lo+Z#YcO0=fN_pd>}uw3m{<3bBZGr!8OtwM6JHElUONbghXynq262JCqRuP-=uh2d&@i^KYF#r6QYg=&v)<`EoJ)wm29Q~TN@p5tgADA!{Jq?|C zay#17s7jP!Ld!;Pl#Ql>DGx^oGZ619GhBb`DL9^%8lKG`cilkfQrLJUi$vEI2zN;@ zqNH08*3MP$L{UIAQGZ+=9A|%b0sKHg#dT(nVvE&4{Q#Y?pvh!-h}anqUAb+uW$&Pz z(2;%z>Sqx5k0F{^V$Gm?1hL|Eqy0@vm^CrqkV8h8DaCt`RMx{l(CYjCKMEW(LO0Q( zA9=4J$mDy_&}>e-i~!MQkuX5UTOf~njDCrulsH359Gdhq%*0vW04mHP!Vu6O@7y*c z^-Cf>E^*$N1W^dFcLzSP9!2Tb01F^K|J!R=yGU<(!F=gekd6Kz`2wR%1qdYJ?=TeC z@sp@)Lz0AYXrgaEA*U0ak?91UU|6 zO+lh)Szq%)luK*Z#MlSfBqE~dR#^lQKm5E>5qg;|5Mf<}4E#92-&#V~MWKWHPdMyS z{b|E{Bx@ih(&r&;Q-LWx(b((Ih8OaPxmB(6UZy>0C#LJ^ewUHTeGr$>4_*VWdPRik zGz94nDK_x~ASR?>EPt>wZoZl-A#nsk1AOQ($n&)^0S|;W7S?3KoLUR>WyZtY^FKYm zc5cf&OEtYcpOlmQgA@BMDQ&ovgeH6gm?UcG-=-aw*lYHJWDUe%PUpbS()J7M8`N3| zpM3Z;4D{Brz=rERkzQIO)9tI0O2n_S4^4%T48DHF<=%OJ>|jA8!UZvpBEWl>pbyBI z(+KPPyr%06E1t&knX@L2e&64i;bo+io2s&TL%N&v=M19f-p0I!r0RsCQ$kF|mfGL$ z7K?|wS>a+du`UORQd3}o1`64oidU@Uq~Fng(Jp|iKIu{08r|*mezrR{ufx`SGroJQ zktZzAt38?i^6n9taZY)Lbk9jmKpj!$QiQGtkmV6J0et7!_HFl6m9 zyMKY!niMeCcH2%lJm~@U4kI%%$XevaFSgLx+e_um*WEppEE503Bln5}U?Et|9xKO9_H%WFoH!UvJ-E#0^GNg21+N8btJ&#ii_QzS*M znEYgXatusRBAWPIdmAP@XMjJV2RV=EK|0{i3!+3Ho6lo;fp_?Hbl@S}?Mr{0T>FZT zvxaaTUwEWbn*Jzq@aw)9A-I}xvwVy{?DQ>j*{Iyn<*S%|{kiRy{zD{A|n%pch`!gZ+LAUale#mG|B~yn% zj*po*7O1RgDOu}($ZSUZQMgdKvF1}51<2e@(`|NPQ{jS=CCVkZVdU!sJ?c_S6(NKH zhrirzpb97ie#`o3l)MGYK*Os|b^_p)`{!48?5=NLPp;yE{S%{84c(Ac^v6yM0Uz!6 z?)|X$Bh@n0v$Z}C(0o~8siXfRmQ@+7J&!sVy#4W26%hIan)EsW3P8l|ygq_u{SfZP z)Gc@R5{$18jr{ML`UoLp%GIOsMDA_^S)odsNc8MIiJ(Q0!u8ow8mh?dNJd_Fa@p1W z;6}AU`teRTB-9Fwqqua>;tq80rFPDr!7b^!zP&ZI@x?e)L>#&Q@ribhXqAd99Lc-r%uOX7YHE2S0!;3QD#hAWO4 zf9*l*d~5~e5JE>kXv|tGz)aeNF#l|QDc-HJn7xKOdJ#*R#^9A<(bf z0n7Y+y%)0frA@d`^YyI8KfT<1(lul8Xfd|(RWcum>V= z-c_w9a{15Dts3MeOpq{8#B_Mv8NGLxahz*pX3oy>|8RCdC|&bTXHY8XJ;et}NajXw zpp9)1>oTlwXYE4i%GR*r6d!ZWJc(42SDZORm5OF+*ZNNGF#!8aVN*%|cw()(aucIQ ztoJ37XtwgFNw2-JFGf+;)dKLlY9w%>i0>9{lMe*c+qW#5b=+RvlT=P*$cZhO?bP$ya6k}Qw~4{D=@2eTDK?e z_dL5GEV{2f$@Q$bjz5F;nAT1G7A4Lkvh@;o@m{?UjY}r?+5=Jt_d&R;C1y!!mB*~b zxO$%bBt?>=NxNt>NIQk8s!k4Unc9I95WbX9*j#oA@WgsHN8DGFmx&v~qeRJ$y@%Z6 z@8}GPeg1H-Sypem@(s4{7`$GIw4lH1O7LAPyXC7Fb!@u?ZDd*)6Nl=^qTw}H5N){R z%A~>yQFtHG_1}`j`ZgSK)X{32)k{Vmw^px6gtAG#Jv0m^13XzX?^X9T;ZCN3p?6K< z`MpE;1eG>e0joXZ-}t-&NX006 zdDExdX(Z5~8^P8Ef!$u{0X!|pAR>?z;9E>BTz+ccB*#r5r41xsK6Ti&AKczXE1@|E z+En0arBs^fbxt;ib$(;6vQ#2sy~ezF;Y@%DO(*+BYQBhntdzomvsTlpzceW-E`R(3zB^Z%o-=8qdU}bA~bIGYKrOnizTM7J}$+ z@}cz5Z0%Pjtr*(Rk6<+2YUoGas02##$$OF&RW_NOOJ=#wz3q*GI0NVJHlT+B6QD^n~qwD_^j|HUKWwuA4W{!y(tz6vsvI=iToh`#6_=MAm8C zn&O2{a+*ir75PsuZmhN0KhYJ}=;-r@w*qE(#Og(keDgm1vd z>q7=TYRGe*Q%m;uqt}-a$)mp{2zsdF!Lo8 zbZPdYan`KVqvLq$4VWO6t#yhP_oOm!@A>?!Acu_SV?Jsqq{^_U^cuNlw`Uz$sOz`7 z7-BB2yZe{##8tlx3%qk(#icX!v6K~8ttoY+oDjCQpF`vl2x>Jo{xU1ZseU9e772g! zSFw`@5XyqxIc>1bk>$7`{LcqbRo1Y8%xwM!xSR9BWX0m!?1I%R*-U^!-u} zV&Yi0exp;L;iqwSTc4Fz03#l{Vfd|m?XCOgHvh6OJVQT1Fm3?6#)#g+iszZv@khd5 zTMn~U6SzfKwc{aSaT5+nLgY>!nrqP!GDz?_X!(^P)L#lDtkyg=b3U4qLJz2}Z1y%= za>`&|kmv5`yQe`6#hs^Me!BAw?{rHAlzWefZ+l?K3MaD56d&YT3}cEdxxjdmlG_hK zU+Y%91`?3L{)ECo(p*r}EsF5WHI8!p)m`Rua*MzrwfU-!_C#C;;|`;0ZvqVzGAw#m z3_UrQ=-qwO)*^LrvX#=WiwZ3tPs=3Ww9*3cP;;-v*;vAS#J7LF8O;iR;6mZvp78pv zNxW1dw;Lb7hK$CX+ewJt7VPsRE_srP4)a>GQ|+t{hj|wUp3|o54*i;DO>r^=`}Jp> z?v383g^y#xIp#9lX}cbiIfj&7uwl7%iIY#nEXDwisPQ+~IgX&KPwV5rn#}7YOEC~{M(AMb-p8w67&TNDi4}8S3Cf>hgoR+U6 zLYUx3~P0eNt**zlCu-W!g#h8uL+wDJVls?V>wz%2pZIPg0 z>%ktcl+EuXU1Hog5^CtK-m=|WAk|G~bIU(<7MtAt`9VnsvmRe1cajzz>+_p9Zvnj+ z5q`A!=q0+Ivffh8l5evPv641MWz{@e@bbs!^={k}Vw%2A+Tz*8lnr6g_wIeZ`Hr{h z^}Sa6JV~z8aA32YejO!O@WXv$VH8DWot^h8+(sm!_dY2{ij_aT4%6<=!S>_2HJ9yn zG88Nir(etA<)6bQeGYa}Kd0b}CvYv_R!Q72sZ)fX+Z{vs;ro{@PBJC5`n!)3tui#~ zAbWJByc*Yi|J6tRY*aJ*pFaqg0Tb;3N7`iqVUfvX^l}=b|S{Z+D869vd>qNoK>q;yLa(VcR%!4`kSKwF_ZP$ zJjqYF=N1g+L;>!x=SXphuq8jbd1DPj zlr2+5vvy{jmK8U5e9npF1DH6KwH`O6h>&V+cyaOzr9GdzGxgZ3J)lS-wR;>9YA}7b z@}^7n#d+Nzm+zHG+b0^r7R&ku!i%PHf~}4_YBxsYH=U(%@wBeBU%}-UO|0<-LGll! zn`}|*-5yaxTJP)pb5*1t&<0`frg?O`$--WVpN!I3Q&kbJX=~p~iY*X(n0P2KXSH%z zRm%F9D?rd4Xm_bm?1NE^u_W6V-O$4DLQNn#POuOS=g?&Am$LhW1J&yE64UK73e!)! z4Vi}{pTSMZOqatr6>C1YeB>^$t`cUebj4FSo0w9Q)STF7ODNW2lb;`foUoqp_7&06 z2pR3Qea>WLUsHKKZ89si(iXtouhndaQs|K=eoYtnh6jBXWU?DACKIYHZSOy} z^AT3~uM)+fOOo|tmp#gm;aLbipTbzn~yKBJ#H{F<4ABt7%pLxq;4 zkyn$?By@af4b(5!_k7;sR2@^R6r&=@ac=t8y=iXf%3dcb_HFN|yOZ`Tu01nGEV)Kk z)*2}L)gsg<}x90Ye_Y`Iu+t zRp<`zpKg`(pNkoNPRUJ3rj&75J*s4D-cY+)Gs~@Gv?f!y@6F|c5ndJJ2SKvauwKuJ z0mWZ{gUax{>Z*u2>z#Aw$mALiUIu;T)=9)3W`Mj>T;xTf)t4+YOx$|ixdhw;6h0ma4Nvq|1 z0*~S2n*mlnb#&Y7s$pm+z>N8-ar>@|47>70Mpa51DH{ZS$+Yn#A4J7|<5Oq!P$q>h z#(EZQtmqP_&huMA!xs)$@s_j+_Qso}o)X*BD3|xV@(t0^y<5rFuuG<#yHG@IEACx8 z9KYlssE}9B8WM27&2Xqhr0!|7(8_`P+a)^Vqo5Q7QAIBU` zliE91=LvO!?aNwp8k>~*_2;y%jmb0a6&gLK!zSv+>;EWchWpD*EL2ZzC){6O)f@Ny z>!Y~mUkEf|70%x@Bd%g(X4Gudeg_))UHABy@JwCP%E}ibm!@zWH{r-43e~>WD|%zE z?0pR*W$q}ZNmbk9UjEhZBwpB?m3`P`G@s zhv1N_@f<_q40ozY#L+dEeHMgAQZyI&_HW5#wa+!Pzu{PI-)ZzXmkAlJEqt;4%4U;^ zHQ&LvY3++ZXW`rD(U@CcZu~gYNj(sy?^(;nZWJ9k?IpxS-QC8hZMJ63;`0Wpw|D;N zF&8f`ZGL9=&0^ImKm<2eZaKc#)>x*o=;ER{@YWCT++JtwkrAUVzAA8jTCTD0OJLQ7 z7P~n;i|F@tnt8b5wX~8$!28u{OHs1#kDFvJuH6Sd#cDrWawtN4tAOms4m+$nE~EG$yS|kw zaNHxK7nhkFp$W${QLOwAK>~TdcTZUOem4&RmSthlJadaI6}a-gXHKNWi`vG<7&6ja zMC5y45-a8TUN;ET;a^<4J9}nL zFR9*n%6?_sRMCA!OFK&Nrz(+?rbVPub&I5nP4z5oU7&cC&xCd4`@x$Oo@vA3j?V`t zE>^k^-dYQrtvB!LPzQ;Sq6jn#GYkuegeRX$qOU~FZfV#G;J_Z#^e8ty)VT<(2oAW4P#=d|DF8 z_WLAi1R`GmyD3cJv9y({UNa)W=V-q@Ho%WJW#r5W{T)=VT7_u$b1^?G0NzSxp3*@} zUJ-!#Q`u?zyK!b=kld<=-e;jsSCN1sm$rE24_g|q6wTIGrCZHStoiTP+EZqf@B6C5SDvk>g(>lgWE~m#9~?di(-) zZ$MH0H^RdNH2BYC@ z@d_nFLtsP4jQ=zJhx%I_3+aUAQfkyVdO)1!Y_#!`dj6Z+^x+DQq_r^;_pc;VZE9uG z?5a{$l^(9BN-y=frUWQCA34R~+r)lF!ACk& z?uN%B>qfD>XZCiaD%OycXbMJtFY-9mAg=!*Bf;v%+eOV*5KJ=(zlsg~g*bebZOZ=} zX8q+i?IvgGkGt=BKdZ`xrA=X=)d+HRA`xUui7I^J%Z(P^?UYF@*1dB(#?$vZF5BUNhdLeG~vsL)%jnj z_mR(YT+o%9a?*k{TJ{@D#S)zS(Ut&3%L+2mX;SXjq zb6AJ}m6<=B0~d0jF*IRp>j%}!a?J}ATKJ(4a!xjhW9Dx|ALO6^Hcf@K=%P}PDSEi; z9Bc-3WU1Q7S&NdtL~;cGTjtRsBvn`RPI=r&h#@*IQihn z{M)?ukS?YuZBzxmX+^|WM!kb&HOK+FfBeo_BBR)K^NTXBYQ3413k2r>O1U)VRU>Mx zTXPf#9_*&nkESSMsitX5cWfAJJ#&~eOY64WAEp=L$nGigdb0AYRfaU$@q308op!hO zM6v{-6T0F@&+Lw37%HtLIOg8?<8>r9XY(X7h$yGq&618%>>f+8W!a_xwB$+Z(+@$Q zzfzx&Uo@aa+{X{@UH(S2(}kz}79hy9-jpOy=pWeFJ&sGo2QO$myh@XNyN8F?Z0+(W5ldC5q*ikj*~Rp>wlydqcJMWy$?+4{h@poP#fJT-686a5nDh zplqmIeQ-kY-1I2cJksV8O$|A^2H!BsB5oKPpR}-$H!=Q}H%X7tNXy0_R8daW)VeVA z^AqbpE4VmOU!SIOadZ%*id5iuQ7SF}z_iB0COwHUpG6UY0aBU?-w1;mq@hpAC<6t! z9P**PPuXRJlcDvR#f`wepyRIn56T z`nh(;Kkq-|q6I&}WX{LO7k&8+NQWi*0c^I8q*FlX*=jAkk}H(K2R6A7aSIdBOWTx} zV^~eZO>{JmXQA%8M%9x91q8%ii#lzoMH5FUHZn868`cl!e!Ixbj-U7L9^mkGvvdP; zau0d$o%JUA=sc>yVMlNHnM4r(>5{-fyYlyWN9-2f^P=+DP-CD`$X9F8I6KIo3EY(u z;3^&84mJk=_^B2qBhNVaBBcLyoL0p}JFR2AP00#8o}+^NX6TrzCc)ZShBoftZcI?*5Yv)5=s=7F$Z;Y28J2JL_C0pXO?u{YD~Orse1N z&uWhin?bcflhH=)T~4ssC89zZ=&45Wyb@4pHE z@>vub%|e1tH~1HOL{mGv^Io`uy2N`9>gDDd-6tX$LaHwGrSpPLQ^QJVrYc zbm3aoB$ozaM?mJ?yFeX9pGGazGf`>nAsyc)y*+qRmdVnllO|xM%@AB}Z6+#Yy;lcL z9>VVXVqS!Jh)9vr`3_P00ioo8xhLnI z;LsWVL|T@@ zbHJ$9#XNs=Ra-N7Q8yR*Up~Sv-@*M$adp=6+>n-{QsLGx5dT~Et+ZC3qso<|yqY>p@POlM`!B?p-A0A?o8kjX($81ot3Q1U zn^pf!vwNwaD7-s_xl%@FKZXrb0Y4;)#I{KdlXc=n& z<&dSIk`Bt$_BzBemkGC;U-6-=iB&#hD5=gtYvgK%xc=?a24{TBCS zoyK+hE1m5%7zs5iX5xI(-{QP!zY&M9iCM`s8cBofECfNE&RZFr!BP6PpS}a=mh)ko zm?EG-L%WNV;n%%?`uBINXl2;!^;a@jTqfj3&yg16&+*}tj2scZ!}|t7b=R>z^R(!l ze1gnsheA2gEZxBOIhLMMzj3|xNs3DMB?<79sAsAF(u)~eCdl+#QKcRX%t3v*m!G~H zM7*;dUx{+cVlQjGSaj;&KI)E+E5|{i;^KdP4{dzAVpmh^@x;OaCP4jlW)FX++)oOYVTrcct;l01@bqUYl9{tH|m&stDgMDNM1hKysH`;nusn%(t?>_%ydn{uD zUrc~;4z-UpwNE>s-%5yyyFtKAv)9IixX3woSj4PIbq0gYh`;LDef_-lH?gP*F;N{D`>zGV6(+_u!bknRiwRENHr8z%OA>{J(uir%>V zokf0`ATw6$MX}#iG!oTU$7)z`l`=BCUdeOHvd`>s$j<$h%X~4aWl})HO#vBP|LIOX zzQZy4X|?>7`$@ax;}P8QN8{xgM0YKc<=QVYCl?iU% zON-ut-txtI*2xXfqT0L*V?g7X^V9^BVK?#mv8nQ_g=`MW%?^_Z85EU+18I9!Qesha zqhtYj^6xYo+Fj1&Y?S~U%brhD*ac$Ya6dXlO1ii--=QzK~(93agfW=D>#&Gm-nr%b1b&DA7EQ{w`BVU zg*3IbzDo^{(KGSA3pmAA$5F%Lsm!&nLM-l{oV1BtdBX3dt4t=#Y<4%t!yh4rQmIRp z#rnVURJ`Gf-OQ_jhemh)`G)BH^nwX#xjPWO=ag%Ar#GG(2e>txKLC#*)pZ7{ues{1@6oVsRYXGljc~nX$c!wsk8LtN7&hGUG)s@ z*n`_Gpsi=$3r;e=ai)n|aBu*t<6z4}5#tR_gUcw#jc2SX~n2bT2x3!^^|b?}3n_SZgrS72>^gAR8pv6TSPz z6w#?W!8Wh})9qS6vRN+?K3vl}Uz#kHk777Id2PyMQFk*2#pE82Cj)p9_NxA9NT__$ zKy}j5=GnCK?u5tmA}tVrkK^2)?+l%_4_RR!q>{PkTrRtVID-vgPAM;Gyf`fSl{J4obtg^9H({Fpxr2^!Lua3f)hCAI}|fHM~J~QD2E(Rp1GD z8n1D=iDC-GHHN*(y!AP3kGI;-=gd`SY#{!^z`7L5niq#@Z+kfR)V`SeZ~$3U?g8c_ zYxvWVN!Tv+Lf9@DdE@FNvF%M>q-!I(TV{cFgu zGrsmTSLwvqKm#oAL_^`ci;}m6IPYdKuZFHv*v}!nB$=OyWjOx&;4f=f@|}~t=iP<* zU#?`ZyNRwboOd&4#?S*H9$)7DT$&`@Z@)d<^>m zSd43_4UTnHA?39bX7(&a9^o_^!<32wE=AkXXsdm82Zp<<+Kks8gyZ#{)yg3Z#V7>e%~$SC?LEKoYPL4acdy?4 zX)zXH)F>s*>qrRoNkM1xgBCt%9=;AL;jMh(-+9jFA5h&8L+R1?XA%{aWZ1rceyxTR znhX<_9f9+gyY7q=RCxQji;-g_n5$O1%FIkT0O#$}y;A{GR_=?(c6OZ||B0N~9P8f; zcJ8-jFQKzvf1ZpM7MZSFAiKdK-8cL`7;}T4C3k_T4O^Z9W@{d%R7XS0?4^%n1tis- zE%kY^p;Dynd6_Xm4#*DrC#v2~ta5i@QL`mB&6c)->k$M~QH7tZG!vO>W&rnTzNY~I z7zQ=8mG_ve6O5b_*?~DBRt?Gn1!pQ-`|Ulugx1#{VpnHjS$D}^-Kk9nNveW^q!Ac%m}Oe1s4Oh63$1&R=oB&S6#$n4Z3WxF9V|D+Uf+cSlWzhF#}dkN1#2H-QN02ebZz5Q?caf^d%MdHOYcbZ zdgDrV1L4-IK8&5cQ;UVIK~;TM6@@3@AjUtKD=JaNZh4U@i2zhyva)v*N~F@RMm_(M zVWwr zE#0~1?R*6hr*gK2^a%mX*rO$K0MN+#a9Z0O^A36VhCdNkcJ5B|HBL1;E2Y*2^6~n& zJ#MnTtNcUY%qw{Vj{@S4XIr-soZ}B?-bajoIP;?0GvvC=)Xb1^-sK5v{JaAM2vO`+ zz4cz&2ddHK?ZozYhkfEWy{_2Xn1N3bVHXAXI{`C zex4MGjhM1-R{bYuUPA90g)$R!VX(2ND=(%SBrcou3gtbeSwhN1-DqSoO80Y@3x{Di zI+{YGeFs2CY=h*$a8Dq~|Bt~l7ko6kM=3_I$j#@?dx*_&j6vLa z;Xl#Nlq%&GUV_Px5>7s+B$H;Qgp3VUh~1@cLl%W6yq0mzXl^b&hOQ`upZh{oT6^^p z!q`6Sl%WE0Y`;Ic3A+hR+O z0U7F*96c?ymq%Kp56q?ZQ(kLiDl1^4Z2*LJgtCY%D> z8+i~nk#L~1yACc&V;ED>C=YUa@aaM`MI^ApY0uz+8vrn!r>s&pEFSyxQM2GXZWKpv zV$yrqJ*%0*-opjaS(%XI~#z z!K?~lIn^=xVTMq8>^3y*{oU+WQ?ijMjtLs7*-)P0zpe#s=MQogWOC|#+wcmlj}Me# zWFWU{_+!5%CoP<)plMNsBKxv-vf0|_a%nMqrHl+~IMX&sb8+52;5s2!vvN6whmT=q zW(bJZDJ8ZbH^##_&;h;drL+%AY;J@XoT6H+yhT52q?i%j?b2yBE7`o+RXRWZAItQI zTQ5g-v*0oqOT!IV`TBz&$*V2_a#ehgB;SOt(XZFjGvj7^e#ZS}*K5Nf+FUUwFtd6N zY;6SA_y5DDH`}#e!#*ctZ*TeRjh;qdMeK za%s{$mR7fS4++y+XwRrB@0C4Nsdv~}2D0w?_P>1YY4Q0K+93YY)$?u_8awE4%Sw+n z80EZKp`LzT=xg%Q9nV5NJX9mtGM^W+Jm$&2e2t!ITo2r z-`NzEw#fU#rzt9quHLIblSCFP)L6m6SwqaPPMj>cDG8g%?$v%)FKKKJ^>TE`Tvi-A>x$&x zU3u-qqcSJ~o@@00!VJ zYV8SXvwt zVA@ZS>d@)g;lFwJ&Mff4{DyfS$75frSFvu>WYaSVC)T~5XwTg;S~pda`cOQpMy*NCxI-V?WW&d{-m?f9f@$lwt@9A?I?A2l*&neAKCxz z-fVACbRgND99=Qgd0Eqb`%kgpAJLR1lOI>o&(Z@PR9oW1As%I}RN=kaUiE18QD>iAApe4u{rZ1=0p2(l9qOcFs zp3}0D1SKtL#TC@@e5?Eg43WIsAo86845EEh*O<4g=k1~v+BEGh?F!AXvE&g}_-+x) zEa_)i=7YuR6j^@hlr`~zoY>*&wU@zSu15z~7J@{I`6D7aZ)t6`ZEK9n6EHTy(i*i` zy;jalYi&_HwSk*2IqIdURAtd5wPC|iXX6@L%GkE^ifOgR%C069INjOAF_pw4QmeBJ zFzfsCk)urTl<}!O0SC8V#pciZjQND8FPAixCq>3s)o5%0crB=zjy&OaxLO`Bk73M% z?*~HqibSHW_2{gUtbKjh%-|#!Tc-83u$ijvf6U3*1UNryN=IT0`-0_sRPYyDt#iNr zW#W4vGSb+9f9&MTMN)~>nLgt|!z}HaIP5rxD!V z!XNiFHm+?8SlGKLqH;Dj;}+tU?C2>Gk6OOy_c{zKoik^XOn9UfgWt_jUd!NJ!eJ6~ zeT{{~fI;frR`!q~->VA*+?ZNNHjN*t2##4YSM|gr3^ZJb-w*mx_`H2nz&^(mI7ESG zQmF@JS@Z}Rs6J1ejk9y67Y^+cj4#gZge+FmSx7AvJBKVzvp`<-RI)Exud>ZVc{rlB zzA${T{>Q>txf7kKmHpvnqg+E3Xh-zR9oV8bV{5^8koR~c?&=))_SXNCZ?8^?MhUHpGvt=9)5!)11jLA@DH+Dq1t;?x&(%8y>ykm|) zk!_C8tBOs~YRN0VuHh;V7nhz*1D^abq~%rCr1C#M-cV~wQ_M9&+g(z8`yI#v zFg{rMV>XdrL)?G=r(F`tLs7BMg0g{m^UtX!41?)!o`hXek|?g>b`H~DNYzSX$M(un zVVGE&w4@l5R=A3FBf*Ww}5VGla(=;L)Vo3L586I zn0Ye22xz_3Y)n^{N`U$z-(q!6S&Xpb7l~4a(F$!V5AgiV?(*iiLW^=%Dyu}we2U6a zKaaB!{~$dTV(pCOgf0hx5hUF-1RqrVL3Q~qe);!BZ_@4Z(O(Wn=POK3_dr{C;i=!XPlZ`;#0PXaNXS> zWWmR@)@TNq!7_}(mJfih1a-~x;%$d}AD`X|6TU_05OOjPLG{Jx7xZ>^82r45(R!j9 z^bwztrFK#h8Xt>)Jfu)ssZ12JDMUZzg!;Cx3DybM+ILD0Bg;>5xR#V~8k~$rE!ag# zUDu$QNB1V_vd^pkfEnR~dL|4X3rJ8-&wBn*o7s7uxBOu@YZov_B``FO0si~Na9&h* zTq&Ka1c%xJvGezUIHvi}u0E{_fuPoL@SD=m)HHup5kd^fQq;DtFcsepiq;KzyGTmY zDefJ2q^LY$(RwF!0?y8r(6`@ea%b(>p4ElJYj^BHKdb@dh-?Q<2@Gk+xXf%a_>BJR zCZpX!!*|O}_uTU;eRH{g0gRGqreV?Vk6>j9jY9AVa6q@w-a0!Z90v1_6M-NIsjWrp zl18jA%I*&!Gbf~<0!Xho%qO4l!SP-wtn$lV8UcqCgz5*7P{atDi7~&I$o0r4jwfe0 zW~<2&)cLo$XmSK&X%rRQjl?#C$(s4*E3sH4UbVM|t~_y@Ceyn_kOFvzwZMVjz*F4O zkl#Cn(SEtqAbU}ZFyYYF=`c!DV)!m)8r3JvW2X-39CE8^*@YO|&Jg;?YpK9t45~vK!;T3RF6kJ)0kM z2CKBpowba7!}q9#|Hp4wwhgYA;7n9k0w43!bUBpT0$eUHWAzqd zzG~@NkAA1?DWs7;nt!K<&&V9y1iApXV<~x&yjtX6^9;|TWz5!V8nNUgCKW8So^T|) z@rY`1*gq2?2g5{tM0rz&)@dJhpr8tk(NKOd#2{GR)KHYa1oLnWug zl0RlP4(>FJ?)n38vH*h;p*KUR7Mt-2e5xEl}G;)$jT^in5qX8r(jx*PpTx zck)Q@=HMX+#@TQkhMZ9nndb6ZIX_oGp+%ZC?PUu6CyrS`qU7U6VZmhJ4&H^kv$qZ2 z1ryV&?CqgT@V_y9RAItzCQCv`llH%d4rne=nug0qN)jkdmOm?kh%p}ePq>n(4S49@ zCk_6U_*hBUEF3{EFg*y)lAlSEvkAQM#7pdv(|El2`2X8WI@d3Qy$D1IP7>n$l`vDL7UXg$4LhG80adZ?4CaSw zUBS#+F8GgYDJFFL1H=>LP_7+Egm|&Eh~sE9Ma9_pxz9fuy^z)`g3dsukHEi4bdbFp zaHd*U4cSloRLB=AF*sboio zs?Po&MOk2ib-gD8yO0~PMBPVOa|N+RrY3v{^C^4CtxQ^eV4<8pAf3_(L7vSRdU0#c z+N8LyFz4Lcy*dH?sW1g5-Jac9KjPK5eXMfFUN>a9O+*>z*r)o+kQ5mdh2eQWo{-sV%A6O9^$ z6zmudW!~p^GZ$<>bo#(}Mn8QBpzL_!V{{vmz{h=0gm5pu{e0=Oo;}v90MmM1yoP9O z$rIF%Yej9z>^Zz0Z_@)S#uG ziD;R4M9#_Oet9b&0TZ(NHH~7%4I{92<^Jwf_SCSlL(>N{ba!8XI$J8mTH(ta&U}J{ zxv$VCIpgM;-KqBPyORz<0uDJw(oGiJ)bUxs~CW@==L74)thL!%#}Zk3UeLz4PS5eaD7-Sm&Ia_R=y+lRk^E+ z971^fOd|O9KO};vMo2Rgb8;WFKNfue9a{l{tVx8r3;MCB?-iZs3J*DpRF382_@-Vv z)n)Qi1h&BJDwia$0XdLSwH-VCMsqZ;{5Vuk3eaf&{bsV)KZk9=m0Ore6OrQhfWAawXN>a z@AGD%4e|7pHi*r8(RvtwW5)OBv0_%u)G1$A2evN6Boeg7)I?0T)(%Ho3ly0wW)qFU zXiq&pon8O^Go&swB}wVTf2a#PX)RVYBKZl|7$!My_CopVY421gBL`Etigl3j%5PY9 z>bCC`2z81-NLxx_>)i@xg9d*&fxk)qzcPH1|z6>|HI;43>k-v|@YGVV=B13eUAlV~PTvsb$8zL=xjroPn5r zakj&!k z=+7NT#uIbA_Hvp|fXDA^P%Ft$-xl2+{O;(jcC8xKyl3$gmFM6s@b#I;S8o!TM39 z==WDpC91Mpw>C+PhhH-|fF+=NXyZSVZ%A_T2jA##JV3$}Bl%38R7=wIj&C2H-ZCk8 zu^?~#KpEblBVl(~S-^nJ%R(AN8^W1)!7#m8QPX^e@Wzo`76ksvUI=lBxdv(tT zxGs{fls5B~{=L8c*Px2z)RhEk^O`E}=0riynKO^Z?4GB0Z$JMk`$ZrKAI zOyu+U8y$H@(g?H(@;JNR#FWO@hk0CS>gdE}CM-6Vy7){!-7+OAL2Sl!~`tpk~KipL?6J;XPyCK~a^1=!Y#Y&1+4YnMdH^=OAW0Gc!7} ztSnbqGs8}%n4O_(rt%LV-7fBbF@76UE(fBQufX&0B^$v+Nz_N7CC@0MgBDhVh(vgf zNpxfCh(nG@?arBTX!A=Oxk_s?cfI(WMrj%sdHVy)Tur(kyqB4!V63vcZ;4>7COMO~ zMmC+oX;dj3=HaY2L*YxY2m3O1>FcuBz=wu4YrcSD97zV@#2s3hmSMs?^%d-Y$&7`u z@`s1|P@3q6%{JfJd%e{AdE2@#OO5a`#&XX$nV0xLmEzRMEzfDkMvrkLG$+=_|Tu0 z+DH38?7d}JmF>1R?3_+?0-}I`Qc9zA3n(xt1xZOM3F#70P&%XpM5G%;Qc_Y-DFtZ| zkPbyUL`tN;aqF|ze)sq8eeCbYcO37Z_s?29&O5IA$}!Gyj&t~8k$i_^8ZPo^MffO1>d*gmEyi5wyWX=>}1b>m* zHs1Z}@)9XPV)E6v#3=Y*TLV*sG4j2q%uYDV_{L(UfloU9BIrUSQV=^CVoHeYj$hVA?XvYkIw-_*U8j#qM6ceE#g6{tP6+1oDAp*k1gRQ<8@eHrXEl4X6eY09nt^fs?$;IP`8BlKtj;B!hZh-Z-<< z?S1$XKR8D*A;TG*%qZ~s+(cyF1D-RdUXP0KiGc4_-3 z5By66NCD>6zFkSvE~q!ZOiKfG*d3S$rXa?vmX1SJ|EO7-1{imcIlyuQ{7)n={gS0N z5cj+JF{!bmL}rcO<2+5;CX z4^$sbIw3S)|L~N?1t|ClWoL?yE&skVv;V{Y@$Vo0t5>@G41bfeX|(7tg9qQgitcJ7 zWxgo~VuHzfF}1}7uk9u@{w#cIJvZgT)~7YibR7CpbMqW+}i02W@t zv)vHvE8B^v*9dP~>_9>rkPbP!*gC)K0ZiZ{x}R|$dK=~ZqEtF09CGbgzmBO6 zAHbOVJ-FNz`Fy$SCd_0u`o|EzXUNi`z4@8)+6@SBlR-nS#2MGVio~ zeBn>qvJ1ilhR=glDiD^X_l4~-@ zA;~B)Gi7ID&USgv%p%eSsY+V*q;?hJUy&|IE$v7bBp<4`tE!Yy{w-1}!fLMlQ&MJ% zO2LWpjjZz`dn+~St%t5NeN7gtURGnn)pfKgm7N2g`&aq!EfIN;M^Zi{mbFLJdYb2b zXVXsG68{8|?F9>&(121}QHDz5nAyEzc{Z(r+|k^Zk0Gsd&s$-BkAV@%qeH6mA+nt- z7wfh@@-;v_!`d`XYu5`=TG9;uE#*(q=8u0?pqFrUZB%-H_B0-eQLz37KF`_{mco4f z>e2$;ikllzJGGB2nW&q$p`wEgZBez$itd43))N97P+C+>5!EcY)d+-A7V!iYqzp8Y z1F&r>=G!y1M|6Ayw|@T) z{~C7lhK$zF>&m~w{Z{3>{PZh_&bR19xE-G~NXENN^V6VB=cgxjawWNmbrap}d-pcV zsMjsN5~&O26~vdewhL5j<|iH>LVUjl3uT^i{9zfsiUZ2OgG-kFPXiFtNMRRV!PH78 zke|ustbI8Fa_X}$Yp3>19K_)iULvSHC)y5Ttv*gx6&a3gJU$yC>GU>Yv1nW~z3qjjZ9VNZ*}{drnfAE7y38rU zF%ZPeZ@=(*pM-f#2?JTC@<(wA05aHk&wpw~8RmAJyX)b*B|E&Zek_dIF!1#10(*&s?}dZjU9YB@EASh+f%Mq73f#1ZA{=*g(Bg7%d>M^a&$uQ-EiJB_g>5ja<-+OA(Fih^l(a|cHYUBH?&*RtcFy8nI+}i=emZ`UCoL7r){IDNJBtQ$m z!7H;nak*!o-;die_??umzf%~km zpQ5|U_SSt5qcSwvbR$nS_Z_btvua?9R7`9>8Rwu+R`mVIU#GJfFl{41&NK)aw^`l@ zj@7#W(q6MLi&wb6In(N zv2&mwcXPH_Ga=Wd+L@T$B<=lKkS2bj8+5T{3-$U}{qZv`NIpREnV`@krb%jQ=s36T*9P@DVSr9+pJLy+O>kb{duXG6G=#{ng zC#I<^COXdF+pkg#ZoBq-ag$CmfMi5CedkjB9Au4ka{Ii(o#j}8kMiT}=P+~~$i)sT<`_-SK#1b~Ti(TIGsE3KkGMJ9Qy zv5D0shK>?;Rsx8XY9i?o5@#UuwFLABUDVuwGrc92N##;cpq>8BbefmfRh(|KXD{}pu9tq-Gn)ynj8Yo-5^ zoMH%;&lckVW~Q~hKntHopsBRe?WlL;3b^OxWYh$7bLafY2x0?vU0x}c;Xd$5Z$34V zA!+srehuD(Wi$n8HG?%#DS=*`^gzNptJ~rsW|_a&pUm@=t%zc%Fy}xFa^!1m3EtI= zridPjUONE(s4mZvS+;yJuk?Co&+s)2*N58kqPsy!`rTw<22_<9m2W+{#Luzde@ePh z@58Ai61p=6Ak|q;EhjMz5gdO+@3uBg$S#o&8$-C>@r@=*|NMsCbf(Mv9!kd|*Ly)%{OZ1y@sx)O-N}`Ljhz8x@*jo)V%SpjTkXMW&CLwrX5zlo_=nD(Y4m^ zk!XIJk`l>2ghf+JJu(2sPb%t88fF=ZLd?BmTF=e6+;hs}%JaaMbwK-*SU~AV8~ytZ z$k;qM2zX_ee@6dg$4yxD{(lzzrXE92gmy=4VJ3;e7jS5l?mT0T&!&4e1Zy?(xoTHd zN`x9HQ0B9wU$I*YnMnyw_M4ZK&&PdOmy%;6V+?T+(o71#s4@DiHoUz4w0AY5a(|Suf_Fn0 zcK@kA;OY-C3d;yOVwu~KL0KL`tn&@UU~-vAW^E{WVHe1pOneQb^puON z5|^G{k~+;>BAq!3Q3k=bQ=jjG66>j3hQN?IC*zl;0h#-`+l0J96DI@V3;+Kip!`ZTw)(39u`P7x@Foo z^o~a1n$rv1#`=?~3i^k`k0XEYpyJ0i<84lRT!{zZ}8Z%!C zj;;uri{2$4Bbc4hYnMXQ>X!QTJ~EyI2Hac&MmG-l`_g9|&M&x`!)>Ywp>+(C4sZK2u5jG;gANsv0OB&iZE7pq{oPPtgS-ernF zwzq}GPf?5r!irWMhCtz;N+G|IWA4{^8M*XZ$-_dQX;cc}9ibJJ-yg}-iNfs}9}=|j zfp4NaUoZp_Gl>8GFTiuBk?(a!il_451xb;}CqoC{>T^sZgcvBq`BO;{GU4-?_6Z2# z!%d_}xm~;aX_DxblqYu)5pWZiFYh3wbW@q%tZlmPJ5sVm@qHc*kC6|U)NLO9gJ7pC zKs@^RqTA;Di|V@M7(k_-Lm>*`{7eXWq0-=gIN=l_%9TFNyE(@J;zWIGSC@FW!n$>U zGMdJ;J%ohWJ_Mb4D0Fsux(0*@k1xXh56N)r@;C#Kv&#Zsr z^f;O1=wQ|R_#y^w@sao7eM9YsWgpSypCcv4AAD*M;gMKw!wah0Kp0Sfu=?>uL~m)d z>fV`LcR*v90VP0zPwQR?oQ4s@I0#7XfH!iGwq_0mHT*tSbg4m>AJcZ7c_-D(gV}j` z5%;NRp<;srDcAY%w<$So`≺LqpF|&YWC!TvdjiF9*!pQe~a`FL9k^QH%gOFi-k8 zQZWq*>k{N5VzZ_Id7zw;L{SI^M;H9L8t~o_A@>;Z-3M%fHHEoPYq-= zYZFj#oL2<_ox#>T^`kM?)25I<+3L*-UcSQmMyf-K%y(O0bmx;Bhz@Tks^gN$ZGO6V@?!k^ zw|O&crq~95E()fb=$GAbY;x$tZ}8DB-e#K$!IvaCKI@agz)(Q9shu-XJhzgNYGz^a z)}qEOb@w1PAy3?MvHyOc@6Nq1myvXm>|%%;W~A{qAeqcim>b`vl`X%3VOQj=EudAe z=Pe$T*wrj#|GC0t`S$1Tc_YAG)8^+COOOsl{@rEBIH(m;oG&i_NgoDHn(BgDAgUR- zj5&TPsBPzZtB914E2xEXjcowhqc!2#K(}M?NSihI$NtngjA)pSu%`mOy|j85D0CxLgM#92x z1r|^o#P;#tWlLk^ej;TU0Ne=zm(l=D_E5cT%YS60t={Xv9B%-Xqnyc z|DUMs%5Q0;lNOh>?Z>IH9=7 zFFGW^^^|*P;?Z}a>ZuTRKZ9Nn{LL#Y>|CSkM^R=8TP_4H z74tihp_R|yU$LLeaOVj;hM_x#!fIep&tf^ygcOhNPaI}zN-(Z^_mA2XzrVI|qEcd3 zLNE2Q(<;ACW>b57!V%wsd(66Aq7qH%otYgIV0vqBbmWIjn~a9?~W40wq3;Te^Zp z=-)yHf!T1v(eEJJj&pq+)H8N6O%W+Ok+h}!XoHq6>;w9%jMw1F-iOjI*UJfq6SyEhfjQ>2_0G9xcnkF8cpIBB<8P4dvG^BNch?VQKOq;u{JrYO=IRVNzepY>7oP+FA>txzB=}5r6c8;@O;b(Q6KYvh542~X@^u3p* zFyL6S=*ik1v2hze8h+L71*6=Zd+tu>?~Sy13)dfSb`#P@!-!+eEW9YAY^E0Kc&lqS zCo{&fkIZLA7V7M(JA*@DXhTc%`MZM`p>Jvfmp3Qp#~imN*WsesxsinuU1}m?AW#p!#++Ivt*ObP^ioodqqF(6r|_fot|7*#CxFt~FAjW2EX0eFqYE ze(4pzFBr;Gve9?o#t<+VdA&B)?pd5p)loB!;Y+&Ud3Z5&dy<_$uZ)d-Z=F^3xA(k zw-Zk;qGM#?2dkZMN!&(GaKvoXR}wTQtB=oslld6R_a>c>j~2d;ayJc0Pj6Ivc?Sl? z$(dk!fjIXR8fz)vem@MID52>ukcN3id@1=DTK8K)i9e2+1c%Gdw0$d~jS<+sA74pH zm!8G>vg0IrfyKuswtPPp2k#2|`>xKjIGjx?2?cSk3_LL=-udBNK@P`~RlS?F=vMYv z=iNkxL3KZBYHG9Unb@;pvhgQSSR7p|d~0NE3Mql*VLwT)2kAAbbSt=Bsq7m8hAjcw z!EBLMEqx5S1@;9_|jhLPO!@`P0k%53vGD$54eNHxQ$!65tSW$?Jc591>Ya8VSPiOf3()_p^o9`xS@aF-JbXV_icHM z$ZgRqIGh3%>69-H0`%$3^n@Re`A$QxDBMvF#{O*;%#V+f(DkQZ6|Oz;8K80-BlI1D zi<<%|J`p=49z!SgE#$xXItg3tFw{ImWE^6kes{&xY`1^Zjjv$C#O?07C5YT-IjP6Q zEhd9OVW)|4xOSA)J94L&y1QwQqI5cS|x922Gm%sC%_XaZ2MYd#SJ;h#X z-m1{6d0D=q@rCW;*eMj&6lOyEYy&$M_thJ|ZcQ1PI50$TZw!o5A>TR_;yUK5MGCW( z8D}4ay9rli3A|P${P$@7?fpgtLLc#PqX_!^AI`nPC)ZMt?z?9*a9jNT);7W z1zI-Eujld4p?|=`?_YT+bJ|xJ9^NE>Zt~XI_^S_#I0tG$h_9VgfrF&NeP1%Y3G>^& z&%|^Lo)`TbKA!U1IsOMqN>BR-2Nmw#eW{^&Yp&pY;Mj3r6BsXY>8I{=5nj+43*C;1 z=0iO_bWj^sOKjY3N5WTPk9Bt*^CKR#r5A!r=I#~of7=vi(5i| zOQpX+7jYe~w(78UoAdXHofxYT<;MtU^WPXF&Q*jT!e?kvov-~#?5&}>Su8P%81!@K ziBK}uE)&=)clQQO9U9Qtyk172qDw;yb{Z17rY3d5VS>Ech>9F5^awy#WDh&wp<`fcEsQMd z9z*tgWU}Ql!DTVgcC!R1EE%$~SUwoT`f9)wQ*d@m(8pk)vKvQ!L1_pFSPCSM*i%9g z+AShB0%77u3n$E&{yatwd4o(&&{z1VOjz0qH(4OrpcUDd3Ou425|KRWUn4f$u26+# z{8^D8c9WB))G_bR^K>q|ZF)NJyAxOr*sP4)`mQ(^^1P|zwa#brbu>K|FJ)`49*of)MZ3HVNXAhmt`nh59< zBEDN;L{r<{qNPuZFip=J9__|$E)A-L3r}Go$;e%5@a1ApjHBD) z_v2@-(Sbf{##3%>T;FTuYilyD;2a_Xx#8B9`_shdq4eshi{ko75fBh3?Wk`1vvd42 zyaLojXx*}49v>VB^r=Zhh&onS1!j`RWh91g3O=_JQd}O)WtQ%18(~los-1x6B#i^^ zQVAld;_@ZF=z*?9it{w!`q6e(LINBMvQ@m+eVVsMiiWX!nN6HEPB+<`(uo+bD2V zXOdU>ZI6SPOzDtG=9B!kQKszdOqw8vS5z{EVx}dsAiQFyN$$BQP`cSPMY> zt=g+sFM6*2R>G_GK#%-h`)-zBSV=lQo7Z@O(+#HPXZ|Ke3>`5eYU2!PCq4QT&HXTo ziPNb%zO$sz$k~iP)5yzw8a)zJZ?-9P+YwouD(p{cFDCMde0RGFLINcmcr!|V!-{KTvLB5OlK zC=<(B$uE^c`@!~GN6nS(03F{M&etDFU1-$%+V#NE6Js=Sg{LSqZ}zLJRMaB6)C&Ly zEZI(i6O+8a{xOcq8{RmfYg1_UX&C%hF~(CmhMd(H}=bF-6NRXE6!!;&Q-QYii}}y=IvP)7=ZahQiFop;2R=Y<@9X2;I`_hSE16bZ_BeTN$AI7} zEY)>Iu}j?r^`$21BR1^)aU)PpW>62tM67J_rr`)LRIJdynyrIxu?J7={|N|8)pi1v zeZ|D4y~>Z&wy|%trl0ny2~vp^+wzU2O+Ru^OR8=*DicTrGuc11y7(t{j+n$ z7cIhx#tjPM_^y_#&#(q4`Y> zH_)dgm=L|A7RahIj@$pldqNxzDIWMaFrnu-3L9J!Q0B3UE~}A0+#Re7(fMpf*thq5 zzV(UQ)Jyx%9`i4POaTUqgPa-t^4rz+R@g8IdZ=2-jJ{Zd=zG0EdjzaiS#1}=_`G-v-HRH%@^qhud|O)Sel#2T#vmh9G3=O1GE{t}1YG3%nqGglA&& z{m2pEL8ONDy)!ksP$}I`{K4C7`>Zj`mrp(T-&SaxA-R3KZ{;O;w3&jJA1I0$DSW?@ zk*4p|%1pWKChSY{WqI!#SW+FEC)46%++0E}YB`gwEwngBRvO%LeRN{Zy^EQ{Bff3K z)NyhmQQOC?k3~%T9_!LW`_XcgHgcjz2ZLSl1jm00L-$n(EVHVO8=J(zq41CTrb*jN zgqG%`Y2;~s4|7$s8=nlI`bO^RQY(cH&k2!_2o0N_WoIF`)(-Z`&Pbrh8QeAYTH*Cx zW-odp>le_jU@p_}d0>L5A-8993DydKXyAI}~c~BX)jo=RDr`ab2QHgsdcmm{CJ1cb)AbZI6-CDEc;5BX^ z_&!tw4;V4Y^kC&FzQU5Bi7&B&vEkHfVT)yogWq?Yz7txqh0lk)GV0Tm)%|+s4ApJx zysUzs8^9pJQ(E4DSh3E)?Yl&`P`Rpar7!~IGn*x7@~HKS3u109FvQ8#nJ0;I=QG)R z1Sr(`Iqw&$8nUhD4)y6b>}6vF$~r$6wb)qWC0=}46lP=19DBhl{l1WMZ^0$2N{bPZ zov1au@@(aA0(w`QqS>D#P+rG?y8E%mPFD6ENV&ES7o}F#>>prH{KEN};FTjmVU4ew zO_JVR*C?*TImx4Z8%}$Bu$t~Os>tBOP$OPp!rA(t#B!J5OV^p3&-JW7VVR9M<}Nx7 zrIswUi4sKbm@L}Lk5(R45h<#eFZX7J(s=xAScfB&%oZ7ER>N0DIirUU6uvhmHfQA* zJ`hZ9&U>9CdM6=9FJQMfir1MuRwSa{-frttzq**Svo4^uE3b+@(1`>Fv-QV$gNOvG z1JqT@G>p6ZnWEtA9(b6xed~MAqv+QJ(Ox#nxQ+WJrvpoJtSCgYMNAv)hYO2~&y6cn z*}7%q6gjYK;ri6?01n3GUKbKp9t+#54JCI0YzF>kgNxr$r6T4hlWn zzY_aGe|E%q2iH%+dMu#sBw*7Mp6G8ZJurWp?rEn*Y&JX&3%PQ#U&7|O0euj5J}+jL z-6zg{WE#jl%f2Y+YcrNusK7qR-rSG{Px#fuVM2T#Xbq>%lq_f7qI>XqwW4u0u zM!N;$Y*g4%_r-!BcS$50&YUoeZ~*`HH;!)xqQVh!t%^zlf~{nMGp;2%5Qag;fjt_- ztyk+~Zf)*rel?R^;rg=+b}EX9r7Lzn-iJt8maGEdP>|g)Lh2j9)a z>UE_WQIhp^So7Lkfuc<9TKBZRseBgu@p& zv8FGyzBH%Kw^)od`@#O8L;8$Ze|TQw0@%UxR;dDWi^YQI#E_@65kZRAu31sS?EeEd z5z7g|A;RJjU}XQu%`K8kWiV@gHhy)i{T%?bCJcZ|#n(?!pu^7^5nAk!&fojizv7S7 zBkf2Q={s>3!}yk2op54LNrmT1e$?{i=U!>{H~MStG}yTY49$!-4(F+}>cCDt0|))u z_~_Y7Pjl~vcJHNtTtOtzM<-Ha^YjQRY7++TAW<4wPfLFGR@*v~-!du#8g*`oGmRcQ zhewdDEds>8nXap={f3dF_M9HYdc+4c)^4R)sT@{X?}nFZnr>B@EeffO|5zBr^^G`r z9q=4ws)>D3v+-MaM)G#~fT}Jv&>Boz) zY8O#xs{1eQE-vJ|J@QL^w|Hf|YX{)lj@eM^Pv9iIc0k*H<47it zg;}=_egd<^4IL7ObhB??@h~@$oyRnn5{p(wY#zfJ50sLMk;Z4mB{Qp0+p^B}!^m4r z%;c{+tazL@!J;I(#N}8Ms+h=ImP;nXwOiH9p_q>NiRhHj6x4=I?Ie! zu6TPCR%M$cri#~;WTT^XUc~0LW>+HBM8wi1@AmcyRCwvx_hlS{k5!&yGYi;iZS`P4 zSf04X$NsZon!^2WAkOqKX*=6GozqsI{K82hDpOgdvJ3~~Y<(L)*ci_m&BLH4g9kr* zdxZGEcKRz@bX37|3kJ-d&9t!YaMwjJ{er)0(Et@nsY$XdOKbDJ*1}3&PHJgBvopDw z0-!+(md1oAEK>@+p_f1U>_xQhY2rIl=`vsk-ozvt(|tYvXDdDhe$6r=?GD%$WqNGR zSIw(+U!xQAUa-_SYibiO3U%GQhv-QZR*tmUF9|wte(Y~&$Ks=PCF6Pgj$vq!LrgAX zpQjUru4M5a=(YrQ;bbS`1@Ob;<5Ax)pu+)BU^kp94zfLL--S7($s|grL78uDe?l8twYK=qP$i9rdUh8E-%ftP0@jLWV zROm#Nup59x62Z6hM##>>-uqD*04A4@r;edmVPc3R2>C&u>dMh+<2LTyQul-pD0WQaZl`SmLx^T&T#9_S-4e|5+i>2`!40RM-B~9TLI;?E@Sc+9@*S! z*q-5Eu6DbsQ+I#VBMyNO=~UI=naoY&yA2T7WJaQ&^wNQHkkqWZN_+;|EeTyg(CANs zVGGBRp+}T*Ryd05e%)+(eYX*Q-Nm23oEv1YuK^|Xdl#o_1eG>w4I(o$?KU#jkgswn#GF2>}x9e;6c-pc6xN6s~u`C{;HmT!j5t#qVFU zho+qYop=|YajYM~c$C-@uownr7_rOB$xR4oZ(e`dWir}lAlhN}B>?C0KmVCSiFb88 z01QQ{vR7uqsV^LRtcSoq#?TAZoif(KVaV|vis_z zlmg1ix&bwvS;4ijzfSQ>V95Z^EflsN!~2V>mbbO=m-*we5Q~qa4=FD_h&>SFhMUWw7dF-zdQ98y_PP!A!xfNx^F1aYN}p!|N|yo=IT+!R`o% z=s}^d!Z4PducztLMPhjxm589D`!ge-;2`S+Yx#WYqfQ6Xdndy7h~XDvQc;-yLC7u= zdkp|%((7;XpbugbHTX!dGB=@<)RKim5sS9N$MGTnf)r9xC&7o|LD1#MAlSqku~!7) zfof4VSjG#y~4oc{i&vAB9;_?FYG&w5~`^+F%A)7o73^HG4e>9Q36E_=z};qicB}>!^wZ6D;SKPXb){e`FqfCWpOMHOygSL> zQ4ir|)k#+wf^f<3+^Nja)m(f50RcH@=ZgL>U*zrV?D~JKo%H=bG5r6{f&U*UKhkGd zd|FtxAE50t3KNh_#R6c^O9VLXg`)KT?UFG1C;z7<(WdBf%GHPABVvaDxBaIX8x(b8 z9!6sUG10>X871qPp*y+ELT|%(tHG3I7j$ff^XFgd48nT<`(l9K)sP*}4Bt$YIM)Fl zKz?|_Xcr!93@s-3?oWz5uc9hLiBoM2Kz`}h`?? z=p5lL(l}By*cYV8{FZE`BZJ-S3;eV0(~|Pjh`5O>Ah5{pi^LwHT^z1I#^FJ0^S@5> zJ7YQgkD*7TzB*iX#IPvF)19P!5Rv?F!qo|IP!=w6^i3o(w>c9{DVZJ#^Fk!{ATg0D zuedm_x%qnDt5;6{?S4f?MU`1oqr14cm|0mVWWD|SG4B68I2Znh)8x$=l3*#^-p1$^ zv%}3+`bqhvc+f$jfh+}i!q7)xlwhGnve+`RBCygV6L{%}VbxNYOP#^c(fRo_zx`); z*lAy1r%xY$xMDa{eR7fX%=B#1y;z6}h^#r1(t)w?H}V`aql_spHL_5}V$oFh(6B21 ziAgH3K@Vh_v%GhdrB8QGv7msTgnK;GuxaicG4r%L^PVg}mz@ojo zW_U9PYnhcv+i)9Dz;{R~)=eJYHxpk;{GSkFGs9t7EULBn(-GstSIO~AWy)Zn!sNZs z6hm;1o!TzvL|bYE=9$80Fh~cUdJd^3qDyu?z#Ur{C`{U19#%)_fO>Wnngd%>hT!H1 z@j;*l33l*Rc|f6zQlP!H^R;tCEwKN3LU1$5w)Dw+gE_~Athia{tVxw;V!v*?+W}Fj zbgAg=Y=wf;96%454_7DF7jn{cU3NY7yfsw&JSs8~7~Sb3$&Hr6s)?GO(e;^oFRqmT zGy%e*>|NKDia`puIS31LAwT|e4Tpu5b%3#u*$zjJavIR=E6F}&_--_-#}W= zOof7V&YbJ7IKrzdiiHbh2a$=W%j(%J00mfScj8I! zf*Da*pUKd6TIG#FRjJvv2P&VC6eJ{FC`Kl5`cxKPsy9hPFxFC_c{QKf! z;TX9k)gTgxTU4l8TLF!~3_ZdFA?HPnt19E`!ZrgONW_&M62l@mD~B2r=LZ@gve*^N zt!AvO{qb@f689|hu(qf1e&_M5=8I}o|0_eg8s;seDjx(@h9@RLO?&b0BZj`&`9=M4Aeb=)4K*#!#Loejt_XYpNgJqg*RVrQ)x1_NRlUp zpd0YwGAs@=+`#-fEvuAE?2uMJ#%=M3<~N_#EI;}7j4vhF^7Lql!^--%_x8uM4iuBB z>89gO10|-uU%vF{8B!b}M(;0mWo*`Ok=-Dn*UE!?kT*R=rm+#A6fQX1~0#@l5N{D1(=X2(qt zmG2#PZdeH0(09`*I7z;B&ugy zd|9FFX6qdYt!%7RKw*oK>V%cN3CWYRrh69vU=DjG)!cf1?i%k@?3Tr1zgf}zc_gB@ za-}gYXJgP@2B$)49d>=c^gbUtW`*A`8UlHWM569W8%CA2ZstqxEZVqTXC2V10Lixw z;ceuk1k8$kPfuXo_39BmFxSGmnP;V2Sl8&JLZmdY3TT@n+h1Uv{Ya{qM+!HUuL{10 zt=4NDDRt1@FWbKKp*tt9B6homFy+m49>9`n1qTxz9G)BR z0Q@h*>k8q^9;%sfYo@dq8U#Zp$3po*xu-dEBKb?m*H1G}W3P{QgFu`5an8Fh{gMd8 zN3K|OB`enH2%6#LL)?h7FK+MSxgL$*Z(z$Ry!iCPJ#)dRoQwgpID=WM0?}~p^o2KU zObO(=sUaF80^(&g9V73-U90Rq*F$0O;8uHEfm|6nj6i9z+m^e7P{`a(X^yGsf|>}k&Y;RR3}a3SI+9`$kdQpuZ_6{Ld#t*9(MAigSENz##bT! z0BY9Gq;F__Q?1?=I=r1tkyOE(f9im1?gB|qvH~Tb8R!{+akNe^hT;28BesTDcrY;p zUJ!$*5hHNF-iMr`Lr7*744AbCn(7WS5orxC!$sQ&v}$^9=oKA{Qo-%P^u1Pg(A#dX z^_`Fvk+a*1S$&r7lie5+a>M-Xo#stY&8%`v`hX{ePUNQ9^-J`Q3eRB(Ez<9uCA5TS zfAKw&i=7l2Z>`&HmpT>)tLOiAk&ZH&`$16LE@2T}U%Pl(y3enVXH_26jBpNh_!3Vr zgmO$-+3#mJzL;gH0{()R&Rx8X!N_-WyLOAtsyKJR7FLGLL+`~_*IGON)6hA2hlwhfVhn6sveHjz+Sr?#@l4B1r>*O<7y|A14N2 z6ejRlbT=SaWn`|G!C6oouu$=}mVnA!FF9UR-<99BPw9$|FGCg9vsPWW(^~K|`an0> zP$hm0Og`JSN900^+3h}iJ{`54BrEgeWlCRol3{L^_>_N=Y4g}| z@@NeQb-;`T$Yb^^$DG=SA7fHMZ303Wi^f>dlYlTiNakSmJ;%mkh4Lg1*C^T06GtiG z@rh+N*j7}WDUU#r^PwOWbP%EWBdOr|psA{4(Uzsm9Khg0upJaFI@e@*~R_f3)xb<0m zW5q-oFeDo{>wv=k549JySeotPc(w`a0w9Ds?}?+w6Ox=Cf^-C%~;T2}l@c*1G~ z5><&`ap8ndsTAXtom;udQs%7Ep>^$Zj9oU5ryQ=2T%K(tZzz*JI?-AQ-;yoT6YT1Y zoa-k7I1Sb^^WS@*VD=?a8RBu0S3V&BV?5pBiv$v4vYL}88?qUuK!35&Cwrs+i+`u! z)*8U7?T{qIuzQCmh~0GD39=qS2;R4~NjPtTOHc}7Tx$X<#&%=1JHi#;qK)3)4Hs$t zK|M>RJaLNn5+}+!Jtp?bOUZh2bU*Gsv--z8Ooz~gy(2l!ABvB0mtpbSeG}{{SFKGg z&10ocT=5ZN*0oVoDO!dsx=XUUK87kmn@cKqHkNsgwN#dFINyPJK0Px2!_8@IVX@^A-z-(i<&HyUtL1OO)&STa;aa(XnBUx&v5V!k^ z@%|jdF7_l)odq>(8q1$AmW_1q>m4o|quh2)f0``KLJ+d2ZAx$`S;_kp{xxK#2r zym#*d3&>*mK3+MLE&M`Yr>i<)tF#4j9_A`8`rxq?82tvS#;AJ|X^-Lo2m8Jq&6%J>ip%IkPTikhc+{MbHu;&#|$^HdC;fZ&%8 zY#r9(dh>?gujEfG8%boj0tQsfu z>< zqjr`zY@CnA1Lpd1{FE+v3tq;Yzbq#nm`ucT1t=FFBz3`_WaWzu(sV+ws6&Z3!|&3i zv;Nl9)*f|}19~xUn2!Q5tKkF}dKO#Tt1dF>I!CML@gC?a2p^SfT1q9l|LmCx`${3s z341Nc;V70Wpb>UhMd_ZqWpg`0)A&UcA0bulF-F=^+l%EDz2$-N141@deLL}Xxh5Ya z+|jdu-!o2nY&;e7XxRRe=vXe4Fp!~60rb}`gOMABAF{W)_Roq|TpvQ{O6F!M2Ww9& z0!P3jrV#xtY4c{mD*AfZS+SI|2rb(Q!nHyn-74j|SG)@iq_+ghGL$xNOx`!&bCk>P z*^mF?{_CR-W05Scsg4KWlz8rIj~7WA-8zcAjx7dZf%tX_f%JPF%*n8C0Eq0Y2-9?V zx0|J+64Dnnw5cIZh55UlzisYb8I_hjbLzaHwT70XH*)My;M|2JHOjPkl~wvZzWSb& zIm0aH#9uK$K(GqX!RsJuGE;ptBl)E*;^X9JJ&mg{AFhjI(7Mxdjz}m7p@37BwI$Q_ zApFNdi(n*5_#RHE-lWkt7Uq@7NO;py+cocJT^n2e?Qe?bqT)yHIzf`0ER|nT2y!OK zd{T09Uu!x?NYHKjFa{!L=S=ko7{wen5$a3X(=xITO+qKoy0(^%5VXsL^`e_t+Nw`S zObeepe|riaaI(LEW_dI}UrzLhrtvH>nl}h9IFH&wn>Qf}$9U#LGL4n2h{20dP_1t{ z&@Po8=dxB_j0SP}QM)%>We~LyasPN4csdB~->S+_j7df)=o&Rx3g}y2coFGw3qZ^0 zwBRFOPc?}EvW5iStXi2i{+0XkrDrl7$DnZiFKRNs>(q3|+VGHyCtmFkd02P1THl-iZhxQVp9La zHd8Qnq;U#HYS=u4Y_BPkn81Y5MOg4nEpJ|gL4(?`by7iGaXyti-#S$A{)u*`BS>gK z&W^ zxue5_cJdS7wTOMy2rUD!7UEu*Kbc29xEJtF0_Ff(fM!3#pKPKBiWksIFuFFRMCT9> zRxS@%FIWpN-B<$`9T8CC-;XPF(~wAR9A@)}NHi;ahRAakKNzq!gs?wGY2<-rih`-g zw;`ng*e)M7S_(%E(r;oq8^9L3w3JEG(-xA}K(9&)ZDnrgpo8GZ61-Z!n{G{-?gISd znnM#iOw!ZnEC}PPfp9=1k{XRjywusec!v1m^J2Fuz)o)mz=kH~>OM>*e268+)@Krv z!j0G98LjR3B$9Oa@Z87PPd}}i^Pd%s2 z2X#CX_P=mhl{dr=K;=t(Tg~Y7?{HgqVm-JcOi0Q&)C@s1@-qI0DWid&@YwP{SPEni zWh_)x!+-ssVxO=6`;qD~{HUd^tzck~(myZ|%YLf4kPbaV*5lFf|Frkz;ZVPA-I%y;IyoY#4s%jfesRjDn3c*VIBx;e67qj+UrP8X<%D`C%Sv%(V( zQW0ki_=z3Rl!XVuy_+7tRgnQJ!hR_~B*1$({2e{G091Fllg@Ash(00mIK}5sd&nt^ z5I#$_6>&E>PHf{w>5D9b=Fzjel?XErZ|{`u-6{GRO2YM!vVOcY1HO`wq7OHx*u9GXsWlF@S%xZ`hI4(Avtxjs_NlYvs?$Ro+n`SG;o+MZci{=B66F$qm%1~>3o*qv(t`um;u)94U6f$2SOO4aOr%{mEl-( z(`q6-c$k2*L2?Qdo1s^>)% zu>dPAFdm-3o2yfF@W_1OC2wopZVsF{+{wDo0jN72r-zqYxx=%;QlPB|p9YhnfbXaA zzDkYd4S_=__ayE^Wj;7jzHRsb`2F?h_m4Ahg4Y>BX0pl({pv7;;<>KRxd(RNeei*> zFzNwtRjFWpOq9>p{ukOq?{p)53*NaSS;W?IHM$y?z(GV936@))GZ zXyiH+=>Rd1af$}e}@H2p&UILqq z1NtiEFl}AnpH`OI&O69h{se?Aj}SPAj0HV|_Gc_~U@SkD>lQGUzmfemgRvmgXask} zC?VwucpaUuJG@7~Rp-Dmf*EyjcST^WMn+v62Do1l^22{Z{>fhf(?Xt&d)F2$h0il^ z-6}@P&=JNRM{ox$x1R?19bLG5>qAdkCCY#fV+sNmLNE_}6qIHyH=2u3X#=*Uh(=9D z>)KjP0BWu8|0Uc6SYj%B zRO5Do&_^h=3itg#VlRg$AI%I-yV?E)>*4f1#K^$cB7pFd!)Gadl#vIsWHGOTFHivo zP<)lC1YWj+H{zC-dj^QI-DsEF)k?tOj-$BpS;kdw_+BjfUJW5CN{F)#!Jl=x`1AaL zN~MWH5@BLv2++7%X`3x@chA7H!zWzHr1e37M?)&s{oJ=6BZ46?cp{L#Ccqpb{*Apr z)4YN09SIz3)5J2pG;zF1Y0{sI&wrS-`L;BLt>X zE(lj7oCxvU_pK&j!f(Fkl>93e=r zV$-o3r=f`}DC9bAs&5vllMX-t0a9sL@SM}ol{`T3wSw*`z?M}2LAD7Xw>>cOd0$q7 zgEHng4uj%&w3{4Ybc59J7cLJj2m+bBp$BU~@2vyHS$^Qj9vvPAHmd<>#|t1CSAJ+6 z$F3^7O@kW4OJB-Orcg4Ch8}vGn(;>i-uNqdplVdmEb;L;)bz%A)3~FJgp&337r#wI z2S$M)(l+8R={{G`LyUh3@agfOy9HS7fTJqhDVzzBoaLK=sZ<0QUV|L^0SL%`Y>#On z8pP&=eV`8%3OYTS;N!LG1iB$&pa(;>W_bqtzrLFEoc5HI;N(rxkWKqt&4PCUJBBNh zhMN_bjkVF#JMk}X9Fbydp_XxsHO0Ww6Em8!C!FvK?Q$HvfJ3c9zY7qoxe2R zt%pv*Pnp8SjJbY%%Juh4setbqj!-`Z_2PDR|~^3AD-J_{oQH5`l1Jj%jb>p*Q0L3 z6)bcM3jtpMfxSjzYscs*1au!AzD5X#wGk)JNV))w+;2R^CD=bszv`+f0YFB*FklBD z(=JXWTjk_Ml}kamf{LyPOjBYXt|}S|s>DLp8R2$5aPpE9a|o2Q3iyipkymME*%?t` zezX$DL92-ai60lgnn)utC{H*I<(I%}(1#dk|I#uPdc%+c9MUVOFtF<~z7z?aFM*ul zZpfZT97#!^lX>s5n6VO*qbw@hWEY}zrQ7k;`Q80`!vU{IA`7g%Ssa533Wizm{Yz2 zkhzKzWkLx+XUVrRT7bx{@bOskB;a1AflSylkLt~M;XrR)nfXWw zAuudrlB*m?B0L4w2EN~&BcD_xkmpLSfRX4LwRUJ3Jmu>yngn`ov9Y1eL;qaT*uD7$ z$hwnzTDKK-eIOMmDe&8Z{oiJFZr7YI@6=j?eza}!M|M*2ir4A~FPppreV;yw;r$Rl zx>RQMJ^NY+Sk-T*K`+k)_JsW49m2%mxs04)?Pu{YXKj*q=I)bz$Y;*#owNJ^5&(MT z07?<=TG`7xJ{qNnPh6djuKwB=`vTyKJ^1IF7|FAq_=zF}5)$<^9?h7_;fQ%nEA^S6 zIn`>CedS|bX|J18xyzz~!l6F99973VyWRoB*wnwU-0?%N=Zc(O^Jqzvu}jbMARDm1 zq{j=t}@Z>JKi!IvtiBv5?4Lis?BRkU@2VlWJ!6l0(9-|B89uEXkONco_iKZ z?$^qr130nahNkf`BYdx)>X8<0Z@LoHxt)gk>&A-lpn^Axfs5i}A>*>V|Pq&+>-A!S);Uy0ZA3KsY$IMxinYsomkzt%+PB>*0uH9g+Hd#}BwHR8Hnd3SqyuW)kdi$a{n+@76u}9Wsh1IMt zZHSN3dAZ`ARh_U+!pVZ%xVjX&^yQQn%bn^7suGZ+2Jp`thVg=8!4GCDeG$_+ zOV6ScKK(FWQLs|IF)7+BkNhGjr!;&{vQ(g*RTAlNOwk{BEoVVXe|R|Nd{Ih>dC3Kd zt%vT&KK8&kRwuk`a$Sl~_1eDIV^ZcShM1D9N;HdINf44%p}4es|N6qfTb8MY-^{B% zUnc5At{gml$M^9`>dTwEsPVP%b7*K&no9)uq%PpkoEjW{W0 z;w$8lc{M8%ERak~^G8O;})cn@1!SEZXSHZm)>2*pNlrH3ha?76l$pwHEDeWBZy6}#0iEZ_4HbKHQqbe z63CIX5N3b>{(~npN%W`2V}@1Khj+3yT*;npFdB!P8Y>z^+sPDi=pTTvTC1k1;VnHb)Vo^+8$%) z4jm93GxZ-2UdK={)4$QvX7R39wPQR;!cCi(U_Zo}twQ zrK&IYmu@RMQT4<|Mss>8@^9`LHTRu-;RQPyA1II2(ey}z;3Fp}r_HW>y{#BR)8h;o zKB&zVgS6d_!tvP|wIjzmNy zTpHRQOB1^`_m!#T7aMn8=ICk^_O-85J96JKzPahk`y;%5Z!&Y2KQ7+ocZ?UJ>m6}i z5!+k3(CAmPu8Lw@mse-_a9nmid|sfS&U!Xy*1S3I@?3pltb84KDyharEk8vQeIfQXz0i#k9T?x`P$ji zP4^z=w+orbj(fmNj%HSoGtbx$nj{qHri=#h^x55QqB`Re8JoYGysL+#psg1;s;gw{ z*_8a2M#OjuL7BThlIj~x@*aR1Ur(esiERp~>Jb1k6D{;rDditi-f2scct&;E#XZBW zu>D{)xb?=)=5RF$rP^C;*6OTtdKeMC^-U7ZGuQG{+rm}5fC2BWqjz9LIT?8G-4hwB z0`(>{DSUoT3Qu$O3vHH33<^t5-2E=EZO!!0>QD829Ah%> z8M&{f?K12YsGk#5q~3q2oKM=>{eJ$*&7kGmYi6t7X-!Hz<>f1ooVa*0xhC`cSa(fS zi(0>7`9^l=Pk&&rSy(k;G@=}QJ={BVKK1=*8N8j}+CBOq$IiJ+?!@e*v&^2z+Wq}Y zg&|)Ot4bXvQi;)%EBYh-Jy%>b*Ai{FNUlw0=G(MICXj@T*F4gJZH+rF1o6{u^K0D@ zU_XP&-V2V zvy+_mXDz&^xPfk`O=HQw`Jn}GJ>bG_=>xJc!BLM)Q?bTX>W z#@&6sGIt|dFgON(LI3rvwl~=}j<DD|eiOlIXLAi3-YhLPcJJX=Yv7eW)!C#~=>F$xEoK2-e(5`^`=Xc`fpJJEfDFCpLv!w;eOS&SYbt z(c6wGG<>PuebBiQgk8ouA3~jBUty4{o=L2Pxy#2G8q-YCQ?CGwCd<2-vh>0RJ-97` z#tRvS4Nf{yL5gn5nRYcWXGb5+c$v@Vl{WUXrxoxjb>eZDvOaD#s+o9>WsGcVo6S^j%~!T#XzP7-Lo402aHv-_+HO?Q{tsOlHHz+ zAf)KU?4k9`PrB>nCY%)_f`Z1LGSAF9wZRV9`CFEMzT>7>=B$|`N(q>_+ti#*duvd4 z=)3wC4;Mw@ma9mna2=)z2O12-b(H(#@x)j^Y|-UPCV@`b8RA@wxqS4Ai)Xo7O+N`0 z*9omyr76tK?c|a$TLcWMgf+XA{Nq3+7IvapOz|N;vXBRH$!vHP`iHgi8|moDySg(^ z$4!Xk)C-klmc&j77q{2|hIS%)%Cba*I6h0Du(h&Z-B)_V;E{1W{l-cpD==?K;ZnWe zUukV^y^^0Rhn?l{enTv=BQD1SvY~+KR@cF%pd&4TZYAm;TC>#I)o(~I`ByZ}w>4JQ zH`@czX zXBGFX7Ol}g=ft!T_w!JNGg*io8s$YtS%Oi^cY#+nV#Iw0x)54inPiiTM1**6rpH*vHdbji*)j zY3Ln;*_$uzj`6!}%AzNa-^&J%UtL$$X)By%+6%fIah{!HRt0RA-e{h79l}>Qw{6gIylz|mjne#tOx0&d#_7q_?G zwf7IkGVXj%b+*KWj$I>(6gCd!+6UqqpCs(PDC25##R0n%B$0Scv@g{aWTkpPNc{Hf zI-~74$zSKg#}*Sh^)h&ZxyHR~JkM#0i~<&P(?`dYt5?tDOW)^w%SaY@>qZFdRBgu> zi%>2WrDtxd2Ms`F}>?7clMq)8K)7^is)OU$|vO9zyGC? z`DJ6w@T|JJ9(h4EIi8VE8W!TzwxciIAyt;*8DWky(;#@eL|a;bt6a&WG53RU*Ro*jx-e<67}$qp{;Zwo7huDk2pclK6?wp zafMwhy-wESEz0@*_*(Y^Y#ZrX5CXlkRV~)k;g4HCQw7M$e{?@zmLK~jwnOJC4?M9R3HIO-6+Nygr@;s%19c@u(Qs=u( zI|D@sFK5`feTLOCrcnhRJz6oVpSQyw-E!n zM4(Hoa$dJDi4ha`dV$OB8%n{ctae9Dd`5s^IP7zlm zrYPb}Ns+ftXm&5cG%*yO>^aDpqT{j}D~&aR^6&7eS|{#$P@)?IxR#cD+d{SZDIA}aNC8dboRvqfpD!;HmJ#Zl3DC$KmXD)i0GA}@ zQ$`7;BnR`DDNYSkW*#WJ$o$LoUjT|}oHYyAX()a2V+Rt7t@Eo_=Hro4p}q$wA{8Nx3w;k%utReR^*s=) zWJLXeYP_f_47kra6Frw9<2M1N;HX$(^9>cIO^3Q7fUhEzV?WlO4O$E_9i>o)3b2Yj z5i6uH+>)GZ7VVxo0%cEm>shQJu!PdByYg%HFF>ti5PFeKV|)>PCshQv4fv0APd`K& zI%d!DdYj{v+Q$*k9)3ivbK62U6!4BsLrIC)V+$gLKp&1jVbVTMY>MMH0dP!hwam*G zpP&R53KgRL4fz@_eD@y}@E0};>3?Bbthy1F_`wy*zJs}kp_we$M{;f$Yxq)SJu6W^ zD4t{1Q#`9sJ=fb1l(;GwEl|ns{_Bpf^I^apShGwQmw%j)G${S`tzXgt!`R&g=7L<^C#b8@^=#QGb=cHgZEkaqrRPe*;%&f6$=2DyW2WlySe|+T`KE8c z_;7P#yqP)ne#lLFnnIh|=jlq}vXEz#8rRuD0hqG_Br*9-FM-JEwQ-=Xy$R`qRl!zJ z0X0847vLjn)HQF7q3y8+S-S6{*=W=vq>T1aec(RjL>1pO6ue?aCGa}CoV?Ph(+h(w zd%SkCEERwGoMR&9YCO5(-0Vv}bG8L9_Y{C0z3E)++SMR>%Ywz=ySqUl@w{$w?_Jvm zKm|Qr01(*?6@#yivEz*oG1({*sPjUE2MjHSUc(@Z-Io+zL(QJiAe(qKDpyBaacdap z@bVr`=Z9?q!P<#)D?k%HcQq$hhO6jh=&_x#9p=w?Tw!M`TonE7^7+yJYA@Y>XbnwJ zzo=2;;#GobzpwE*th_70@(UMd2bH}j?|8jK<1O;iB_WMB%Pa1jRvmv2;99&vr86)u zuj!hN?lV&yd`Z$%e`y2bEi+*}rmK)m_k%`jk5 zE}n4cG+B<0vcfd&rYKF~n=eq9c?n|^qs>a8D5wl0EeaP;gX$EEn4OVsKhj;5cAKpm za|f%D%c-FiAH?Yc4F}2x{ZF8RESjUP&eSNa(?QiR4dM^xJzH3W*3IzuIs{4B*sEm|^CJKcG2xS^^5N z&rvw1E8 zv?yqGB@0JU$5ed<8S-zPT9I#9iHe`?RSso}rfa_66~#vp91~QA z4};1ub79n{7&s=;gt*bC2wH)T{As_Ap@n1oejEeaOH@??Ns0?A5?vVG+zrRX>Jce@ zieN)?m%(mNp<2~($D^YQ9351aN_-3DzR2HkI z-xbY&rc&RtisPUhU1U1w!vne1RYsl+@8Pt%z3HRW{@?*;RFM?R&-~+i{>6auZwk`@ z_}`no8gQvu!<#Gtgjn#Jb=ecJc;CPEXAqw9r!+M}i!k%5Dfx&An8JtZ!9S$4-_J=I zA@1Z;(zXwyjTVYx>X5KEGLPO(8Q6%S#2xqb)+NNmz*a}ibpi0RujR0}pxtv-LGf_p z@wdQJW|R)OXWw7dT4>&)4BnVg@$s6SW2e6PMS{T-2f4%0)3$p!n-3Bj1k%4L=)`91hzR8%V|@YwHe~1LN0CSpu!L-%1ldtV-1?)! zHJY9GO+o<9PXXN|jDfiQQ3BsykJ^01agR*!7^d|ZK)HI$Ey*-MIWb9VNtGa${Ku~s z#lVvpz^5{X`o>-%EePC398VCur`cUbjlm;{-hqD1bqtu(T_;>HV6o4*zDG0u+928#@hZHF`bL@{s-S3y7!W^LL^c_Q}cgql%#R5og35bf<uPgP z!Aaob-S*J>=+cvWWi~PKhd>u5U&phI1*3`k3Oqmsx{?tXnSvxbG|QerWTEgDx}c#D z>*yiAIxW2gUdawBamq;TU~2>mlDkQQ_+@6F>95%1N4OFMxCmVl2|KVd-k`_Jlmks) zf|4C}go?8D;=PVsg1GrtI)`2odY}~;xkY+CK!pDE#&;tvBS1jo&rkn*%KXy||8TI! z#r6GgszNYr1ppRlWa34t@=cgfMsjLnw6RMpbh%7GF*Levh zrtn~(L$Xym@A8j94XIZyFrw1PcR}3Iz}lmP8&tdPIB=Y>w5iSx&|vl%qE|pV*^5j>D z&hLJ3mi5odxp!KU%>LclKBbE>JD= zX3Ae3)_24DKXEYE_74!uMKhnid=w=R2yiBU`XUJH76m*;eNPZf4}(^f0rcx0ca{x6 zXwPAWMcsDPWgdd)^vyf?Bh?82FK5T^w^w2iG))+6Q4iGNw z_LVnSA0FzF>(@ZdKQ&bWn%NQgQ=1j7Jn^0fWdaDW@M@$Cp|sb7DGL$on?eBneCI_aF7O(>CuKB6<0AsLr6R}FCcF_) z{)0jS!bb@zDtlr#VtM_*>6~|61wzFjs*B9=^_JLvL?AXFCk8T!4bp)8T;MkqfWStk zV^|{%{7leeXrDcsk?H~NX%Hzno&(42qhCmTi4e93M}mL>nxtB0Y|yUHV2brHW8tfv z)>dgdKuG@q_V`a0w+~rd#*H)o!s33%O#a6Q8;bu1OGEJ6=mO}5BKe@A)*YqbEWvSu z;({5}VIv`{kee0Rhe8C33LxsHDe9_bIn`>U6xbRC;MzV<%E$G;(DZcYsZ_Dy|oQR@U)(>ctI*QscqMZ*@>`3RxS zc>QeW=AYA0E+eP*6n`Ig99fl!=|O2nFd_B*Z9r#{XACOxgLpA;`P->R%>6#N762nW^Q=T;%|#4S z2|?ryl<^6>!FwGA21^AmQn=m{KHB8%`|;mjfA@aO`IaC_mPeoY{g3D< z2@Gvfb7vgGU;j?U3U91ez54T;e;xHdKk@5Ue1GvjH|ozf Date: Tue, 31 May 2022 02:44:26 +0800 Subject: [PATCH 010/104] Fix broken unit test for `--gtest_filter='*StorageDeltaMergeTest*:*RegionKVStoreTest*'` (#4903) close pingcap/tiflash#4904 --- .../tests/gtest_dm_storage_delta_merge.cpp | 20 ++++-- .../Transaction/tests/gtest_kvstore.cpp | 23 ++++++- tests/docker/config/tics_dt.toml | 13 +--- .../docker/config/tiflash_dt_async_grpc.toml | 69 ------------------- .../tiflash_dt_disable_local_tunnel.toml | 69 ------------------- 5 files changed, 38 insertions(+), 156 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp index a26471cfe01..1cb735a2b65 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include #include #include @@ -168,6 +169,8 @@ try } EXPECT_EQ(total_segment_rows, num_rows_read); storage->drop(); + // remove the storage from TiFlash context manually + storage->removeFromTMTContext(); } CATCH @@ -252,6 +255,8 @@ try ASSERT_EQ(storage->getDatabaseName(), new_db_name); storage->drop(); + // remove the storage from TiFlash context manually + storage->removeFromTMTContext(); } CATCH @@ -315,6 +320,8 @@ try ASSERT_EQ(sort_desc.front().nulls_direction, sort_desc2.front().nulls_direction); storage->drop(); + // remove the storage from TiFlash context manually + storage->removeFromTMTContext(); } CATCH @@ -609,6 +616,8 @@ try sample.insert(DB::tests::createColumn( Strings(100, "a"), "col2")); + constexpr TiDB::TableID table_id = 1; + const String table_name = fmt::format("t_{}", table_id); Context ctx = DMTestEnv::getContext(); std::shared_ptr storage; @@ -631,12 +640,11 @@ try path.remove(true); // primary_expr_ast - const String table_name = "t_1233"; ASTPtr astptr(new ASTIdentifier(table_name, ASTIdentifier::Kind::Table)); astptr->children.emplace_back(new ASTIdentifier("col1")); TiDB::TableInfo tidb_table_info; - tidb_table_info.id = 1; + tidb_table_info.id = table_id; storage = StorageDeltaMerge::create("TiFlash", /* db_name= */ "default", @@ -692,8 +700,8 @@ try { Field res; c->get(i, res); - ASSERT(!res.isNull()); - ASSERT(res.get() == 1); + ASSERT_TRUE(!res.isNull()); + ASSERT_EQ(res.get(), table_id); } } } @@ -701,6 +709,8 @@ try in->readSuffix(); ASSERT_EQ(num_rows_read, sample.rows()); storage->drop(); + // remove the storage from TiFlash context manually + storage->removeFromTMTContext(); } CATCH @@ -848,6 +858,8 @@ try ASSERT_LT(read_data(), num_rows_write); } storage->drop(); + // remove the storage from TiFlash context manually + storage->removeFromTMTContext(); } CATCH diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp index e93a117cc1c..2378871f71f 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp @@ -47,8 +47,7 @@ RegionPtr makeRegion(UInt64 id, const std::string start_key, const std::string e class RegionKVStoreTest : public ::testing::Test { public: - RegionKVStoreTest() - = default; + RegionKVStoreTest() = default; static void SetUpTestCase() {} static void testBasic(); @@ -1372,12 +1371,30 @@ void RegionKVStoreTest::testBasic() } } -TEST_F(RegionKVStoreTest, run) +TEST_F(RegionKVStoreTest, Basic) try { testBasic(); +} +CATCH + +TEST_F(RegionKVStoreTest, KVStore) +try +{ testKVStore(); +} +CATCH + +TEST_F(RegionKVStoreTest, Region) +try +{ testRegion(); +} +CATCH + +TEST_F(RegionKVStoreTest, ReadIndex) +try +{ testReadIndex(); } CATCH diff --git a/tests/docker/config/tics_dt.toml b/tests/docker/config/tics_dt.toml index 89147f80c7d..56bef659cb7 100644 --- a/tests/docker/config/tics_dt.toml +++ b/tests/docker/config/tics_dt.toml @@ -13,25 +13,16 @@ # limitations under the License. tmp_path = "/tmp/tiflash/data/tmp" -display_name = "TiFlash" -# specify paths used for store data, multiple path should be seperated by comma path = "/tmp/tiflash/data/db" -capacity = "107374182400" -# multi-paths example -# path = "/tmp/tiflash/data/db,/tmp/tiflash1,/tmp/tiflash2" -# capacity = "0,0,0" +capacity = "107374182400" # 100GB mark_cache_size = 5368709120 minmax_index_cache_size = 5368709120 tcp_port = 9000 http_port = 8123 + [logger] count = 10 errorlog = "/tmp/tiflash/log/error.log" size = "1000M" log = "/tmp/tiflash/log/server.log" level = "trace" -[application] -runAsDaemon = true -[raft] -# specify which storage engine we use. tmt or dt -storage_engine = "dt" diff --git a/tests/docker/config/tiflash_dt_async_grpc.toml b/tests/docker/config/tiflash_dt_async_grpc.toml index 3c67c37db33..bf31c61cfa8 100644 --- a/tests/docker/config/tiflash_dt_async_grpc.toml +++ b/tests/docker/config/tiflash_dt_async_grpc.toml @@ -13,71 +13,15 @@ # limitations under the License. tmp_path = "/tmp/tiflash/data/tmp" -display_name = "TiFlash" -## Deprecated storage path setting style. Check [storage] section for new style. path = "/tmp/tiflash/data/db" capacity = "10737418240" -## Deprecated storage path setting style of multi-disks. Check [storage] section for new style. -# path = "/tmp/tiflash/data/db,/tmp/tiflash1,/tmp/tiflash2" -# capacity = "0" mark_cache_size = 5368709120 minmax_index_cache_size = 5368709120 tcp_port = 9000 http_port = 8123 -## Storage paths settings. -# [storage] -## The storage format version in storage engine. Valid values: 1, 2 (experimental). -## format_version = 1 - -## If there are multiple SSD disks on the machine, -## specify the path list on `storage.main.dir` can improve TiFlash performance. - -## If there are multiple disks with different IO metrics (e.g. one SSD and some HDDs) -## on the machine, -## set `storage.latest.dir` to store the latest data on SSD (disks with higher IOPS metrics) -## set `storage.main.dir` to store the main data on HDD (disks with lower IOPS metrics) -## can improve TiFlash performance. - -# [storage.main] -## The path to store main data. -# e.g. -# dir = [ "/data0/tiflash" ] -# or -# dir = [ "/data0/tiflash", "/data1/tiflash" ] - -## Store capacity of each path, i.e. max data size allowed. -## If it is not set, or is set to 0s, the actual disk capacity is used. -## Note that we don't support human-readable big numbers(like "10GB") yet. -## Please set in the specified number of bytes. -# e.g. -# capacity = [ 10737418240, 10737418240 ] - -# [storage.latest] -## The path(s) to store latest data. -## If not set, it will be the same with `storage.main.dir`. -# dir = [ ] - -## Store capacity of each path, i.e. max data size allowed. -## If it is not set, or is set to 0s, the actual disk capacity is used. -# e.g. -# capacity = [ 10737418240, 10737418240 ] - -# [storage.raft] -## The path(s) to store Raft data. -## If not set, it will be the paths in `storage.latest.dir` appended with "/kvstore". -# dir = [ ] - -# [storage.io_rate_limit] -## The max I/O bandwith. Default value is 0 and I/O rate limit is disabled. -# max_bytes_per_sec = 268435456 -## max_read_bytes_per_sec and max_write_bytes_per_sec are the same meaning as max_bytes_per_sec, -## but for disk that read bandwidth and write bandwith are calculated separatly, such as GCP's persistent disks. -# max_read_bytes_per_sec = 0 -# max_write_bytes_per_sec = 0 - [flash] tidb_status_addr = "tidb0:10080" service_addr = "0.0.0.0:3930" @@ -100,22 +44,9 @@ size = "1000M" log = "/tmp/tiflash/log/server.log" level = "trace" -[application] -runAsDaemon = true - [raft] pd_addr = "pd0:2379" ignore_databases = "system,default" -# specify which storage engine we use. tmt or dt -storage_engine = "dt" -# Deprecated Raft data storage path setting style. Check [storage.raft] section for new style. -# If it is not set, it will be the first path of "path" appended with "/kvstore". -# kvstore_path = "" - -[raft.snapshot] -# The way to apply snapshot data -# The value is one of "block" / "file1" / "file2". -# method = "file1" [profiles] [profiles.default] diff --git a/tests/docker/config/tiflash_dt_disable_local_tunnel.toml b/tests/docker/config/tiflash_dt_disable_local_tunnel.toml index 23b82909776..1fb166a9a19 100644 --- a/tests/docker/config/tiflash_dt_disable_local_tunnel.toml +++ b/tests/docker/config/tiflash_dt_disable_local_tunnel.toml @@ -13,71 +13,15 @@ # limitations under the License. tmp_path = "/tmp/tiflash/data/tmp" -display_name = "TiFlash" -## Deprecated storage path setting style. Check [storage] section for new style. path = "/tmp/tiflash/data/db" capacity = "10737418240" -## Deprecated storage path setting style of multi-disks. Check [storage] section for new style. -# path = "/tmp/tiflash/data/db,/tmp/tiflash1,/tmp/tiflash2" -# capacity = "0" mark_cache_size = 5368709120 minmax_index_cache_size = 5368709120 tcp_port = 9000 http_port = 8123 -## Storage paths settings. -# [storage] -## The storage format version in storage engine. Valid values: 1, 2 (experimental). -## format_version = 1 - -## If there are multiple SSD disks on the machine, -## specify the path list on `storage.main.dir` can improve TiFlash performance. - -## If there are multiple disks with different IO metrics (e.g. one SSD and some HDDs) -## on the machine, -## set `storage.latest.dir` to store the latest data on SSD (disks with higher IOPS metrics) -## set `storage.main.dir` to store the main data on HDD (disks with lower IOPS metrics) -## can improve TiFlash performance. - -# [storage.main] -## The path to store main data. -# e.g. -# dir = [ "/data0/tiflash" ] -# or -# dir = [ "/data0/tiflash", "/data1/tiflash" ] - -## Store capacity of each path, i.e. max data size allowed. -## If it is not set, or is set to 0s, the actual disk capacity is used. -## Note that we don't support human-readable big numbers(like "10GB") yet. -## Please set in the specified number of bytes. -# e.g. -# capacity = [ 10737418240, 10737418240 ] - -# [storage.latest] -## The path(s) to store latest data. -## If not set, it will be the same with `storage.main.dir`. -# dir = [ ] - -## Store capacity of each path, i.e. max data size allowed. -## If it is not set, or is set to 0s, the actual disk capacity is used. -# e.g. -# capacity = [ 10737418240, 10737418240 ] - -# [storage.raft] -## The path(s) to store Raft data. -## If not set, it will be the paths in `storage.latest.dir` appended with "/kvstore". -# dir = [ ] - -# [storage.io_rate_limit] -## The max I/O bandwith. Default value is 0 and I/O rate limit is disabled. -# max_bytes_per_sec = 268435456 -## max_read_bytes_per_sec and max_write_bytes_per_sec are the same meaning as max_bytes_per_sec, -## but for disk that read bandwidth and write bandwith are calculated separatly, such as GCP's persistent disks. -# max_read_bytes_per_sec = 0 -# max_write_bytes_per_sec = 0 - [flash] tidb_status_addr = "tidb0:10080" service_addr = "0.0.0.0:3930" @@ -100,22 +44,9 @@ size = "1000M" log = "/tmp/tiflash/log/server.log" level = "trace" -[application] -runAsDaemon = true - [raft] pd_addr = "pd0:2379" ignore_databases = "system,default" -# specify which storage engine we use. tmt or dt -storage_engine = "dt" -# Deprecated Raft data storage path setting style. Check [storage.raft] section for new style. -# If it is not set, it will be the first path of "path" appended with "/kvstore". -# kvstore_path = "" - -[raft.snapshot] -# The way to apply snapshot data -# The value is one of "block" / "file1" / "file2". -# method = "file1" [profiles] [profiles.default] From 6afdd7496c093e08c5ef3f92c166e2cb50641595 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 31 May 2022 16:40:27 +0800 Subject: [PATCH 011/104] Remove useless code (#5004) close pingcap/tiflash#4998 --- dbms/src/Debug/dbgFuncRegion.cpp | 12 +- dbms/src/Server/RaftConfigParser.cpp | 6 +- .../Storages/Transaction/ApplySnapshot.cpp | 92 +-------- dbms/src/Storages/Transaction/KVStore.h | 7 +- .../Storages/Transaction/PartitionStreams.cpp | 54 ----- dbms/src/Storages/Transaction/ProxyFFI.cpp | 40 +--- dbms/src/Storages/Transaction/ProxyFFI.h | 1 - dbms/src/Storages/Transaction/Region.cpp | 41 ---- dbms/src/Storages/Transaction/Region.h | 1 - .../Transaction/RegionBlockReader.cpp | 23 +-- .../Storages/Transaction/RegionBlockReader.h | 69 +------ dbms/src/Storages/Transaction/RegionTable.h | 15 -- .../Storages/Transaction/StorageEngineType.h | 16 +- dbms/src/Storages/Transaction/TMTContext.cpp | 12 +- dbms/src/Storages/Transaction/TMTContext.h | 1 - .../Transaction/tests/gtest_kvstore.cpp | 187 ++++++++++-------- etc/config-template.toml | 4 - 17 files changed, 135 insertions(+), 446 deletions(-) diff --git a/dbms/src/Debug/dbgFuncRegion.cpp b/dbms/src/Debug/dbgFuncRegion.cpp index 7924c086508..b2024eac1d8 100644 --- a/dbms/src/Debug/dbgFuncRegion.cpp +++ b/dbms/src/Debug/dbgFuncRegion.cpp @@ -40,7 +40,7 @@ extern const int UNKNOWN_TABLE; // put_region(region_id, start, end, database_name, table_name[, partition-name]) void dbgFuncPutRegion(Context & context, const ASTs & args, DBGInvoker::Printer output) { - RegionID region_id = static_cast(safeGet(typeid_cast(*args[0]).value)); + auto region_id = static_cast(safeGet(typeid_cast(*args[0]).value)); bool has_partition_id = false; size_t args_size = args.size(); if (dynamic_cast(args[args_size - 1].get()) != nullptr) @@ -81,8 +81,8 @@ void dbgFuncPutRegion(Context & context, const ASTs & args, DBGInvoker::Printer } else { - HandleID start = static_cast(safeGet(typeid_cast(*args[1]).value)); - HandleID end = static_cast(safeGet(typeid_cast(*args[2]).value)); + auto start = static_cast(safeGet(typeid_cast(*args[1]).value)); + auto end = static_cast(safeGet(typeid_cast(*args[2]).value)); TMTContext & tmt = context.getTMTContext(); RegionPtr region = RegionBench::createRegion(table_id, region_id, start, end); @@ -107,7 +107,7 @@ void dbgFuncTryFlushRegion(Context & context, const ASTs & args, DBGInvoker::Pri throw Exception("Args not matched, should be: region-id", ErrorCodes::BAD_ARGUMENTS); } - RegionID region_id = static_cast(safeGet(typeid_cast(*args[0]).value)); + auto region_id = static_cast(safeGet(typeid_cast(*args[0]).value)); TMTContext & tmt = context.getTMTContext(); tmt.getRegionTable().tryFlushRegion(region_id); @@ -160,7 +160,7 @@ void dbgFuncDumpAllRegion(Context & context, const ASTs & args, DBGInvoker::Prin if (args.empty()) throw Exception("Args not matched, should be: table_id", ErrorCodes::BAD_ARGUMENTS); - TableID table_id = static_cast(safeGet(typeid_cast(*args[0]).value)); + auto table_id = static_cast(safeGet(typeid_cast(*args[0]).value)); bool ignore_none = false; if (args.size() > 1) @@ -190,7 +190,7 @@ void dbgFuncRemoveRegion(Context & context, const ASTs & args, DBGInvoker::Print if (args.empty()) throw Exception("Args not matched, should be: region_id", ErrorCodes::BAD_ARGUMENTS); - RegionID region_id = static_cast(safeGet(typeid_cast(*args[0]).value)); + auto region_id = static_cast(safeGet(typeid_cast(*args[0]).value)); TMTContext & tmt = context.getTMTContext(); KVStorePtr & kvstore = tmt.getKVStore(); diff --git a/dbms/src/Server/RaftConfigParser.cpp b/dbms/src/Server/RaftConfigParser.cpp index 8e146dd842e..2f0a88855cd 100644 --- a/dbms/src/Server/RaftConfigParser.cpp +++ b/dbms/src/Server/RaftConfigParser.cpp @@ -92,11 +92,7 @@ TiFlashRaftConfig TiFlashRaftConfig::parseSettings(Poco::Util::LayeredConfigurat { String snapshot_method = config.getString("raft.snapshot.method"); std::transform(snapshot_method.begin(), snapshot_method.end(), snapshot_method.begin(), [](char ch) { return std::tolower(ch); }); - if (snapshot_method == "block") - { - res.snapshot_apply_method = TiDB::SnapshotApplyMethod::Block; - } - else if (snapshot_method == "file1") + if (snapshot_method == "file1") { res.snapshot_apply_method = TiDB::SnapshotApplyMethod::DTFile_Directory; } diff --git a/dbms/src/Storages/Transaction/ApplySnapshot.cpp b/dbms/src/Storages/Transaction/ApplySnapshot.cpp index 3ed04d5ecbf..2df95fead93 100644 --- a/dbms/src/Storages/Transaction/ApplySnapshot.cpp +++ b/dbms/src/Storages/Transaction/ApplySnapshot.cpp @@ -261,75 +261,6 @@ void KVStore::onSnapshot(const RegionPtrWrap & new_region_wrap, RegionPtr old_re extern RegionPtrWithBlock::CachePtr GenRegionPreDecodeBlockData(const RegionPtr &, Context &); -/// `preHandleSnapshotToBlock` read data from SSTFiles and predoced the data as a block -RegionPreDecodeBlockDataPtr KVStore::preHandleSnapshotToBlock( - RegionPtr new_region, - const SSTViewVec snaps, - uint64_t /*index*/, - uint64_t /*term*/, - TMTContext & tmt) -{ - RegionPreDecodeBlockDataPtr cache{nullptr}; - { - decltype(bg_gc_region_data)::value_type tmp; - std::lock_guard lock(bg_gc_region_data_mutex); - if (!bg_gc_region_data.empty()) - { - tmp.swap(bg_gc_region_data.back()); - bg_gc_region_data.pop_back(); - } - } - - Stopwatch watch; - auto & ctx = tmt.getContext(); - SCOPE_EXIT({ GET_METRIC(tiflash_raft_command_duration_seconds, type_apply_snapshot_predecode).Observe(watch.elapsedSeconds()); }); - - { - LOG_FMT_INFO(log, "Pre-handle snapshot {} with {} TiKV sst files", new_region->toString(false), snaps.len); - // Iterator over all SST files and insert key-values into `new_region` - for (UInt64 i = 0; i < snaps.len; ++i) - { - const auto & snapshot = snaps.views[i]; - auto sst_reader = SSTReader{proxy_helper, snapshot}; - - uint64_t kv_size = 0; - while (sst_reader.remained()) - { - auto key = sst_reader.key(); - auto value = sst_reader.value(); - new_region->insert(snaps.views[i].type, TiKVKey(key.data, key.len), TiKVValue(value.data, value.len)); - ++kv_size; - sst_reader.next(); - } - - LOG_FMT_INFO(log, - "Decode {} got [cf: {}, kv size: {}]", - std::string_view(snapshot.path.data, snapshot.path.len), - CFToName(snapshot.type), - kv_size); - // Note that number of keys in different cf will be aggregated into one metrics - GET_METRIC(tiflash_raft_process_keys, type_apply_snapshot).Increment(kv_size); - } - { - LOG_FMT_INFO(log, "Start to pre-decode {} into block", new_region->toString()); - auto block_cache = GenRegionPreDecodeBlockData(new_region, ctx); - if (block_cache) - { - std::stringstream ss; - block_cache->toString(ss); - LOG_FMT_INFO(log, "Got pre-decode block cache {}", ss.str()); - } - else - LOG_FMT_INFO(log, "Got empty pre-decode block cache"); - - cache = std::move(block_cache); - } - LOG_FMT_INFO(log, "Pre-handle snapshot {} cost {}ms", new_region->toString(false), watch.elapsedMilliseconds()); - } - - return cache; -} - std::vector KVStore::preHandleSnapshotToFiles( RegionPtr new_region, const SSTViewVec snaps, @@ -473,8 +404,8 @@ void KVStore::handlePreApplySnapshot(const RegionPtrWrap & new_region, TMTContex LOG_FMT_INFO(log, "{} apply snapshot success", new_region->toString(false)); } -template void KVStore::handlePreApplySnapshot(const RegionPtrWithBlock &, TMTContext &); template void KVStore::handlePreApplySnapshot(const RegionPtrWithSnapshotFiles &, TMTContext &); + template void KVStore::checkAndApplySnapshot(const RegionPtrWithBlock &, TMTContext &); template void KVStore::checkAndApplySnapshot(const RegionPtrWithSnapshotFiles &, TMTContext &); template void KVStore::onSnapshot(const RegionPtrWithBlock &, RegionPtr, UInt64, TMTContext &); @@ -521,10 +452,7 @@ void KVStore::handleApplySnapshot( TMTContext & tmt) { auto new_region = genRegionPtr(std::move(region), peer_id, index, term); - if (snapshot_apply_method == TiDB::SnapshotApplyMethod::Block) - handlePreApplySnapshot(RegionPtrWithBlock{new_region, preHandleSnapshotToBlock(new_region, snaps, index, term, tmt)}, tmt); - else - handlePreApplySnapshot(RegionPtrWithSnapshotFiles{new_region, preHandleSnapshotToFiles(new_region, snaps, index, term, tmt)}, tmt); + handlePreApplySnapshot(RegionPtrWithSnapshotFiles{new_region, preHandleSnapshotToFiles(new_region, snaps, index, term, tmt)}, tmt); } EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec snaps, UInt64 index, UInt64 term, TMTContext & tmt) @@ -543,15 +471,12 @@ EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec fiu_do_on(FailPoints::force_set_sst_decode_rand, { static int num_call = 0; - switch (num_call++ % 3) + switch (num_call++ % 2) { case 0: - snapshot_apply_method = TiDB::SnapshotApplyMethod::Block; - break; - case 1: snapshot_apply_method = TiDB::SnapshotApplyMethod::DTFile_Directory; break; - case 2: + case 1: snapshot_apply_method = TiDB::SnapshotApplyMethod::DTFile_Single; break; default: @@ -576,15 +501,6 @@ EngineStoreApplyRes KVStore::handleIngestSST(UInt64 region_id, const SSTViewVec } }; - if (snapshot_apply_method == TiDB::SnapshotApplyMethod::Block) - { - // try to flush remain data in memory. - func_try_flush(); - region->handleIngestSSTInMemory(snaps, index, term); - // after `handleIngestSSTInMemory`, all data are stored in `region`, try to flush committed data into storage - func_try_flush(); - } - else { // try to flush remain data in memory. func_try_flush(); diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index ef851d67958..8673cae3ff3 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -109,12 +109,7 @@ class KVStore final : private boost::noncopyable EngineStoreApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt); void handleApplySnapshot(metapb::Region && region, uint64_t peer_id, const SSTViewVec, uint64_t index, uint64_t term, TMTContext & tmt); - RegionPreDecodeBlockDataPtr preHandleSnapshotToBlock( - RegionPtr new_region, - const SSTViewVec, - uint64_t index, - uint64_t term, - TMTContext & tmt); + std::vector /* */ preHandleSnapshotToFiles( RegionPtr new_region, const SSTViewVec, diff --git a/dbms/src/Storages/Transaction/PartitionStreams.cpp b/dbms/src/Storages/Transaction/PartitionStreams.cpp index 13840159ebb..ada792c80f7 100644 --- a/dbms/src/Storages/Transaction/PartitionStreams.cpp +++ b/dbms/src/Storages/Transaction/PartitionStreams.cpp @@ -353,60 +353,6 @@ void RegionTable::writeBlockByRegion( data_list_to_remove = std::move(*data_list_read); } -RegionTable::ReadBlockByRegionRes RegionTable::readBlockByRegion(const TiDB::TableInfo & table_info, - const ColumnsDescription & columns [[maybe_unused]], - const Names & column_names_to_read, - const RegionPtr & region, - RegionVersion region_version, - RegionVersion conf_version, - bool resolve_locks, - Timestamp start_ts, - const std::unordered_set * bypass_lock_ts, - RegionScanFilterPtr scan_filter) -{ - if (!region) - throw Exception(std::string(__PRETTY_FUNCTION__) + ": region is null", ErrorCodes::LOGICAL_ERROR); - - // Tiny optimization for queries that need only handle, tso, delmark. - bool need_value = column_names_to_read.size() != 3; - auto region_data_lock = resolveLocksAndReadRegionData( - table_info.id, - region, - start_ts, - bypass_lock_ts, - region_version, - conf_version, - resolve_locks, - need_value); - - return std::visit(variant_op::overloaded{ - [&](RegionDataReadInfoList & data_list_read) -> ReadBlockByRegionRes { - /// Read region data as block. - Block block; - // FIXME: remove this deprecated function - assert(0); - { - auto reader = RegionBlockReader(nullptr); - bool ok = reader.setStartTs(start_ts) - .setFilter(scan_filter) - .read(block, data_list_read, /*force_decode*/ true); - if (!ok) - // TODO: Enrich exception message. - throw Exception("Read region " + std::to_string(region->id()) + " of table " - + std::to_string(table_info.id) + " failed", - ErrorCodes::LOGICAL_ERROR); - } - return block; - }, - [&](LockInfoPtr & lock_value) -> ReadBlockByRegionRes { - assert(lock_value); - throw LockException(region->id(), std::move(lock_value)); - }, - [](RegionException::RegionReadStatus & s) -> ReadBlockByRegionRes { return s; }, - }, - region_data_lock); -} - RegionTable::ResolveLocksAndWriteRegionRes RegionTable::resolveLocksAndWriteRegion(TMTContext & tmt, const TiDB::TableID table_id, const RegionPtr & region, diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index 58e7f5ad2e5..cc7d1e10a49 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -304,25 +304,13 @@ RawRustPtrWrap::~RawRustPtrWrap() RustGcHelper::instance().gcRustPtr(ptr, type); } RawRustPtrWrap::RawRustPtrWrap(RawRustPtrWrap && src) + : RawRustPtr() { RawRustPtr & tar = (*this); tar = src; src.ptr = nullptr; } -struct PreHandledSnapshotWithBlock -{ - ~PreHandledSnapshotWithBlock() { CurrentMetrics::sub(CurrentMetrics::RaftNumSnapshotsPendingApply); } - PreHandledSnapshotWithBlock(const RegionPtr & region_, RegionPtrWithBlock::CachePtr && cache_) - : region(region_) - , cache(std::move(cache_)) - { - CurrentMetrics::add(CurrentMetrics::RaftNumSnapshotsPendingApply); - } - RegionPtr region; - RegionPtrWithBlock::CachePtr cache; -}; - struct PreHandledSnapshotWithFiles { ~PreHandledSnapshotWithFiles() { CurrentMetrics::sub(CurrentMetrics::RaftNumSnapshotsPendingApply); } @@ -362,13 +350,6 @@ RawCppPtr PreHandleSnapshot( switch (kvstore->applyMethod()) { - case TiDB::SnapshotApplyMethod::Block: - { - // Pre-decode as a block - auto new_region_block_cache = kvstore->preHandleSnapshotToBlock(new_region, snaps, index, term, tmt); - auto * res = new PreHandledSnapshotWithBlock{new_region, std::move(new_region_block_cache)}; - return GenRawCppPtr(res, RawCppPtrTypeImpl::PreHandledSnapshotWithBlock); - } case TiDB::SnapshotApplyMethod::DTFile_Directory: case TiDB::SnapshotApplyMethod::DTFile_Single: { @@ -391,18 +372,12 @@ RawCppPtr PreHandleSnapshot( template void ApplyPreHandledSnapshot(EngineStoreServerWrap * server, PreHandledSnapshot * snap) { - static_assert( - std::is_same_v || std::is_same_v, - "Unknown pre-handled snapshot type"); + static_assert(std::is_same_v, "Unknown pre-handled snapshot type"); try { auto & kvstore = server->tmt->getKVStore(); - if constexpr (std::is_same_v) - { - kvstore->handlePreApplySnapshot(RegionPtrWithBlock{snap->region, std::move(snap->cache)}, *server->tmt); - } - else if constexpr (std::is_same_v) + if constexpr (std::is_same_v) { kvstore->handlePreApplySnapshot(RegionPtrWithSnapshotFiles{snap->region, std::move(snap->ingest_ids)}, *server->tmt); } @@ -418,12 +393,6 @@ void ApplyPreHandledSnapshot(EngineStoreServerWrap * server, RawVoidPtr res, Raw { switch (static_cast(type)) { - case RawCppPtrTypeImpl::PreHandledSnapshotWithBlock: - { - auto * snap = reinterpret_cast(res); - ApplyPreHandledSnapshot(server, snap); - break; - } case RawCppPtrTypeImpl::PreHandledSnapshotWithFiles: { auto * snap = reinterpret_cast(res); @@ -445,9 +414,6 @@ void GcRawCppPtr(RawVoidPtr ptr, RawCppPtrType type) case RawCppPtrTypeImpl::String: delete reinterpret_cast(ptr); break; - case RawCppPtrTypeImpl::PreHandledSnapshotWithBlock: - delete reinterpret_cast(ptr); - break; case RawCppPtrTypeImpl::PreHandledSnapshotWithFiles: delete reinterpret_cast(ptr); break; diff --git a/dbms/src/Storages/Transaction/ProxyFFI.h b/dbms/src/Storages/Transaction/ProxyFFI.h index 5d87af94f30..e1c01599275 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.h +++ b/dbms/src/Storages/Transaction/ProxyFFI.h @@ -56,7 +56,6 @@ enum class RawCppPtrTypeImpl : RawCppPtrType { None = 0, String, - PreHandledSnapshotWithBlock, PreHandledSnapshotWithFiles, WakerNotifier, }; diff --git a/dbms/src/Storages/Transaction/Region.cpp b/dbms/src/Storages/Transaction/Region.cpp index e021de3d978..aa75eabb4b9 100644 --- a/dbms/src/Storages/Transaction/Region.cpp +++ b/dbms/src/Storages/Transaction/Region.cpp @@ -720,47 +720,6 @@ EngineStoreApplyRes Region::handleWriteRaftCmd(const WriteCmdsView & cmds, UInt6 return EngineStoreApplyRes::None; } -void Region::handleIngestSSTInMemory(const SSTViewVec snaps, UInt64 index, UInt64 term) -{ - if (index <= appliedIndex()) - return; - - { - std::unique_lock lock(mutex); - - for (UInt64 i = 0; i < snaps.len; ++i) - { - const auto & snapshot = snaps.views[i]; - auto sst_reader = SSTReader{proxy_helper, snapshot}; - - LOG_FMT_INFO(log, - "{} begin to ingest sst of cf {} at [term: {}, index: {}]", - this->toString(false), - CFToName(snapshot.type), - term, - index); - - uint64_t kv_size = 0; - while (sst_reader.remained()) - { - auto key = sst_reader.key(); - auto value = sst_reader.value(); - doInsert(snaps.views[i].type, TiKVKey(key.data, key.len), TiKVValue(value.data, value.len)); - ++kv_size; - sst_reader.next(); - } - - LOG_FMT_INFO(log, - "{} finish to ingest sst of kv count {}", - this->toString(false), - kv_size); - GET_METRIC(tiflash_raft_process_keys, type_ingest_sst).Increment(kv_size); - } - meta.setApplied(index, term); - } - meta.notifyAll(); -} - void Region::finishIngestSSTByDTFile(RegionPtr && rhs, UInt64 index, UInt64 term) { if (index <= appliedIndex()) diff --git a/dbms/src/Storages/Transaction/Region.h b/dbms/src/Storages/Transaction/Region.h index b31ae0cdc49..06b18de379a 100644 --- a/dbms/src/Storages/Transaction/Region.h +++ b/dbms/src/Storages/Transaction/Region.h @@ -191,7 +191,6 @@ class Region : public std::enable_shared_from_this TableID getMappedTableID() const; EngineStoreApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 index, UInt64 term, TMTContext & tmt); - void handleIngestSSTInMemory(const SSTViewVec snaps, UInt64 index, UInt64 term); void finishIngestSSTByDTFile(RegionPtr && rhs, UInt64 index, UInt64 term); UInt64 getSnapshotEventFlag() const { return snapshot_event_flag; } diff --git a/dbms/src/Storages/Transaction/RegionBlockReader.cpp b/dbms/src/Storages/Transaction/RegionBlockReader.cpp index 32be7302775..af351f4a6b0 100644 --- a/dbms/src/Storages/Transaction/RegionBlockReader.cpp +++ b/dbms/src/Storages/Transaction/RegionBlockReader.cpp @@ -58,7 +58,7 @@ bool RegionBlockReader::readImpl(Block & block, const RegionDataReadInfoList & d const auto & pk_column_ids = schema_snapshot->pk_column_ids; const auto & pk_pos_map = schema_snapshot->pk_pos_map; - SortedColumnIDWithPosConstIter column_ids_iter = read_column_ids.begin(); + auto column_ids_iter = read_column_ids.begin(); size_t next_column_pos = 0; /// every table in tiflash must have an extra handle column, it either @@ -112,25 +112,6 @@ bool RegionBlockReader::readImpl(Block & block, const RegionDataReadInfoList & d size_t index = 0; for (const auto & [pk, write_type, commit_ts, value_ptr] : data_list) { - // Ignore data after the start_ts. - if (commit_ts > start_ts) - continue; - - bool should_skip = false; - if constexpr (pk_type != TMTPKType::STRING) - { - if constexpr (pk_type == TMTPKType::UINT64) - { - should_skip = scan_filter != nullptr && scan_filter->filter(static_cast(pk)); - } - else - { - should_skip = scan_filter != nullptr && scan_filter->filter(static_cast(pk)); - } - } - if (should_skip) - continue; - /// set delmark and version column delmark_data.emplace_back(write_type == Region::DelFlag); version_data.emplace_back(commit_ts); @@ -186,7 +167,7 @@ bool RegionBlockReader::readImpl(Block & block, const RegionDataReadInfoList & d { // The pk_type must be Int32/Uint32 or more narrow type // so cannot tell its' exact type here, just use `insert(Field)` - HandleID handle_value(static_cast(pk)); + auto handle_value(static_cast(pk)); raw_pk_column->insert(Field(handle_value)); if (unlikely(raw_pk_column->getInt(index) != handle_value)) { diff --git a/dbms/src/Storages/Transaction/RegionBlockReader.h b/dbms/src/Storages/Transaction/RegionBlockReader.h index 860e0d149e6..ec633e805c0 100644 --- a/dbms/src/Storages/Transaction/RegionBlockReader.h +++ b/dbms/src/Storages/Transaction/RegionBlockReader.h @@ -37,79 +37,12 @@ using ManageableStoragePtr = std::shared_ptr; struct ColumnsDescription; class Block; -class RegionScanFilter -{ - bool is_full_range_scan; - std::vector> int64_ranges; - std::vector> uint64_ranges; - - bool isValidHandle(UInt64 handle) - { - for (const auto & range : uint64_ranges) - { - if (handle >= range.first && handle < range.second) - { - return true; - } - } - return false; - } - bool isValidHandle(Int64 handle) - { - for (const auto & range : int64_ranges) - { - if (handle >= range.first && handle < range.second) - { - return true; - } - } - return false; - } - -public: - RegionScanFilter( - bool is_full_range_scan_, - std::vector> int64_ranges_, - std::vector> uint64_ranges_) - : is_full_range_scan(is_full_range_scan_) - , int64_ranges(std::move(int64_ranges_)) - , uint64_ranges(std::move(uint64_ranges_)) - {} - bool filter(UInt64 handle) { return !is_full_range_scan && !isValidHandle(handle); } - bool filter(Int64 handle) { return !is_full_range_scan && !isValidHandle(handle); } - bool isFullRangeScan() { return is_full_range_scan; } - const std::vector> & getUInt64Ranges() { return uint64_ranges; } - const std::vector> & getInt64Ranges() { return int64_ranges; } -}; - -using RegionScanFilterPtr = std::shared_ptr; - /// The Reader to read the region data in `data_list` and decode based on the given table_info and columns, as a block. class RegionBlockReader : private boost::noncopyable { - RegionScanFilterPtr scan_filter; - Timestamp start_ts = std::numeric_limits::max(); - public: RegionBlockReader(DecodingStorageSchemaSnapshotConstPtr schema_snapshot_); - inline RegionBlockReader & setFilter(RegionScanFilterPtr filter) - { - scan_filter = std::move(filter); - return *this; - } - - /// Set the `start_ts` for reading data. The `start_ts` is `Timestamp::max` if not set. - /// - /// Data with commit_ts > start_ts will be ignored. This is for the sake of decode safety on read, - /// i.e. as data keeps being synced to region cache while the schema for a specific read is fixed, - /// we'll always have newer data than schema, only ignoring them can guarantee the decode safety. - inline RegionBlockReader & setStartTs(Timestamp tso) - { - start_ts = tso; - return *this; - } - /// Read `data_list` as a block. /// /// On decode error, i.e. column number/type mismatch, will do force apply schema, @@ -117,7 +50,7 @@ class RegionBlockReader : private boost::noncopyable /// Moreover, exception will be thrown if we see fatal decode error meanwhile `force_decode` is true. /// /// `RegionBlockReader::read` is the common routine used by both 'flush' and 'read' processes of TXN engine (Delta-Tree, TXN-MergeTree), - /// each of which will use carefully adjusted 'start_ts' and 'force_decode' with appropriate error handling/retry to get what they want. + /// each of which will use carefully adjusted 'force_decode' with appropriate error handling/retry to get what they want. bool read(Block & block, const RegionDataReadInfoList & data_list, bool force_decode); private: diff --git a/dbms/src/Storages/Transaction/RegionTable.h b/dbms/src/Storages/Transaction/RegionTable.h index b30a905541a..717b1cd568f 100644 --- a/dbms/src/Storages/Transaction/RegionTable.h +++ b/dbms/src/Storages/Transaction/RegionTable.h @@ -146,21 +146,6 @@ class RegionTable : private boost::noncopyable Poco::Logger * log, bool lock_region = true); - /// Read the data of the given region into block, take good care of learner read and locks. - /// Assuming that the schema has been properly synced by outer, i.e. being new enough to decode data before start_ts, - /// we directly ask RegionBlockReader::read to perform a read with the given start_ts and force_decode being true. - using ReadBlockByRegionRes = std::variant; - static ReadBlockByRegionRes readBlockByRegion(const TiDB::TableInfo & table_info, - const ColumnsDescription & columns, - const Names & column_names_to_read, - const RegionPtr & region, - RegionVersion region_version, - RegionVersion conf_version, - bool resolve_locks, - Timestamp start_ts, - const std::unordered_set * bypass_lock_ts, - RegionScanFilterPtr scan_filter = nullptr); - /// Check transaction locks in region, and write committed data in it into storage engine if check passed. Otherwise throw an LockException. /// The write logic is the same as #writeBlockByRegion, with some extra checks about region version and conf_version. using ResolveLocksAndWriteRegionRes = std::variant; diff --git a/dbms/src/Storages/Transaction/StorageEngineType.h b/dbms/src/Storages/Transaction/StorageEngineType.h index f202d15a769..3d103ca60c1 100644 --- a/dbms/src/Storages/Transaction/StorageEngineType.h +++ b/dbms/src/Storages/Transaction/StorageEngineType.h @@ -33,7 +33,7 @@ enum class StorageEngine enum class SnapshotApplyMethod : std::int32_t { - Block = 1, + DEPRECATED_Block = 1, // Invalid if the storage engine is not DeltaTree DTFile_Directory, DTFile_Single, @@ -43,14 +43,12 @@ inline const std::string applyMethodToString(SnapshotApplyMethod method) { switch (method) { - case SnapshotApplyMethod::Block: - return "block"; - case SnapshotApplyMethod::DTFile_Directory: - return "file1"; - case SnapshotApplyMethod::DTFile_Single: - return "file2"; - default: - return "unknown(" + std::to_string(static_cast(method)) + ")"; + case SnapshotApplyMethod::DTFile_Directory: + return "file1"; + case SnapshotApplyMethod::DTFile_Single: + return "file2"; + default: + return "unknown(" + std::to_string(static_cast(method)) + ")"; } return "unknown"; } diff --git a/dbms/src/Storages/Transaction/TMTContext.cpp b/dbms/src/Storages/Transaction/TMTContext.cpp index 719784edaf2..3c7468cbd64 100644 --- a/dbms/src/Storages/Transaction/TMTContext.cpp +++ b/dbms/src/Storages/Transaction/TMTContext.cpp @@ -36,6 +36,8 @@ extern const uint64_t DEFAULT_WAIT_INDEX_TIMEOUT_MS = 5 * 60 * 1000; const int64_t DEFAULT_WAIT_REGION_READY_TIMEOUT_SEC = 20 * 60; +const int64_t DEFAULT_READ_INDEX_WORKER_TICK_MS = 10; + TMTContext::TMTContext(Context & context_, const TiFlashRaftConfig & raft_config, const pingcap::ClusterConfig & cluster_config) : context(context_) , kvstore(std::make_shared(context, raft_config.snapshot_apply_method)) @@ -56,7 +58,7 @@ TMTContext::TMTContext(Context & context_, const TiFlashRaftConfig & raft_config , replica_read_max_thread(1) , batch_read_index_timeout_ms(DEFAULT_BATCH_READ_INDEX_TIMEOUT_MS) , wait_index_timeout_ms(DEFAULT_WAIT_INDEX_TIMEOUT_MS) - , read_index_worker_tick_ms(10) + , read_index_worker_tick_ms(DEFAULT_READ_INDEX_WORKER_TICK_MS) , wait_region_ready_timeout_sec(DEFAULT_WAIT_REGION_READY_TIMEOUT_SEC) {} @@ -149,12 +151,6 @@ SchemaSyncerPtr TMTContext::getSchemaSyncer() const return schema_syncer; } -void TMTContext::setSchemaSyncer(SchemaSyncerPtr rhs) -{ - std::lock_guard lock(mutex); - schema_syncer = rhs; -} - pingcap::pd::ClientPtr TMTContext::getPDClient() const { return cluster->pd_client; @@ -194,7 +190,7 @@ void TMTContext::reloadConfig(const Poco::Util::AbstractConfiguration & config) t = t >= 0 ? t : std::numeric_limits::max(); // set -1 to wait infinitely t; }); - read_index_worker_tick_ms = config.getUInt64(READ_INDEX_WORKER_TICK_MS, 10 /*10ms*/); + read_index_worker_tick_ms = config.getUInt64(READ_INDEX_WORKER_TICK_MS, DEFAULT_READ_INDEX_WORKER_TICK_MS); } { LOG_FMT_INFO( diff --git a/dbms/src/Storages/Transaction/TMTContext.h b/dbms/src/Storages/Transaction/TMTContext.h index 27e0482b787..bd592dad315 100644 --- a/dbms/src/Storages/Transaction/TMTContext.h +++ b/dbms/src/Storages/Transaction/TMTContext.h @@ -84,7 +84,6 @@ class TMTContext : private boost::noncopyable explicit TMTContext(Context & context_, const TiFlashRaftConfig & raft_config, const pingcap::ClusterConfig & cluster_config_); SchemaSyncerPtr getSchemaSyncer() const; - void setSchemaSyncer(SchemaSyncerPtr); pingcap::pd::ClientPtr getPDClient() const; diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp index 2378871f71f..f0cafce3914 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp @@ -310,10 +310,13 @@ void RegionKVStoreTest::testRaftMergeRollback(KVStore & kvs, TMTContext & tmt) } } region->setStateApplying(); + try { - kvs.handleAdminRaftCmd(std::move(request), - std::move(response), + raft_cmdpb::AdminRequest first_request = request; + raft_cmdpb::AdminResponse first_response = response; + kvs.handleAdminRaftCmd(std::move(first_request), + std::move(first_response), region_id, 32, 6, @@ -925,12 +928,14 @@ void RegionKVStoreTest::testKVStore() TiKVValue lock_value = RecordKVFormat::encodeLockCfValue(Region::DelFlag, "pk", 77, 0); RegionBench::setupDelRequest(request.add_requests(), ColumnFamilyName::Lock, lock_key); } - ASSERT_EQ(kvs.handleWriteRaftCmd(std::move(request), 1, 7, 6, ctx.getTMTContext()), + raft_cmdpb::RaftCmdRequest first_request = request; + ASSERT_EQ(kvs.handleWriteRaftCmd(std::move(first_request), 1, 7, 6, ctx.getTMTContext()), EngineStoreApplyRes::None); RegionBench::setupDelRequest(request.add_requests(), ColumnFamilyName::Write, TiKVKey("illegal key")); // index <= appliedIndex(), ignore - ASSERT_EQ(kvs.handleWriteRaftCmd(std::move(request), 1, 7, 6, ctx.getTMTContext()), + raft_cmdpb::RaftCmdRequest second_request; + ASSERT_EQ(kvs.handleWriteRaftCmd(std::move(second_request), 1, 7, 6, ctx.getTMTContext()), EngineStoreApplyRes::None); try { @@ -972,13 +977,24 @@ void RegionKVStoreTest::testKVStore() request.mutable_compact_log(); request.set_cmd_type(::raft_cmdpb::AdminCmdType::CompactLog); - ASSERT_EQ(kvs.handleAdminRaftCmd(std::move(request), std::move(response), 7, 22, 6, ctx.getTMTContext()), EngineStoreApplyRes::Persist); - ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(response), 7, 23, 6, ctx.getTMTContext()), EngineStoreApplyRes::None); + raft_cmdpb::AdminRequest first_request = request; + raft_cmdpb::AdminResponse first_response = response; + + ASSERT_EQ(kvs.handleAdminRaftCmd(std::move(first_request), std::move(first_response), 7, 22, 6, ctx.getTMTContext()), EngineStoreApplyRes::Persist); + + raft_cmdpb::AdminResponse second_response = response; + ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(second_response), 7, 23, 6, ctx.getTMTContext()), EngineStoreApplyRes::None); request.set_cmd_type(::raft_cmdpb::AdminCmdType::ComputeHash); - ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(response), 7, 24, 6, ctx.getTMTContext()), EngineStoreApplyRes::None); + + raft_cmdpb::AdminResponse third_response = response; + ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(third_response), 7, 24, 6, ctx.getTMTContext()), EngineStoreApplyRes::None); request.set_cmd_type(::raft_cmdpb::AdminCmdType::VerifyHash); - ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(response), 7, 25, 6, ctx.getTMTContext()), EngineStoreApplyRes::None); - ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(response), 8192, 5, 6, ctx.getTMTContext()), EngineStoreApplyRes::NotFound); + + raft_cmdpb::AdminResponse fourth_response = response; + ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(fourth_response), 7, 25, 6, ctx.getTMTContext()), EngineStoreApplyRes::None); + + raft_cmdpb::AdminResponse fifth_response = response; + ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(fifth_response), 8192, 5, 6, ctx.getTMTContext()), EngineStoreApplyRes::NotFound); { kvs.setRegionCompactLogConfig(0, 0, 0); request.set_cmd_type(::raft_cmdpb::AdminCmdType::CompactLog); @@ -994,62 +1010,12 @@ void RegionKVStoreTest::testKVStore() } { auto ori_snapshot_apply_method = kvs.snapshot_apply_method; - kvs.snapshot_apply_method = TiDB::SnapshotApplyMethod::Block; + kvs.snapshot_apply_method = TiDB::SnapshotApplyMethod::DTFile_Single; SCOPE_EXIT({ kvs.snapshot_apply_method = ori_snapshot_apply_method; }); - { - { - auto region = makeRegion(22, RecordKVFormat::genKey(1, 55), RecordKVFormat::genKey(1, 65)); - kvs.checkAndApplySnapshot(region, ctx.getTMTContext()); - } - try - { - auto region = makeRegion(20, RecordKVFormat::genKey(1, 55), RecordKVFormat::genKey(1, 65)); - kvs.checkAndApplySnapshot(region, ctx.getTMTContext()); // overlap, but not tombstone - ASSERT_TRUE(false); - } - catch (Exception & e) - { - ASSERT_EQ(e.message(), "range of region 20 is overlapped with 22, state: region { id: 22 }"); - } - { - const auto * ori_ptr = proxy_helper.proxy_ptr.inner; - proxy_helper.proxy_ptr.inner = nullptr; - SCOPE_EXIT({ - proxy_helper.proxy_ptr.inner = ori_ptr; - }); - try - { - auto region = makeRegion(20, RecordKVFormat::genKey(1, 55), RecordKVFormat::genKey(1, 65)); - kvs.checkAndApplySnapshot(region, ctx.getTMTContext()); - ASSERT_TRUE(false); - } - catch (Exception & e) - { - ASSERT_EQ(e.message(), "getRegionLocalState meet internal error: RaftStoreProxyPtr is none"); - } - } - - { - proxy_instance.getRegion(22)->setSate(({ - raft_serverpb::RegionLocalState s; - s.set_state(::raft_serverpb::PeerState::Tombstone); - s; - })); - auto region = makeRegion(20, RecordKVFormat::genKey(1, 55), RecordKVFormat::genKey(1, 65)); - kvs.checkAndApplySnapshot(region, ctx.getTMTContext()); // overlap, tombstone, remove previous one - ASSERT_EQ(nullptr, kvs.getRegion(22)); - ASSERT_NE(nullptr, kvs.getRegion(20)); - - auto state = proxy_helper.getRegionLocalState(8192); - ASSERT_EQ(state.state(), raft_serverpb::PeerState::Tombstone); - } - - kvs.handleDestroy(20, ctx.getTMTContext()); - } auto region_id = 19; auto region = makeRegion(region_id, RecordKVFormat::genKey(1, 50), RecordKVFormat::genKey(1, 60)); auto region_id_str = std::to_string(19); @@ -1076,7 +1042,7 @@ void RegionKVStoreTest::testKVStore() 8, 5, ctx.getTMTContext()); - ASSERT_EQ(kvs.getRegion(19)->dataInfo(), "[default 2 ]"); + ASSERT_EQ(kvs.getRegion(19)->checkIndex(8), true); try { kvs.handleApplySnapshot( @@ -1091,22 +1057,85 @@ void RegionKVStoreTest::testKVStore() catch (Exception & e) { ASSERT_EQ(e.message(), "[region 19] already has newer apply-index 8 than 6, should not happen"); - ASSERT_EQ(kvs.getRegion(19)->dataInfo(), "[default 2 ]"); // apply-snapshot do not work } - kvs.handleApplySnapshot( - region->getMetaRegion(), - 2, - {}, // empty - 8, // same index - 5, - ctx.getTMTContext()); - ASSERT_EQ(kvs.getRegion(19)->dataInfo(), "[default 2 ]"); // apply-snapshot do not work - region = makeRegion(19, RecordKVFormat::genKey(1, 50), RecordKVFormat::genKey(1, 60)); - region->handleWriteRaftCmd({}, 10, 10, ctx.getTMTContext()); - kvs.checkAndApplySnapshot(region, ctx.getTMTContext()); - ASSERT_EQ(kvs.getRegion(19)->dataInfo(), "[]"); + } + + { + { + auto region = makeRegion(22, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100)); + auto ingest_ids = kvs.preHandleSnapshotToFiles( + region, + {}, + 9, + 5, + ctx.getTMTContext()); + kvs.checkAndApplySnapshot(RegionPtrWithSnapshotFiles{region, std::move(ingest_ids)}, ctx.getTMTContext()); + } + try + { + auto region = makeRegion(20, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100)); + auto ingest_ids = kvs.preHandleSnapshotToFiles( + region, + {}, + 9, + 5, + ctx.getTMTContext()); + kvs.checkAndApplySnapshot(RegionPtrWithSnapshotFiles{region, std::move(ingest_ids)}, ctx.getTMTContext()); // overlap, but not tombstone + ASSERT_TRUE(false); + } + catch (Exception & e) + { + ASSERT_EQ(e.message(), "range of region 20 is overlapped with 22, state: region { id: 22 }"); + } + + { + const auto * ori_ptr = proxy_helper.proxy_ptr.inner; + proxy_helper.proxy_ptr.inner = nullptr; + SCOPE_EXIT({ + proxy_helper.proxy_ptr.inner = ori_ptr; + }); + + try + { + auto region = makeRegion(20, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100)); + auto ingest_ids = kvs.preHandleSnapshotToFiles( + region, + {}, + 10, + 5, + ctx.getTMTContext()); + kvs.checkAndApplySnapshot(RegionPtrWithSnapshotFiles{region, std::move(ingest_ids)}, ctx.getTMTContext()); + ASSERT_TRUE(false); + } + catch (Exception & e) + { + ASSERT_EQ(e.message(), "getRegionLocalState meet internal error: RaftStoreProxyPtr is none"); + } + } + + { + proxy_instance.getRegion(22)->setSate(({ + raft_serverpb::RegionLocalState s; + s.set_state(::raft_serverpb::PeerState::Tombstone); + s; + })); + auto region = makeRegion(20, RecordKVFormat::genKey(55, 50), RecordKVFormat::genKey(55, 100)); + auto ingest_ids = kvs.preHandleSnapshotToFiles( + region, + {}, + 10, + 5, + ctx.getTMTContext()); + kvs.checkAndApplySnapshot(RegionPtrWithSnapshotFiles{region, std::move(ingest_ids)}, ctx.getTMTContext()); // overlap, tombstone, remove previous one + + auto state = proxy_helper.getRegionLocalState(8192); + ASSERT_EQ(state.state(), raft_serverpb::PeerState::Tombstone); + } + + kvs.handleDestroy(20, ctx.getTMTContext()); } } + { auto region_id = 19; auto region_id_str = std::to_string(19); @@ -1125,11 +1154,6 @@ void RegionKVStoreTest::testKVStore() RegionMockTest mock_test(ctx.getTMTContext().getKVStore(), region); { - auto ori_snapshot_apply_method = kvs.snapshot_apply_method; - kvs.snapshot_apply_method = TiDB::SnapshotApplyMethod::Block; - SCOPE_EXIT({ - kvs.snapshot_apply_method = ori_snapshot_apply_method; - }); // Mocking ingest a SST for column family "Write" std::vector sst_views; sst_views.push_back(SSTView{ @@ -1142,9 +1166,10 @@ void RegionKVStoreTest::testKVStore() 100, 1, ctx.getTMTContext()); - ASSERT_EQ(kvs.getRegion(19)->dataInfo(), "[default 2 ]"); + ASSERT_EQ(kvs.getRegion(19)->checkIndex(100), true); } } + { raft_cmdpb::AdminRequest request; raft_cmdpb::AdminResponse response; @@ -1154,7 +1179,7 @@ void RegionKVStoreTest::testKVStore() try { - kvs.handleAdminRaftCmd(std::move(request), std::move(response), 19, 110, 6, ctx.getTMTContext()); + kvs.handleAdminRaftCmd(std::move(request), std::move(response), 1, 110, 6, ctx.getTMTContext()); ASSERT_TRUE(false); } catch (Exception & e) diff --git a/etc/config-template.toml b/etc/config-template.toml index cad45dc8105..f56a6a095d4 100644 --- a/etc/config-template.toml +++ b/etc/config-template.toml @@ -130,10 +130,6 @@ # pd_addr = "pd0:2379" # specify which storage engine we use. tmt or dt TODO: Remove deprecated tmt engine # storage_engine = "dt" -[raft.snapshot] -# The way to apply snapshot data -# The value is one of "block" / "file1" -# method = "file1" [status] # The port through which Prometheus pulls metrics information. From a0ed1a650cdc7cf600d37a59668aa59e05d3b7db Mon Sep 17 00:00:00 2001 From: hehechen Date: Tue, 31 May 2022 23:42:27 +0800 Subject: [PATCH 012/104] Add mix mode UT (#5012) close pingcap/tiflash#5028 --- .../Page/V3/tests/gtest_page_storage.cpp | 33 ++ .../V3/tests/gtest_page_storage_mix_mode.cpp | 312 ++++++++++++++++++ 2 files changed, 345 insertions(+) diff --git a/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp b/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp index ce2ba0adaf4..f7ba33c46c8 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp @@ -1408,6 +1408,39 @@ try } CATCH +TEST_F(PageStorageTest, TruncateBlobFile) +try +{ + const size_t buf_sz = 1024; + char c_buff[buf_sz]; + + for (size_t i = 0; i < buf_sz; ++i) + { + c_buff[i] = i % 0xff; + } + + { + WriteBatch batch; + batch.putPage(1, 0, std::make_shared(c_buff, buf_sz), buf_sz, {}); + page_storage->write(std::move(batch)); + } + + auto blob_file = Poco::File(getTemporaryPath() + "/blobfile_1"); + + page_storage = reopenWithConfig(config); + EXPECT_GT(blob_file.getSize(), 0); + + { + WriteBatch batch; + batch.delPage(1); + page_storage->write(std::move(batch)); + } + page_storage = reopenWithConfig(config); + page_storage->gc(/*not_skip*/ false, nullptr, nullptr); + EXPECT_EQ(blob_file.getSize(), 0); +} +CATCH + } // namespace PS::V3::tests } // namespace DB diff --git a/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp b/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp index 97eac841018..5517539b898 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp @@ -490,6 +490,318 @@ try } CATCH +// v2 put 1, v2 ref 2->1, get snapshot s1, v3 del 1, read s1 +TEST_F(PageStorageMixedTest, RefWithSnapshot) +try +{ + const size_t buf_sz = 1024; + char c_buff[buf_sz] = {0}; + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, 0, buff, buf_sz); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + batch.putRefPage(2, 1); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::MIX_MODE); + ASSERT_EQ(page_reader_mix->getNormalPageId(2), 1); + } + + auto snapshot_mix_mode = page_reader_mix->getSnapshot("ReadWithSnapshotAfterDelOrigin"); + { + auto page_reader_mix_with_snap = storage_pool_mix->newLogReader(nullptr, snapshot_mix_mode); + const auto & page1 = page_reader_mix_with_snap.read(1); + ASSERT_PAGE_EQ(c_buff, buf_sz, page1, 1); + } + + { + WriteBatch batch; + batch.delPage(1); + page_writer_mix->write(std::move(batch), nullptr); + } + + { + auto page_reader_mix_with_snap = storage_pool_mix->newLogReader(nullptr, snapshot_mix_mode); + const auto & page1 = page_reader_mix_with_snap.read(1); + ASSERT_PAGE_EQ(c_buff, buf_sz, page1, 1); + } +} +CATCH + +// v2 put 1, v2 ref 2->1, get snapshot s1, v3 del 1, v3 del 2, read s1 +TEST_F(PageStorageMixedTest, RefWithDelSnapshot) +try +{ + const size_t buf_sz = 1024; + char c_buff[buf_sz] = {0}; + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, 0, buff, buf_sz); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + batch.putRefPage(2, 1); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::MIX_MODE); + ASSERT_EQ(page_reader_mix->getNormalPageId(2), 1); + } + + auto snapshot_mix_mode = page_reader_mix->getSnapshot("ReadWithSnapshotAfterDelOrigin"); + { + auto page_reader_mix_with_snap = storage_pool_mix->newLogReader(nullptr, snapshot_mix_mode); + const auto & page1 = page_reader_mix_with_snap.read(1); + ASSERT_PAGE_EQ(c_buff, buf_sz, page1, 1); + } + + { + WriteBatch batch; + batch.delPage(1); + batch.delPage(2); + page_writer_mix->write(std::move(batch), nullptr); + } + + { + auto page_reader_mix_with_snap = storage_pool_mix->newLogReader(nullptr, snapshot_mix_mode); + const auto & page1 = page_reader_mix_with_snap.read(1); + ASSERT_PAGE_EQ(c_buff, buf_sz, page1, 1); + } +} +CATCH + +// v2 put 1, v2 ref 2->1, v3 del 1, get snapshot s1, v3 del 2, use s1 read 2 +TEST_F(PageStorageMixedTest, RefWithDelSnapshot2) +try +{ + const size_t buf_sz = 1024; + char c_buff[buf_sz] = {0}; + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, 0, buff, buf_sz); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + batch.putRefPage(2, 1); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::MIX_MODE); + ASSERT_EQ(page_reader_mix->getNormalPageId(2), 1); + } + + { + WriteBatch batch; + batch.delPage(1); + page_writer_mix->write(std::move(batch), nullptr); + } + + auto snapshot_mix_mode = page_reader_mix->getSnapshot("ReadWithSnapshotAfterDelOrigin"); + { + auto page_reader_mix_with_snap = storage_pool_mix->newLogReader(nullptr, snapshot_mix_mode); + const auto & page1 = page_reader_mix_with_snap.read(2); + ASSERT_PAGE_EQ(c_buff, buf_sz, page1, 2); + } + + { + WriteBatch batch; + batch.delPage(2); + page_writer_mix->write(std::move(batch), nullptr); + } + + { + auto page_reader_mix_with_snap = storage_pool_mix->newLogReader(nullptr, snapshot_mix_mode); + const auto & page1 = page_reader_mix_with_snap.read(2); + ASSERT_PAGE_EQ(c_buff, buf_sz, page1, 2); + } +} +CATCH + +// v2 put 1, v2 del 1, v3 put 2, v3 del 2 +TEST_F(PageStorageMixedTest, GetMaxId) +try +{ + const size_t buf_sz = 1024; + char c_buff[buf_sz] = {0}; + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, 0, buff, buf_sz); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + batch.delPage(1); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(2, 0, buff, buf_sz); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::MIX_MODE); + ASSERT_EQ(storage_pool_mix->newLogPageId(), 3); + } + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(3, 0, buff, buf_sz); + page_writer_mix->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + batch.delPage(3); + page_writer_mix->write(std::move(batch), nullptr); + } + + { + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::MIX_MODE); + ASSERT_EQ(storage_pool_mix->newLogPageId(), 4); + } +} +CATCH + + +TEST_F(PageStorageMixedTest, ReuseV2ID) +try +{ + const size_t buf_sz = 1024; + char c_buff[buf_sz] = {0}; + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, 0, buff, buf_sz); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + batch.delPage(1); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::ONLY_V3); + ASSERT_EQ(storage_pool_mix->newLogPageId(), 1); + } + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, 0, buff, buf_sz); + page_writer_mix->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + batch.delPage(1); + page_writer_mix->write(std::move(batch), nullptr); + } + + { + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::ONLY_V3); + ASSERT_EQ(storage_pool_mix->newLogPageId(), 2); + } +} +CATCH + +// v2 put 1, v3 ref 2->1, reload, check max id, get snapshot s1, v3 del 1, get snapshot s2, v3 del 2, get snapshot s3, check snapshots +TEST_F(PageStorageMixedTest, V3RefV2WithSnapshot) +try +{ + const size_t buf_sz = 1024; + char c_buff[buf_sz] = {0}; + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, 0, buff, buf_sz); + page_writer_v2->write(std::move(batch), nullptr); + } + + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::MIX_MODE); + + { + WriteBatch batch; + batch.putRefPage(2, 1); + page_writer_mix->write(std::move(batch), nullptr); + } + + { + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::MIX_MODE); + ASSERT_EQ(page_reader_mix->getNormalPageId(2), 1); + ASSERT_EQ(storage_pool_mix->newLogPageId(), 3); + } + + auto snapshot_before_del = page_reader_mix->getSnapshot("ReadWithSnapshotBeforeDelOrigin"); + + { + WriteBatch batch; + batch.delPage(1); + page_writer_mix->write(std::move(batch), nullptr); + } + + auto snapshot_after_del_origin = page_reader_mix->getSnapshot("ReadWithSnapshotAfterDelOrigin"); + + { + auto page_reader_mix_with_snap = storage_pool_mix->newLogReader(nullptr, snapshot_before_del); + const auto & page1 = page_reader_mix_with_snap.read(1); + ASSERT_PAGE_EQ(c_buff, buf_sz, page1, 1); + } + + { + WriteBatch batch; + batch.delPage(2); + page_writer_mix->write(std::move(batch), nullptr); + } + + auto snapshot_after_del_all = page_reader_mix->getSnapshot("ReadWithSnapshotAfterDelAll"); + + { + auto page_reader_mix_with_snap = storage_pool_mix->newLogReader(nullptr, snapshot_after_del_origin); + const auto & page1 = page_reader_mix_with_snap.read(2); + ASSERT_PAGE_EQ(c_buff, buf_sz, page1, 2); + } + + { + auto page_reader_mix_with_snap = storage_pool_mix->newLogReader(nullptr, snapshot_after_del_origin); + const auto & page1 = page_reader_mix_with_snap.read(2); + ASSERT_PAGE_EQ(c_buff, buf_sz, page1, 2); + } + + { + auto page_reader_mix_with_snap = storage_pool_mix->newLogReader(nullptr, snapshot_after_del_all); + ASSERT_ANY_THROW(page_reader_mix_with_snap.read(2)); + } +} +CATCH TEST_F(PageStorageMixedTest, MockDTIngest) try From 800715c798c6d56b6bc26e1ff83dff6a5eb6d0dd Mon Sep 17 00:00:00 2001 From: lidezhu <47731263+lidezhu@users.noreply.github.com> Date: Wed, 1 Jun 2022 11:52:26 +0800 Subject: [PATCH 013/104] support mix_mode in dt_workload (#5011) ref pingcap/tiflash#3594 --- .../DeltaMerge/tools/workload/DTWorkload.cpp | 2 +- .../DeltaMerge/tools/workload/MainEntry.cpp | 29 ++++++++++++++----- .../DeltaMerge/tools/workload/Options.cpp | 20 ++++++++++--- .../DeltaMerge/tools/workload/Options.h | 4 ++- dbms/src/TestUtils/TiFlashTestEnv.cpp | 4 +-- dbms/src/TestUtils/TiFlashTestEnv.h | 3 +- 6 files changed, 45 insertions(+), 17 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.cpp b/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.cpp index e5c7fd30f40..a6113f91d91 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.cpp +++ b/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.cpp @@ -233,7 +233,7 @@ void DTWorkload::verifyHandle(uint64_t r) } for (size_t i = 0; i < handle_col->size(); i++) { - // Handle must be int64 or uint64. Currently, TableGenterator would ensure this limit. + // Handle must be int64 or uint64. Currently, TableGenerator would ensure this limit. uint64_t h = handle_col->getInt(i); uint64_t store_ts = ts_col->getInt(i); diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/MainEntry.cpp b/dbms/src/Storages/DeltaMerge/tools/workload/MainEntry.cpp index e18a6ef30a2..f79d414f20b 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/MainEntry.cpp +++ b/dbms/src/Storages/DeltaMerge/tools/workload/MainEntry.cpp @@ -124,13 +124,26 @@ void run(WorkloadOptions & opts) // Table Schema auto table_gen = TableGenerator::create(opts); auto table_info = table_gen->get(opts.table_id, opts.table_name); - // In this for loop, destory DeltaMergeStore gracefully and recreate it. - for (uint64_t i = 0; i < opts.verify_round; i++) + // In this for loop, destroy DeltaMergeStore gracefully and recreate it. + auto run_test = [&]() { + for (uint64_t i = 0; i < opts.verify_round; i++) + { + DTWorkload workload(opts, handle_table, table_info); + workload.run(i); + stats.push_back(workload.getStat()); + LOG_FMT_INFO(log, "No.{} Workload {} {}", i, opts.write_key_distribution, stats.back().toStrings()); + } + }; + run_test(); + + if (opts.ps_run_mode == DB::PageStorageRunMode::MIX_MODE) { - DTWorkload workload(opts, handle_table, table_info); - workload.run(i); - stats.push_back(workload.getStat()); - LOG_FMT_INFO(log, "No.{} Workload {} {}", i, opts.write_key_distribution, stats.back().toStrings()); + // clear statistic in DB::PageStorageRunMode::ONLY_V2 + stats.clear(); + auto & global_context = TiFlashTestEnv::getGlobalContext(); + global_context.setPageStorageRunMode(DB::PageStorageRunMode::MIX_MODE); + global_context.initializeGlobalStoragePoolIfNeed(global_context.getPathPool()); + run_test(); } } catch (...) @@ -254,8 +267,9 @@ int DTWorkload::mainEntry(int argc, char ** argv) // or the logging in global context won't be output to // the log file init(opts); - TiFlashTestEnv::initializeGlobalContext(opts.work_dirs, opts.enable_ps_v3); + // For mixed mode, we need to run the test in ONLY_V2 mode first. + TiFlashTestEnv::initializeGlobalContext(opts.work_dirs, opts.ps_run_mode == PageStorageRunMode::ONLY_V3 ? PageStorageRunMode::ONLY_V3 : PageStorageRunMode::ONLY_V2); if (opts.testing_type == "daily_perf") { dailyPerformanceTest(opts); @@ -277,7 +291,6 @@ int DTWorkload::mainEntry(int argc, char ** argv) runAndRandomKill(opts); } } - TiFlashTestEnv::shutdown(); return 0; } diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/Options.cpp b/dbms/src/Storages/DeltaMerge/tools/workload/Options.cpp index 0d2b14d916b..1c6409f3c53 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/Options.cpp +++ b/dbms/src/Storages/DeltaMerge/tools/workload/Options.cpp @@ -44,7 +44,7 @@ std::string WorkloadOptions::toString(std::string seperator) const fmt::format("read_stream_count {}{}", read_stream_count, seperator) + // fmt::format("testing_type {}{}", testing_type, seperator) + // fmt::format("log_write_request {}{}", log_write_request, seperator) + // - fmt::format("enable_ps_v3 {}{}", enable_ps_v3, seperator) + // + fmt::format("ps_run_mode {}{}", ps_run_mode, seperator) + // fmt::format("bg_thread_count {}{}", bg_thread_count, seperator) + // fmt::format("table_id {}{}", table_id, seperator) + // fmt::format("table_name {}{}", table_name, seperator); @@ -88,7 +88,7 @@ std::pair WorkloadOptions::parseOptions(int argc, char * argv // ("log_write_request", value()->default_value(false), "") // // - ("enable_ps_v3", value()->default_value(true), "") // + ("ps_run_mode", value()->default_value(2, "possible value: 1(only_v2), 2(only_v3), 3(mix_mode), and note that in mix_mode, the test will run twice, first round in only_v2 mode and second round in mix_mode")) // // ("bg_thread_count", value()->default_value(4), "") // // @@ -155,8 +155,20 @@ std::pair WorkloadOptions::parseOptions(int argc, char * argv testing_type = vm["testing_type"].as(); log_write_request = vm["log_write_request"].as(); - - enable_ps_v3 = vm["enable_ps_v3"].as(); + switch (vm["ps_run_mode"].as()) + { + case static_cast(PageStorageRunMode::ONLY_V2): + ps_run_mode = PageStorageRunMode::ONLY_V2; + break; + case static_cast(PageStorageRunMode::ONLY_V3): + ps_run_mode = PageStorageRunMode::ONLY_V3; + break; + case static_cast(PageStorageRunMode::MIX_MODE): + ps_run_mode = PageStorageRunMode::MIX_MODE; + break; + default: + return {false, fmt::format("unknown ps_run_mode {}.", vm["ps_run_mode"].as())}; + } bg_thread_count = vm["bg_thread_count"].as(); diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/Options.h b/dbms/src/Storages/DeltaMerge/tools/workload/Options.h index 17c7a5ba61f..f017daf2d8a 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/Options.h +++ b/dbms/src/Storages/DeltaMerge/tools/workload/Options.h @@ -14,6 +14,8 @@ #pragma once +#include + #include #include @@ -53,7 +55,7 @@ struct WorkloadOptions bool log_write_request; - bool enable_ps_v3; + PageStorageRunMode ps_run_mode; uint64_t bg_thread_count; diff --git a/dbms/src/TestUtils/TiFlashTestEnv.cpp b/dbms/src/TestUtils/TiFlashTestEnv.cpp index bd05e5826db..cbd42b57550 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.cpp +++ b/dbms/src/TestUtils/TiFlashTestEnv.cpp @@ -28,7 +28,7 @@ namespace DB::tests { std::unique_ptr TiFlashTestEnv::global_context = nullptr; -void TiFlashTestEnv::initializeGlobalContext(Strings testdata_path, bool enable_ps_v3) +void TiFlashTestEnv::initializeGlobalContext(Strings testdata_path, PageStorageRunMode ps_run_mode) { // set itself as global context global_context = std::make_unique(DB::Context::createGlobal()); @@ -68,7 +68,7 @@ void TiFlashTestEnv::initializeGlobalContext(Strings testdata_path, bool enable_ global_context->getPathCapacity(), global_context->getFileProvider()); - global_context->setPageStorageRunMode(enable_ps_v3 ? PageStorageRunMode::ONLY_V3 : PageStorageRunMode::ONLY_V2); + global_context->setPageStorageRunMode(ps_run_mode); global_context->initializeGlobalStoragePoolIfNeed(global_context->getPathPool()); LOG_FMT_INFO(Logger::get("TiFlashTestEnv"), "Storage mode : {}", static_cast(global_context->getPageStorageRunMode())); diff --git a/dbms/src/TestUtils/TiFlashTestEnv.h b/dbms/src/TestUtils/TiFlashTestEnv.h index 0264d87ef9f..dafecf6e1de 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.h +++ b/dbms/src/TestUtils/TiFlashTestEnv.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include @@ -88,7 +89,7 @@ class TiFlashTestEnv static Context getContext(const DB::Settings & settings = DB::Settings(), Strings testdata_path = {}); - static void initializeGlobalContext(Strings testdata_path = {}, bool enable_ps_v3 = true); + static void initializeGlobalContext(Strings testdata_path = {}, PageStorageRunMode ps_run_mode = PageStorageRunMode::ONLY_V3); static Context & getGlobalContext() { return *global_context; } static void shutdown(); From 380f6cd560247d2a53b39200a743a36c893ac91f Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 1 Jun 2022 19:50:27 +0800 Subject: [PATCH 014/104] enable new function test framework by default (#5029) ref pingcap/tiflash#4830 --- .../tests/gtest_arithmetic_functions.cpp | 30 ++-- dbms/src/Functions/tests/gtest_bitand.cpp | 148 +++++++++--------- dbms/src/Functions/tests/gtest_bitnot.cpp | 30 ++-- dbms/src/Functions/tests/gtest_bitor.cpp | 148 +++++++++--------- dbms/src/Functions/tests/gtest_bitxor.cpp | 148 +++++++++--------- dbms/src/Functions/tests/gtest_date_add.cpp | 32 ++-- .../gtest_date_or_datetime_to_something.cpp | 2 +- dbms/src/Functions/tests/gtest_date_sub.cpp | 32 ++-- .../tests/gtest_datetime_daymonthyear.cpp | 6 +- .../tests/gtest_duration_pushdown.cpp | 12 +- .../tests/gtest_functions_round_with_frac.cpp | 15 +- dbms/src/Functions/tests/gtest_ifnull.cpp | 23 +-- .../Functions/tests/gtest_inet_aton_ntoa.cpp | 2 +- .../Functions/tests/gtest_is_true_false.cpp | 8 +- .../Functions/tests/gtest_least_greatest.cpp | 4 +- dbms/src/Functions/tests/gtest_logical.cpp | 8 +- dbms/src/Functions/tests/gtest_regexp.cpp | 62 ++++---- .../src/Functions/tests/gtest_string_left.cpp | 13 +- .../Functions/tests/gtest_string_lrtrim.cpp | 20 +-- .../src/Functions/tests/gtest_strings_cmp.cpp | 18 +-- .../Functions/tests/gtest_strings_format.cpp | 12 +- .../Functions/tests/gtest_strings_right.cpp | 3 +- .../Functions/tests/gtest_strings_search.cpp | 97 +++++++----- .../tests/gtest_strings_tidb_concat.cpp | 5 +- .../Functions/tests/gtest_strings_trim.cpp | 14 +- dbms/src/Functions/tests/gtest_substring.cpp | 2 +- .../Functions/tests/gtest_substring_index.cpp | 4 +- .../Functions/tests/gtest_unix_timestamp.cpp | 8 +- dbms/src/TestUtils/FunctionTestUtils.h | 10 +- 29 files changed, 465 insertions(+), 451 deletions(-) diff --git a/dbms/src/Functions/tests/gtest_arithmetic_functions.cpp b/dbms/src/Functions/tests/gtest_arithmetic_functions.cpp index 1a542c908ee..1d548a4c2d2 100644 --- a/dbms/src/Functions/tests/gtest_arithmetic_functions.cpp +++ b/dbms/src/Functions/tests/gtest_arithmetic_functions.cpp @@ -128,6 +128,10 @@ try null_or_zero_field.push_back(Field(DecimalField(0, 0))); std::vector values{10, 2, 20, 8, 10, 0, 30, 8, 16, 4}; + /// The precision of a non-zero DecimalField must not less than minDecimalPrecision + /// the decimal_128_factor is used to make sure the precision is big enough + Int128 decimal_128_factor = 10000000000; + decimal_128_factor *= 1000000000; const size_t size = 10; @@ -172,7 +176,7 @@ try if (col2_type->onlyNull()) continue; auto c1 = nullable_decimal_type_1->createColumnConst(size, Null()); - auto c2 = col2_type->createColumnConst(size, Field(DecimalField(2, 0))); + auto c2 = col2_type->createColumnConst(size, Field(DecimalField(2 * decimal_128_factor, 0))); auto col1 = ColumnWithTypeAndName(std::move(c1), nullable_decimal_type_1, "col1"); auto col2 = ColumnWithTypeAndName(std::move(c2), col2_type, "col2"); auto result = executeFunction(func_name, {col1, col2}); @@ -193,7 +197,7 @@ try continue; if (!col2_value.isNull() && col2_type->onlyNull()) continue; - auto c1 = col1_type->createColumnConst(size, Field(DecimalField(100, 2))); + auto c1 = col1_type->createColumnConst(size, Field(DecimalField(100 * decimal_128_factor, 2))); auto c2 = col2_type->createColumnConst(size, col2_value); auto col1 = ColumnWithTypeAndName(std::move(c1), col1_type, "col1"); auto col2 = ColumnWithTypeAndName(std::move(c2), col2_type, "col2"); @@ -211,11 +215,11 @@ try { if (col1_type->onlyNull() || col2_type->onlyNull()) continue; - auto c1 = col1_type->createColumnConst(size, Field(DecimalField(1000, 2))); - auto c2 = col2_type->createColumnConst(size, Field(DecimalField(2, 0))); + auto c1 = col1_type->createColumnConst(size, Field(DecimalField(1000 * decimal_128_factor, 2))); + auto c2 = col2_type->createColumnConst(size, Field(DecimalField(2 * decimal_128_factor, 0))); auto col1 = ColumnWithTypeAndName(std::move(c1), col1_type, "col1"); auto col2 = ColumnWithTypeAndName(std::move(c2), col2_type, "col2"); - auto res_col = executeFunction(func_name, {col1, col2}).column; + auto res_col = executeFunction(func_name, {col1, col2}, nullptr, false).column; ASSERT_TRUE(size == res_col->size()); Field res_field; for (size_t i = 0; i < size; i++) @@ -246,7 +250,7 @@ try if (col1_type->isNullable() && col1_null_map[i]) c1_mutable->insert(Null()); else - c1_mutable->insert(Field(DecimalField(values[i], 2))); + c1_mutable->insert(Field(DecimalField(values[i] * decimal_128_factor, 2))); } auto c2 = col2_type->createColumnConst(values.size(), col2_value); @@ -271,9 +275,9 @@ try if (col1_type->isNullable() && col1_null_map[i]) c1_mutable->insert(Null()); else - c1_mutable->insert(Field(DecimalField(values[i], 2))); + c1_mutable->insert(Field(DecimalField(values[i] * decimal_128_factor, 2))); } - auto c2 = col2_type->createColumnConst(values.size(), Field(DecimalField(2, 0))); + auto c2 = col2_type->createColumnConst(values.size(), Field(DecimalField(2 * decimal_128_factor, 0))); auto col1 = ColumnWithTypeAndName(std::move(c1_mutable), col1_type, "col1"); auto col2 = ColumnWithTypeAndName(std::move(c2), col2_type, "col2"); @@ -312,7 +316,7 @@ try if (col2_type->isNullable() && col2_null_map[i]) c2->insert(Null()); else - c2->insert(Field(DecimalField(values[i], 2))); + c2->insert(Field(DecimalField(values[i] * decimal_128_factor, 2))); } auto col1 = ColumnWithTypeAndName(std::move(c1), col1_type, "col1"); auto col2 = ColumnWithTypeAndName(std::move(c2), col2_type, "col2"); @@ -334,14 +338,14 @@ try if (values[i] != 0) value *= values[i]; } - auto c1 = col1_type->createColumnConst(size, Field(DecimalField(value, 2))); + auto c1 = col1_type->createColumnConst(size, Field(DecimalField(value * decimal_128_factor, 2))); auto c2 = col2_type->createColumn(); for (size_t i = 0; i < values.size(); i++) { if (col2_type->isNullable() && col2_null_map[i]) c2->insert(Null()); else - c2->insert(Field(DecimalField(values[i], 0))); + c2->insert(Field(DecimalField(values[i] * decimal_128_factor, 0))); } auto col1 = ColumnWithTypeAndName(std::move(c1), col1_type, "col1"); auto col2 = ColumnWithTypeAndName(std::move(c2), col2_type, "col2"); @@ -377,11 +381,11 @@ try if (col1_type->isNullable() && col1_null_map[i]) c1->insert(Null()); else - c1->insert(Field(DecimalField(values[i], 2))); + c1->insert(Field(DecimalField(values[i] * decimal_128_factor, 2))); if (col2_type->isNullable() && col2_null_map[i]) c2->insert(Null()); else - c2->insert(Field(DecimalField(values[i], 0))); + c2->insert(Field(DecimalField(values[i] * decimal_128_factor, 0))); } auto col1 = ColumnWithTypeAndName(std::move(c1), col1_type, "col1"); auto col2 = ColumnWithTypeAndName(std::move(c2), col2_type, "col2"); diff --git a/dbms/src/Functions/tests/gtest_bitand.cpp b/dbms/src/Functions/tests/gtest_bitand.cpp index b77bc5e8547..88c70847d98 100644 --- a/dbms/src/Functions/tests/gtest_bitand.cpp +++ b/dbms/src/Functions/tests/gtest_bitand.cpp @@ -40,7 +40,7 @@ class TestFunctionBitAnd : public DB::tests::FunctionTest TEST_F(TestFunctionBitAnd, Simple) try { - ASSERT_BITAND(createColumn>({-1, 1}), createColumn>({0, 0}), createColumn>({0, 0})); + ASSERT_BITAND(createColumn>({-1, 1}), createColumn>({0, 0}), createColumn>({0, 0})); } CATCH @@ -49,21 +49,21 @@ TEST_F(TestFunctionBitAnd, TypePromotion) try { // Type Promotion - ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); // Type Promotion across signed/unsigned - ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn>({0}), createColumn>({0})); } CATCH @@ -71,51 +71,51 @@ TEST_F(TestFunctionBitAnd, Nullable) try { // Non Nullable - ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); - ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); - ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); - ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); + ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); + ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); + ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); + ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); - ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); - ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); + ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); + ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); - ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); - ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); - ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); - ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); + ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); + ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); + ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); + ASSERT_BITAND(createColumn({1}), createColumn({0}), createColumn({0})); // Across Nullable and non-Nullable - ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); + ASSERT_BITAND(createColumn({1}), createColumn>({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); - ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); + ASSERT_BITAND(createColumn>({1}), createColumn({0}), createColumn>({0})); } CATCH @@ -129,48 +129,48 @@ try /// 4. ColumnConst, value != null /// 5. ColumnConst, value = null - ASSERT_BITAND(createColumn({0, 0, 1, 1}), createColumn({0, 1, 0, 1}), createColumn({0, 0, 0, 1})); - ASSERT_BITAND(createColumn({0, 0, 1, 1}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 0, std::nullopt, std::nullopt})); - ASSERT_BITAND(createColumn({0, 0, 1, 1}), createConstColumn(4, 0), createColumn({0, 0, 0, 0})); - ASSERT_BITAND(createColumn({0, 0, 1, 1}), createConstColumn>(4, 0), createColumn>({0, 0, 0, 0})); - ASSERT_BITAND(createColumn({0, 0, 1, 1}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); // become const in wrapInNullable - - ASSERT_BITAND(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn({0, 1, 0, 1}), createColumn>({0, 1, std::nullopt, std::nullopt})); - ASSERT_BITAND(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); - ASSERT_BITAND(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn(4, 0), createColumn>({0, 0, std::nullopt, std::nullopt})); - ASSERT_BITAND(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, 0), createColumn>({0, 0, std::nullopt, std::nullopt})); - ASSERT_BITAND(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); - - ASSERT_BITAND(createConstColumn(4, 0), createColumn({0, 1, 0, 1}), createColumn({0, 0, 0, 0})); - ASSERT_BITAND(createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 0, std::nullopt, std::nullopt})); - ASSERT_BITAND(createConstColumn(4, 0), createConstColumn(4, 0), createConstColumn(4, 0)); - ASSERT_BITAND(createConstColumn(4, 0), createConstColumn>(4, 0), createConstColumn>(4, 0)); - ASSERT_BITAND(createConstColumn(4, 0), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); - - ASSERT_BITAND(createConstColumn>(4, 0), createColumn({0, 1, 0, 1}), createColumn>({0, 0, 0, 0})); - ASSERT_BITAND(createConstColumn>(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 0, std::nullopt, std::nullopt})); - ASSERT_BITAND(createConstColumn>(4, 0), createConstColumn(4, 0), createConstColumn>(4, 0)); - ASSERT_BITAND(createConstColumn>(4, 0), createConstColumn>(4, 0), createConstColumn>(4, 0)); - ASSERT_BITAND(createConstColumn>(4, 0), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); - - ASSERT_BITAND(createConstColumn>(4, std::nullopt), createColumn({0, 1, 0, 1}), createConstColumn>(4, std::nullopt)); - ASSERT_BITAND(createConstColumn>(4, std::nullopt), createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt)); - ASSERT_BITAND(createConstColumn>(4, std::nullopt), createConstColumn(4, 0), createConstColumn>(4, std::nullopt)); - ASSERT_BITAND(createConstColumn>(4, std::nullopt), createConstColumn>(4, 0), createConstColumn>(4, std::nullopt)); - ASSERT_BITAND(createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + ASSERT_BITAND(createColumn({0, 0, 1, 1}), createColumn({0, 1, 0, 1}), createColumn({0, 0, 0, 1})); + ASSERT_BITAND(createColumn({0, 0, 1, 1}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 0, std::nullopt, std::nullopt})); + ASSERT_BITAND(createColumn({0, 0, 1, 1}), createConstColumn(4, 0), createColumn({0, 0, 0, 0})); + ASSERT_BITAND(createColumn({0, 0, 1, 1}), createConstColumn>(4, 0), createColumn({0, 0, 0, 0})); + ASSERT_BITAND(createColumn({0, 0, 1, 1}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); // become const in wrapInNullable + + ASSERT_BITAND(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn({0, 1, 0, 1}), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITAND(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITAND(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn(4, 0), createColumn>({0, 0, std::nullopt, std::nullopt})); + ASSERT_BITAND(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, 0), createColumn>({0, 0, std::nullopt, std::nullopt})); + ASSERT_BITAND(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + + ASSERT_BITAND(createConstColumn(4, 0), createColumn({0, 1, 0, 1}), createColumn({0, 0, 0, 0})); + ASSERT_BITAND(createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 0, std::nullopt, std::nullopt})); + ASSERT_BITAND(createConstColumn(4, 0), createConstColumn(4, 0), createConstColumn(4, 0)); + ASSERT_BITAND(createConstColumn(4, 0), createConstColumn>(4, 0), createConstColumn(4, 0)); + ASSERT_BITAND(createConstColumn(4, 0), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + + ASSERT_BITAND(createConstColumn>(4, 0), createColumn({0, 1, 0, 1}), createColumn({0, 0, 0, 0})); + ASSERT_BITAND(createConstColumn>(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 0, std::nullopt, std::nullopt})); + ASSERT_BITAND(createConstColumn>(4, 0), createConstColumn(4, 0), createConstColumn(4, 0)); + ASSERT_BITAND(createConstColumn>(4, 0), createConstColumn>(4, 0), createConstColumn(4, 0)); + ASSERT_BITAND(createConstColumn>(4, 0), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + + ASSERT_BITAND(createConstColumn>(4, std::nullopt), createColumn({0, 1, 0, 1}), createConstColumn>(4, std::nullopt)); + ASSERT_BITAND(createConstColumn>(4, std::nullopt), createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt)); + ASSERT_BITAND(createConstColumn>(4, std::nullopt), createConstColumn(4, 0), createConstColumn>(4, std::nullopt)); + ASSERT_BITAND(createConstColumn>(4, std::nullopt), createConstColumn(4, 0), createConstColumn>(4, std::nullopt)); + ASSERT_BITAND(createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); } CATCH TEST_F(TestFunctionBitAnd, Boundary) try { - ASSERT_BITAND(createColumn({127, 127, -128, -128}), createColumn({0, 255, 0, 255}), createColumn({0, 127, 0, -128})); - ASSERT_BITAND(createColumn({127, 127, -128, -128}), createColumn({0, 65535, 0, 65535}), createColumn({0, 127, 0, -128})); - ASSERT_BITAND(createColumn({32767, 32767, -32768, -32768}), createColumn({0, 255, 0, 255}), createColumn({0, 255, 0, 0})); + ASSERT_BITAND(createColumn({127, 127, -128, -128}), createColumn({0, 255, 0, 255}), createColumn({0, 127, 0, 128})); + ASSERT_BITAND(createColumn({127, 127, -128, -128}), createColumn({0, 65535, 0, 65535}), createColumn({0, 127, 0, 65408})); + ASSERT_BITAND(createColumn({32767, 32767, -32768, -32768}), createColumn({0, 255, 0, 255}), createColumn({0, 255, 0, 0})); ASSERT_BITAND(createColumn({0, 0, 1, 1, -1, -1, INT64_MAX, INT64_MAX, INT64_MIN, INT64_MIN}), createColumn({0, UINT64_MAX, 0, UINT64_MAX, 0, UINT64_MAX, 0, UINT64_MAX, 0, UINT64_MAX}), - createColumn({0, 0, 0, 1, 0, -1, 0, INT64_MAX, 0, INT64_MIN})); + createColumn({0, 0, 0, 1, 0, UINT64_MAX, 0, INT64_MAX, 0, 9223372036854775808ull})); } CATCH diff --git a/dbms/src/Functions/tests/gtest_bitnot.cpp b/dbms/src/Functions/tests/gtest_bitnot.cpp index 6b96e59718e..0b542d87961 100644 --- a/dbms/src/Functions/tests/gtest_bitnot.cpp +++ b/dbms/src/Functions/tests/gtest_bitnot.cpp @@ -41,7 +41,7 @@ class TestFunctionBitNot : public DB::tests::FunctionTest TEST_F(TestFunctionBitNot, Simple) try { - ASSERT_BITNOT(createColumn>({-1, 1}), createColumn>({0, -2})); + ASSERT_BITNOT(createColumn>({-1, 1}), createColumn>({0, UINT64_MAX - 1})); } CATCH @@ -49,21 +49,21 @@ CATCH TEST_F(TestFunctionBitNot, TypeTest) try { - ASSERT_BITNOT(createColumn>({1}), createColumn>({-2})); - ASSERT_BITNOT(createColumn>({1}), createColumn>({-2})); - ASSERT_BITNOT(createColumn>({1}), createColumn>({-2})); - ASSERT_BITNOT(createColumn>({1}), createColumn>({-2})); - - ASSERT_BITNOT(createColumn>({1}), createColumn>({UINT8_MAX - 1})); - ASSERT_BITNOT(createColumn>({1}), createColumn>({UINT16_MAX - 1})); - ASSERT_BITNOT(createColumn>({1}), createColumn>({UINT32_MAX - 1})); + ASSERT_BITNOT(createColumn>({1}), createColumn>({UINT64_MAX - 1})); + ASSERT_BITNOT(createColumn>({1}), createColumn>({UINT64_MAX - 1})); + ASSERT_BITNOT(createColumn>({1}), createColumn>({UINT64_MAX - 1})); + ASSERT_BITNOT(createColumn>({1}), createColumn>({UINT64_MAX - 1})); + + ASSERT_BITNOT(createColumn>({1}), createColumn>({UINT64_MAX - 1})); + ASSERT_BITNOT(createColumn>({1}), createColumn>({UINT64_MAX - 1})); + ASSERT_BITNOT(createColumn>({1}), createColumn>({UINT64_MAX - 1})); ASSERT_BITNOT(createColumn>({1}), createColumn>({UINT64_MAX - 1})); - ASSERT_BITNOT(createColumn({0, 0, 1, 1}), createColumn({-1, -1, -2, -2})); - ASSERT_BITNOT(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({-1, -2, std::nullopt, std::nullopt})); - ASSERT_BITNOT(createConstColumn(4, 0), createConstColumn(4, -1)); - ASSERT_BITNOT(createConstColumn>(4, 0), createConstColumn>(4, -1)); - ASSERT_BITNOT(createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + ASSERT_BITNOT(createColumn({0, 0, 1, 1}), createColumn({UINT64_MAX, UINT64_MAX, UINT64_MAX - 1, UINT64_MAX - 1})); + ASSERT_BITNOT(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({UINT64_MAX, UINT64_MAX - 1, std::nullopt, std::nullopt})); + ASSERT_BITNOT(createConstColumn(4, 0), createConstColumn(4, UINT64_MAX)); + ASSERT_BITNOT(createConstColumn>(4, 0), createConstColumn(4, UINT64_MAX)); + ASSERT_BITNOT(createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); } CATCH @@ -71,7 +71,7 @@ TEST_F(TestFunctionBitNot, Boundary) try { ASSERT_BITNOT(createColumn({0, 1, -1, INT64_MAX, INT64_MIN}), - createColumn({-1, -2, 0, INT64_MIN, INT64_MAX})); + createColumn({UINT64_MAX, UINT64_MAX - 1, 0, static_cast(INT64_MAX) + 1, INT64_MAX})); ASSERT_BITNOT(createColumn({0, 1, UINT64_MAX}), createColumn({UINT64_MAX, UINT64_MAX - 1, 0})); diff --git a/dbms/src/Functions/tests/gtest_bitor.cpp b/dbms/src/Functions/tests/gtest_bitor.cpp index a1611659913..fb4794c31b2 100644 --- a/dbms/src/Functions/tests/gtest_bitor.cpp +++ b/dbms/src/Functions/tests/gtest_bitor.cpp @@ -40,7 +40,7 @@ class TestFunctionBitOr : public DB::tests::FunctionTest TEST_F(TestFunctionBitOr, Simple) try { - ASSERT_BITOR(createColumn>({-1, 1}), createColumn>({0, 0}), createColumn>({-1, 1})); + ASSERT_BITOR(createColumn>({-1, 1}), createColumn>({0, 0}), createColumn>({UINT64_MAX, 1})); } CATCH @@ -49,21 +49,21 @@ TEST_F(TestFunctionBitOr, TypePromotion) try { // Type Promotion - ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); // Type Promotion across signed/unsigned - ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); } CATCH @@ -71,51 +71,51 @@ TEST_F(TestFunctionBitOr, Nullable) try { // Non Nullable - ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITOR(createColumn({1}), createColumn({0}), createColumn({1})); // Across Nullable and non-Nullable - ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITOR(createColumn>({1}), createColumn({0}), createColumn>({1})); } CATCH @@ -129,48 +129,48 @@ try /// 4. ColumnConst, value != null /// 5. ColumnConst, value = null - ASSERT_BITOR(createColumn({0, 0, 1, 1}), createColumn({0, 1, 0, 1}), createColumn({0, 1, 1, 1})); - ASSERT_BITOR(createColumn({0, 0, 1, 1}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); - ASSERT_BITOR(createColumn({0, 0, 1, 1}), createConstColumn(4, 0), createColumn({0, 0, 1, 1})); - ASSERT_BITOR(createColumn({0, 0, 1, 1}), createConstColumn>(4, 0), createColumn>({0, 0, 1, 1})); - ASSERT_BITOR(createColumn({0, 0, 1, 1}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); // become const in wrapInNullable - - ASSERT_BITOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn({0, 1, 0, 1}), createColumn>({0, 1, std::nullopt, std::nullopt})); - ASSERT_BITOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); - ASSERT_BITOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt})); - ASSERT_BITOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt})); - ASSERT_BITOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); - - ASSERT_BITOR(createConstColumn(4, 0), createColumn({0, 1, 0, 1}), createColumn({0, 1, 0, 1})); - ASSERT_BITOR(createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); - ASSERT_BITOR(createConstColumn(4, 0), createConstColumn(4, 0), createConstColumn(4, 0)); - ASSERT_BITOR(createConstColumn(4, 0), createConstColumn>(4, 0), createConstColumn>(4, 0)); - ASSERT_BITOR(createConstColumn(4, 0), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); - - ASSERT_BITOR(createConstColumn>(4, 0), createColumn({0, 1, 0, 1}), createColumn>({0, 1, 0, 1})); - ASSERT_BITOR(createConstColumn>(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); - ASSERT_BITOR(createConstColumn>(4, 0), createConstColumn(4, 0), createConstColumn>(4, 0)); - ASSERT_BITOR(createConstColumn>(4, 0), createConstColumn>(4, 0), createConstColumn>(4, 0)); - ASSERT_BITOR(createConstColumn>(4, 0), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); - - ASSERT_BITOR(createConstColumn>(4, std::nullopt), createColumn({0, 1, 0, 1}), createConstColumn>(4, std::nullopt)); - ASSERT_BITOR(createConstColumn>(4, std::nullopt), createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt)); - ASSERT_BITOR(createConstColumn>(4, std::nullopt), createConstColumn(4, 0), createConstColumn>(4, std::nullopt)); - ASSERT_BITOR(createConstColumn>(4, std::nullopt), createConstColumn>(4, 0), createConstColumn>(4, std::nullopt)); - ASSERT_BITOR(createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + ASSERT_BITOR(createColumn({0, 0, 1, 1}), createColumn({0, 1, 0, 1}), createColumn({0, 1, 1, 1})); + ASSERT_BITOR(createColumn({0, 0, 1, 1}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITOR(createColumn({0, 0, 1, 1}), createConstColumn(4, 0), createColumn({0, 0, 1, 1})); + ASSERT_BITOR(createColumn({0, 0, 1, 1}), createConstColumn>(4, 0), createColumn({0, 0, 1, 1})); + ASSERT_BITOR(createColumn({0, 0, 1, 1}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); // become const in wrapInNullable + + ASSERT_BITOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn({0, 1, 0, 1}), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + + ASSERT_BITOR(createConstColumn(4, 0), createColumn({0, 1, 0, 1}), createColumn({0, 1, 0, 1})); + ASSERT_BITOR(createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITOR(createConstColumn(4, 0), createConstColumn(4, 0), createConstColumn(4, 0)); + ASSERT_BITOR(createConstColumn(4, 0), createConstColumn>(4, 0), createConstColumn(4, 0)); + ASSERT_BITOR(createConstColumn(4, 0), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + + ASSERT_BITOR(createConstColumn>(4, 0), createColumn({0, 1, 0, 1}), createColumn({0, 1, 0, 1})); + ASSERT_BITOR(createConstColumn>(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITOR(createConstColumn>(4, 0), createConstColumn(4, 0), createConstColumn(4, 0)); + ASSERT_BITOR(createConstColumn>(4, 0), createConstColumn>(4, 0), createConstColumn(4, 0)); + ASSERT_BITOR(createConstColumn>(4, 0), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + + ASSERT_BITOR(createConstColumn>(4, std::nullopt), createColumn({0, 1, 0, 1}), createConstColumn>(4, std::nullopt)); + ASSERT_BITOR(createConstColumn>(4, std::nullopt), createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt)); + ASSERT_BITOR(createConstColumn>(4, std::nullopt), createConstColumn(4, 0), createConstColumn>(4, std::nullopt)); + ASSERT_BITOR(createConstColumn>(4, std::nullopt), createConstColumn(4, 0), createConstColumn>(4, std::nullopt)); + ASSERT_BITOR(createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); } CATCH TEST_F(TestFunctionBitOr, Boundary) try { - ASSERT_BITOR(createColumn({127, 127, -128, -128}), createColumn({0, 255, 0, 255}), createColumn({127, -1, -128, -1})); - ASSERT_BITOR(createColumn({127, 127, -128, -128}), createColumn({0, 65535, 0, 65535}), createColumn({127, -1, -128, -1})); - ASSERT_BITOR(createColumn({32767, 32767, -32768, -32768}), createColumn({0, 255, 0, 255}), createColumn({32767, 32767, -32768, -32513})); + ASSERT_BITOR(createColumn({127, 127, -128, -128}), createColumn({0, 255, 0, 255}), createColumn({127, 255, 18446744073709551488ull, UINT64_MAX})); + ASSERT_BITOR(createColumn({127, 127, -128, -128}), createColumn({0, 65535, 0, 65535}), createColumn({127, 65535, 18446744073709551488ull, UINT64_MAX})); + ASSERT_BITOR(createColumn({32767, 32767, -32768, -32768}), createColumn({0, 255, 0, 255}), createColumn({32767, 32767, 18446744073709518848ull, 18446744073709519103ull})); ASSERT_BITOR(createColumn({0, 0, 1, 1, -1, -1, INT64_MAX, INT64_MAX, INT64_MIN, INT64_MIN}), createColumn({0, UINT64_MAX, 0, UINT64_MAX, 0, UINT64_MAX, 0, UINT64_MAX, 0, UINT64_MAX}), - createColumn({0, -1, 1, -1, -1, -1, INT64_MAX, -1, INT64_MIN, -1})); + createColumn({0, UINT64_MAX, 1, UINT64_MAX, UINT64_MAX, UINT64_MAX, INT64_MAX, UINT64_MAX, 9223372036854775808ull, UINT64_MAX})); } CATCH diff --git a/dbms/src/Functions/tests/gtest_bitxor.cpp b/dbms/src/Functions/tests/gtest_bitxor.cpp index 60f3027d2b5..85cc51a8a49 100644 --- a/dbms/src/Functions/tests/gtest_bitxor.cpp +++ b/dbms/src/Functions/tests/gtest_bitxor.cpp @@ -40,7 +40,7 @@ class TestFunctionBitXor : public DB::tests::FunctionTest TEST_F(TestFunctionBitXor, Simple) try { - ASSERT_BITXOR(createColumn>({-1, 1}), createColumn>({0, 0}), createColumn>({-1, 1})); + ASSERT_BITXOR(createColumn>({-1, 1}), createColumn>({0, 0}), createColumn>({UINT64_MAX, 1})); } CATCH @@ -49,21 +49,21 @@ TEST_F(TestFunctionBitXor, TypePromotion) try { // Type Promotion - ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); // Type Promotion across signed/unsigned - ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn>({0}), createColumn>({1})); } CATCH @@ -71,51 +71,51 @@ TEST_F(TestFunctionBitXor, Nullable) try { // Non Nullable - ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); - ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITXOR(createColumn({1}), createColumn({0}), createColumn({1})); // Across Nullable and non-Nullable - ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn({1}), createColumn>({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); - ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITXOR(createColumn>({1}), createColumn({0}), createColumn>({1})); } CATCH @@ -129,48 +129,48 @@ try /// 4. ColumnConst, value != null /// 5. ColumnConst, value = null - ASSERT_BITXOR(createColumn({0, 0, 1, 1}), createColumn({0, 1, 0, 1}), createColumn({0, 1, 1, 0})); - ASSERT_BITXOR(createColumn({0, 0, 1, 1}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); - ASSERT_BITXOR(createColumn({0, 0, 1, 1}), createConstColumn(4, 0), createColumn({0, 0, 1, 1})); - ASSERT_BITXOR(createColumn({0, 0, 1, 1}), createConstColumn>(4, 0), createColumn>({0, 0, 1, 1})); - ASSERT_BITXOR(createColumn({0, 0, 1, 1}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); // become const in wrapInNullable - - ASSERT_BITXOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn({0, 1, 0, 1}), createColumn>({0, 0, std::nullopt, std::nullopt})); - ASSERT_BITXOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 0, std::nullopt, std::nullopt})); - ASSERT_BITXOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt})); - ASSERT_BITXOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt})); - ASSERT_BITXOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); - - ASSERT_BITXOR(createConstColumn(4, 0), createColumn({0, 1, 0, 1}), createColumn({0, 1, 0, 1})); - ASSERT_BITXOR(createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); - ASSERT_BITXOR(createConstColumn(4, 0), createConstColumn(4, 0), createConstColumn(4, 0)); - ASSERT_BITXOR(createConstColumn(4, 0), createConstColumn>(4, 0), createConstColumn>(4, 0)); - ASSERT_BITXOR(createConstColumn(4, 0), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); - - ASSERT_BITXOR(createConstColumn>(4, 0), createColumn({0, 1, 0, 1}), createColumn>({0, 1, 0, 1})); - ASSERT_BITXOR(createConstColumn>(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); - ASSERT_BITXOR(createConstColumn>(4, 0), createConstColumn(4, 0), createConstColumn>(4, 0)); - ASSERT_BITXOR(createConstColumn>(4, 0), createConstColumn>(4, 0), createConstColumn>(4, 0)); - ASSERT_BITXOR(createConstColumn>(4, 0), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); - - ASSERT_BITXOR(createConstColumn>(4, std::nullopt), createColumn({0, 1, 0, 1}), createConstColumn>(4, std::nullopt)); - ASSERT_BITXOR(createConstColumn>(4, std::nullopt), createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt)); - ASSERT_BITXOR(createConstColumn>(4, std::nullopt), createConstColumn(4, 0), createConstColumn>(4, std::nullopt)); - ASSERT_BITXOR(createConstColumn>(4, std::nullopt), createConstColumn>(4, 0), createConstColumn>(4, std::nullopt)); - ASSERT_BITXOR(createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + ASSERT_BITXOR(createColumn({0, 0, 1, 1}), createColumn({0, 1, 0, 1}), createColumn({0, 1, 1, 0})); + ASSERT_BITXOR(createColumn({0, 0, 1, 1}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITXOR(createColumn({0, 0, 1, 1}), createConstColumn(4, 0), createColumn({0, 0, 1, 1})); + ASSERT_BITXOR(createColumn({0, 0, 1, 1}), createConstColumn>(4, 0), createColumn({0, 0, 1, 1})); + ASSERT_BITXOR(createColumn({0, 0, 1, 1}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); // become const in wrapInNullable + + ASSERT_BITXOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn({0, 1, 0, 1}), createColumn>({0, 0, std::nullopt, std::nullopt})); + ASSERT_BITXOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 0, std::nullopt, std::nullopt})); + ASSERT_BITXOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITXOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITXOR(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + + ASSERT_BITXOR(createConstColumn(4, 0), createColumn({0, 1, 0, 1}), createColumn({0, 1, 0, 1})); + ASSERT_BITXOR(createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITXOR(createConstColumn(4, 0), createConstColumn(4, 0), createConstColumn(4, 0)); + ASSERT_BITXOR(createConstColumn(4, 0), createConstColumn>(4, 0), createConstColumn(4, 0)); + ASSERT_BITXOR(createConstColumn(4, 0), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + + ASSERT_BITXOR(createConstColumn>(4, 0), createColumn({0, 1, 0, 1}), createColumn({0, 1, 0, 1})); + ASSERT_BITXOR(createConstColumn>(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITXOR(createConstColumn>(4, 0), createConstColumn(4, 0), createConstColumn(4, 0)); + ASSERT_BITXOR(createConstColumn>(4, 0), createConstColumn>(4, 0), createConstColumn(4, 0)); + ASSERT_BITXOR(createConstColumn>(4, 0), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + + ASSERT_BITXOR(createConstColumn>(4, std::nullopt), createColumn({0, 1, 0, 1}), createConstColumn>(4, std::nullopt)); + ASSERT_BITXOR(createConstColumn>(4, std::nullopt), createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt)); + ASSERT_BITXOR(createConstColumn>(4, std::nullopt), createConstColumn(4, 0), createConstColumn>(4, std::nullopt)); + ASSERT_BITXOR(createConstColumn>(4, std::nullopt), createConstColumn(4, 0), createConstColumn>(4, std::nullopt)); + ASSERT_BITXOR(createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); } CATCH TEST_F(TestFunctionBitXor, Boundary) try { - ASSERT_BITXOR(createColumn({127, 127, -128, -128}), createColumn({0, 255, 0, 255}), createColumn({127, -128, -128, 127})); - ASSERT_BITXOR(createColumn({127, 127, -128, -128}), createColumn({0, 65535, 0, 65535}), createColumn({127, -128, -128, 127})); - ASSERT_BITXOR(createColumn({32767, 32767, -32768, -32768}), createColumn({0, 255, 0, 255}), createColumn({32767, 32512, -32768, -32513})); + ASSERT_BITXOR(createColumn({127, 127, -128, -128}), createColumn({0, 255, 0, 255}), createColumn({127, 128, 18446744073709551488ull, 18446744073709551487ull})); + ASSERT_BITXOR(createColumn({127, 127, -128, -128}), createColumn({0, 65535, 0, 65535}), createColumn({127, 65408, 18446744073709551488ull, 18446744073709486207ull})); + ASSERT_BITXOR(createColumn({32767, 32767, -32768, -32768}), createColumn({0, 255, 0, 255}), createColumn({32767, 32512, 18446744073709518848ull, 18446744073709519103ull})); ASSERT_BITXOR(createColumn({0, 0, 1, 1, -1, -1, INT64_MAX, INT64_MAX, INT64_MIN, INT64_MIN}), createColumn({0, UINT64_MAX, 0, UINT64_MAX, 0, UINT64_MAX, 0, UINT64_MAX, 0, UINT64_MAX}), - createColumn({0, -1, 1, -2, -1, 0, INT64_MAX, INT64_MIN, INT64_MIN, INT64_MAX})); + createColumn({0, UINT64_MAX, 1, UINT64_MAX - 1, UINT64_MAX, 0, INT64_MAX, 9223372036854775808ull, 9223372036854775808ull, INT64_MAX})); } CATCH diff --git a/dbms/src/Functions/tests/gtest_date_add.cpp b/dbms/src/Functions/tests/gtest_date_add.cpp index 19effb11fb6..d87209c0805 100644 --- a/dbms/src/Functions/tests/gtest_date_add.cpp +++ b/dbms/src/Functions/tests/gtest_date_add.cpp @@ -114,58 +114,58 @@ TEST_F(Dateadd, dateAddStringRealUnitTest) { ASSERT_COLUMN_EQ( toNullableVec({"2012-12-14", "2012-12-14 12:12:12", "2012-12-14", "2012-12-14 12:12:12"}), - executeFunction("addDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(1.6)))); + executeFunction("addDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(1.6)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-13", "2012-12-13 12:12:12", "2012-12-13", "2012-12-13 12:12:12"}), - executeFunction("addDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(1.4)))); + executeFunction("addDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(1.4)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-14", "2012-12-14 12:12:12", "2012-12-14", "2012-12-14 12:12:12"}), - executeFunction("addDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toFloatVec({1.6, 1.6, 1.6, 1.6})))); + executeFunction("addDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toFloatVec({1.6, 1.6, 1.6, 1.6})))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-13", "2012-12-13 12:12:12", "2012-12-13", "2012-12-13 12:12:12"}), - executeFunction("addDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toFloatVec({1.4, 1.4, 1.4, 1.4})))); + executeFunction("addDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toFloatVec({1.4, 1.4, 1.4, 1.4})))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-26", "2012-12-26 12:12:12", "2012-12-26", "2012-12-26 12:12:12"}), - executeFunction("addWeeks", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(1.6)))); + executeFunction("addWeeks", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(1.6)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-19", "2012-12-19 12:12:12", "2012-12-19", "2012-12-19 12:12:12"}), - executeFunction("addWeeks", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(1.4)))); + executeFunction("addWeeks", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(1.4)))); ASSERT_COLUMN_EQ( toNullableVec({"2013-02-12", "2013-02-12 12:12:12", "2013-02-12", "2013-02-12 12:12:12"}), - executeFunction("addMonths", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(1.6)))); + executeFunction("addMonths", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(1.6)))); ASSERT_COLUMN_EQ( toNullableVec({"2013-01-12", "2013-01-12 12:12:12", "2013-01-12", "2013-01-12 12:12:12"}), - executeFunction("addMonths", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(1.4)))); + executeFunction("addMonths", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(1.4)))); ASSERT_COLUMN_EQ( toNullableVec({"2014-12-12", "2014-12-12 12:12:12", "2014-12-12", "2014-12-12 12:12:12"}), - executeFunction("addYears", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(1.6)))); + executeFunction("addYears", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(1.6)))); ASSERT_COLUMN_EQ( toNullableVec({"2013-12-12", "2013-12-12 12:12:12", "2013-12-12", "2013-12-12 12:12:12"}), - executeFunction("addYears", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(1.4)))); + executeFunction("addYears", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(1.4)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-12 02:00:00", "2012-12-12 14:12:12", "2012-12-12 02:00:00", "2012-12-12 14:12:12"}), - executeFunction("addHours", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(1.6)))); + executeFunction("addHours", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(1.6)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-12 01:00:00", "2012-12-12 13:12:12", "2012-12-12 01:00:00", "2012-12-12 13:12:12"}), - executeFunction("addHours", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(1.4)))); + executeFunction("addHours", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(1.4)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-12 00:02:00", "2012-12-12 12:14:12", "2012-12-12 00:02:00", "2012-12-12 12:14:12"}), - executeFunction("addMinutes", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(1.6)))); + executeFunction("addMinutes", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(1.6)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-12 00:01:00", "2012-12-12 12:13:12", "2012-12-12 00:01:00", "2012-12-12 12:13:12"}), - executeFunction("addMinutes", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(1.4)))); + executeFunction("addMinutes", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(1.4)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-12 00:00:02", "2012-12-12 12:12:14", "2012-12-12 00:00:02", "2012-12-12 12:12:14"}), - executeFunction("addSeconds", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(1.6)))); + executeFunction("addSeconds", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(1.6)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-12 00:00:01", "2012-12-12 12:12:13", "2012-12-12 00:00:01", "2012-12-12 12:12:13"}), - executeFunction("addSeconds", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(1.4)))); + executeFunction("addSeconds", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(1.4)))); } } // namespace DB::tests diff --git a/dbms/src/Functions/tests/gtest_date_or_datetime_to_something.cpp b/dbms/src/Functions/tests/gtest_date_or_datetime_to_something.cpp index 115218e5550..206c13b1ef6 100644 --- a/dbms/src/Functions/tests/gtest_date_or_datetime_to_something.cpp +++ b/dbms/src/Functions/tests/gtest_date_or_datetime_to_something.cpp @@ -66,7 +66,7 @@ try MyDateTime(2020, 10, 10, 0, 0, 0, 0).toPackedUInt()) .column; input_col = ColumnWithTypeAndName(data_col_ptr, data_type_ptr, "input"); - output_col = createConstColumn>(4, 4); + output_col = createConstColumn(4, 4); ASSERT_COLUMN_EQ(output_col, executeFunction(func_name, input_col)); // ColumnConst(null) diff --git a/dbms/src/Functions/tests/gtest_date_sub.cpp b/dbms/src/Functions/tests/gtest_date_sub.cpp index ad6751bac09..ee45e4e212b 100644 --- a/dbms/src/Functions/tests/gtest_date_sub.cpp +++ b/dbms/src/Functions/tests/gtest_date_sub.cpp @@ -114,58 +114,58 @@ TEST_F(Datesub, dateSubStringRealUnitTest) { ASSERT_COLUMN_EQ( toNullableVec({"2012-12-14", "2012-12-14 12:12:12", "2012-12-14", "2012-12-14 12:12:12"}), - executeFunction("subtractDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(-1.6)))); + executeFunction("subtractDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(-1.6)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-13", "2012-12-13 12:12:12", "2012-12-13", "2012-12-13 12:12:12"}), - executeFunction("subtractDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(-1.4)))); + executeFunction("subtractDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(-1.4)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-14", "2012-12-14 12:12:12", "2012-12-14", "2012-12-14 12:12:12"}), - executeFunction("subtractDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toFloatVec({-1.6, -1.6, -1.6, -1.6})))); + executeFunction("subtractDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toFloatVec({-1.6, -1.6, -1.6, -1.6})))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-13", "2012-12-13 12:12:12", "2012-12-13", "2012-12-13 12:12:12"}), - executeFunction("subtractDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toFloatVec({-1.4, -1.4, -1.4, -1.4})))); + executeFunction("subtractDays", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toFloatVec({-1.4, -1.4, -1.4, -1.4})))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-26", "2012-12-26 12:12:12", "2012-12-26", "2012-12-26 12:12:12"}), - executeFunction("subtractWeeks", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(-1.6)))); + executeFunction("subtractWeeks", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(-1.6)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-19", "2012-12-19 12:12:12", "2012-12-19", "2012-12-19 12:12:12"}), - executeFunction("subtractWeeks", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(-1.4)))); + executeFunction("subtractWeeks", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(-1.4)))); ASSERT_COLUMN_EQ( toNullableVec({"2013-02-12", "2013-02-12 12:12:12", "2013-02-12", "2013-02-12 12:12:12"}), - executeFunction("subtractMonths", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(-1.6)))); + executeFunction("subtractMonths", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(-1.6)))); ASSERT_COLUMN_EQ( toNullableVec({"2013-01-12", "2013-01-12 12:12:12", "2013-01-12", "2013-01-12 12:12:12"}), - executeFunction("subtractMonths", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(-1.4)))); + executeFunction("subtractMonths", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(-1.4)))); ASSERT_COLUMN_EQ( toNullableVec({"2014-12-12", "2014-12-12 12:12:12", "2014-12-12", "2014-12-12 12:12:12"}), - executeFunction("subtractYears", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(-1.6)))); + executeFunction("subtractYears", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(-1.6)))); ASSERT_COLUMN_EQ( toNullableVec({"2013-12-12", "2013-12-12 12:12:12", "2013-12-12", "2013-12-12 12:12:12"}), - executeFunction("subtractYears", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(-1.4)))); + executeFunction("subtractYears", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(-1.4)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-12 02:00:00", "2012-12-12 14:12:12", "2012-12-12 02:00:00", "2012-12-12 14:12:12"}), - executeFunction("subtractHours", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(-1.6)))); + executeFunction("subtractHours", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(-1.6)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-12 01:00:00", "2012-12-12 13:12:12", "2012-12-12 01:00:00", "2012-12-12 13:12:12"}), - executeFunction("subtractHours", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(-1.4)))); + executeFunction("subtractHours", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(-1.4)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-12 00:02:00", "2012-12-12 12:14:12", "2012-12-12 00:02:00", "2012-12-12 12:14:12"}), - executeFunction("subtractMinutes", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(-1.6)))); + executeFunction("subtractMinutes", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(-1.6)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-12 00:01:00", "2012-12-12 12:13:12", "2012-12-12 00:01:00", "2012-12-12 12:13:12"}), - executeFunction("subtractMinutes", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(-1.4)))); + executeFunction("subtractMinutes", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(-1.4)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-12 00:00:02", "2012-12-12 12:12:14", "2012-12-12 00:00:02", "2012-12-12 12:12:14"}), - executeFunction("subtractSeconds", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(-1.6)))); + executeFunction("subtractSeconds", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(-1.6)))); ASSERT_COLUMN_EQ( toNullableVec({"2012-12-12 00:00:01", "2012-12-12 12:12:13", "2012-12-12 00:00:01", "2012-12-12 12:12:13"}), - executeFunction("subtractSeconds", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("round", toConst(-1.4)))); + executeFunction("subtractSeconds", toNullableVec({"20121212", "20121212121212", "2012-12-12", "2012-12-12 12:12:12"}), executeFunction("tidbRound", toConst(-1.4)))); } } // namespace DB::tests diff --git a/dbms/src/Functions/tests/gtest_datetime_daymonthyear.cpp b/dbms/src/Functions/tests/gtest_datetime_daymonthyear.cpp index 99254821062..1cc7d799014 100644 --- a/dbms/src/Functions/tests/gtest_datetime_daymonthyear.cpp +++ b/dbms/src/Functions/tests/gtest_datetime_daymonthyear.cpp @@ -82,9 +82,9 @@ class TestDateTimeDayMonthYear : public DB::tests::FunctionTest .column, makeNullable(std::make_shared
    ()), "result"); - result_day = createConstColumn>(1, {day}); - result_month = createConstColumn>(1, {month}); - result_year = createConstColumn>(1, {year}); + result_day = createConstColumn(1, {day}); + result_month = createConstColumn(1, {month}); + result_year = createConstColumn(1, {year}); } else { diff --git a/dbms/src/Functions/tests/gtest_duration_pushdown.cpp b/dbms/src/Functions/tests/gtest_duration_pushdown.cpp index bd6fddaeb84..4501a4c9fae 100644 --- a/dbms/src/Functions/tests/gtest_duration_pushdown.cpp +++ b/dbms/src/Functions/tests/gtest_duration_pushdown.cpp @@ -43,8 +43,10 @@ try result_col, executeFunction( "FunctionConvertDurationFromNanos", - createColumn>({-1, 0, 1, {}, INT64_MAX, INT64_MIN, (838 * 3600 + 59 * 60 + 59) * 1000000000L, -(838 * 3600 + 59 * 60 + 59) * 1000000000L}), - createConstColumn(8, 1))); + {createColumn>({-1, 0, 1, {}, INT64_MAX, INT64_MIN, (838 * 3600 + 59 * 60 + 59) * 1000000000L, -(838 * 3600 + 59 * 60 + 59) * 1000000000L}), + createConstColumn(8, 1)}, + nullptr, + true)); ColumnWithTypeAndName result_col2( createConstColumn(3, 3).column, @@ -54,8 +56,10 @@ try result_col2, executeFunction( "FunctionConvertDurationFromNanos", - createConstColumn(3, 3), - createConstColumn(3, 2))); + {createConstColumn(3, 3), + createConstColumn(3, 2)}, + nullptr, + true)); } CATCH diff --git a/dbms/src/Functions/tests/gtest_functions_round_with_frac.cpp b/dbms/src/Functions/tests/gtest_functions_round_with_frac.cpp index 0c85285029e..403b88259a4 100644 --- a/dbms/src/Functions/tests/gtest_functions_round_with_frac.cpp +++ b/dbms/src/Functions/tests/gtest_functions_round_with_frac.cpp @@ -312,9 +312,10 @@ TEST_F(TestFunctionsRoundWithFrac, IntConstInput) // const signed - const frac for (size_t i = 0; i < size; ++i) { - ASSERT_COLUMN_EQ(createConstColumn>(1, int32_result[i]), + bool frac_data_null = !frac_data[i].has_value(); + ASSERT_COLUMN_EQ(frac_data_null ? createConstColumn>(1, int32_result[i]) : createConstColumn(1, int32_result[i].value()), execute(createConstColumn(1, int32_input), createConstColumn>(1, frac_data[i]))); - ASSERT_COLUMN_EQ(createConstColumn>(1, uint32_result[i]), + ASSERT_COLUMN_EQ(frac_data_null ? createConstColumn>(1, uint32_result[i]) : createConstColumn(1, uint32_result[i].value()), execute(createConstColumn(1, uint32_input), createConstColumn>(1, frac_data[i]))); ASSERT_COLUMN_EQ(createConstColumn>(1, {}), execute(createConstColumn>(1, {}), createConstColumn>(1, frac_data[i]))); @@ -441,22 +442,22 @@ try { auto frac = createColumn>({3, 2, 1, 0, -1, -2, -3, -4, -5, -6, {}}); - ASSERT_COLUMN_EQ(column({max_prec, 3}, - {"98765.432", "98765.430", "98765.400", "98765.000", "98770.000", "98800.000", "99000.000", "100000.000", "100000.000", "0.000", {}}), + ASSERT_COLUMN_EQ(createColumn>(std::make_tuple(9, 3), + {"98765.432", "98765.430", "98765.400", "98765.000", "98770.000", "98800.000", "99000.000", "100000.000", "100000.000", "0.000", {}}), this->execute(constColumn({max_prec - 1, 3}, 11, "98765.432"), frac)); ASSERT_COLUMN_EQ(constColumn({max_prec, 3}, 11, {}), this->execute(constColumn({max_prec - 1, 3}, 11, {}), frac)); } // const input & frac - ASSERT_COLUMN_EQ(constColumn({max_prec - 1, 2}, 1, "0.03"), + ASSERT_COLUMN_EQ(createConstColumn(std::make_tuple(3, 2), 1, "0.03"), this->execute(constColumn({max_prec - 1, 3}, 1, "0.025"), createConstColumn(1, 2))); ASSERT_COLUMN_EQ( constColumn({max_prec - 1, 2}, 1, {}), this->execute(constColumn({max_prec - 1, 3}, 1, {}), createConstColumn(1, 2))); - ASSERT_COLUMN_EQ(constColumn({max_prec - 3, 0}, 1, {}), + ASSERT_COLUMN_EQ(createConstColumn>(std::make_tuple(1, 0), 1, {}, "", 0), this->execute(constColumn({max_prec - 1, 3}, 1, "0.025"), createConstColumn>(1, {}))); - ASSERT_COLUMN_EQ(createConstColumn(std::make_tuple(max_prec, 5), 100, "1." + String(5, '0')), + ASSERT_COLUMN_EQ(createConstColumn(std::make_tuple(6, 5), 100, "1." + String(5, '0')), this->execute(createConstColumn(std::make_tuple(max_prec - 5, 0), 100, "1"), createConstColumn(100, 5))); } CATCH diff --git a/dbms/src/Functions/tests/gtest_ifnull.cpp b/dbms/src/Functions/tests/gtest_ifnull.cpp index 6d4bd17089a..bd319b59d92 100644 --- a/dbms/src/Functions/tests/gtest_ifnull.cpp +++ b/dbms/src/Functions/tests/gtest_ifnull.cpp @@ -34,19 +34,15 @@ class TestIfNull : public DB::tests::FunctionTest protected: ColumnWithTypeAndName executeIfNull(const ColumnWithTypeAndName & first_column, const ColumnWithTypeAndName & second_column) { - auto is_null_column = executeFunction("isNull", first_column); - auto not_null_column = executeFunction("assumeNotNull", first_column); - return executeFunction("multiIf", is_null_column, second_column, not_null_column); + return executeFunction("ifNull", first_column, second_column); } DataTypePtr getReturnTypeForIfNull(const DataTypePtr & type_1, const DataTypePtr & type_2) { - const static auto cond_type = std::make_shared(); ColumnsWithTypeAndName input_columns{ - {nullptr, cond_type, ""}, - {nullptr, removeNullable(type_1), ""}, + {nullptr, type_1, ""}, {nullptr, type_2, ""}, }; - return getReturnTypeForFunction(context, "multiIf", input_columns); + return getReturnTypeForFunction(context, "ifNull", input_columns); } template ColumnWithTypeAndName createIntegerColumnInternal(const std::vector & signed_input, const std::vector unsigned_input, const std::vector & null_map) @@ -150,12 +146,12 @@ try } else { - ASSERT_COLUMN_EQ(col_2.type->isNullable() ? expr_data_2_nullable_vector : expr_data_2_vector, executeIfNull(col_1, col_2)); + ASSERT_COLUMN_EQ(expr_data_2_vector, executeIfNull(col_1, col_2)); } } else { - if (col_2.type->isNullable()) + if (col_2.column->isNullAt(0)) { ASSERT_COLUMN_EQ(expr_data_1_nullable_vector, executeIfNull(col_1, col_2)); } @@ -185,14 +181,7 @@ try } else { - if (col_2.type->isNullable()) - { - ASSERT_COLUMN_EQ(createNullableColumn(vector_const_result, {0, 0, 0, 0, 0}), executeIfNull(col_1, col_2)); - } - else - { - ASSERT_COLUMN_EQ(createColumn(vector_const_result), executeIfNull(col_1, col_2)); - } + ASSERT_COLUMN_EQ(createColumn(vector_const_result), executeIfNull(col_1, col_2)); } } } diff --git a/dbms/src/Functions/tests/gtest_inet_aton_ntoa.cpp b/dbms/src/Functions/tests/gtest_inet_aton_ntoa.cpp index 756dc7e610a..cb37252d17e 100644 --- a/dbms/src/Functions/tests/gtest_inet_aton_ntoa.cpp +++ b/dbms/src/Functions/tests/gtest_inet_aton_ntoa.cpp @@ -142,7 +142,7 @@ try // const non-null column ASSERT_COLUMN_EQ( - createConstColumn>(1, "0.0.0.1"), + createConstColumn(1, "0.0.0.1"), executeFunction(func_name, createConstColumn>(1, 1))); // normal cases diff --git a/dbms/src/Functions/tests/gtest_is_true_false.cpp b/dbms/src/Functions/tests/gtest_is_true_false.cpp index 520728b4380..400166c685e 100644 --- a/dbms/src/Functions/tests/gtest_is_true_false.cpp +++ b/dbms/src/Functions/tests/gtest_is_true_false.cpp @@ -124,10 +124,10 @@ CATCH createColumn>({0, 1, 1, std::nullopt}), \ executeFunction("isTrueWithNull", createColumn>({0, 1, static_cast(-1), std::nullopt}))); \ ASSERT_COLUMN_EQ( \ - createConstColumn>(5, 0), \ + createConstColumn(5, 0), \ executeFunction("isTrueWithNull", createConstColumn>(5, 0))); \ ASSERT_COLUMN_EQ( \ - createConstColumn>(5, 1), \ + createConstColumn(5, 1), \ executeFunction("isTrueWithNull", createConstColumn>(5, 2))); \ ASSERT_COLUMN_EQ( \ createConstColumn>(5, std::nullopt), \ @@ -194,10 +194,10 @@ CATCH createColumn>({1, 0, 0, std::nullopt}), \ executeFunction("isFalseWithNull", createColumn>({0, 1, static_cast(-1), std::nullopt}))); \ ASSERT_COLUMN_EQ( \ - createConstColumn>(5, 1), \ + createConstColumn(5, 1), \ executeFunction("isFalseWithNull", createConstColumn>(5, 0))); \ ASSERT_COLUMN_EQ( \ - createConstColumn>(5, 0), \ + createConstColumn(5, 0), \ executeFunction("isFalseWithNull", createConstColumn>(5, 2))); \ ASSERT_COLUMN_EQ( \ createConstColumn>(5, std::nullopt), \ diff --git a/dbms/src/Functions/tests/gtest_least_greatest.cpp b/dbms/src/Functions/tests/gtest_least_greatest.cpp index bc57cc531a1..cbf7552fdc1 100644 --- a/dbms/src/Functions/tests/gtest_least_greatest.cpp +++ b/dbms/src/Functions/tests/gtest_least_greatest.cpp @@ -154,7 +154,7 @@ try // const-const least ASSERT_COLUMN_EQ( - createConstColumn>(1, -3), + createConstColumn(1, -3), executeFunction( func_name, createConstColumn>(1, 5), @@ -323,7 +323,7 @@ try // const-const greatest ASSERT_COLUMN_EQ( - createConstColumn>(1, 5), + createConstColumn(1, 5), executeFunction( func_name, createConstColumn>(1, 5), diff --git a/dbms/src/Functions/tests/gtest_logical.cpp b/dbms/src/Functions/tests/gtest_logical.cpp index 7988989cc88..29fd4d6e133 100644 --- a/dbms/src/Functions/tests/gtest_logical.cpp +++ b/dbms/src/Functions/tests/gtest_logical.cpp @@ -46,7 +46,7 @@ try createColumn>({1, 0}))); // const, const ASSERT_COLUMN_EQ( - createConstColumn>(1, 1), + createConstColumn(1, 1), executeFunction( func_name, createConstColumn>(1, 1), @@ -82,7 +82,7 @@ try createColumn>({1, 0}))); // const, const ASSERT_COLUMN_EQ( - createConstColumn>(1, 1), + createConstColumn(1, 1), executeFunction( func_name, createConstColumn>(1, 1), @@ -118,7 +118,7 @@ try createColumn>({1, 0}))); // const, const ASSERT_COLUMN_EQ( - createConstColumn>(1, 0), + createConstColumn(1, 0), executeFunction( func_name, createConstColumn>(1, 1), @@ -146,7 +146,7 @@ try createColumn>({0, 1, {}}))); // const ASSERT_COLUMN_EQ( - createConstColumn>(1, 0), + createConstColumn(1, 0), executeFunction( func_name, createConstColumn>(1, 1))); diff --git a/dbms/src/Functions/tests/gtest_regexp.cpp b/dbms/src/Functions/tests/gtest_regexp.cpp index 25314e9f5e7..d3eb93a0790 100644 --- a/dbms/src/Functions/tests/gtest_regexp.cpp +++ b/dbms/src/Functions/tests/gtest_regexp.cpp @@ -1828,15 +1828,15 @@ TEST_F(Regexp, testRegexp) for (size_t i = 0; i < row_size; i++) { /// test regexp(const, const) - ASSERT_COLUMN_EQ(input_string_nulls[i] || pattern_nulls[i] ? const_uint8_null_column : createConstColumn>(row_size, results[i]), + ASSERT_COLUMN_EQ(input_string_nulls[i] || pattern_nulls[i] ? const_uint8_null_column : createConstColumn(row_size, results[i]), executeFunction("regexp", input_string_nulls[i] ? const_string_null_column : createConstColumn>(row_size, input_strings[i]), pattern_nulls[i] ? const_string_null_column : createConstColumn>(row_size, patterns[i]))); /// test regexp(const, const, const) - ASSERT_COLUMN_EQ(input_string_nulls[i] || pattern_nulls[i] || match_type_nulls[i] ? const_uint8_null_column : createConstColumn>(row_size, results_with_match_type[i]), + ASSERT_COLUMN_EQ(input_string_nulls[i] || pattern_nulls[i] || match_type_nulls[i] ? const_uint8_null_column : createConstColumn(row_size, results_with_match_type[i]), executeFunction("regexp", input_string_nulls[i] ? const_string_null_column : createConstColumn>(row_size, input_strings[i]), pattern_nulls[i] ? const_string_null_column : createConstColumn>(row_size, patterns[i]), match_type_nulls[i] ? const_string_null_column : createConstColumn>(row_size, match_types[i]))); /// test regexp(const, const, const) with binary collator - ASSERT_COLUMN_EQ(input_string_nulls[i] || pattern_nulls[i] || match_type_nulls[i] ? const_uint8_null_column : createConstColumn>(row_size, results_with_match_type_collator[i]), + ASSERT_COLUMN_EQ(input_string_nulls[i] || pattern_nulls[i] || match_type_nulls[i] ? const_uint8_null_column : createConstColumn(row_size, results_with_match_type_collator[i]), executeFunction("regexp", {input_string_nulls[i] ? const_string_null_column : createConstColumn>(row_size, input_strings[i]), pattern_nulls[i] ? const_string_null_column : createConstColumn>(row_size, patterns[i]), match_type_nulls[i] ? const_string_null_column : createConstColumn>(row_size, match_types[i])}, binary_collator)); } /// case 3 regexp(vector, const[, const]) @@ -1920,20 +1920,13 @@ TEST_F(Regexp, testRegexpCustomerCases) } else if (isColumnConstNotNull(input_column) && isColumnConstNotNull(pattern_column)) { - if (input_column.type->isNullable() || pattern_column.type->isNullable()) - { - ASSERT_COLUMN_EQ(createConstColumn>(5, 1), - executeFunction("regexp", input_column, pattern_column)); - } - else - { - ASSERT_COLUMN_EQ(createConstColumn(5, 1), - executeFunction("regexp", input_column, pattern_column)); - } + ASSERT_COLUMN_EQ(createConstColumn(5, 1), + executeFunction("regexp", input_column, pattern_column)); } else { - bool result_nullable = input_column.type->isNullable() || pattern_column.type->isNullable(); + bool result_nullable = (input_column.type->isNullable() && !isColumnConstNotNull(input_column)) + || (pattern_column.type->isNullable() && !isColumnConstNotNull(pattern_column)); if (!result_nullable) { ASSERT_COLUMN_EQ(createColumn({1, 1, 1, 1, 1}), @@ -2067,28 +2060,29 @@ TEST_F(Regexp, testRegexpReplace) auto const_string_null_column = createConstColumn>(row_size, {}); auto const_int64_null_column = createConstColumn>(row_size, {}); + /// regexp_replace is not supported in TiDB yet, so use raw function test /// case 1. regexp_replace(const, const, const [, const, const ,const]) for (size_t i = 0; i < match_types.size(); i++) { /// test regexp_replace(str, pattern, replacement) ASSERT_COLUMN_EQ(createConstColumn(row_size, results[i]), - executeFunction("replaceRegexpAll", createConstColumn(row_size, input_strings[i]), createConstColumn(row_size, patterns[i]), createConstColumn(row_size, replacements[i]))); + executeFunction("replaceRegexpAll", {createConstColumn(row_size, input_strings[i]), createConstColumn(row_size, patterns[i]), createConstColumn(row_size, replacements[i])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos) ASSERT_COLUMN_EQ(createConstColumn(row_size, results_with_pos[i]), - executeFunction("replaceRegexpAll", createConstColumn(row_size, input_strings[i]), createConstColumn(row_size, patterns[i]), createConstColumn(row_size, replacements[i]), createConstColumn(row_size, pos[i]))); + executeFunction("replaceRegexpAll", {createConstColumn(row_size, input_strings[i]), createConstColumn(row_size, patterns[i]), createConstColumn(row_size, replacements[i]), createConstColumn(row_size, pos[i])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos, occ) ASSERT_COLUMN_EQ(createConstColumn(row_size, results_with_pos_occ[i]), - executeFunction("replaceRegexpAll", createConstColumn(row_size, input_strings[i]), createConstColumn(row_size, patterns[i]), createConstColumn(row_size, replacements[i]), createConstColumn(row_size, pos[i]), createConstColumn(row_size, occ[i]))); + executeFunction("replaceRegexpAll", {createConstColumn(row_size, input_strings[i]), createConstColumn(row_size, patterns[i]), createConstColumn(row_size, replacements[i]), createConstColumn(row_size, pos[i]), createConstColumn(row_size, occ[i])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos, occ, match_type) ASSERT_COLUMN_EQ(createConstColumn(row_size, results_with_pos_occ_match_type[i]), - executeFunction("replaceRegexpAll", createConstColumn(row_size, input_strings[i]), createConstColumn(row_size, patterns[i]), createConstColumn(row_size, replacements[i]), createConstColumn(row_size, pos[i]), createConstColumn(row_size, occ[i]), createConstColumn(row_size, match_types[i]))); + executeFunction("replaceRegexpAll", {createConstColumn(row_size, input_strings[i]), createConstColumn(row_size, patterns[i]), createConstColumn(row_size, replacements[i]), createConstColumn(row_size, pos[i]), createConstColumn(row_size, occ[i]), createConstColumn(row_size, match_types[i])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos, occ, match_type) with binary collator ASSERT_COLUMN_EQ(createConstColumn(row_size, results_with_pos_occ_match_type_binary[i]), - executeFunction("replaceRegexpAll", {createConstColumn(row_size, input_strings[i]), createConstColumn(row_size, patterns[i]), createConstColumn(row_size, replacements[i]), createConstColumn(row_size, pos[i]), createConstColumn(row_size, occ[i]), createConstColumn(row_size, match_types[i])}, binary_collator)); + executeFunction("replaceRegexpAll", {createConstColumn(row_size, input_strings[i]), createConstColumn(row_size, patterns[i]), createConstColumn(row_size, replacements[i]), createConstColumn(row_size, pos[i]), createConstColumn(row_size, occ[i]), createConstColumn(row_size, match_types[i])}, binary_collator, true)); } /// case 2. regexp_replace(const, const, const [, const, const ,const]) with null value @@ -2097,74 +2091,74 @@ TEST_F(Regexp, testRegexpReplace) /// test regexp_replace(str, pattern, replacement) bool null_result = input_string_nulls[i] || pattern_nulls[i] || replacement_nulls[i]; ASSERT_COLUMN_EQ(null_result ? const_string_null_column : createConstColumn>(row_size, results[i]), - executeFunction("replaceRegexpAll", input_string_nulls[i] ? const_string_null_column : createConstColumn>(row_size, input_strings[i]), pattern_nulls[i] ? const_string_null_column : createConstColumn>(row_size, patterns[i]), replacement_nulls[i] ? const_string_null_column : createConstColumn>(row_size, replacements[i]))); + executeFunction("replaceRegexpAll", {input_string_nulls[i] ? const_string_null_column : createConstColumn>(row_size, input_strings[i]), pattern_nulls[i] ? const_string_null_column : createConstColumn>(row_size, patterns[i]), replacement_nulls[i] ? const_string_null_column : createConstColumn>(row_size, replacements[i])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos) null_result = null_result || pos_nulls[i]; ASSERT_COLUMN_EQ(null_result ? const_string_null_column : createConstColumn>(row_size, results_with_pos[i]), - executeFunction("replaceRegexpAll", input_string_nulls[i] ? const_string_null_column : createConstColumn>(row_size, input_strings[i]), pattern_nulls[i] ? const_string_null_column : createConstColumn>(row_size, patterns[i]), replacement_nulls[i] ? const_string_null_column : createConstColumn>(row_size, replacements[i]), pos_nulls[i] ? const_int64_null_column : createConstColumn>(row_size, pos[i]))); + executeFunction("replaceRegexpAll", {input_string_nulls[i] ? const_string_null_column : createConstColumn>(row_size, input_strings[i]), pattern_nulls[i] ? const_string_null_column : createConstColumn>(row_size, patterns[i]), replacement_nulls[i] ? const_string_null_column : createConstColumn>(row_size, replacements[i]), pos_nulls[i] ? const_int64_null_column : createConstColumn>(row_size, pos[i])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos, occ) null_result = null_result || occ_nulls[i]; ASSERT_COLUMN_EQ(null_result ? const_string_null_column : createConstColumn>(row_size, results_with_pos_occ[i]), - executeFunction("replaceRegexpAll", input_string_nulls[i] ? const_string_null_column : createConstColumn>(row_size, input_strings[i]), pattern_nulls[i] ? const_string_null_column : createConstColumn>(row_size, patterns[i]), replacement_nulls[i] ? const_string_null_column : createConstColumn>(row_size, replacements[i]), pos_nulls[i] ? const_int64_null_column : createConstColumn>(row_size, pos[i]), occ_nulls[i] ? const_int64_null_column : createConstColumn>(row_size, occ[i]))); + executeFunction("replaceRegexpAll", {input_string_nulls[i] ? const_string_null_column : createConstColumn>(row_size, input_strings[i]), pattern_nulls[i] ? const_string_null_column : createConstColumn>(row_size, patterns[i]), replacement_nulls[i] ? const_string_null_column : createConstColumn>(row_size, replacements[i]), pos_nulls[i] ? const_int64_null_column : createConstColumn>(row_size, pos[i]), occ_nulls[i] ? const_int64_null_column : createConstColumn>(row_size, occ[i])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos, occ, match_type) null_result = null_result || match_type_nulls[i]; ASSERT_COLUMN_EQ(null_result ? const_string_null_column : createConstColumn>(row_size, results_with_pos_occ_match_type[i]), - executeFunction("replaceRegexpAll", input_string_nulls[i] ? const_string_null_column : createConstColumn>(row_size, input_strings[i]), pattern_nulls[i] ? const_string_null_column : createConstColumn>(row_size, patterns[i]), replacement_nulls[i] ? const_string_null_column : createConstColumn>(row_size, replacements[i]), pos_nulls[i] ? const_int64_null_column : createConstColumn>(row_size, pos[i]), occ_nulls[i] ? const_int64_null_column : createConstColumn>(row_size, occ[i]), match_type_nulls[i] ? const_string_null_column : createConstColumn>(row_size, match_types[i]))); + executeFunction("replaceRegexpAll", {input_string_nulls[i] ? const_string_null_column : createConstColumn>(row_size, input_strings[i]), pattern_nulls[i] ? const_string_null_column : createConstColumn>(row_size, patterns[i]), replacement_nulls[i] ? const_string_null_column : createConstColumn>(row_size, replacements[i]), pos_nulls[i] ? const_int64_null_column : createConstColumn>(row_size, pos[i]), occ_nulls[i] ? const_int64_null_column : createConstColumn>(row_size, occ[i]), match_type_nulls[i] ? const_string_null_column : createConstColumn>(row_size, match_types[i])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos, occ, match_type) with binary collator ASSERT_COLUMN_EQ(null_result ? const_string_null_column : createConstColumn>(row_size, results_with_pos_occ_match_type_binary[i]), - executeFunction("replaceRegexpAll", {input_string_nulls[i] ? const_string_null_column : createConstColumn>(row_size, input_strings[i]), pattern_nulls[i] ? const_string_null_column : createConstColumn>(row_size, patterns[i]), replacement_nulls[i] ? const_string_null_column : createConstColumn>(row_size, replacements[i]), pos_nulls[i] ? const_int64_null_column : createConstColumn>(row_size, pos[i]), occ_nulls[i] ? const_int64_null_column : createConstColumn>(row_size, occ[i]), match_type_nulls[i] ? const_string_null_column : createConstColumn>(row_size, match_types[i])}, binary_collator)); + executeFunction("replaceRegexpAll", {input_string_nulls[i] ? const_string_null_column : createConstColumn>(row_size, input_strings[i]), pattern_nulls[i] ? const_string_null_column : createConstColumn>(row_size, patterns[i]), replacement_nulls[i] ? const_string_null_column : createConstColumn>(row_size, replacements[i]), pos_nulls[i] ? const_int64_null_column : createConstColumn>(row_size, pos[i]), occ_nulls[i] ? const_int64_null_column : createConstColumn>(row_size, occ[i]), match_type_nulls[i] ? const_string_null_column : createConstColumn>(row_size, match_types[i])}, binary_collator, true)); } /// case 3 regexp_replace(vector, const, const[, const, const, const]) { /// test regexp_replace(str, pattern, replacement) ASSERT_COLUMN_EQ(createColumn(vec_results), - executeFunction("replaceRegexpAll", createColumn(input_strings), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]))); + executeFunction("replaceRegexpAll", {createColumn(input_strings), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos) ASSERT_COLUMN_EQ(createColumn(vec_results_with_pos), - executeFunction("replaceRegexpAll", createColumn(input_strings), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0]))); + executeFunction("replaceRegexpAll", {createColumn(input_strings), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos, occ) ASSERT_COLUMN_EQ(createColumn(vec_results_with_pos_occ), - executeFunction("replaceRegexpAll", createColumn(input_strings), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0]), createConstColumn(row_size, occ[0]))); + executeFunction("replaceRegexpAll", {createColumn(input_strings), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0]), createConstColumn(row_size, occ[0])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos, occ, match_type) ASSERT_COLUMN_EQ(createColumn(vec_results_with_pos_occ_match_type), - executeFunction("replaceRegexpAll", createColumn(input_strings), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0]), createConstColumn(row_size, occ[0]), createConstColumn(row_size, match_types[0]))); + executeFunction("replaceRegexpAll", {createColumn(input_strings), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0]), createConstColumn(row_size, occ[0]), createConstColumn(row_size, match_types[0])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos, occ, match_type) with binary collator ASSERT_COLUMN_EQ(createColumn(vec_results_with_pos_occ_match_type_binary), - executeFunction("replaceRegexpAll", {createColumn(input_strings), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0]), createConstColumn(row_size, occ[0]), createConstColumn(row_size, match_types[0])}, binary_collator)); + executeFunction("replaceRegexpAll", {createColumn(input_strings), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0]), createConstColumn(row_size, occ[0]), createConstColumn(row_size, match_types[0])}, binary_collator, true)); } /// case 4 regexp_replace(vector, const, const[, const, const, const]) with null value { /// test regexp_replace(str, pattern, replacement) ASSERT_COLUMN_EQ(createNullableVectorColumn(vec_results, input_string_nulls), - executeFunction("replaceRegexpAll", createNullableVectorColumn(input_strings, input_string_nulls), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]))); + executeFunction("replaceRegexpAll", {createNullableVectorColumn(input_strings, input_string_nulls), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos) ASSERT_COLUMN_EQ(createNullableVectorColumn(vec_results_with_pos, input_string_nulls), - executeFunction("replaceRegexpAll", createNullableVectorColumn(input_strings, input_string_nulls), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0]))); + executeFunction("replaceRegexpAll", {createNullableVectorColumn(input_strings, input_string_nulls), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos, occ) ASSERT_COLUMN_EQ(createNullableVectorColumn(vec_results_with_pos_occ, input_string_nulls), - executeFunction("replaceRegexpAll", createNullableVectorColumn(input_strings, input_string_nulls), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0]), createConstColumn(row_size, occ[0]))); + executeFunction("replaceRegexpAll", {createNullableVectorColumn(input_strings, input_string_nulls), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0]), createConstColumn(row_size, occ[0])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos, occ, match_type) ASSERT_COLUMN_EQ(createNullableVectorColumn(vec_results_with_pos_occ_match_type, input_string_nulls), - executeFunction("replaceRegexpAll", createNullableVectorColumn(input_strings, input_string_nulls), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0]), createConstColumn(row_size, occ[0]), createConstColumn(row_size, match_types[0]))); + executeFunction("replaceRegexpAll", {createNullableVectorColumn(input_strings, input_string_nulls), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0]), createConstColumn(row_size, occ[0]), createConstColumn(row_size, match_types[0])}, nullptr, true)); /// test regexp_replace(str, pattern, replacement, pos, occ, match_type) with binary collator ASSERT_COLUMN_EQ(createNullableVectorColumn(vec_results_with_pos_occ_match_type_binary, input_string_nulls), - executeFunction("replaceRegexpAll", {createNullableVectorColumn(input_strings, input_string_nulls), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0]), createConstColumn(row_size, occ[0]), createConstColumn(row_size, match_types[0])}, binary_collator)); + executeFunction("replaceRegexpAll", {createNullableVectorColumn(input_strings, input_string_nulls), createConstColumn(row_size, patterns[0]), createConstColumn(row_size, replacements[0]), createConstColumn(row_size, pos[0]), createConstColumn(row_size, occ[0]), createConstColumn(row_size, match_types[0])}, binary_collator, true)); } } } // namespace tests diff --git a/dbms/src/Functions/tests/gtest_string_left.cpp b/dbms/src/Functions/tests/gtest_string_left.cpp index ca491bfe6fa..42a16ee6fff 100644 --- a/dbms/src/Functions/tests/gtest_string_left.cpp +++ b/dbms/src/Functions/tests/gtest_string_left.cpp @@ -27,8 +27,7 @@ namespace tests class StringLeftTest : public DB::tests::FunctionTest { public: - // leftUTF8(str,len) = substrUTF8(str,const 1,len) - static constexpr auto func_name = "substringUTF8"; + static constexpr auto func_name = "leftUTF8"; template void testBoundary() @@ -53,14 +52,14 @@ class StringLeftTest : public DB::tests::FunctionTest template void test(const std::optional & str, const std::optional & length, const std::optional & result) { - const auto start_column = createConstColumn(1, 1); auto inner_test = [&](bool is_str_const, bool is_length_const) { bool is_one_of_args_null_const = (is_str_const && !str.has_value()) || (is_length_const && !length.has_value()); bool is_result_const = (is_str_const && is_length_const) || is_one_of_args_null_const; - auto expected_res_column = is_result_const ? createConstColumn>(1, result) : createColumn>({result}); + auto expected_res_column = is_result_const ? (is_one_of_args_null_const ? createConstColumn>(1, result) : createConstColumn(1, result.value())) + : createColumn>({result}); auto str_column = is_str_const ? createConstColumn>(1, str) : createColumn>({str}); auto length_column = is_length_const ? createConstColumn>(1, length) : createColumn>({length}); - auto actual_res_column = executeFunction(func_name, str_column, start_column, length_column); + auto actual_res_column = executeFunction(func_name, str_column, length_column); ASSERT_COLUMN_EQ(expected_res_column, actual_res_column); }; std::vector is_consts = {true, false}; @@ -78,7 +77,6 @@ class StringLeftTest : public DB::tests::FunctionTest executeFunction( func_name, is_str_const ? createConstColumn>(1, "") : createColumn>({""}), - createConstColumn(1, 1), is_length_const ? createConstColumn>(1, 0) : createColumn>({0})), Exception); }; @@ -132,7 +130,6 @@ try executeFunction( func_name, createColumn>({big_string, origin_str, origin_str, mixed_language_str}), - createConstColumn(8, 1), createColumn>({22, 12, 22, english_str.size()}))); // case 2 String second_case_string = "abc"; @@ -141,14 +138,12 @@ try executeFunction( func_name, createColumn>({second_case_string, second_case_string, second_case_string, second_case_string, second_case_string, second_case_string, second_case_string, second_case_string}), - createConstColumn(8, 1), createColumn>({0, 1, 0, 1, 0, 0, 1, 1}))); ASSERT_COLUMN_EQ( createColumn>({"", "a", "", "a", "", "", "a", "a"}), executeFunction( func_name, createConstColumn>(8, second_case_string), - createConstColumn(8, 1), createColumn>({0, 1, 0, 1, 0, 0, 1, 1}))); } CATCH diff --git a/dbms/src/Functions/tests/gtest_string_lrtrim.cpp b/dbms/src/Functions/tests/gtest_string_lrtrim.cpp index 409c8ed715b..df52257613d 100644 --- a/dbms/src/Functions/tests/gtest_string_lrtrim.cpp +++ b/dbms/src/Functions/tests/gtest_string_lrtrim.cpp @@ -39,16 +39,16 @@ try { // ltrim(const) ASSERT_COLUMN_EQ( - createConstColumn>(5, "x "), + createConstColumn(5, "x "), executeFunction("tidbLTrim", createConstColumn>(5, " x "))); ASSERT_COLUMN_EQ( - createConstColumn>(5, "测试 "), + createConstColumn(5, "测试 "), executeFunction("tidbLTrim", createConstColumn>(5, " 测试 "))); ASSERT_COLUMN_EQ( - createConstColumn>(5, "x x x"), + createConstColumn(5, "x x x"), executeFunction("tidbLTrim", createConstColumn>(5, "x x x"))); ASSERT_COLUMN_EQ( - createConstColumn>(5, "测 试 "), + createConstColumn(5, "测 试 "), executeFunction("tidbLTrim", createConstColumn>(5, "测 试 "))); ASSERT_COLUMN_EQ( createConstColumn(5, "x "), @@ -83,16 +83,16 @@ try // rtrim(const) ASSERT_COLUMN_EQ( - createConstColumn>(5, " x"), + createConstColumn(5, " x"), executeFunction("tidbRTrim", createConstColumn>(5, " x "))); ASSERT_COLUMN_EQ( - createConstColumn>(5, " 测试"), + createConstColumn(5, " 测试"), executeFunction("tidbRTrim", createConstColumn>(5, " 测试 "))); ASSERT_COLUMN_EQ( - createConstColumn>(5, "x x x"), + createConstColumn(5, "x x x"), executeFunction("tidbRTrim", createConstColumn>(5, "x x x"))); ASSERT_COLUMN_EQ( - createConstColumn>(5, "测 试"), + createConstColumn(5, "测 试"), executeFunction("tidbRTrim", createConstColumn>(5, "测 试 "))); ASSERT_COLUMN_EQ( createConstColumn(5, " x"), @@ -225,10 +225,10 @@ try input_iter++, lres_iter++, rres_iter++) { ASSERT_COLUMN_EQ( - createConstColumn>(5, *lres_iter), + createConstColumn(5, *lres_iter), executeFunction("tidbLTrim", createConstColumn>(5, *input_iter))); ASSERT_COLUMN_EQ( - createConstColumn>(5, *rres_iter), + createConstColumn(5, *rres_iter), executeFunction("tidbRTrim", createConstColumn>(5, *input_iter))); ASSERT_COLUMN_EQ( createConstColumn(5, *lres_iter), diff --git a/dbms/src/Functions/tests/gtest_strings_cmp.cpp b/dbms/src/Functions/tests/gtest_strings_cmp.cpp index ca9ae6562c7..7e94a7ed30e 100644 --- a/dbms/src/Functions/tests/gtest_strings_cmp.cpp +++ b/dbms/src/Functions/tests/gtest_strings_cmp.cpp @@ -43,7 +43,7 @@ try ASSERT_COLUMN_EQ(createColumn>({1, 0, -1, std::nullopt}), executeFunction("strcmp", {createConstColumn>(4, "b"), createColumn>({"a", "b", "c", std::nullopt})})); // constant with constant - ASSERT_COLUMN_EQ(createConstColumn>(1, -1), executeFunction("strcmp", {createConstColumn>(1, "a"), createConstColumn>(1, "b")})); + ASSERT_COLUMN_EQ(createConstColumn(1, -1), executeFunction("strcmp", {createConstColumn>(1, "a"), createConstColumn>(1, "b")})); // constant with nullable ASSERT_COLUMN_EQ(createColumn>({-1}), executeFunction("strcmp", {createColumn({"a"}), createColumn>({"b"})})); @@ -65,10 +65,10 @@ try ASSERT_COLUMN_EQ(createColumn>({-1, 1, 0, std::nullopt, std::nullopt}), executeFunction("strcmp", {createColumn>({"", "123", "", "", std::nullopt}), createColumn>({"123", "", "", std::nullopt, ""})}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); // column with constant - ASSERT_COLUMN_EQ(createColumn>({-1}), executeFunction("strcmp", {createColumn({"a"}), createConstColumn>(1, "b")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); - ASSERT_COLUMN_EQ(createColumn>({-1}), executeFunction("strcmp", {createColumn({"A"}), createConstColumn>(1, "b")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); - ASSERT_COLUMN_EQ(createColumn>({-1}), executeFunction("strcmp", {createColumn({"A"}), createConstColumn>(1, "B")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); - ASSERT_COLUMN_EQ(createColumn>({-1, 0, 1, 1}), executeFunction("strcmp", {createColumn({"A", "B", "C", "D"}), createConstColumn>(4, "B")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); + ASSERT_COLUMN_EQ(createColumn({-1}), executeFunction("strcmp", {createColumn({"a"}), createConstColumn>(1, "b")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); + ASSERT_COLUMN_EQ(createColumn({-1}), executeFunction("strcmp", {createColumn({"A"}), createConstColumn>(1, "b")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); + ASSERT_COLUMN_EQ(createColumn({-1}), executeFunction("strcmp", {createColumn({"A"}), createConstColumn>(1, "B")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); + ASSERT_COLUMN_EQ(createColumn({-1, 0, 1, 1}), executeFunction("strcmp", {createColumn({"A", "B", "C", "D"}), createConstColumn>(4, "B")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); // constant with column ASSERT_COLUMN_EQ(createColumn>({1, 0, -1, std::nullopt}), executeFunction("strcmp", {createConstColumn>(4, "b"), createColumn>({"a", "b", "c", std::nullopt})}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); @@ -76,10 +76,10 @@ try ASSERT_COLUMN_EQ(createColumn>({1, 0, -1, std::nullopt}), executeFunction("strcmp", {createConstColumn>(4, "b"), createColumn>({"A", "B", "C", std::nullopt})}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); // constant with constant - ASSERT_COLUMN_EQ(createConstColumn>(1, -1), executeFunction("strcmp", {createConstColumn>(1, "a"), createConstColumn>(1, "b")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); - ASSERT_COLUMN_EQ(createConstColumn>(1, -1), executeFunction("strcmp", {createConstColumn>(1, "A"), createConstColumn>(1, "b")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); - ASSERT_COLUMN_EQ(createConstColumn>(1, -1), executeFunction("strcmp", {createConstColumn>(1, "a"), createConstColumn>(1, "B")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); - ASSERT_COLUMN_EQ(createConstColumn>(1, -1), executeFunction("strcmp", {createConstColumn>(1, "A"), createConstColumn>(1, "B")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); + ASSERT_COLUMN_EQ(createConstColumn(1, -1), executeFunction("strcmp", {createConstColumn>(1, "a"), createConstColumn>(1, "b")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); + ASSERT_COLUMN_EQ(createConstColumn(1, -1), executeFunction("strcmp", {createConstColumn>(1, "A"), createConstColumn>(1, "b")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); + ASSERT_COLUMN_EQ(createConstColumn(1, -1), executeFunction("strcmp", {createConstColumn>(1, "a"), createConstColumn>(1, "B")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); + ASSERT_COLUMN_EQ(createConstColumn(1, -1), executeFunction("strcmp", {createConstColumn>(1, "A"), createConstColumn>(1, "B")}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); // constant with nullable ASSERT_COLUMN_EQ(createColumn>({-1}), executeFunction("strcmp", {createColumn({"a"}), createColumn>({"b"})}, TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI))); diff --git a/dbms/src/Functions/tests/gtest_strings_format.cpp b/dbms/src/Functions/tests/gtest_strings_format.cpp index 2adc17afb93..2d571a9bb1b 100644 --- a/dbms/src/Functions/tests/gtest_strings_format.cpp +++ b/dbms/src/Functions/tests/gtest_strings_format.cpp @@ -62,6 +62,8 @@ class StringFormat : public DB::tests::FunctionTest FieldType(static_cast(-9999999), 4), FieldType(static_cast(-3333330), 4)}), createConstColumn>(4, 3))); + /// known issue https://github.com/pingcap/tiflash/issues/4891 + /* ASSERT_COLUMN_EQ( createColumn>({"-999.9999", "-1,000", "-1,000", "-999.999900000000000000000000000000", "-999.99990", "-1,000.0", "-1,000.00"}), executeFunction( @@ -80,6 +82,7 @@ class StringFormat : public DB::tests::FunctionTest 1, FieldType(static_cast(-9999999), 4)), createConstColumn>(1, 3))); + */ ASSERT_COLUMN_EQ( createColumn>({"12,332.1000", "12,332", "12,332.300000000000000000000000000000", "-12,332.30000", "-1,000.0", "-333.33", {}}), executeFunction( @@ -105,6 +108,8 @@ class StringFormat : public DB::tests::FunctionTest FieldType(static_cast(-9999999), 4), FieldType(static_cast(-3333330), 4)}), createConstColumn>(4, 3))); + /// known issue https://github.com/pingcap/tiflash/issues/4891 + /* ASSERT_COLUMN_EQ( createColumn>({"-999.9999", "-1,000", "-999.999900000000000000000000000000", "-999.99990", "-1,000.0", "-1,000.00"}), executeFunction( @@ -123,6 +128,7 @@ class StringFormat : public DB::tests::FunctionTest 1, FieldType(static_cast(-9999999), 4)), createConstColumn>(1, 3))); + */ } template @@ -226,7 +232,7 @@ try createColumn>({4, 0, -1, 31, 5, 1, 2}))); /// const, const ASSERT_COLUMN_EQ( - createConstColumn>(1, "-1,000.000"), + createConstColumn(1, "-1,000.000"), executeFunction( func_name, createConstColumn>(1, -999.9999), @@ -256,7 +262,7 @@ try createColumn>({4, 0, 31, 5, 1, 2}))); /// const, const ASSERT_COLUMN_EQ( - createConstColumn>(1, "-1,000.000"), + createConstColumn(1, "-1,000.000"), executeFunction( func_name, createConstColumn>(1, -999.9999), @@ -265,7 +271,7 @@ try /// float32, int /// const, const ASSERT_COLUMN_EQ( - createConstColumn>(1, "12.123"), + createConstColumn(1, "12.123"), executeFunction( func_name, createConstColumn>(1, 12.1235), diff --git a/dbms/src/Functions/tests/gtest_strings_right.cpp b/dbms/src/Functions/tests/gtest_strings_right.cpp index 1dbe8c7d5e3..210cb651ec6 100644 --- a/dbms/src/Functions/tests/gtest_strings_right.cpp +++ b/dbms/src/Functions/tests/gtest_strings_right.cpp @@ -54,7 +54,8 @@ class StringRightTest : public DB::tests::FunctionTest auto inner_test = [&](bool is_str_const, bool is_length_const) { bool is_one_of_args_null_const = (is_str_const && !str.has_value()) || (is_length_const && !length.has_value()); bool is_result_const = (is_str_const && is_length_const) || is_one_of_args_null_const; - auto expected_res_column = is_result_const ? createConstColumn>(1, result) : createColumn>({result}); + auto expected_res_column = is_result_const ? (is_one_of_args_null_const ? createConstColumn>(1, result) : createConstColumn(1, result.value())) + : createColumn>({result}); auto str_column = is_str_const ? createConstColumn>(1, str) : createColumn>({str}); auto length_column = is_length_const ? createConstColumn>(1, length) : createColumn>({length}); auto actual_res_column = executeFunction(func_name, str_column, length_column); diff --git a/dbms/src/Functions/tests/gtest_strings_search.cpp b/dbms/src/Functions/tests/gtest_strings_search.cpp index 58bf1b34487..544ebc34df6 100644 --- a/dbms/src/Functions/tests/gtest_strings_search.cpp +++ b/dbms/src/Functions/tests/gtest_strings_search.cpp @@ -24,6 +24,7 @@ namespace tests class StringMatch : public FunctionTest { protected: + const String func_name = "like3Args"; const String long_str = "abcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzab" "cdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdef" "ghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyzabcdefghijkl" @@ -33,6 +34,8 @@ class StringMatch : public FunctionTest const String long_pattern = "abcdefghijklmnopqrstuvwxyz_bcdefghijklmnopqrstuvwxyzabcdefghijklmnopqrstuvwxyz%abcdefghijklmnopqrstuvwxyz"; + ColumnWithTypeAndName escape = createConstColumn(1, static_cast('\\')); + static ColumnWithTypeAndName toNullableVec(const std::vector> & v) { return createColumn>(v); @@ -122,10 +125,9 @@ try auto haystack = createColumn(haystack_raw, "haystack"); auto needle = createColumn(needle_raw, "needle"); - auto escape = createConstColumn(1, static_cast('\\')); auto expected = createColumn(result_raw, "result"); - auto result = executeFunction("like3Args", {haystack, needle, escape}); + auto result = executeFunction(func_name, {haystack, needle, escape}); ASSERT_COLUMN_EQ(expected, result); @@ -155,10 +157,9 @@ try auto nullable_haystack = createColumn>(nullable_haystack_raw, "haystack"); auto nullable_needle = createColumn>(nullable_needle_raw, "needle"); - auto nullable_escape = createConstColumn>(1, static_cast('\\')); auto nullable_expected = createColumn>(nullable_result_raw, "result"); - auto nullable_result = executeFunction("like3Args", {nullable_haystack, nullable_needle, nullable_escape}); + auto nullable_result = executeFunction(func_name, {nullable_haystack, nullable_needle, escape}); ASSERT_COLUMN_EQ(nullable_expected, nullable_result); } @@ -196,10 +197,9 @@ try auto haystack = createConstColumn>(1, cas.src); auto needle = createColumn>(needle_raw); - auto escape = createConstColumn>(1, static_cast('\\')); auto expected = createColumn>(result_raw); - auto result = executeFunction("like3Args", {haystack, needle, escape}); + auto result = executeFunction(func_name, {haystack, needle, escape}); ASSERT_COLUMN_EQ(expected, result); } } @@ -213,16 +213,18 @@ TEST_F(StringMatch, LikeVectorWithVector) ASSERT_COLUMN_EQ( toNullableVec(expect), executeFunction( - "like", + func_name, toNullableVec(haystack), - toNullableVec(needle))); + toNullableVec(needle), + escape)); ASSERT_COLUMN_EQ( toVec(expect), executeFunction( - "like", + func_name, toVec(haystack), - toVec(needle))); + toVec(needle), + escape)); std::vector> haystack_null = {{}, "a"}; std::vector> needle_null = {"我_tif%", {}}; @@ -230,9 +232,10 @@ TEST_F(StringMatch, LikeVectorWithVector) ASSERT_COLUMN_EQ( toNullableVec(expect_null), executeFunction( - "like", + func_name, toNullableVec(haystack_null), - toNullableVec(needle_null))); + toNullableVec(needle_null), + escape)); } TEST_F(StringMatch, LikeConstWithVector) @@ -243,32 +246,36 @@ TEST_F(StringMatch, LikeConstWithVector) ASSERT_COLUMN_EQ( toNullableVec(expect), executeFunction( - "like", + func_name, toConst("abcaba"), - toNullableVec(needle))); + toNullableVec(needle), + escape)); ASSERT_COLUMN_EQ( toVec(expect), executeFunction( - "like", + func_name, toConst("abcaba"), - toVec(needle))); + toVec(needle), + escape)); ASSERT_COLUMN_EQ( toVec(expect1), executeFunction( - "like", + func_name, toConst(long_str), - toVec(needle))); + toVec(needle), + escape)); std::vector> needle_null = {{}}; std::vector> expect_null = {{}}; ASSERT_COLUMN_EQ( toNullableVec(expect_null), executeFunction( - "like", + func_name, toConst("abc"), - toNullableVec(needle_null))); + toNullableVec(needle_null), + escape)); } TEST_F(StringMatch, LikeVectorWithConst) @@ -281,46 +288,52 @@ TEST_F(StringMatch, LikeVectorWithConst) ASSERT_COLUMN_EQ( toNullableVec(expect), executeFunction( - "like", + func_name, toNullableVec(haystack), - toConst("%aa%"))); + toConst("%aa%"), + escape)); ASSERT_COLUMN_EQ( toVec(expect), executeFunction( - "like", + func_name, toVec(haystack), - toConst("%aa%"))); + toConst("%aa%"), + escape)); ASSERT_COLUMN_EQ( toVec(expect1), executeFunction( - "like", + func_name, toVec(haystack), - toConst("%爱tif%"))); + toConst("%爱tif%"), + escape)); ASSERT_COLUMN_EQ( toVec(expect2), executeFunction( - "like", + func_name, toVec(haystack), - toConst("%不爱tif%"))); + toConst("%不爱tif%"), + escape)); ASSERT_COLUMN_EQ( toVec(expect3), executeFunction( - "like", + func_name, toVec(haystack), - toConst(long_pattern))); + toConst(long_pattern), + escape)); std::vector> haystack_null = {{}}; std::vector> expect_null = {{}}; ASSERT_COLUMN_EQ( toNullableVec(expect_null), executeFunction( - "like", + func_name, toNullableVec(haystack_null), - toConst("abc"))); + toConst("abc"), + escape)); } TEST_F(StringMatch, LikeConstWithConst) @@ -328,30 +341,34 @@ TEST_F(StringMatch, LikeConstWithConst) ASSERT_COLUMN_EQ( toConst(1), executeFunction( - "like", + func_name, toConst("resaasfe"), - toConst("%aa%"))); + toConst("%aa%"), + escape)); ASSERT_COLUMN_EQ( toConst(0), executeFunction( - "like", + func_name, toConst("abcde"), - toConst("%aa%"))); + toConst("%aa%"), + escape)); ASSERT_COLUMN_EQ( toConst(1), executeFunction( - "like", + func_name, toConst("我爱tiflash"), - toConst("%爱tif%"))); + toConst("%爱tif%"), + escape)); ASSERT_COLUMN_EQ( toConst(0), executeFunction( - "like", + func_name, toConst("我爱tiflash"), - toConst("%不爱tif%"))); + toConst("%不爱tif%"), + escape)); } } // namespace tests diff --git a/dbms/src/Functions/tests/gtest_strings_tidb_concat.cpp b/dbms/src/Functions/tests/gtest_strings_tidb_concat.cpp index a0c06a5d6a8..24295bc079b 100644 --- a/dbms/src/Functions/tests/gtest_strings_tidb_concat.cpp +++ b/dbms/src/Functions/tests/gtest_strings_tidb_concat.cpp @@ -28,6 +28,7 @@ class StringTidbConcat : public DB::tests::FunctionTest static constexpr auto func_name = "tidbConcat"; using Type = Nullable; + using NotNullType = String; InferredDataVector test_strings = {"", "www.pingcap", "中文.测.试。。。", {}}; }; @@ -45,7 +46,7 @@ try createColumn({value}))); // const ASSERT_COLUMN_EQ( - createConstColumn(1, value), + value.has_value() ? createConstColumn(1, value.value()) : createConstColumn(1, value), executeFunction( StringTidbConcat::func_name, createConstColumn(1, value))); @@ -64,7 +65,7 @@ try // all args is const or has only null const auto is_result_const = (is_value1_const && is_value2_const) || (!value1.has_value() && is_value1_const) || (!value2.has_value() && is_value2_const); ASSERT_COLUMN_EQ( - is_result_const ? createConstColumn(1, result) : createColumn({result}), + is_result_const ? (is_result_not_null ? createConstColumn(1, result.value()) : createConstColumn(1, result)) : createColumn({result}), executeFunction( StringTidbConcat::func_name, is_value1_const ? createConstColumn(1, value1) : createColumn({value1}), diff --git a/dbms/src/Functions/tests/gtest_strings_trim.cpp b/dbms/src/Functions/tests/gtest_strings_trim.cpp index be2616c4818..55c4063abb5 100644 --- a/dbms/src/Functions/tests/gtest_strings_trim.cpp +++ b/dbms/src/Functions/tests/gtest_strings_trim.cpp @@ -600,37 +600,37 @@ try { // trim(const) ASSERT_COLUMN_EQ( - createConstColumn>(5, "x"), + createConstColumn(5, "x"), executeFunction("tidbTrim", createConstColumn>(5, " x "))); // trim(const from const) ASSERT_COLUMN_EQ( - createConstColumn>(5, "a"), + createConstColumn(5, "a"), executeFunction("tidbTrim", createConstColumn>(5, "xax"), createConstColumn>(5, "x"))); // trim(leading|trailing|both const from const) ASSERT_COLUMN_EQ( - createConstColumn>(5, "a"), + createConstColumn(5, "a"), executeFunction("tidbTrim", createConstColumn>(5, "xax"), createConstColumn>(5, "x"), createConstColumn>(5, 0))); ASSERT_COLUMN_EQ( - createConstColumn>(5, "a"), + createConstColumn(5, "a"), executeFunction("tidbTrim", createConstColumn>(5, "xax"), createConstColumn>(5, "x"), createConstColumn>(5, 1))); ASSERT_COLUMN_EQ( - createConstColumn>(5, "ax"), + createConstColumn(5, "ax"), executeFunction("tidbTrim", createConstColumn>(5, "xax"), createConstColumn>(5, "x"), createConstColumn>(5, 2))); ASSERT_COLUMN_EQ( - createConstColumn>(5, "xa"), + createConstColumn(5, "xa"), executeFunction("tidbTrim", createConstColumn>(5, "xax"), createConstColumn>(5, "x"), @@ -955,7 +955,7 @@ try res_itr++, input_itr++) { ASSERT_COLUMN_EQ( - createConstColumn>(5, *res_itr), + input_itr->has_value() ? createConstColumn(5, res_itr->value()) : createConstColumn>(5, *res_itr), executeFunction("tidbTrim", createConstColumn>(5, *input_itr), createConstColumn>(5, "啊啊"), diff --git a/dbms/src/Functions/tests/gtest_substring.cpp b/dbms/src/Functions/tests/gtest_substring.cpp index 1506b3340a6..374e38970f1 100644 --- a/dbms/src/Functions/tests/gtest_substring.cpp +++ b/dbms/src/Functions/tests/gtest_substring.cpp @@ -41,7 +41,7 @@ try createConstColumn>(4, 4))); // const, const, const ASSERT_COLUMN_EQ( - createConstColumn>(1, "www."), + createConstColumn(1, "www."), executeFunction( "substringUTF8", createConstColumn>(1, "www.pingcap.com"), diff --git a/dbms/src/Functions/tests/gtest_substring_index.cpp b/dbms/src/Functions/tests/gtest_substring_index.cpp index a8208b45042..9bb4675c329 100644 --- a/dbms/src/Functions/tests/gtest_substring_index.cpp +++ b/dbms/src/Functions/tests/gtest_substring_index.cpp @@ -75,7 +75,9 @@ class SubstringIndexTest : public DB::tests::FunctionTest auto inner_test = [&](bool is_str_const, bool is_delim_const, bool is_count_const) { bool is_one_of_args_null_const = (is_str_const && !str.has_value()) || (is_delim_const && !delim.has_value()) || (is_count_const && !count.has_value()); bool is_result_const = (is_str_const && is_delim_const && is_count_const) || is_one_of_args_null_const; - auto expected_res_column = is_result_const ? createConstColumn>(1, result) : createColumn>({result}); + if (is_result_const && !is_one_of_args_null_const && !result.has_value()) + throw Exception("Should not reach here"); + auto expected_res_column = is_result_const ? (is_one_of_args_null_const ? createConstColumn>(1, result) : createConstColumn(1, result.value())) : createColumn>({result}); auto str_column = is_str_const ? createConstColumn>(1, str) : createColumn>({str}); auto delim_column = is_delim_const ? createConstColumn>(1, delim) : createColumn>({delim}); auto count_column = is_count_const ? createConstColumn>(1, count) : createColumn>({count}); diff --git a/dbms/src/Functions/tests/gtest_unix_timestamp.cpp b/dbms/src/Functions/tests/gtest_unix_timestamp.cpp index 1fa02141c4c..15909a90af9 100644 --- a/dbms/src/Functions/tests/gtest_unix_timestamp.cpp +++ b/dbms/src/Functions/tests/gtest_unix_timestamp.cpp @@ -86,16 +86,16 @@ try executeFunction(func_name_dec, createConstColumn(std::make_tuple(6), 10, date_time_with_fsp_data[0]))); /// case 2, func(nullable(not null const)) ASSERT_COLUMN_EQ( - createConstColumn>(10, date_result[0]), + createConstColumn(10, date_result[0]), executeFunction(func_name_int, createConstColumn>(10, date_data[0]))); ASSERT_COLUMN_EQ( - createConstColumn>(10, date_time_int_result[0]), + createConstColumn(10, date_time_int_result[0]), executeFunction(func_name_int, createConstColumn>(std::make_tuple(0), 10, date_time_data[0]))); ASSERT_COLUMN_EQ( - createConstColumn>(std::make_tuple(12, 0), 10, date_time_decimal_result[0]), + createConstColumn(std::make_tuple(12, 0), 10, date_time_decimal_result[0]), executeFunction(func_name_dec, createConstColumn>(std::make_tuple(0), 10, date_time_data[0]))); ASSERT_COLUMN_EQ( - createConstColumn>(std::make_tuple(18, 6), 10, date_time_with_fsp_result[0]), + createConstColumn(std::make_tuple(18, 6), 10, date_time_with_fsp_result[0]), executeFunction(func_name_dec, createConstColumn>(std::make_tuple(6), 10, date_time_with_fsp_data[0]))); /// case 3, func(nullable(null const)) ASSERT_COLUMN_EQ( diff --git a/dbms/src/TestUtils/FunctionTestUtils.h b/dbms/src/TestUtils/FunctionTestUtils.h index 615a58ebda5..7704c69a89f 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.h +++ b/dbms/src/TestUtils/FunctionTestUtils.h @@ -536,7 +536,7 @@ ColumnWithTypeAndName executeFunction( const String & func_name, const ColumnsWithTypeAndName & columns, const TiDB::TiDBCollatorPtr & collator = nullptr, - bool raw_function_test = true); + bool raw_function_test = false); ColumnWithTypeAndName executeFunction( Context & context, @@ -544,7 +544,7 @@ ColumnWithTypeAndName executeFunction( const ColumnNumbers & argument_column_numbers, const ColumnsWithTypeAndName & columns, const TiDB::TiDBCollatorPtr & collator = nullptr, - bool raw_function_test = true); + bool raw_function_test = false); template ColumnWithTypeAndName executeFunction( @@ -562,7 +562,7 @@ DataTypePtr getReturnTypeForFunction( const String & func_name, const ColumnsWithTypeAndName & columns, const TiDB::TiDBCollatorPtr & collator = nullptr, - bool raw_function_test = true); + bool raw_function_test = false); template ColumnWithTypeAndName createNullableColumn(InferredDataVector init_vec, const std::vector & null_map, const String name = "") @@ -687,7 +687,7 @@ class FunctionTest : public ::testing::Test const String & func_name, const ColumnsWithTypeAndName & columns, const TiDB::TiDBCollatorPtr & collator = nullptr, - bool raw_function_test = true) + bool raw_function_test = false) { return DB::tests::executeFunction(context, func_name, columns, collator, raw_function_test); } @@ -704,7 +704,7 @@ class FunctionTest : public ::testing::Test const ColumnNumbers & argument_column_numbers, const ColumnsWithTypeAndName & columns, const TiDB::TiDBCollatorPtr & collator = nullptr, - bool raw_function_test = true) + bool raw_function_test = false) { return DB::tests::executeFunction(context, func_name, argument_column_numbers, columns, collator, raw_function_test); } From 187a591269fb2dc0c3fd81b1f3aeb671963dc8c8 Mon Sep 17 00:00:00 2001 From: hehechen Date: Wed, 1 Jun 2022 21:14:27 +0800 Subject: [PATCH 015/104] Tiflash pagectl support encrypted data (#5003) close pingcap/tiflash#4962 --- dbms/src/Server/CLIService.h | 237 ++++++++++++++++++ dbms/src/Server/CMakeLists.txt | 4 + dbms/src/Server/DTTool/DTTool.h | 200 +-------------- dbms/src/Server/config_tools.h.in | 1 + dbms/src/Server/main.cpp | 6 + dbms/src/Storages/Page/CMakeLists.txt | 1 + dbms/src/Storages/Page/V2/CMakeLists.txt | 14 +- dbms/src/Storages/Page/V2/gc/DataCompactor.h | 7 +- dbms/src/Storages/Page/V3/BlobStore.h | 2 +- dbms/src/Storages/Page/V3/PageDirectory.h | 4 +- dbms/src/Storages/Page/V3/PageStorageImpl.h | 2 +- .../src/Storages/Page/V3/tests/CMakeLists.txt | 8 +- dbms/src/Storages/Page/tools/CMakeLists.txt | 17 ++ .../Page/tools/PageCtl/CMakeLists.txt | 24 ++ dbms/src/Storages/Page/tools/PageCtl/Main.cpp | 20 ++ .../Storages/Page/tools/PageCtl/MainEntry.cpp | 69 +++++ .../Page/tools/PageCtl/PageStorageCtl.h | 24 ++ .../PageCtl/PageStorageCtlV2.cpp} | 33 +-- .../Page/tools/PageCtl/PageStorageCtlV2.h | 17 ++ .../PageCtl/PageStorageCtlV3.cpp} | 86 +++++-- .../Page/tools/PageCtl/PageStorageCtlV3.h | 17 ++ 21 files changed, 523 insertions(+), 270 deletions(-) create mode 100644 dbms/src/Server/CLIService.h create mode 100644 dbms/src/Storages/Page/tools/CMakeLists.txt create mode 100644 dbms/src/Storages/Page/tools/PageCtl/CMakeLists.txt create mode 100644 dbms/src/Storages/Page/tools/PageCtl/Main.cpp create mode 100644 dbms/src/Storages/Page/tools/PageCtl/MainEntry.cpp create mode 100644 dbms/src/Storages/Page/tools/PageCtl/PageStorageCtl.h rename dbms/src/Storages/Page/{V2/tests/page_storage_ctl.cpp => tools/PageCtl/PageStorageCtlV2.cpp} (93%) create mode 100644 dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV2.h rename dbms/src/Storages/Page/{V3/tests/page_storage_ctl.cpp => tools/PageCtl/PageStorageCtlV3.cpp} (87%) create mode 100644 dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV3.h diff --git a/dbms/src/Server/CLIService.h b/dbms/src/Server/CLIService.h new file mode 100644 index 00000000000..18c9d61260f --- /dev/null +++ b/dbms/src/Server/CLIService.h @@ -0,0 +1,237 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include +#include +using RaftStoreFFIFunc = void (*)(int argc, const char * const * argv, const DB::EngineStoreServerHelper *); + +template +struct CLIService : public BaseDaemon +{ + struct TiFlashProxyConfig + { + static const std::string config_prefix; + std::vector args; + std::unordered_map val_map; + bool is_proxy_runnable = false; + + static constexpr char ENGINE_STORE_VERSION[] = "engine-version"; + static constexpr char ENGINE_STORE_GIT_HASH[] = "engine-git-hash"; + static constexpr char ENGINE_STORE_ADDRESS[] = "engine-addr"; + static constexpr char ENGINE_STORE_ADVERTISE_ADDRESS[] = "advertise-engine-addr"; + static constexpr char PD_ENDPOINTS[] = "pd-endpoints"; + static constexpr char ENGINE_LABEL[] = "engine-label"; + static constexpr char ENGINE_LABEL_VALUE[] = "tiflash"; + + explicit TiFlashProxyConfig(Poco::Util::LayeredConfiguration & config); + }; + + struct RaftStoreProxyRunner : boost::noncopyable + { + struct RunRaftStoreProxyParms + { + const DB::EngineStoreServerHelper * helper; + const TiFlashProxyConfig & conf; + const RaftStoreFFIFunc ffi_function; + + /// set big enough stack size to avoid runtime error like stack-overflow. + size_t stack_size = 1024 * 1024 * 20; + }; + + explicit RaftStoreProxyRunner(RunRaftStoreProxyParms && parms_); + + void join(); + + void run(); + + private: + static void * runRaftStoreProxyFfi(void * pv); + + private: + RunRaftStoreProxyParms parms; + pthread_t thread; + }; + + Func func; + RaftStoreFFIFunc ffi_function; + const Args & args; + std::unique_ptr global_context; + + explicit CLIService(Func func_, const Args & args_, const std::string & config_file, RaftStoreFFIFunc ffi_function = nullptr); + + int main(const std::vector &) override; +}; + +template +CLIService::TiFlashProxyConfig::TiFlashProxyConfig(Poco::Util::LayeredConfiguration & config) +{ + if (!config.has(config_prefix)) + return; + + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_prefix, keys); + { + std::unordered_map args_map; + for (const auto & key : keys) + { + const auto k = config_prefix + "." + key; + args_map[key] = config.getString(k); + } + args_map[PD_ENDPOINTS] = config.getString("raft.pd_addr"); + args_map[ENGINE_STORE_VERSION] = TiFlashBuildInfo::getReleaseVersion(); + args_map[ENGINE_STORE_GIT_HASH] = TiFlashBuildInfo::getGitHash(); + if (!args_map.count(ENGINE_STORE_ADDRESS)) + args_map[ENGINE_STORE_ADDRESS] = config.getString("flash.service_addr"); + else + args_map[ENGINE_STORE_ADVERTISE_ADDRESS] = args_map[ENGINE_STORE_ADDRESS]; + args_map[ENGINE_LABEL] = ENGINE_LABEL_VALUE; + + for (auto && [k, v] : args_map) + { + val_map.emplace("--" + k, std::move(v)); + } + } + + args.push_back("TiFlash Proxy"); + for (const auto & v : val_map) + { + args.push_back(v.first.data()); + args.push_back(v.second.data()); + } + is_proxy_runnable = true; +} +template +CLIService::RaftStoreProxyRunner::RaftStoreProxyRunner(CLIService::RaftStoreProxyRunner::RunRaftStoreProxyParms && parms_) + : parms(std::move(parms_)) +{} +template +void CLIService::RaftStoreProxyRunner::join() +{ + if (!parms.conf.is_proxy_runnable) + return; + pthread_join(thread, nullptr); +} +template +void CLIService::RaftStoreProxyRunner::run() +{ + if (!parms.conf.is_proxy_runnable) + return; + pthread_attr_t attribute; + pthread_attr_init(&attribute); + pthread_attr_setstacksize(&attribute, parms.stack_size); + pthread_create(&thread, &attribute, runRaftStoreProxyFfi, &parms); + pthread_attr_destroy(&attribute); +} +template +void * CLIService::RaftStoreProxyRunner::runRaftStoreProxyFfi(void * pv) +{ + auto & parms = *static_cast(pv); + if (nullptr == parms.ffi_function) + { + throw DB::Exception("proxy is not available"); + } + parms.ffi_function(static_cast(parms.conf.args.size()), parms.conf.args.data(), parms.helper); + return nullptr; +} + +template +CLIService::CLIService(Func func_, const Args & args_, const std::string & config_file, RaftStoreFFIFunc ffi_function) + : func(std::move(func_)) + , ffi_function(ffi_function) + , args(args_) +{ + config_path = config_file; + ConfigProcessor config_processor(config_file); + auto loaded_config = config_processor.loadConfig(); + BaseDaemon::config().add(loaded_config.configuration); + BaseDaemon::config().setString("config-file", config_file); +} + +template +int CLIService::main(const std::vector &) +{ + using namespace DB; + TiFlashProxyConfig proxy_conf(config()); + EngineStoreServerWrap tiflash_instance_wrap{}; + auto helper = GetEngineStoreServerHelper( + &tiflash_instance_wrap); + + typename RaftStoreProxyRunner::RunRaftStoreProxyParms parms{&helper, proxy_conf, ffi_function}; + RaftStoreProxyRunner proxy_runner(std::move(parms)); + + proxy_runner.run(); + + if (proxy_conf.is_proxy_runnable) + { + while (!tiflash_instance_wrap.proxy_helper) + std::this_thread::sleep_for(std::chrono::milliseconds(200)); + } + + SCOPE_EXIT({ + if (!proxy_conf.is_proxy_runnable) + { + proxy_runner.join(); + return; + } + tiflash_instance_wrap.status = EngineStoreServerStatus::Terminated; + tiflash_instance_wrap.tmt = nullptr; + proxy_runner.join(); + }); + + global_context = std::make_unique(Context::createGlobal()); + global_context->setGlobalContext(*global_context); + global_context->setApplicationType(Context::ApplicationType::SERVER); + + /// Init File Provider + if (proxy_conf.is_proxy_runnable) + { + bool enable_encryption = tiflash_instance_wrap.proxy_helper->checkEncryptionEnabled(); + if (enable_encryption) + { + auto method = tiflash_instance_wrap.proxy_helper->getEncryptionMethod(); + enable_encryption = (method != EncryptionMethod::Plaintext); + } + KeyManagerPtr key_manager = std::make_shared(&tiflash_instance_wrap); + global_context->initializeFileProvider(key_manager, enable_encryption); + } + else + { + KeyManagerPtr key_manager = std::make_shared(false); + global_context->initializeFileProvider(key_manager, false); + } + + return func(*global_context, args); +} + + +template +inline const std::string CLIService::TiFlashProxyConfig::config_prefix = "flash.proxy"; diff --git a/dbms/src/Server/CMakeLists.txt b/dbms/src/Server/CMakeLists.txt index 6c3d289dea6..63cf6d0e1f9 100644 --- a/dbms/src/Server/CMakeLists.txt +++ b/dbms/src/Server/CMakeLists.txt @@ -22,6 +22,7 @@ option(ENABLE_CLICKHOUSE_SERVER "Enable server" ${ENABLE_CLICKHOUSE_ALL}) option(ENABLE_CLICKHOUSE_CLIENT "Enable client" ${ENABLE_CLICKHOUSE_ALL}) option(ENABLE_TIFLASH_DTTOOL "Enable dttool: tools to manage dmfile" ${ENABLE_CLICKHOUSE_ALL}) option(ENABLE_TIFLASH_DTWORKLOAD "Enable dtworkload: tools to test and stress DeltaTree" ${ENABLE_CLICKHOUSE_ALL}) +option(ENABLE_TIFLASH_PAGECTL "Enable pagectl: tools to debug page storage" ${ENABLE_CLICKHOUSE_ALL}) configure_file (config_tools.h.in ${CMAKE_CURRENT_BINARY_DIR}/config_tools.h) @@ -135,6 +136,9 @@ endif () if (ENABLE_TIFLASH_DTWORKLOAD) target_link_libraries(tiflash dt-workload-lib) endif () +if (ENABLE_TIFLASH_PAGECTL) + target_link_libraries(tiflash page-ctl-lib) +endif () # install always because depian package want this files: install (TARGETS tiflash RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT tiflash) diff --git a/dbms/src/Server/DTTool/DTTool.h b/dbms/src/Server/DTTool/DTTool.h index 911c29bf98b..6236bd6cdb9 100644 --- a/dbms/src/Server/DTTool/DTTool.h +++ b/dbms/src/Server/DTTool/DTTool.h @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -76,205 +77,6 @@ int migrateEntry(const std::vector & opts, RaftStoreFFIFunc ffi_fun namespace DTTool { -template -struct CLIService : public BaseDaemon -{ - struct TiFlashProxyConfig - { - static const std::string config_prefix; - std::vector args; - std::unordered_map val_map; - bool is_proxy_runnable = false; - - static constexpr char ENGINE_STORE_VERSION[] = "engine-version"; - static constexpr char ENGINE_STORE_GIT_HASH[] = "engine-git-hash"; - static constexpr char ENGINE_STORE_ADDRESS[] = "engine-addr"; - static constexpr char ENGINE_STORE_ADVERTISE_ADDRESS[] = "advertise-engine-addr"; - static constexpr char PD_ENDPOINTS[] = "pd-endpoints"; - - explicit TiFlashProxyConfig(Poco::Util::LayeredConfiguration & config); - }; - - struct RaftStoreProxyRunner : boost::noncopyable - { - struct RunRaftStoreProxyParms - { - const DB::EngineStoreServerHelper * helper; - const TiFlashProxyConfig & conf; - const RaftStoreFFIFunc ffi_function; - - /// set big enough stack size to avoid runtime error like stack-overflow. - size_t stack_size = 1024 * 1024 * 20; - }; - - explicit RaftStoreProxyRunner(RunRaftStoreProxyParms && parms_); - - void join(); - - void run(); - - private: - static void * runRaftStoreProxyFfi(void * pv); - - private: - RunRaftStoreProxyParms parms; - pthread_t thread; - }; - - Func func; - RaftStoreFFIFunc ffi_function; - const Args & args; - std::unique_ptr global_context; - - explicit CLIService(Func func_, const Args & args_, const std::string & config_file, RaftStoreFFIFunc ffi_function = nullptr); - - int main(const std::vector &) override; -}; - -template -CLIService::TiFlashProxyConfig::TiFlashProxyConfig(Poco::Util::LayeredConfiguration & config) -{ - if (!config.has(config_prefix)) - return; - - Poco::Util::AbstractConfiguration::Keys keys; - config.keys(config_prefix, keys); - { - std::unordered_map args_map; - for (const auto & key : keys) - { - const auto k = config_prefix + "." + key; - args_map[key] = config.getString(k); - } - args_map[PD_ENDPOINTS] = config.getString("raft.pd_addr"); - args_map[ENGINE_STORE_VERSION] = TiFlashBuildInfo::getReleaseVersion(); - args_map[ENGINE_STORE_GIT_HASH] = TiFlashBuildInfo::getGitHash(); - if (!args_map.count(ENGINE_STORE_ADDRESS)) - args_map[ENGINE_STORE_ADDRESS] = config.getString("flash.service_addr"); - else - args_map[ENGINE_STORE_ADVERTISE_ADDRESS] = args_map[ENGINE_STORE_ADDRESS]; - - for (auto && [k, v] : args_map) - { - val_map.emplace("--" + k, std::move(v)); - } - } - - args.push_back("TiFlash Proxy"); - for (const auto & v : val_map) - { - args.push_back(v.first.data()); - args.push_back(v.second.data()); - } - is_proxy_runnable = true; -} -template -CLIService::RaftStoreProxyRunner::RaftStoreProxyRunner(CLIService::RaftStoreProxyRunner::RunRaftStoreProxyParms && parms_) - : parms(std::move(parms_)) -{} -template -void CLIService::RaftStoreProxyRunner::join() -{ - if (!parms.conf.is_proxy_runnable) - return; - pthread_join(thread, nullptr); -} -template -void CLIService::RaftStoreProxyRunner::run() -{ - if (!parms.conf.is_proxy_runnable) - return; - pthread_attr_t attribute; - pthread_attr_init(&attribute); - pthread_attr_setstacksize(&attribute, parms.stack_size); - pthread_create(&thread, &attribute, runRaftStoreProxyFfi, &parms); - pthread_attr_destroy(&attribute); -} -template -void * CLIService::RaftStoreProxyRunner::runRaftStoreProxyFfi(void * pv) -{ - auto & parms = *static_cast(pv); - if (nullptr == parms.ffi_function) - { - throw DB::Exception("proxy is not available"); - } - parms.ffi_function(static_cast(parms.conf.args.size()), parms.conf.args.data(), parms.helper); - return nullptr; -} - -template -CLIService::CLIService(Func func_, const Args & args_, const std::string & config_file, RaftStoreFFIFunc ffi_function) - : func(std::move(func_)) - , ffi_function(ffi_function) - , args(args_) -{ - config_path = config_file; - ConfigProcessor config_processor(config_file); - auto loaded_config = config_processor.loadConfig(); - BaseDaemon::config().add(loaded_config.configuration); - BaseDaemon::config().setString("config-file", config_file); -} - -template -int CLIService::main(const std::vector &) -{ - using namespace DB; - TiFlashProxyConfig proxy_conf(config()); - EngineStoreServerWrap tiflash_instance_wrap{}; - auto helper = GetEngineStoreServerHelper( - &tiflash_instance_wrap); - - typename RaftStoreProxyRunner::RunRaftStoreProxyParms parms{&helper, proxy_conf, ffi_function}; - RaftStoreProxyRunner proxy_runner(std::move(parms)); - - proxy_runner.run(); - - if (proxy_conf.is_proxy_runnable) - { - while (!tiflash_instance_wrap.proxy_helper) - std::this_thread::sleep_for(std::chrono::milliseconds(200)); - } - - SCOPE_EXIT({ - if (!proxy_conf.is_proxy_runnable) - { - proxy_runner.join(); - return; - } - tiflash_instance_wrap.status = EngineStoreServerStatus::Terminated; - tiflash_instance_wrap.tmt = nullptr; - proxy_runner.join(); - }); - - global_context = std::make_unique(Context::createGlobal()); - global_context->setGlobalContext(*global_context); - global_context->setApplicationType(Context::ApplicationType::SERVER); - - /// Init File Provider - if (proxy_conf.is_proxy_runnable) - { - bool enable_encryption = tiflash_instance_wrap.proxy_helper->checkEncryptionEnabled(); - if (enable_encryption) - { - auto method = tiflash_instance_wrap.proxy_helper->getEncryptionMethod(); - enable_encryption = (method != EncryptionMethod::Plaintext); - } - KeyManagerPtr key_manager = std::make_shared(&tiflash_instance_wrap); - global_context->initializeFileProvider(key_manager, enable_encryption); - } - else - { - KeyManagerPtr key_manager = std::make_shared(false); - global_context->initializeFileProvider(key_manager, false); - } - - return func(*global_context, args); -} - - -template -inline const std::string CLIService::TiFlashProxyConfig::config_prefix = "flash.proxy"; - namespace detail { using namespace DB; diff --git a/dbms/src/Server/config_tools.h.in b/dbms/src/Server/config_tools.h.in index 03df94cc8e1..61aa3f41591 100644 --- a/dbms/src/Server/config_tools.h.in +++ b/dbms/src/Server/config_tools.h.in @@ -6,3 +6,4 @@ #cmakedefine01 ENABLE_CLICKHOUSE_CLIENT #cmakedefine01 ENABLE_TIFLASH_DTTOOL #cmakedefine01 ENABLE_TIFLASH_DTWORKLOAD +#cmakedefine01 ENABLE_TIFLASH_PAGECTL diff --git a/dbms/src/Server/main.cpp b/dbms/src/Server/main.cpp index ace9dfc80ee..11cccf84729 100644 --- a/dbms/src/Server/main.cpp +++ b/dbms/src/Server/main.cpp @@ -38,6 +38,9 @@ #if ENABLE_TIFLASH_DTWORKLOAD #include #endif +#if ENABLE_TIFLASH_PAGECTL +#include +#endif #include #include @@ -103,6 +106,9 @@ std::pair clickhouse_applications[] = { #endif #if ENABLE_TIFLASH_DTWORKLOAD {"dtworkload", DB::DM::tests::DTWorkload::mainEntry}, +#endif +#if ENABLE_TIFLASH_PAGECTL + {"pagectl", DB::PageStorageCtl::mainEntry}, #endif {"version", mainEntryVersion}, {"errgen", mainExportError}}; diff --git a/dbms/src/Storages/Page/CMakeLists.txt b/dbms/src/Storages/Page/CMakeLists.txt index 1883f0bc0aa..cead83fa126 100644 --- a/dbms/src/Storages/Page/CMakeLists.txt +++ b/dbms/src/Storages/Page/CMakeLists.txt @@ -13,6 +13,7 @@ # limitations under the License. add_subdirectory(V2) +add_subdirectory(tools) # PageStorage Stress test if (ENABLE_V3_PAGESTORAGE) diff --git a/dbms/src/Storages/Page/V2/CMakeLists.txt b/dbms/src/Storages/Page/V2/CMakeLists.txt index e840960b653..f994358f2b1 100644 --- a/dbms/src/Storages/Page/V2/CMakeLists.txt +++ b/dbms/src/Storages/Page/V2/CMakeLists.txt @@ -42,16 +42,4 @@ add_library(page_storage_v2 EXCLUDE_FROM_ALL ) target_include_directories(page_storage_v2 PUBLIC ${TiFlash_SOURCE_DIR}/contrib/tiflash-proxy/raftstore-proxy/ffi/src) target_link_libraries(page_storage_v2 clickhouse_common_io cpptoml - kv_client tipb) # TODO: remove dependency on these libs. Now we need them for DB::Context - -### Build a control binary for PageStorage -## For `page_ctl`, we need to define `PAGE_STORAGE_UTIL_DEBUGGGING` -add_executable(page_ctl EXCLUDE_FROM_ALL - tests/page_storage_ctl.cpp - ${page_storage_v2_headers} ${page_storage_v2_sources} - ${io_base_headers} ${io_base_sources} -) -target_include_directories(page_ctl PUBLIC ${TiFlash_SOURCE_DIR}/contrib/tiflash-proxy/raftstore-proxy/ffi/src) -target_link_libraries(page_ctl clickhouse_common_io cpptoml) -target_compile_options(page_ctl PRIVATE -Wno-format) -target_compile_definitions(page_ctl PRIVATE PAGE_STORAGE_UTIL_DEBUGGGING DBMS_PUBLIC_GTEST) + kv_client tipb) # TODO: remove dependency on these libs. Now we need them for DB::Context \ No newline at end of file diff --git a/dbms/src/Storages/Page/V2/gc/DataCompactor.h b/dbms/src/Storages/Page/V2/gc/DataCompactor.h index 1ddd36f87dd..eede1775cdf 100644 --- a/dbms/src/Storages/Page/V2/gc/DataCompactor.h +++ b/dbms/src/Storages/Page/V2/gc/DataCompactor.h @@ -60,10 +60,6 @@ class DataCompactor : private boost::noncopyable std::tuple tryMigrate(const PageFileSet & page_files, SnapshotPtr && snapshot, const WritingFilesSnapshot & writing_files); -#ifndef DBMS_PUBLIC_GTEST -private: -#endif - /** * Collect valid page of snapshot. * Return { @@ -72,6 +68,9 @@ class DataCompactor : private boost::noncopyable * } */ static ValidPages collectValidPagesInPageFile(const SnapshotPtr & snapshot); +#ifndef DBMS_PUBLIC_GTEST +private: +#endif struct CompactCandidates { diff --git a/dbms/src/Storages/Page/V3/BlobStore.h b/dbms/src/Storages/Page/V3/BlobStore.h index 5a3e98400d1..24bf4652123 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.h +++ b/dbms/src/Storages/Page/V3/BlobStore.h @@ -316,7 +316,7 @@ class BlobStore : private Allocator BlobFilePtr getBlobFile(BlobFileId blob_id); friend class PageDirectoryFactory; - friend class PageStorageControl; + friend class PageStorageControlV3; #ifndef DBMS_PUBLIC_GTEST private: diff --git a/dbms/src/Storages/Page/V3/PageDirectory.h b/dbms/src/Storages/Page/V3/PageDirectory.h index 39b5a05a40a..bd7c433022f 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory.h +++ b/dbms/src/Storages/Page/V3/PageDirectory.h @@ -255,7 +255,7 @@ class VersionedPageEntries being_ref_count, entries.size()); } - friend class PageStorageControl; + friend class PageStorageControlV3; private: mutable std::mutex m; @@ -376,7 +376,7 @@ class PageDirectory DISALLOW_COPY_AND_MOVE(PageDirectory); friend class PageDirectoryFactory; - friend class PageStorageControl; + friend class PageStorageControlV3; private: // Only `std::map` is allow for `MVCCMap`. Cause `std::map::insert` ensure that diff --git a/dbms/src/Storages/Page/V3/PageStorageImpl.h b/dbms/src/Storages/Page/V3/PageStorageImpl.h index f49601ce2ad..b4cdd425e59 100644 --- a/dbms/src/Storages/Page/V3/PageStorageImpl.h +++ b/dbms/src/Storages/Page/V3/PageStorageImpl.h @@ -115,7 +115,7 @@ class PageStorageImpl : public DB::PageStorage #endif friend class PageDirectoryFactory; - friend class PageStorageControl; + friend class PageStorageControlV3; #ifndef DBMS_PUBLIC_GTEST private: #endif diff --git a/dbms/src/Storages/Page/V3/tests/CMakeLists.txt b/dbms/src/Storages/Page/V3/tests/CMakeLists.txt index 8bab6afcded..355247c9eba 100644 --- a/dbms/src/Storages/Page/V3/tests/CMakeLists.txt +++ b/dbms/src/Storages/Page/V3/tests/CMakeLists.txt @@ -26,10 +26,4 @@ add_executable(gtests_page_storage_v3 ${ps_v3_gtest_sources} ${TiFlash_SOURCE_DI target_link_libraries(gtests_page_storage_v3 page_storage_v3 gtest_main) target_compile_options(gtests_page_storage_v3 PRIVATE -Wno-unknown-pragmas) target_compile_definitions(gtests_page_storage_v3 PRIVATE DBMS_PUBLIC_GTEST) -add_check(gtests_page_storage_v3) - - -add_executable(page_storage_ctl EXCLUDE_FROM_ALL page_storage_ctl.cpp) -target_compile_definitions(page_storage_ctl PUBLIC DBMS_PUBLIC_GTEST) -target_link_libraries(page_storage_ctl dbms page_storage_v3) -target_compile_options(page_storage_ctl PRIVATE -Wno-format -lc++) # turn off printf format check +add_check(gtests_page_storage_v3) \ No newline at end of file diff --git a/dbms/src/Storages/Page/tools/CMakeLists.txt b/dbms/src/Storages/Page/tools/CMakeLists.txt new file mode 100644 index 00000000000..629dedd01fc --- /dev/null +++ b/dbms/src/Storages/Page/tools/CMakeLists.txt @@ -0,0 +1,17 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +include_directories (${CMAKE_CURRENT_BINARY_DIR}) + +add_subdirectory (PageCtl EXCLUDE_FROM_ALL) diff --git a/dbms/src/Storages/Page/tools/PageCtl/CMakeLists.txt b/dbms/src/Storages/Page/tools/PageCtl/CMakeLists.txt new file mode 100644 index 00000000000..576b5e07a0f --- /dev/null +++ b/dbms/src/Storages/Page/tools/PageCtl/CMakeLists.txt @@ -0,0 +1,24 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +include_directories (${CMAKE_CURRENT_BINARY_DIR}) + +add_library(page-ctl-lib MainEntry.cpp PageStorageCtlV3.cpp PageStorageCtlV2.cpp ${page-ctl-src}) +target_include_directories(page-ctl-lib PUBLIC ${TiFlash_SOURCE_DIR}/libs/libdaemon/include) +target_link_libraries(page-ctl-lib dbms daemon tiflash-dttool-entry-object clickhouse-server-lib) +target_compile_options(page-ctl-lib PRIVATE -Wno-format) + +add_executable(page-ctl Main.cpp) +target_link_libraries(page-ctl page-ctl-lib dbms clickhouse_functions clickhouse-server-lib) +target_compile_options(page-ctl PRIVATE -Wno-format) diff --git a/dbms/src/Storages/Page/tools/PageCtl/Main.cpp b/dbms/src/Storages/Page/tools/PageCtl/Main.cpp new file mode 100644 index 00000000000..ae9901ec864 --- /dev/null +++ b/dbms/src/Storages/Page/tools/PageCtl/Main.cpp @@ -0,0 +1,20 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include + +int main(int argc, char ** argv) +{ + return DB::PageStorageCtl::mainEntry(argc, argv); +} diff --git a/dbms/src/Storages/Page/tools/PageCtl/MainEntry.cpp b/dbms/src/Storages/Page/tools/PageCtl/MainEntry.cpp new file mode 100644 index 00000000000..69b41435c34 --- /dev/null +++ b/dbms/src/Storages/Page/tools/PageCtl/MainEntry.cpp @@ -0,0 +1,69 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include + +#include +#include +namespace DB +{ +int PageStorageCtl::mainEntry(int argc, char ** argv) +{ + namespace po = boost::program_options; + using po::value; + po::options_description desc("Allowed commands"); + desc.add_options()("help,h", "produce help message") // + ("page_storage_version,V", value(), "PageStorage Version: 2 means PageStorage V2, 3 means PageStorage V3.\n"); + po::variables_map options; + po::parsed_options parsed = po::command_line_parser(argc, argv) + .options(desc) + .allow_unregistered() + .run(); + po::store(parsed, options); + po::notify(options); + if (options.count("page_storage_version") == 0 && options.count("help") == 0) + { + std::cerr << "Invalid arg page_storage_version." << std::endl; + std::cerr << desc << std::endl; + exit(0); + } + if (options.count("page_storage_version") > 0) + { + int ps_version = options["page_storage_version"].as(); + if (ps_version == 3) + { + pageStorageV3CtlEntry(argc - 2, argv + 2); + return 0; + } + else if (ps_version == 2) + { + return pageStorageV2CtlEntry(argc - 2, argv + 2); + } + else + { + std::cerr << "Invalid arg page_storage_version." << std::endl; + std::cerr << desc << std::endl; + exit(0); + } + } + if (options.count("help") > 0) + { + std::cerr << desc << std::endl; + exit(0); + } + return 0; +} +} // namespace DB diff --git a/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtl.h b/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtl.h new file mode 100644 index 00000000000..c8f35a7750a --- /dev/null +++ b/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtl.h @@ -0,0 +1,24 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +namespace DB +{ +class PageStorageCtl +{ +public: + static int mainEntry(int argc, char ** argv); +}; +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Storages/Page/V2/tests/page_storage_ctl.cpp b/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV2.cpp similarity index 93% rename from dbms/src/Storages/Page/V2/tests/page_storage_ctl.cpp rename to dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV2.cpp index 2fb7e31d70e..f9488c4dfd9 100644 --- a/dbms/src/Storages/Page/V2/tests/page_storage_ctl.cpp +++ b/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV2.cpp @@ -28,34 +28,21 @@ using namespace DB::PS::V2; DB::WriteBatch::SequenceID debugging_recover_stop_sequence = 0; -/* some exported global vars */ -namespace DB -{ -#if __APPLE__ && __clang__ -__thread bool is_background_thread = false; -#else -thread_local bool is_background_thread = false; -#endif -} // namespace DB -/* some exported global vars */ -void Usage(const char * prog) +void Usage() { fprintf(stderr, R"HELP( -Usage: %s +Usage: mode == 1 -> dump all page entries 2 -> dump valid page entries - param: %s 2 [max-recover-sequence] + param: 2 [max-recover-sequence] 3 -> check all page entries and page data checksum 4 -> list capacity of all page files 5 -> list all page files 1000 -> gc files - param: %s 1000 [run-gc-times=1] [min-gc-file-num=10] [min-gc-bytes=134217728] [max-gc-valid-rate=0.35] -)HELP", - prog, - prog, - prog); + param: 1000 [run-gc-times=1] [min-gc-file-num=10] [min-gc-bytes=134217728] [max-gc-valid-rate=0.35] + )HELP"); } void printPageEntry(const DB::PageId pid, const DB::PageEntry & entry) @@ -117,7 +104,7 @@ PageStorage::Config parse_storage_config(int argc, char ** argv, Poco::Logger * return config; } -int main(int argc, char ** argv) +int pageStorageV2CtlEntry(int argc, char ** argv) try { (void)argc; @@ -125,7 +112,7 @@ try if (argc < 3) { - Usage(argv[0]); + Usage(); return 1; } @@ -153,7 +140,7 @@ try LOG_FMT_INFO(logger, "Running with [mode={}]", mode); break; default: - Usage(argv[0]); + Usage(); return 1; } @@ -271,13 +258,13 @@ void dump_all_entries(PageFileSet & page_files, int32_t mode) id_and_caches.emplace_back(std::make_pair(record.page_id, record.entry)); break; case DB::WriteBatch::WriteType::DEL: - printf("DEL\t%lld\n", // + printf("DEL\t%lld\t%llu\t%u\n", // record.page_id, page_file.getFileId(), page_file.getLevel()); break; case DB::WriteBatch::WriteType::REF: - printf("REF\t%lld\t%lld\t\n", // + printf("REF\t%lld\t%lld\t\t%llu\t%u\n", // record.page_id, record.ori_page_id, page_file.getFileId(), diff --git a/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV2.h b/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV2.h new file mode 100644 index 00000000000..6d573ffaba7 --- /dev/null +++ b/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV2.h @@ -0,0 +1,17 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +int pageStorageV2CtlEntry(int argc, char ** argv); diff --git a/dbms/src/Storages/Page/V3/tests/page_storage_ctl.cpp b/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV3.cpp similarity index 87% rename from dbms/src/Storages/Page/V3/tests/page_storage_ctl.cpp rename to dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV3.cpp index 7ea8da6892a..c9b871c67d2 100644 --- a/dbms/src/Storages/Page/V3/tests/page_storage_ctl.cpp +++ b/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV3.cpp @@ -12,19 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include +#include #include -#include -#include #include -#include -#include -#include +#include #include #include #include -#include #include #include @@ -32,6 +27,9 @@ namespace DB::PS::V3 { +extern "C" { +void run_raftstore_proxy_ffi(int argc, const char * const * argv, const DB::EngineStoreServerHelper *); +} struct ControlOptions { enum DisplayType @@ -49,6 +47,8 @@ struct ControlOptions UInt64 query_ns_id = DB::TEST_NAMESPACE_ID; UInt64 check_page_id = UINT64_MAX; bool enable_fo_check = true; + bool is_imitative = true; + String config_file_path; static ControlOptions parse(int argc, char ** argv); }; @@ -65,12 +65,12 @@ ControlOptions ControlOptions::parse(int argc, char ** argv) ("display_mode,D", value()->default_value(1), "Display Mode: 1 is summary information,\n 2 is display all of stored page and version chaim(will be very long),\n 3 is display all blobs(in disk) data distribution. \n 4 is check every data is valid.") // ("enable_fo_check,E", value()->default_value(true), "Also check the evert field offsets. This options only works when `display_mode` is 4.") // ("query_ns_id,N", value()->default_value(DB::TEST_NAMESPACE_ID), "When used `check_page_id`/`query_page_id`/`query_blob_id` to query results. You can specify a namespace id.")("check_page_id,C", value()->default_value(UINT64_MAX), "Check a single Page id, display the exception if meet. And also will check the field offsets.") // - ("query_page_id,W", value()->default_value(UINT64_MAX), "Quert a single Page id, and print its version chaim.") // - ("query_blob_id,B", value()->default_value(UINT32_MAX), "Quert a single Blob id, and print its data distribution."); + ("query_page_id,W", value()->default_value(UINT64_MAX), "Query a single Page id, and print its version chaim.") // + ("query_blob_id,B", value()->default_value(UINT32_MAX), "Query a single Blob id, and print its data distribution.")("imitative,I", value()->default_value(true), "Use imitative context instead. (encryption is not supported in this mode so that no need to set config_file_path)")("config_file_path", value(), "Path to TiFlash config (tiflash.toml)."); static_assert(sizeof(DB::PageId) == sizeof(UInt64)); - static_assert(sizeof(DB::BlobFileId) == sizeof(UInt32)); + static_assert(sizeof(DB::BlobFileId) == sizeof(UInt64)); po::variables_map options; po::store(po::parse_command_line(argc, argv, desc), options); @@ -97,6 +97,21 @@ ControlOptions ControlOptions::parse(int argc, char ** argv) opt.enable_fo_check = options["enable_fo_check"].as(); opt.check_page_id = options["check_page_id"].as(); opt.query_ns_id = options["query_ns_id"].as(); + opt.is_imitative = options["imitative"].as(); + if (opt.is_imitative && options.count("config_file_path") != 0) + { + std::cerr << "config_file_path is not allowed in imitative mode" << std::endl; + exit(0); + } + else if (!opt.is_imitative && options.count("config_file_path") == 0) + { + std::cerr << "config_file_path is required in proxy mode" << std::endl; + exit(0); + } + if (options.count("config_file_path") != 0) + { + opt.config_file_path = options["config_file_path"].as(); + } if (opt.display_mode < DisplayType::DISPLAY_SUMMARY_INFO || opt.display_mode > DisplayType::CHECK_ALL_DATA_CRC) { @@ -108,15 +123,39 @@ ControlOptions ControlOptions::parse(int argc, char ** argv) return opt; } -class PageStorageControl +class PageStorageControlV3 { public: - explicit PageStorageControl(const ControlOptions & options_) + explicit PageStorageControlV3(const ControlOptions & options_) : options(options_) { } void run() + { + try + { + if (options.is_imitative) + { + Context context = Context::createGlobal(); + getPageStorageV3Info(context, options); + } + else + { + PageDirectory::MVCCMapType type; + CLIService service(getPageStorageV3Info, options, options.config_file_path, run_raftstore_proxy_ffi); + service.run({""}); + } + } + catch (...) + { + DB::tryLogCurrentException("exception thrown"); + std::abort(); // Finish testing if some error happened. + } + } + +private: + static int getPageStorageV3Info(Context & context, const ControlOptions & options) { DB::PSDiskDelegatorPtr delegator; if (options.paths.size() == 1) @@ -128,13 +167,20 @@ class PageStorageControl delegator = std::make_shared(options.paths); } - auto key_manager = std::make_shared(false); - auto file_provider = std::make_shared(key_manager, false); - + FileProviderPtr file_provider_ptr; + if (options.is_imitative) + { + auto key_manager = std::make_shared(false); + file_provider_ptr = std::make_shared(key_manager, false); + } + else + { + file_provider_ptr = context.getFileProvider(); + } BlobStore::Config blob_config; PageStorage::Config config; - PageStorageImpl ps_v3("PageStorageControl", delegator, config, file_provider); + PageStorageImpl ps_v3("PageStorageControlV3", delegator, config, file_provider_ptr); ps_v3.restore(); PageDirectory::MVCCMapType & mvcc_table_directory = ps_v3.page_directory->mvcc_table_directory; @@ -171,9 +217,9 @@ class PageStorageControl std::cout << "Invalid display mode." << std::endl; break; } + return 0; } -private: static String getBlobsInfo(BlobStore & blob_store, UInt32 blob_id) { auto stat_info = [](const BlobStore::BlobStats::BlobStatPtr & stat, const String & path) { @@ -469,9 +515,9 @@ class PageStorageControl } // namespace DB::PS::V3 using namespace DB::PS::V3; -int main(int argc, char ** argv) + +void pageStorageV3CtlEntry(int argc, char ** argv) { const auto & options = ControlOptions::parse(argc, argv); - PageStorageControl(options).run(); - return 0; + PageStorageControlV3(options).run(); } diff --git a/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV3.h b/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV3.h new file mode 100644 index 00000000000..21a929ee599 --- /dev/null +++ b/dbms/src/Storages/Page/tools/PageCtl/PageStorageCtlV3.h @@ -0,0 +1,17 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +void pageStorageV3CtlEntry(int argc, char ** argv); From 226dff62b1c045eebbf16ebb3f459f26328ee2ad Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger <60744015+Lloyd-Pottiger@users.noreply.github.com> Date: Thu, 2 Jun 2022 12:14:27 +0800 Subject: [PATCH 016/104] update tiflash proxy (#5043) ref pingcap/tiflash#4879 --- contrib/tiflash-proxy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 7578b816399..ca2f51f94e5 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 7578b8163992ce933074135f8687ad447d88ea9b +Subproject commit ca2f51f94e55bdd23749dcc02ab4afb94eeb5ae5 From 2ce9529f1069185f82b3018e5b96152fcd02b601 Mon Sep 17 00:00:00 2001 From: lidezhu <47731263+lidezhu@users.noreply.github.com> Date: Thu, 2 Jun 2022 13:13:44 +0800 Subject: [PATCH 017/104] Fix potential data inconsistency under heavy ddl operation (#5044) * fix decoding error under heavy ddl operation * small fix --- dbms/src/Common/FailPoint.cpp | 3 ++- dbms/src/Debug/dbgFuncSchema.cpp | 18 +++++++++++++++++- dbms/src/Storages/IManageableStorage.h | 7 +++++-- dbms/src/Storages/StorageDeltaMerge.cpp | 13 ++++++++----- dbms/src/Storages/StorageDeltaMerge.h | 9 +++++++-- .../DecodingStorageSchemaSnapshot.h | 7 ++++--- .../Storages/Transaction/PartitionStreams.cpp | 12 ++++++------ .../Transaction/tests/RowCodecTestUtils.h | 4 ++-- dbms/src/TiDB/Schema/SchemaBuilder.cpp | 13 ++++++++++++- dbms/src/TiDB/Schema/TiDBSchemaSyncer.h | 9 +++++++++ .../ddl/alter_column_when_pk_is_handle.test | 18 +++++++++++++++++- 11 files changed, 89 insertions(+), 24 deletions(-) diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index 921dd5bf748..c6c3caa44ad 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -65,7 +65,8 @@ std::unordered_map> FailPointHelper::f M(exception_when_read_from_log) \ M(exception_mpp_hash_build) \ M(exception_before_drop_segment) \ - M(exception_after_drop_segment) + M(exception_after_drop_segment) \ + M(exception_between_schema_change_in_the_same_diff) #define APPLY_FOR_FAILPOINTS(M) \ M(skip_check_segment_update) \ diff --git a/dbms/src/Debug/dbgFuncSchema.cpp b/dbms/src/Debug/dbgFuncSchema.cpp index 8b73ddc23a3..c388015dc10 100644 --- a/dbms/src/Debug/dbgFuncSchema.cpp +++ b/dbms/src/Debug/dbgFuncSchema.cpp @@ -34,6 +34,7 @@ namespace DB { namespace ErrorCodes { +extern const int FAIL_POINT_ERROR; extern const int UNKNOWN_TABLE; } // namespace ErrorCodes @@ -62,7 +63,22 @@ void dbgFuncRefreshSchemas(Context & context, const ASTs &, DBGInvoker::Printer { TMTContext & tmt = context.getTMTContext(); auto schema_syncer = tmt.getSchemaSyncer(); - schema_syncer->syncSchemas(context); + try + { + schema_syncer->syncSchemas(context); + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::FAIL_POINT_ERROR) + { + output(e.message()); + return; + } + else + { + throw; + } + } output("schemas refreshed"); } diff --git a/dbms/src/Storages/IManageableStorage.h b/dbms/src/Storages/IManageableStorage.h index e41d092ca87..ebf84c592e4 100644 --- a/dbms/src/Storages/IManageableStorage.h +++ b/dbms/src/Storages/IManageableStorage.h @@ -157,12 +157,15 @@ class IManageableStorage : public IStorage /// when `need_block` is true, it will try return a cached block corresponding to DecodingStorageSchemaSnapshotConstPtr, /// and `releaseDecodingBlock` need to be called when the block is free /// when `need_block` is false, it will just return an nullptr - virtual std::pair getSchemaSnapshotAndBlockForDecoding(bool /* need_block */) + /// This method must be called under the protection of table structure lock + virtual std::pair getSchemaSnapshotAndBlockForDecoding(const TableStructureLockHolder & /* table_structure_lock */, bool /* need_block */) { throw Exception("Method getDecodingSchemaSnapshot is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); }; - virtual void releaseDecodingBlock(Int64 /* schema_version */, BlockUPtr /* block */) + /// The `block_decoding_schema_version` is just an internal version for `DecodingStorageSchemaSnapshot`, + /// And it has no relation with the table schema version. + virtual void releaseDecodingBlock(Int64 /* block_decoding_schema_version */, BlockUPtr /* block */) { throw Exception("Method getDecodingSchemaSnapshot is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index fc73e28e23a..67d32c73a05 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -901,14 +901,16 @@ void StorageDeltaMerge::deleteRows(const Context & context, size_t delete_rows) LOG_FMT_ERROR(log, "Rows after delete range not match, expected: {}, got: {}", (total_rows - delete_rows), after_delete_rows); } -std::pair StorageDeltaMerge::getSchemaSnapshotAndBlockForDecoding(bool need_block) +std::pair StorageDeltaMerge::getSchemaSnapshotAndBlockForDecoding(const TableStructureLockHolder & table_structure_lock, bool need_block) { + (void)table_structure_lock; std::lock_guard lock{decode_schema_mutex}; - if (!decoding_schema_snapshot || decoding_schema_snapshot->schema_version < tidb_table_info.schema_version) + if (!decoding_schema_snapshot || decoding_schema_changed) { auto & store = getAndMaybeInitStore(); - decoding_schema_snapshot = std::make_shared(store->getStoreColumns(), tidb_table_info, store->getHandle()); + decoding_schema_snapshot = std::make_shared(store->getStoreColumns(), tidb_table_info, store->getHandle(), decoding_schema_version++); cache_blocks.clear(); + decoding_schema_changed = false; } if (need_block) @@ -930,10 +932,10 @@ std::pair StorageDeltaMerg } } -void StorageDeltaMerge::releaseDecodingBlock(Int64 schema_version, BlockUPtr block_ptr) +void StorageDeltaMerge::releaseDecodingBlock(Int64 block_decoding_schema_version, BlockUPtr block_ptr) { std::lock_guard lock{decode_schema_mutex}; - if (!decoding_schema_snapshot || schema_version < decoding_schema_snapshot->schema_version) + if (!decoding_schema_snapshot || block_decoding_schema_version < decoding_schema_snapshot->decoding_schema_version) return; if (cache_blocks.size() >= max_cached_blocks_num) return; @@ -1113,6 +1115,7 @@ try updateTableColumnInfo(); } } + decoding_schema_changed = true; SortDescription pk_desc = getPrimarySortDescription(); ColumnDefines store_columns = getStoreColumnDefines(); diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index e304c713b7b..79ee225d237 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -151,9 +151,9 @@ class StorageDeltaMerge size_t getRowKeyColumnSize() const override { return rowkey_column_size; } - std::pair getSchemaSnapshotAndBlockForDecoding(bool /* need_block */) override; + std::pair getSchemaSnapshotAndBlockForDecoding(const TableStructureLockHolder & table_structure_lock, bool /* need_block */) override; - void releaseDecodingBlock(Int64 schema_version, BlockUPtr block) override; + void releaseDecodingBlock(Int64 block_decoding_schema_version, BlockUPtr block) override; bool initStoreIfDataDirExist() override; @@ -238,6 +238,11 @@ class StorageDeltaMerge mutable std::mutex decode_schema_mutex; DecodingStorageSchemaSnapshotPtr decoding_schema_snapshot; + // The following two members must be used under the protection of table structure lock + bool decoding_schema_changed = false; + // internal version for `decoding_schema_snapshot` + Int64 decoding_schema_version = 1; + // avoid creating block every time when decoding row std::vector cache_blocks; // avoid creating too many cached blocks(the typical num should be less and equal than raft apply thread) diff --git a/dbms/src/Storages/Transaction/DecodingStorageSchemaSnapshot.h b/dbms/src/Storages/Transaction/DecodingStorageSchemaSnapshot.h index 6cedbe3f0c0..c636d9e60ab 100644 --- a/dbms/src/Storages/Transaction/DecodingStorageSchemaSnapshot.h +++ b/dbms/src/Storages/Transaction/DecodingStorageSchemaSnapshot.h @@ -67,13 +67,14 @@ struct DecodingStorageSchemaSnapshot bool pk_is_handle; bool is_common_handle; TMTPKType pk_type = TMTPKType::UNSPECIFIED; - Int64 schema_version = DEFAULT_UNSPECIFIED_SCHEMA_VERSION; + // an internal increasing version for `DecodingStorageSchemaSnapshot`, has no relation with the table schema version + Int64 decoding_schema_version; - DecodingStorageSchemaSnapshot(DM::ColumnDefinesPtr column_defines_, const TiDB::TableInfo & table_info_, const DM::ColumnDefine & original_handle_) + DecodingStorageSchemaSnapshot(DM::ColumnDefinesPtr column_defines_, const TiDB::TableInfo & table_info_, const DM::ColumnDefine & original_handle_, Int64 decoding_schema_version_) : column_defines{std::move(column_defines_)} , pk_is_handle{table_info_.pk_is_handle} , is_common_handle{table_info_.is_common_handle} - , schema_version{table_info_.schema_version} + , decoding_schema_version{decoding_schema_version_} { std::unordered_map column_lut; for (size_t i = 0; i < table_info_.columns.size(); i++) diff --git a/dbms/src/Storages/Transaction/PartitionStreams.cpp b/dbms/src/Storages/Transaction/PartitionStreams.cpp index ada792c80f7..4b2ca6c07a8 100644 --- a/dbms/src/Storages/Transaction/PartitionStreams.cpp +++ b/dbms/src/Storages/Transaction/PartitionStreams.cpp @@ -114,14 +114,14 @@ static void writeRegionDataToStorage( /// Read region data as block. Stopwatch watch; - Int64 block_schema_version = DEFAULT_UNSPECIFIED_SCHEMA_VERSION; + Int64 block_decoding_schema_version = -1; BlockUPtr block_ptr = nullptr; if (need_decode) { LOG_FMT_TRACE(log, "{} begin to decode table {}, region {}", FUNCTION_NAME, table_id, region->id()); DecodingStorageSchemaSnapshotConstPtr decoding_schema_snapshot; - std::tie(decoding_schema_snapshot, block_ptr) = storage->getSchemaSnapshotAndBlockForDecoding(true); - block_schema_version = decoding_schema_snapshot->schema_version; + std::tie(decoding_schema_snapshot, block_ptr) = storage->getSchemaSnapshotAndBlockForDecoding(lock, true); + block_decoding_schema_version = decoding_schema_snapshot->decoding_schema_version; auto reader = RegionBlockReader(decoding_schema_snapshot); if (!reader.read(*block_ptr, data_list_read, force_decode)) @@ -153,7 +153,7 @@ static void writeRegionDataToStorage( write_part_cost = watch.elapsedMilliseconds(); GET_METRIC(tiflash_raft_write_data_to_storage_duration_seconds, type_write).Observe(write_part_cost / 1000.0); if (need_decode) - storage->releaseDecodingBlock(block_schema_version, std::move(block_ptr)); + storage->releaseDecodingBlock(block_decoding_schema_version, std::move(block_ptr)); LOG_FMT_TRACE(log, "{}: table {}, region {}, cost [region decode {}, write part {}] ms", FUNCTION_NAME, table_id, region->id(), region_decode_cost, write_part_cost); return true; @@ -455,7 +455,7 @@ RegionPtrWithBlock::CachePtr GenRegionPreDecodeBlockData(const RegionPtr & regio } DecodingStorageSchemaSnapshotConstPtr decoding_schema_snapshot; - std::tie(decoding_schema_snapshot, std::ignore) = storage->getSchemaSnapshotAndBlockForDecoding(false); + std::tie(decoding_schema_snapshot, std::ignore) = storage->getSchemaSnapshotAndBlockForDecoding(lock, false); res_block = createBlockSortByColumnID(decoding_schema_snapshot); auto reader = RegionBlockReader(decoding_schema_snapshot); if (!reader.read(res_block, *data_list_read, force_decode)) @@ -508,7 +508,7 @@ AtomicGetStorageSchema(const RegionPtr & region, TMTContext & tmt) auto table_lock = storage->lockStructureForShare(getThreadName()); dm_storage = std::dynamic_pointer_cast(storage); // only dt storage engine support `getSchemaSnapshotAndBlockForDecoding`, other engine will throw exception - std::tie(schema_snapshot, std::ignore) = storage->getSchemaSnapshotAndBlockForDecoding(false); + std::tie(schema_snapshot, std::ignore) = storage->getSchemaSnapshotAndBlockForDecoding(table_lock, false); std::tie(std::ignore, drop_lock) = std::move(table_lock).release(); return true; }; diff --git a/dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h b/dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h index c28ea531afe..20b395a9952 100644 --- a/dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h +++ b/dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h @@ -285,11 +285,11 @@ inline DecodingStorageSchemaSnapshotConstPtr getDecodingStorageSchemaSnapshot(co if (handle_id != EXTRA_HANDLE_COLUMN_ID) { auto iter = std::find_if(store_columns.begin(), store_columns.end(), [&](const ColumnDefine & cd) { return cd.id == handle_id; }); - return std::make_shared(std::make_shared(store_columns), table_info, *iter); + return std::make_shared(std::make_shared(store_columns), table_info, *iter, /* decoding_schema_version_ */ 1); } else { - return std::make_shared(std::make_shared(store_columns), table_info, store_columns[0]); + return std::make_shared(std::make_shared(store_columns), table_info, store_columns[0], /* decoding_schema_version_ */ 1); } } diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.cpp b/dbms/src/TiDB/Schema/SchemaBuilder.cpp index 64d118eec3e..99e540e6c95 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.cpp +++ b/dbms/src/TiDB/Schema/SchemaBuilder.cpp @@ -61,6 +61,7 @@ extern const char exception_before_step_2_rename_in_exchange_partition[]; extern const char exception_after_step_2_in_exchange_partition[]; extern const char exception_before_step_3_rename_in_exchange_partition[]; extern const char exception_after_step_3_in_exchange_partition[]; +extern const char exception_between_schema_change_in_the_same_diff[]; } // namespace FailPoints bool isReservedDatabase(Context & context, const String & database_name) @@ -336,6 +337,7 @@ void SchemaBuilder::applyAlterPhysicalTable(DBInfoPtr db_inf FmtBuffer fmt_buf; fmt_buf.fmtAppend("Detected schema changes: {}: ", name_mapper.debugCanonicalName(*db_info, *table_info)); for (const auto & schema_change : schema_changes) + { for (const auto & command : schema_change.first) { if (command.type == AlterCommand::ADD_COLUMN) @@ -347,6 +349,7 @@ void SchemaBuilder::applyAlterPhysicalTable(DBInfoPtr db_inf else if (command.type == AlterCommand::RENAME_COLUMN) fmt_buf.fmtAppend("RENAME COLUMN from {} to {}, ", command.column_name, command.new_column_name); } + } return fmt_buf.toString(); }; LOG_DEBUG(log, log_str()); @@ -355,8 +358,16 @@ void SchemaBuilder::applyAlterPhysicalTable(DBInfoPtr db_inf // Using original table info with updated columns instead of using new_table_info directly, // so that other changes (RENAME commands) won't be saved. // Also, updating schema_version as altering column is structural. - for (const auto & schema_change : schema_changes) + for (size_t i = 0; i < schema_changes.size(); i++) { + if (i > 0) + { + /// If there are multiple schema change in the same diff, + /// the table schema version will be set to the latest schema version after the first schema change is applied. + /// Throw exception in the middle of the schema change to mock the case that there is a race between data decoding and applying different schema change. + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_between_schema_change_in_the_same_diff); + } + const auto & schema_change = schema_changes[i]; /// Update column infos by applying schema change in this step. schema_change.second(orig_table_info); /// Update schema version aggressively for the sake of correctness. diff --git a/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h b/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h index b682abf1af4..8aab2b3302e 100644 --- a/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h +++ b/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h @@ -28,6 +28,11 @@ namespace DB { +namespace ErrorCodes +{ +extern const int FAIL_POINT_ERROR; +}; + template struct TiDBSchemaSyncer : public SchemaSyncer { @@ -177,6 +182,10 @@ struct TiDBSchemaSyncer : public SchemaSyncer } catch (Exception & e) { + if (e.code() == ErrorCodes::FAIL_POINT_ERROR) + { + throw; + } GET_METRIC(tiflash_schema_apply_count, type_failed).Increment(); LOG_FMT_WARNING(log, "apply diff meets exception : {} \n stack is {}", e.displayText(), e.getStackTrace().toString()); return false; diff --git a/tests/fullstack-test2/ddl/alter_column_when_pk_is_handle.test b/tests/fullstack-test2/ddl/alter_column_when_pk_is_handle.test index ca92828e6cf..df0aa13823a 100644 --- a/tests/fullstack-test2/ddl/alter_column_when_pk_is_handle.test +++ b/tests/fullstack-test2/ddl/alter_column_when_pk_is_handle.test @@ -37,9 +37,25 @@ mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t | 1 | world | 0.00 | 2 | NULL | +---+-------+------+------+------+ -# Need to apply a lossy type change to reorganize data. issue#3714 +=> DBGInvoke __enable_schema_sync_service('false') + +>> DBGInvoke __enable_fail_point(exception_between_schema_change_in_the_same_diff) + +# stop decoding data +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# Need to apply a lossy type change to reorganize data. issue#3714 mysql> alter table test.t modify c decimal(6,3) +# refresh schema and hit the `exception_between_schema_change_in_the_same_diff` failpoint +>> DBGInvoke __refresh_schemas() + +>> DBGInvoke __disable_fail_point(exception_between_schema_change_in_the_same_diff) + +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke __enable_schema_sync_service('true') + mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t +---+-------+-------+------+------+ | a | b | c | d | e | From 20d2198e5c2a191154431272f907f8758a8bd17e Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Thu, 2 Jun 2022 18:10:28 +0800 Subject: [PATCH 018/104] *: fix invalid fmt format string in CreatingSetsBlockInputStream.cpp (#5053) ref pingcap/tiflash#5052 --- dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp index 47e7a56aa35..22545327edd 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp @@ -228,7 +228,7 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) if (done_with_set && done_with_join && done_with_table) { - if (IProfilingBlockInputStream * profiling_in = dynamic_cast(&*subquery.source)) + if (auto * profiling_in = dynamic_cast(&*subquery.source)) profiling_in->cancel(false); break; @@ -248,7 +248,7 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) watch.stop(); size_t head_rows = 0; - if (IProfilingBlockInputStream * profiling_in = dynamic_cast(&*subquery.source)) + if (auto * profiling_in = dynamic_cast(&*subquery.source)) { const BlockStreamProfileInfo & profile_info = profiling_in->getProfileInfo(); @@ -272,7 +272,7 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) if (subquery.table) msg.fmtAppend("Table with {} rows. ", head_rows); - msg.fmtAppend("In {.3f} sec. ", watch.elapsedSeconds()); + msg.fmtAppend("In {:.3f} sec. ", watch.elapsedSeconds()); msg.fmtAppend("using {} threads.", subquery.join ? subquery.join->getBuildConcurrency() : 1); return msg.toString(); }; From 677ad751e33a083c0311ee02114078b53f5f5e43 Mon Sep 17 00:00:00 2001 From: lidezhu <47731263+lidezhu@users.noreply.github.com> Date: Fri, 3 Jun 2022 09:16:27 +0800 Subject: [PATCH 019/104] increase bg gc check interval (#5056) close pingcap/tiflash#5057 --- dbms/src/Interpreters/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 87d007c101f..9361e0525d2 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -272,7 +272,7 @@ struct Settings M(SettingUInt64, dt_segment_delta_small_column_file_size, 8388608, "Determine whether a column file in delta is small or not. 8MB by default.") \ M(SettingUInt64, dt_segment_stable_pack_rows, DEFAULT_MERGE_BLOCK_SIZE, "Expected stable pack rows in DeltaTree Engine.") \ M(SettingFloat, dt_segment_wait_duration_factor, 1, "The factor of wait duration in a write stall.") \ - M(SettingUInt64, dt_bg_gc_check_interval, 5, "Background gc thread check interval, the unit is second.") \ + M(SettingUInt64, dt_bg_gc_check_interval, 60, "Background gc thread check interval, the unit is second.") \ M(SettingInt64, dt_bg_gc_max_segments_to_check_every_round, 100, "Max segments to check in every gc round, value less than or equal to 0 means gc no segments.") \ M(SettingFloat, dt_bg_gc_ratio_threhold_to_trigger_gc, 1.2, "Trigger segment's gc when the ratio of invalid version exceed this threhold. Values smaller than or equal to 1.0 means gc all " \ "segments") \ From b49a78780156bcc4769a9017d3bb0351c43e7d8c Mon Sep 17 00:00:00 2001 From: JaySon Date: Fri, 3 Jun 2022 16:12:28 +0800 Subject: [PATCH 020/104] PageStorage: Fix pages are not deleted under some cases (#5069) close pingcap/tiflash#5054 --- dbms/src/Storages/DeltaMerge/StoragePool.cpp | 18 +-- dbms/src/Storages/Page/Page.h | 3 +- dbms/src/Storages/Page/V2/PageEntries.h | 26 ++-- .../V3/tests/gtest_page_storage_mix_mode.cpp | 126 ++++++++++++++++++ 4 files changed, 149 insertions(+), 24 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/StoragePool.cpp b/dbms/src/Storages/DeltaMerge/StoragePool.cpp index fa765cd9b1d..752898f9c75 100644 --- a/dbms/src/Storages/DeltaMerge/StoragePool.cpp +++ b/dbms/src/Storages/DeltaMerge/StoragePool.cpp @@ -515,21 +515,9 @@ void StoragePool::dataRegisterExternalPagesCallbacks(const ExternalPageCallbacks } case PageStorageRunMode::MIX_MODE: { - // When PageStorage run as Mix Mode. - // We need both get alive pages from V2 and V3 which will feedback for the DM. - // But V2 and V3 won't GC in the same time. So V3 need proxy V2 external pages callback. - // When V3 GC happend, scan the external pages from V3, in remover will scanner all of external pages from V2. - ExternalPageCallbacks mix_mode_callbacks; - - mix_mode_callbacks.scanner = callbacks.scanner; - mix_mode_callbacks.remover = [this, callbacks](const ExternalPageCallbacks::PathAndIdsVec & path_and_ids_vec, const std::set & valid_ids) { - // ns_id won't used on V2 - auto v2_valid_page_ids = data_storage_v2->getAliveExternalPageIds(ns_id); - v2_valid_page_ids.insert(valid_ids.begin(), valid_ids.end()); - callbacks.remover(path_and_ids_vec, v2_valid_page_ids); - }; - mix_mode_callbacks.ns_id = ns_id; - data_storage_v3->registerExternalPagesCallbacks(mix_mode_callbacks); + // We have transformed all pages from V2 to V3 in `restore`, so + // only need to register callbacks for V3. + data_storage_v3->registerExternalPagesCallbacks(callbacks); break; } default: diff --git a/dbms/src/Storages/Page/Page.h b/dbms/src/Storages/Page/Page.h index b54b25033dd..5328490e5ad 100644 --- a/dbms/src/Storages/Page/Page.h +++ b/dbms/src/Storages/Page/Page.h @@ -128,12 +128,13 @@ struct PageEntry String toDebugString() const { - return fmt::format("PageEntry{{file: {}, offset: 0x{:X}, size: {}, checksum: 0x{:X}, tag: {}, field_offsets_size: {}}}", + return fmt::format("PageEntry{{file: {}, offset: 0x{:X}, size: {}, checksum: 0x{:X}, tag: {}, ref: {}, field_offsets_size: {}}}", file_id, offset, size, checksum, tag, + ref, field_offsets.size()); } diff --git a/dbms/src/Storages/Page/V2/PageEntries.h b/dbms/src/Storages/Page/V2/PageEntries.h index c99e0dade6b..0a0504b0cb5 100644 --- a/dbms/src/Storages/Page/V2/PageEntries.h +++ b/dbms/src/Storages/Page/V2/PageEntries.h @@ -252,7 +252,7 @@ class PageEntriesMixin protected: template - void decreasePageRef(PageId page_id); + void decreasePageRef(PageId page_id, bool keep_tombstone); void copyEntries(const PageEntriesMixin & rhs) { @@ -370,8 +370,10 @@ void PageEntriesMixin::del(PageId page_id) const size_t num_erase = page_ref.erase(page_id); if (num_erase > 0) { - // decrease origin page's ref counting - decreasePageRef(normal_page_id); + // decrease origin page's ref counting, this method can + // only called by base, so we should remove the entry if + // the ref count down to zero + decreasePageRef(normal_page_id, /*keep_tombstone=*/false); } } @@ -392,7 +394,9 @@ void PageEntriesMixin::ref(const PageId ref_id, const PageId page_id) // if RefPage{ref-id} -> Page{normal_page_id} already exists, just ignore if (ori_ref->second == normal_page_id) return; - decreasePageRef(ori_ref->second); + // this method can only called by base, so we should remove the entry if + // the ref count down to zero + decreasePageRef(ori_ref->second, /*keep_tombstone=*/false); } // build ref page_ref[ref_id] = normal_page_id; @@ -408,7 +412,7 @@ void PageEntriesMixin::ref(const PageId ref_id, const PageId page_id) template template -void PageEntriesMixin::decreasePageRef(const PageId page_id) +void PageEntriesMixin::decreasePageRef(const PageId page_id, bool keep_tombstone) { auto iter = normal_pages.find(page_id); if constexpr (must_exist) @@ -421,8 +425,11 @@ void PageEntriesMixin::decreasePageRef(const PageId page_id) if (iter != normal_pages.end()) { auto & entry = iter->second; - entry.ref -= 1; - if (entry.ref == 0) + if (entry.ref > 0) + { + entry.ref -= 1; + } + if (!keep_tombstone && entry.ref == 0) { normal_pages.erase(iter); } @@ -620,7 +627,10 @@ class PageEntriesForDelta : public PageEntriesMixin { ref_deletions.insert(page_id); } - decreasePageRef(page_id); + // If this is the base version, we should remove the entry if + // the ref count down to zero. Otherwise it is the delta version + // we should keep a tombstone. + decreasePageRef(page_id, /*keep_tombstone=*/!this->isBase()); } for (auto it : rhs.page_ref) { diff --git a/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp b/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp index 5517539b898..078daa3e5b4 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -19,6 +20,8 @@ #include #include #include +#include +#include #include namespace DB @@ -879,6 +882,129 @@ try } CATCH + +TEST_F(PageStorageMixedTest, RefV2External2) +try +{ + auto logger = DB::Logger::get("PageStorageMixedTest"); + { + WriteBatch batch; + batch.putExternal(100, 0); + batch.putRefPage(101, 100); + batch.delPage(100); + batch.putExternal(102, 0); + page_writer_v2->write(std::move(batch), nullptr); + } + + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::MIX_MODE); + { + WriteBatch batch; + batch.putExternal(100, 0); + batch.putRefPage(101, 100); + batch.delPage(100); + batch.putExternal(102, 0); + page_writer_mix->writeIntoV3(std::move(batch), nullptr); + } + { + auto snap = storage_pool_mix->log_storage_v2->getSnapshot("zzz"); // must hold + // after transform to v3, delete these from v2 + WriteBatch batch; + batch.delPage(100); + batch.delPage(101); + batch.delPage(102); + page_writer_mix->writeIntoV2(std::move(batch), nullptr); + } + + { + LOG_FMT_INFO(logger, "first check alive id in v2"); + auto alive_dt_ids_in_v2 = storage_pool_mix->log_storage_v2->getAliveExternalPageIds(TEST_NAMESPACE_ID); + EXPECT_EQ(alive_dt_ids_in_v2.size(), 0); + + storage_pool_mix->log_storage_v3->gc(false, nullptr, nullptr); + auto alive_dt_ids_in_v3 = storage_pool_mix->log_storage_v3->getAliveExternalPageIds(TEST_NAMESPACE_ID); + ASSERT_EQ(alive_dt_ids_in_v3.size(), 2); + auto iter = alive_dt_ids_in_v3.begin(); + EXPECT_EQ(*iter, 100); + iter++; + EXPECT_EQ(*iter, 102); + } + + { + LOG_FMT_INFO(logger, "remove 100, create 105"); + StorageSnapshot snap(*storage_pool_mix, nullptr, "xxx", true); // must hold and write + // write delete again + WriteBatch batch; + batch.delPage(100); + batch.putExternal(105, 0); + page_writer_mix->write(std::move(batch), nullptr); + LOG_FMT_INFO(logger, "done"); + } + { + LOG_FMT_INFO(logger, "remove 101, create 106"); + StorageSnapshot snap(*storage_pool_mix, nullptr, "xxx", true); // must hold and write + // write delete again + WriteBatch batch; + batch.delPage(101); + batch.putExternal(106, 0); + page_writer_mix->write(std::move(batch), nullptr); + LOG_FMT_INFO(logger, "done"); + } + { + LOG_FMT_INFO(logger, "remove 102, create 107"); + StorageSnapshot snap(*storage_pool_mix, nullptr, "xxx", true); // must hold and write + // write delete again + WriteBatch batch; + batch.delPage(102); + batch.putExternal(107, 0); + page_writer_mix->write(std::move(batch), nullptr); + LOG_FMT_INFO(logger, "done"); + } + + { + LOG_FMT_INFO(logger, "second check alive id in v2"); + auto alive_dt_ids_in_v2 = storage_pool_mix->log_storage_v2->getAliveExternalPageIds(TEST_NAMESPACE_ID); + EXPECT_EQ(alive_dt_ids_in_v2.size(), 0) << fmt::format("{}", alive_dt_ids_in_v2); + + storage_pool_mix->log_storage_v3->gc(false, nullptr, nullptr); + auto alive_dt_ids_in_v3 = storage_pool_mix->log_storage_v3->getAliveExternalPageIds(TEST_NAMESPACE_ID); + ASSERT_EQ(alive_dt_ids_in_v3.size(), 3) << fmt::format("{}", alive_dt_ids_in_v3); + auto iter = alive_dt_ids_in_v3.begin(); + EXPECT_EQ(*iter, 105); + iter++; + EXPECT_EQ(*iter, 106); + iter++; + EXPECT_EQ(*iter, 107); + } + { + LOG_FMT_INFO(logger, "third check alive id in v2"); + auto alive_dt_ids_in_v2 = storage_pool_mix->log_storage_v2->getAliveExternalPageIds(TEST_NAMESPACE_ID); + EXPECT_EQ(alive_dt_ids_in_v2.size(), 0) << fmt::format("{}", alive_dt_ids_in_v2); + + storage_pool_mix->log_storage_v3->gc(false, nullptr, nullptr); + auto alive_dt_ids_in_v3 = storage_pool_mix->log_storage_v3->getAliveExternalPageIds(TEST_NAMESPACE_ID); + ASSERT_EQ(alive_dt_ids_in_v3.size(), 3) << fmt::format("{}", alive_dt_ids_in_v3); + auto iter = alive_dt_ids_in_v3.begin(); + EXPECT_EQ(*iter, 105); + iter++; + EXPECT_EQ(*iter, 106); + iter++; + EXPECT_EQ(*iter, 107); + } + + { + // cleanup v3 + WriteBatch batch; + batch.delPage(100); + batch.delPage(101); + batch.delPage(102); + batch.delPage(105); + batch.delPage(106); + batch.delPage(107); + page_writer_mix->write(std::move(batch), nullptr); + } +} +CATCH + TEST_F(PageStorageMixedTest, ReadWithSnapshotAfterMergeDelta) try { From a66c082deb2bdd5bcee47de4f626b4abac314dd8 Mon Sep 17 00:00:00 2001 From: lidezhu <47731263+lidezhu@users.noreply.github.com> Date: Fri, 3 Jun 2022 19:04:27 +0800 Subject: [PATCH 021/104] Fix unstable drop table unit test (#5059) close pingcap/tiflash#5060 --- .../tests/gtest_dm_storage_delta_merge.cpp | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp index 1cb735a2b65..2dc65c256df 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp @@ -717,14 +717,27 @@ CATCH TEST(StorageDeltaMergeTest, RestoreAfterClearData) try { - Context ctx = DMTestEnv::getContext(); - auto & settings = ctx.getSettingsRef(); + auto & global_settings = ::DB::tests::TiFlashTestEnv::getGlobalContext().getSettingsRef(); + // store the old value to restore global_context settings after the test finish to avoid influence other tests + auto old_global_settings = global_settings; + SCOPE_EXIT({ + global_settings = old_global_settings; + }); + // change the settings to make it more easy to trigger splitting segments + Settings settings; settings.dt_segment_limit_rows = 11; settings.dt_segment_limit_size = 20; settings.dt_segment_delta_limit_rows = 7; settings.dt_segment_delta_limit_size = 20; settings.dt_segment_force_split_size = 100; settings.dt_segment_delta_cache_limit_size = 20; + + // we need change the settings in both the ctx we get just below and the global_context above. + // because when processing write request, `DeltaMergeStore` will call `checkSegmentUpdate` with the context we just get below. + // and when initialize `DeltaMergeStore`, it will call `checkSegmentUpdate` with the global_context above. + // so we need to make the settings in these two contexts consistent. + global_settings = settings; + Context ctx = DMTestEnv::getContext(settings); std::shared_ptr storage; DataTypes data_types; Names column_names; From e3a44124f17482bab9039f1b60fc275fe5acfbf2 Mon Sep 17 00:00:00 2001 From: JaySon Date: Mon, 6 Jun 2022 18:14:29 +0800 Subject: [PATCH 022/104] Fix broken ut SegmentDeletionRelevantPlaceTest (#4607) close pingcap/tiflash#4606 --- .../Management/tests/gtest_manual_compact.cpp | 2 +- dbms/src/Interpreters/IDAsPathUpgrader.cpp | 1 + .../Storages/DeltaMerge/tests/CMakeLists.txt | 2 +- .../tests/{dm_basic_include.h => DMTestEnv.h} | 0 .../DeltaMerge/tests/MultiSegmentTestUtil.h | 5 +- .../tests/bank/DeltaMergeStoreProxy.h | 2 +- .../Storages/DeltaMerge/tests/bank/main.cpp | 2 +- .../DeltaMerge/tests/gtest_convert_column.cpp | 3 +- .../DeltaMerge/tests/gtest_data_streams.cpp | 2 +- .../tests/gtest_dm_delta_merge_store.cpp | 5 +- .../tests/gtest_dm_delta_value_space.cpp | 3 +- .../DeltaMerge/tests/gtest_dm_file.cpp | 3 +- .../tests/gtest_dm_minmax_index.cpp | 4 +- .../DeltaMerge/tests/gtest_dm_segment.cpp | 215 ++++++++++-------- .../tests/gtest_dm_segment_common_handle.cpp | 3 +- .../tests/gtest_dm_storage_delta_merge.cpp | 4 +- .../DeltaMerge/tests/gtest_dm_utils.cpp | 3 +- .../DeltaMerge/tests/gtest_version_filter.cpp | 2 +- .../DeltaMerge/tests/stress/DMStressProxy.h | 2 +- 19 files changed, 138 insertions(+), 125 deletions(-) rename dbms/src/Storages/DeltaMerge/tests/{dm_basic_include.h => DMTestEnv.h} (100%) diff --git a/dbms/src/Flash/Management/tests/gtest_manual_compact.cpp b/dbms/src/Flash/Management/tests/gtest_manual_compact.cpp index 8ec3eb54406..4527892e353 100644 --- a/dbms/src/Flash/Management/tests/gtest_manual_compact.cpp +++ b/dbms/src/Flash/Management/tests/gtest_manual_compact.cpp @@ -17,8 +17,8 @@ #include #include #include +#include #include -#include #include #include #include diff --git a/dbms/src/Interpreters/IDAsPathUpgrader.cpp b/dbms/src/Interpreters/IDAsPathUpgrader.cpp index 8c807b537e9..9aa3dcb8dd0 100644 --- a/dbms/src/Interpreters/IDAsPathUpgrader.cpp +++ b/dbms/src/Interpreters/IDAsPathUpgrader.cpp @@ -487,6 +487,7 @@ bool IDAsPathUpgrader::needUpgrade() if (db_info.engine != "TiFlash") { has_old_db_engine = true; + LOG_FMT_INFO(log, "Find old style of database engine, doing upgrade [path={}] [engine={}]", database_metadata_file, db_info.engine); } } diff --git a/dbms/src/Storages/DeltaMerge/tests/CMakeLists.txt b/dbms/src/Storages/DeltaMerge/tests/CMakeLists.txt index 8d5854ffb5d..fd02bcebd2f 100644 --- a/dbms/src/Storages/DeltaMerge/tests/CMakeLists.txt +++ b/dbms/src/Storages/DeltaMerge/tests/CMakeLists.txt @@ -21,7 +21,7 @@ macro(grep_gtest_sources BASE_DIR DST_VAR) endmacro() # attach all dm gtest sources grep_gtest_sources(${TiFlash_SOURCE_DIR}/dbms/src/Storages/DeltaMerge/tests dm_gtest_sources) -add_executable(gtests_dm EXCLUDE_FROM_ALL ${dm_gtest_sources} dm_basic_include.h) +add_executable(gtests_dm EXCLUDE_FROM_ALL ${dm_gtest_sources} DMTestEnv.h) target_link_libraries(gtests_dm gtest_main dbms clickhouse_functions) add_check(gtests_dm) diff --git a/dbms/src/Storages/DeltaMerge/tests/dm_basic_include.h b/dbms/src/Storages/DeltaMerge/tests/DMTestEnv.h similarity index 100% rename from dbms/src/Storages/DeltaMerge/tests/dm_basic_include.h rename to dbms/src/Storages/DeltaMerge/tests/DMTestEnv.h diff --git a/dbms/src/Storages/DeltaMerge/tests/MultiSegmentTestUtil.h b/dbms/src/Storages/DeltaMerge/tests/MultiSegmentTestUtil.h index bc6b7d5c3e6..7c5b0b2416d 100644 --- a/dbms/src/Storages/DeltaMerge/tests/MultiSegmentTestUtil.h +++ b/dbms/src/Storages/DeltaMerge/tests/MultiSegmentTestUtil.h @@ -22,6 +22,7 @@ #include #include #include +#include #include #include #include @@ -30,7 +31,6 @@ #include #include -#include "dm_basic_include.h" namespace DB { @@ -43,7 +43,6 @@ namespace DM { namespace tests { - /// Helper class to test with multiple segments. /// You can call `prepareSegments` to prepare multiple segments. After that, /// you can use `verifyExpectedRowsForAllSegments` to verify the expectation for each segment. @@ -157,4 +156,4 @@ class MultiSegmentTestUtil : private boost::noncopyable } // namespace tests } // namespace DM -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/tests/bank/DeltaMergeStoreProxy.h b/dbms/src/Storages/DeltaMerge/tests/bank/DeltaMergeStoreProxy.h index 99264c77dc6..8edbbbc55b4 100644 --- a/dbms/src/Storages/DeltaMerge/tests/bank/DeltaMergeStoreProxy.h +++ b/dbms/src/Storages/DeltaMerge/tests/bank/DeltaMergeStoreProxy.h @@ -18,9 +18,9 @@ #include #include #include +#include #include #include -#include #include #include diff --git a/dbms/src/Storages/DeltaMerge/tests/bank/main.cpp b/dbms/src/Storages/DeltaMerge/tests/bank/main.cpp index 115e170c48b..b90ad132e25 100644 --- a/dbms/src/Storages/DeltaMerge/tests/bank/main.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/bank/main.cpp @@ -17,8 +17,8 @@ #include #include #include +#include #include -#include #include #include diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_convert_column.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_convert_column.cpp index 928a256349b..efc67de611e 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_convert_column.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_convert_column.cpp @@ -18,10 +18,9 @@ #include #include #include +#include #include -#include "dm_basic_include.h" - namespace DB { namespace DM diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_data_streams.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_data_streams.cpp index f0c2f49c30b..00f31bc97e7 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_data_streams.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_data_streams.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp index 35e6c3d00c6..e934f7a2049 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp @@ -27,6 +27,8 @@ #include #include #include +#include +#include #include #include #include @@ -36,9 +38,6 @@ #include #include -#include "MultiSegmentTestUtil.h" -#include "dm_basic_include.h" - namespace DB { namespace FailPoints diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp index 2b64fd90c09..40c399353b6 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_value_space.cpp @@ -18,14 +18,13 @@ #include #include #include +#include #include #include #include #include -#include "dm_basic_include.h" - namespace CurrentMetrics { extern const Metric DT_SnapshotOfRead; diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp index dfd4419fe38..23062f4ffdf 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_file.cpp @@ -20,13 +20,12 @@ #include #include #include +#include #include #include #include -#include "dm_basic_include.h" - namespace DB { namespace FailPoints diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp index 31fd99faf01..96c0070b73b 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp @@ -22,7 +22,7 @@ #include #include #include -#include +#include #include #include @@ -41,7 +41,7 @@ static const String DEFAULT_COL_NAME = "2020-09-26"; class DMMinMaxIndexTest : public ::testing::Test { public: - DMMinMaxIndexTest() {} + DMMinMaxIndexTest() = default; protected: static void SetUpTestCase() {} diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp index 5726cfa132d..deec5646d33 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment.cpp @@ -18,16 +18,17 @@ #include #include #include +#include +#include #include #include #include #include +#include #include #include -#include "dm_basic_include.h" - namespace CurrentMetrics { extern const Metric DT_SnapshotOfRead; @@ -50,12 +51,10 @@ extern DMFilePtr writeIntoNewDMFile(DMContext & dm_context, // DMFileBlockOutputStream::Flags flags); namespace tests { -class Segment_test : public DB::base::TiFlashStorageTestBasic +class SegmentTest : public DB::base::TiFlashStorageTestBasic { public: - Segment_test() - : storage_pool() - {} + SegmentTest() = default; public: static void SetUpTestCase() {} @@ -63,7 +62,7 @@ class Segment_test : public DB::base::TiFlashStorageTestBasic void SetUp() override { TiFlashStorageTestBasic::SetUp(); - table_columns_ = std::make_shared(); + table_columns = std::make_shared(); segment = reload(); ASSERT_EQ(segment->segmentId(), DELTA_MERGE_FIRST_SEGMENT_ID); @@ -79,43 +78,43 @@ class Segment_test : public DB::base::TiFlashStorageTestBasic ColumnDefinesPtr cols = (!pre_define_columns) ? DMTestEnv::getDefaultColumns() : pre_define_columns; setColumns(cols); - return Segment::newSegment(*dm_context_, table_columns_, RowKeyRange::newAll(false, 1), storage_pool->newMetaPageId(), 0); + return Segment::newSegment(*dm_context, table_columns, RowKeyRange::newAll(false, 1), storage_pool->newMetaPageId(), 0); } // setColumns should update dm_context at the same time void setColumns(const ColumnDefinesPtr & columns) { - *table_columns_ = *columns; + *table_columns = *columns; - dm_context_ = std::make_unique(*db_context, - *storage_path_pool, - *storage_pool, - 0, - /*min_version_*/ 0, - settings.not_compress_columns, - false, - 1, - db_context->getSettingsRef()); + dm_context = std::make_unique(*db_context, + *storage_path_pool, + *storage_pool, + 0, + /*min_version_*/ 0, + settings.not_compress_columns, + false, + 1, + db_context->getSettingsRef()); } - const ColumnDefinesPtr & tableColumns() const { return table_columns_; } + const ColumnDefinesPtr & tableColumns() const { return table_columns; } - DMContext & dmContext() { return *dm_context_; } + DMContext & dmContext() { return *dm_context; } protected: /// all these var lives as ref in dm_context std::unique_ptr storage_path_pool; std::unique_ptr storage_pool; - ColumnDefinesPtr table_columns_; + ColumnDefinesPtr table_columns; DM::DeltaMergeStore::Settings settings; /// dm_context - std::unique_ptr dm_context_; + std::unique_ptr dm_context; // the segment we are going to test SegmentPtr segment; }; -TEST_F(Segment_test, WriteRead) +TEST_F(SegmentTest, WriteRead) try { const size_t num_rows_write = 100; @@ -124,11 +123,11 @@ try // write to segment segment->write(dmContext(), block); // estimate segment - auto estimatedRows = segment->getEstimatedRows(); - ASSERT_EQ(estimatedRows, block.rows()); + auto estimated_rows = segment->getEstimatedRows(); + ASSERT_EQ(estimated_rows, block.rows()); - auto estimatedBytes = segment->getEstimatedBytes(); - ASSERT_EQ(estimatedBytes, block.bytes()); + auto estimated_bytes = segment->getEstimatedBytes(); + ASSERT_EQ(estimated_bytes, block.bytes()); } { @@ -212,7 +211,7 @@ try } CATCH -TEST_F(Segment_test, WriteRead2) +TEST_F(SegmentTest, WriteRead2) try { const size_t num_rows_write = dmContext().stable_pack_rows; @@ -249,7 +248,7 @@ try } CATCH -TEST_F(Segment_test, WriteReadMultiRange) +TEST_F(SegmentTest, WriteReadMultiRange) try { const size_t num_rows_write = 100; @@ -258,11 +257,11 @@ try // write to segment segment->write(dmContext(), block); // estimate segment - auto estimatedRows = segment->getEstimatedRows(); - ASSERT_EQ(estimatedRows, block.rows()); + auto estimated_rows = segment->getEstimatedRows(); + ASSERT_EQ(estimated_rows, block.rows()); - auto estimatedBytes = segment->getEstimatedBytes(); - ASSERT_EQ(estimatedBytes, block.bytes()); + auto estimated_bytes = segment->getEstimatedBytes(); + ASSERT_EQ(estimated_bytes, block.bytes()); } { @@ -356,7 +355,7 @@ try } CATCH -TEST_F(Segment_test, ReadWithMoreAdvacedDeltaIndex) +TEST_F(SegmentTest, ReadWithMoreAdvacedDeltaIndex) try { // Test the case that reading rows with an advance DeltaIndex @@ -421,27 +420,21 @@ try } CATCH -class SegmentDeletionRelevantPlace_test - : public Segment_test +class SegmentDeletionRelevantPlaceTest + : public SegmentTest , public testing::WithParamInterface { - DB::Settings getSettings() - { - DB::Settings settings; - auto enable_relevant_place = GetParam(); - - if (enable_relevant_place) - settings.set("dt_enable_relevant_place", "1"); - else - settings.set("dt_enable_relevant_place", "0"); - return settings; - } }; -TEST_P(SegmentDeletionRelevantPlace_test, ShareDelteRangeIndex) +TEST_P(SegmentDeletionRelevantPlaceTest, ShareDelteRangeIndex) try { + Settings my_settings; + const auto enable_relevant_place = GetParam(); + my_settings.dt_enable_relevant_place = enable_relevant_place; + this->reload({}, std::move(my_settings)); + const size_t num_rows_write = 300; { // write to segment @@ -467,27 +460,54 @@ try { HandleRange remove(100, 200); - segment->write(dmContext(), {RowKeyRange::fromHandleRange(remove)}); + segment->write(dmContext(), /*delete_range*/ {RowKeyRange::fromHandleRange(remove)}); } // The first call of get_rows below will place the DeleteRange into delta index. + // If relevant place is enabled, the placed deletes in delta-tree-index is not + // pushed forward since we do not fully apply the delete range [100, 200). auto rows1 = get_rows(RowKeyRange::fromHandleRange(HandleRange(0, 150))); + { + auto delta = segment->getDelta(); + auto placed_rows = delta->getPlacedDeltaRows(); + auto placed_deletes = delta->getPlacedDeltaDeletes(); + ASSERT_EQ(placed_rows, num_rows_write); + EXPECT_EQ(placed_deletes, enable_relevant_place ? 0 : 1); + } auto rows2 = get_rows(RowKeyRange::fromHandleRange(HandleRange(150, 300))); + { + auto delta = segment->getDelta(); + auto placed_rows = delta->getPlacedDeltaRows(); + auto placed_deletes = delta->getPlacedDeltaDeletes(); + ASSERT_EQ(placed_rows, num_rows_write); + EXPECT_EQ(placed_deletes, enable_relevant_place ? 0 : 1); + } + // Query with range [0, 300) will push the placed deletes forward no matter + // relevant place is enable or not. + auto rows3 = get_rows(RowKeyRange::fromHandleRange(HandleRange(0, 300))); + { + auto delta = segment->getDelta(); + auto placed_rows = delta->getPlacedDeltaRows(); + auto placed_deletes = delta->getPlacedDeltaDeletes(); + ASSERT_EQ(placed_rows, num_rows_write); + EXPECT_EQ(placed_deletes, 1); + } - ASSERT_EQ(rows1, (size_t)100); - ASSERT_EQ(rows2, (size_t)100); + ASSERT_EQ(rows1, 100); + ASSERT_EQ(rows2, 100); + ASSERT_EQ(rows3, 200); } CATCH -INSTANTIATE_TEST_CASE_P(WhetherEnableRelevantPlace, SegmentDeletionRelevantPlace_test, testing::Values(true, false)); +INSTANTIATE_TEST_CASE_P(WhetherEnableRelevantPlace, SegmentDeletionRelevantPlaceTest, testing::Values(true, false)); -class SegmentDeletion_test - : public Segment_test +class SegmentDeletionTest + : public SegmentTest , public testing::WithParamInterface> { }; -TEST_P(SegmentDeletion_test, DeleteDataInDelta) +TEST_P(SegmentDeletionTest, DeleteDataInDelta) try { const size_t num_rows_write = 100; @@ -565,7 +585,7 @@ try } CATCH -TEST_P(SegmentDeletion_test, DeleteDataInStable) +TEST_P(SegmentDeletionTest, DeleteDataInStable) try { const size_t num_rows_write = 100; @@ -651,7 +671,7 @@ try } CATCH -TEST_P(SegmentDeletion_test, DeleteDataInStableAndDelta) +TEST_P(SegmentDeletionTest, DeleteDataInStableAndDelta) try { const size_t num_rows_write = 100; @@ -738,9 +758,9 @@ try } CATCH -INSTANTIATE_TEST_CASE_P(WhetherReadOrMergeDeltaBeforeDeleteRange, SegmentDeletion_test, testing::Combine(testing::Bool(), testing::Bool())); +INSTANTIATE_TEST_CASE_P(WhetherReadOrMergeDeltaBeforeDeleteRange, SegmentDeletionTest, testing::Combine(testing::Bool(), testing::Bool())); -TEST_F(Segment_test, DeleteRead) +TEST_F(SegmentTest, DeleteRead) try { const size_t num_rows_write = 64; @@ -946,7 +966,7 @@ try } CATCH -TEST_F(Segment_test, Split) +TEST_F(SegmentTest, Split) try { const size_t num_rows_write_per_batch = 100; @@ -1046,7 +1066,7 @@ try } CATCH -TEST_F(Segment_test, SplitFail) +TEST_F(SegmentTest, SplitFail) try { const size_t num_rows_write = 100; @@ -1066,7 +1086,7 @@ try } CATCH -TEST_F(Segment_test, Restore) +TEST_F(SegmentTest, Restore) try { // compare will compares the given segments. @@ -1158,7 +1178,7 @@ try } CATCH -TEST_F(Segment_test, MassiveSplit) +TEST_F(SegmentTest, MassiveSplit) try { Settings settings = dmContext().db_context.getSettings(); @@ -1242,52 +1262,51 @@ try } CATCH -enum Segment_test_Mode +enum SegmentTestMode { V1_BlockOnly, V2_BlockOnly, V2_FileOnly, }; -String testModeToString(const ::testing::TestParamInfo & info) +String testModeToString(const ::testing::TestParamInfo & info) { const auto mode = info.param; switch (mode) { - case Segment_test_Mode::V1_BlockOnly: + case SegmentTestMode::V1_BlockOnly: return "V1_BlockOnly"; - case Segment_test_Mode::V2_BlockOnly: + case SegmentTestMode::V2_BlockOnly: return "V2_BlockOnly"; - case Segment_test_Mode::V2_FileOnly: + case SegmentTestMode::V2_FileOnly: return "V2_FileOnly"; default: return "Unknown"; } } -class Segment_test_2 : public Segment_test - , public testing::WithParamInterface +class SegmentTest2 : public SegmentTest + , public testing::WithParamInterface { public: - Segment_test_2() - : Segment_test() - {} + SegmentTest2() = default; + void SetUp() override { mode = GetParam(); switch (mode) { - case Segment_test_Mode::V1_BlockOnly: + case SegmentTestMode::V1_BlockOnly: setStorageFormat(1); break; - case Segment_test_Mode::V2_BlockOnly: - case Segment_test_Mode::V2_FileOnly: + case SegmentTestMode::V2_BlockOnly: + case SegmentTestMode::V2_FileOnly: setStorageFormat(2); break; } - Segment_test::SetUp(); + SegmentTest::SetUp(); } std::pair genDMFile(DMContext & context, const Block & block) @@ -1305,7 +1324,7 @@ class Segment_test_2 : public Segment_test delegator.addDTFile(file_id, dmfile->getBytesOnDisk(), store_path); - auto & pk_column = block.getByPosition(0).column; + const auto & pk_column = block.getByPosition(0).column; auto min_pk = pk_column->getInt(0); auto max_pk = pk_column->getInt(block.rows() - 1); HandleRange range(min_pk, max_pk + 1); @@ -1313,10 +1332,10 @@ class Segment_test_2 : public Segment_test return {RowKeyRange::fromHandleRange(range), {file_id}}; } - Segment_test_Mode mode; + SegmentTestMode mode; }; -TEST_P(Segment_test_2, FlushDuringSplitAndMerge) +TEST_P(SegmentTest2, FlushDuringSplitAndMerge) try { size_t row_offset = 0; @@ -1327,11 +1346,11 @@ try row_offset += 100; switch (mode) { - case Segment_test_Mode::V1_BlockOnly: - case Segment_test_Mode::V2_BlockOnly: + case SegmentTestMode::V1_BlockOnly: + case SegmentTestMode::V2_BlockOnly: segment->write(dmContext(), std::move(block)); break; - case Segment_test_Mode::V2_FileOnly: + case SegmentTestMode::V2_FileOnly: { auto delegate = dmContext().path_pool.getStableDiskDelegator(); auto file_provider = dmContext().db_context.getFileProvider(); @@ -1430,9 +1449,9 @@ try } CATCH -INSTANTIATE_TEST_CASE_P(Segment_test_Mode, // - Segment_test_2, - testing::Values(Segment_test_Mode::V1_BlockOnly, Segment_test_Mode::V2_BlockOnly, Segment_test_Mode::V2_FileOnly), +INSTANTIATE_TEST_CASE_P(SegmentTestMode, // + SegmentTest2, + testing::Values(SegmentTestMode::V1_BlockOnly, SegmentTestMode::V2_BlockOnly, SegmentTestMode::V2_FileOnly), testModeToString); enum class SegmentWriteType @@ -1440,12 +1459,12 @@ enum class SegmentWriteType ToDisk, ToCache }; -class Segment_DDL_test - : public Segment_test +class SegmentDDLTest + : public SegmentTest , public testing::WithParamInterface> { }; -String paramToString(const ::testing::TestParamInfo & info) +String paramToString(const ::testing::TestParamInfo & info) { const auto [write_type, flush_before_ddl] = info.param; @@ -1455,7 +1474,7 @@ String paramToString(const ::testing::TestParamInfo } /// Mock a col from i8 -> i32 -TEST_P(Segment_DDL_test, AlterInt8ToInt32) +TEST_P(SegmentDDLTest, AlterInt8ToInt32) try { const String column_name_i8_to_i32 = "i8_to_i32"; @@ -1627,7 +1646,7 @@ try } CATCH -TEST_P(Segment_DDL_test, AddColumn) +TEST_P(SegmentDDLTest, AddColumn) try { const String new_column_name = "i8"; @@ -1795,7 +1814,7 @@ try } CATCH -TEST_F(Segment_test, CalculateDTFileProperty) +TEST_F(SegmentTest, CalculateDTFileProperty) try { Settings settings = dmContext().db_context.getSettings(); @@ -1836,7 +1855,7 @@ try } CATCH -TEST_F(Segment_test, CalculateDTFilePropertyWithPropertyFileDeleted) +TEST_F(SegmentTest, CalculateDTFilePropertyWithPropertyFileDeleted) try { Settings settings = dmContext().db_context.getSettings(); @@ -1857,10 +1876,10 @@ try } { - auto & stable = segment->getStable(); - auto & dmfiles = stable->getDMFiles(); + const auto & stable = segment->getStable(); + const auto & dmfiles = stable->getDMFiles(); ASSERT_GT(dmfiles[0]->getPacks(), (size_t)1); - auto & dmfile = dmfiles[0]; + const auto & dmfile = dmfiles[0]; auto file_path = dmfile->path(); // check property file exists and then delete it ASSERT_EQ(Poco::File(file_path + "/property").exists(), true); @@ -1877,7 +1896,7 @@ try // calculate the StableProperty for packs in the key range [0, num_rows_write_every_round) stable->calculateStableProperty(dmContext(), range, false); ASSERT_EQ(stable->isStablePropertyCached(), true); - auto & property = stable->getStableProperty(); + const auto & property = stable->getStableProperty(); ASSERT_EQ(property.gc_hint_version, std::numeric_limits::max()); ASSERT_EQ(property.num_versions, num_rows_write_every_round); ASSERT_EQ(property.num_puts, num_rows_write_every_round); @@ -1887,7 +1906,7 @@ try CATCH INSTANTIATE_TEST_CASE_P(SegmentWriteType, - Segment_DDL_test, + SegmentDDLTest, ::testing::Combine( // ::testing::Values(SegmentWriteType::ToDisk, SegmentWriteType::ToCache), ::testing::Bool()), diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment_common_handle.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment_common_handle.cpp index 1cc61663a2f..6359a3db184 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment_common_handle.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_segment_common_handle.cpp @@ -15,14 +15,13 @@ #include #include #include +#include #include #include #include #include -#include "dm_basic_include.h" - namespace DB { namespace DM diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp index 2dc65c256df..f929e153847 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_storage_delta_merge.cpp @@ -31,6 +31,8 @@ #include #include #include +#include +#include #include #include #include @@ -41,8 +43,6 @@ #include -#include "dm_basic_include.h" - namespace DB { namespace FailPoints diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_utils.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_utils.cpp index 340dad1ff6e..26a5b1132e4 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_utils.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_utils.cpp @@ -14,10 +14,9 @@ #include #include +#include #include -#include "dm_basic_include.h" - namespace DB { namespace DM diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_version_filter.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_version_filter.cpp index 59052e2e8b2..16b1729bea1 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_version_filter.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_version_filter.cpp @@ -15,7 +15,7 @@ #include #include #include -#include +#include namespace DB { diff --git a/dbms/src/Storages/DeltaMerge/tests/stress/DMStressProxy.h b/dbms/src/Storages/DeltaMerge/tests/stress/DMStressProxy.h index eb4d937e8d4..0571eafae83 100644 --- a/dbms/src/Storages/DeltaMerge/tests/stress/DMStressProxy.h +++ b/dbms/src/Storages/DeltaMerge/tests/stress/DMStressProxy.h @@ -18,7 +18,7 @@ #include #include #include -#include +#include #include #include From 5847f1c235b996eb6fb970029da926909e1819fd Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Wed, 8 Jun 2022 09:06:30 +0800 Subject: [PATCH 023/104] Add `Precompiled header` for modules & Refine by PImpl to accelerate release build (#5047) ref pingcap/tiflash#4909 --- CMakeLists.txt | 8 ++ cmake/find_ccache.cmake | 14 ++++ dbms/CMakeLists.txt | 32 ++++++++ dbms/pch-common.h | 23 ++++++ dbms/pch-dbms.h | 18 +++++ dbms/pch-kvpb.h | 26 +++++++ dbms/pch-stl.h | 34 ++++++++ dbms/src/Common/FmtUtils.h | 10 +-- dbms/src/Common/TiFlashException.cpp | 77 ++++++++++++++++++- dbms/src/Common/TiFlashException.h | 58 ++++---------- .../DataStreams/TiRemoteBlockInputStream.h | 1 - .../src/Flash/Coprocessor/CoprocessorReader.h | 2 +- .../Coprocessor/DAGStorageInterpreter.cpp | 1 + .../Flash/Coprocessor/DAGStorageInterpreter.h | 2 +- .../Flash/Coprocessor/GenSchemaAndColumn.cpp | 1 + dbms/src/Flash/Coprocessor/RemoteRequest.cpp | 2 + dbms/src/Flash/Coprocessor/RemoteRequest.h | 3 +- .../Flash/Coprocessor/TablesRegionsInfo.cpp | 1 + .../src/Flash/Coprocessor/TablesRegionsInfo.h | 3 +- .../Management/tests/gtest_manual_compact.cpp | 1 + dbms/src/Flash/Mpp/MinTSOScheduler.h | 11 ++- dbms/src/Interpreters/Context.h | 1 - dbms/src/Server/CLIService.h | 1 - dbms/src/Storages/Transaction/KVStore.cpp | 36 +++++---- dbms/src/Storages/Transaction/KVStore.h | 21 +++-- dbms/src/Storages/Transaction/ProxyFFI.cpp | 2 + .../Storages/Transaction/ReadIndexWorker.cpp | 7 +- dbms/src/Storages/Transaction/TMTContext.h | 6 -- dbms/src/TestUtils/ColumnsToTiPBExpr.cpp | 1 + dbms/src/TiDB/Schema/TiDBSchemaSyncer.h | 1 - .../scripts/build-tiflash-release.sh | 2 + 31 files changed, 307 insertions(+), 99 deletions(-) create mode 100644 dbms/pch-common.h create mode 100644 dbms/pch-dbms.h create mode 100644 dbms/pch-kvpb.h create mode 100644 dbms/pch-stl.h diff --git a/CMakeLists.txt b/CMakeLists.txt index f1dd740b0b4..4e14c205f18 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -48,6 +48,14 @@ endif () set(CMAKE_EXPORT_COMPILE_COMMANDS 1) option (USE_CCACHE "Set to OFF to disable ccache" ON) +if (USE_CCACHE) + set(NOT_USE_CCACHE 0) +else() + set(NOT_USE_CCACHE 1) +endif() + +option(ENABLE_PCH "Enable `Precompiled header`" OFF) + include (cmake/find_ccache.cmake) if (NOT CMAKE_BUILD_TYPE OR CMAKE_BUILD_TYPE STREQUAL "None") diff --git a/cmake/find_ccache.cmake b/cmake/find_ccache.cmake index 910caebb046..71fa337a922 100644 --- a/cmake/find_ccache.cmake +++ b/cmake/find_ccache.cmake @@ -24,6 +24,20 @@ if (USE_CCACHE AND CCACHE_FOUND AND NOT CMAKE_CXX_COMPILER_LAUNCHER MATCHES "cca message ("${CCACHE_CONFIG}") set_property (GLOBAL PROPERTY RULE_LAUNCH_COMPILE ${CCACHE_FOUND}) set_property (GLOBAL PROPERTY RULE_LAUNCH_LINK ${CCACHE_FOUND}) + + if (ENABLE_PCH) + execute_process (COMMAND ${CCACHE_FOUND} --get-config sloppiness OUTPUT_VARIABLE _CCACHE_SLOPPINESS OUTPUT_STRIP_TRAILING_WHITESPACE) + string (FIND "${_CCACHE_SLOPPINESS}" "pch_defines" _CCACHE_SLOPPINESS_RES) + if (NOT _CCACHE_SLOPPINESS_RES STREQUAL "-1") + string (FIND "${_CCACHE_SLOPPINESS}" "time_macros" _CCACHE_SLOPPINESS_RES) + endif () + + if (_CCACHE_SLOPPINESS_RES STREQUAL "-1") + message(WARNING "`Precompiled header` won't be cached by ccache, sloppiness = `${CCACHE_SLOPPINESS}`,please execute `ccache -o sloppiness=pch_defines,time_macros`") + set (ENABLE_PCH FALSE CACHE BOOL "" FORCE) + endif () + endif () + else () message (STATUS "Not using ccache ${CCACHE_FOUND}, USE_CCACHE=${USE_CCACHE}") endif () diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 21cf06cbe31..cce11bd6997 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -259,6 +259,16 @@ target_include_directories (clickhouse_common_io BEFORE PRIVATE ${COMMON_INCLUDE # https://cmake.org/pipermail/cmake/2016-May/063400.html target_link_libraries (clickhouse_common_io PUBLIC ${TIFLASH_XXHASH_LIBRARY}) +function(add_target_pch context target) + if (ENABLE_PCH) + message(STATUS "Add PCH `${context}` for target `${target}`") + target_precompile_headers(${target} PRIVATE ${context}) + endif () + if(${ARGC} GREATER 2) + add_target_pch(${context} ${ARGN}) + endif() +endfunction() + if (ENABLE_TESTS) include (${TiFlash_SOURCE_DIR}/cmake/find_gtest.cmake) @@ -297,6 +307,8 @@ if (ENABLE_TESTS) target_compile_options(gtests_dbms PRIVATE -Wno-unknown-pragmas -Wno-deprecated-copy) add_check(gtests_dbms) + add_target_pch("pch-dbms.h" gtests_dbms) + grep_bench_sources(${TiFlash_SOURCE_DIR}/dbms dbms_bench_sources) add_executable(bench_dbms EXCLUDE_FROM_ALL ${dbms_bench_sources} @@ -342,3 +354,23 @@ if (TEST_COVERAGE AND CMAKE_BUILD_TYPE STREQUAL "Debug") ) endif () endif () + +# dbms +add_target_pch("pch-dbms.h" dbms flash_service) +add_target_pch("pch-common.h" clickhouse_common_io clickhouse_functions clickhouse_aggregate_functions) +add_target_pch("pch-common.h" clickhouse_parsers clickhouse_storages_system dt-workload-lib clickhouse-server-lib) + +# common +add_target_pch("pch-kvpb.h" kv_client) + +add_target_pch("pch-stl.h" ${Boost_SYSTEM_LIBRARY} cctz ${RE2_LIBRARY} ${RE2_ST_LIBRARY}) + +# grpc +add_target_pch("$<$:${CMAKE_CURRENT_SOURCE_DIR}/pch-stl.h>" grpc grpc++) + +# pb +add_target_pch("pch-stl.h" libprotobuf kvproto tipb libprotoc) + +# poco +add_target_pch("pch-stl.h" Net Crypto Util Data NetSSL) +add_target_pch("$<$:${CMAKE_CURRENT_SOURCE_DIR}/pch-stl.h>" XML Foundation JSON) diff --git a/dbms/pch-common.h b/dbms/pch-common.h new file mode 100644 index 00000000000..878254a3529 --- /dev/null +++ b/dbms/pch-common.h @@ -0,0 +1,23 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include +#include + +#include "pch-stl.h" diff --git a/dbms/pch-dbms.h b/dbms/pch-dbms.h new file mode 100644 index 00000000000..60687073bf8 --- /dev/null +++ b/dbms/pch-dbms.h @@ -0,0 +1,18 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include "pch-common.h" +#include "pch-kvpb.h" diff --git a/dbms/pch-kvpb.h b/dbms/pch-kvpb.h new file mode 100644 index 00000000000..d74bfc6bb89 --- /dev/null +++ b/dbms/pch-kvpb.h @@ -0,0 +1,26 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +/** +There may be unexpected behaviors for `ccache` to deal with PCH which includes those header files generated by tools: + 'xxx' has been modified since the precompiled header 'xxx' was built: mtime changed + `Precompiled header includes xxx.h, which has a new mtime` +*/ +#include +#include +#include + +#include "pch-stl.h" diff --git a/dbms/pch-stl.h b/dbms/pch-stl.h new file mode 100644 index 00000000000..01b3123650d --- /dev/null +++ b/dbms/pch-stl.h @@ -0,0 +1,34 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include \ No newline at end of file diff --git a/dbms/src/Common/FmtUtils.h b/dbms/src/Common/FmtUtils.h index 71f93a34078..ab37194ed10 100644 --- a/dbms/src/Common/FmtUtils.h +++ b/dbms/src/Common/FmtUtils.h @@ -14,8 +14,6 @@ #pragma once -#include -#include #include namespace DB @@ -32,9 +30,9 @@ class FmtBuffer return *this; } - FmtBuffer & append(StringRef s) + FmtBuffer & append(std::string_view s) { - buffer.append(s.data, s.data + s.size); + buffer.append(s.data(), s.data() + s.size()); return *this; } @@ -55,7 +53,7 @@ class FmtBuffer FmtBuffer & joinStr( Iter first, Iter end, - StringRef delimiter) + std::string_view delimiter) { auto func = [](const auto & s, FmtBuffer & fb) { fb.append(s); @@ -68,7 +66,7 @@ class FmtBuffer Iter first, Iter end, FF && toStringFunc, // void (const auto &, FmtBuffer &) - StringRef delimiter) + std::string_view delimiter) { if (first == end) return *this; diff --git a/dbms/src/Common/TiFlashException.cpp b/dbms/src/Common/TiFlashException.cpp index bd28a57c093..11ee229ded0 100644 --- a/dbms/src/Common/TiFlashException.cpp +++ b/dbms/src/Common/TiFlashException.cpp @@ -18,6 +18,32 @@ namespace DB { +struct TiFlashErrorRegistry::Errors : std::map, TiFlashError> +{ +}; + +TiFlashErrorRegistry::Errors & TiFlashErrorRegistry::errors() +{ + return *inner_data; +} + +TiFlashErrorRegistry::Errors & TiFlashErrorRegistry::errors() const +{ + return *inner_data; +} + +TiFlashErrorRegistry::TiFlashErrorRegistry() + : inner_data(new Errors{}) +{ + initialize(); +} + +TiFlashErrorRegistry::~TiFlashErrorRegistry() +{ + delete inner_data; + inner_data = nullptr; +} + void TiFlashErrorRegistry::initialize() { // Used to check uniqueness of classes @@ -46,9 +72,9 @@ void TiFlashErrorRegistry::initialize() void TiFlashErrorRegistry::registerError(const std::string & error_class, const std::string & error_code, const std::string & description, const std::string & workaround, const std::string & message_template) { TiFlashError error{error_class, error_code, description, workaround, message_template}; - if (all_errors.find({error_class, error_code}) == all_errors.end()) + if (errors().find({error_class, error_code}) == errors().end()) { - all_errors.emplace(std::make_pair(error_class, error_code), std::move(error)); + errors().emplace(std::make_pair(error_class, error_code), std::move(error)); } else { @@ -77,4 +103,51 @@ std::string TiFlashException::standardText() const return text; } +std::optional TiFlashErrorRegistry::get(const std::string & error_class, const std::string & error_code) const +{ + auto error = errors().find({error_class, error_code}); + if (error != errors().end()) + { + return error->second; + } + else + { + return {}; + } +} +std::optional TiFlashErrorRegistry::get(const std::string & error_class, int error_code) const +{ + return get(error_class, std::to_string(error_code)); +} + +std::vector TiFlashErrorRegistry::allErrors() const +{ + std::vector res; + res.reserve(errors().size()); + for (const auto & error : errors()) + { + res.push_back(error.second); + } + return res; +} + +TiFlashError TiFlashErrorRegistry::simpleGet(const std::string & error_class, const std::string & error_code) +{ + auto & m_instance = instance(); + auto error = m_instance.get(error_class, error_code); + if (error.has_value()) + { + return error.value(); + } + else + { + throw Exception("Unregistered TiFlashError: FLASH:" + error_class + ":" + error_code); + } +} +TiFlashError TiFlashErrorRegistry::simpleGet(const std::string & error_class, int error_code) +{ + return simpleGet(error_class, std::to_string(error_code)); +} + + } // namespace DB diff --git a/dbms/src/Common/TiFlashException.h b/dbms/src/Common/TiFlashException.h index 2026571859e..3b4e3d75813 100644 --- a/dbms/src/Common/TiFlashException.h +++ b/dbms/src/Common/TiFlashException.h @@ -194,56 +194,19 @@ class TiFlashErrorRegistry : public ext::Singleton public: friend ext::Singleton; - static TiFlashError simpleGet(const std::string & error_class, const std::string & error_code) - { - auto & m_instance = instance(); - auto error = m_instance.get(error_class, error_code); - if (error.has_value()) - { - return error.value(); - } - else - { - throw Exception("Unregistered TiFlashError: FLASH:" + error_class + ":" + error_code); - } - } + static TiFlashError simpleGet(const std::string & error_class, const std::string & error_code); - static TiFlashError simpleGet(const std::string & error_class, int error_code) - { - return simpleGet(error_class, std::to_string(error_code)); - } + static TiFlashError simpleGet(const std::string & error_class, int error_code); - std::optional get(const std::string & error_class, const std::string & error_code) const - { - auto error = all_errors.find({error_class, error_code}); - if (error != all_errors.end()) - { - return error->second; - } - else - { - return {}; - } - } + std::optional get(const std::string & error_class, const std::string & error_code) const; - std::optional get(const std::string & error_class, int error_code) const - { - return get(error_class, std::to_string(error_code)); - } + std::optional get(const std::string & error_class, int error_code) const; - std::vector allErrors() const - { - std::vector res; - res.reserve(all_errors.size()); - for (const auto & error : all_errors) - { - res.push_back(error.second); - } - return res; - } + std::vector allErrors() const; protected: - TiFlashErrorRegistry() { initialize(); } + TiFlashErrorRegistry(); + ~TiFlashErrorRegistry(); private: void registerError(const std::string & error_class, const std::string & error_code, const std::string & description, const std::string & workaround, const std::string & message_template = ""); @@ -252,8 +215,13 @@ class TiFlashErrorRegistry : public ext::Singleton void initialize(); + struct Errors; + + Errors & errors(); + Errors & errors() const; + private: - std::map, TiFlashError> all_errors; + Errors * inner_data; // PImpl }; /// TiFlashException implements TiDB's standardized error. diff --git a/dbms/src/DataStreams/TiRemoteBlockInputStream.h b/dbms/src/DataStreams/TiRemoteBlockInputStream.h index f8e313a25be..f249bf1a0dc 100644 --- a/dbms/src/DataStreams/TiRemoteBlockInputStream.h +++ b/dbms/src/DataStreams/TiRemoteBlockInputStream.h @@ -22,7 +22,6 @@ #include #include #include -#include #include #include diff --git a/dbms/src/Flash/Coprocessor/CoprocessorReader.h b/dbms/src/Flash/Coprocessor/CoprocessorReader.h index 8a3eb471e54..25c07cff49c 100644 --- a/dbms/src/Flash/Coprocessor/CoprocessorReader.h +++ b/dbms/src/Flash/Coprocessor/CoprocessorReader.h @@ -20,7 +20,6 @@ #include #include #include -#include #include #include @@ -29,6 +28,7 @@ #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wunused-parameter" +#pragma GCC diagnostic ignored "-Wdeprecated-declarations" #include #include #include diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index 11a1b7e2d3e..df7e504d2c4 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -33,6 +33,7 @@ #include #include #include +#include #include #pragma GCC diagnostic push diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h index d86274a1e22..0425abe04db 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h @@ -24,7 +24,6 @@ #include #include #include -#include #include #include #include @@ -33,6 +32,7 @@ namespace DB { +class TMTContext; using TablesRegionInfoMap = std::unordered_map>; /// DAGStorageInterpreter encapsulates operations around storage during interprete stage. /// It's only intended to be used by DAGQueryBlockInterpreter. diff --git a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp index e7964021709..be3475f714f 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include #include +#include namespace DB { diff --git a/dbms/src/Flash/Coprocessor/RemoteRequest.cpp b/dbms/src/Flash/Coprocessor/RemoteRequest.cpp index 086cdb43d20..d3b5c202136 100644 --- a/dbms/src/Flash/Coprocessor/RemoteRequest.cpp +++ b/dbms/src/Flash/Coprocessor/RemoteRequest.cpp @@ -13,8 +13,10 @@ // limitations under the License. #include +#include #include #include +#include namespace DB { diff --git a/dbms/src/Flash/Coprocessor/RemoteRequest.h b/dbms/src/Flash/Coprocessor/RemoteRequest.h index 1e42e18a7bd..5af3f66298c 100644 --- a/dbms/src/Flash/Coprocessor/RemoteRequest.h +++ b/dbms/src/Flash/Coprocessor/RemoteRequest.h @@ -17,11 +17,12 @@ #include #include #include -#include #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wunused-parameter" +#pragma GCC diagnostic ignored "-Wdeprecated-declarations" #include +#include #include #pragma GCC diagnostic pop diff --git a/dbms/src/Flash/Coprocessor/TablesRegionsInfo.cpp b/dbms/src/Flash/Coprocessor/TablesRegionsInfo.cpp index 7c4ddedabf4..ab4a0f82e95 100644 --- a/dbms/src/Flash/Coprocessor/TablesRegionsInfo.cpp +++ b/dbms/src/Flash/Coprocessor/TablesRegionsInfo.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB { diff --git a/dbms/src/Flash/Coprocessor/TablesRegionsInfo.h b/dbms/src/Flash/Coprocessor/TablesRegionsInfo.h index dccb8b95466..f80a44b92a3 100644 --- a/dbms/src/Flash/Coprocessor/TablesRegionsInfo.h +++ b/dbms/src/Flash/Coprocessor/TablesRegionsInfo.h @@ -15,7 +15,7 @@ #pragma once #include #include -#include + #pragma GCC diagnostic push #pragma GCC diagnostic ignored "-Wunused-parameter" @@ -25,6 +25,7 @@ namespace DB { +class TMTContext; struct SingleTableRegions { RegionInfoMap local_regions; diff --git a/dbms/src/Flash/Management/tests/gtest_manual_compact.cpp b/dbms/src/Flash/Management/tests/gtest_manual_compact.cpp index 4527892e353..df6c881c306 100644 --- a/dbms/src/Flash/Management/tests/gtest_manual_compact.cpp +++ b/dbms/src/Flash/Management/tests/gtest_manual_compact.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Flash/Mpp/MinTSOScheduler.h b/dbms/src/Flash/Mpp/MinTSOScheduler.h index 17ab1f4dfa3..dbc0cb84cc3 100644 --- a/dbms/src/Flash/Mpp/MinTSOScheduler.h +++ b/dbms/src/Flash/Mpp/MinTSOScheduler.h @@ -15,12 +15,19 @@ #pragma once #include -#include -#include #include namespace DB { +class MinTSOScheduler; +using MPPTaskSchedulerPtr = std::unique_ptr; + +class MPPTaskManager; +using MPPTaskManagerPtr = std::shared_ptr; + +struct MPPQueryTaskSet; +using MPPQueryTaskSetPtr = std::shared_ptr; + /// scheduling tasks in the set according to the tso order under the soft limit of threads, but allow the min_tso query to preempt threads under the hard limit of threads. /// The min_tso query avoids the deadlock resulted from threads competition among nodes. /// schedule tasks under the lock protection of the task manager. diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 63aefcbece9..5d5c39263c6 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -14,7 +14,6 @@ #pragma once -#include #include #include #include diff --git a/dbms/src/Server/CLIService.h b/dbms/src/Server/CLIService.h index 18c9d61260f..9078fa991f3 100644 --- a/dbms/src/Server/CLIService.h +++ b/dbms/src/Server/CLIService.h @@ -25,7 +25,6 @@ #include #include #include -#include #include #include diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 2b82220cbc4..318a04c6ed9 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include #include @@ -38,15 +39,15 @@ extern const int TABLE_IS_DROPPED; } // namespace ErrorCodes KVStore::KVStore(Context & context, TiDB::SnapshotApplyMethod snapshot_apply_method_) - : region_persister(context, region_manager) + : region_persister(std::make_unique(context, region_manager)) , raft_cmd_res(std::make_unique()) , snapshot_apply_method(snapshot_apply_method_) , log(&Poco::Logger::get("KVStore")) + , region_compact_log_period(120) + , region_compact_log_min_rows(40 * 1024) + , region_compact_log_min_bytes(32 * 1024 * 1024) { // default config about compact-log: period 120s, rows 40k, bytes 32MB. - REGION_COMPACT_LOG_PERIOD = 120; - REGION_COMPACT_LOG_MIN_ROWS = 40 * 1024; - REGION_COMPACT_LOG_MIN_BYTES = 32 * 1024 * 1024; } void KVStore::restore(const TiFlashRaftProxyHelper * proxy_helper) @@ -55,7 +56,7 @@ void KVStore::restore(const TiFlashRaftProxyHelper * proxy_helper) auto manage_lock = genRegionWriteLock(task_lock); this->proxy_helper = proxy_helper; - manage_lock.regions = region_persister.restore(proxy_helper); + manage_lock.regions = region_persister->restore(proxy_helper); LOG_FMT_INFO(log, "Restored {} regions", manage_lock.regions.size()); @@ -166,7 +167,7 @@ void KVStore::tryPersist(RegionID region_id) if (region) { LOG_FMT_INFO(log, "Try to persist {}", region->toString(false)); - region_persister.persist(*region); + region_persister->persist(*region); LOG_FMT_INFO(log, "After persisted {}, cache {} bytes", region->toString(false), region->dataSize()); } } @@ -182,7 +183,7 @@ void KVStore::gcRegionPersistedCache(Seconds gc_persist_period) if (now < (last_gc_time.load() + gc_persist_period)) return; last_gc_time = now; - region_persister.gc(); + region_persister->gc(); } void KVStore::removeRegion(RegionID region_id, bool remove_data, RegionTable & region_table, const KVStoreTaskLock & task_lock, const RegionTaskLock & region_lock) @@ -203,7 +204,7 @@ void KVStore::removeRegion(RegionID region_id, bool remove_data, RegionTable & r } } - region_persister.drop(region_id, region_lock); + region_persister->drop(region_id, region_lock); LOG_FMT_INFO(log, "Persisted [region {}] deleted", region_id); region_table.removeRegion(region_id, remove_data, region_lock); @@ -306,9 +307,9 @@ void KVStore::handleDestroy(UInt64 region_id, TMTContext & tmt, const KVStoreTas void KVStore::setRegionCompactLogConfig(UInt64 sec, UInt64 rows, UInt64 bytes) { - REGION_COMPACT_LOG_PERIOD = sec; - REGION_COMPACT_LOG_MIN_ROWS = rows; - REGION_COMPACT_LOG_MIN_BYTES = bytes; + region_compact_log_period = sec; + region_compact_log_min_rows = rows; + region_compact_log_min_bytes = bytes; LOG_FMT_INFO( log, @@ -321,7 +322,7 @@ void KVStore::setRegionCompactLogConfig(UInt64 sec, UInt64 rows, UInt64 bytes) void KVStore::persistRegion(const Region & region, const RegionTaskLock & region_task_lock, const char * caller) { LOG_FMT_INFO(log, "Start to persist {}, cache size: {} bytes for `{}`", region.toString(true), region.dataSize(), caller); - region_persister.persist(region, region_task_lock); + region_persister->persist(region, region_task_lock); LOG_FMT_DEBUG(log, "Persist {} done", region.toString(false)); } @@ -362,8 +363,8 @@ EngineStoreApplyRes KVStore::handleUselessAdminRaftCmd( LOG_FMT_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}", curr_region.toString(false), rows, size_bytes); - if (rows >= REGION_COMPACT_LOG_MIN_ROWS.load(std::memory_order_relaxed) - || size_bytes >= REGION_COMPACT_LOG_MIN_BYTES.load(std::memory_order_relaxed)) + if (rows >= region_compact_log_min_rows.load(std::memory_order_relaxed) + || size_bytes >= region_compact_log_min_bytes.load(std::memory_order_relaxed)) { // if rows or bytes more than threshold, flush cache and perist mem data. return true; @@ -372,7 +373,7 @@ EngineStoreApplyRes KVStore::handleUselessAdminRaftCmd( { // if thhere is little data in mem, wait until time interval reached threshold. // use random period so that lots of regions will not be persisted at same time. - auto compact_log_period = std::rand() % REGION_COMPACT_LOG_PERIOD.load(std::memory_order_relaxed); // NOLINT + auto compact_log_period = std::rand() % region_compact_log_period.load(std::memory_order_relaxed); // NOLINT return !(curr_region.lastCompactLogTime() + Seconds{compact_log_period} > Clock::now()); } } @@ -765,4 +766,9 @@ KVStore::~KVStore() releaseReadIndexWorkers(); } +FileUsageStatistics KVStore::getFileUsageStatistics() const +{ + return region_persister->getFileUsageStatistics(); +} + } // namespace DB diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index 8673cae3ff3..bb45e65d18b 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -16,20 +16,19 @@ #include #include -#include #include - namespace TiDB { struct TableInfo; } namespace DB { +class Context; namespace RegionBench { extern void concurrentBatchInsert(const TiDB::TableInfo &, Int64, Int64, Int64, UInt64, UInt64, Context &); -} +} // namespace RegionBench namespace DM { enum class FileConvertJobType; @@ -40,7 +39,6 @@ namespace tests class RegionKVStoreTest; } -class Context; class IAST; using ASTPtr = std::shared_ptr; using ASTs = std::vector; @@ -71,6 +69,8 @@ using RegionPreDecodeBlockDataPtr = std::unique_ptr; class ReadIndexWorkerManager; using BatchReadIndexRes = std::vector>; class ReadIndexStressTest; +struct FileUsageStatistics; +class RegionPersister; /// TODO: brief design document. class KVStore final : private boost::noncopyable @@ -157,10 +157,7 @@ class KVStore final : private boost::noncopyable ~KVStore(); - FileUsageStatistics getFileUsageStatistics() const - { - return region_persister.getFileUsageStatistics(); - } + FileUsageStatistics getFileUsageStatistics() const; private: friend class MockTiDB; @@ -229,7 +226,7 @@ class KVStore final : private boost::noncopyable private: RegionManager region_manager; - RegionPersister region_persister; + std::unique_ptr region_persister; std::atomic last_gc_time = Timepoint::min(); @@ -242,9 +239,9 @@ class KVStore final : private boost::noncopyable Poco::Logger * log; - std::atomic REGION_COMPACT_LOG_PERIOD; - std::atomic REGION_COMPACT_LOG_MIN_ROWS; - std::atomic REGION_COMPACT_LOG_MIN_BYTES; + std::atomic region_compact_log_period; + std::atomic region_compact_log_min_rows; + std::atomic region_compact_log_min_bytes; mutable std::mutex bg_gc_region_data_mutex; std::list bg_gc_region_data; diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index cc7d1e10a49..8a40ca9b15e 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -24,6 +24,8 @@ #include #include +#include + #define CHECK_PARSE_PB_BUFF_IMPL(n, a, b, c) \ do \ { \ diff --git a/dbms/src/Storages/Transaction/ReadIndexWorker.cpp b/dbms/src/Storages/Transaction/ReadIndexWorker.cpp index 97a8f4b3e0b..3223c815989 100644 --- a/dbms/src/Storages/Transaction/ReadIndexWorker.cpp +++ b/dbms/src/Storages/Transaction/ReadIndexWorker.cpp @@ -20,6 +20,7 @@ #include #include +#include namespace DB { @@ -174,7 +175,7 @@ struct BlockedReadIndexHelper : BlockedReadIndexHelperTrait return waker.waitFor(tm); } - virtual ~BlockedReadIndexHelper() = default; + ~BlockedReadIndexHelper() override = default; private: AsyncWaker & waker; @@ -193,7 +194,7 @@ struct BlockedReadIndexHelperV3 : BlockedReadIndexHelperTrait return notifier.blockedWaitFor(tm); } - virtual ~BlockedReadIndexHelperV3() = default; + ~BlockedReadIndexHelperV3() override = default; private: AsyncWaker::Notifier & notifier; @@ -342,7 +343,7 @@ struct RegionReadIndexNotifier : AsyncNotifier notify->wake(); } - virtual ~RegionReadIndexNotifier() = default; + ~RegionReadIndexNotifier() override = default; RegionReadIndexNotifier( RegionID region_id_, diff --git a/dbms/src/Storages/Transaction/TMTContext.h b/dbms/src/Storages/Transaction/TMTContext.h index bd592dad315..8e26c0da88c 100644 --- a/dbms/src/Storages/Transaction/TMTContext.h +++ b/dbms/src/Storages/Transaction/TMTContext.h @@ -34,15 +34,9 @@ using SchemaSyncerPtr = std::shared_ptr; class BackgroundService; using BackGroundServicePtr = std::unique_ptr; -class MinTSOScheduler; -using MPPTaskSchedulerPtr = std::unique_ptr; - class MPPTaskManager; using MPPTaskManagerPtr = std::shared_ptr; -struct MPPQueryTaskSet; -using MPPQueryTaskSetPtr = std::shared_ptr; - class GCManager; using GCManagerPtr = std::shared_ptr; diff --git a/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp b/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp index 2c3bf243176..dcf727614b1 100644 --- a/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp +++ b/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include namespace DB diff --git a/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h b/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h index 8aab2b3302e..4fdba195acb 100644 --- a/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h +++ b/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include diff --git a/release-centos7-llvm/scripts/build-tiflash-release.sh b/release-centos7-llvm/scripts/build-tiflash-release.sh index bb62e4743f6..42993b51afe 100755 --- a/release-centos7-llvm/scripts/build-tiflash-release.sh +++ b/release-centos7-llvm/scripts/build-tiflash-release.sh @@ -42,6 +42,7 @@ SRCPATH=$( ) NPROC=${NPROC:-$(nproc || grep -c ^processor /proc/cpuinfo)} ENABLE_THINLTO=${ENABLE_THINLTO:-ON} +ENABLE_PCH=${ENABLE_PCH:-ON} INSTALL_DIR="${SRCPATH}/release-centos7-llvm/tiflash" rm -rf ${INSTALL_DIR} && mkdir -p ${INSTALL_DIR} @@ -59,6 +60,7 @@ cmake -S "${SRCPATH}" \ -DRUN_HAVE_STD_REGEX=0 \ -DENABLE_THINLTO=${ENABLE_THINLTO} \ -DTHINLTO_JOBS=${NPROC} \ + -DENABLE_PCH=${ENABLE_PCH} \ -GNinja cmake --build . --target tiflash --parallel ${NPROC} From fdab3f52572abd84e7b00106a20cd2a18554fdec Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Wed, 8 Jun 2022 11:02:30 +0800 Subject: [PATCH 024/104] Test: Mock Input columns for operator tests (#5041) ref pingcap/tiflash#4609 --- .../MockExchangeReceiverInputStream.cpp | 16 ++ .../MockExchangeReceiverInputStream.h | 6 +- dbms/src/Flash/Coprocessor/DAGContext.cpp | 9 + dbms/src/Flash/Coprocessor/DAGContext.h | 9 +- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 49 ++-- dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 6 +- dbms/src/Flash/Coprocessor/MockSourceStream.h | 60 +++++ dbms/src/Flash/tests/gtest_executor.cpp | 230 ++++++++++++++++++ dbms/src/Flash/tests/gtest_interpreter.cpp | 28 +-- dbms/src/TestUtils/ExecutorTestUtils.cpp | 133 ++++++++++ ...rpreterTestUtils.h => ExecutorTestUtils.h} | 38 ++- dbms/src/TestUtils/InterpreterTestUtils.cpp | 73 ------ dbms/src/TestUtils/mockExecutor.cpp | 92 ++++++- dbms/src/TestUtils/mockExecutor.h | 26 +- .../TestUtils/tests/gtest_mock_executors.cpp | 6 +- 15 files changed, 652 insertions(+), 129 deletions(-) create mode 100644 dbms/src/Flash/Coprocessor/MockSourceStream.h create mode 100644 dbms/src/Flash/tests/gtest_executor.cpp create mode 100644 dbms/src/TestUtils/ExecutorTestUtils.cpp rename dbms/src/TestUtils/{InterpreterTestUtils.h => ExecutorTestUtils.h} (61%) delete mode 100644 dbms/src/TestUtils/InterpreterTestUtils.cpp diff --git a/dbms/src/DataStreams/MockExchangeReceiverInputStream.cpp b/dbms/src/DataStreams/MockExchangeReceiverInputStream.cpp index b1de3e23914..3f46bb46cc8 100644 --- a/dbms/src/DataStreams/MockExchangeReceiverInputStream.cpp +++ b/dbms/src/DataStreams/MockExchangeReceiverInputStream.cpp @@ -30,6 +30,22 @@ MockExchangeReceiverInputStream::MockExchangeReceiverInputStream(const tipb::Exc } } +MockExchangeReceiverInputStream::MockExchangeReceiverInputStream(ColumnsWithTypeAndName columns, size_t max_block_size) + : columns(columns) + , output_index(0) + , max_block_size(max_block_size) +{ + rows = 0; + for (const auto & elem : columns) + { + if (elem.column) + { + assert(rows == 0 || rows == elem.column->size()); + rows = elem.column->size(); + } + } +} + ColumnPtr MockExchangeReceiverInputStream::makeColumn(ColumnWithTypeAndName elem) const { auto column = elem.type->createColumn(); diff --git a/dbms/src/DataStreams/MockExchangeReceiverInputStream.h b/dbms/src/DataStreams/MockExchangeReceiverInputStream.h index 24ae80d4f62..8c0a5b85822 100644 --- a/dbms/src/DataStreams/MockExchangeReceiverInputStream.h +++ b/dbms/src/DataStreams/MockExchangeReceiverInputStream.h @@ -26,7 +26,11 @@ class MockExchangeReceiverInputStream : public IProfilingBlockInputStream { public: MockExchangeReceiverInputStream(const tipb::ExchangeReceiver & receiver, size_t max_block_size, size_t rows_); - Block getHeader() const override { return Block(columns); } + MockExchangeReceiverInputStream(ColumnsWithTypeAndName columns, size_t max_block_size); + Block getHeader() const override + { + return Block(columns); + } String getName() const override { return "MockExchangeReceiver"; } ColumnsWithTypeAndName columns; size_t output_index; diff --git a/dbms/src/Flash/Coprocessor/DAGContext.cpp b/dbms/src/Flash/Coprocessor/DAGContext.cpp index 17fb6553eab..1736e0b6cec 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.cpp +++ b/dbms/src/Flash/Coprocessor/DAGContext.cpp @@ -271,4 +271,13 @@ const SingleTableRegions & DAGContext::getTableRegionsInfoByTableID(Int64 table_ return tables_regions_info.getTableRegionInfoByTableID(table_id); } +ColumnsWithTypeAndName DAGContext::columnsForTest(String executor_id) +{ + auto it = columns_for_test_map.find(executor_id); + if (unlikely(it == columns_for_test_map.end())) + { + throw DB::Exception("Don't have columns for mock source executors"); + } + return it->second; +} } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index e3e5efdcbc6..c20eb3a367e 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -298,6 +298,10 @@ class DAGContext } bool isTest() const { return is_test; } + void setColumnsForTest(std::unordered_map & columns_for_test_map_) { columns_for_test_map = columns_for_test_map_; } + ColumnsWithTypeAndName columnsForTest(String executor_id); + + bool columnsForTestEmpty() { return columns_for_test_map.empty(); } void cancelAllExchangeReceiver(); @@ -317,8 +321,8 @@ class DAGContext Clock::time_point read_wait_index_end_timestamp{Clock::duration::zero()}; String table_scan_executor_id; String tidb_host = "Unknown"; - bool collect_execution_summaries; - bool return_executor_id; + bool collect_execution_summaries{}; + bool return_executor_id{}; bool is_mpp_task = false; bool is_root_mpp_task = false; bool is_batch_cop = false; @@ -372,6 +376,7 @@ class DAGContext std::vector subqueries; bool is_test = false; /// switch for test, do not use it in production. + std::unordered_map columns_for_test_map; /// , for multiple sources }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 5fac49faaed..86d6428c92a 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -42,6 +42,7 @@ #include #include #include +#include #include #include #include @@ -159,13 +160,22 @@ AnalysisResult analyzeExpressions( // for tests, we need to mock tableScan blockInputStream as the source stream. void DAGQueryBlockInterpreter::handleMockTableScan(const TiDBTableScan & table_scan, DAGPipeline & pipeline) { - auto names_and_types = genNamesAndTypes(table_scan); - auto columns_with_type_and_name = getColumnWithTypeAndName(names_and_types); - analyzer = std::make_unique(std::move(names_and_types), context); - for (size_t i = 0; i < max_streams; ++i) + if (context.getDAGContext()->columnsForTestEmpty() || context.getDAGContext()->columnsForTest(table_scan.getTableScanExecutorID()).empty()) + { + auto names_and_types = genNamesAndTypes(table_scan); + auto columns_with_type_and_name = getColumnWithTypeAndName(names_and_types); + analyzer = std::make_unique(std::move(names_and_types), context); + for (size_t i = 0; i < max_streams; ++i) + { + auto mock_table_scan_stream = std::make_shared(columns_with_type_and_name, context.getSettingsRef().max_block_size); + pipeline.streams.emplace_back(mock_table_scan_stream); + } + } + else { - auto mock_table_scan_stream = std::make_shared(columns_with_type_and_name, context.getSettingsRef().max_block_size); - pipeline.streams.emplace_back(mock_table_scan_stream); + auto [names_and_types, mock_table_scan_streams] = mockSourceStream(context, max_streams, log, table_scan.getTableScanExecutorID()); + analyzer = std::make_unique(std::move(names_and_types), context); + pipeline.streams.insert(pipeline.streams.end(), mock_table_scan_streams.begin(), mock_table_scan_streams.end()); } } @@ -266,7 +276,8 @@ void DAGQueryBlockInterpreter::handleJoin(const tipb::Join & join, DAGPipeline & stream->setExtraInfo( fmt::format("join build, build_side_root_executor_id = {}", dagContext().getJoinExecuteInfoMap()[query_block.source_name].build_side_root_executor_id)); }); - executeUnion(build_pipeline, max_streams, log, /*ignore_block=*/true, "for join"); + // for test, join executor need the return blocks to output. + executeUnion(build_pipeline, max_streams, log, /*ignore_block=*/!dagContext().isTest(), "for join"); right_query.source = build_pipeline.firstStream(); right_query.join = join_ptr; @@ -491,19 +502,29 @@ void DAGQueryBlockInterpreter::handleExchangeReceiver(DAGPipeline & pipeline) analyzer = std::make_unique(std::move(source_columns), context); } +// for tests, we need to mock ExchangeReceiver blockInputStream as the source stream. void DAGQueryBlockInterpreter::handleMockExchangeReceiver(DAGPipeline & pipeline) { - for (size_t i = 0; i < max_streams; ++i) + if (context.getDAGContext()->columnsForTestEmpty() || context.getDAGContext()->columnsForTest(query_block.source_name).empty()) { - // use max_block_size / 10 to determine the mock block's size - pipeline.streams.push_back(std::make_shared(query_block.source->exchange_receiver(), context.getSettingsRef().max_block_size, context.getSettingsRef().max_block_size / 10)); + for (size_t i = 0; i < max_streams; ++i) + { + // use max_block_size / 10 to determine the mock block's size + pipeline.streams.push_back(std::make_shared(query_block.source->exchange_receiver(), context.getSettingsRef().max_block_size, context.getSettingsRef().max_block_size / 10)); + } + NamesAndTypes source_columns; + for (const auto & col : pipeline.firstStream()->getHeader()) + { + source_columns.emplace_back(col.name, col.type); + } + analyzer = std::make_unique(std::move(source_columns), context); } - NamesAndTypes source_columns; - for (const auto & col : pipeline.firstStream()->getHeader()) + else { - source_columns.emplace_back(col.name, col.type); + auto [names_and_types, mock_exchange_streams] = mockSourceStream(context, max_streams, log, query_block.source_name); + analyzer = std::make_unique(std::move(names_and_types), context); + pipeline.streams.insert(pipeline.streams.end(), mock_exchange_streams.begin(), mock_exchange_streams.end()); } - analyzer = std::make_unique(std::move(source_columns), context); } void DAGQueryBlockInterpreter::handleProjection(DAGPipeline & pipeline, const tipb::Projection & projection) diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index 741aa7b5e26..a67ebf20aa5 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -83,11 +83,13 @@ BlockIO InterpreterDAG::execute() DAGPipeline pipeline; pipeline.streams = streams; /// add union to run in parallel if needed - if (dagContext().isMPPTask()) + if (unlikely(dagContext().isTest())) + executeUnion(pipeline, max_streams, dagContext().log, /*ignore_block=*/false, "for test"); + else if (dagContext().isMPPTask()) /// MPPTask do not need the returned blocks. executeUnion(pipeline, max_streams, dagContext().log, /*ignore_block=*/true, "for mpp"); else - executeUnion(pipeline, max_streams, dagContext().log, false, "for non mpp"); + executeUnion(pipeline, max_streams, dagContext().log, /*ignore_block=*/false, "for non mpp"); if (dagContext().hasSubquery()) { const Settings & settings = context.getSettingsRef(); diff --git a/dbms/src/Flash/Coprocessor/MockSourceStream.h b/dbms/src/Flash/Coprocessor/MockSourceStream.h new file mode 100644 index 00000000000..039cba22e3d --- /dev/null +++ b/dbms/src/Flash/Coprocessor/MockSourceStream.h @@ -0,0 +1,60 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +#pragma once + +#include +#include +#include +#include +#include + +namespace DB +{ +template +std::pair>> mockSourceStream(Context & context, size_t max_streams, DB::LoggerPtr log, String executor_id) +{ + ColumnsWithTypeAndName columns_with_type_and_name; + NamesAndTypes names_and_types; + size_t rows = 0; + std::vector> mock_source_streams; + columns_with_type_and_name = context.getDAGContext()->columnsForTest(executor_id); + for (const auto & col : columns_with_type_and_name) + { + if (rows == 0) + rows = col.column->size(); + RUNTIME_ASSERT(rows == col.column->size(), log, "each column must has same size"); + names_and_types.push_back({col.name, col.type}); + } + size_t row_for_each_stream = rows / max_streams; + size_t rows_left = rows - row_for_each_stream * max_streams; + size_t start = 0; + for (size_t i = 0; i < max_streams; ++i) + { + ColumnsWithTypeAndName columns_for_stream; + size_t row_for_current_stream = row_for_each_stream + (i < rows_left ? 1 : 0); + for (const auto & column_with_type_and_name : columns_with_type_and_name) + { + columns_for_stream.push_back( + ColumnWithTypeAndName( + column_with_type_and_name.column->cut(start, row_for_current_stream), + column_with_type_and_name.type, + column_with_type_and_name.name)); + } + start += row_for_current_stream; + mock_source_streams.emplace_back(std::make_shared(columns_for_stream, context.getSettingsRef().max_block_size)); + } + RUNTIME_ASSERT(start == rows, log, "mock source streams' total size must same as user input"); + return {names_and_types, mock_source_streams}; +} +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/tests/gtest_executor.cpp b/dbms/src/Flash/tests/gtest_executor.cpp new file mode 100644 index 00000000000..64c60f14bb6 --- /dev/null +++ b/dbms/src/Flash/tests/gtest_executor.cpp @@ -0,0 +1,230 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +namespace DB +{ +namespace tests +{ +class ExecutorTestRunner : public DB::tests::ExecutorTest +{ +public: + void initializeContext() override + { + ExecutorTest::initializeContext(); + context.addMockTable({"test_db", "test_table"}, + {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, + {toNullableVec("s1", {"banana", {}, "banana"}), + toNullableVec("s2", {"apple", {}, "banana"})}); + context.addExchangeReceiver("exchange1", + {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}, + {toNullableVec("s1", {"banana", {}, "banana"}), + toNullableVec("s2", {"apple", {}, "banana"})}); + + context.addExchangeReceiver("exchange_r_table", + {{"s1", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, + {toNullableVec("s", {"banana", "banana"}), + toNullableVec("join_c", {"apple", "banana"})}); + + context.addExchangeReceiver("exchange_l_table", + {{"s1", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, + {toNullableVec("s", {"banana", "banana"}), + toNullableVec("join_c", {"apple", "banana"})}); + + context.addMockTable({"test_db", "r_table"}, + {{"s", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, + {toVec("s", {"banana", "banana"}), + toVec("join_c", {"apple", "banana"})}); + + context.addMockTable({"test_db", "r_table_2"}, + {{"s", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, + {toVec("s", {"banana", "banana", "banana"}), + toVec("join_c", {"apple", "apple", "apple"})}); + + context.addMockTable({"test_db", "l_table"}, + {{"s", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}, + {toVec("s", {"banana", "banana"}), + toVec("join_c", {"apple", "banana"})}); + } +}; + +TEST_F(ExecutorTestRunner, Filter) +try +{ + auto request = context + .scan("test_db", "test_table") + .filter(eq(col("s1"), col("s2"))) + .build(context); + { + executeStreams(request, + {toNullableVec({"banana"}), + toNullableVec({"banana"})}); + } + + request = context.receive("exchange1") + .filter(eq(col("s1"), col("s2"))) + .build(context); + { + executeStreams(request, + {toNullableVec({"banana"}), + toNullableVec({"banana"})}); + } +} +CATCH + +TEST_F(ExecutorTestRunner, JoinWithTableScan) +try +{ + auto request = context + .scan("test_db", "l_table") + .join(context.scan("test_db", "r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) + .topN("join_c", false, 2) + .build(context); + { + String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 2\n" + " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" + " table_scan_0 | {<0, String>, <1, String>}\n" + " table_scan_1 | {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 2); + + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 5); + + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}); + } + request = context + .scan("test_db", "l_table") + .join(context.scan("test_db", "r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) + .project({"s", "join_c"}) + .topN("join_c", false, 2) + .build(context); + { + String expected = "topn_4 | order_by: {(<1, String>, desc: false)}, limit: 2\n" + " project_3 | {<0, String>, <1, String>}\n" + " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" + " table_scan_0 | {<0, String>, <1, String>}\n" + " table_scan_1 | {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 2); + } + + request = context + .scan("test_db", "l_table") + .join(context.scan("test_db", "r_table_2"), {col("join_c")}, ASTTableJoin::Kind::Left) + .topN("join_c", false, 4) + .build(context); + { + String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 4\n" + " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" + " table_scan_0 | {<0, String>, <1, String>}\n" + " table_scan_1 | {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + executeStreams(request, + {toNullableVec({"banana", "banana", "banana", "banana"}), + toNullableVec({"apple", "apple", "apple", "banana"}), + toNullableVec({"banana", "banana", "banana", {}}), + toNullableVec({"apple", "apple", "apple", {}})}, + 2); + executeStreams(request, + {toNullableVec({"banana", "banana", "banana", "banana"}), + toNullableVec({"apple", "apple", "apple", "banana"}), + toNullableVec({"banana", "banana", "banana", {}}), + toNullableVec({"apple", "apple", "apple", {}})}, + 3); + } +} +CATCH + +TEST_F(ExecutorTestRunner, JoinWithExchangeReceiver) +try +{ + auto request = context + .receive("exchange_l_table") + .join(context.receive("exchange_r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) + .topN("join_c", false, 2) + .build(context); + { + String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 2\n" + " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" + " exchange_receiver_0 | type:PassThrough, {<0, String>, <1, String>}\n" + " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 2); + + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 5); + + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}); + } +} +CATCH + +TEST_F(ExecutorTestRunner, JoinWithTableScanAndReceiver) +try +{ + auto request = context + .scan("test_db", "l_table") + .join(context.receive("exchange_r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) + .topN("join_c", false, 2) + .build(context); + { + String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 2\n" + " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" + " table_scan_0 | {<0, String>, <1, String>}\n" + " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + executeStreams(request, + {toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})}, + 2); + } +} +CATCH + +} // namespace tests +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp index aed9d9e90f9..a6bb8ff1702 100644 --- a/dbms/src/Flash/tests/gtest_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_interpreter.cpp @@ -12,19 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include #include namespace DB { namespace tests { -class InterpreterExecuteTest : public DB::tests::InterpreterTest +class InterpreterExecuteTest : public DB::tests::ExecutorTest { public: void initializeContext() override { - InterpreterTest::initializeContext(); + ExecutorTest::initializeContext(); context.addMockTable({"test_db", "test_table"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "test_table_1"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); @@ -47,7 +47,7 @@ try .build(context); { String expected = R"( -Union: +Union: SharedQuery x 10: Expression: MergeSorting, limit = 10 @@ -72,7 +72,7 @@ Union: { String expected = R"( -Union: +Union: SharedQuery x 10: Limit, limit = 10 Union: @@ -100,7 +100,7 @@ try .build(context); { String expected = R"( -Union: +Union: Expression x 10: Expression: Expression: @@ -122,7 +122,7 @@ Union: .build(context); { String expected = R"( -Union: +Union: Expression x 10: Expression: Expression: @@ -147,7 +147,7 @@ Union: .build(context); { String expected = R"( -Union: +Union: Expression x 10: Expression: Expression: @@ -181,7 +181,7 @@ Union: .build(context); { String expected = R"( -Union: +Union: SharedQuery x 10: Limit, limit = 10 Union: @@ -244,7 +244,7 @@ CreatingSets HashJoinProbe: Expression: MockTableScan - Union: + Union: Expression x 10: Expression: HashJoinProbe: @@ -260,7 +260,7 @@ CreatingSets .build(context); { String expected = R"( -Union: +Union: Expression x 10: Expression: Expression: @@ -283,7 +283,7 @@ Union: .build(context); { String expected = R"( -Union: +Union: MockExchangeSender x 10 Expression: Expression: @@ -331,7 +331,7 @@ CreatingSets HashJoinProbe: Expression: MockExchangeReceiver - Union: + Union: Expression x 10: Expression: HashJoinProbe: @@ -373,7 +373,7 @@ CreatingSets HashJoinProbe: Expression: MockExchangeReceiver - Union: + Union: MockExchangeSender x 10 Expression: Expression: diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp new file mode 100644 index 00000000000..67a21d12286 --- /dev/null +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -0,0 +1,133 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +namespace DB::tests +{ +DAGContext & ExecutorTest::getDAGContext() +{ + assert(dag_context_ptr != nullptr); + return *dag_context_ptr; +} + +void ExecutorTest::initializeContext() +{ + dag_context_ptr = std::make_unique(1024); + context = MockDAGRequestContext(TiFlashTestEnv::getContext()); + dag_context_ptr->log = Logger::get("executorTest"); +} + +void ExecutorTest::SetUpTestCase() +{ + try + { + DB::registerFunctions(); + DB::registerAggregateFunctions(); + } + catch (DB::Exception &) + { + // Maybe another test has already registered, ignore exception here. + } +} + +void ExecutorTest::initializeClientInfo() +{ + context.context.setCurrentQueryId("test"); + ClientInfo & client_info = context.context.getClientInfo(); + client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; + client_info.interface = ClientInfo::Interface::GRPC; +} + +void ExecutorTest::executeInterpreter(const String & expected_string, const std::shared_ptr & request, size_t concurrency) +{ + DAGContext dag_context(*request, "interpreter_test", concurrency); + context.context.setDAGContext(&dag_context); + // Currently, don't care about regions information in interpreter tests. + DAGQuerySource dag(context.context); + auto res = executeQuery(dag, context.context, false, QueryProcessingStage::Complete); + FmtBuffer fb; + res.in->dumpTree(fb); + ASSERT_EQ(Poco::trim(expected_string), Poco::trim(fb.toString())); +} + +namespace +{ +Block mergeBlocks(Blocks blocks) +{ + if (blocks.empty()) + return {}; + + Block sample_block = blocks.back(); + std::vector actual_cols; + for (const auto & column : sample_block.getColumnsWithTypeAndName()) + { + actual_cols.push_back(column.type->createColumn()); + } + for (const auto & block : blocks) + { + for (size_t i = 0; i < block.columns(); ++i) + { + for (size_t j = 0; j < block.rows(); ++j) + { + actual_cols[i]->insert((*(block.getColumnsWithTypeAndName())[i].column)[j]); + } + } + } + + ColumnsWithTypeAndName actual_columns; + for (size_t i = 0; i < actual_cols.size(); ++i) + actual_columns.push_back({std::move(actual_cols[i]), sample_block.getColumnsWithTypeAndName()[i].type, sample_block.getColumnsWithTypeAndName()[i].name, sample_block.getColumnsWithTypeAndName()[i].column_id}); + return Block(actual_columns); +} + +void readBlock(BlockInputStreamPtr stream, const ColumnsWithTypeAndName & expect_columns) +{ + Blocks actual_blocks; + Block except_block(expect_columns); + stream->readPrefix(); + while (auto block = stream->read()) + { + actual_blocks.push_back(block); + } + stream->readSuffix(); + Block actual_block = mergeBlocks(actual_blocks); + ASSERT_BLOCK_EQ(except_block, actual_block); +} +} // namespace + +void ExecutorTest::executeStreams(const std::shared_ptr & request, std::unordered_map & source_columns_map, const ColumnsWithTypeAndName & expect_columns, size_t concurrency) +{ + DAGContext dag_context(*request, "executor_test", concurrency); + dag_context.setColumnsForTest(source_columns_map); + context.context.setDAGContext(&dag_context); + // Currently, don't care about regions information in tests. + DAGQuerySource dag(context.context); + readBlock(executeQuery(dag, context.context, false, QueryProcessingStage::Complete).in, expect_columns); +} + +void ExecutorTest::executeStreams(const std::shared_ptr & request, const ColumnsWithTypeAndName & expect_columns, size_t concurrency) +{ + executeStreams(request, context.executorIdColumnsMap(), expect_columns, concurrency); +} + +void ExecutorTest::dagRequestEqual(const String & expected_string, const std::shared_ptr & actual) +{ + ASSERT_EQ(Poco::trim(expected_string), Poco::trim(ExecutorSerializer().serialize(actual.get()))); +} + +} // namespace DB::tests diff --git a/dbms/src/TestUtils/InterpreterTestUtils.h b/dbms/src/TestUtils/ExecutorTestUtils.h similarity index 61% rename from dbms/src/TestUtils/InterpreterTestUtils.h rename to dbms/src/TestUtils/ExecutorTestUtils.h index 28d44d3a5f2..977b46abbd2 100644 --- a/dbms/src/TestUtils/InterpreterTestUtils.h +++ b/dbms/src/TestUtils/ExecutorTestUtils.h @@ -27,7 +27,7 @@ namespace DB::tests { void executeInterpreter(const std::shared_ptr & request, Context & context); -class InterpreterTest : public ::testing::Test +class ExecutorTest : public ::testing::Test { protected: void SetUp() override @@ -37,7 +37,7 @@ class InterpreterTest : public ::testing::Test } public: - InterpreterTest() + ExecutorTest() : context(TiFlashTestEnv::getContext()) {} static void SetUpTestCase(); @@ -52,6 +52,40 @@ class InterpreterTest : public ::testing::Test void executeInterpreter(const String & expected_string, const std::shared_ptr & request, size_t concurrency); + void executeStreams( + const std::shared_ptr & request, + std::unordered_map & source_columns_map, + const ColumnsWithTypeAndName & expect_columns, + size_t concurrency = 1); + void executeStreams( + const std::shared_ptr & request, + const ColumnsWithTypeAndName & expect_columns, + size_t concurrency = 1); + + template + ColumnWithTypeAndName toNullableVec(const std::vector::FieldType>> & v) + { + return createColumn>(v); + } + + template + ColumnWithTypeAndName toVec(const std::vector::FieldType> & v) + { + return createColumn(v); + } + + template + ColumnWithTypeAndName toNullableVec(String name, const std::vector::FieldType>> & v) + { + return createColumn>(v, name); + } + + template + ColumnWithTypeAndName toVec(String name, const std::vector::FieldType> & v) + { + return createColumn(v, name); + } + protected: MockDAGRequestContext context; std::unique_ptr dag_context_ptr; diff --git a/dbms/src/TestUtils/InterpreterTestUtils.cpp b/dbms/src/TestUtils/InterpreterTestUtils.cpp deleted file mode 100644 index 2cc096d4095..00000000000 --- a/dbms/src/TestUtils/InterpreterTestUtils.cpp +++ /dev/null @@ -1,73 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include -#include -#include -#include -#include -namespace DB::tests -{ -DAGContext & InterpreterTest::getDAGContext() -{ - assert(dag_context_ptr != nullptr); - return *dag_context_ptr; -} - -void InterpreterTest::initializeContext() -{ - dag_context_ptr = std::make_unique(1024); - context = MockDAGRequestContext(TiFlashTestEnv::getContext()); - dag_context_ptr->log = Logger::get("interpreterTest"); -} - -void InterpreterTest::SetUpTestCase() -{ - try - { - DB::registerFunctions(); - DB::registerAggregateFunctions(); - } - catch (DB::Exception &) - { - // Maybe another test has already registered, ignore exception here. - } -} - -void InterpreterTest::initializeClientInfo() -{ - context.context.setCurrentQueryId("test"); - ClientInfo & client_info = context.context.getClientInfo(); - client_info.query_kind = ClientInfo::QueryKind::INITIAL_QUERY; - client_info.interface = ClientInfo::Interface::GRPC; -} - -void InterpreterTest::executeInterpreter(const String & expected_string, const std::shared_ptr & request, size_t concurrency) -{ - DAGContext dag_context(*request, "interpreter_test", concurrency); - context.context.setDAGContext(&dag_context); - // Currently, don't care about regions information in interpreter tests. - DAGQuerySource dag(context.context); - auto res = executeQuery(dag, context.context, false, QueryProcessingStage::Complete); - FmtBuffer fb; - res.in->dumpTree(fb); - ASSERT_EQ(Poco::trim(expected_string), Poco::trim(fb.toString())); -} - -void InterpreterTest::dagRequestEqual(const String & expected_string, const std::shared_ptr & actual) -{ - ASSERT_EQ(Poco::trim(expected_string), Poco::trim(ExecutorSerializer().serialize(actual.get()))); -} - -} // namespace DB::tests diff --git a/dbms/src/TestUtils/mockExecutor.cpp b/dbms/src/TestUtils/mockExecutor.cpp index 3313aae6a93..af939002cff 100644 --- a/dbms/src/TestUtils/mockExecutor.cpp +++ b/dbms/src/TestUtils/mockExecutor.cpp @@ -23,6 +23,8 @@ #include #include +#include + namespace DB::tests { ASTPtr buildColumn(const String & column_name) @@ -274,52 +276,116 @@ DAGRequestBuilder & DAGRequestBuilder::buildAggregation(ASTPtr agg_funcs, ASTPtr return *this; } -void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfoList & columns) +void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfoList & columnInfos) { - std::vector v_column_info(columns.size()); + std::vector v_column_info(columnInfos.size()); size_t i = 0; - for (const auto & info : columns) + for (const auto & info : columnInfos) { v_column_info[i++] = std::move(info); } mock_tables[name.first + "." + name.second] = v_column_info; } -void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfos & columns) +void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfos & columnInfos) { - mock_tables[db + "." + table] = columns; + mock_tables[db + "." + table] = columnInfos; } -void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfos & columns) +void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfos & columnInfos) { - mock_tables[name.first + "." + name.second] = columns; + mock_tables[name.first + "." + name.second] = columnInfos; } -void MockDAGRequestContext::addExchangeRelationSchema(String name, const MockColumnInfos & columns) +void MockDAGRequestContext::addExchangeRelationSchema(String name, const MockColumnInfos & columnInfos) { - exchange_schemas[name] = columns; + exchange_schemas[name] = columnInfos; } -void MockDAGRequestContext::addExchangeRelationSchema(String name, const MockColumnInfoList & columns) +void MockDAGRequestContext::addExchangeRelationSchema(String name, const MockColumnInfoList & columnInfos) { - std::vector v_column_info(columns.size()); + std::vector v_column_info(columnInfos.size()); size_t i = 0; - for (const auto & info : columns) + for (const auto & info : columnInfos) { v_column_info[i++] = std::move(info); } exchange_schemas[name] = v_column_info; } +void MockDAGRequestContext::addMockTableColumnData(const String & db, const String & table, ColumnsWithTypeAndName columns) +{ + mock_table_columns[db + "." + table] = columns; +} + +void MockDAGRequestContext::addMockTableColumnData(const MockTableName & name, ColumnsWithTypeAndName columns) +{ + mock_table_columns[name.first + "." + name.second] = columns; +} + +void MockDAGRequestContext::addExchangeReceiverColumnData(const String & name, ColumnsWithTypeAndName columns) +{ + mock_exchange_columns[name] = columns; +} + +void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfoList & columnInfos, ColumnsWithTypeAndName columns) +{ + addMockTable(db, table, columnInfos); + addMockTableColumnData(db, table, columns); +} + +void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfos & columnInfos, ColumnsWithTypeAndName columns) +{ + addMockTable(db, table, columnInfos); + addMockTableColumnData(db, table, columns); +} + +void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfoList & columnInfos, ColumnsWithTypeAndName columns) +{ + addMockTable(name, columnInfos); + addMockTableColumnData(name, columns); +} + +void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfos & columnInfos, ColumnsWithTypeAndName columns) +{ + addMockTable(name, columnInfos); + addMockTableColumnData(name, columns); +} + +void MockDAGRequestContext::addExchangeReceiver(const String & name, MockColumnInfos columnInfos, ColumnsWithTypeAndName columns) +{ + addExchangeRelationSchema(name, columnInfos); + addExchangeReceiverColumnData(name, columns); +} + +void MockDAGRequestContext::addExchangeReceiver(const String & name, MockColumnInfoList columnInfos, ColumnsWithTypeAndName columns) +{ + addExchangeRelationSchema(name, columnInfos); + addExchangeReceiverColumnData(name, columns); +} + DAGRequestBuilder MockDAGRequestContext::scan(String db_name, String table_name) { - return DAGRequestBuilder(index).mockTable({db_name, table_name}, mock_tables[db_name + "." + table_name]); + auto builder = DAGRequestBuilder(index).mockTable({db_name, table_name}, mock_tables[db_name + "." + table_name]); + // If don't have related columns, user must pass input columns as argument of executeStreams in order to run Executors Tests. + // If user don't want to test executors, it will be safe to run Interpreter Tests. + if (mock_table_columns.find(db_name + "." + table_name) != mock_table_columns.end()) + { + executor_id_columns_map[builder.getRoot()->name] = mock_table_columns[db_name + "." + table_name]; + } + return builder; } DAGRequestBuilder MockDAGRequestContext::receive(String exchange_name) { auto builder = DAGRequestBuilder(index).exchangeReceiver(exchange_schemas[exchange_name]); receiver_source_task_ids_map[builder.getRoot()->name] = {}; + // If don't have related columns, user must pass input columns as argument of executeStreams in order to run Executors Tests. + // If user don't want to test executors, it will be safe to run Interpreter Tests. + if (mock_exchange_columns.find(exchange_name) != mock_exchange_columns.end()) + { + executor_id_columns_map[builder.getRoot()->name] = mock_exchange_columns[exchange_name]; + } return builder; } } // namespace DB::tests \ No newline at end of file diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index 2f6d3542ebb..88d98158b74 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include #include @@ -122,11 +123,23 @@ class MockDAGRequestContext return DAGRequestBuilder(index); } - void addMockTable(const MockTableName & name, const MockColumnInfoList & columns); - void addMockTable(const String & db, const String & table, const MockColumnInfos & columns); - void addMockTable(const MockTableName & name, const MockColumnInfos & columns); - void addExchangeRelationSchema(String name, const MockColumnInfos & columns); - void addExchangeRelationSchema(String name, const MockColumnInfoList & columns); + void addMockTable(const MockTableName & name, const MockColumnInfoList & columnInfos); + void addMockTable(const String & db, const String & table, const MockColumnInfos & columnInfos); + void addMockTable(const MockTableName & name, const MockColumnInfos & columnInfos); + void addExchangeRelationSchema(String name, const MockColumnInfos & columnInfos); + void addExchangeRelationSchema(String name, const MockColumnInfoList & columnInfos); + void addMockTableColumnData(const String & db, const String & table, ColumnsWithTypeAndName columns); + void addMockTable(const String & db, const String & table, const MockColumnInfoList & columnInfos, ColumnsWithTypeAndName columns); + void addMockTable(const String & db, const String & table, const MockColumnInfos & columnInfos, ColumnsWithTypeAndName columns); + void addMockTable(const MockTableName & name, const MockColumnInfoList & columnInfos, ColumnsWithTypeAndName columns); + void addMockTable(const MockTableName & name, const MockColumnInfos & columnInfos, ColumnsWithTypeAndName columns); + void addMockTableColumnData(const MockTableName & name, ColumnsWithTypeAndName columns); + void addExchangeReceiverColumnData(const String & name, ColumnsWithTypeAndName columns); + void addExchangeReceiver(const String & name, MockColumnInfos columnInfos, ColumnsWithTypeAndName columns); + void addExchangeReceiver(const String & name, MockColumnInfoList columnInfos, ColumnsWithTypeAndName columns); + + std::unordered_map & executorIdColumnsMap() { return executor_id_columns_map; } + DAGRequestBuilder scan(String db_name, String table_name); DAGRequestBuilder receive(String exchange_name); @@ -134,6 +147,9 @@ class MockDAGRequestContext size_t index; std::unordered_map mock_tables; std::unordered_map exchange_schemas; + std::unordered_map mock_table_columns; + std::unordered_map mock_exchange_columns; + std::unordered_map executor_id_columns_map; /// public: // Currently don't support task_id, so the following to structure is useless, diff --git a/dbms/src/TestUtils/tests/gtest_mock_executors.cpp b/dbms/src/TestUtils/tests/gtest_mock_executors.cpp index 6dbf791669f..214148fe47f 100644 --- a/dbms/src/TestUtils/tests/gtest_mock_executors.cpp +++ b/dbms/src/TestUtils/tests/gtest_mock_executors.cpp @@ -12,19 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include #include namespace DB { namespace tests { -class MockDAGRequestTest : public DB::tests::InterpreterTest +class MockDAGRequestTest : public DB::tests::ExecutorTest { public: void initializeContext() override { - InterpreterTest::initializeContext(); + ExecutorTest::initializeContext(); context.addMockTable({"test_db", "test_table"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "test_table_1"}, {{"s1", TiDB::TP::TypeLong}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); From 5b61ae70550624d3bf0b5ca6bac89013ed5a6a4b Mon Sep 17 00:00:00 2001 From: bestwoody <89765764+bestwoody@users.noreply.github.com> Date: Wed, 8 Jun 2022 14:36:30 +0800 Subject: [PATCH 025/104] Improve the performance of partition table in extreme case (#4988) close pingcap/tiflash#4474 --- dbms/src/DataStreams/MultiplexInputStream.h | 246 ++++++++++++++++++ .../Coprocessor/DAGStorageInterpreter.cpp | 23 +- 2 files changed, 264 insertions(+), 5 deletions(-) create mode 100644 dbms/src/DataStreams/MultiplexInputStream.h diff --git a/dbms/src/DataStreams/MultiplexInputStream.h b/dbms/src/DataStreams/MultiplexInputStream.h new file mode 100644 index 00000000000..4fa33262e66 --- /dev/null +++ b/dbms/src/DataStreams/MultiplexInputStream.h @@ -0,0 +1,246 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include + +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} // namespace ErrorCodes + +class MultiPartitionStreamPool +{ +public: + MultiPartitionStreamPool() = default; + + void addPartitionStreams(const BlockInputStreams & cur_streams) + { + if (cur_streams.empty()) + return; + std::unique_lock lk(mu); + streams_queue_by_partition.push_back( + std::make_shared>>()); + for (const auto & stream : cur_streams) + streams_queue_by_partition.back()->push(stream); + added_streams.insert(added_streams.end(), cur_streams.begin(), cur_streams.end()); + } + + std::shared_ptr pickOne() + { + std::unique_lock lk(mu); + if (streams_queue_by_partition.empty()) + return nullptr; + if (streams_queue_id >= static_cast(streams_queue_by_partition.size())) + streams_queue_id = 0; + + auto & q = *streams_queue_by_partition[streams_queue_id]; + std::shared_ptr ret = nullptr; + assert(!q.empty()); + ret = q.front(); + q.pop(); + if (q.empty()) + streams_queue_id = removeQueue(streams_queue_id); + else + streams_queue_id = nextQueueId(streams_queue_id); + return ret; + } + + int exportAddedStreams(BlockInputStreams & ret_streams) + { + std::unique_lock lk(mu); + for (auto & stream : added_streams) + ret_streams.push_back(stream); + return added_streams.size(); + } + + int addedStreamsCnt() + { + std::unique_lock lk(mu); + return added_streams.size(); + } + +private: + int removeQueue(int queue_id) + { + streams_queue_by_partition[queue_id] = nullptr; + if (queue_id != static_cast(streams_queue_by_partition.size()) - 1) + { + swap(streams_queue_by_partition[queue_id], streams_queue_by_partition.back()); + streams_queue_by_partition.pop_back(); + return queue_id; + } + else + { + streams_queue_by_partition.pop_back(); + return 0; + } + } + + int nextQueueId(int queue_id) const + { + if (queue_id + 1 < static_cast(streams_queue_by_partition.size())) + return queue_id + 1; + else + return 0; + } + + static void swap(std::shared_ptr>> & a, + std::shared_ptr>> & b) + { + a.swap(b); + } + + std::vector< + std::shared_ptr>>> + streams_queue_by_partition; + std::vector> added_streams; + int streams_queue_id = 0; + std::mutex mu; +}; + +class MultiplexInputStream final : public IProfilingBlockInputStream +{ +private: + static constexpr auto NAME = "Multiplex"; + +public: + MultiplexInputStream( + std::shared_ptr & shared_pool, + const String & req_id) + : log(Logger::get(NAME, req_id)) + , shared_pool(shared_pool) + { + shared_pool->exportAddedStreams(children); + size_t num_children = children.size(); + if (num_children > 1) + { + Block header = children.at(0)->getHeader(); + for (size_t i = 1; i < num_children; ++i) + assertBlocksHaveEqualStructure( + children[i]->getHeader(), + header, + "MULTIPLEX"); + } + } + + String getName() const override { return NAME; } + + ~MultiplexInputStream() override + { + try + { + if (!all_read) + cancel(false); + } + catch (...) + { + tryLogCurrentException(log, __PRETTY_FUNCTION__); + } + } + + /** Different from the default implementation by trying to stop all sources, + * skipping failed by execution. + */ + void cancel(bool kill) override + { + if (kill) + is_killed = true; + + bool old_val = false; + if (!is_cancelled.compare_exchange_strong( + old_val, + true, + std::memory_order_seq_cst, + std::memory_order_relaxed)) + return; + + if (cur_stream) + { + if (IProfilingBlockInputStream * child = dynamic_cast(&*cur_stream)) + { + child->cancel(kill); + } + } + } + + Block getHeader() const override { return children.at(0)->getHeader(); } + +protected: + /// Do nothing, to make the preparation when underlying InputStream is picked from the pool + void readPrefix() override + { + } + + /** The following options are possible: + * 1. `readImpl` function is called until it returns an empty block. + * Then `readSuffix` function is called and then destructor. + * 2. `readImpl` function is called. At some point, `cancel` function is called perhaps from another thread. + * Then `readSuffix` function is called and then destructor. + * 3. At any time, the object can be destroyed (destructor called). + */ + + Block readImpl() override + { + if (all_read) + return {}; + + Block ret; + while (!cur_stream || !(ret = cur_stream->read())) + { + if (cur_stream) + cur_stream->readSuffix(); // release old inputstream + cur_stream = shared_pool->pickOne(); + if (!cur_stream) + { // shared_pool is empty + all_read = true; + return {}; + } + cur_stream->readPrefix(); + } + return ret; + } + + /// Called either after everything is read, or after cancel. + void readSuffix() override + { + if (!all_read && !is_cancelled) + throw Exception("readSuffix called before all data is read", ErrorCodes::LOGICAL_ERROR); + + if (cur_stream) + { + cur_stream->readSuffix(); + cur_stream = nullptr; + } + } + +private: + LoggerPtr log; + + std::shared_ptr shared_pool; + std::shared_ptr cur_stream; + + bool all_read = false; +}; + +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index df7e504d2c4..14cddd94730 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -634,6 +635,9 @@ void DAGStorageInterpreter::buildLocalStreams(DAGPipeline & pipeline, size_t max if (total_local_region_num == 0) return; const auto table_query_infos = generateSelectQueryInfos(); + bool has_multiple_partitions = table_query_infos.size() > 1; + // MultiPartitionStreamPool will be disabled in no partition mode or single-partition case + std::shared_ptr stream_pool = has_multiple_partitions ? std::make_shared() : nullptr; for (const auto & table_query_info : table_query_infos) { DAGPipeline current_pipeline; @@ -642,9 +646,6 @@ void DAGStorageInterpreter::buildLocalStreams(DAGPipeline & pipeline, size_t max size_t region_num = query_info.mvcc_query_info->regions_query_info.size(); if (region_num == 0) continue; - /// calculate weighted max_streams for each partition, note at least 1 stream is needed for each partition - size_t current_max_streams = table_query_infos.size() == 1 ? max_streams : (max_streams * region_num + total_local_region_num - 1) / total_local_region_num; - QueryProcessingStage::Enum from_stage = QueryProcessingStage::FetchColumns; assert(storages_with_structure_lock.find(table_id) != storages_with_structure_lock.end()); auto & storage = storages_with_structure_lock[table_id].storage; @@ -654,7 +655,7 @@ void DAGStorageInterpreter::buildLocalStreams(DAGPipeline & pipeline, size_t max { try { - current_pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, current_max_streams); + current_pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); // After getting streams from storage, we need to validate whether Regions have changed or not after learner read. // (by calling `validateQueryInfo`). In case the key ranges of Regions have changed (Region merge/split), those `streams` @@ -778,7 +779,19 @@ void DAGStorageInterpreter::buildLocalStreams(DAGPipeline & pipeline, size_t max throw; } } - pipeline.streams.insert(pipeline.streams.end(), current_pipeline.streams.begin(), current_pipeline.streams.end()); + if (has_multiple_partitions) + stream_pool->addPartitionStreams(current_pipeline.streams); + else + pipeline.streams.insert(pipeline.streams.end(), current_pipeline.streams.begin(), current_pipeline.streams.end()); + } + if (has_multiple_partitions) + { + String req_info = dag_context.isMPPTask() ? dag_context.getMPPTaskId().toString() : ""; + int exposed_streams_cnt = std::min(static_cast(max_streams), stream_pool->addedStreamsCnt()); + for (int i = 0; i < exposed_streams_cnt; ++i) + { + pipeline.streams.push_back(std::make_shared(stream_pool, req_info)); + } } } From 167d39ff16696ec5d62f3a738b2f60c8c1db8563 Mon Sep 17 00:00:00 2001 From: jiaqizho Date: Thu, 9 Jun 2022 17:34:31 +0800 Subject: [PATCH 026/104] Fix some fail cases when enable TASN (#5086) close pingcap/tiflash#5085 --- .../Management/tests/gtest_manual_compact.cpp | 8 +- dbms/src/Storages/DeltaMerge/StoragePool.cpp | 4 +- .../DeltaMerge/tests/MultiSegmentTestUtil.h | 3 + .../tests/gtest_dm_delta_merge_store.cpp | 3 - .../Page/V3/tests/gtest_blob_store.cpp | 3 + .../Page/V3/tests/gtest_page_directory.cpp | 102 +++++++++--------- .../Page/V3/tests/gtest_wal_store.cpp | 48 ++++----- dbms/src/TestUtils/ColumnsToTiPBExpr.cpp | 2 + 8 files changed, 87 insertions(+), 86 deletions(-) diff --git a/dbms/src/Flash/Management/tests/gtest_manual_compact.cpp b/dbms/src/Flash/Management/tests/gtest_manual_compact.cpp index df6c881c306..1e9da93ffe3 100644 --- a/dbms/src/Flash/Management/tests/gtest_manual_compact.cpp +++ b/dbms/src/Flash/Management/tests/gtest_manual_compact.cpp @@ -14,7 +14,6 @@ #include #include -#include #include #include #include @@ -48,7 +47,6 @@ class BasicManualCompactTest BasicManualCompactTest() { - log = &Poco::Logger::get(DB::base::TiFlashStorageTestBasic::getCurrentFullTestName()); pk_type = GetParam(); } @@ -63,7 +61,7 @@ class BasicManualCompactTest setupStorage(); // In tests let's only compact one segment. - db_context->setSetting("manual_compact_more_until_ms", UInt64(0)); + db_context->setSetting("manual_compact_more_until_ms", Field(UInt64(0))); // Split into 4 segments, and prepare some delta data for first 3 segments. helper = std::make_unique(*db_context); @@ -116,8 +114,6 @@ class BasicManualCompactTest std::unique_ptr manager; DM::tests::DMTestEnv::PkType pk_type; - - [[maybe_unused]] Poco::Logger * log; }; @@ -315,7 +311,7 @@ CATCH TEST_P(BasicManualCompactTest, CompactMultiple) try { - db_context->setSetting("manual_compact_more_until_ms", UInt64(60 * 1000)); // Hope it's long enough! + db_context->setSetting("manual_compact_more_until_ms", Field(UInt64(60 * 1000))); // Hope it's long enough! auto request = ::kvrpcpb::CompactRequest(); request.set_physical_table_id(TABLE_ID); diff --git a/dbms/src/Storages/DeltaMerge/StoragePool.cpp b/dbms/src/Storages/DeltaMerge/StoragePool.cpp index 752898f9c75..2791a74e9e3 100644 --- a/dbms/src/Storages/DeltaMerge/StoragePool.cpp +++ b/dbms/src/Storages/DeltaMerge/StoragePool.cpp @@ -624,8 +624,8 @@ PageId StoragePool::newDataPageIdForDTFile(StableDiskDelegator & delegator, cons auto existed_path = delegator.getDTFilePath(dtfile_id, /*throw_on_not_exist=*/false); fiu_do_on(FailPoints::force_set_dtfile_exist_when_acquire_id, { - static size_t fail_point_called = 0; - if (existed_path.empty() && fail_point_called % 10 == 0) + static std::atomic fail_point_called(0); + if (existed_path.empty() && fail_point_called.load() % 10 == 0) { existed_path = ""; } diff --git a/dbms/src/Storages/DeltaMerge/tests/MultiSegmentTestUtil.h b/dbms/src/Storages/DeltaMerge/tests/MultiSegmentTestUtil.h index 7c5b0b2416d..787a521ded3 100644 --- a/dbms/src/Storages/DeltaMerge/tests/MultiSegmentTestUtil.h +++ b/dbms/src/Storages/DeltaMerge/tests/MultiSegmentTestUtil.h @@ -88,6 +88,7 @@ class MultiSegmentTestUtil : private boost::noncopyable // Check there is only one segment ASSERT_EQ(store->segments.size(), 1); const auto & [_key, seg] = *store->segments.begin(); + (void)_key; ASSERT_EQ(seg->getDelta()->getRows(), n_avg_rows_per_segment * 4); ASSERT_EQ(seg->getStable()->getRows(), 0); @@ -108,6 +109,7 @@ class MultiSegmentTestUtil : private boost::noncopyable auto segment_idx = 0; for (auto & [_key, seg] : store->segments) { + (void)_key; LOG_FMT_INFO(log, "Segment #{}: Range = {}", segment_idx, seg->getRowKeyRange().toDebugString()); ASSERT_EQ(seg->getDelta()->getRows(), 0); ASSERT_GT(seg->getStable()->getRows(), 0); // We don't check the exact rows of each segment. @@ -147,6 +149,7 @@ class MultiSegmentTestUtil : private boost::noncopyable auto segment_idx = 0; for (auto & [_key, seg] : store->segments) { + (void)_key; ASSERT_EQ(seg->getDelta()->getRows(), expected_delta_rows[segment_idx]) << "Assert failed for segment #" << segment_idx; ASSERT_EQ(seg->getStable()->getRows(), expected_stable_rows[segment_idx]) << "Assert failed for segment #" << segment_idx; segment_idx++; diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp index e934f7a2049..d46e1b7aa36 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp @@ -3564,7 +3564,6 @@ class DeltaMergeStoreMergeDeltaBySegmentTest public: DeltaMergeStoreMergeDeltaBySegmentTest() { - log = &Poco::Logger::get(DB::base::TiFlashStorageTestBasic::getCurrentFullTestName()); std::tie(ps_ver, pk_type) = GetParam(); } @@ -3607,8 +3606,6 @@ class DeltaMergeStoreMergeDeltaBySegmentTest UInt64 ps_ver; DMTestEnv::PkType pk_type; - - [[maybe_unused]] Poco::Logger * log; }; INSTANTIATE_TEST_CASE_P( diff --git a/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp b/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp index 048140ed04f..94bb69045ba 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp @@ -82,6 +82,7 @@ try stats.restoreByEntry(PageEntryV3{ .file_id = file_id1, .size = 128, + .padded_size = 0, .tag = 0, .offset = 1024, .checksum = 0x4567, @@ -89,6 +90,7 @@ try stats.restoreByEntry(PageEntryV3{ .file_id = file_id1, .size = 512, + .padded_size = 0, .tag = 0, .offset = 2048, .checksum = 0x4567, @@ -96,6 +98,7 @@ try stats.restoreByEntry(PageEntryV3{ .file_id = file_id2, .size = 512, + .padded_size = 0, .tag = 0, .offset = 2048, .checksum = 0x4567, diff --git a/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp b/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp index 6e2b0efa1ea..83e07f75d37 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp @@ -75,7 +75,7 @@ try auto snap0 = dir->createSnapshot(); EXPECT_ENTRY_NOT_EXIST(dir, 1, snap0); - PageEntryV3 entry1{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry1{.file_id = 1, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(1, entry1); @@ -85,7 +85,7 @@ try auto snap1 = dir->createSnapshot(); EXPECT_ENTRY_EQ(entry1, dir, 1, snap1); - PageEntryV3 entry2{.file_id = 2, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry2{.file_id = 2, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(2, entry2); @@ -102,7 +102,7 @@ try EXPECT_ENTRIES_EQ(expected_entries, dir, ids, snap2); } - PageEntryV3 entry2_v2{.file_id = 2 + 102, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry2_v2{.file_id = 2 + 102, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.del(2); @@ -123,7 +123,7 @@ try auto snap0 = dir->createSnapshot(); EXPECT_ENTRY_NOT_EXIST(dir, page_id, snap0); - PageEntryV3 entry1{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry1{.file_id = 1, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(page_id, entry1); @@ -133,7 +133,7 @@ try auto snap1 = dir->createSnapshot(); EXPECT_ENTRY_EQ(entry1, dir, page_id, snap1); - PageEntryV3 entry2{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x1234, .checksum = 0x4567}; + PageEntryV3 entry2{.file_id = 1, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x1234, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(page_id, entry2); @@ -151,7 +151,7 @@ try // Put identical page within one `edit` page_id++; - PageEntryV3 entry3{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x12345, .checksum = 0x4567}; + PageEntryV3 entry3{.file_id = 1, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x12345, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(page_id, entry1); @@ -172,8 +172,8 @@ CATCH TEST_F(PageDirectoryTest, ApplyPutDelRead) try { - PageEntryV3 entry1{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry2{.file_id = 2, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry1{.file_id = 1, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry2{.file_id = 2, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(1, entry1); @@ -185,8 +185,8 @@ try EXPECT_ENTRY_EQ(entry1, dir, 1, snap1); EXPECT_ENTRY_EQ(entry2, dir, 2, snap1); - PageEntryV3 entry3{.file_id = 3, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry4{.file_id = 4, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry3{.file_id = 3, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry4{.file_id = 4, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.del(2); @@ -217,8 +217,8 @@ CATCH TEST_F(PageDirectoryTest, ApplyUpdateOnRefEntries) try { - PageEntryV3 entry1{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry2{.file_id = 2, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry1{.file_id = 1, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry2{.file_id = 2, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(1, entry1); @@ -236,14 +236,14 @@ try EXPECT_ENTRY_EQ(entry2, dir, 3, snap1); // Update on ref page is not allowed - PageEntryV3 entry_updated{.file_id = 999, .size = 16, .tag = 0, .offset = 0x123, .checksum = 0x123}; + PageEntryV3 entry_updated{.file_id = 999, .size = 16, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x123}; { PageEntriesEdit edit; edit.put(3, entry_updated); ASSERT_ANY_THROW(dir->apply(std::move(edit))); } - PageEntryV3 entry_updated2{.file_id = 777, .size = 16, .tag = 0, .offset = 0x123, .checksum = 0x123}; + PageEntryV3 entry_updated2{.file_id = 777, .size = 16, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x123}; { PageEntriesEdit edit; edit.put(2, entry_updated2); @@ -255,8 +255,8 @@ CATCH TEST_F(PageDirectoryTest, ApplyDeleteOnRefEntries) try { - PageEntryV3 entry1{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry2{.file_id = 2, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry1{.file_id = 1, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry2{.file_id = 2, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(1, entry1); @@ -305,8 +305,8 @@ CATCH TEST_F(PageDirectoryTest, ApplyRefOnRefEntries) try { - PageEntryV3 entry1{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry2{.file_id = 2, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry1{.file_id = 1, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry2{.file_id = 2, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(1, entry1); @@ -343,8 +343,8 @@ CATCH TEST_F(PageDirectoryTest, ApplyDuplicatedRefEntries) try { - PageEntryV3 entry1{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry2{.file_id = 2, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry1{.file_id = 1, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry2{.file_id = 2, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(1, entry1); @@ -410,8 +410,8 @@ CATCH TEST_F(PageDirectoryTest, ApplyCollapseDuplicatedRefEntries) try { - PageEntryV3 entry1{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry2{.file_id = 2, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry1{.file_id = 1, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry2{.file_id = 2, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(1, entry1); @@ -447,9 +447,9 @@ CATCH TEST_F(PageDirectoryTest, ApplyRefToNotExistEntry) try { - PageEntryV3 entry1{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry2{.file_id = 2, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry3{.file_id = 3, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry1{.file_id = 1, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry2{.file_id = 2, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry3{.file_id = 3, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(1, entry1); @@ -628,12 +628,12 @@ try } CATCH -#define INSERT_BLOBID_ENTRY(BLOBID, VERSION) \ - PageEntryV3 entry_v##VERSION{.file_id = (BLOBID), .size = (VERSION), .tag = 0, .offset = 0x123, .checksum = 0x4567}; \ +#define INSERT_BLOBID_ENTRY(BLOBID, VERSION) \ + PageEntryV3 entry_v##VERSION{.file_id = (BLOBID), .size = (VERSION), .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; \ entries.createNewEntry(PageVersion(VERSION), entry_v##VERSION); #define INSERT_ENTRY(VERSION) INSERT_BLOBID_ENTRY(1, VERSION) -#define INSERT_GC_ENTRY(VERSION, EPOCH) \ - PageEntryV3 entry_gc_v##VERSION##_##EPOCH{.file_id = 2, .size = 100 * (VERSION) + (EPOCH), .tag = 0, .offset = 0x234, .checksum = 0x5678}; \ +#define INSERT_GC_ENTRY(VERSION, EPOCH) \ + PageEntryV3 entry_gc_v##VERSION##_##EPOCH{.file_id = 2, .size = 100 * (VERSION) + (EPOCH), .padded_size = 0, .tag = 0, .offset = 0x234, .checksum = 0x5678}; \ entries.createNewEntry(PageVersion((VERSION), (EPOCH)), entry_gc_v##VERSION##_##EPOCH); class VersionedEntriesTest : public ::testing::Test @@ -1271,12 +1271,12 @@ class PageDirectoryGCTest : public PageDirectoryTest { }; -#define INSERT_ENTRY_TO(PAGE_ID, VERSION, BLOB_FILE_ID) \ - PageEntryV3 entry_v##VERSION{.file_id = (BLOB_FILE_ID), .size = (VERSION), .tag = 0, .offset = 0x123, .checksum = 0x4567}; \ - { \ - PageEntriesEdit edit; \ - edit.put((PAGE_ID), entry_v##VERSION); \ - dir->apply(std::move(edit)); \ +#define INSERT_ENTRY_TO(PAGE_ID, VERSION, BLOB_FILE_ID) \ + PageEntryV3 entry_v##VERSION{.file_id = (BLOB_FILE_ID), .size = (VERSION), .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; \ + { \ + PageEntriesEdit edit; \ + edit.put((PAGE_ID), entry_v##VERSION); \ + dir->apply(std::move(edit)); \ } // Insert an entry into mvcc directory #define INSERT_ENTRY(PAGE_ID, VERSION) INSERT_ENTRY_TO(PAGE_ID, VERSION, 1) @@ -1566,7 +1566,7 @@ try INSERT_ENTRY_ACQ_SNAP(page_id, 5); INSERT_ENTRY(another_page_id, 6); INSERT_ENTRY(another_page_id, 7); - PageEntryV3 entry_v8{.file_id = 1, .size = 8, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_v8{.file_id = 1, .size = 8, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.del(page_id); @@ -1756,7 +1756,7 @@ TEST_F(PageDirectoryGCTest, GCOnRefedEntries) try { // 10->entry1, 11->10=>11->entry1; del 10->entry1 - PageEntryV3 entry1{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry1{.file_id = 1, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(10, entry1); @@ -1793,7 +1793,7 @@ TEST_F(PageDirectoryGCTest, GCOnRefedEntries2) try { // 10->entry1, 11->10=>11->entry1; del 10->entry1 - PageEntryV3 entry1{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry1{.file_id = 1, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(10, entry1); @@ -1836,7 +1836,7 @@ TEST_F(PageDirectoryGCTest, UpsertOnRefedEntries) try { // 10->entry1, 11->10, 12->10 - PageEntryV3 entry1{.file_id = 1, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry1{.file_id = 1, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(10, entry1); @@ -1860,7 +1860,7 @@ try } // upsert 10->entry2 - PageEntryV3 entry2{.file_id = 2, .size = 1024, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry2{.file_id = 2, .size = 1024, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; auto full_gc_entries = dir->getEntriesByBlobIds({1}); @@ -2024,10 +2024,10 @@ try return d; }; - PageEntryV3 entry_1_v1{.file_id = 1, .size = 1, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_1_v2{.file_id = 1, .size = 2, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_2_v1{.file_id = 2, .size = 1, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_2_v2{.file_id = 2, .size = 2, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_1_v1{.file_id = 1, .size = 1, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_1_v2{.file_id = 1, .size = 2, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_2_v1{.file_id = 2, .size = 1, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_2_v2{.file_id = 2, .size = 2, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(1, entry_1_v1); @@ -2055,8 +2055,8 @@ try // 10->ext, 11->10, del 10->ext // 50->entry, 51->50, 52->51=>50, del 50 - PageEntryV3 entry_50{.file_id = 1, .size = 50, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_60{.file_id = 1, .size = 90, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_50{.file_id = 1, .size = 50, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_60{.file_id = 1, .size = 90, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; { PageEntriesEdit edit; edit.del(2); @@ -2218,9 +2218,9 @@ try Poco::File(fmt::format("{}/{}{}", path, BlobFile::BLOB_PREFIX_NAME, file_id1)).createFile(); Poco::File(fmt::format("{}/{}{}", path, BlobFile::BLOB_PREFIX_NAME, file_id2)).createFile(); - PageEntryV3 entry_1_v1{.file_id = file_id1, .size = 7890, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_5_v1{.file_id = file_id2, .size = 255, .tag = 0, .offset = 0x100, .checksum = 0x4567}; - PageEntryV3 entry_5_v2{.file_id = file_id2, .size = 255, .tag = 0, .offset = 0x400, .checksum = 0x4567}; + PageEntryV3 entry_1_v1{.file_id = file_id1, .size = 7890, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_5_v1{.file_id = file_id2, .size = 255, .padded_size = 0, .tag = 0, .offset = 0x100, .checksum = 0x4567}; + PageEntryV3 entry_5_v2{.file_id = file_id2, .size = 255, .padded_size = 0, .tag = 0, .offset = 0x400, .checksum = 0x4567}; { PageEntriesEdit edit; edit.put(1, entry_1_v1); @@ -2275,8 +2275,8 @@ CATCH TEST_F(PageDirectoryGCTest, CleanAfterDecreaseRef) try { - PageEntryV3 entry_50_1{.file_id = 1, .size = 7890, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_50_2{.file_id = 2, .size = 7890, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_50_1{.file_id = 1, .size = 7890, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_50_2{.file_id = 2, .size = 7890, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; auto restore_from_edit = [](const PageEntriesEdit & edit) { auto ctx = ::DB::tests::TiFlashTestEnv::getContext(); diff --git a/dbms/src/Storages/Page/V3/tests/gtest_wal_store.cpp b/dbms/src/Storages/Page/V3/tests/gtest_wal_store.cpp index 6d47adabbc5..b4e6c2d9204 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_wal_store.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_wal_store.cpp @@ -34,8 +34,8 @@ namespace DB::PS::V3::tests { TEST(WALSeriTest, AllPuts) { - PageEntryV3 entry_p1{.file_id = 1, .size = 1, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_p2{.file_id = 1, .size = 2, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p1{.file_id = 1, .size = 1, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p2{.file_id = 1, .size = 2, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; PageVersion ver20(/*seq=*/20); PageEntriesEdit edit; edit.put(1, entry_p1); @@ -56,8 +56,8 @@ TEST(WALSeriTest, AllPuts) TEST(WALSeriTest, PutsAndRefsAndDels) try { - PageEntryV3 entry_p3{.file_id = 1, .size = 3, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_p5{.file_id = 1, .size = 5, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p3{.file_id = 1, .size = 3, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p5{.file_id = 1, .size = 5, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; PageVersion ver21(/*seq=*/21); PageEntriesEdit edit; edit.put(3, entry_p3); @@ -104,9 +104,9 @@ CATCH TEST(WALSeriTest, Upserts) { - PageEntryV3 entry_p1_2{.file_id = 2, .size = 1, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_p3_2{.file_id = 2, .size = 3, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_p5_2{.file_id = 2, .size = 5, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p1_2{.file_id = 2, .size = 1, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p3_2{.file_id = 2, .size = 3, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p5_2{.file_id = 2, .size = 5, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; PageVersion ver20_1(/*seq=*/20, /*epoch*/ 1); PageVersion ver21_1(/*seq=*/21, /*epoch*/ 1); PageEntriesEdit edit; @@ -164,7 +164,7 @@ TEST(WALSeriTest, RefExternalAndEntry) { PageEntriesEdit edit; - PageEntryV3 entry_p1_2{.file_id = 2, .size = 1, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p1_2{.file_id = 2, .size = 1, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; edit.varEntry(1, ver1_0, entry_p1_2, 2); edit.varDel(1, ver2_0); edit.varRef(2, ver3_0, 1); @@ -405,8 +405,8 @@ try ASSERT_NE(wal, nullptr); // Stage 2. Apply with only puts - PageEntryV3 entry_p1{.file_id = 1, .size = 1, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_p2{.file_id = 1, .size = 2, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p1{.file_id = 1, .size = 1, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p2{.file_id = 1, .size = 2, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; PageVersion ver20(/*seq=*/20); { PageEntriesEdit edit; @@ -435,8 +435,8 @@ try } // Stage 3. Apply with puts and refs - PageEntryV3 entry_p3{.file_id = 1, .size = 3, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_p5{.file_id = 1, .size = 5, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p3{.file_id = 1, .size = 3, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p5{.file_id = 1, .size = 5, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; PageVersion ver21(/*seq=*/21); { PageEntriesEdit edit; @@ -468,9 +468,9 @@ try // Stage 4. Apply with delete and upsert - PageEntryV3 entry_p1_2{.file_id = 2, .size = 1, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_p3_2{.file_id = 2, .size = 3, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_p5_2{.file_id = 2, .size = 5, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p1_2{.file_id = 2, .size = 1, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p3_2{.file_id = 2, .size = 3, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p5_2{.file_id = 2, .size = 5, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; PageVersion ver20_1(/*seq=*/20, /*epoch*/ 1); PageVersion ver21_1(/*seq=*/21, /*epoch*/ 1); { @@ -514,8 +514,8 @@ try std::vector size_each_edit; // Stage 1. Apply with only puts - PageEntryV3 entry_p1{.file_id = 1, .size = 1, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_p2{.file_id = 1, .size = 2, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p1{.file_id = 1, .size = 1, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p2{.file_id = 1, .size = 2, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; PageVersion ver20(/*seq=*/20); { PageEntriesEdit edit; @@ -526,8 +526,8 @@ try } // Stage 2. Apply with puts and refs - PageEntryV3 entry_p3{.file_id = 1, .size = 3, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_p5{.file_id = 1, .size = 5, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p3{.file_id = 1, .size = 3, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p5{.file_id = 1, .size = 5, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; PageVersion ver21(/*seq=*/21); { PageEntriesEdit edit; @@ -540,9 +540,9 @@ try } // Stage 3. Apply with delete and upsert - PageEntryV3 entry_p1_2{.file_id = 2, .size = 1, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_p3_2{.file_id = 2, .size = 3, .tag = 0, .offset = 0x123, .checksum = 0x4567}; - PageEntryV3 entry_p5_2{.file_id = 2, .size = 5, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p1_2{.file_id = 2, .size = 1, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p3_2{.file_id = 2, .size = 3, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry_p5_2{.file_id = 2, .size = 5, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; PageVersion ver20_1(/*seq=*/20, /*epoch*/ 1); PageVersion ver21_1(/*seq=*/21, /*epoch*/ 1); { @@ -615,7 +615,7 @@ try PageVersion ver(/*seq*/ 32); for (size_t i = 0; i < num_edits_test; ++i) { - PageEntryV3 entry{.file_id = 2, .size = 1, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry{.file_id = 2, .size = 1, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; PageEntriesEdit edit; const size_t num_pages_put = d_20(rd); for (size_t p = 0; p < num_pages_put; ++p) @@ -660,7 +660,7 @@ try .persisted_log_files = persisted_log_files}; PageEntriesEdit snap_edit; - PageEntryV3 entry{.file_id = 2, .size = 1, .tag = 0, .offset = 0x123, .checksum = 0x4567}; + PageEntryV3 entry{.file_id = 2, .size = 1, .padded_size = 0, .tag = 0, .offset = 0x123, .checksum = 0x4567}; std::uniform_int_distribution<> d_10000(0, 10000); // just fill in some random entry for (size_t i = 0; i < 70; ++i) diff --git a/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp b/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp index dcf727614b1..ea19ff08dd3 100644 --- a/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp +++ b/dbms/src/TestUtils/ColumnsToTiPBExpr.cpp @@ -36,6 +36,7 @@ void columnToTiPBExpr(tipb::Expr * expr, const ColumnWithTypeAndName column, siz if (column.column->isColumnNullable()) { auto [col, null_map] = removeNullable(column.column.get()); + (void)null_map; is_const = col->isColumnConst(); } } @@ -97,6 +98,7 @@ void columnsToTiPBExprForTiDBCast( if (type_column.column->isColumnNullable()) { auto [col, null_map] = removeNullable(type_column.column.get()); + (void)null_map; is_const = col->isColumnConst(); } } From ba725cc09fe85074b663e8fbafa117d7c3b7af87 Mon Sep 17 00:00:00 2001 From: JaySon Date: Thu, 9 Jun 2022 20:18:30 +0800 Subject: [PATCH 027/104] PageStorage: Fix entry.tag after full gc && add more debug message (#5094) ref pingcap/tiflash#5076, close pingcap/tiflash#5093 --- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 5 +- dbms/src/Storages/Page/PageUtil.h | 2 +- .../Page/V2/tests/gtest_page_util.cpp | 3 ++ dbms/src/Storages/Page/V3/BlobStore.cpp | 42 +++++++++------- dbms/src/Storages/Page/V3/BlobStore.h | 2 +- .../Page/V3/tests/gtest_blob_store.cpp | 9 ++-- .../Page/V3/tests/gtest_page_storage.cpp | 49 +++++++++++++++++++ 7 files changed, 87 insertions(+), 25 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index a74404f3dbb..195ed5c53c2 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -34,6 +34,7 @@ #include #include #include +#include #include #include @@ -1137,9 +1138,11 @@ BlockInputStreams DeltaMergeStore::readRaw(const Context & db_context, } fiu_do_on(FailPoints::force_slow_page_storage_snapshot_release, { - std::thread thread_hold_snapshots([tasks]() { + std::thread thread_hold_snapshots([this, tasks]() { + LOG_FMT_WARNING(log, "failpoint force_slow_page_storage_snapshot_release begin"); std::this_thread::sleep_for(std::chrono::seconds(5 * 60)); (void)tasks; + LOG_FMT_WARNING(log, "failpoint force_slow_page_storage_snapshot_release end"); }); thread_hold_snapshots.detach(); }); diff --git a/dbms/src/Storages/Page/PageUtil.h b/dbms/src/Storages/Page/PageUtil.h index cebcbdb27f2..b0d8f0f88c8 100644 --- a/dbms/src/Storages/Page/PageUtil.h +++ b/dbms/src/Storages/Page/PageUtil.h @@ -281,7 +281,7 @@ void readFile(T & file, } if (unlikely(bytes_read != expected_bytes)) - throw DB::TiFlashException(fmt::format("No enough data in file {}, read bytes: {} , expected bytes: {}", file->getFileName(), bytes_read, expected_bytes), + throw DB::TiFlashException(fmt::format("No enough data in file {}, read bytes: {}, expected bytes: {}, offset: {}", file->getFileName(), bytes_read, expected_bytes, offset), Errors::PageStorage::FileSizeNotMatch); } diff --git a/dbms/src/Storages/Page/V2/tests/gtest_page_util.cpp b/dbms/src/Storages/Page/V2/tests/gtest_page_util.cpp index e72c7a87541..c4dd2178eb9 100644 --- a/dbms/src/Storages/Page/V2/tests/gtest_page_util.cpp +++ b/dbms/src/Storages/Page/V2/tests/gtest_page_util.cpp @@ -17,6 +17,7 @@ #include #include #include +#include namespace DB { @@ -30,6 +31,7 @@ namespace tests static const std::string FileName = "page_util_test"; TEST(PageUtilsTest, ReadWriteFile) +try { ::remove(FileName.c_str()); @@ -52,6 +54,7 @@ TEST(PageUtilsTest, ReadWriteFile) ::remove(FileName.c_str()); } +CATCH TEST(PageUtilsTest, FileNotExists) { diff --git a/dbms/src/Storages/Page/V3/BlobStore.cpp b/dbms/src/Storages/Page/V3/BlobStore.cpp index d5f71841b91..37a4fd429f4 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.cpp +++ b/dbms/src/Storages/Page/V3/BlobStore.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -555,7 +556,7 @@ void BlobStore::read(PageIDAndEntriesV3 & entries, const PageHandler & handler, for (const auto & [page_id_v3, entry] : entries) { - auto blob_file = read(entry.file_id, entry.offset, data_buf, entry.size, read_limiter); + auto blob_file = read(page_id_v3, entry.file_id, entry.offset, data_buf, entry.size, read_limiter); if constexpr (BLOBSTORE_CHECKSUM_ON_READ) { @@ -635,7 +636,7 @@ PageMap BlobStore::read(FieldReadInfos & to_read, const ReadLimiterPtr & read_li // TODO: Continuously fields can read by one system call. const auto [beg_offset, end_offset] = entry.getFieldOffsets(field_index); const auto size_to_read = end_offset - beg_offset; - auto blob_file = read(entry.file_id, entry.offset + beg_offset, write_offset, size_to_read, read_limiter); + auto blob_file = read(page_id_v3, entry.file_id, entry.offset + beg_offset, write_offset, size_to_read, read_limiter); fields_offset_in_page.emplace(field_index, read_size_this_entry); if constexpr (BLOBSTORE_CHECKSUM_ON_READ) @@ -732,7 +733,7 @@ PageMap BlobStore::read(PageIDAndEntriesV3 & entries, const ReadLimiterPtr & rea PageMap page_map; for (const auto & [page_id_v3, entry] : entries) { - auto blob_file = read(entry.file_id, entry.offset, pos, entry.size, read_limiter); + auto blob_file = read(page_id_v3, entry.file_id, entry.offset, pos, entry.size, read_limiter); if constexpr (BLOBSTORE_CHECKSUM_ON_READ) { @@ -797,7 +798,7 @@ Page BlobStore::read(const PageIDAndEntryV3 & id_entry, const ReadLimiterPtr & r free(p, buf_size); }); - auto blob_file = read(entry.file_id, entry.offset, data_buf, buf_size, read_limiter); + auto blob_file = read(page_id_v3, entry.file_id, entry.offset, data_buf, buf_size, read_limiter); if constexpr (BLOBSTORE_CHECKSUM_ON_READ) { ChecksumClass digest; @@ -824,11 +825,20 @@ Page BlobStore::read(const PageIDAndEntryV3 & id_entry, const ReadLimiterPtr & r return page; } -BlobFilePtr BlobStore::read(BlobFileId blob_id, BlobFileOffset offset, char * buffers, size_t size, const ReadLimiterPtr & read_limiter, bool background) +BlobFilePtr BlobStore::read(const PageIdV3Internal & page_id_v3, BlobFileId blob_id, BlobFileOffset offset, char * buffers, size_t size, const ReadLimiterPtr & read_limiter, bool background) { assert(buffers != nullptr); - auto blob_file = getBlobFile(blob_id); - blob_file->read(buffers, offset, size, read_limiter, background); + BlobFilePtr blob_file = getBlobFile(blob_id); + try + { + blob_file->read(buffers, offset, size, read_limiter, background); + } + catch (DB::Exception & e) + { + // add debug message + e.addMessage(fmt::format("(error while reading page data [page_id={}] [blob_id={}] [offset={}] [size={}] [background={}])", page_id_v3, blob_id, offset, size, background)); + e.rethrow(); + } return blob_file; } @@ -1117,21 +1127,15 @@ PageEntriesEdit BlobStore::gc(std::map & std::tie(blobfile_id, file_offset_beg) = getPosFromStats(next_alloc_size); } - PageEntryV3 new_entry; - - read(file_id, entry.offset, data_pos, entry.size, read_limiter, /*background*/ true); - - // No need do crc again, crc won't be changed. - new_entry.checksum = entry.checksum; - - // Need copy the field_offsets - new_entry.field_offsets = entry.field_offsets; - - // Entry size won't be changed. - new_entry.size = entry.size; + // Read the data into buffer by old entry + read(page_id, file_id, entry.offset, data_pos, entry.size, read_limiter, /*background*/ true); + // Most vars of the entry is not changed, but the file id and offset + // need to be updated. + PageEntryV3 new_entry = entry; new_entry.file_id = blobfile_id; new_entry.offset = file_offset_beg + offset_in_data; + new_entry.padded_size = 0; // reset padded size to be zero offset_in_data += new_entry.size; data_pos += new_entry.size; diff --git a/dbms/src/Storages/Page/V3/BlobStore.h b/dbms/src/Storages/Page/V3/BlobStore.h index 24bf4652123..6b139b98557 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.h +++ b/dbms/src/Storages/Page/V3/BlobStore.h @@ -296,7 +296,7 @@ class BlobStore : private Allocator PageEntriesEdit handleLargeWrite(DB::WriteBatch & wb, const WriteLimiterPtr & write_limiter = nullptr); - BlobFilePtr read(BlobFileId blob_id, BlobFileOffset offset, char * buffers, size_t size, const ReadLimiterPtr & read_limiter = nullptr, bool background = false); + BlobFilePtr read(const PageIdV3Internal & page_id_v3, BlobFileId blob_id, BlobFileOffset offset, char * buffers, size_t size, const ReadLimiterPtr & read_limiter = nullptr, bool background = false); /** * Ask BlobStats to get a span from BlobStat. diff --git a/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp b/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp index 94bb69045ba..fdd08c7cb8e 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp @@ -534,7 +534,8 @@ TEST_F(BlobStoreTest, testWriteRead) ASSERT_EQ(record.entry.file_id, 1); // Read directly from the file - blob_store.read(record.entry.file_id, + blob_store.read(buildV3Id(TEST_NAMESPACE_ID, page_id), + record.entry.file_id, record.entry.offset, c_buff_read + index * buff_size, record.entry.size, @@ -634,7 +635,8 @@ TEST_F(BlobStoreTest, testWriteReadWithIOLimiter) { for (const auto & record : edits[i].getRecords()) { - blob_store.read(record.entry.file_id, + blob_store.read(buildV3Id(TEST_NAMESPACE_ID, page_id), + record.entry.file_id, record.entry.offset, c_buff_read + i * buff_size, record.entry.size, @@ -812,7 +814,8 @@ TEST_F(BlobStoreTest, testFeildOffsetWriteRead) ASSERT_EQ(check_field_sizes, offsets); // Read - blob_store.read(record.entry.file_id, + blob_store.read(buildV3Id(TEST_NAMESPACE_ID, page_id), + record.entry.file_id, record.entry.offset, c_buff_read + index * buff_size, record.entry.size, diff --git a/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp b/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp index f7ba33c46c8..f9ef25cb973 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_storage.cpp @@ -1441,6 +1441,55 @@ try } CATCH +TEST_F(PageStorageTest, EntryTagAfterFullGC) +try +{ + { + PageStorage::Config config; + config.blob_heavy_gc_valid_rate = 1.0; /// always run full gc + page_storage = reopenWithConfig(config); + } + + const size_t buf_sz = 1024; + char c_buff[buf_sz]; + + for (size_t i = 0; i < buf_sz; ++i) + { + c_buff[i] = i % 0xff; + } + + PageId page_id = 120; + UInt64 tag = 12345; + { + WriteBatch batch; + batch.putPage(page_id, tag, std::make_shared(c_buff, buf_sz), buf_sz, {}); + page_storage->write(std::move(batch)); + } + + { + auto entry = page_storage->getEntry(page_id); + ASSERT_EQ(entry.tag, tag); + auto page = page_storage->read(page_id); + for (size_t i = 0; i < buf_sz; ++i) + { + EXPECT_EQ(*(page.data.begin() + i), static_cast(i % 0xff)); + } + } + + auto done_full_gc = page_storage->gc(); + EXPECT_TRUE(done_full_gc); + + { + auto entry = page_storage->getEntry(page_id); + ASSERT_EQ(entry.tag, tag); + auto page = page_storage->read(page_id); + for (size_t i = 0; i < buf_sz; ++i) + { + EXPECT_EQ(*(page.data.begin() + i), static_cast(i % 0xff)); + } + } +} +CATCH } // namespace PS::V3::tests } // namespace DB From a9b322ab2a467827d0ce0574336fb214468362d8 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger <60744015+Lloyd-Pottiger@users.noreply.github.com> Date: Fri, 10 Jun 2022 10:54:31 +0800 Subject: [PATCH 028/104] feat: add page_stress_testing as a subcommand of tiflash (#5038) close pingcap/tiflash#5037 --- CMakeLists.txt | 3 - dbms/CMakeLists.txt | 13 ++-- dbms/src/Server/CMakeLists.txt | 4 ++ dbms/src/Server/config_tools.h.in | 1 + dbms/src/Server/main.cpp | 8 ++- dbms/src/Storages/CMakeLists.txt | 9 ++- .../DeltaMerge/tools/workload/Main.cpp | 22 ------ .../{tools => }/workload/CMakeLists.txt | 3 - .../{tools => }/workload/DTWorkload.cpp | 20 +++--- .../{tools => }/workload/DTWorkload.h | 2 +- .../{tools => }/workload/DataGenerator.cpp | 16 +++-- .../{tools => }/workload/DataGenerator.h | 2 +- .../DeltaMerge/{tools => }/workload/Handle.h | 18 ++--- .../{tools => }/workload/KeyGenerator.cpp | 10 +-- .../{tools => }/workload/KeyGenerator.h | 4 +- .../{tools => }/workload/Limiter.cpp | 10 +-- .../DeltaMerge/{tools => }/workload/Limiter.h | 2 +- .../{tools => }/workload/MainEntry.cpp | 8 +-- .../{tools => }/workload/Options.cpp | 4 +- .../DeltaMerge/{tools => }/workload/Options.h | 0 .../workload/ReadColumnsGenerator.h | 4 +- .../{tools => }/workload/TableGenerator.cpp | 8 +-- .../{tools => }/workload/TableGenerator.h | 2 +- .../{tools => }/workload/TimestampGenerator.h | 0 .../DeltaMerge/{tools => }/workload/Utils.cpp | 8 +-- .../DeltaMerge/{tools => }/workload/Utils.h | 0 dbms/src/Storages/Page/CMakeLists.txt | 10 --- .../Page/stress/stress_page_storage.cpp | 46 ------------ .../tools => Page/workload}/CMakeLists.txt | 7 +- .../workload/HeavyMemoryCostInGC.cpp | 5 +- .../Page/{stress => }/workload/HeavyRead.cpp | 7 +- .../workload/HeavySkewWriteRead.cpp | 7 +- .../Page/{stress => }/workload/HeavyWrite.cpp | 7 +- .../workload/HighValidBigFileGC.cpp | 5 +- .../workload/HoldSnapshotsLongTime.cpp | 7 +- dbms/src/Storages/Page/workload/MainEntry.cpp | 70 +++++++++++++++++++ .../Page/{stress => }/workload/Normal.cpp | 5 +- .../{stress => workload}/PSBackground.cpp | 6 +- .../Page/{stress => workload}/PSBackground.h | 5 +- .../Page/{stress => workload}/PSRunnable.cpp | 13 ++-- .../Page/{stress => workload}/PSRunnable.h | 37 +++++----- .../Page/{stress => workload}/PSStressEnv.cpp | 7 +- .../Page/{stress => workload}/PSStressEnv.h | 3 + .../Page/{stress => workload}/PSWorkload.cpp | 5 +- .../Page/{stress => workload}/PSWorkload.h | 30 ++++---- .../workload/PageStorageInMemoryCapacity.cpp | 12 ++-- .../workload/ThousandsOfOffset.cpp | 7 +- 47 files changed, 270 insertions(+), 212 deletions(-) delete mode 100644 dbms/src/Storages/DeltaMerge/tools/workload/Main.cpp rename dbms/src/Storages/DeltaMerge/{tools => }/workload/CMakeLists.txt (86%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/DTWorkload.cpp (94%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/DTWorkload.h (97%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/DataGenerator.cpp (95%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/DataGenerator.h (96%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/Handle.h (90%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/KeyGenerator.cpp (92%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/KeyGenerator.h (92%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/Limiter.cpp (77%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/Limiter.h (96%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/MainEntry.cpp (97%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/Options.cpp (98%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/Options.h (100%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/ReadColumnsGenerator.h (93%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/TableGenerator.cpp (97%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/TableGenerator.h (96%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/TimestampGenerator.h (100%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/Utils.cpp (94%) rename dbms/src/Storages/DeltaMerge/{tools => }/workload/Utils.h (100%) delete mode 100644 dbms/src/Storages/Page/stress/stress_page_storage.cpp rename dbms/src/Storages/{DeltaMerge/tools => Page/workload}/CMakeLists.txt (53%) rename dbms/src/Storages/Page/{stress => }/workload/HeavyMemoryCostInGC.cpp (96%) rename dbms/src/Storages/Page/{stress => }/workload/HeavyRead.cpp (94%) rename dbms/src/Storages/Page/{stress => }/workload/HeavySkewWriteRead.cpp (95%) rename dbms/src/Storages/Page/{stress => }/workload/HeavyWrite.cpp (94%) rename dbms/src/Storages/Page/{stress => }/workload/HighValidBigFileGC.cpp (97%) rename dbms/src/Storages/Page/{stress => }/workload/HoldSnapshotsLongTime.cpp (95%) create mode 100644 dbms/src/Storages/Page/workload/MainEntry.cpp rename dbms/src/Storages/Page/{stress => }/workload/Normal.cpp (95%) rename dbms/src/Storages/Page/{stress => workload}/PSBackground.cpp (96%) rename dbms/src/Storages/Page/{stress => workload}/PSBackground.h (97%) rename dbms/src/Storages/Page/{stress => workload}/PSRunnable.cpp (97%) rename dbms/src/Storages/Page/{stress => workload}/PSRunnable.h (90%) rename dbms/src/Storages/Page/{stress => workload}/PSStressEnv.cpp (97%) rename dbms/src/Storages/Page/{stress => workload}/PSStressEnv.h (98%) rename dbms/src/Storages/Page/{stress => workload}/PSWorkload.cpp (98%) rename dbms/src/Storages/Page/{stress => workload}/PSWorkload.h (92%) rename dbms/src/Storages/Page/{stress => }/workload/PageStorageInMemoryCapacity.cpp (96%) rename dbms/src/Storages/Page/{stress => }/workload/ThousandsOfOffset.cpp (97%) diff --git a/CMakeLists.txt b/CMakeLists.txt index 4e14c205f18..f2ec9f3316b 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -432,9 +432,6 @@ else (ENABLE_FAILPOINTS) message (STATUS "Failpoints are disabled") endif (ENABLE_FAILPOINTS) -# Enable PageStorage V3 test. -option (ENABLE_V3_PAGESTORAGE "Enables V3 PageStorage" ON) - # Flags for test coverage option (TEST_COVERAGE "Enables flags for test coverage" OFF) option (TEST_COVERAGE_XML "Output XML report for test coverage" OFF) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index cce11bd6997..e1e52fab73b 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -91,12 +91,10 @@ add_headers_and_sources(dbms src/Storages/Page/V2/VersionSet) add_headers_and_sources(dbms src/Storages/Page/V2/gc) add_headers_and_sources(dbms src/WindowFunctions) add_headers_and_sources(dbms src/TiDB/Schema) -if (ENABLE_V3_PAGESTORAGE) - add_headers_and_sources(dbms src/Storages/Page/V3) - add_headers_and_sources(dbms src/Storages/Page/V3/LogFile) - add_headers_and_sources(dbms src/Storages/Page/V3/WAL) - add_headers_and_sources(dbms src/Storages/Page/V3/spacemap) -endif() +add_headers_and_sources(dbms src/Storages/Page/V3) +add_headers_and_sources(dbms src/Storages/Page/V3/LogFile) +add_headers_and_sources(dbms src/Storages/Page/V3/WAL) +add_headers_and_sources(dbms src/Storages/Page/V3/spacemap) add_headers_and_sources(dbms src/Storages/Page/) add_headers_and_sources(dbms src/TiDB) add_headers_and_sources(dbms src/Client) @@ -323,6 +321,9 @@ if (ENABLE_TESTS) if (ENABLE_TIFLASH_DTWORKLOAD) target_link_libraries(bench_dbms dt-workload-lib) endif () + if (ENABLE_TIFLASH_PAGEWORKLOAD) + target_link_libraries(bench_dbms page-workload-lib) + endif () add_check(bench_dbms) endif () diff --git a/dbms/src/Server/CMakeLists.txt b/dbms/src/Server/CMakeLists.txt index 63cf6d0e1f9..104b4f34e4a 100644 --- a/dbms/src/Server/CMakeLists.txt +++ b/dbms/src/Server/CMakeLists.txt @@ -22,6 +22,7 @@ option(ENABLE_CLICKHOUSE_SERVER "Enable server" ${ENABLE_CLICKHOUSE_ALL}) option(ENABLE_CLICKHOUSE_CLIENT "Enable client" ${ENABLE_CLICKHOUSE_ALL}) option(ENABLE_TIFLASH_DTTOOL "Enable dttool: tools to manage dmfile" ${ENABLE_CLICKHOUSE_ALL}) option(ENABLE_TIFLASH_DTWORKLOAD "Enable dtworkload: tools to test and stress DeltaTree" ${ENABLE_CLICKHOUSE_ALL}) +option(ENABLE_TIFLASH_PAGEWORKLOAD "Enable pageworkload: tools to test and stress PageStorage" ${ENABLE_CLICKHOUSE_ALL}) option(ENABLE_TIFLASH_PAGECTL "Enable pagectl: tools to debug page storage" ${ENABLE_CLICKHOUSE_ALL}) configure_file (config_tools.h.in ${CMAKE_CURRENT_BINARY_DIR}/config_tools.h) @@ -136,6 +137,9 @@ endif () if (ENABLE_TIFLASH_DTWORKLOAD) target_link_libraries(tiflash dt-workload-lib) endif () +if (ENABLE_TIFLASH_PAGEWORKLOAD) + target_link_libraries(tiflash page-workload-lib) +endif() if (ENABLE_TIFLASH_PAGECTL) target_link_libraries(tiflash page-ctl-lib) endif () diff --git a/dbms/src/Server/config_tools.h.in b/dbms/src/Server/config_tools.h.in index 61aa3f41591..03a478a6473 100644 --- a/dbms/src/Server/config_tools.h.in +++ b/dbms/src/Server/config_tools.h.in @@ -6,4 +6,5 @@ #cmakedefine01 ENABLE_CLICKHOUSE_CLIENT #cmakedefine01 ENABLE_TIFLASH_DTTOOL #cmakedefine01 ENABLE_TIFLASH_DTWORKLOAD +#cmakedefine01 ENABLE_TIFLASH_PAGEWORKLOAD #cmakedefine01 ENABLE_TIFLASH_PAGECTL diff --git a/dbms/src/Server/main.cpp b/dbms/src/Server/main.cpp index 11cccf84729..dbcaa4f38fc 100644 --- a/dbms/src/Server/main.cpp +++ b/dbms/src/Server/main.cpp @@ -36,7 +36,10 @@ #include #endif #if ENABLE_TIFLASH_DTWORKLOAD -#include +#include +#endif +#if ENABLE_TIFLASH_PAGEWORKLOAD +#include #endif #if ENABLE_TIFLASH_PAGECTL #include @@ -107,6 +110,9 @@ std::pair clickhouse_applications[] = { #if ENABLE_TIFLASH_DTWORKLOAD {"dtworkload", DB::DM::tests::DTWorkload::mainEntry}, #endif +#if ENABLE_TIFLASH_PAGEWORKLOAD + {"pageworkload", DB::PS::tests::StressWorkload::mainEntry}, +#endif #if ENABLE_TIFLASH_PAGECTL {"pagectl", DB::PageStorageCtl::mainEntry}, #endif diff --git a/dbms/src/Storages/CMakeLists.txt b/dbms/src/Storages/CMakeLists.txt index 90cc7a01d5b..68a2e6c9a74 100644 --- a/dbms/src/Storages/CMakeLists.txt +++ b/dbms/src/Storages/CMakeLists.txt @@ -15,16 +15,15 @@ add_subdirectory (System) add_subdirectory (Page) add_subdirectory (DeltaMerge/File/dtpb) -add_subdirectory (DeltaMerge/tools) +add_subdirectory (DeltaMerge/workload) +add_subdirectory (Page/workload) if (ENABLE_TESTS) add_subdirectory (tests EXCLUDE_FROM_ALL) add_subdirectory (Transaction/tests EXCLUDE_FROM_ALL) add_subdirectory (Page/V2/tests EXCLUDE_FROM_ALL) - if (ENABLE_V3_PAGESTORAGE) - add_subdirectory (Page/V3 EXCLUDE_FROM_ALL) - add_subdirectory (Page/V3/tests EXCLUDE_FROM_ALL) - endif () + add_subdirectory (Page/V3 EXCLUDE_FROM_ALL) + add_subdirectory (Page/V3/tests EXCLUDE_FROM_ALL) add_subdirectory (DeltaMerge/tests EXCLUDE_FROM_ALL) endif () diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/Main.cpp b/dbms/src/Storages/DeltaMerge/tools/workload/Main.cpp deleted file mode 100644 index 092c8a89a42..00000000000 --- a/dbms/src/Storages/DeltaMerge/tools/workload/Main.cpp +++ /dev/null @@ -1,22 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include - -using namespace DB::DM::tests; - -int main(int argc, char ** argv) -{ - return DTWorkload::mainEntry(argc, argv); -} diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/CMakeLists.txt b/dbms/src/Storages/DeltaMerge/workload/CMakeLists.txt similarity index 86% rename from dbms/src/Storages/DeltaMerge/tools/workload/CMakeLists.txt rename to dbms/src/Storages/DeltaMerge/workload/CMakeLists.txt index 7227f1cf563..7a83cbec57c 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/CMakeLists.txt +++ b/dbms/src/Storages/DeltaMerge/workload/CMakeLists.txt @@ -18,6 +18,3 @@ set(dt-workload-src MainEntry.cpp DTWorkload.cpp KeyGenerator.cpp TableGenerator add_library(dt-workload-lib ${dt-workload-src}) target_link_libraries(dt-workload-lib dbms clickhouse_functions clickhouse-server-lib) - -add_executable(dt-workload Main.cpp ${dt-workload-src}) -target_link_libraries(dt-workload dbms gtest clickhouse_functions clickhouse-server-lib) diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.cpp b/dbms/src/Storages/DeltaMerge/workload/DTWorkload.cpp similarity index 94% rename from dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.cpp rename to dbms/src/Storages/DeltaMerge/workload/DTWorkload.cpp index a6113f91d91..a53a1b9ebbd 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.cpp +++ b/dbms/src/Storages/DeltaMerge/workload/DTWorkload.cpp @@ -19,16 +19,16 @@ #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 diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.h b/dbms/src/Storages/DeltaMerge/workload/DTWorkload.h similarity index 97% rename from dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.h rename to dbms/src/Storages/DeltaMerge/workload/DTWorkload.h index 26cc5b6e07c..1ee5ba6b871 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/DTWorkload.h +++ b/dbms/src/Storages/DeltaMerge/workload/DTWorkload.h @@ -73,7 +73,7 @@ class ThreadStat class Statistics { public: - Statistics(int write_thread_count = 0, int read_thread_count = 0) + explicit Statistics(int write_thread_count = 0, int read_thread_count = 0) : init_ms(0) , write_stats(write_thread_count) , read_stats(read_thread_count) diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/DataGenerator.cpp b/dbms/src/Storages/DeltaMerge/workload/DataGenerator.cpp similarity index 95% rename from dbms/src/Storages/DeltaMerge/tools/workload/DataGenerator.cpp rename to dbms/src/Storages/DeltaMerge/workload/DataGenerator.cpp index be6ff1dcbbe..479977d46d1 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/DataGenerator.cpp +++ b/dbms/src/Storages/DeltaMerge/workload/DataGenerator.cpp @@ -13,11 +13,11 @@ // limitations under the License. #include -#include -#include -#include -#include -#include +#include +#include +#include +#include +#include #include #include @@ -33,7 +33,7 @@ class RandomDataGenerator : public DataGenerator , rand_gen(std::random_device()()) {} - virtual std::tuple get(uint64_t key) override + std::tuple get(uint64_t key) override { Block block; // Generate 'rowkeys'. @@ -227,7 +227,9 @@ class RandomDataGenerator : public DataGenerator struct tm randomLocalTime() { time_t t = randomUTCTimestamp(); - struct tm res; + struct tm res + { + }; if (localtime_r(&t, &res) == nullptr) { throw std::invalid_argument(fmt::format("localtime_r({}) ret {}", t, strerror(errno))); diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/DataGenerator.h b/dbms/src/Storages/DeltaMerge/workload/DataGenerator.h similarity index 96% rename from dbms/src/Storages/DeltaMerge/tools/workload/DataGenerator.h rename to dbms/src/Storages/DeltaMerge/workload/DataGenerator.h index e32de4591e6..cd29f1a3a80 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/DataGenerator.h +++ b/dbms/src/Storages/DeltaMerge/workload/DataGenerator.h @@ -27,7 +27,7 @@ class DataGenerator public: static std::unique_ptr create(const WorkloadOptions & opts, const TableInfo & table_info, TimestampGenerator & ts_gen); virtual std::tuple get(uint64_t key) = 0; - virtual ~DataGenerator() {} + virtual ~DataGenerator() = default; }; std::string blockToString(const Block & block); diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/Handle.h b/dbms/src/Storages/DeltaMerge/workload/Handle.h similarity index 90% rename from dbms/src/Storages/DeltaMerge/tools/workload/Handle.h rename to dbms/src/Storages/DeltaMerge/workload/Handle.h index eb117a4fddd..c4949c15a1f 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/Handle.h +++ b/dbms/src/Storages/DeltaMerge/workload/Handle.h @@ -16,7 +16,7 @@ #include #include -#include +#include #include #include @@ -40,7 +40,7 @@ class HandleLock static constexpr uint64_t default_lock_count = 4096; static std::unique_ptr create(const TableInfo & table_info); - HandleLock(uint64_t lock_count = default_lock_count) + explicit HandleLock(uint64_t lock_count = default_lock_count) : rmtxs(lock_count) {} @@ -51,14 +51,14 @@ class HandleLock std::vector> getLocks(const std::vector & handles) { - std::vector indexes; + std::vector indexes(handles.size()); for (const auto & h : handles) { indexes.push_back(index(h)); } // Sort mutex indexes to avoid dead lock. sort(indexes.begin(), indexes.end()); - std::vector> locks; + std::vector> locks(indexes.size()); for (auto i : indexes) { locks.push_back(getLockByIndex(i)); @@ -105,7 +105,7 @@ class HandleTable std::lock_guard lock(mtx); handle_to_ts[handle] = ts; Record r{handle, ts}; - if (wal != nullptr && wal->write((char *)&r, sizeof(r)) != sizeof(r)) + if (wal != nullptr && wal->write(reinterpret_cast(&r), sizeof(r)) != sizeof(r)) { throw std::runtime_error(fmt::format("write ret {}", strerror(errno))); } @@ -134,8 +134,8 @@ class HandleTable try { PosixRandomAccessFile f(fname, -1); - Record r; - while (f.read((char *)&r, sizeof(r)) == sizeof(r)) + Record r{}; + while (f.read(reinterpret_cast(&r), sizeof(r)) == sizeof(r)) { handle_to_ts[r.handle] = r.ts; } @@ -156,7 +156,7 @@ class HandleTable for (const auto & pa : handle_to_ts) { Record r{pa.first, pa.second}; - if (f.write((char *)&r, sizeof(r)) != sizeof(r)) + if (f.write(reinterpret_cast(&r), sizeof(r)) != sizeof(r)) { throw std::runtime_error(fmt::format("write ret {}", strerror(errno))); } @@ -191,7 +191,7 @@ class SharedHandleTable public: static constexpr uint64_t default_shared_count = 4096; - SharedHandleTable(uint64_t max_key_count, const std::string & waldir = "", uint64_t shared_cnt = default_shared_count) + explicit SharedHandleTable(uint64_t max_key_count, const std::string & waldir = "", uint64_t shared_cnt = default_shared_count) : tables(shared_cnt) { uint64_t max_key_count_per_shared = max_key_count / default_shared_count + 1; diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/KeyGenerator.cpp b/dbms/src/Storages/DeltaMerge/workload/KeyGenerator.cpp similarity index 92% rename from dbms/src/Storages/DeltaMerge/tools/workload/KeyGenerator.cpp rename to dbms/src/Storages/DeltaMerge/workload/KeyGenerator.cpp index bb2f2253279..f899ec71b4b 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/KeyGenerator.cpp +++ b/dbms/src/Storages/DeltaMerge/workload/KeyGenerator.cpp @@ -12,8 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include -#include +#include +#include #include #include @@ -31,7 +31,7 @@ class IncrementalKeyGenerator : public KeyGenerator , key(0) {} - virtual uint64_t get64() override + uint64_t get64() override { return key.fetch_add(1, std::memory_order_relaxed) % key_count + start_key; } @@ -54,7 +54,7 @@ class UniformDistributionKeyGenerator : public KeyGenerator , uniform_dist(0, key_count) {} - virtual uint64_t get64() override + uint64_t get64() override { std::lock_guard lock(mtx); return uniform_dist(rand_gen); @@ -78,7 +78,7 @@ class NormalDistributionKeyGenerator : public KeyGenerator , normal_dist(key_count / 2.0, key_count / 20.0) {} - virtual uint64_t get64() override + uint64_t get64() override { std::lock_guard lock(mtx); return normal_dist(rand_gen); diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/KeyGenerator.h b/dbms/src/Storages/DeltaMerge/workload/KeyGenerator.h similarity index 92% rename from dbms/src/Storages/DeltaMerge/tools/workload/KeyGenerator.h rename to dbms/src/Storages/DeltaMerge/workload/KeyGenerator.h index 447f3ffc27a..7c8b8fd0080 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/KeyGenerator.h +++ b/dbms/src/Storages/DeltaMerge/workload/KeyGenerator.h @@ -23,8 +23,8 @@ class KeyGenerator public: static std::unique_ptr create(const WorkloadOptions & opts); - KeyGenerator() {} - virtual ~KeyGenerator() {} + KeyGenerator() = default; + virtual ~KeyGenerator() = default; virtual uint64_t get64() = 0; }; diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/Limiter.cpp b/dbms/src/Storages/DeltaMerge/workload/Limiter.cpp similarity index 77% rename from dbms/src/Storages/DeltaMerge/tools/workload/Limiter.cpp rename to dbms/src/Storages/DeltaMerge/workload/Limiter.cpp index 73764d27bc5..65f9e3ce72c 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/Limiter.cpp +++ b/dbms/src/Storages/DeltaMerge/workload/Limiter.cpp @@ -13,8 +13,8 @@ // limitations under the License. #include -#include -#include +#include +#include #include #include @@ -24,10 +24,10 @@ namespace DB::DM::tests class ConstantLimiter : public Limiter { public: - ConstantLimiter(uint64_t rate_per_sec) + explicit ConstantLimiter(uint64_t rate_per_sec) : limiter(rate_per_sec, LimiterType::UNKNOW) {} - virtual void request() override + void request() override { limiter.request(1); } @@ -38,7 +38,7 @@ class ConstantLimiter : public Limiter std::unique_ptr Limiter::create(const WorkloadOptions & opts) { - uint64_t per_sec = std::ceil(static_cast(opts.max_write_per_sec / opts.write_thread_count)); + uint64_t per_sec = std::ceil(opts.max_write_per_sec * 1.0 / opts.write_thread_count); return std::make_unique(per_sec); } diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/Limiter.h b/dbms/src/Storages/DeltaMerge/workload/Limiter.h similarity index 96% rename from dbms/src/Storages/DeltaMerge/tools/workload/Limiter.h rename to dbms/src/Storages/DeltaMerge/workload/Limiter.h index e2892b178a2..da2d31c7915 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/Limiter.h +++ b/dbms/src/Storages/DeltaMerge/workload/Limiter.h @@ -23,6 +23,6 @@ class Limiter public: static std::unique_ptr create(const WorkloadOptions & opts); virtual void request() = 0; - virtual ~Limiter() {} + virtual ~Limiter() = default; }; } // namespace DB::DM::tests \ No newline at end of file diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/MainEntry.cpp b/dbms/src/Storages/DeltaMerge/workload/MainEntry.cpp similarity index 97% rename from dbms/src/Storages/DeltaMerge/tools/workload/MainEntry.cpp rename to dbms/src/Storages/DeltaMerge/workload/MainEntry.cpp index f79d414f20b..88cf0b6322f 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/MainEntry.cpp +++ b/dbms/src/Storages/DeltaMerge/workload/MainEntry.cpp @@ -14,10 +14,10 @@ #include #include -#include -#include -#include -#include +#include +#include +#include +#include #include #include #include diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/Options.cpp b/dbms/src/Storages/DeltaMerge/workload/Options.cpp similarity index 98% rename from dbms/src/Storages/DeltaMerge/tools/workload/Options.cpp rename to dbms/src/Storages/DeltaMerge/workload/Options.cpp index 1c6409f3c53..8545d22ca8d 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/Options.cpp +++ b/dbms/src/Storages/DeltaMerge/workload/Options.cpp @@ -13,8 +13,8 @@ // limitations under the License. #include -#include -#include +#include +#include #include #include diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/Options.h b/dbms/src/Storages/DeltaMerge/workload/Options.h similarity index 100% rename from dbms/src/Storages/DeltaMerge/tools/workload/Options.h rename to dbms/src/Storages/DeltaMerge/workload/Options.h diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/ReadColumnsGenerator.h b/dbms/src/Storages/DeltaMerge/workload/ReadColumnsGenerator.h similarity index 93% rename from dbms/src/Storages/DeltaMerge/tools/workload/ReadColumnsGenerator.h rename to dbms/src/Storages/DeltaMerge/workload/ReadColumnsGenerator.h index 180409f89e1..c881bb148a2 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/ReadColumnsGenerator.h +++ b/dbms/src/Storages/DeltaMerge/workload/ReadColumnsGenerator.h @@ -14,7 +14,7 @@ #pragma once -#include +#include #include @@ -28,7 +28,7 @@ class ReadColumnsGenerator return std::make_unique(table_info); } - ReadColumnsGenerator(const TableInfo & table_info_) + explicit ReadColumnsGenerator(const TableInfo & table_info_) : table_info(table_info_) , rand_gen(std::random_device()()) , uniform_dist(0, table_info_.columns->size() - 1) diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/TableGenerator.cpp b/dbms/src/Storages/DeltaMerge/workload/TableGenerator.cpp similarity index 97% rename from dbms/src/Storages/DeltaMerge/tools/workload/TableGenerator.cpp rename to dbms/src/Storages/DeltaMerge/workload/TableGenerator.cpp index cf52e808ab1..ec29a476d6a 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/TableGenerator.cpp +++ b/dbms/src/Storages/DeltaMerge/workload/TableGenerator.cpp @@ -15,8 +15,8 @@ #include #include #include -#include -#include +#include +#include #include #include @@ -237,7 +237,7 @@ class RandomTableGenerator : public TableGenerator , rand_gen(std::random_device()()) {} - virtual TableInfo get(int64_t table_id, std::string table_name) override + TableInfo get(int64_t table_id, std::string table_name) override { TableInfo table_info; @@ -293,7 +293,7 @@ class RandomTableGenerator : public TableGenerator class ConstantTableGenerator : public TableGenerator { - virtual TableInfo get(int64_t table_id, std::string table_name) override + TableInfo get(int64_t table_id, std::string table_name) override { TableInfo table_info; diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/TableGenerator.h b/dbms/src/Storages/DeltaMerge/workload/TableGenerator.h similarity index 96% rename from dbms/src/Storages/DeltaMerge/tools/workload/TableGenerator.h rename to dbms/src/Storages/DeltaMerge/workload/TableGenerator.h index aba5c1590b7..b88bf2b72e2 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/TableGenerator.h +++ b/dbms/src/Storages/DeltaMerge/workload/TableGenerator.h @@ -38,6 +38,6 @@ class TableGenerator virtual TableInfo get(int64_t table_id, std::string table_name) = 0; - virtual ~TableGenerator() {} + virtual ~TableGenerator() = default; }; } // namespace DB::DM::tests \ No newline at end of file diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/TimestampGenerator.h b/dbms/src/Storages/DeltaMerge/workload/TimestampGenerator.h similarity index 100% rename from dbms/src/Storages/DeltaMerge/tools/workload/TimestampGenerator.h rename to dbms/src/Storages/DeltaMerge/workload/TimestampGenerator.h diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/Utils.cpp b/dbms/src/Storages/DeltaMerge/workload/Utils.cpp similarity index 94% rename from dbms/src/Storages/DeltaMerge/tools/workload/Utils.cpp rename to dbms/src/Storages/DeltaMerge/workload/Utils.cpp index 1cefae724c6..80d9f788016 100644 --- a/dbms/src/Storages/DeltaMerge/tools/workload/Utils.cpp +++ b/dbms/src/Storages/DeltaMerge/workload/Utils.cpp @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include #include #include @@ -83,7 +83,7 @@ std::string fieldToString(const DataTypePtr & data_type, const Field & f) } else if (t == Field::Types::Which::Decimal256) { - auto i = f.get(); + const auto & i = f.get(); auto scale = dynamic_cast(data_type.get())->getScale(); return i.toString(scale); } @@ -105,8 +105,8 @@ std::vector colToVec(const DataTypePtr & data_type, const ColumnPtr std::string blockToString(const Block & block) { std::string s = "id name type values\n"; - auto & cols = block.getColumnsWithTypeAndName(); - for (auto & col : cols) + const auto & cols = block.getColumnsWithTypeAndName(); + for (const auto & col : cols) { s += fmt::format("{} {} {} {}\n", col.column_id, col.name, col.type->getFamilyName(), colToVec(col.type, col.column)); } diff --git a/dbms/src/Storages/DeltaMerge/tools/workload/Utils.h b/dbms/src/Storages/DeltaMerge/workload/Utils.h similarity index 100% rename from dbms/src/Storages/DeltaMerge/tools/workload/Utils.h rename to dbms/src/Storages/DeltaMerge/workload/Utils.h diff --git a/dbms/src/Storages/Page/CMakeLists.txt b/dbms/src/Storages/Page/CMakeLists.txt index cead83fa126..f208dc84be2 100644 --- a/dbms/src/Storages/Page/CMakeLists.txt +++ b/dbms/src/Storages/Page/CMakeLists.txt @@ -14,13 +14,3 @@ add_subdirectory(V2) add_subdirectory(tools) - -# PageStorage Stress test -if (ENABLE_V3_PAGESTORAGE) - add_headers_and_sources(page_stress_testing stress) - add_headers_and_sources(page_stress_testing stress/workload) - add_executable(page_stress_testing EXCLUDE_FROM_ALL ${page_stress_testing_sources}) - target_link_libraries(page_stress_testing dbms page_storage_v3) - target_include_directories(page_stress_testing PRIVATE stress) - target_compile_options(page_stress_testing PRIVATE -Wno-format -lc++) # turn off printf format check -endif() \ No newline at end of file diff --git a/dbms/src/Storages/Page/stress/stress_page_storage.cpp b/dbms/src/Storages/Page/stress/stress_page_storage.cpp deleted file mode 100644 index 818be710363..00000000000 --- a/dbms/src/Storages/Page/stress/stress_page_storage.cpp +++ /dev/null @@ -1,46 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include -#include - -namespace DB -{ -// Define is_background_thread for this binary -// It is required for `RateLimiter` but we do not link with `BackgroundProcessingPool`. -#if __APPLE__ && __clang__ -__thread bool is_background_thread = false; -#else -thread_local bool is_background_thread = false; -#endif -} // namespace DB - -int main(int argc, char ** argv) -try -{ - StressEnv::initGlobalLogger(); - auto env = StressEnv::parse(argc, argv); - env.setup(); - - auto & mamager = StressWorkloadManger::getInstance(); - mamager.setEnv(env); - mamager.runWorkload(); - - return StressEnvStatus::getInstance().isSuccess(); -} -catch (...) -{ - DB::tryLogCurrentException(""); - exit(-1); -} diff --git a/dbms/src/Storages/DeltaMerge/tools/CMakeLists.txt b/dbms/src/Storages/Page/workload/CMakeLists.txt similarity index 53% rename from dbms/src/Storages/DeltaMerge/tools/CMakeLists.txt rename to dbms/src/Storages/Page/workload/CMakeLists.txt index 36270a0c8e4..5c8ecb34d97 100644 --- a/dbms/src/Storages/DeltaMerge/tools/CMakeLists.txt +++ b/dbms/src/Storages/Page/workload/CMakeLists.txt @@ -14,4 +14,9 @@ include_directories (${CMAKE_CURRENT_BINARY_DIR}) -add_subdirectory (workload EXCLUDE_FROM_ALL) +set (page-workload-src HeavyMemoryCostInGC.cpp HeavyRead.cpp HeavySkewWriteRead.cpp HeavyWrite.cpp HighValidBigFileGC.cpp HoldSnapshotsLongTime.cpp Normal.cpp + PageStorageInMemoryCapacity.cpp ThousandsOfOffset.cpp MainEntry.cpp Normal.cpp PageStorageInMemoryCapacity.cpp PSBackground.cpp PSRunnable.cpp PSStressEnv.cpp PSWorkload.cpp) + +add_library (page-workload-lib ${page-workload-src}) +target_link_libraries (page-workload-lib dbms clickhouse_functions clickhouse-server-lib) +target_compile_options (page-workload-lib PRIVATE -Wno-format -lc++) \ No newline at end of file diff --git a/dbms/src/Storages/Page/stress/workload/HeavyMemoryCostInGC.cpp b/dbms/src/Storages/Page/workload/HeavyMemoryCostInGC.cpp similarity index 96% rename from dbms/src/Storages/Page/stress/workload/HeavyMemoryCostInGC.cpp rename to dbms/src/Storages/Page/workload/HeavyMemoryCostInGC.cpp index 40595f0cb59..7e745e29fc2 100644 --- a/dbms/src/Storages/Page/stress/workload/HeavyMemoryCostInGC.cpp +++ b/dbms/src/Storages/Page/workload/HeavyMemoryCostInGC.cpp @@ -12,8 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include +namespace DB::PS::tests +{ class HeavyMemoryCostInGC : public StressWorkload , public StressWorkloadFunc @@ -81,3 +83,4 @@ class HeavyMemoryCostInGC }; REGISTER_WORKLOAD(HeavyMemoryCostInGC) +} // namespace DB::PS::tests diff --git a/dbms/src/Storages/Page/stress/workload/HeavyRead.cpp b/dbms/src/Storages/Page/workload/HeavyRead.cpp similarity index 94% rename from dbms/src/Storages/Page/stress/workload/HeavyRead.cpp rename to dbms/src/Storages/Page/workload/HeavyRead.cpp index 15aeb1320cf..a67c435e84c 100644 --- a/dbms/src/Storages/Page/stress/workload/HeavyRead.cpp +++ b/dbms/src/Storages/Page/workload/HeavyRead.cpp @@ -12,8 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include +namespace DB::PS::tests +{ class HeavyRead : public StressWorkload , public StressWorkloadFunc { @@ -69,4 +71,5 @@ class HeavyRead : public StressWorkload } }; -REGISTER_WORKLOAD(HeavyRead) \ No newline at end of file +REGISTER_WORKLOAD(HeavyRead) +} // namespace DB::PS::tests \ No newline at end of file diff --git a/dbms/src/Storages/Page/stress/workload/HeavySkewWriteRead.cpp b/dbms/src/Storages/Page/workload/HeavySkewWriteRead.cpp similarity index 95% rename from dbms/src/Storages/Page/stress/workload/HeavySkewWriteRead.cpp rename to dbms/src/Storages/Page/workload/HeavySkewWriteRead.cpp index 78ffa5b60e0..805bf105358 100644 --- a/dbms/src/Storages/Page/stress/workload/HeavySkewWriteRead.cpp +++ b/dbms/src/Storages/Page/workload/HeavySkewWriteRead.cpp @@ -12,8 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include +namespace DB::PS::tests +{ class HeavySkewWriteRead : public StressWorkload , public StressWorkloadFunc { @@ -84,4 +86,5 @@ class HeavySkewWriteRead : public StressWorkload } }; -REGISTER_WORKLOAD(HeavySkewWriteRead) \ No newline at end of file +REGISTER_WORKLOAD(HeavySkewWriteRead) +} // namespace DB::PS::tests \ No newline at end of file diff --git a/dbms/src/Storages/Page/stress/workload/HeavyWrite.cpp b/dbms/src/Storages/Page/workload/HeavyWrite.cpp similarity index 94% rename from dbms/src/Storages/Page/stress/workload/HeavyWrite.cpp rename to dbms/src/Storages/Page/workload/HeavyWrite.cpp index 265b289db56..8dfd7f810f7 100644 --- a/dbms/src/Storages/Page/stress/workload/HeavyWrite.cpp +++ b/dbms/src/Storages/Page/workload/HeavyWrite.cpp @@ -12,8 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include +namespace DB::PS::tests +{ class HeavyWrite : public StressWorkload , public StressWorkloadFunc { @@ -71,4 +73,5 @@ class HeavyWrite : public StressWorkload } }; -REGISTER_WORKLOAD(HeavyWrite) \ No newline at end of file +REGISTER_WORKLOAD(HeavyWrite) +} // namespace DB::PS::tests \ No newline at end of file diff --git a/dbms/src/Storages/Page/stress/workload/HighValidBigFileGC.cpp b/dbms/src/Storages/Page/workload/HighValidBigFileGC.cpp similarity index 97% rename from dbms/src/Storages/Page/stress/workload/HighValidBigFileGC.cpp rename to dbms/src/Storages/Page/workload/HighValidBigFileGC.cpp index 866782c9578..a9af6aebb76 100644 --- a/dbms/src/Storages/Page/stress/workload/HighValidBigFileGC.cpp +++ b/dbms/src/Storages/Page/workload/HighValidBigFileGC.cpp @@ -12,8 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include +namespace DB::PS::tests +{ class HighValidBigFileGCWorkload : public StressWorkload , public StressWorkloadFunc @@ -129,3 +131,4 @@ class HighValidBigFileGCWorkload }; REGISTER_WORKLOAD(HighValidBigFileGCWorkload) +} // namespace DB::PS::tests \ No newline at end of file diff --git a/dbms/src/Storages/Page/stress/workload/HoldSnapshotsLongTime.cpp b/dbms/src/Storages/Page/workload/HoldSnapshotsLongTime.cpp similarity index 95% rename from dbms/src/Storages/Page/stress/workload/HoldSnapshotsLongTime.cpp rename to dbms/src/Storages/Page/workload/HoldSnapshotsLongTime.cpp index b49347fc858..f02fbf65bcd 100644 --- a/dbms/src/Storages/Page/stress/workload/HoldSnapshotsLongTime.cpp +++ b/dbms/src/Storages/Page/workload/HoldSnapshotsLongTime.cpp @@ -12,8 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include +namespace DB::PS::tests +{ class HoldSnapshotsLongTime : public StressWorkload , public StressWorkloadFunc { @@ -93,4 +95,5 @@ class HoldSnapshotsLongTime : public StressWorkload } }; -REGISTER_WORKLOAD(HoldSnapshotsLongTime) \ No newline at end of file +REGISTER_WORKLOAD(HoldSnapshotsLongTime) +} // namespace DB::PS::tests \ No newline at end of file diff --git a/dbms/src/Storages/Page/workload/MainEntry.cpp b/dbms/src/Storages/Page/workload/MainEntry.cpp new file mode 100644 index 00000000000..ac82e1ea4bc --- /dev/null +++ b/dbms/src/Storages/Page/workload/MainEntry.cpp @@ -0,0 +1,70 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +#include +#include + +using namespace DB::PS::tests; + +int StressWorkload::mainEntry(int argc, char ** argv) +{ + { + // maybe due to sequence of linking, REGISTER_WORKLOAD is not visible to main function in dbms/src/Server/main.cpp + // cause that REGISTER_WORKLOAD will not be triggered before mainEntry + // we do this to trigger REGISTER_WORKLOAD explicitly. + void _work_load_register_named_HeavyMemoryCostInGC(); + void (*f)() = _work_load_register_named_HeavyMemoryCostInGC; + (void)f; + void _work_load_register_named_HeavyRead(); + f = _work_load_register_named_HeavyRead; + (void)f; + void _work_load_register_named_HeavySkewWriteRead(); + f = _work_load_register_named_HeavySkewWriteRead; + (void)f; + void _work_load_register_named_HeavyWrite(); + f = _work_load_register_named_HeavyWrite; + (void)f; + void _work_load_register_named_HighValidBigFileGCWorkload(); + f = _work_load_register_named_HighValidBigFileGCWorkload; + (void)f; + void _work_load_register_named_HoldSnapshotsLongTime(); + f = _work_load_register_named_HoldSnapshotsLongTime; + (void)f; + void _work_load_register_named_PageStorageInMemoryCapacity(); + f = _work_load_register_named_PageStorageInMemoryCapacity; + (void)f; + void _work_load_register_named_NormalWorkload(); + f = _work_load_register_named_NormalWorkload; + (void)f; + void _work_load_register_named_ThousandsOfOffset(); + f = _work_load_register_named_ThousandsOfOffset; + (void)f; + } + try + { + StressEnv::initGlobalLogger(); + auto env = StressEnv::parse(argc, argv); + env.setup(); + + auto & mamager = StressWorkloadManger::getInstance(); + mamager.setEnv(env); + mamager.runWorkload(); + + return StressEnvStatus::getInstance().isSuccess(); + } + catch (...) + { + DB::tryLogCurrentException(""); + exit(-1); + } +} \ No newline at end of file diff --git a/dbms/src/Storages/Page/stress/workload/Normal.cpp b/dbms/src/Storages/Page/workload/Normal.cpp similarity index 95% rename from dbms/src/Storages/Page/stress/workload/Normal.cpp rename to dbms/src/Storages/Page/workload/Normal.cpp index 0323b857613..57229395809 100644 --- a/dbms/src/Storages/Page/stress/workload/Normal.cpp +++ b/dbms/src/Storages/Page/workload/Normal.cpp @@ -12,8 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include +namespace DB::PS::tests +{ class NormalWorkload : public StressWorkload , public StressWorkloadFunc @@ -77,3 +79,4 @@ class NormalWorkload }; REGISTER_WORKLOAD(NormalWorkload) +} // namespace DB::PS::tests diff --git a/dbms/src/Storages/Page/stress/PSBackground.cpp b/dbms/src/Storages/Page/workload/PSBackground.cpp similarity index 96% rename from dbms/src/Storages/Page/stress/PSBackground.cpp rename to dbms/src/Storages/Page/workload/PSBackground.cpp index af7329e8348..247bea23dcc 100644 --- a/dbms/src/Storages/Page/stress/PSBackground.cpp +++ b/dbms/src/Storages/Page/workload/PSBackground.cpp @@ -13,11 +13,14 @@ // limitations under the License. #include -#include #include #include +#include #include + +namespace DB::PS::tests +{ void PSMetricsDumper::onTime(Poco::Timer & /*timer*/) { for (auto & metric : metrics) @@ -107,3 +110,4 @@ void StressTimeout::start() { timeout_timer.start(Poco::TimerCallback(*this, &StressTimeout::onTime)); } +} // namespace DB::PS::tests diff --git a/dbms/src/Storages/Page/stress/PSBackground.h b/dbms/src/Storages/Page/workload/PSBackground.h similarity index 97% rename from dbms/src/Storages/Page/stress/PSBackground.h rename to dbms/src/Storages/Page/workload/PSBackground.h index 8c22458c5e8..c91dad1361f 100644 --- a/dbms/src/Storages/Page/stress/PSBackground.h +++ b/dbms/src/Storages/Page/workload/PSBackground.h @@ -15,14 +15,16 @@ #pragma once #include #include -#include #include +#include namespace CurrentMetrics { extern const Metric PSMVCCSnapshotsList; } +namespace DB::PS::tests +{ class PSMetricsDumper { public: @@ -162,3 +164,4 @@ class StressTimeout Poco::Timer timeout_timer; }; using StressTimeoutPtr = std::shared_ptr; +} // namespace DB::PS::tests diff --git a/dbms/src/Storages/Page/stress/PSRunnable.cpp b/dbms/src/Storages/Page/workload/PSRunnable.cpp similarity index 97% rename from dbms/src/Storages/Page/stress/PSRunnable.cpp rename to dbms/src/Storages/Page/workload/PSRunnable.cpp index 5d6c8ecc5c6..5e9774ccc99 100644 --- a/dbms/src/Storages/Page/stress/PSRunnable.cpp +++ b/dbms/src/Storages/Page/workload/PSRunnable.cpp @@ -16,14 +16,16 @@ #include #include #include -#include #include #include +#include #include #include #include +namespace DB::PS::tests +{ void PSRunnable::run() try { @@ -69,7 +71,7 @@ DB::ReadBufferPtr PSWriter::genRandomData(const DB::PageId pageId, DB::MemHolder std::uniform_int_distribution<> dist(0, 3000); const size_t buff_sz = approx_page_mb * DB::MB + dist(size_gen); - char * buff = static_cast(malloc(buff_sz)); + char * buff = static_cast(malloc(buff_sz)); // NOLINT if (buff == nullptr) { throw DB::Exception("Alloc fix memory failed.", DB::ErrorCodes::LOGICAL_ERROR); @@ -78,7 +80,7 @@ DB::ReadBufferPtr PSWriter::genRandomData(const DB::PageId pageId, DB::MemHolder const char buff_ch = pageId % 0xFF; memset(buff, buff_ch, buff_sz); - holder = DB::createMemHolder(buff, [&](char * p) { free(p); }); + holder = DB::createMemHolder(buff, [&](char * p) { free(p); }); // NOLINT return std::make_shared(const_cast(buff), buff_sz); } @@ -88,7 +90,7 @@ void PSWriter::updatedRandomData() size_t memory_size = approx_page_mb * DB::MB * 2; if (memory == nullptr) { - memory = static_cast(malloc(memory_size)); + memory = static_cast(malloc(memory_size)); // NOLINT if (memory == nullptr) { throw DB::Exception("Alloc fix memory failed.", DB::ErrorCodes::LOGICAL_ERROR); @@ -147,7 +149,7 @@ void PSCommonWriter::updatedRandomData() if (memory == nullptr) { - memory = static_cast(malloc(memory_size)); + memory = static_cast(malloc(memory_size)); // NOLINT if (memory == nullptr) { throw DB::Exception("Alloc fix memory failed.", DB::ErrorCodes::LOGICAL_ERROR); @@ -415,3 +417,4 @@ DB::PageId PSIncreaseWriter::genRandomPageId() { return static_cast(begin_page_id++); } +} // namespace DB::PS::tests diff --git a/dbms/src/Storages/Page/stress/PSRunnable.h b/dbms/src/Storages/Page/workload/PSRunnable.h similarity index 90% rename from dbms/src/Storages/Page/stress/PSRunnable.h rename to dbms/src/Storages/Page/workload/PSRunnable.h index 3ddcd73c093..b723236391d 100644 --- a/dbms/src/Storages/Page/stress/PSRunnable.h +++ b/dbms/src/Storages/Page/workload/PSRunnable.h @@ -13,12 +13,14 @@ // limitations under the License. #pragma once -#include #include #include +#include const DB::PageId MAX_PAGE_ID_DEFAULT = 1000; +namespace DB::PS::tests +{ class PSRunnable : public Poco::Runnable { public: @@ -46,7 +48,7 @@ class PSWriter : public PSRunnable gen.seed(time(nullptr)); } - virtual ~PSWriter() + ~PSWriter() override { if (memory != nullptr) { @@ -54,7 +56,7 @@ class PSWriter : public PSRunnable } } - virtual String description() override + String description() override { return fmt::format("(Stress Test Writer {})", index); } @@ -67,7 +69,7 @@ class PSWriter : public PSRunnable static void fillAllPages(const PSPtr & ps); - virtual bool runImpl() override; + bool runImpl() override; protected: virtual DB::PageId genRandomPageId(); @@ -91,11 +93,11 @@ class PSCommonWriter : public PSWriter : PSWriter(ps_, index_) {} - virtual void updatedRandomData() override; + void updatedRandomData() override; - virtual String description() override { return fmt::format("(Stress Test Common Writer {})", index); } + String description() override { return fmt::format("(Stress Test Common Writer {})", index); } - virtual bool runImpl() override; + bool runImpl() override; void setBatchBufferNums(size_t numbers); @@ -120,7 +122,7 @@ class PSCommonWriter : public PSWriter DB::PageFieldSizes data_sizes = {}; - virtual DB::PageId genRandomPageId() override; + DB::PageId genRandomPageId() override; virtual size_t genBufferSize(); }; @@ -154,7 +156,7 @@ class PSWindowWriter : public PSCommonWriter void setNormalDistributionSigma(size_t sigma); protected: - virtual DB::PageId genRandomPageId() override; + DB::PageId genRandomPageId() override; protected: size_t window_size = 100; @@ -170,12 +172,12 @@ class PSIncreaseWriter : public PSCommonWriter String description() override { return fmt::format("(Stress Test Increase Writer {})", index); } - virtual bool runImpl() override; + bool runImpl() override; void setPageRange(size_t page_range); protected: - virtual DB::PageId genRandomPageId() override; + DB::PageId genRandomPageId() override; protected: size_t begin_page_id = 1; @@ -192,9 +194,9 @@ class PSReader : public PSRunnable gen.seed(time(nullptr)); } - virtual String description() override { return fmt::format("(Stress Test PSReader {})", index); } + String description() override { return fmt::format("(Stress Test PSReader {})", index); } - virtual bool runImpl() override; + bool runImpl() override; void setPageReadOnce(size_t page_read_once); @@ -242,7 +244,7 @@ class PSWindowReader : public PSReader void setWriterNums(size_t writer_nums); protected: - virtual DB::PageIds genRandomPageIds() override; + DB::PageIds genRandomPageIds() override; protected: size_t window_size = 100; @@ -261,12 +263,13 @@ class PSSnapshotReader : public PSReader : PSReader(ps_, index_) {} - virtual bool runImpl() override; + bool runImpl() override; void setSnapshotGetIntervalMs(size_t snapshot_get_interval_ms_); protected: - size_t snapshots_hold_num; + size_t snapshots_hold_num = 0; size_t snapshot_get_interval_ms = 0; std::list snapshots; -}; \ No newline at end of file +}; +} // namespace DB::PS::tests diff --git a/dbms/src/Storages/Page/stress/PSStressEnv.cpp b/dbms/src/Storages/Page/workload/PSStressEnv.cpp similarity index 97% rename from dbms/src/Storages/Page/stress/PSStressEnv.cpp rename to dbms/src/Storages/Page/workload/PSStressEnv.cpp index 7d680cd43c0..f5cead0a158 100644 --- a/dbms/src/Storages/Page/stress/PSStressEnv.cpp +++ b/dbms/src/Storages/Page/workload/PSStressEnv.cpp @@ -16,18 +16,20 @@ #include #include #include -#include -#include #include #include #include #include #include #include +#include +#include #include #include +namespace DB::PS::tests +{ Poco::Logger * StressEnv::logger; void StressEnv::initGlobalLogger() { @@ -146,3 +148,4 @@ void StressEnv::setup() init_pages = true; setupSignal(); } +} // namespace DB::PS::tests diff --git a/dbms/src/Storages/Page/stress/PSStressEnv.h b/dbms/src/Storages/Page/workload/PSStressEnv.h similarity index 98% rename from dbms/src/Storages/Page/stress/PSStressEnv.h rename to dbms/src/Storages/Page/workload/PSStressEnv.h index 1c7d8ee761f..e67cb325430 100644 --- a/dbms/src/Storages/Page/stress/PSStressEnv.h +++ b/dbms/src/Storages/Page/workload/PSStressEnv.h @@ -25,6 +25,8 @@ namespace Poco class Logger; } +namespace DB::PS::tests +{ using PSPtr = std::shared_ptr; enum StressEnvStat @@ -124,3 +126,4 @@ struct StressEnv void setup(); }; +} // namespace DB::PS::tests diff --git a/dbms/src/Storages/Page/stress/PSWorkload.cpp b/dbms/src/Storages/Page/workload/PSWorkload.cpp similarity index 98% rename from dbms/src/Storages/Page/stress/PSWorkload.cpp rename to dbms/src/Storages/Page/workload/PSWorkload.cpp index ce1f8d92ce0..81f13527f48 100644 --- a/dbms/src/Storages/Page/stress/PSWorkload.cpp +++ b/dbms/src/Storages/Page/workload/PSWorkload.cpp @@ -14,12 +14,14 @@ #include #include -#include #include #include #include +#include #include +namespace DB::PS::tests +{ void StressWorkload::onDumpResult() { UInt64 time_interval = stop_watch.elapsedMilliseconds(); @@ -177,3 +179,4 @@ void StressWorkloadManger::runWorkload() } } } +} // namespace DB::PS::tests diff --git a/dbms/src/Storages/Page/stress/PSWorkload.h b/dbms/src/Storages/Page/workload/PSWorkload.h similarity index 92% rename from dbms/src/Storages/Page/stress/PSWorkload.h rename to dbms/src/Storages/Page/workload/PSWorkload.h index cb099b4203a..eaaaf4eba5b 100644 --- a/dbms/src/Storages/Page/stress/PSWorkload.h +++ b/dbms/src/Storages/Page/workload/PSWorkload.h @@ -16,15 +16,17 @@ #include #include -#include -#include -#include #include #include #include +#include +#include +#include #include #define NORMAL_WORKLOAD 0 +namespace DB::PS::tests +{ template class StressWorkloadFunc { @@ -45,6 +47,8 @@ class StressWorkloadFunc class StressWorkload { public: + static int mainEntry(int argc, char ** argv); + explicit StressWorkload(StressEnv options_) : options(options_) {} @@ -189,13 +193,15 @@ class StressWorkloadManger StressEnv options; }; -#define REGISTER_WORKLOAD(WORKLOAD) \ - static void __attribute__((constructor)) _work_load_register_named_##WORKLOAD(void) \ - { \ - StressWorkloadManger::getInstance().reg( \ - WORKLOAD::nameFunc(), \ - WORKLOAD::maskFunc(), \ - [](const StressEnv & opts) -> std::shared_ptr { \ - return std::make_shared(opts); \ - }); \ +#define REGISTER_WORKLOAD(WORKLOAD) \ + void __attribute__((constructor)) _work_load_register_named_##WORKLOAD(void) \ + { \ + StressWorkloadManger::getInstance().reg( \ + WORKLOAD::nameFunc(), \ + WORKLOAD::maskFunc(), \ + [](const StressEnv & opts) -> std::shared_ptr { \ + return std::make_shared(opts); \ + }); \ } + +} // namespace DB::PS::tests diff --git a/dbms/src/Storages/Page/stress/workload/PageStorageInMemoryCapacity.cpp b/dbms/src/Storages/Page/workload/PageStorageInMemoryCapacity.cpp similarity index 96% rename from dbms/src/Storages/Page/stress/workload/PageStorageInMemoryCapacity.cpp rename to dbms/src/Storages/Page/workload/PageStorageInMemoryCapacity.cpp index 190cbf6b323..6ab321d1a10 100644 --- a/dbms/src/Storages/Page/stress/workload/PageStorageInMemoryCapacity.cpp +++ b/dbms/src/Storages/Page/workload/PageStorageInMemoryCapacity.cpp @@ -12,13 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include #include #include #include #include - #ifdef __APPLE__ #include @@ -27,6 +26,8 @@ #include #endif +namespace DB::PS::tests +{ class PageStorageInMemoryCapacity : public StressWorkload , public StressWorkloadFunc { @@ -89,14 +90,14 @@ class PageStorageInMemoryCapacity : public StressWorkload } FILE * file = fopen("/proc/meminfo", "r"); - if (file != NULL) + if (file != nullptr) { char buffer[128]; #define MEMORY_TOTAL_LABEL "MemTotal:" while (fgets(buffer, 128, file)) { if ((strncmp((buffer), (MEMORY_TOTAL_LABEL), strlen(MEMORY_TOTAL_LABEL)) == 0) - && sscanf(buffer + strlen(MEMORY_TOTAL_LABEL), " %32llu kB", &total_mem)) + && sscanf(buffer + strlen(MEMORY_TOTAL_LABEL), " %32llu kB", &total_mem)) // NOLINT { break; } @@ -174,4 +175,5 @@ class PageStorageInMemoryCapacity : public StressWorkload } }; -REGISTER_WORKLOAD(PageStorageInMemoryCapacity) \ No newline at end of file +REGISTER_WORKLOAD(PageStorageInMemoryCapacity) +} // namespace DB::PS::tests \ No newline at end of file diff --git a/dbms/src/Storages/Page/stress/workload/ThousandsOfOffset.cpp b/dbms/src/Storages/Page/workload/ThousandsOfOffset.cpp similarity index 97% rename from dbms/src/Storages/Page/stress/workload/ThousandsOfOffset.cpp rename to dbms/src/Storages/Page/workload/ThousandsOfOffset.cpp index 3a215f76769..5a02ef48d68 100644 --- a/dbms/src/Storages/Page/stress/workload/ThousandsOfOffset.cpp +++ b/dbms/src/Storages/Page/workload/ThousandsOfOffset.cpp @@ -12,8 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include +#include +namespace DB::PS::tests +{ class ThousandsOfOffset : public StressWorkload , public StressWorkloadFunc { @@ -168,4 +170,5 @@ class ThousandsOfOffset : public StressWorkload } }; -REGISTER_WORKLOAD(ThousandsOfOffset) \ No newline at end of file +REGISTER_WORKLOAD(ThousandsOfOffset) +} // namespace DB::PS::tests \ No newline at end of file From 1e3207d3794d8870fc8afad895017191851e4cca Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger <60744015+Lloyd-Pottiger@users.noreply.github.com> Date: Fri, 10 Jun 2022 12:32:30 +0800 Subject: [PATCH 029/104] feat: add hardware information of server as an attribute (#5090) close pingcap/tiflash#4879 --- dbms/src/Server/CMakeLists.txt | 1 + dbms/src/Server/Server.cpp | 30 +++-- dbms/src/Server/Server.h | 8 +- dbms/src/Server/ServerInfo.cpp | 199 +++++++++++++++++++++++++++++++++ dbms/src/Server/ServerInfo.h | 99 ++++++++++++++++ 5 files changed, 327 insertions(+), 10 deletions(-) create mode 100644 dbms/src/Server/ServerInfo.cpp create mode 100644 dbms/src/Server/ServerInfo.h diff --git a/dbms/src/Server/CMakeLists.txt b/dbms/src/Server/CMakeLists.txt index 104b4f34e4a..2948bb076db 100644 --- a/dbms/src/Server/CMakeLists.txt +++ b/dbms/src/Server/CMakeLists.txt @@ -34,6 +34,7 @@ add_library (clickhouse-server-lib NotFoundHandler.cpp PingRequestHandler.cpp RootRequestHandler.cpp + ServerInfo.cpp Server.cpp StatusFile.cpp TCPHandler.cpp diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 04676ef969d..3e2c29de76c 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include "Server.h" - #include #include #include @@ -56,6 +54,8 @@ #include #include #include +#include +#include #include #include #include @@ -72,7 +72,6 @@ #include #include #include -#include #include #include @@ -1049,6 +1048,23 @@ int Server::main(const std::vector & /*args*/) LOG_FMT_INFO(log, "tiflash proxy thread is joined"); }); + /// get CPU/memory/disk info of this server + if (tiflash_instance_wrap.proxy_helper) + { + diagnosticspb::ServerInfoRequest request; + request.set_tp(static_cast(1)); + diagnosticspb::ServerInfoResponse response; + std::string req = request.SerializeAsString(); + auto * helper = tiflash_instance_wrap.proxy_helper; + helper->fn_server_info(helper->proxy_ptr, strIntoView(&req), &response); + server_info.parseSysInfo(response); + LOG_FMT_INFO(log, "ServerInfo: {}", server_info.debugString()); + } + else + { + LOG_FMT_INFO(log, "TiFlashRaftProxyHelper is null, failed to get server info"); + } + CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::get()); // print necessary grpc log. @@ -1410,10 +1426,10 @@ int Server::main(const std::vector & /*args*/) // on ARM processors it can show only enabled at current moment cores LOG_FMT_INFO( log, - "Available RAM = {}; physical cores = {}; threads = {}.", - formatReadableSizeWithBinarySuffix(getMemoryAmount()), - getNumberOfPhysicalCPUCores(), - std::thread::hardware_concurrency()); + "Available RAM = {}; physical cores = {}; logical cores = {}.", + server_info.memory_info.capacity, + server_info.cpu_info.physical_cores, + server_info.cpu_info.logical_cores); } LOG_FMT_INFO(log, "Ready for connections."); diff --git a/dbms/src/Server/Server.h b/dbms/src/Server/Server.h index 278349f2aa4..07c5b955a92 100644 --- a/dbms/src/Server/Server.h +++ b/dbms/src/Server/Server.h @@ -14,10 +14,10 @@ #pragma once +#include +#include #include -#include "IServer.h" - /** Server provides three interfaces: * 1. HTTP - simple interface for any applications. * 2. TCP - interface for native clickhouse-client and for server to server internal communications. @@ -39,7 +39,7 @@ class Server : public BaseDaemon return BaseDaemon::config(); } - virtual const TiFlashSecurityConfig & securityConfig() const override { return security_config; }; + const TiFlashSecurityConfig & securityConfig() const override { return security_config; }; Poco::Logger & logger() const override { @@ -70,6 +70,8 @@ class Server : public BaseDaemon TiFlashSecurityConfig security_config; + ServerInfo server_info; + class FlashGrpcServerHolder; class TcpHttpServersHolder; }; diff --git a/dbms/src/Server/ServerInfo.cpp b/dbms/src/Server/ServerInfo.cpp new file mode 100644 index 00000000000..9cba40c4775 --- /dev/null +++ b/dbms/src/Server/ServerInfo.cpp @@ -0,0 +1,199 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +#include + +namespace DB +{ +using diagnosticspb::ServerInfoItem; +using diagnosticspb::ServerInfoResponse; + +void ServerInfo::parseCPUInfo(const diagnosticspb::ServerInfoItem & cpu_info_item) +{ + for (const auto & pair : cpu_info_item.pairs()) + { + const auto & key = pair.key(); + if (key == "cpu-logical-cores") + { + cpu_info.logical_cores = static_cast(std::stoi(pair.value())); + } + else if (key == "cpu-physical-cores") + { + cpu_info.physical_cores = static_cast(std::stoi(pair.value())); + } + else if (key == "cpu-frequency") + { + cpu_info.frequency = pair.value(); + } + else if (key == "l1-cache-size") + { + cpu_info.l1_cache_size = static_cast(std::stoull(pair.value())); + } + else if (key == "l1-cache-line-size") + { + cpu_info.l1_cache_line_size = static_cast(std::stoi(pair.value())); + } + else if (key == "l2-cache-size") + { + cpu_info.l2_cache_size = static_cast(std::stoull(pair.value())); + } + else if (key == "l2-cache-line-size") + { + cpu_info.l2_cache_line_size = static_cast(std::stoi(pair.value())); + } + else if (key == "l3-cache-size") + { + cpu_info.l3_cache_size = static_cast(std::stoull(pair.value())); + } + else if (key == "l3-cache-line-size") + { + cpu_info.l3_cache_line_size = static_cast(std::stoi(pair.value())); + } + else if (key == "cpu-arch") + { + cpu_info.arch = pair.value(); + } + } +} + +void ServerInfo::parseDiskInfo(const diagnosticspb::ServerInfoItem & disk_info_item) +{ + Disk disk; + disk.name = disk_info_item.name(); + for (const auto & pair : disk_info_item.pairs()) + { + const auto & key = pair.key(); + if (key == "type") + { + if (pair.value() == "HDD") + { + disk.disk_type = Disk::DiskType::HDD; + } + else if (pair.value() == "SSD") + { + disk.disk_type = Disk::DiskType::SSD; + } + else + { + disk.disk_type = Disk::DiskType::UNKNOWN; + } + } + else if (key == "total") + { + disk.total_space = static_cast(std::stoull(pair.value())); + } + else if (key == "free") + { + disk.free_space = static_cast(std::stoull(pair.value())); + } + else if (key == "path") + { + disk.mount_point = pair.value(); + } + else if (key == "fstype") + { + disk.fs_type = pair.value(); + } + } + disk_infos.push_back(disk); +} + +void ServerInfo::parseMemoryInfo(const diagnosticspb::ServerInfoItem & memory_info_item) +{ + for (const auto & pair : memory_info_item.pairs()) + { + if (pair.key() == "capacity") + { + memory_info.capacity = std::stoull(pair.value()); + } + } +} + +void ServerInfo::parseSysInfo(const diagnosticspb::ServerInfoResponse & sys_info_response) +{ + for (const auto & item : sys_info_response.items()) + { + const auto & tp = item.tp(); + if (tp == "cpu") + { + parseCPUInfo(item); + } + else if (tp == "disk") + { + parseDiskInfo(item); + } + else if (tp == "memory") + { + parseMemoryInfo(item); + } + } +} + +String ServerInfo::debugString() const +{ + FmtBuffer fmt_buf; + // append cpu info + fmt_buf.fmtAppend("CPU: \n" + " logical cores: {}\n" + " physical cores: {}\n" + " frequency: {}\n" + " l1 cache size: {}\n" + " l1 cache line size: {}\n" + " l2 cache size: {}\n" + " l2 cache line size: {}\n" + " l3 cache size: {}\n" + " l3 cache line size: {}\n" + " arch: {}\n", + cpu_info.logical_cores, + cpu_info.physical_cores, + cpu_info.frequency, + cpu_info.l1_cache_size, + cpu_info.l1_cache_line_size, + cpu_info.l2_cache_size, + cpu_info.l2_cache_line_size, + cpu_info.l3_cache_size, + cpu_info.l3_cache_line_size, + cpu_info.arch); + // append disk info + { + const static String disk_type_str[] = {"UNKNOWN", "HDD", "SSD"}; + for (const auto & disk_info : disk_infos) + { + fmt_buf.fmtAppend("Disk: \n" + " name: {}\n" + " type: {}\n" + " total space: {}\n" + " free space: {}\n" + " mount point: {}\n" + " fstype: {}\n", + disk_info.name, + disk_type_str[static_cast(disk_info.disk_type)], + disk_info.total_space, + disk_info.free_space, + disk_info.mount_point, + disk_info.fs_type); + } + } + // append memory info + fmt_buf.fmtAppend("Memory: \n" + " capacity: {}\n", + memory_info.capacity); + + return fmt_buf.toString(); +} + +} // namespace DB \ No newline at end of file diff --git a/dbms/src/Server/ServerInfo.h b/dbms/src/Server/ServerInfo.h new file mode 100644 index 00000000000..9663bd37568 --- /dev/null +++ b/dbms/src/Server/ServerInfo.h @@ -0,0 +1,99 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once +#include +#include +#include + +#include +#include + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" +#ifdef __clang__ +#pragma GCC diagnostic ignored "-Wdeprecated-declarations" +#endif +#include +#pragma GCC diagnostic pop + +namespace DB +{ +class ServerInfo +{ +public: + struct CPUInfo + { + /// number of logical CPU cores + UInt16 logical_cores = std::thread::hardware_concurrency(); + /// number of physical CPU cores + UInt16 physical_cores = getNumberOfPhysicalCPUCores(); + /// number of L1 cache size + /// units: Byte + UInt32 l1_cache_size = 16384; // 16KB (typical value) + /// number of L2 cache size + /// units: Byte + UInt32 l2_cache_size = 65536; // 64KB (typical value) + /// number of L3 cache size + /// units: Byte + UInt32 l3_cache_size = 2097152; // 2MB (typical value) + /// number of L1 cache line size + UInt8 l1_cache_line_size = 64; // 64B (typical value) + /// number of L2 cache line size + UInt8 l2_cache_line_size = 64; // 64B (typical value) + /// number of L3 cache line size + UInt8 l3_cache_line_size = 64; // 64B (typical value) + /// CPU architecture + String arch; + /// CPU frequency + String frequency; + }; + + struct Disk + { + String name; + enum DiskType + { + UNKNOWN = 0, + HDD = 1, + SSD = 2 + }; + DiskType disk_type; + UInt64 total_space = 0; + UInt64 free_space = 0; + String mount_point; + String fs_type; + }; + using DiskInfo = std::vector; + + struct MemoryInfo + { + /// total memory size + /// units: Byte + UInt64 capacity = getMemoryAmount(); + }; + + ServerInfo() = default; + ~ServerInfo() = default; + void parseCPUInfo(const diagnosticspb::ServerInfoItem & cpu_info_item); + void parseDiskInfo(const diagnosticspb::ServerInfoItem & disk_info_item); + void parseMemoryInfo(const diagnosticspb::ServerInfoItem & memory_info_item); + void parseSysInfo(const diagnosticspb::ServerInfoResponse & sys_info_response); + String debugString() const; + + CPUInfo cpu_info; + DiskInfo disk_infos; + MemoryInfo memory_info; +}; +} // namespace DB From 8916afe6eb0fccea897df172473b6bb0a98acae0 Mon Sep 17 00:00:00 2001 From: xufei Date: Fri, 10 Jun 2022 17:28:31 +0800 Subject: [PATCH 030/104] move `tunnel_map` to MPPTunnelSet (#5123) ref pingcap/tiflash#5095 --- dbms/src/Flash/Mpp/MPPTask.cpp | 46 +++++++------------- dbms/src/Flash/Mpp/MPPTask.h | 3 -- dbms/src/Flash/Mpp/MPPTunnelSet.cpp | 65 +++++++++++++++++++++++++++++ dbms/src/Flash/Mpp/MPPTunnelSet.h | 21 ++++++---- 4 files changed, 92 insertions(+), 43 deletions(-) diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 0f18ad582b4..8f9ca8e55e5 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -56,6 +56,7 @@ MPPTask::MPPTask(const mpp::TaskMeta & meta_, const ContextPtr & context_) , id(meta.start_ts(), meta.task_id()) , log(Logger::get("MPPTask", id.toString())) , mpp_task_statistics(id, meta.address()) + , needed_threads(0) , schedule_state(ScheduleState::WAITING) {} @@ -78,18 +79,14 @@ MPPTask::~MPPTask() void MPPTask::closeAllTunnels(const String & reason) { - for (auto & it : tunnel_map) - { - it.second->close(reason); - } + if (likely(tunnel_set)) + tunnel_set->close(reason); } void MPPTask::finishWrite() { - for (const auto & it : tunnel_map) - { - it.second->writeDone(); - } + RUNTIME_ASSERT(tunnel_set != nullptr, log, "mpp task without tunnel set"); + tunnel_set->finishWrite(); } void MPPTask::run() @@ -97,15 +94,13 @@ void MPPTask::run() newThreadManager()->scheduleThenDetach(true, "MPPTask", [self = shared_from_this()] { self->runImpl(); }); } -void MPPTask::registerTunnel(const MPPTaskId & id, MPPTunnelPtr tunnel) +void MPPTask::registerTunnel(const MPPTaskId & task_id, MPPTunnelPtr tunnel) { if (status == CANCELLED) throw Exception("the tunnel " + tunnel->id() + " can not been registered, because the task is cancelled"); - if (tunnel_map.find(id) != tunnel_map.end()) - throw Exception("the tunnel " + tunnel->id() + " has been registered"); - - tunnel_map[id] = tunnel; + RUNTIME_ASSERT(tunnel_set != nullptr, log, "mpp task without tunnel set"); + tunnel_set->registerTunnel(task_id, tunnel); } std::pair MPPTask::getTunnel(const ::mpp::EstablishMPPConnectionRequest * request) @@ -120,8 +115,9 @@ std::pair MPPTask::getTunnel(const ::mpp::EstablishMPPConn } MPPTaskId receiver_id{request->receiver_meta().start_ts(), request->receiver_meta().task_id()}; - auto it = tunnel_map.find(receiver_id); - if (it == tunnel_map.end()) + RUNTIME_ASSERT(tunnel_set != nullptr, log, "mpp task without tunnel set"); + auto tunnel_ptr = tunnel_set->getTunnelById(receiver_id); + if (tunnel_ptr == nullptr) { auto err_msg = fmt::format( "can't find tunnel ({} + {})", @@ -129,7 +125,7 @@ std::pair MPPTask::getTunnel(const ::mpp::EstablishMPPConn request->receiver_meta().task_id()); return {nullptr, err_msg}; } - return {it->second, ""}; + return {tunnel_ptr, ""}; } void MPPTask::unregisterTask() @@ -211,7 +207,7 @@ void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) } // register tunnels - tunnel_set = std::make_shared(); + tunnel_set = std::make_shared(log->identifier()); std::chrono::seconds timeout(task_request.timeout()); for (int i = 0; i < exchange_sender.encoded_task_meta_size(); i++) @@ -225,7 +221,6 @@ void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) MPPTunnelPtr tunnel = std::make_shared(task_meta, task_request.meta(), timeout, context->getSettingsRef().max_threads, is_local, is_async, log->identifier()); LOG_FMT_DEBUG(log, "begin to register the tunnel {}", tunnel->id()); registerTunnel(MPPTaskId{task_meta.start_ts(), task_meta.task_id()}, tunnel); - tunnel_set->addTunnel(tunnel); if (!dag_context->isRootMPPTask()) { FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_mpp_register_tunnel_for_non_root_mpp_task); @@ -369,19 +364,8 @@ void MPPTask::runImpl() void MPPTask::writeErrToAllTunnels(const String & e) { - for (auto & it : tunnel_map) - { - try - { - FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_mpp_write_err_to_tunnel); - it.second->write(getPacketWithError(e), true); - } - catch (...) - { - it.second->close("Failed to write error msg to tunnel"); - tryLogCurrentException(log, "Failed to write error " + e + " to tunnel: " + it.second->id()); - } - } + RUNTIME_ASSERT(tunnel_set != nullptr, log, "mpp task without tunnel set"); + tunnel_set->writeError(e); } void MPPTask::cancel(const String & reason) diff --git a/dbms/src/Flash/Mpp/MPPTask.h b/dbms/src/Flash/Mpp/MPPTask.h index c34cae49699..ee434a2f2ff 100644 --- a/dbms/src/Flash/Mpp/MPPTask.h +++ b/dbms/src/Flash/Mpp/MPPTask.h @@ -123,9 +123,6 @@ class MPPTask : public std::enable_shared_from_this MPPTunnelSetPtr tunnel_set; - // which targeted task we should send data by which tunnel. - std::unordered_map tunnel_map; - MPPTaskManager * manager = nullptr; const LoggerPtr log; diff --git a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp index 12de07d4a18..500e9501b08 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp @@ -13,11 +13,17 @@ // limitations under the License. #include +#include #include +#include #include namespace DB { +namespace FailPoints +{ +extern const char exception_during_mpp_write_err_to_tunnel[]; +} // namespace FailPoints namespace { inline mpp::MPPDataPacket serializeToPacket(const tipb::SelectResponse & response) @@ -108,6 +114,65 @@ void MPPTunnelSetBase::write(mpp::MPPDataPacket & packet, int16_t partit tunnels[partition_id]->write(packet); } +template +void MPPTunnelSetBase::writeError(const String & msg) +{ + for (auto & tunnel : tunnels) + { + try + { + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_mpp_write_err_to_tunnel); + tunnel->write(getPacketWithError(msg), true); + } + catch (...) + { + tunnel->close("Failed to write error msg to tunnel"); + tryLogCurrentException(log, "Failed to write error " + msg + " to tunnel: " + tunnel->id()); + } + } +} + +template +void MPPTunnelSetBase::registerTunnel(const MPPTaskId & id, const TunnelPtr & tunnel) +{ + if (id_to_index_map.find(id) != id_to_index_map.end()) + throw Exception("the tunnel " + tunnel->id() + " has been registered"); + + id_to_index_map[id] = tunnels.size(); + tunnels.push_back(tunnel); + if (!tunnel->isLocal()) + { + remote_tunnel_cnt++; + } +} + +template +void MPPTunnelSetBase::close(const String & reason) +{ + for (auto & tunnel : tunnels) + tunnel->close(reason); +} + +template +void MPPTunnelSetBase::finishWrite() +{ + for (auto & tunnel : tunnels) + { + tunnel->writeDone(); + } +} + +template +typename MPPTunnelSetBase::TunnelPtr MPPTunnelSetBase::getTunnelById(const MPPTaskId & id) +{ + auto it = id_to_index_map.find(id); + if (it == id_to_index_map.end()) + { + return nullptr; + } + return tunnels[it->second]; +} + /// Explicit template instantiations - to avoid code bloat in headers. template class MPPTunnelSetBase; diff --git a/dbms/src/Flash/Mpp/MPPTunnelSet.h b/dbms/src/Flash/Mpp/MPPTunnelSet.h index f2279b945cb..021c609f516 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSet.h +++ b/dbms/src/Flash/Mpp/MPPTunnelSet.h @@ -14,6 +14,7 @@ #pragma once +#include #include #ifdef __clang__ #pragma clang diagnostic push @@ -32,6 +33,9 @@ class MPPTunnelSetBase : private boost::noncopyable { public: using TunnelPtr = std::shared_ptr; + explicit MPPTunnelSetBase(const String & req_id) + : log(Logger::get("MPPTunnelSet", req_id)) + {} void clearExecutionSummaries(tipb::SelectResponse & response); @@ -50,17 +54,14 @@ class MPPTunnelSetBase : private boost::noncopyable // this is a partition writing. void write(tipb::SelectResponse & response, int16_t partition_id); void write(mpp::MPPDataPacket & packet, int16_t partition_id); + void writeError(const String & msg); + void close(const String & reason); + void finishWrite(); + void registerTunnel(const MPPTaskId & id, const TunnelPtr & tunnel); - uint16_t getPartitionNum() const { return tunnels.size(); } + TunnelPtr getTunnelById(const MPPTaskId & id); - void addTunnel(const TunnelPtr & tunnel) - { - tunnels.push_back(tunnel); - if (!tunnel->isLocal()) - { - remote_tunnel_cnt++; - } - } + uint16_t getPartitionNum() const { return tunnels.size(); } int getRemoteTunnelCnt() { @@ -71,6 +72,8 @@ class MPPTunnelSetBase : private boost::noncopyable private: std::vector tunnels; + std::unordered_map id_to_index_map; + const LoggerPtr log; int remote_tunnel_cnt = 0; }; From 8a81342c104a78c9d84b2877c111a4cb87e3cdff Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger <60744015+Lloyd-Pottiger@users.noreply.github.com> Date: Fri, 10 Jun 2022 21:42:30 +0800 Subject: [PATCH 031/104] add some notes about `getMemoryAmount()` and `getNumberOfPhysicalCPUCores()` (#5126) close pingcap/tiflash#5125 --- dbms/src/Common/getNumberOfPhysicalCPUCores.h | 1 + dbms/src/Storages/DeltaMerge/workload/Handle.h | 6 ++++-- libs/libcommon/include/common/getMemoryAmount.h | 1 + 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/dbms/src/Common/getNumberOfPhysicalCPUCores.h b/dbms/src/Common/getNumberOfPhysicalCPUCores.h index 6f7eaef4bb4..b3ab65a66e5 100644 --- a/dbms/src/Common/getNumberOfPhysicalCPUCores.h +++ b/dbms/src/Common/getNumberOfPhysicalCPUCores.h @@ -15,4 +15,5 @@ #pragma once /// Get number of CPU cores without hyper-threading. +/// Note: do not support environment under resource isolation mechanism like Docker, CGroup. unsigned getNumberOfPhysicalCPUCores(); diff --git a/dbms/src/Storages/DeltaMerge/workload/Handle.h b/dbms/src/Storages/DeltaMerge/workload/Handle.h index c4949c15a1f..2bbd1bd409d 100644 --- a/dbms/src/Storages/DeltaMerge/workload/Handle.h +++ b/dbms/src/Storages/DeltaMerge/workload/Handle.h @@ -51,14 +51,16 @@ class HandleLock std::vector> getLocks(const std::vector & handles) { - std::vector indexes(handles.size()); + std::vector indexes; + indexes.reserve(handles.size()); for (const auto & h : handles) { indexes.push_back(index(h)); } // Sort mutex indexes to avoid dead lock. sort(indexes.begin(), indexes.end()); - std::vector> locks(indexes.size()); + std::vector> locks; + locks.reserve(indexes.size()); for (auto i : indexes) { locks.push_back(getLockByIndex(i)); diff --git a/libs/libcommon/include/common/getMemoryAmount.h b/libs/libcommon/include/common/getMemoryAmount.h index 98aa87661c3..0807c6f8e12 100644 --- a/libs/libcommon/include/common/getMemoryAmount.h +++ b/libs/libcommon/include/common/getMemoryAmount.h @@ -19,5 +19,6 @@ /** * Returns the size of physical memory (RAM) in bytes. * Returns 0 on unsupported platform +* Note: do not support environment under resource isolation mechanism like Docker, CGroup. */ uint64_t getMemoryAmount(); From 10bcb06bcd379601014c70b6dd9173bb960b3c32 Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Mon, 13 Jun 2022 12:52:32 +0800 Subject: [PATCH 032/104] *: remove TableFunctionFile (#5098) ref pingcap/tiflash#2019 --- dbms/src/Storages/StorageFile.cpp | 348 ------------------ dbms/src/Storages/StorageFile.h | 100 ----- dbms/src/Storages/registerStorages.cpp | 2 - dbms/src/TableFunctions/TableFunctionFile.cpp | 98 ----- dbms/src/TableFunctions/TableFunctionFile.h | 39 -- .../TableFunctions/registerTableFunctions.cpp | 2 - 6 files changed, 589 deletions(-) delete mode 100644 dbms/src/Storages/StorageFile.cpp delete mode 100644 dbms/src/Storages/StorageFile.h delete mode 100644 dbms/src/TableFunctions/TableFunctionFile.cpp delete mode 100644 dbms/src/TableFunctions/TableFunctionFile.h diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp deleted file mode 100644 index 4dec4fd5ea0..00000000000 --- a/dbms/src/Storages/StorageFile.cpp +++ /dev/null @@ -1,348 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include -#include - -#include -#include - -#include -#include - -#include -#include -#include - -#include -#include -#include - -#include -#include - -#include - -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_WRITE_TO_FILE_DESCRIPTOR; - extern const int CANNOT_SEEK_THROUGH_FILE; - extern const int DATABASE_ACCESS_DENIED; - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int UNKNOWN_IDENTIFIER; - extern const int INCORRECT_FILE_NAME; - extern const int FILE_DOESNT_EXIST; - extern const int EMPTY_LIST_OF_COLUMNS_PASSED; -}; - - -static std::string getTablePath(const std::string & db_dir_path, const std::string & table_name, const std::string & format_name) -{ - return db_dir_path + escapeForFileName(table_name) + "/data." + escapeForFileName(format_name); -} - -/// Both db_dir_path and table_path must be converted to absolute paths (in particular, path cannot contain '..'). -static void checkCreationIsAllowed(Context & context_global, const std::string & db_dir_path, const std::string & table_path, int table_fd) -{ - if (context_global.getApplicationType() != Context::ApplicationType::SERVER) - return; - - if (table_fd >= 0) - throw Exception("Using file descriptor as source of storage isn't allowed for server daemons", ErrorCodes::DATABASE_ACCESS_DENIED); - else if (!startsWith(table_path, db_dir_path)) - throw Exception("Part path " + table_path + " is not inside " + db_dir_path, ErrorCodes::DATABASE_ACCESS_DENIED); - - Poco::File table_path_poco_file = Poco::File(table_path); - if (!table_path_poco_file.exists()) - throw Exception("File " + table_path + " is not exist", ErrorCodes::FILE_DOESNT_EXIST); - else if (table_path_poco_file.isDirectory()) - throw Exception("File " + table_path + " must not be a directory", ErrorCodes::INCORRECT_FILE_NAME); -} - - -StorageFile::StorageFile( - const std::string & table_path_, - int table_fd_, - const std::string & db_dir_path, - const std::string & table_name_, - const std::string & format_name_, - const ColumnsDescription & columns_, - Context & context_) - : IStorage(columns_), - table_name(table_name_), format_name(format_name_), context_global(context_), table_fd(table_fd_) -{ - if (table_fd < 0) /// Will use file - { - use_table_fd = false; - - if (!table_path_.empty()) /// Is user's file - { - Poco::Path poco_path = Poco::Path(table_path_); - if (poco_path.isRelative()) - poco_path = Poco::Path(db_dir_path, poco_path); - - path = poco_path.absolute().toString(); - checkCreationIsAllowed(context_global, db_dir_path, path, table_fd); - is_db_table = false; - } - else /// Is DB's file - { - if (db_dir_path.empty()) - throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); - - path = getTablePath(db_dir_path, table_name, format_name); - is_db_table = true; - Poco::File(Poco::Path(path).parent()).createDirectories(); - } - } - else /// Will use FD - { - checkCreationIsAllowed(context_global, db_dir_path, path, table_fd); - - is_db_table = false; - use_table_fd = true; - - /// Save initial offset, it will be used for repeating SELECTs - /// If FD isn't seekable (lseek returns -1), then the second and subsequent SELECTs will fail. - table_fd_init_offset = lseek(table_fd, 0, SEEK_CUR); - } -} - - -class StorageFileBlockInputStream : public IProfilingBlockInputStream -{ -public: - StorageFileBlockInputStream(StorageFile & storage_, const Context & context, size_t max_block_size) - : storage(storage_) - { - if (storage.use_table_fd) - { - storage.rwlock.lock(); - - /// We could use common ReadBuffer and WriteBuffer in storage to leverage cache - /// and add ability to seek unseekable files, but cache sync isn't supported. - - if (storage.table_fd_was_used) /// We need seek to initial position - { - if (storage.table_fd_init_offset < 0) - throw Exception("File descriptor isn't seekable, inside " + storage.getName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE); - - /// ReadBuffer's seek() doesn't make sence, since cache is empty - if (lseek(storage.table_fd, storage.table_fd_init_offset, SEEK_SET) < 0) - throwFromErrno("Cannot seek file descriptor, inside " + storage.getName(), ErrorCodes::CANNOT_SEEK_THROUGH_FILE); - } - - storage.table_fd_was_used = true; - read_buf = std::make_unique(storage.table_fd); - } - else - { - storage.rwlock.lock_shared(); - - read_buf = std::make_unique(storage.path); - } - - reader = FormatFactory().getInput(storage.format_name, *read_buf, storage.getSampleBlock(), context, max_block_size); - } - - ~StorageFileBlockInputStream() override - { - if (storage.use_table_fd) - storage.rwlock.unlock(); - else - storage.rwlock.unlock_shared(); - } - - String getName() const override - { - return storage.getName(); - } - - Block readImpl() override - { - return reader->read(); - } - - Block getHeader() const override { return reader->getHeader(); }; - - void readPrefixImpl() override - { - reader->readPrefix(); - } - - void readSuffixImpl() override - { - reader->readSuffix(); - } - -private: - StorageFile & storage; - Block sample_block; - std::unique_ptr read_buf; - BlockInputStreamPtr reader; -}; - - -BlockInputStreams StorageFile::read( - const Names & /*column_names*/, - const SelectQueryInfo & /*query_info*/, - const Context & context, - QueryProcessingStage::Enum & /*processed_stage*/, - size_t max_block_size, - unsigned /*num_streams*/) -{ - return BlockInputStreams(1, std::make_shared(*this, context, max_block_size)); -} - - -class StorageFileBlockOutputStream : public IBlockOutputStream -{ -public: - explicit StorageFileBlockOutputStream(StorageFile & storage_) - : storage(storage_), lock(storage.rwlock) - { - if (storage.use_table_fd) - { - /** NOTE: Using real file binded to FD may be misleading: - * SELECT *; INSERT insert_data; SELECT *; last SELECT returns initil_fd_data + insert_data - * INSERT data; SELECT *; last SELECT returns only insert_data - */ - storage.table_fd_was_used = true; - write_buf = std::make_unique(storage.table_fd); - } - else - { - write_buf = std::make_unique(storage.path, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); - } - - writer = FormatFactory().getOutput(storage.format_name, *write_buf, storage.getSampleBlock(), storage.context_global); - } - - Block getHeader() const override { return storage.getSampleBlock(); } - - void write(const Block & block) override - { - writer->write(block); - } - - void writePrefix() override - { - writer->writePrefix(); - } - - void writeSuffix() override - { - writer->writeSuffix(); - } - - void flush() override - { - writer->flush(); - } - -private: - StorageFile & storage; - std::unique_lock lock; - std::unique_ptr write_buf; - BlockOutputStreamPtr writer; -}; - -BlockOutputStreamPtr StorageFile::write( - const ASTPtr & /*query*/, - const Settings & /*settings*/) -{ - return std::make_shared(*this); -} - - -void StorageFile::drop() -{ - /// Extra actions are not required. -} - - -void StorageFile::rename(const String & new_path_to_db, const String & /*new_database_name*/, const String & new_table_name) -{ - if (!is_db_table) - throw Exception("Can't rename table '" + table_name + "' binded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED); - - std::unique_lock lock(rwlock); - - std::string path_new = getTablePath(new_path_to_db, new_table_name, format_name); - Poco::File(Poco::Path(path_new).parent()).createDirectories(); - Poco::File(path).renameTo(path_new); - - path = std::move(path_new); -} - - -void registerStorageFile(StorageFactory & factory) -{ - factory.registerStorage("File", [](const StorageFactory::Arguments & args) - { - ASTs & engine_args = args.engine_args; - - if (!(engine_args.size() == 1 || engine_args.size() == 2)) - throw Exception( - "Storage File requires 1 or 2 arguments: name of used format and source.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); - String format_name = static_cast(*engine_args[0]).value.safeGet(); - - int source_fd = -1; - String source_path; - if (engine_args.size() >= 2) - { - /// Will use FD if engine_args[1] is int literal or identifier with std* name - - if (const ASTIdentifier * identifier = typeid_cast(engine_args[1].get())) - { - if (identifier->name == "stdin") - source_fd = STDIN_FILENO; - else if (identifier->name == "stdout") - source_fd = STDOUT_FILENO; - else if (identifier->name == "stderr") - source_fd = STDERR_FILENO; - else - throw Exception("Unknown identifier '" + identifier->name + "' in second arg of File storage constructor", - ErrorCodes::UNKNOWN_IDENTIFIER); - } - else if (const ASTLiteral * literal = typeid_cast(engine_args[1].get())) - { - auto type = literal->value.getType(); - if (type == Field::Types::Int64) - source_fd = static_cast(literal->value.get()); - else if (type == Field::Types::UInt64) - source_fd = static_cast(literal->value.get()); - else if (type == Field::Types::String) - source_path = literal->value.get(); - } - } - - return StorageFile::create( - source_path, source_fd, - args.data_path, - args.table_name, format_name, args.columns, - args.context); - }); -} - -} diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h deleted file mode 100644 index ca46f7f366e..00000000000 --- a/dbms/src/Storages/StorageFile.h +++ /dev/null @@ -1,100 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#pragma once - -#include -#include -#include -#include - -#include -#include -#include - - -namespace DB -{ -class StorageFileBlockInputStream; -class StorageFileBlockOutputStream; - -class StorageFile : public ext::SharedPtrHelper - , public IStorage -{ -public: - std::string getName() const override - { - return "File"; - } - - std::string getTableName() const override - { - return table_name; - } - - BlockInputStreams read( - const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned num_streams) override; - - BlockOutputStreamPtr write( - const ASTPtr & query, - const Settings & settings) override; - - void drop() override; - - void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; - - String getDataPath() const override { return path; } - -protected: - friend class StorageFileBlockInputStream; - friend class StorageFileBlockOutputStream; - - /** there are three options (ordered by priority): - - use specified file descriptor if (fd >= 0) - - use specified table_path if it isn't empty - - create own table inside data/db/table/ - */ - StorageFile( - const std::string & table_path_, - int table_fd_, - const std::string & db_dir_path, - const std::string & table_name_, - const std::string & format_name_, - const ColumnsDescription & columns_, - Context & context_); - -private: - std::string table_name; - std::string format_name; - Context & context_global; - - std::string path; - int table_fd = -1; - - bool is_db_table = true; /// Table is stored in real database, not user's file - bool use_table_fd = false; /// Use table_fd insted of path - std::atomic table_fd_was_used{false}; /// To detect repeating reads from stdin - off_t table_fd_init_offset = -1; /// Initial position of fd, used for repeating reads - - mutable std::shared_mutex rwlock; - - Poco::Logger * log = &Poco::Logger::get("StorageFile"); -}; - -} // namespace DB diff --git a/dbms/src/Storages/registerStorages.cpp b/dbms/src/Storages/registerStorages.cpp index a709be0b017..ddf815316ab 100644 --- a/dbms/src/Storages/registerStorages.cpp +++ b/dbms/src/Storages/registerStorages.cpp @@ -27,7 +27,6 @@ void registerStorageNull(StorageFactory & factory); void registerStorageMerge(StorageFactory & factory); void registerStorageBuffer(StorageFactory & factory); void registerStorageMemory(StorageFactory & factory); -void registerStorageFile(StorageFactory & factory); void registerStorageDictionary(StorageFactory & factory); void registerStorageSet(StorageFactory & factory); void registerStorageJoin(StorageFactory & factory); @@ -47,7 +46,6 @@ void registerStorages() registerStorageMerge(factory); registerStorageBuffer(factory); registerStorageMemory(factory); - registerStorageFile(factory); registerStorageDictionary(factory); registerStorageSet(factory); registerStorageJoin(factory); diff --git a/dbms/src/TableFunctions/TableFunctionFile.cpp b/dbms/src/TableFunctions/TableFunctionFile.cpp deleted file mode 100644 index 0ff1a5b443f..00000000000 --- a/dbms/src/TableFunctions/TableFunctionFile.cpp +++ /dev/null @@ -1,98 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#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 DATABASE_ACCESS_DENIED; -} // namespace ErrorCodes - -StoragePtr TableFunctionFile::executeImpl(const ASTPtr & ast_function, const Context & context) const -{ - // Parse args - ASTs & args_func = typeid_cast(*ast_function).children; - - if (args_func.size() != 1) - throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::LOGICAL_ERROR); - - ASTs & args = typeid_cast(*args_func.at(0)).children; - - if (args.size() != 3) - throw Exception("Table function '" + getName() + "' requires exactly 3 arguments: path, format and structure.", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - for (size_t i = 0; i < 3; ++i) - args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(args[i], context); - - std::string path = static_cast(*args[0]).value.safeGet(); - std::string format = static_cast(*args[1]).value.safeGet(); - std::string structure = static_cast(*args[2]).value.safeGet(); - - // Create sample block - std::vector structure_vals; - boost::split(structure_vals, structure, boost::algorithm::is_any_of(" ,"), boost::algorithm::token_compress_on); - - if (structure_vals.size() % 2 != 0) - throw Exception("Odd number of elements in section structure: must be a list of name type pairs", ErrorCodes::LOGICAL_ERROR); - - Block sample_block; - const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); - - for (size_t i = 0, size = structure_vals.size(); i < size; i += 2) - { - ColumnWithTypeAndName column; - column.name = structure_vals[i]; - column.type = data_type_factory.get(structure_vals[i + 1]); - column.column = column.type->createColumn(); - sample_block.insert(std::move(column)); - } - - // Create table - StoragePtr storage = StorageFile::create( - path, - -1, - context.getUserFilesPath(), - getName(), - format, - ColumnsDescription{sample_block.getNamesAndTypesList()}, - const_cast(context)); - - storage->startup(); - - return storage; -} - - -void registerTableFunctionFile(TableFunctionFactory & factory) -{ - factory.registerFunction(); -} - -} // namespace DB diff --git a/dbms/src/TableFunctions/TableFunctionFile.h b/dbms/src/TableFunctions/TableFunctionFile.h deleted file mode 100644 index dda367c2679..00000000000 --- a/dbms/src/TableFunctions/TableFunctionFile.h +++ /dev/null @@ -1,39 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#pragma once - -#include - - -namespace DB -{ -/* file(path, format, structure) - creates a temporary storage from file - * - * - * The file must be in the clickhouse data directory. - * The relative path begins with the clickhouse data directory. - */ -class TableFunctionFile : public ITableFunction -{ -public: - static constexpr auto name = "file"; - std::string getName() const override { return name; } - -private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context) const override; -}; - - -} // namespace DB diff --git a/dbms/src/TableFunctions/registerTableFunctions.cpp b/dbms/src/TableFunctions/registerTableFunctions.cpp index 2eac0ce0548..bfe219eec62 100644 --- a/dbms/src/TableFunctions/registerTableFunctions.cpp +++ b/dbms/src/TableFunctions/registerTableFunctions.cpp @@ -22,7 +22,6 @@ namespace DB void registerTableFunctionMerge(TableFunctionFactory & factory); void registerTableFunctionNumbers(TableFunctionFactory & factory); void registerTableFunctionCatBoostPool(TableFunctionFactory & factory); -void registerTableFunctionFile(TableFunctionFactory & factory); void registerTableFunctions() { auto & factory = TableFunctionFactory::instance(); @@ -30,7 +29,6 @@ void registerTableFunctions() registerTableFunctionMerge(factory); registerTableFunctionNumbers(factory); registerTableFunctionCatBoostPool(factory); - registerTableFunctionFile(factory); } } // namespace DB From d54fcc7723606ea92ffa787b0d06ae408f303fb5 Mon Sep 17 00:00:00 2001 From: Grace Cai Date: Mon, 13 Jun 2022 14:44:33 +0800 Subject: [PATCH 033/104] README.md: update URL of TiDB Cloud free trial (#5133) close pingcap/tiflash#4936, ref pingcap/tidb#35316 --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index 8a2217b9a42..02af727105b 100644 --- a/README.md +++ b/README.md @@ -12,7 +12,7 @@ TiFlash repository is based on [ClickHouse](https://github.com/ClickHouse/ClickH ### Start with TiDB Cloud -Quickly explore TiFlash with [a free trial of TiDB Cloud](https://tidbcloud.com/signup). +Quickly explore TiFlash with [a free trial of TiDB Cloud](https://tidbcloud.com/free-trial). See [TiDB Cloud Quick Start Guide](https://docs.pingcap.com/tidbcloud/tidb-cloud-quickstart). From 9c8a58884b3f61afd17a4c1a132dbd78495f3037 Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Mon, 13 Jun 2022 15:46:32 +0800 Subject: [PATCH 034/104] *: remove CatBoost (#5131) ref pingcap/tiflash#2019 --- dbms/src/Storages/StorageCatBoostPool.cpp | 287 ------------------ dbms/src/Storages/StorageCatBoostPool.h | 100 ------ .../TableFunctionCatBoostPool.cpp | 68 ----- .../TableFunctionCatBoostPool.h | 35 --- .../TableFunctions/registerTableFunctions.cpp | 3 +- 5 files changed, 1 insertion(+), 492 deletions(-) delete mode 100644 dbms/src/Storages/StorageCatBoostPool.cpp delete mode 100644 dbms/src/Storages/StorageCatBoostPool.h delete mode 100644 dbms/src/TableFunctions/TableFunctionCatBoostPool.cpp delete mode 100644 dbms/src/TableFunctions/TableFunctionCatBoostPool.h diff --git a/dbms/src/Storages/StorageCatBoostPool.cpp b/dbms/src/Storages/StorageCatBoostPool.cpp deleted file mode 100644 index 317cac21d52..00000000000 --- a/dbms/src/Storages/StorageCatBoostPool.cpp +++ /dev/null @@ -1,287 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_OPEN_FILE; - extern const int CANNOT_PARSE_TEXT; - extern const int DATABASE_ACCESS_DENIED; -} - -namespace -{ -class CatBoostDatasetBlockInputStream : public IProfilingBlockInputStream -{ -public: - - CatBoostDatasetBlockInputStream(const std::string & file_name, const std::string & format_name, - const Block & sample_block, const Context & context, size_t max_block_size) - : file_name(file_name), format_name(format_name) - { - read_buf = std::make_unique(file_name); - reader = FormatFactory().getInput(format_name, *read_buf, sample_block, context, max_block_size); - } - - String getName() const override - { - return "CatBoostDataset"; - } - - Block readImpl() override - { - return reader->read(); - } - - void readPrefixImpl() override - { - reader->readPrefix(); - } - - void readSuffixImpl() override - { - reader->readSuffix(); - } - - Block getHeader() const override { return sample_block; }; - -private: - Block sample_block; - std::unique_ptr read_buf; - BlockInputStreamPtr reader; - std::string file_name; - std::string format_name; -}; - -} - -static boost::filesystem::path canonicalPath(std::string && path) -{ - return boost::filesystem::canonical(boost::filesystem::path(path)); -} - -static std::string resolvePath(const boost::filesystem::path & base_path, std::string && path) -{ - boost::filesystem::path resolved_path(path); - if (!resolved_path.is_absolute()) - return (base_path / resolved_path).string(); - return resolved_path.string(); -} - -static void checkCreationIsAllowed(const String & base_path, const String & path) -{ - if (base_path != path.substr(0, base_path.size())) - throw Exception( - "Using file descriptor or user specified path as source of storage isn't allowed for server daemons", - ErrorCodes::DATABASE_ACCESS_DENIED); -} - - -StorageCatBoostPool::StorageCatBoostPool(const Context & context, - String column_description_file_name_, - String data_description_file_name_) - : column_description_file_name(std::move(column_description_file_name_)), - data_description_file_name(std::move(data_description_file_name_)) -{ - auto base_path = canonicalPath(context.getPath()); - column_description_file_name = resolvePath(base_path, std::move(column_description_file_name)); - data_description_file_name = resolvePath(base_path, std::move(data_description_file_name)); - if (context.getApplicationType() == Context::ApplicationType::SERVER) - { - const auto & base_path_str = base_path.string(); - checkCreationIsAllowed(base_path_str, column_description_file_name); - checkCreationIsAllowed(base_path_str, data_description_file_name); - } - - parseColumnDescription(); - createSampleBlockAndColumns(); -} - -std::string StorageCatBoostPool::getColumnTypesString(const ColumnTypesMap & columnTypesMap) -{ - std::string types_string; - bool first = true; - for (const auto & value : columnTypesMap) - { - if (!first) - types_string.append(", "); - - first = false; - types_string += value.first; - } - - return types_string; -} - -void StorageCatBoostPool::checkDatasetDescription() -{ - std::ifstream in(data_description_file_name); - if (!in.good()) - throw Exception("Cannot open file: " + data_description_file_name, ErrorCodes::CANNOT_OPEN_FILE); - - std::string line; - if (!std::getline(in, line)) - throw Exception("File is empty: " + data_description_file_name, ErrorCodes::CANNOT_PARSE_TEXT); - - size_t columns_count = 1; - for (char sym : line) - if (sym == '\t') - ++columns_count; - - columns_description.resize(columns_count); -} - -void StorageCatBoostPool::parseColumnDescription() -{ - /// NOTE: simple parsing - /// TODO: use ReadBufferFromFile - - checkDatasetDescription(); - - std::ifstream in(column_description_file_name); - if (!in.good()) - throw Exception("Cannot open file: " + column_description_file_name, ErrorCodes::CANNOT_OPEN_FILE); - - std::string line; - size_t line_num = 0; - auto column_types_map = getColumnTypesMap(); - auto column_types_string = getColumnTypesString(column_types_map); - - /// Enumerate default names for columns as Auxiliary, Auxiliary1, Auxiliary2, ... - std::map columns_per_type_count; - - while (std::getline(in, line)) - { - ++line_num; - std::string str_line_num = std::to_string(line_num); - - if (line.empty()) - continue; - - std::istringstream iss(line); - std::vector tokens; - std::string token; - while (std::getline(iss, token, '\t')) - tokens.push_back(token); - - if (tokens.size() != 2 && tokens.size() != 3) - throw Exception("Cannot parse column description at line " + str_line_num + " '" + line + "' " - + ": expected 2 or 3 columns, got " + std::to_string(tokens.size()), - ErrorCodes::CANNOT_PARSE_TEXT); - - std::string str_id = tokens[0]; - std::string col_type = tokens[1]; - std::string col_alias = tokens.size() > 2 ? tokens[2] : ""; - - size_t num_id; - try - { - num_id = std::stoull(str_id); - } - catch (std::exception & e) - { - throw Exception("Cannot parse column index at row " + str_line_num + ": " + e.what(), - ErrorCodes::CANNOT_PARSE_TEXT); - } - - if (num_id >= columns_description.size()) - throw Exception("Invalid index at row " + str_line_num + ": " + str_id - + ", expected in range [0, " + std::to_string(columns_description.size()) + ")", - ErrorCodes::CANNOT_PARSE_TEXT); - - if (column_types_map.count(col_type) == 0) - throw Exception("Invalid column type: " + col_type + ", expected: " + column_types_string, - ErrorCodes::CANNOT_PARSE_TEXT); - - auto type = column_types_map[col_type]; - - std::string col_name; - - bool is_feature_column = type == DatasetColumnType::Num || type == DatasetColumnType::Categ; - auto & col_number = columns_per_type_count[type]; - /// If column is not feature skip '0' after the name (to use 'Target' instead of 'Target0'). - col_name = col_type + (is_feature_column || col_number ? std::to_string(col_number) : ""); - ++col_number; - - columns_description[num_id] = ColumnDescription(col_name, col_alias, type); - } -} - -void StorageCatBoostPool::createSampleBlockAndColumns() -{ - ColumnsDescription columns; - NamesAndTypesList cat_columns; - NamesAndTypesList num_columns; - sample_block.clear(); - for (auto & desc : columns_description) - { - DataTypePtr type; - if (desc.column_type == DatasetColumnType::Categ - || desc.column_type == DatasetColumnType::Auxiliary - || desc.column_type == DatasetColumnType::DocId) - type = std::make_shared(); - else - type = std::make_shared(); - - if (desc.column_type == DatasetColumnType::Categ) - cat_columns.emplace_back(desc.column_name, type); - else if (desc.column_type == DatasetColumnType::Num) - num_columns.emplace_back(desc.column_name, type); - else - columns.materialized.emplace_back(desc.column_name, type); - - if (!desc.alias.empty()) - { - auto alias = std::make_shared(desc.column_name); - columns.defaults[desc.alias] = {ColumnDefaultKind::Alias, alias}; - columns.aliases.emplace_back(desc.alias, type); - } - - sample_block.insert(ColumnWithTypeAndName(type, desc.column_name)); - } - columns.ordinary.insert(columns.ordinary.end(), num_columns.begin(), num_columns.end()); - columns.ordinary.insert(columns.ordinary.end(), cat_columns.begin(), cat_columns.end()); - - setColumns(columns); -} - -BlockInputStreams StorageCatBoostPool::read(const Names & column_names, - const SelectQueryInfo & /*query_info*/, - const Context & context, - QueryProcessingStage::Enum & /*processed_stage*/, - size_t max_block_size, - unsigned /*threads*/) -{ - auto stream = std::make_shared( - data_description_file_name, "TSV", sample_block, context, max_block_size); - - auto filter_stream = std::make_shared(stream, column_names, false); - return { filter_stream }; -} - -} diff --git a/dbms/src/Storages/StorageCatBoostPool.h b/dbms/src/Storages/StorageCatBoostPool.h deleted file mode 100644 index 0f4f7c2cede..00000000000 --- a/dbms/src/Storages/StorageCatBoostPool.h +++ /dev/null @@ -1,100 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#pragma once - -#include -#include - -#include - -namespace DB -{ -class StorageCatBoostPool : public ext::SharedPtrHelper - , public IStorage -{ -public: - std::string getName() const override { return "CatBoostPool"; } - - std::string getTableName() const override { return table_name; } - - BlockInputStreams read(const Names & column_names, - const SelectQueryInfo & query_info, - const Context & context, - QueryProcessingStage::Enum & processed_stage, - size_t max_block_size, - unsigned threads) override; - -private: - String table_name; - - String column_description_file_name; - String data_description_file_name; - Block sample_block; - - enum class DatasetColumnType - { - Target, - Num, - Categ, - Auxiliary, - DocId, - Weight, - Baseline - }; - - using ColumnTypesMap = std::map; - - ColumnTypesMap getColumnTypesMap() const - { - return { - {"Target", DatasetColumnType::Target}, - {"Num", DatasetColumnType::Num}, - {"Categ", DatasetColumnType::Categ}, - {"Auxiliary", DatasetColumnType::Auxiliary}, - {"DocId", DatasetColumnType::DocId}, - {"Weight", DatasetColumnType::Weight}, - {"Baseline", DatasetColumnType::Baseline}, - }; - }; - - std::string getColumnTypesString(const ColumnTypesMap & columnTypesMap); - - struct ColumnDescription - { - std::string column_name; - std::string alias; - DatasetColumnType column_type; - - ColumnDescription() - : column_type(DatasetColumnType::Num) - {} - ColumnDescription(std::string column_name, std::string alias, DatasetColumnType column_type) - : column_name(std::move(column_name)) - , alias(std::move(alias)) - , column_type(column_type) - {} - }; - - std::vector columns_description; - - void checkDatasetDescription(); - void parseColumnDescription(); - void createSampleBlockAndColumns(); - -protected: - StorageCatBoostPool(const Context & context, String column_description_file_name, String data_description_file_name); -}; - -} // namespace DB diff --git a/dbms/src/TableFunctions/TableFunctionCatBoostPool.cpp b/dbms/src/TableFunctions/TableFunctionCatBoostPool.cpp deleted file mode 100644 index ab5cd7e5849..00000000000 --- a/dbms/src/TableFunctions/TableFunctionCatBoostPool.cpp +++ /dev/null @@ -1,68 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include -#include -#include -#include -#include -#include - - -namespace DB -{ -namespace ErrorCodes -{ -extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; -extern const int BAD_ARGUMENTS; -} // namespace ErrorCodes - - -StoragePtr TableFunctionCatBoostPool::executeImpl(const ASTPtr & ast_function, const Context & context) const -{ - ASTs & args_func = typeid_cast(*ast_function).children; - - std::string err = "Table function '" + getName() + "' requires 2 parameters: " - + "column descriptions file, dataset description file"; - - if (args_func.size() != 1) - throw Exception(err, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - ASTs & args = typeid_cast(*args_func.at(0)).children; - - if (args.size() != 2) - throw Exception(err, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - auto getStringLiteral = [](const IAST & node, const char * description) { - auto lit = typeid_cast(&node); - if (!lit) - throw Exception(description + String(" must be string literal (in single quotes)."), ErrorCodes::BAD_ARGUMENTS); - - if (lit->value.getType() != Field::Types::String) - throw Exception(description + String(" must be string literal (in single quotes)."), ErrorCodes::BAD_ARGUMENTS); - - return safeGet(lit->value); - }; - String column_descriptions_file = getStringLiteral(*args[0], "Column descriptions file"); - String dataset_description_file = getStringLiteral(*args[1], "Dataset description file"); - - return StorageCatBoostPool::create(context, column_descriptions_file, dataset_description_file); -} - -void registerTableFunctionCatBoostPool(TableFunctionFactory & factory) -{ - factory.registerFunction(); -} - -} // namespace DB diff --git a/dbms/src/TableFunctions/TableFunctionCatBoostPool.h b/dbms/src/TableFunctions/TableFunctionCatBoostPool.h deleted file mode 100644 index 0b5b32dfffe..00000000000 --- a/dbms/src/TableFunctions/TableFunctionCatBoostPool.h +++ /dev/null @@ -1,35 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#pragma once - -#include - - -namespace DB -{ -/* catboostPool('column_descriptions_file', 'dataset_description_file') - * Create storage from CatBoost dataset. - */ -class TableFunctionCatBoostPool : public ITableFunction -{ -public: - static constexpr auto name = "catBoostPool"; - std::string getName() const override { return name; } - -private: - StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context) const override; -}; - -} // namespace DB diff --git a/dbms/src/TableFunctions/registerTableFunctions.cpp b/dbms/src/TableFunctions/registerTableFunctions.cpp index bfe219eec62..8449077cc96 100644 --- a/dbms/src/TableFunctions/registerTableFunctions.cpp +++ b/dbms/src/TableFunctions/registerTableFunctions.cpp @@ -21,14 +21,13 @@ namespace DB { void registerTableFunctionMerge(TableFunctionFactory & factory); void registerTableFunctionNumbers(TableFunctionFactory & factory); -void registerTableFunctionCatBoostPool(TableFunctionFactory & factory); + void registerTableFunctions() { auto & factory = TableFunctionFactory::instance(); registerTableFunctionMerge(factory); registerTableFunctionNumbers(factory); - registerTableFunctionCatBoostPool(factory); } } // namespace DB From f4c2e012fc3067193412aca91c6b26e30173c722 Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Mon, 13 Jun 2022 16:16:35 +0800 Subject: [PATCH 035/104] Test: Mock window function, refactor window function tests (#5021) ref pingcap/tiflash#4609, close pingcap/tiflash#5081 --- .../DataStreams/WindowBlockInputStream.cpp | 18 + dbms/src/DataStreams/WindowBlockInputStream.h | 2 + dbms/src/Debug/astToExecutor.cpp | 199 +++++++- dbms/src/Debug/astToExecutor.h | 56 ++- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 4 +- .../Coprocessor/DAGQueryBlockInterpreter.h | 2 - dbms/src/Flash/Coprocessor/DAGUtils.cpp | 5 +- .../Coprocessor/collectOutputFieldTypes.cpp | 26 +- dbms/src/Flash/tests/gtest_interpreter.cpp | 80 ++++ dbms/src/Interpreters/WindowDescription.cpp | 36 +- dbms/src/Interpreters/WindowDescription.h | 4 + dbms/src/TestUtils/ExecutorTestUtils.cpp | 34 +- dbms/src/TestUtils/ExecutorTestUtils.h | 57 +-- dbms/src/TestUtils/FunctionTestUtils.cpp | 32 ++ dbms/src/TestUtils/FunctionTestUtils.h | 27 ++ dbms/src/TestUtils/executorSerializer.cpp | 66 ++- dbms/src/TestUtils/mockExecutor.cpp | 56 ++- dbms/src/TestUtils/mockExecutor.h | 19 +- .../TestUtils/tests/gtest_mock_executors.cpp | 12 + .../tests/gtest_window_functions.cpp | 436 ++++++------------ 20 files changed, 823 insertions(+), 348 deletions(-) diff --git a/dbms/src/DataStreams/WindowBlockInputStream.cpp b/dbms/src/DataStreams/WindowBlockInputStream.cpp index bc63db52873..2cc61df8104 100644 --- a/dbms/src/DataStreams/WindowBlockInputStream.cpp +++ b/dbms/src/DataStreams/WindowBlockInputStream.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include namespace DB @@ -574,4 +575,21 @@ void WindowBlockInputStream::tryCalculate() peer_group_number = 1; } } + +void WindowBlockInputStream::appendInfo(FmtBuffer & buffer) const +{ + buffer.append(", function: {"); + buffer.joinStr( + window_description.window_functions_descriptions.begin(), + window_description.window_functions_descriptions.end(), + [&](const auto & func, FmtBuffer & b) { + b.append(func.window_function->getName()); + }, + ", "); + buffer.fmtAppend( + "}}, frame: {{type: {}, boundary_begin: {}, boundary_end: {}}}", + frameTypeToString(window_description.frame.type), + boundaryTypeToString(window_description.frame.begin_type), + boundaryTypeToString(window_description.frame.end_type)); +} } // namespace DB diff --git a/dbms/src/DataStreams/WindowBlockInputStream.h b/dbms/src/DataStreams/WindowBlockInputStream.h index 46b18dec1ee..0ef23aa9f6f 100644 --- a/dbms/src/DataStreams/WindowBlockInputStream.h +++ b/dbms/src/DataStreams/WindowBlockInputStream.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include #include @@ -169,6 +170,7 @@ class WindowBlockInputStream : public IProfilingBlockInputStream protected: Block readImpl() override; + void appendInfo(FmtBuffer & buffer) const override; LoggerPtr log; diff --git a/dbms/src/Debug/astToExecutor.cpp b/dbms/src/Debug/astToExecutor.cpp index 82f894905e6..fec76d7a085 100644 --- a/dbms/src/Debug/astToExecutor.cpp +++ b/dbms/src/Debug/astToExecutor.cpp @@ -24,13 +24,13 @@ #include #include #include -#include #include #include #include namespace DB { +using ASTPartitionByElement = ASTOrderByElement; void literalFieldToTiPBExpr(const ColumnInfo & ci, const Field & val_field, tipb::Expr * expr, Int32 collator_id) { *(expr->mutable_field_type()) = columnInfoToFieldType(ci); @@ -190,6 +190,12 @@ std::unordered_map agg_func_name_to_sig({ {"group_concat", tipb::ExprType::GroupConcat}, }); +std::unordered_map window_func_name_to_sig({ + {"RowNumber", tipb::ExprType::RowNumber}, + {"Rank", tipb::ExprType::Rank}, + {"DenseRank", tipb::ExprType::DenseRank}, +}); + DAGColumnInfo toNullableDAGColumnInfo(const DAGColumnInfo & input) { DAGColumnInfo output = input; @@ -1343,6 +1349,105 @@ void Join::toMPPSubPlan(size_t & executor_index, const DAGProperties & propertie exchange_map[left_exchange_receiver->name] = std::make_pair(left_exchange_receiver, left_exchange_sender); exchange_map[right_exchange_receiver->name] = std::make_pair(right_exchange_receiver, right_exchange_sender); } + +bool Window::toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t collator_id, const MPPInfo & mpp_info, const Context & context) +{ + tipb_executor->set_tp(tipb::ExecType::TypeWindow); + tipb_executor->set_executor_id(name); + tipb::Window * window = tipb_executor->mutable_window(); + auto & input_schema = children[0]->output_schema; + for (const auto & expr : func_descs) + { + tipb::Expr * window_expr = window->add_func_desc(); + const auto * window_func = typeid_cast(expr.get()); + for (const auto & arg : window_func->arguments->children) + { + tipb::Expr * func = window_expr->add_children(); + astToPB(input_schema, arg, func, collator_id, context); + } + auto window_sig_it = window_func_name_to_sig.find(window_func->name); + if (window_sig_it == window_func_name_to_sig.end()) + throw Exception(fmt::format("Unsupported window function {}", window_func->name), ErrorCodes::LOGICAL_ERROR); + auto window_sig = window_sig_it->second; + window_expr->set_tp(window_sig); + auto * ft = window_expr->mutable_field_type(); + // TODO: Maybe more window functions with different field type. + ft->set_tp(TiDB::TypeLongLong); + ft->set_flag(TiDB::ColumnFlagBinary); + ft->set_collate(collator_id); + ft->set_flen(21); + ft->set_decimal(-1); + } + + for (const auto & child : order_by_exprs) + { + auto * elem = typeid_cast(child.get()); + if (!elem) + throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); + tipb::ByItem * by = window->add_order_by(); + by->set_desc(elem->direction < 0); + tipb::Expr * expr = by->mutable_expr(); + astToPB(children[0]->output_schema, elem->children[0], expr, collator_id, context); + } + + for (const auto & child : partition_by_exprs) + { + auto * elem = typeid_cast(child.get()); + if (!elem) + throw Exception("Invalid partition by element", ErrorCodes::LOGICAL_ERROR); + tipb::ByItem * by = window->add_partition_by(); + by->set_desc(elem->direction < 0); + tipb::Expr * expr = by->mutable_expr(); + astToPB(children[0]->output_schema, elem->children[0], expr, collator_id, context); + } + + if (frame.type.has_value()) + { + tipb::WindowFrame * mut_frame = window->mutable_frame(); + mut_frame->set_type(frame.type.value()); + if (frame.start.has_value()) + { + auto * start = mut_frame->mutable_start(); + start->set_offset(std::get<2>(frame.start.value())); + start->set_unbounded(std::get<1>(frame.start.value())); + start->set_type(std::get<0>(frame.start.value())); + } + + if (frame.end.has_value()) + { + auto * end = mut_frame->mutable_end(); + end->set_offset(std::get<2>(frame.end.value())); + end->set_unbounded(std::get<1>(frame.end.value())); + end->set_type(std::get<0>(frame.end.value())); + } + } + + auto * children_executor = window->mutable_child(); + return children[0]->toTiPBExecutor(children_executor, collator_id, mpp_info, context); +} + +bool Sort::toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t collator_id, const MPPInfo & mpp_info, const Context & context) +{ + tipb_executor->set_tp(tipb::ExecType::TypeSort); + tipb_executor->set_executor_id(name); + tipb::Sort * sort = tipb_executor->mutable_sort(); + sort->set_ispartialsort(is_partial_sort); + + for (const auto & child : by_exprs) + { + auto * elem = typeid_cast(child.get()); + if (!elem) + throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); + tipb::ByItem * by = sort->add_byitems(); + by->set_desc(elem->direction < 0); + tipb::Expr * expr = by->mutable_expr(); + astToPB(children[0]->output_schema, elem->children[0], expr, collator_id, context); + } + + auto * children_executor = sort->mutable_child(); + return children[0]->toTiPBExecutor(children_executor, collator_id, mpp_info, context); +} + } // namespace mock ExecutorPtr compileTableScan(size_t & executor_index, TableInfo & table_info, String & table_alias, bool append_pk_column) @@ -1561,11 +1666,101 @@ ExecutorPtr compileExchangeSender(ExecutorPtr input, size_t & executor_index, ti return exchange_sender; } - ExecutorPtr compileExchangeReceiver(size_t & executor_index, DAGSchema schema) { ExecutorPtr exchange_receiver = std::make_shared(executor_index, schema); return exchange_receiver; } +ExecutorPtr compileWindow(ExecutorPtr input, size_t & executor_index, ASTPtr func_desc_list, ASTPtr partition_by_expr_list, ASTPtr order_by_expr_list, mock::MockWindowFrame frame) +{ + std::vector partition_columns; + if (partition_by_expr_list != nullptr) + { + for (const auto & child : partition_by_expr_list->children) + { + auto * elem = typeid_cast(child.get()); + if (!elem) + throw Exception("Invalid partition by element", ErrorCodes::LOGICAL_ERROR); + partition_columns.push_back(child); + compileExpr(input->output_schema, elem->children[0]); + } + } + + std::vector order_columns; + if (order_by_expr_list != nullptr) + { + for (const auto & child : order_by_expr_list->children) + { + auto * elem = typeid_cast(child.get()); + if (!elem) + throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); + order_columns.push_back(child); + compileExpr(input->output_schema, elem->children[0]); + } + } + + DAGSchema output_schema; + output_schema.insert(output_schema.end(), input->output_schema.begin(), input->output_schema.end()); + + std::vector window_exprs; + if (func_desc_list != nullptr) + { + for (const auto & expr : func_desc_list->children) + { + const auto * func = typeid_cast(expr.get()); + window_exprs.push_back(expr); + std::vector children_ci; + for (const auto & arg : func->arguments->children) + { + children_ci.push_back(compileExpr(input->output_schema, arg)); + } + // TODO: add more window functions + TiDB::ColumnInfo ci; + switch (window_func_name_to_sig[func->name]) + { + case tipb::ExprType::RowNumber: + case tipb::ExprType::Rank: + case tipb::ExprType::DenseRank: + { + ci.tp = TiDB::TypeLongLong; + ci.flag = TiDB::ColumnFlagBinary; + break; + } + default: + throw Exception(fmt::format("Unsupported window function {}", func->name), ErrorCodes::LOGICAL_ERROR); + } + output_schema.emplace_back(std::make_pair(func->getColumnName(), ci)); + } + } + + ExecutorPtr window = std::make_shared( + executor_index, + output_schema, + window_exprs, + std::move(partition_columns), + std::move(order_columns), + frame); + window->children.push_back(input); + return window; +} + +ExecutorPtr compileSort(ExecutorPtr input, size_t & executor_index, ASTPtr order_by_expr_list, bool is_partial_sort) +{ + std::vector order_columns; + if (order_by_expr_list != nullptr) + { + for (const auto & child : order_by_expr_list->children) + { + auto * elem = typeid_cast(child.get()); + if (!elem) + throw Exception("Invalid order by element", ErrorCodes::LOGICAL_ERROR); + order_columns.push_back(child); + compileExpr(input->output_schema, elem->children[0]); + } + } + ExecutorPtr sort = std::make_shared(executor_index, input->output_schema, std::move(order_columns), is_partial_sort); + sort->children.push_back(input); + return sort; +} } // namespace DB \ No newline at end of file diff --git a/dbms/src/Debug/astToExecutor.h b/dbms/src/Debug/astToExecutor.h index 37d3f22b6e1..cbd2e5ade3a 100644 --- a/dbms/src/Debug/astToExecutor.h +++ b/dbms/src/Debug/astToExecutor.h @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -28,6 +29,8 @@ #include #include +#include + namespace DB { namespace ErrorCodes @@ -272,6 +275,54 @@ struct Join : Executor void toMPPSubPlan(size_t & executor_index, const DAGProperties & properties, std::unordered_map, std::shared_ptr>> & exchange_map) override; }; + +using MockWindowFrameBound = std::tuple; + +struct MockWindowFrame +{ + std::optional type; + std::optional start; + std::optional end; + // TODO: support calcFuncs +}; + +struct Window : Executor +{ + std::vector func_descs; + std::vector partition_by_exprs; + std::vector order_by_exprs; + MockWindowFrame frame; + + Window(size_t & index_, const DAGSchema & output_schema_, std::vector func_descs_, std::vector partition_by_exprs_, std::vector order_by_exprs_, MockWindowFrame frame_) + : Executor(index_, "window_" + std::to_string(index_), output_schema_) + , func_descs(std::move(func_descs_)) + , partition_by_exprs(std::move(partition_by_exprs_)) + , order_by_exprs(order_by_exprs_) + , frame(frame_) + { + } + // Currently only use Window Executor in Unit Test which don't call columnPrume. + // TODO: call columnPrune in unit test and further benchmark test to eliminate compute process. + void columnPrune(std::unordered_set &) override { throw Exception("Should not reach here"); } + bool toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; +}; + +struct Sort : Executor +{ + std::vector by_exprs; + bool is_partial_sort; + + Sort(size_t & index_, const DAGSchema & output_schema_, std::vector by_exprs_, bool is_partial_sort_) + : Executor(index_, "sort_" + std::to_string(index_), output_schema_) + , by_exprs(by_exprs_) + , is_partial_sort(is_partial_sort_) + { + } + // Currently only use Sort Executor in Unit Test which don't call columnPrume. + // TODO: call columnPrune in unit test and further benchmark test to eliminate compute process. + void columnPrune(std::unordered_set &) override { throw Exception("Should not reach here"); } + bool toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t collator_id, const MPPInfo & mpp_info, const Context & context) override; +}; } // namespace mock using ExecutorPtr = std::shared_ptr; @@ -294,8 +345,9 @@ ExecutorPtr compileExchangeSender(ExecutorPtr input, size_t & executor_index, ti ExecutorPtr compileExchangeReceiver(size_t & executor_index, DAGSchema schema); -void literalFieldToTiPBExpr(const ColumnInfo & ci, const Field & field, tipb::Expr * expr, Int32 collator_id); +ExecutorPtr compileWindow(ExecutorPtr input, size_t & executor_index, ASTPtr func_desc_list, ASTPtr partition_by_expr_list, ASTPtr order_by_expr_list, mock::MockWindowFrame frame); -//TODO: add compileWindow +ExecutorPtr compileSort(ExecutorPtr input, size_t & executor_index, ASTPtr order_by_expr_list, bool is_partial_sort); +void literalFieldToTiPBExpr(const ColumnInfo & ci, const Field & field, tipb::Expr * expr, Int32 collator_id); } // namespace DB \ No newline at end of file diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 3b7112af02d..9f201006a88 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -151,11 +151,9 @@ class DAGExpressionAnalyzer : private boost::noncopyable void appendCastAfterWindow( const ExpressionActionsPtr & actions, const tipb::Window & window, - const size_t window_columns_start_index); + size_t window_columns_start_index); -#ifndef DBMS_PUBLIC_GTEST private: -#endif NamesAndTypes buildOrderColumns( const ExpressionActionsPtr & actions, const ::google::protobuf::RepeatedPtrField & order_by); diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index e68c4f91cee..0b3b2db9623 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -54,9 +54,7 @@ class DAGQueryBlockInterpreter BlockInputStreams execute(); -#ifndef DBMS_PUBLIC_GTEST private: -#endif void executeImpl(DAGPipeline & pipeline); void handleMockTableScan(const TiDBTableScan & table_scan, DAGPipeline & pipeline); void handleTableScan(const TiDBTableScan & table_scan, DAGPipeline & pipeline); diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 87f58131c8c..bea26fe9f99 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -29,7 +29,6 @@ #include namespace DB { - const Int8 VAR_SIZE = 0; extern const String uniq_raw_res_name; @@ -770,6 +769,10 @@ const String & getFunctionName(const tipb::Expr & expr) { return getAggFunctionName(expr); } + else if (isWindowFunctionExpr(expr)) + { + return getWindowFunctionName(expr); + } else { auto it = scalar_func_map.find(expr.sig()); diff --git a/dbms/src/Flash/Coprocessor/collectOutputFieldTypes.cpp b/dbms/src/Flash/Coprocessor/collectOutputFieldTypes.cpp index b68279faa13..87744c553e0 100644 --- a/dbms/src/Flash/Coprocessor/collectOutputFieldTypes.cpp +++ b/dbms/src/Flash/Coprocessor/collectOutputFieldTypes.cpp @@ -45,19 +45,36 @@ bool collectForAgg(std::vector & output_field_types, const tipb { for (const auto & expr : agg.agg_func()) { - if (!exprHasValidFieldType(expr)) + if (unlikely(!exprHasValidFieldType(expr))) throw TiFlashException("Agg expression without valid field type", Errors::Coprocessor::BadRequest); output_field_types.push_back(expr.field_type()); } for (const auto & expr : agg.group_by()) { - if (!exprHasValidFieldType(expr)) + if (unlikely(!exprHasValidFieldType(expr))) throw TiFlashException("Group by expression without valid field type", Errors::Coprocessor::BadRequest); output_field_types.push_back(expr.field_type()); } return false; } +bool collectForExecutor(std::vector & output_field_types, const tipb::Executor & executor); +bool collectForWindow(std::vector & output_field_types, const tipb::Executor & executor) +{ + // collect output_field_types of child + getChildren(executor).forEach([&output_field_types](const tipb::Executor & child) { + traverseExecutorTree(child, [&output_field_types](const tipb::Executor & e) { return collectForExecutor(output_field_types, e); }); + }); + + for (const auto & expr : executor.window().func_desc()) + { + if (unlikely(!exprHasValidFieldType(expr))) + throw TiFlashException("Window expression without valid field type", Errors::Coprocessor::BadRequest); + output_field_types.push_back(expr.field_type()); + } + return false; +} + bool collectForReceiver(std::vector & output_field_types, const tipb::ExchangeReceiver & receiver) { for (const auto & field_type : receiver.field_types()) @@ -82,7 +99,6 @@ bool collectForTableScan(std::vector & output_field_types, cons return false; } -bool collectForExecutor(std::vector & output_field_types, const tipb::Executor & executor); bool collectForJoin(std::vector & output_field_types, const tipb::Executor & executor) { // collect output_field_types of children @@ -147,8 +163,8 @@ bool collectForExecutor(std::vector & output_field_types, const case tipb::ExecType::TypeWindow: // Window will only be pushed down in mpp mode. // In mpp mode, ExchangeSender or Sender will return output_field_types directly. - // If not in mpp mode, window executor type is invalid. - throw TiFlashException("Window executor type is invalid in non-mpp mode, should not reach here.", Errors::Coprocessor::Internal); + // If not in mpp mode or debug mode, window executor type is invalid. + return collectForWindow(output_field_types, executor); case tipb::ExecType::TypeExchangeReceiver: return collectForReceiver(output_field_types, executor.exchange_receiver()); case tipb::ExecType::TypeTableScan: diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp index a6bb8ff1702..ba7d8fd15ee 100644 --- a/dbms/src/Flash/tests/gtest_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_interpreter.cpp @@ -385,5 +385,85 @@ CreatingSets } CATCH +TEST_F(InterpreterExecuteTest, Window) +try +{ + auto request = context + .scan("test_db", "test_table") + .sort({{"s1", true}, {"s2", false}}, true) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame()) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Expression: + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table") + .sort({{"s1", true}, {"s2", false}}, true) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame()) + .project({"s1", "s2", "RowNumber()"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + Expression: + Expression: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Expression: + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .sort({{"s1", true}, {"s2", false}}, true) + .project({"s1", "s2", "s3"}) + .window(RowNumber(), {"s1", true}, {"s1", false}, buildDefaultRowsFrame()) + .project({"s1", "s2", "s3", "RowNumber()"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + Expression: + Expression: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Union: + Expression x 10: + Expression: + Expression: + SharedQuery: + Expression: + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + } // namespace tests } // namespace DB \ No newline at end of file diff --git a/dbms/src/Interpreters/WindowDescription.cpp b/dbms/src/Interpreters/WindowDescription.cpp index 2ab407bb18e..09d96411673 100644 --- a/dbms/src/Interpreters/WindowDescription.cpp +++ b/dbms/src/Interpreters/WindowDescription.cpp @@ -44,7 +44,7 @@ WindowFrame::FrameType getFrameTypeFromTipb(const tipb::WindowFrameType & type) return WindowFrame::FrameType::Groups; default: throw Exception(ErrorCodes::BAD_ARGUMENTS, - "Unknowed frame type {}", + "Unknown frame type {}", type); } } @@ -60,4 +60,38 @@ void WindowDescription::setWindowFrame(const tipb::WindowFrame & frame_) frame.end_preceding = (frame_.end().type() == tipb::WindowBoundType::Preceding); frame.is_default = false; } + +String frameTypeToString(const WindowFrame::FrameType & type) +{ + switch (type) + { + case WindowFrame::FrameType::Rows: + return "Rows"; + case WindowFrame::FrameType::Groups: + return "Groups"; + case WindowFrame::FrameType::Ranges: + return "Ranges"; + default: + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Unknown frame type {}", + type); + } +} + +String boundaryTypeToString(const WindowFrame::BoundaryType & type) +{ + switch (type) + { + case WindowFrame::BoundaryType::Unbounded: + return "Unbounded"; + case WindowFrame::BoundaryType::Current: + return "Current"; + case WindowFrame::BoundaryType::Offset: + return "Offset"; + default: + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Unknown boundary type {}", + type); + } +} } // namespace DB diff --git a/dbms/src/Interpreters/WindowDescription.h b/dbms/src/Interpreters/WindowDescription.h index cdcade1b750..a3c2bac5747 100644 --- a/dbms/src/Interpreters/WindowDescription.h +++ b/dbms/src/Interpreters/WindowDescription.h @@ -87,6 +87,10 @@ struct WindowFrame && other.end_preceding == end_preceding; } }; + +String frameTypeToString(const WindowFrame::FrameType & type); +String boundaryTypeToString(const WindowFrame::BoundaryType & type); + class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; struct WindowDescription diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index 67a21d12286..881ebaf88db 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -12,11 +12,15 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include #include #include + +#include + namespace DB::tests { DAGContext & ExecutorTest::getDAGContext() @@ -34,15 +38,20 @@ void ExecutorTest::initializeContext() void ExecutorTest::SetUpTestCase() { - try - { - DB::registerFunctions(); - DB::registerAggregateFunctions(); - } - catch (DB::Exception &) - { - // Maybe another test has already registered, ignore exception here. - } + auto register_func = [](std::function func) { + try + { + func(); + } + catch (DB::Exception &) + { + // Maybe another test has already registered, ignore exception here. + } + }; + + register_func(DB::registerFunctions); + register_func(DB::registerAggregateFunctions); + register_func(DB::registerWindowFunctions); } void ExecutorTest::initializeClientInfo() @@ -125,6 +134,13 @@ void ExecutorTest::executeStreams(const std::shared_ptr & requ executeStreams(request, context.executorIdColumnsMap(), expect_columns, concurrency); } +void ExecutorTest::executeStreamsWithSingleSource(const std::shared_ptr & request, const ColumnsWithTypeAndName & source_columns, const ColumnsWithTypeAndName & expect_columns, SourceType type, size_t concurrency) +{ + std::unordered_map source_columns_map; + source_columns_map[getSourceName(type)] = source_columns; + executeStreams(request, source_columns_map, expect_columns, concurrency); +} + void ExecutorTest::dagRequestEqual(const String & expected_string, const std::shared_ptr & actual) { ASSERT_EQ(Poco::trim(expected_string), Poco::trim(ExecutorSerializer().serialize(actual.get()))); diff --git a/dbms/src/TestUtils/ExecutorTestUtils.h b/dbms/src/TestUtils/ExecutorTestUtils.h index 977b46abbd2..87bb7115bed 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.h +++ b/dbms/src/TestUtils/ExecutorTestUtils.h @@ -15,15 +15,13 @@ #pragma once #include -#include -#include #include #include #include -#include -#include #include #include +#include + namespace DB::tests { void executeInterpreter(const std::shared_ptr & request, Context & context); @@ -52,6 +50,28 @@ class ExecutorTest : public ::testing::Test void executeInterpreter(const String & expected_string, const std::shared_ptr & request, size_t concurrency); + enum SourceType + { + TableScan, + ExchangeReceiver + }; + + // for single source query, the source executor name is ${type}_0 + static String getSourceName(SourceType type) + { + switch (type) + { + case TableScan: + return "table_scan_0"; + case ExchangeReceiver: + return "exchange_receiver_0"; + default: + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Unknown Executor Source type {}", + type); + } + } + void executeStreams( const std::shared_ptr & request, std::unordered_map & source_columns_map, @@ -62,29 +82,12 @@ class ExecutorTest : public ::testing::Test const ColumnsWithTypeAndName & expect_columns, size_t concurrency = 1); - template - ColumnWithTypeAndName toNullableVec(const std::vector::FieldType>> & v) - { - return createColumn>(v); - } - - template - ColumnWithTypeAndName toVec(const std::vector::FieldType> & v) - { - return createColumn(v); - } - - template - ColumnWithTypeAndName toNullableVec(String name, const std::vector::FieldType>> & v) - { - return createColumn>(v, name); - } - - template - ColumnWithTypeAndName toVec(String name, const std::vector::FieldType> & v) - { - return createColumn(v, name); - } + void executeStreamsWithSingleSource( + const std::shared_ptr & request, + const ColumnsWithTypeAndName & source_columns, + const ColumnsWithTypeAndName & expect_columns, + SourceType type = TableScan, + size_t concurrency = 1); protected: MockDAGRequestContext context; diff --git a/dbms/src/TestUtils/FunctionTestUtils.cpp b/dbms/src/TestUtils/FunctionTestUtils.cpp index dae07f7123b..637fbf51c00 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.cpp +++ b/dbms/src/TestUtils/FunctionTestUtils.cpp @@ -242,5 +242,37 @@ ColumnWithTypeAndName createOnlyNullColumn(size_t size, const String & name) return {std::move(col), data_type, name}; } +ColumnWithTypeAndName toDatetimeVec(String name, const std::vector & v, int fsp) +{ + std::vector::FieldType> vec; + vec.reserve(v.size()); + for (const auto & value_str : v) + { + Field value = parseMyDateTime(value_str, fsp); + vec.push_back(value.template safeGet()); + } + DataTypePtr data_type = std::make_shared(fsp); + return {makeColumn(data_type, vec), data_type, name, 0}; +} + +ColumnWithTypeAndName toNullableDatetimeVec(String name, const std::vector & v, int fsp) +{ + std::vector::FieldType>> vec; + vec.reserve(v.size()); + for (const auto & value_str : v) + { + if (!value_str.empty()) + { + Field value = parseMyDateTime(value_str, fsp); + vec.push_back(value.template safeGet()); + } + else + { + vec.push_back({}); + } + } + DataTypePtr data_type = makeNullable(std::make_shared(fsp)); + return {makeColumn>(data_type, vec), data_type, name, 0}; +} } // namespace tests } // namespace DB diff --git a/dbms/src/TestUtils/FunctionTestUtils.h b/dbms/src/TestUtils/FunctionTestUtils.h index 7704c69a89f..d6b7351df05 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.h +++ b/dbms/src/TestUtils/FunctionTestUtils.h @@ -654,6 +654,33 @@ ColumnWithTypeAndName createNullableColumn( return createNullableColumn(data_type_args, vec, null_map, name, 0); } +template +ColumnWithTypeAndName toNullableVec(const std::vector::FieldType>> & v) +{ + return createColumn>(v); +} + +template +ColumnWithTypeAndName toVec(const std::vector::FieldType> & v) +{ + return createColumn(v); +} + +template +ColumnWithTypeAndName toNullableVec(String name, const std::vector::FieldType>> & v) +{ + return createColumn>(v, name); +} + +template +ColumnWithTypeAndName toVec(String name, const std::vector::FieldType> & v) +{ + return createColumn(v, name); +} + +ColumnWithTypeAndName toDatetimeVec(String name, const std::vector & v, int fsp); + +ColumnWithTypeAndName toNullableDatetimeVec(String name, const std::vector & v, int fsp); class FunctionTest : public ::testing::Test { protected: diff --git a/dbms/src/TestUtils/executorSerializer.cpp b/dbms/src/TestUtils/executorSerializer.cpp index b8d2b039bd2..a0ae4b11270 100644 --- a/dbms/src/TestUtils/executorSerializer.cpp +++ b/dbms/src/TestUtils/executorSerializer.cpp @@ -204,6 +204,66 @@ void serializeExchangeReceiver(const String & executor_id, const tipb::ExchangeR buf.append("}\n"); } +void serializeWindow(const String & executor_id, const tipb::Window & window [[maybe_unused]], FmtBuffer & buf) +{ + buf.fmtAppend("{} | partition_by: {{", executor_id); + buf.joinStr( + window.partition_by().begin(), + window.partition_by().end(), + [&](const auto & partition_by, FmtBuffer & fb) { + fb.append("("); + serializeExpression(partition_by.expr(), buf); + fb.fmtAppend(", desc: {})", partition_by.desc()); + }, + ", "); + buf.append("}}, order_by: {"); + buf.joinStr( + window.order_by().begin(), + window.order_by().end(), + [&](const auto & order_by, FmtBuffer & fb) { + fb.append("("); + serializeExpression(order_by.expr(), buf); + fb.fmtAppend(", desc: {})", order_by.desc()); + }, + ", "); + buf.append("}, func_desc: {"); + buf.joinStr( + window.func_desc().begin(), + window.func_desc().end(), + [&](const auto & func, FmtBuffer &) { + serializeExpression(func, buf); + }, + ", "); + if (window.has_frame()) + { + buf.append("}, frame: {"); + if (window.frame().has_start()) + { + buf.fmtAppend("start<{}, {}, {}>", window.frame().start().type(), window.frame().start().unbounded(), window.frame().start().offset()); + } + if (window.frame().has_end()) + { + buf.fmtAppend(", end<{}, {}, {}>", window.frame().end().type(), window.frame().end().unbounded(), window.frame().end().offset()); + } + } + buf.append("}\n"); +} + +void serializeSort(const String & executor_id, const tipb::Sort & sort [[maybe_unused]], FmtBuffer & buf) +{ + buf.fmtAppend("{} | isPartialSort: {}, partition_by: {{", executor_id, sort.ispartialsort()); + buf.joinStr( + sort.byitems().begin(), + sort.byitems().end(), + [&](const auto & by, FmtBuffer & fb) { + fb.append("("); + serializeExpression(by.expr(), buf); + fb.fmtAppend(", desc: {})", by.desc()); + }, + ", "); + buf.append("}\n"); +} + void ExecutorSerializer::serialize(const tipb::Executor & root_executor, size_t level) { auto append_str = [&level, this](const tipb::Executor & executor) { @@ -248,9 +308,11 @@ void ExecutorSerializer::serialize(const tipb::Executor & root_executor, size_t serializeExchangeSender(executor.executor_id(), executor.exchange_sender(), buf); break; case tipb::ExecType::TypeSort: - throw TiFlashException("Sort executor is not supported", Errors::Coprocessor::Unimplemented); // todo support sort executor. + serializeSort(executor.executor_id(), executor.sort(), buf); + break; case tipb::ExecType::TypeWindow: - throw TiFlashException("Window executor is not supported", Errors::Coprocessor::Unimplemented); // todo support window executor. + serializeWindow(executor.executor_id(), executor.window(), buf); + break; default: throw TiFlashException("Should not reach here", Errors::Coprocessor::Internal); } diff --git a/dbms/src/TestUtils/mockExecutor.cpp b/dbms/src/TestUtils/mockExecutor.cpp index af939002cff..e1ccbdbb010 100644 --- a/dbms/src/TestUtils/mockExecutor.cpp +++ b/dbms/src/TestUtils/mockExecutor.cpp @@ -23,8 +23,6 @@ #include #include -#include - namespace DB::tests { ASTPtr buildColumn(const String & column_name) @@ -54,6 +52,15 @@ ASTPtr buildOrderByItemList(MockOrderByItems order_by_items) return exp_list; } +MockWindowFrame buildDefaultRowsFrame() +{ + MockWindowFrame frame; + frame.type = tipb::WindowFrameType::Rows; + frame.end = {tipb::WindowBoundType::CurrentRow, false, 0}; + frame.start = {tipb::WindowBoundType::CurrentRow, false, 0}; + return frame; +} + // a mock DAGRequest should prepare its time_zone, flags, encode_type and output_schema. void DAGRequestBuilder::initDAGRequest(tipb::DAGRequest & dag_request) { @@ -96,6 +103,9 @@ DAGRequestBuilder & DAGRequestBuilder::mockTable(const String & db, const String TiDB::ColumnInfo ret; ret.tp = column.second; ret.name = column.first; + // TODO: find a way to assign decimal field's flen. + if (ret.tp == TiDB::TP::TypeNewDecimal) + ret.flen = 65; ret.id = i++; table_info.columns.push_back(std::move(ret)); } @@ -276,6 +286,48 @@ DAGRequestBuilder & DAGRequestBuilder::buildAggregation(ASTPtr agg_funcs, ASTPtr return *this; } +DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByItem order_by, MockPartitionByItem partition_by, MockWindowFrame frame) +{ + assert(root); + auto window_func_list = std::make_shared(); + window_func_list->children.push_back(window_func); + root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemList({partition_by}), buildOrderByItemList({order_by}), frame); + return *this; +} + +DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByItems order_by_list, MockPartitionByItems partition_by_list, MockWindowFrame frame) +{ + assert(root); + auto window_func_list = std::make_shared(); + window_func_list->children.push_back(window_func); + root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemList(partition_by_list), buildOrderByItemList(order_by_list), frame); + return *this; +} + +DAGRequestBuilder & DAGRequestBuilder::window(MockAsts window_funcs, MockOrderByItems order_by_list, MockPartitionByItems partition_by_list, MockWindowFrame frame) +{ + assert(root); + auto window_func_list = std::make_shared(); + for (const auto & func : window_funcs) + window_func_list->children.push_back(func); + root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemList(partition_by_list), buildOrderByItemList(order_by_list), frame); + return *this; +} + +DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItem order_by, bool is_partial_sort) +{ + assert(root); + root = compileSort(root, getExecutorIndex(), buildOrderByItemList({order_by}), is_partial_sort); + return *this; +} + +DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItems order_by_list, bool is_partial_sort) +{ + assert(root); + root = compileSort(root, getExecutorIndex(), buildOrderByItemList(order_by_list), is_partial_sort); + return *this; +} + void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfoList & columnInfos) { std::vector v_column_info(columnInfos.size()); diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index 88d98158b74..d52b5ec674a 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -20,9 +20,6 @@ #include #include -#include -#include - namespace DB::tests { using MockColumnInfo = std::pair; @@ -31,8 +28,11 @@ using MockColumnInfoList = std::initializer_list; using MockTableName = std::pair; using MockOrderByItem = std::pair; using MockOrderByItems = std::initializer_list; +using MockPartitionByItem = std::pair; +using MockPartitionByItems = std::initializer_list; using MockColumnNames = std::initializer_list; using MockAsts = std::initializer_list; +using MockWindowFrame = mock::MockWindowFrame; class MockDAGRequestContext; @@ -96,6 +96,13 @@ class DAGRequestBuilder DAGRequestBuilder & aggregation(ASTPtr agg_func, ASTPtr group_by_expr); DAGRequestBuilder & aggregation(MockAsts agg_funcs, MockAsts group_by_exprs); + // window + DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItem order_by, MockPartitionByItem partition_by, MockWindowFrame frame); + DAGRequestBuilder & window(MockAsts window_funcs, MockOrderByItems order_by_list, MockPartitionByItems partition_by_list, MockWindowFrame frame); + DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItems order_by_list, MockPartitionByItems partition_by_list, MockWindowFrame frame); + DAGRequestBuilder & sort(MockOrderByItem order_by, bool is_partial_sort); + DAGRequestBuilder & sort(MockOrderByItems order_by_list, bool is_partial_sort); + private: void initDAGRequest(tipb::DAGRequest & dag_request); DAGRequestBuilder & buildAggregation(ASTPtr agg_funcs, ASTPtr group_by_exprs); @@ -164,6 +171,8 @@ ASTPtr buildLiteral(const Field & field); ASTPtr buildFunction(MockAsts exprs, const String & name); ASTPtr buildOrderByItemList(MockOrderByItems order_by_items); +MockWindowFrame buildDefaultRowsFrame(); + #define col(name) buildColumn((name)) #define lit(field) buildLiteral((field)) #define eq(expr1, expr2) makeASTFunction("equals", (expr1), (expr2)) @@ -174,5 +183,9 @@ ASTPtr buildOrderByItemList(MockOrderByItems order_by_items); #define Or(expr1, expr2) makeASTFunction("or", (expr1), (expr2)) #define NOT(expr) makeASTFunction("not", (expr1), (expr2)) #define Max(expr) makeASTFunction("max", expr) +/// Window functions +#define RowNumber() makeASTFunction("RowNumber") +#define Rank() makeASTFunction("Rank") +#define DenseRank() makeASTFunction("DenseRank") } // namespace DB::tests \ No newline at end of file diff --git a/dbms/src/TestUtils/tests/gtest_mock_executors.cpp b/dbms/src/TestUtils/tests/gtest_mock_executors.cpp index 214148fe47f..8bed0f2fc6c 100644 --- a/dbms/src/TestUtils/tests/gtest_mock_executors.cpp +++ b/dbms/src/TestUtils/tests/gtest_mock_executors.cpp @@ -252,5 +252,17 @@ try } CATCH +TEST_F(MockDAGRequestTest, MockWindow) +try +{ + auto request = context.scan("test_db", "test_table").sort({"s1", false}, true).window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame()).build(context); + { + String expected = "window_2 | partition_by: {(<1, String>, desc: false)}}, order_by: {(<0, String>, desc: true)}, func_desc: {row_number()}, frame: {start<2, false, 0>, end<2, false, 0>}\n" + " sort_1 | isPartialSort: true, partition_by: {(<0, String>, desc: false)}\n" + " table_scan_0 | {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + } +} +CATCH } // namespace tests } // namespace DB \ No newline at end of file diff --git a/dbms/src/WindowFunctions/tests/gtest_window_functions.cpp b/dbms/src/WindowFunctions/tests/gtest_window_functions.cpp index e4205f6f938..3addf73a642 100644 --- a/dbms/src/WindowFunctions/tests/gtest_window_functions.cpp +++ b/dbms/src/WindowFunctions/tests/gtest_window_functions.cpp @@ -12,334 +12,192 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include -#include -#include -#include -#include -#include -#include -#include +#include namespace DB::tests { -class WindowFunction : public DB::tests::FunctionTest +class WindowExecutorTestRunner : public DB::tests::ExecutorTest { -protected: - std::shared_ptr mock_interpreter; - - void SetUp() override - { - DB::tests::FunctionTest::SetUp(); - DB::registerWindowFunctions(); - } - - template - ColumnWithTypeAndName toNullableVec(String name, const std::vector::FieldType>> & v) - { - return createColumn>(v, name); - } - - template - ColumnWithTypeAndName toVec(String name, const std::vector::FieldType> & v) - { - return createColumn(v, name); - } - - template - static ColumnWithTypeAndName toConst(const T s) - { - return createConstColumn(1, s); - } - - static ColumnWithTypeAndName toDatetimeVec(String name, const std::vector & v, int fsp) - { - std::vector::FieldType> vec; - for (const auto & value_str : v) - { - Field value = parseMyDateTime(value_str, fsp); - vec.push_back(value.template safeGet()); - } - DataTypePtr data_type = std::make_shared(fsp); - return {makeColumn(data_type, vec), data_type, name, 0}; - } - - static ColumnWithTypeAndName toNullableDatetimeVec(String name, const std::vector & v, int fsp) - { - std::vector::FieldType>> vec; - for (const auto & value_str : v) - { - if (!value_str.empty()) - { - Field value = parseMyDateTime(value_str, fsp); - vec.push_back(value.template safeGet()); - } - else - { - vec.push_back({}); - } - } - DataTypePtr data_type = makeNullable(std::make_shared(fsp)); - return {makeColumn>(data_type, vec), data_type, name, 0}; - } - - void setMaxBlockSize(int size) - { - context.getSettingsRef().max_block_size.set(size); - } - - void mockInterpreter(std::vector source_columns, Context context) - { - std::vector mock_input_streams_vec = {}; - DAGQueryBlock mock_query_block(0, static_cast>(nullptr)); - std::vector mock_subqueries_for_sets = {}; - mock_interpreter = std::make_shared(context, - mock_input_streams_vec, - mock_query_block, - 1); - - mock_interpreter->analyzer = std::make_unique(std::move(source_columns), context); - } - - void mockExecuteTableScan(DAGPipeline & pipeline, ColumnsWithTypeAndName columns) - { - pipeline.streams.push_back(std::make_shared(columns, context.getSettingsRef().max_block_size)); - mock_interpreter->input_streams_vec.push_back(pipeline.streams); - } - - void mockExecuteWindowOrder(DAGPipeline & pipeline, std::string sort_json_str) +public: + void initializeContext() override { - tipb::Sort sort; - google::protobuf::util::JsonStringToMessage(sort_json_str, &sort); - mock_interpreter->handleWindowOrder(pipeline, sort); - mock_interpreter->input_streams_vec[0] = pipeline.streams; - NamesWithAliases final_project; - for (const auto & column : (*mock_interpreter->analyzer).source_columns) - { - final_project.push_back({column.name, ""}); - } - mockExecuteProject(pipeline, final_project); - } - - void mockExecuteWindow(DAGPipeline & pipeline, std::string window_json_str) - { - tipb::Window window; - google::protobuf::util::JsonStringToMessage(window_json_str, &window); - mock_interpreter->handleWindow(pipeline, window); - mock_interpreter->input_streams_vec[0] = pipeline.streams; - NamesWithAliases final_project; - for (const auto & column : (*mock_interpreter->analyzer).source_columns) - { - final_project.push_back({column.name, ""}); - } - mockExecuteProject(pipeline, final_project); - } - - void mockExecuteProject(DAGPipeline & pipeline, NamesWithAliases & final_project) - { - mock_interpreter->executeProject(pipeline, final_project); - } - - static Block mergeBlocks(Blocks blocks) - { - if (blocks.empty()) - { - return {}; - } - Block sample_block; - std::vector actual_cols; - - for (const auto & block : blocks) - { - if (!sample_block) - { - sample_block = block; - for (const auto & column : block.getColumnsWithTypeAndName()) - { - actual_cols.push_back(column.type->createColumn()); - } - } - - for (size_t i = 0; i < block.columns(); ++i) - { - for (size_t j = 0; j < block.rows(); ++j) - { - actual_cols[i]->insert((*(block.getColumnsWithTypeAndName())[i].column)[j]); - } - } - } - - ColumnsWithTypeAndName actual_columns; - - for (size_t i = 0; i < actual_cols.size(); ++i) - { - actual_columns.push_back({std::move(actual_cols[i]), sample_block.getColumnsWithTypeAndName()[i].type, sample_block.getColumnsWithTypeAndName()[i].name, sample_block.getColumnsWithTypeAndName()[i].column_id}); - } - return Block(actual_columns); - } - - void testOneWindowFunction(const std::vector & source_column_types, const ColumnsWithTypeAndName & source_columns, const ColumnsWithTypeAndName & expect_columns, const std::string window_json_str, const std::string sort_json_str) - { - mockInterpreter(source_column_types, context); - DAGPipeline pipeline; - ExpressionActionsChain chain; - Block except_block(expect_columns); - - mockExecuteTableScan(pipeline, source_columns); - - mockExecuteWindowOrder(pipeline, sort_json_str); - - mockExecuteWindow(pipeline, window_json_str); - - auto stream = pipeline.firstStream(); - - Blocks actual_blocks; - while (Block block = stream->read()) - { - actual_blocks.push_back(block); - } - - Block actual_block = mergeBlocks(actual_blocks); - - if (actual_block) - { - // Check that input columns is properly split to many blocks - ASSERT_EQ(actual_blocks.size(), (actual_block.rows() - 1) / context.getSettingsRef().max_block_size + 1); - } - ASSERT_BLOCK_EQ(except_block, actual_block); + ExecutorTest::initializeContext(); + context.addMockTable( + {"test_db", "test_table"}, + {{"partition", TiDB::TP::TypeLongLong}, {"order", TiDB::TP::TypeLongLong}}, + {toVec("partition", {1, 1, 1, 1, 2, 2, 2, 2}), + toVec("order", {1, 1, 2, 2, 1, 1, 2, 2})}); + context.addMockTable( + {"test_db", "test_table_string"}, + {{"partition", TiDB::TP::TypeString}, {"order", TiDB::TP::TypeString}}, + {toVec("partition", {"banana", "banana", "banana", "banana", "apple", "apple", "apple", "apple"}), + toVec("order", {"apple", "apple", "banana", "banana", "apple", "apple", "banana", "banana"})}); + + context.addMockTable( + {"test_db", "test_table_more_cols"}, + {{"partition1", TiDB::TP::TypeLongLong}, {"partition2", TiDB::TP::TypeLongLong}, {"order1", TiDB::TP::TypeLongLong}, {"order2", TiDB::TP::TypeLongLong}}, + {toVec("partition1", {1, 1, 1, 1, 1, 1, 2, 2, 2, 2, 2, 2}), + toVec("partition2", {1, 1, 1, 2, 2, 2, 1, 1, 1, 2, 2, 2}), + toVec("order1", {2, 1, 1, 2, 1, 1, 2, 1, 1, 2, 1, 1}), + toVec("order2", {2, 2, 1, 2, 2, 1, 2, 2, 1, 2, 2, 1})}); + + context.addMockTable( + {"test_db", "test_table_float64"}, + {{"partition", TiDB::TP::TypeDouble}, {"order", TiDB::TP::TypeDouble}}, + {toVec("partition", {1.00, 1.00, 1.00, 1.00, 2.00, 2.00, 2.00, 2.00}), + toVec("order", {1.00, 1.00, 2.00, 2.00, 1.00, 1.00, 2.00, 2.00})}); + + context.addMockTable( + {"test_db", "test_table_datetime"}, + {{"partition", TiDB::TP::TypeDatetime}, {"order", TiDB::TP::TypeDatetime}}); + + context.addMockTable( + {"test_db", "test_table_for_rank"}, + {{"partition", TiDB::TP::TypeLongLong}, {"order", TiDB::TP::TypeLongLong}}, + {toVec("partition", {1, 1, 1, 1, 2, 2, 2, 2}), + toVec("order", {1, 1, 2, 2, 1, 1, 2, 2})}); } }; -TEST_F(WindowFunction, testWindowFunctionByPartitionAndOrder) +TEST_F(WindowExecutorTestRunner, testWindowFunctionByPartitionAndOrder) try { - setMaxBlockSize(3); - - std::string window_json; - std::string sort_json; - /***** row_number with different types of input *****/ // int - sql : select *, row_number() over w1 from test1 window w1 as (partition by partition_int order by order_int) - window_json = R"({"funcDesc":[{"tp":"RowNumber","sig":"Unspecified","fieldType":{"tp":8,"flag":128,"flen":21,"decimal":-1,"collate":63,"charset":"binary"},"hasDistinct":false}],"partitionBy":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"orderBy":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"frame":{"type":"Rows","start":{"type":"CurrentRow","unbounded":false,"offset":"0"},"end":{"type":"CurrentRow","unbounded":false,"offset":"0"}},"child":{"tp":"TypeSort","executorId":"Sort_12","sort":{"byItems":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"isPartialSort":true,"child":{"tp":"TypeExchangeReceiver","exchangeReceiver":{"encodedTaskMeta":["CIGAkMCV6NP+BRABIg4xMjcuMC4wLjE6MzkzMA=="],"fieldTypes":[{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"}]},"executorId":"ExchangeReceiver_11"}}}})"; - sort_json = R"({"byItems":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"isPartialSort":true,"child":{"tp":"TypeExchangeReceiver","exchangeReceiver":{"encodedTaskMeta":["CIGAkMCV6NP+BRABIg4xMjcuMC4wLjE6MzkzMA=="],"fieldTypes":[{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"}]},"executorId":"ExchangeReceiver_11"}})"; - testOneWindowFunction( - {NameAndTypePair("partition", std::make_shared()), NameAndTypePair("order", std::make_shared())}, - {toVec("partition", {1, 1, 1, 1, 2, 2, 2, 2}), toVec("order", {1, 1, 2, 2, 1, 1, 2, 2})}, - {toVec("partition", {1, 1, 1, 1, 2, 2, 2, 2}), toVec("order", {1, 1, 2, 2, 1, 1, 2, 2}), toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})}, - window_json, - sort_json); + auto request = context + .scan("test_db", "test_table") + .sort({{"partition", false}, {"order", false}, {"partition", false}, {"order", false}}, true) + .window(RowNumber(), {"order", false}, {"partition", false}, buildDefaultRowsFrame()) + .build(context); + executeStreams( + request, + {toNullableVec("partition", {1, 1, 1, 1, 2, 2, 2, 2}), + toNullableVec("order", {1, 1, 2, 2, 1, 1, 2, 2}), + toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})}); // null input - testOneWindowFunction( - {NameAndTypePair("partition", makeNullable(std::make_shared())), NameAndTypePair("order", makeNullable(std::make_shared()))}, + executeStreamsWithSingleSource( + request, {toNullableVec("partition", {}), toNullableVec("order", {})}, - {}, - window_json, - sort_json); + {}); // nullable - testOneWindowFunction( - {NameAndTypePair("partition", makeNullable(std::make_shared())), NameAndTypePair("order", makeNullable(std::make_shared()))}, - {toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2})}, - {toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2}), toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})}, - window_json, - sort_json); + executeStreamsWithSingleSource( + request, + {toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), {toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2})}}, + {toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2}), toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})}); // string - sql : select *, row_number() over w1 from test2 window w1 as (partition by partition_string order by order_string) - window_json = R"({"funcDesc":[{"tp":"RowNumber","sig":"Unspecified","fieldType":{"tp":8,"flag":128,"flen":21,"decimal":-1,"collate":63,"charset":"binary"},"hasDistinct":false}],"partitionBy":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":254,"flag":0,"flen":32,"decimal":0,"collate":46,"charset":"utf8mb4"},"hasDistinct":false},"desc":false}],"orderBy":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":254,"flag":0,"flen":32,"decimal":0,"collate":46,"charset":"utf8mb4"},"hasDistinct":false},"desc":false}],"frame":{"type":"Rows","start":{"type":"CurrentRow","unbounded":false,"offset":"0"},"end":{"type":"CurrentRow","unbounded":false,"offset":"0"}},"child":{"tp":"TypeSort","executorId":"Sort_12","sort":{"byItems":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":254,"flag":0,"flen":32,"decimal":0,"collate":46,"charset":"utf8mb4"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":254,"flag":0,"flen":32,"decimal":0,"collate":46,"charset":"utf8mb4"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":254,"flag":0,"flen":32,"decimal":0,"collate":46,"charset":"utf8mb4"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":254,"flag":0,"flen":32,"decimal":0,"collate":46,"charset":"utf8mb4"},"hasDistinct":false},"desc":false}],"isPartialSort":true,"child":{"tp":"TypeExchangeReceiver","exchangeReceiver":{"encodedTaskMeta":["CIGA8Nz57tP+BRABIg4xMjcuMC4wLjE6MzkzMA=="],"fieldTypes":[{"tp":254,"flag":0,"flen":32,"decimal":0,"collate":46,"charset":"utf8mb4"},{"tp":254,"flag":0,"flen":32,"decimal":0,"collate":46,"charset":"utf8mb4"}]},"executorId":"ExchangeReceiver_11"}}}})"; - sort_json = R"({"byItems":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":254,"flag":0,"flen":32,"decimal":0,"collate":46,"charset":"utf8mb4"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":254,"flag":0,"flen":32,"decimal":0,"collate":46,"charset":"utf8mb4"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":254,"flag":0,"flen":32,"decimal":0,"collate":46,"charset":"utf8mb4"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":254,"flag":0,"flen":32,"decimal":0,"collate":46,"charset":"utf8mb4"},"hasDistinct":false},"desc":false}],"isPartialSort":true,"child":{"tp":"TypeExchangeReceiver","exchangeReceiver":{"encodedTaskMeta":["CIGA8Nz57tP+BRABIg4xMjcuMC4wLjE6MzkzMA=="],"fieldTypes":[{"tp":254,"flag":0,"flen":32,"decimal":0,"collate":46,"charset":"utf8mb4"},{"tp":254,"flag":0,"flen":32,"decimal":0,"collate":46,"charset":"utf8mb4"}]},"executorId":"ExchangeReceiver_11"}})"; - testOneWindowFunction( - {NameAndTypePair("partition", std::make_shared()), NameAndTypePair("order", std::make_shared())}, - {toVec("partition", {"banana", "banana", "banana", "banana", "apple", "apple", "apple", "apple"}), toVec("order", {"apple", "apple", "banana", "banana", "apple", "apple", "banana", "banana"})}, - {toVec("partition", {"apple", "apple", "apple", "apple", "banana", "banana", "banana", "banana"}), toVec("order", {"apple", "apple", "banana", "banana", "apple", "apple", "banana", "banana"}), toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})}, - window_json, - sort_json); + request = context + .scan("test_db", "test_table_string") + .sort({{"partition", false}, {"order", false}, {"partition", false}, {"order", false}}, true) + .window(RowNumber(), {"order", false}, {"partition", false}, buildDefaultRowsFrame()) + .build(context); + + executeStreams( + request, + {toNullableVec("partition", {"apple", "apple", "apple", "apple", "banana", "banana", "banana", "banana"}), + toNullableVec("order", {"apple", "apple", "banana", "banana", "apple", "apple", "banana", "banana"}), + toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})}); // nullable - testOneWindowFunction( - {NameAndTypePair("partition", makeNullable(std::make_shared())), NameAndTypePair("order", makeNullable(std::make_shared()))}, - {toNullableVec("partition", {"banana", "banana", "banana", "banana", {}, "apple", "apple", "apple", "apple"}), toNullableVec("order", {"apple", "apple", "banana", "banana", {}, "apple", "apple", "banana", "banana"})}, - {toNullableVec("partition", {{}, "apple", "apple", "apple", "apple", "banana", "banana", "banana", "banana"}), toNullableVec("order", {{}, "apple", "apple", "banana", "banana", "apple", "apple", "banana", "banana"}), toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})}, - window_json, - sort_json); - - // decimal - sql : select *, row_number() over w1 from test3 window w1 as (partition by partition_float order by order_decimal) - window_json = R"({"funcDesc":[{"tp":"RowNumber","sig":"Unspecified","fieldType":{"tp":8,"flag":128,"flen":21,"decimal":-1,"collate":63,"charset":"binary"},"hasDistinct":false}],"partitionBy":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":246,"flag":0,"flen":6,"decimal":2,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"orderBy":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":246,"flag":0,"flen":6,"decimal":2,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"frame":{"type":"Rows","start":{"type":"CurrentRow","unbounded":false,"offset":"0"},"end":{"type":"CurrentRow","unbounded":false,"offset":"0"}},"child":{"tp":"TypeSort","executorId":"Sort_12","sort":{"byItems":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":246,"flag":0,"flen":6,"decimal":2,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":246,"flag":0,"flen":6,"decimal":2,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":246,"flag":0,"flen":6,"decimal":2,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":246,"flag":0,"flen":6,"decimal":2,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"isPartialSort":true,"child":{"tp":"TypeExchangeReceiver","exchangeReceiver":{"encodedTaskMeta":["CIGAoN3M99P+BRABIg4xMjcuMC4wLjE6MzkzMA=="],"fieldTypes":[{"tp":246,"flag":0,"flen":6,"decimal":2,"collate":63,"charset":"binary"},{"tp":246,"flag":0,"flen":6,"decimal":2,"collate":63,"charset":"binary"}]},"executorId":"ExchangeReceiver_11"}}}})"; - sort_json = R"({"byItems":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":246,"flag":0,"flen":6,"decimal":2,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":246,"flag":0,"flen":6,"decimal":2,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":246,"flag":0,"flen":6,"decimal":2,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":246,"flag":0,"flen":6,"decimal":2,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"isPartialSort":true,"child":{"tp":"TypeExchangeReceiver","exchangeReceiver":{"encodedTaskMeta":["CIGAoN3M99P+BRABIg4xMjcuMC4wLjE6MzkzMA=="],"fieldTypes":[{"tp":246,"flag":0,"flen":6,"decimal":2,"collate":63,"charset":"binary"},{"tp":246,"flag":0,"flen":6,"decimal":2,"collate":63,"charset":"binary"}]},"executorId":"ExchangeReceiver_11"}})"; - testOneWindowFunction( - {NameAndTypePair("partition", std::make_shared()), NameAndTypePair("order", std::make_shared())}, - {toVec("partition", {1.00, 1.00, 1.00, 1.00, 2.00, 2.00, 2.00, 2.00}), toVec("order", {1.00, 1.00, 2.00, 2.00, 1.00, 1.00, 2.00, 2.00})}, - {toVec("partition", {1.00, 1.00, 1.00, 1.00, 2.00, 2.00, 2.00, 2.00}), toVec("order", {1.00, 1.00, 2.00, 2.00, 1.00, 1.00, 2.00, 2.00}), toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})}, - window_json, - sort_json); + executeStreamsWithSingleSource( + request, + {toNullableVec("partition", {"banana", "banana", "banana", "banana", {}, "apple", "apple", "apple", "apple"}), + toNullableVec("order", {"apple", "apple", "banana", "banana", {}, "apple", "apple", "banana", "banana"})}, + {toNullableVec("partition", {{}, "apple", "apple", "apple", "apple", "banana", "banana", "banana", "banana"}), + toNullableVec("order", {{}, "apple", "apple", "banana", "banana", "apple", "apple", "banana", "banana"}), + toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})}); + + // float64 - sql : select *, row_number() over w1 from test3 window w1 as (partition by partition_float order by order_float64) + request = context + .scan("test_db", "test_table_float64") + .sort({{"partition", false}, {"order", false}, {"partition", false}, {"order", false}}, true) + .window(RowNumber(), {"order", false}, {"partition", false}, buildDefaultRowsFrame()) + .build(context); + + executeStreams( + request, + {toNullableVec("partition", {1.00, 1.00, 1.00, 1.00, 2.00, 2.00, 2.00, 2.00}), + toNullableVec("order", {1.00, 1.00, 2.00, 2.00, 1.00, 1.00, 2.00, 2.00}), + toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})}); // nullable - testOneWindowFunction( - {NameAndTypePair("partition", makeNullable(std::make_shared())), NameAndTypePair("order", makeNullable(std::make_shared()))}, - {toNullableVec("partition", {{}, 1.00, 1.00, 1.00, 1.00, 2.00, 2.00, 2.00, 2.00}), toNullableVec("order", {{}, 1.00, 1.00, 2.00, 2.00, 1.00, 1.00, 2.00, 2.00})}, - {toNullableVec("partition", {{}, 1.00, 1.00, 1.00, 1.00, 2.00, 2.00, 2.00, 2.00}), toNullableVec("order", {{}, 1.00, 1.00, 2.00, 2.00, 1.00, 1.00, 2.00, 2.00}), toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})}, - window_json, - sort_json); + executeStreamsWithSingleSource( + request, + {toNullableVec("partition", {{}, 1.00, 1.00, 1.00, 1.00, 2.00, 2.00, 2.00, 2.00}), + toNullableVec("order", {{}, 1.00, 1.00, 2.00, 2.00, 1.00, 1.00, 2.00, 2.00})}, + {toNullableVec("partition", {{}, 1.00, 1.00, 1.00, 1.00, 2.00, 2.00, 2.00, 2.00}), + toNullableVec("order", {{}, 1.00, 1.00, 2.00, 2.00, 1.00, 1.00, 2.00, 2.00}), + toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})}); // datetime - select *, row_number() over w1 from test4 window w1 as (partition by partition_datetime order by order_datetime); - window_json = R"({"funcDesc":[{"tp":"RowNumber","sig":"Unspecified","fieldType":{"tp":8,"flag":128,"flen":21,"decimal":-1,"collate":63,"charset":"binary"},"hasDistinct":false}],"partitionBy":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":12,"flag":128,"flen":26,"decimal":6,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"orderBy":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":12,"flag":128,"flen":26,"decimal":6,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"frame":{"type":"Rows","start":{"type":"CurrentRow","unbounded":false,"offset":"0"},"end":{"type":"CurrentRow","unbounded":false,"offset":"0"}},"child":{"tp":"TypeSort","executorId":"Sort_12","sort":{"byItems":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":12,"flag":128,"flen":26,"decimal":6,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":12,"flag":128,"flen":26,"decimal":6,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":12,"flag":128,"flen":26,"decimal":6,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":12,"flag":128,"flen":26,"decimal":6,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"isPartialSort":true,"child":{"tp":"TypeExchangeReceiver","exchangeReceiver":{"encodedTaskMeta":["CIGAsNmBhdT+BRABIg4xMjcuMC4wLjE6MzkzMA=="],"fieldTypes":[{"tp":12,"flag":128,"flen":26,"decimal":6,"collate":63,"charset":"binary"},{"tp":12,"flag":128,"flen":26,"decimal":6,"collate":63,"charset":"binary"}]},"executorId":"ExchangeReceiver_11"}}}})"; - sort_json = R"({"byItems":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":12,"flag":128,"flen":26,"decimal":6,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":12,"flag":128,"flen":26,"decimal":6,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":12,"flag":128,"flen":26,"decimal":6,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":12,"flag":128,"flen":26,"decimal":6,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"isPartialSort":true,"child":{"tp":"TypeExchangeReceiver","exchangeReceiver":{"encodedTaskMeta":["CIGAsNmBhdT+BRABIg4xMjcuMC4wLjE6MzkzMA=="],"fieldTypes":[{"tp":12,"flag":128,"flen":26,"decimal":6,"collate":63,"charset":"binary"},{"tp":12,"flag":128,"flen":26,"decimal":6,"collate":63,"charset":"binary"}]},"executorId":"ExchangeReceiver_11"}})"; - testOneWindowFunction( - {NameAndTypePair("partition", std::make_shared()), NameAndTypePair("order", std::make_shared())}, - {toDatetimeVec("partition", {"20220101010102", "20220101010102", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010101", "20220101010101"}, 0), + request = context + .scan("test_db", "test_table_datetime") + .sort({{"partition", false}, {"order", false}, {"partition", false}, {"order", false}}, true) + .window(RowNumber(), {"order", false}, {"partition", false}, buildDefaultRowsFrame()) + .build(context); + executeStreamsWithSingleSource( + request, + {toNullableDatetimeVec("partition", {"20220101010102", "20220101010102", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010101", "20220101010101"}, 0), toDatetimeVec("order", {"20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010102", "20220101010102"}, 0)}, - {toDatetimeVec("partition", {"20220101010101", "20220101010101", "20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010102", "20220101010102"}, 0), - toDatetimeVec("order", {"20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010102", "20220101010102"}, 0), - toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})}, - window_json, - sort_json); + {toNullableDatetimeVec("partition", {"20220101010101", "20220101010101", "20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010102", "20220101010102"}, 0), + toNullableDatetimeVec("order", {"20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010102", "20220101010102"}, 0), + toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})}); // nullable - testOneWindowFunction( - {NameAndTypePair("partition", makeNullable(std::make_shared())), NameAndTypePair("order", makeNullable(std::make_shared()))}, + executeStreamsWithSingleSource( + request, {toNullableDatetimeVec("partition", {"20220101010102", {}, "20220101010102", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010101", "20220101010101"}, 0), toNullableDatetimeVec("order", {"20220101010101", {}, "20220101010101", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010102", "20220101010102"}, 0)}, {toNullableDatetimeVec("partition", {{}, "20220101010101", "20220101010101", "20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010102", "20220101010102"}, 0), toNullableDatetimeVec("order", {{}, "20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010102", "20220101010102"}, 0), - toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})}, - window_json, - sort_json); + toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})}); // 2 partiton key and 2 order key // sql : select *, row_number() over w1 from test6 window w1 as (partition by partition_int1, partition_int2 order by order_int1,order_int2) - window_json = R"({"funcDesc":[{"tp":"RowNumber","sig":"Unspecified","fieldType":{"tp":8,"flag":128,"flen":21,"decimal":-1,"collate":63,"charset":"binary"},"hasDistinct":false}],"partitionBy":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"orderBy":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAI=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAM=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"frame":{"type":"Rows","start":{"type":"CurrentRow","unbounded":false,"offset":"0"},"end":{"type":"CurrentRow","unbounded":false,"offset":"0"}},"child":{"tp":"TypeSort","executorId":"Sort_12","sort":{"byItems":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAI=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAM=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAI=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAM=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"isPartialSort":true,"child":{"tp":"TypeExchangeReceiver","exchangeReceiver":{"encodedTaskMeta":["CIKA0Img1If/BRABIg4xMjcuMC4wLjE6MzkzMA=="],"fieldTypes":[{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"}]},"executorId":"ExchangeReceiver_11"}}}})"; - sort_json = R"({"byItems":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAI=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAM=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAI=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAM=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"isPartialSort":true,"child":{"tp":"TypeExchangeReceiver","exchangeReceiver":{"encodedTaskMeta":["CIKA0Img1If/BRABIg4xMjcuMC4wLjE6MzkzMA=="],"fieldTypes":[{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"}]},"executorId":"ExchangeReceiver_11"}})"; - testOneWindowFunction( - {NameAndTypePair("partition1", std::make_shared()), NameAndTypePair("partition2", std::make_shared()), NameAndTypePair("order1", std::make_shared()), NameAndTypePair("order2", std::make_shared())}, - {toVec("partition1", {1, 1, 1, 1, 1, 1, 2, 2, 2, 2, 2, 2}), toVec("partition2", {1, 1, 1, 2, 2, 2, 1, 1, 1, 2, 2, 2}), toVec("order1", {2, 1, 1, 2, 1, 1, 2, 1, 1, 2, 1, 1}), toVec("order2", {2, 2, 1, 2, 2, 1, 2, 2, 1, 2, 2, 1})}, - {toVec("partition1", {1, 1, 1, 1, 1, 1, 2, 2, 2, 2, 2, 2}), toVec("partition2", {1, 1, 1, 2, 2, 2, 1, 1, 1, 2, 2, 2}), toVec("order1", {1, 1, 2, 1, 1, 2, 1, 1, 2, 1, 1, 2}), toVec("order2", {1, 2, 2, 1, 2, 2, 1, 2, 2, 1, 2, 2}), toNullableVec("row_number", {1, 2, 3, 1, 2, 3, 1, 2, 3, 1, 2, 3})}, - window_json, - sort_json); + request = context + .scan("test_db", "test_table_more_cols") + .sort({{"partition1", false}, {"partition2", false}, {"order1", false}, {"order2", false}}, true) + .window(RowNumber(), {{"order1", false}, {"order2", false}}, {{"partition1", false}, {"partition2", false}}, buildDefaultRowsFrame()) + .build(context); + + executeStreams( + request, + {toNullableVec("partition1", {1, 1, 1, 1, 1, 1, 2, 2, 2, 2, 2, 2}), + toNullableVec("partition2", {1, 1, 1, 2, 2, 2, 1, 1, 1, 2, 2, 2}), + toNullableVec("order1", {1, 1, 2, 1, 1, 2, 1, 1, 2, 1, 1, 2}), + toNullableVec("order2", {1, 2, 2, 1, 2, 2, 1, 2, 2, 1, 2, 2}), + toNullableVec("row_number", {1, 2, 3, 1, 2, 3, 1, 2, 3, 1, 2, 3})}); /***** rank, dense_rank *****/ - window_json = R"({"funcDesc":[{"tp":"Rank","sig":"Unspecified","fieldType":{"tp":8,"flag":128,"flen":21,"decimal":-1,"collate":63,"charset":"binary"},"hasDistinct":false},{"tp":"DenseRank","sig":"Unspecified","fieldType":{"tp":8,"flag":128,"flen":21,"decimal":-1,"collate":63,"charset":"binary"},"hasDistinct":false}],"partitionBy":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"orderBy":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"child":{"tp":"TypeSort","executorId":"Sort_12","sort":{"byItems":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"isPartialSort":true,"child":{"tp":"TypeExchangeReceiver","exchangeReceiver":{"encodedTaskMeta":["CIGAsOnl3NP+BRABIg4xMjcuMC4wLjE6MzkzMA=="],"fieldTypes":[{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"}]},"executorId":"ExchangeReceiver_11"}}}})"; - sort_json = R"({"byItems":[{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAA=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false},{"expr":{"tp":"ColumnRef","val":"gAAAAAAAAAE=","sig":"Unspecified","fieldType":{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},"hasDistinct":false},"desc":false}],"isPartialSort":true,"child":{"tp":"TypeExchangeReceiver","exchangeReceiver":{"encodedTaskMeta":["CIGAsOnl3NP+BRABIg4xMjcuMC4wLjE6MzkzMA=="],"fieldTypes":[{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"},{"tp":3,"flag":0,"flen":11,"decimal":0,"collate":63,"charset":"binary"}]},"executorId":"ExchangeReceiver_11"}})"; - testOneWindowFunction( - {NameAndTypePair("partition", std::make_shared()), NameAndTypePair("order", std::make_shared())}, - {toVec("partition", {1, 1, 1, 1, 2, 2, 2, 2}), toVec("order", {1, 1, 2, 2, 1, 1, 2, 2})}, - {toVec("partition", {1, 1, 1, 1, 2, 2, 2, 2}), toVec("order", {1, 1, 2, 2, 1, 1, 2, 2}), toNullableVec("rank", {1, 1, 3, 3, 1, 1, 3, 3}), toNullableVec("dense_rank", {1, 1, 2, 2, 1, 1, 2, 2})}, - window_json, - sort_json); + request = context.scan("test_db", "test_table_for_rank").sort({{"partition", false}, {"order", false}}, true).window({Rank(), DenseRank()}, {{"order", false}}, {{"partition", false}}, MockWindowFrame{}).build(context); + executeStreams( + request, + {toNullableVec("partition", {1, 1, 1, 1, 2, 2, 2, 2}), + toNullableVec("order", {1, 1, 2, 2, 1, 1, 2, 2}), + toNullableVec("rank", {1, 1, 3, 3, 1, 1, 3, 3}), + toNullableVec("dense_rank", {1, 1, 2, 2, 1, 1, 2, 2})}); // nullable - testOneWindowFunction( - {NameAndTypePair("partition", makeNullable(std::make_shared())), NameAndTypePair("order", makeNullable(std::make_shared()))}, - {toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2})}, - {toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2}), toNullableVec("rank", {1, 1, 1, 3, 3, 1, 1, 3, 3}), toNullableVec("dense_rank", {1, 1, 1, 2, 2, 1, 1, 2, 2})}, - window_json, - sort_json); - - testOneWindowFunction( - {NameAndTypePair("partition", makeNullable(std::make_shared())), NameAndTypePair("order", makeNullable(std::make_shared()))}, - {toNullableVec("partition", {{}, {}, 1, 1, 1, 1, 2, 2, 2, 2}), toNullableVec("order", {{}, 1, 1, 1, 2, 2, 1, 1, 2, 2})}, - {toNullableVec("partition", {{}, {}, 1, 1, 1, 1, 2, 2, 2, 2}), toNullableVec("order", {{}, 1, 1, 1, 2, 2, 1, 1, 2, 2}), toNullableVec("rank", {1, 2, 1, 1, 3, 3, 1, 1, 3, 3}), toNullableVec("dense_rank", {1, 2, 1, 1, 2, 2, 1, 1, 2, 2})}, - window_json, - sort_json); + executeStreamsWithSingleSource( + request, + {toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), + toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2})}, + {toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), + toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2}), + toNullableVec("rank", {1, 1, 1, 3, 3, 1, 1, 3, 3}), + toNullableVec("dense_rank", {1, 1, 1, 2, 2, 1, 1, 2, 2})}); + + executeStreamsWithSingleSource( + request, + {toNullableVec("partition", {{}, {}, 1, 1, 1, 1, 2, 2, 2, 2}), + toNullableVec("order", {{}, 1, 1, 1, 2, 2, 1, 1, 2, 2})}, + {toNullableVec("partition", {{}, {}, 1, 1, 1, 1, 2, 2, 2, 2}), + toNullableVec("order", {{}, 1, 1, 1, 2, 2, 1, 1, 2, 2}), + toNullableVec("rank", {1, 2, 1, 1, 3, 3, 1, 1, 3, 3}), + toNullableVec("dense_rank", {1, 2, 1, 1, 2, 2, 1, 1, 2, 2})}); } CATCH + } // namespace DB::tests From 123440cf8b9244f42c721ed67ebc95b474551470 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 14 Jun 2022 12:06:33 +0800 Subject: [PATCH 036/104] Remove the log with high frequency and not useful enough (#5141) ref pingcap/tiflash#5140 --- dbms/src/Storages/tests/gtest_filter_parser.cpp | 2 +- dbms/src/TiDB/Schema/SchemaBuilder-internal.h | 2 +- dbms/src/TiDB/Schema/SchemaBuilder.cpp | 5 ++--- 3 files changed, 4 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/tests/gtest_filter_parser.cpp b/dbms/src/Storages/tests/gtest_filter_parser.cpp index 8820c05d2da..3a554fcf4b6 100644 --- a/dbms/src/Storages/tests/gtest_filter_parser.cpp +++ b/dbms/src/Storages/tests/gtest_filter_parser.cpp @@ -98,7 +98,7 @@ DM::RSOperatorPtr FilterParserTest::generateRsOperator(const String table_info_j DM::ColumnDefines columns_to_read; { NamesAndTypes source_columns; - std::tie(source_columns, std::ignore) = parseColumnsFromTableInfo(table_info, log->getLog()); + std::tie(source_columns, std::ignore) = parseColumnsFromTableInfo(table_info); dag_query = std::make_unique( conditions, DAGPreparedSets(), diff --git a/dbms/src/TiDB/Schema/SchemaBuilder-internal.h b/dbms/src/TiDB/Schema/SchemaBuilder-internal.h index a331205ce8c..94edcbea204 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder-internal.h +++ b/dbms/src/TiDB/Schema/SchemaBuilder-internal.h @@ -35,7 +35,7 @@ struct TableInfo; } namespace DB { -std::tuple parseColumnsFromTableInfo(const TiDB::TableInfo & table_info, Poco::Logger * log); +std::tuple parseColumnsFromTableInfo(const TiDB::TableInfo & table_info); constexpr char tmpNamePrefix[] = "_tiflash_tmp_"; diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.cpp b/dbms/src/TiDB/Schema/SchemaBuilder.cpp index 99e540e6c95..f532ac231e2 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.cpp +++ b/dbms/src/TiDB/Schema/SchemaBuilder.cpp @@ -963,13 +963,12 @@ void SchemaBuilder::applyDropSchema(const String & db_name) } std::tuple -parseColumnsFromTableInfo(const TiDB::TableInfo & table_info, Poco::Logger * log) +parseColumnsFromTableInfo(const TiDB::TableInfo & table_info) { NamesAndTypes columns; std::vector primary_keys; for (const auto & column : table_info.columns) { - LOG_FMT_DEBUG(log, "Analyzing column: {}, type: {}", column.name, static_cast(column.tp)); DataTypePtr type = getDataTypeByColumnInfo(column); columns.emplace_back(column.name, type); if (column.hasPriKeyFlag()) @@ -999,7 +998,7 @@ String createTableStmt( Poco::Logger * log) { LOG_FMT_DEBUG(log, "Analyzing table info : {}", table_info.serialize()); - auto [columns, pks] = parseColumnsFromTableInfo(table_info, log); + auto [columns, pks] = parseColumnsFromTableInfo(table_info); String stmt; WriteBufferFromString stmt_buf(stmt); From 4b3a2ce5a691ea4a3aa400d9ef157484b638982c Mon Sep 17 00:00:00 2001 From: shuke <37901441+shuke987@users.noreply.github.com> Date: Tue, 14 Jun 2022 12:48:33 +0800 Subject: [PATCH 037/104] modify cached_gc_safe_point to atomic to prevent more request to PD (#5079) ref pingcap/tiflash#4928 --- dbms/src/Storages/Transaction/PDTiKVClient.cpp | 4 ++-- dbms/src/Storages/Transaction/PDTiKVClient.h | 8 +++++--- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/Transaction/PDTiKVClient.cpp b/dbms/src/Storages/Transaction/PDTiKVClient.cpp index 5a4b751fd9c..a06f1a3ae64 100644 --- a/dbms/src/Storages/Transaction/PDTiKVClient.cpp +++ b/dbms/src/Storages/Transaction/PDTiKVClient.cpp @@ -22,7 +22,7 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -Timestamp PDClientHelper::cached_gc_safe_point = 0; -std::chrono::time_point PDClientHelper::safe_point_last_update_time; +std::atomic PDClientHelper::cached_gc_safe_point = 0; +std::atomic> PDClientHelper::safe_point_last_update_time; } // namespace DB diff --git a/dbms/src/Storages/Transaction/PDTiKVClient.h b/dbms/src/Storages/Transaction/PDTiKVClient.h index 4986c28f4ac..e5801cc7fae 100644 --- a/dbms/src/Storages/Transaction/PDTiKVClient.h +++ b/dbms/src/Storages/Transaction/PDTiKVClient.h @@ -29,6 +29,8 @@ #include #include +#include + // We define a shared ptr here, because TMTContext / SchemaSyncer / IndexReader all need to // `share` the resource of cluster. using KVClusterPtr = std::shared_ptr; @@ -49,7 +51,7 @@ struct PDClientHelper { // In case we cost too much to update safe point from PD. std::chrono::time_point now = std::chrono::system_clock::now(); - const auto duration = std::chrono::duration_cast(now - safe_point_last_update_time); + const auto duration = std::chrono::duration_cast(now - safe_point_last_update_time.load()); const auto min_interval = std::max(Int64(1), safe_point_update_interval_seconds); // at least one second if (duration.count() < min_interval) return cached_gc_safe_point; @@ -73,8 +75,8 @@ struct PDClientHelper } private: - static Timestamp cached_gc_safe_point; - static std::chrono::time_point safe_point_last_update_time; + static std::atomic cached_gc_safe_point; + static std::atomic> safe_point_last_update_time; }; From ad6b8310fc733045ae7f188c22726620706a513f Mon Sep 17 00:00:00 2001 From: jiaqizho Date: Tue, 14 Jun 2022 14:36:33 +0800 Subject: [PATCH 038/104] Removed a unused proxy status api named test-show (#5136) close pingcap/tiflash#5135 --- .../Transaction/ProxyFFIStatusService.cpp | 23 +------------------ 1 file changed, 1 insertion(+), 22 deletions(-) diff --git a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp index dafacd8947d..792f149f588 100644 --- a/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFIStatusService.cpp @@ -22,26 +22,6 @@ namespace DB { -HttpRequestRes HandleHttpRequestTestShow( - EngineStoreServerWrap *, - std::string_view path, - const std::string & api_name, - std::string_view query, - std::string_view body) -{ - auto * res = RawCppString::New(fmt::format( - "api_name: {}\npath: {}\nquery: {}\nbody: {}", - api_name, - path, - query, - body)); - return HttpRequestRes{ - .status = HttpRequestStatus::Ok, - .res = CppStrWithView{ - .inner = GenRawCppPtr(res, RawCppPtrTypeImpl::String), - .view = BaseBuffView{res->data(), res->size()}}}; -} - HttpRequestRes HandleHttpRequestSyncStatus( EngineStoreServerWrap * server, std::string_view path, @@ -112,8 +92,7 @@ using HANDLE_HTTP_URI_METHOD = HttpRequestRes (*)(EngineStoreServerWrap *, std:: static const std::map AVAILABLE_HTTP_URI = { {"/tiflash/sync-status/", HandleHttpRequestSyncStatus}, - {"/tiflash/store-status", HandleHttpRequestStoreStatus}, - {"/tiflash/test-show", HandleHttpRequestTestShow}}; + {"/tiflash/store-status", HandleHttpRequestStoreStatus}}; uint8_t CheckHttpUriAvailable(BaseBuffView path_) { From 94aa0291482ab44ef36309e984f87a68d5e75122 Mon Sep 17 00:00:00 2001 From: jiaqizho Date: Tue, 14 Jun 2022 15:06:33 +0800 Subject: [PATCH 039/104] Fix blobstore truncate size may not right (#5127) close pingcap/tiflash#5076, close pingcap/tiflash#5134 --- dbms/src/Storages/Page/V3/BlobStore.cpp | 55 ++++++++++++++----- dbms/src/Storages/Page/V3/PageDirectory.cpp | 2 +- dbms/src/Storages/Page/V3/spacemap/SpaceMap.h | 6 +- .../Storages/Page/V3/spacemap/SpaceMapBig.h | 2 +- .../Page/V3/spacemap/SpaceMapRBTree.cpp | 28 +++++++--- .../Page/V3/spacemap/SpaceMapRBTree.h | 2 +- .../Page/V3/spacemap/SpaceMapSTDMap.h | 22 +++++++- .../Storages/Page/V3/tests/gtest_free_map.cpp | 37 +++++++++++++ 8 files changed, 125 insertions(+), 29 deletions(-) diff --git a/dbms/src/Storages/Page/V3/BlobStore.cpp b/dbms/src/Storages/Page/V3/BlobStore.cpp index 37a4fd429f4..3bd0bd9c4fa 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.cpp +++ b/dbms/src/Storages/Page/V3/BlobStore.cpp @@ -851,8 +851,8 @@ struct BlobStoreGCInfo toTypeString("Read-Only Blob", 0), toTypeString("No GC Blob", 1), toTypeString("Full GC Blob", 2), - toTypeString("Truncated Blob", 3), - toTypeString("Big Blob", 4)); + toTypeString("Big Blob", 3), + toTypeTruncateString("Truncated Blob")); } void appendToReadOnlyBlob(const BlobFileId blob_id, double valid_rate) @@ -870,23 +870,24 @@ struct BlobStoreGCInfo blob_gc_info[2].emplace_back(std::make_pair(blob_id, valid_rate)); } - void appendToTruncatedBlob(const BlobFileId blob_id, double valid_rate) + void appendToBigBlob(const BlobFileId blob_id, double valid_rate) { blob_gc_info[3].emplace_back(std::make_pair(blob_id, valid_rate)); } - void appendToBigBlob(const BlobFileId blob_id, double valid_rate) + void appendToTruncatedBlob(const BlobFileId blob_id, UInt64 origin_size, UInt64 truncated_size, double valid_rate) { - blob_gc_info[4].emplace_back(std::make_pair(blob_id, valid_rate)); + blob_gc_truncate_info.emplace_back(std::make_tuple(blob_id, origin_size, truncated_size, valid_rate)); } private: // 1. read only blob // 2. no need gc blob // 3. full gc blob - // 4. need truncate blob - // 5. big blob - std::vector> blob_gc_info[5]; + // 4. big blob + std::vector> blob_gc_info[4]; + + std::vector> blob_gc_truncate_info; String toTypeString(const std::string_view prefix, const size_t index) const { @@ -911,6 +912,32 @@ struct BlobStoreGCInfo return fmt_buf.toString(); } + + String toTypeTruncateString(const std::string_view prefix) const + { + FmtBuffer fmt_buf; + if (blob_gc_truncate_info.empty()) + { + fmt_buf.fmtAppend("{}: [null]", prefix); + } + else + { + fmt_buf.fmtAppend("{}: [", prefix); + fmt_buf.joinStr( + blob_gc_truncate_info.begin(), + blob_gc_truncate_info.end(), + [](const auto arg, FmtBuffer & fb) { + fb.fmtAppend("{} origin: {} truncate: {} rate: {:.2f}", // + std::get<0>(arg), // blob id + std::get<1>(arg), // origin size + std::get<2>(arg), // truncated size + std::get<3>(arg)); // valid rate + }, + ", "); + fmt_buf.append("]"); + } + return fmt_buf.toString(); + } }; std::vector BlobStore::getGCStats() @@ -953,7 +980,7 @@ std::vector BlobStore::getGCStats() } auto lock = stat->lock(); - auto right_margin = stat->smap->getRightMargin(); + auto right_margin = stat->smap->getUsedBoundary(); // Avoid divide by zero if (right_margin == 0) @@ -966,14 +993,13 @@ std::vector BlobStore::getGCStats() stat->sm_valid_rate)); } - LOG_FMT_TRACE(log, "Current blob is empty [blob_id={}, total size(all invalid)={}] [valid_rate={}].", stat->id, stat->sm_total_size, stat->sm_valid_rate); - // If current blob empty, the size of in disk blob may not empty // So we need truncate current blob, and let it be reused. auto blobfile = getBlobFile(stat->id); - LOG_FMT_TRACE(log, "Truncate empty blob file [blob_id={}] to 0.", stat->id); + LOG_FMT_INFO(log, "Current blob file is empty, truncated to zero [blob_id={}] [total_size={}] [valid_rate={}]", stat->id, stat->sm_total_size, stat->sm_valid_rate); blobfile->truncate(right_margin); - blobstore_gc_info.appendToTruncatedBlob(stat->id, stat->sm_valid_rate); + blobstore_gc_info.appendToTruncatedBlob(stat->id, stat->sm_total_size, right_margin, stat->sm_valid_rate); + stat->sm_total_size = right_margin; continue; } @@ -1014,9 +1040,10 @@ std::vector BlobStore::getGCStats() auto blobfile = getBlobFile(stat->id); LOG_FMT_TRACE(log, "Truncate blob file [blob_id={}] [origin size={}] [truncated size={}]", stat->id, stat->sm_total_size, right_margin); blobfile->truncate(right_margin); + blobstore_gc_info.appendToTruncatedBlob(stat->id, stat->sm_total_size, right_margin, stat->sm_valid_rate); + stat->sm_total_size = right_margin; stat->sm_valid_rate = stat->sm_valid_size * 1.0 / stat->sm_total_size; - blobstore_gc_info.appendToTruncatedBlob(stat->id, stat->sm_valid_rate); } } } diff --git a/dbms/src/Storages/Page/V3/PageDirectory.cpp b/dbms/src/Storages/Page/V3/PageDirectory.cpp index e9b754854b8..5eb275f5af5 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory.cpp +++ b/dbms/src/Storages/Page/V3/PageDirectory.cpp @@ -1223,7 +1223,7 @@ bool PageDirectory::tryDumpSnapshot(const ReadLimiterPtr & read_limiter, const W // `being_ref_count` by the function `createSnapshot()`. assert(!files_snap.persisted_log_files.empty()); // should not be empty when `needSave` return true auto log_num = files_snap.persisted_log_files.rbegin()->log_num; - auto identifier = fmt::format("{}_dump_{}", wal->name(), log_num); + auto identifier = fmt::format("{}.dump_{}", wal->name(), log_num); auto snapshot_reader = wal->createReaderForFiles(identifier, files_snap.persisted_log_files, read_limiter); PageDirectoryFactory factory; // we just use the `collapsed_dir` to dump edit of the snapshot, should never call functions like `apply` that diff --git a/dbms/src/Storages/Page/V3/spacemap/SpaceMap.h b/dbms/src/Storages/Page/V3/spacemap/SpaceMap.h index ae44b608de0..d230b2f3e35 100644 --- a/dbms/src/Storages/Page/V3/spacemap/SpaceMap.h +++ b/dbms/src/Storages/Page/V3/spacemap/SpaceMap.h @@ -95,9 +95,11 @@ class SpaceMap virtual std::tuple searchInsertOffset(size_t size) = 0; /** - * Get the offset of the last free block. `[margin_offset, +∞)` is not used at all. + * Get the used boundary of this SpaceMap. + * The return value (`used_boundary`) means that `[used_bounary + 1, +∞)` is safe to be truncated. + * If the `used_boundary` is equal to the `end` of this SpaceMap, it means that there is no space to be truncated. */ - virtual UInt64 getRightMargin() = 0; + virtual UInt64 getUsedBoundary() = 0; /** * Get the accurate max capacity of the space map. diff --git a/dbms/src/Storages/Page/V3/spacemap/SpaceMapBig.h b/dbms/src/Storages/Page/V3/spacemap/SpaceMapBig.h index 22128a09f30..81c2a5cb786 100644 --- a/dbms/src/Storages/Page/V3/spacemap/SpaceMapBig.h +++ b/dbms/src/Storages/Page/V3/spacemap/SpaceMapBig.h @@ -74,7 +74,7 @@ class BigSpaceMap return std::make_pair(size_in_used, size_in_used); } - UInt64 getRightMargin() override + UInt64 getUsedBoundary() override { return end; } diff --git a/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.cpp b/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.cpp index 54275574060..4bd53b93e07 100644 --- a/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.cpp +++ b/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.cpp @@ -84,7 +84,7 @@ static void rb_get_new_entry(struct SmapRbEntry ** entry, UInt64 start, UInt64 c { struct SmapRbEntry * new_entry; - new_entry = static_cast(calloc(1, sizeof(struct SmapRbEntry))); + new_entry = static_cast(calloc(1, sizeof(struct SmapRbEntry))); // NOLINT if (new_entry == nullptr) { return; @@ -115,7 +115,7 @@ inline static void rb_free_entry(struct RbPrivate * private_data, struct SmapRbE private_data->read_index_next = nullptr; } - free(entry); + free(entry); // NOLINT } @@ -419,7 +419,7 @@ std::shared_ptr RBTreeSpaceMap::create(UInt64 start, UInt64 end) { auto ptr = std::shared_ptr(new RBTreeSpaceMap(start, end)); - ptr->rb_tree = static_cast(calloc(1, sizeof(struct RbPrivate))); + ptr->rb_tree = static_cast(calloc(1, sizeof(struct RbPrivate))); // NOLINT if (ptr->rb_tree == nullptr) { return nullptr; @@ -435,7 +435,7 @@ std::shared_ptr RBTreeSpaceMap::create(UInt64 start, UInt64 end) if (!rb_insert_entry(start, end, ptr->rb_tree, ptr->log)) { LOG_FMT_ERROR(ptr->log, "Erorr happend, when mark all space free. [start={}] , [end={}]", start, end); - free(ptr->rb_tree); + free(ptr->rb_tree); // NOLINT return nullptr; } return ptr; @@ -451,7 +451,7 @@ static void rb_free_tree(struct rb_root * root) next = rb_tree_next(node); entry = node_to_entry(node); rb_node_remove(node, root); - free(entry); + free(entry); // NOLINT } } @@ -460,7 +460,7 @@ void RBTreeSpaceMap::freeSmap() if (rb_tree) { rb_free_tree(&rb_tree->root); - free(rb_tree); + free(rb_tree); // NOLINT } } @@ -734,7 +734,7 @@ std::pair RBTreeSpaceMap::getSizes() const } } -UInt64 RBTreeSpaceMap::getRightMargin() +UInt64 RBTreeSpaceMap::getUsedBoundary() { struct rb_node * node = rb_tree_last(&rb_tree->root); if (node == nullptr) @@ -743,6 +743,20 @@ UInt64 RBTreeSpaceMap::getRightMargin() } auto * entry = node_to_entry(node); + + // If the `offset+size` of the last free node is not equal to `end`, it means the range `[last_node.offset, end)` is marked as used, + // then we should return `end` as the used boundary. + // + // eg. + // 1. The spacemap manage a space of `[0, 100]` + // 2. A span {offset=90, size=10} is marked as used, then the free range in SpaceMap is `[0, 90)` + // 3. The return value should be 100 + if (entry->start + entry->count != end) + { + return end; + } + + // Else we should return the offset of last free node return entry->start; } diff --git a/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.h b/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.h index 0393fda081b..04691007a47 100644 --- a/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.h +++ b/dbms/src/Storages/Page/V3/spacemap/SpaceMapRBTree.h @@ -46,7 +46,7 @@ class RBTreeSpaceMap std::pair getSizes() const override; - UInt64 getRightMargin() override; + UInt64 getUsedBoundary() override; protected: RBTreeSpaceMap(UInt64 start, UInt64 end) diff --git a/dbms/src/Storages/Page/V3/spacemap/SpaceMapSTDMap.h b/dbms/src/Storages/Page/V3/spacemap/SpaceMapSTDMap.h index b6ff8797f0f..41ddd77d03a 100644 --- a/dbms/src/Storages/Page/V3/spacemap/SpaceMapSTDMap.h +++ b/dbms/src/Storages/Page/V3/spacemap/SpaceMapSTDMap.h @@ -111,13 +111,29 @@ class STDMapSpaceMap } } - UInt64 getRightMargin() override + UInt64 getUsedBoundary() override { if (free_map.empty()) { - return end - start; + return end; } - return free_map.rbegin()->first; + + const auto & last_node_it = free_map.rbegin(); + + // If the `offset+size` of the last free node is not equal to `end`, it means the range `[last_node.offset, end)` is marked as used, + // then we should return `end` as the used boundary. + // + // eg. + // 1. The spacemap manage a space of `[0, 100]` + // 2. A span {offset=90, size=10} is marked as used, then the free range in SpaceMap is `[0, 90)` + // 3. The return value should be 100 + if (last_node_it->first + last_node_it->second != end) + { + return end; + } + + // Else we should return the offset of last free node + return last_node_it->first; } bool isMarkUnused(UInt64 offset, size_t length) override diff --git a/dbms/src/Storages/Page/V3/tests/gtest_free_map.cpp b/dbms/src/Storages/Page/V3/tests/gtest_free_map.cpp index f7120f000b2..faec139920b 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_free_map.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_free_map.cpp @@ -427,6 +427,43 @@ TEST_P(SpaceMapTest, TestGetMaxCap) } } + +TEST_P(SpaceMapTest, TestGetUsedBoundary) +{ + { + auto smap = SpaceMap::createSpaceMap(test_type, 0, 100); + ASSERT_TRUE(smap->markUsed(50, 10)); + ASSERT_EQ(smap->getUsedBoundary(), 60); + ASSERT_TRUE(smap->markUsed(80, 10)); + ASSERT_EQ(smap->getUsedBoundary(), 90); + + ASSERT_TRUE(smap->markUsed(90, 10)); + ASSERT_EQ(smap->getUsedBoundary(), 100); + } + + { + auto smap = SpaceMap::createSpaceMap(test_type, 0, 100); + ASSERT_TRUE(smap->markUsed(90, 10)); + ASSERT_EQ(smap->getUsedBoundary(), 100); + + ASSERT_TRUE(smap->markUsed(20, 10)); + ASSERT_EQ(smap->getUsedBoundary(), 100); + + ASSERT_TRUE(smap->markFree(90, 10)); + ASSERT_EQ(smap->getUsedBoundary(), 30); + + ASSERT_TRUE(smap->markUsed(90, 10)); + ASSERT_EQ(smap->getUsedBoundary(), 100); + } + + { + auto smap = SpaceMap::createSpaceMap(test_type, 0, 100); + ASSERT_EQ(smap->getUsedBoundary(), 0); + ASSERT_TRUE(smap->markUsed(0, 100)); + ASSERT_EQ(smap->getUsedBoundary(), 100); + } +} + INSTANTIATE_TEST_CASE_P( Type, SpaceMapTest, From bcb837bc25215891a5c2b9a95d2cec085df66053 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Tue, 14 Jun 2022 15:58:33 +0800 Subject: [PATCH 040/104] enhencement: supplement the comment for SchemaActionType (#5139) ref pingcap/tiflash#4862 --- dbms/src/TiDB/Schema/SchemaBuilder.cpp | 10 +++++++++- dbms/src/TiDB/Schema/SchemaGetter.h | 1 + 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.cpp b/dbms/src/TiDB/Schema/SchemaBuilder.cpp index f532ac231e2..ae78923fc61 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.cpp +++ b/dbms/src/TiDB/Schema/SchemaBuilder.cpp @@ -370,7 +370,15 @@ void SchemaBuilder::applyAlterPhysicalTable(DBInfoPtr db_inf const auto & schema_change = schema_changes[i]; /// Update column infos by applying schema change in this step. schema_change.second(orig_table_info); - /// Update schema version aggressively for the sake of correctness. + /// Update schema version aggressively for the sake of correctness(for read part). + /// In read action, we will use table_info.schema_version(storage_version) and TiDBSchemaSyncer.cur_version(global_version) to compare with query_version, to decide whether we can read under this query_version, or we need to make the schema newer. + /// In our comparison logic, we only serve the query when the query schema version meet the criterion: storage_version <= query_version <= global_version(The more detail info you can refer the comments in DAGStorageInterpreter::getAndLockStorages.) + /// And when apply multi diffs here, we only update global_version when all diffs have been applied. + /// So the global_version may be less than the actual "global_version" of the local schema in the process of applying schema changes. + /// And if we don't update the storage_version ahead of time, we may meet the following case when apply multiple diffs: storage_version <= global_version < actual "global_version". + /// If we receive a query with the same version as global_version, we can have the following scenario: storage_version <= global_version == query_version < actual "global_version". + /// And because storage_version <= global_version == query_version meet the criterion of serving the query, the query will be served. But query_version < actual "global_version" indicates that we use a newer schema to server an older query which may cause some inconsistency issue. + /// So we update storage_version aggressively to prevent the above scenario happens. orig_table_info.schema_version = target_version; auto alter_lock = storage->lockForAlter(getThreadName()); storage->alterFromTiDB( diff --git a/dbms/src/TiDB/Schema/SchemaGetter.h b/dbms/src/TiDB/Schema/SchemaGetter.h index cfa5e1c6335..02d2f7a7c88 100644 --- a/dbms/src/TiDB/Schema/SchemaGetter.h +++ b/dbms/src/TiDB/Schema/SchemaGetter.h @@ -28,6 +28,7 @@ namespace DB { +// The enum results are completely the same as the DDL Action listed in the "parser/model/ddl.go" of TiDB codebase, which must be keeping in sync. enum class SchemaActionType : Int8 { None = 0, From 864cfe9933efdd3caca1d94d1d49e270b5c6285a Mon Sep 17 00:00:00 2001 From: xufei Date: Tue, 14 Jun 2022 17:22:33 +0800 Subject: [PATCH 041/104] Some refinements of `mpp_exchange_receiver_map` and `MPPTunnelSet` (#5132) ref pingcap/tiflash#5095 --- dbms/src/Flash/Coprocessor/DAGContext.cpp | 51 +-------- dbms/src/Flash/Coprocessor/DAGContext.h | 14 +-- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 1 + dbms/src/Flash/Coprocessor/InterpreterDAG.cpp | 17 +-- dbms/src/Flash/Mpp/MPPTask.cpp | 100 +++++++++++++----- dbms/src/Flash/Mpp/MPPTask.h | 12 ++- dbms/src/Flash/Mpp/MPPTunnelSet.cpp | 14 +-- dbms/src/Flash/Mpp/MPPTunnelSet.h | 4 +- dbms/src/Interpreters/Context.cpp | 25 +++++ dbms/src/Interpreters/Context.h | 2 + 10 files changed, 130 insertions(+), 110 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.cpp b/dbms/src/Flash/Coprocessor/DAGContext.cpp index 1736e0b6cec..1ef7338a589 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.cpp +++ b/dbms/src/Flash/Coprocessor/DAGContext.cpp @@ -205,60 +205,13 @@ void DAGContext::attachBlockIO(const BlockIO & io_) { io = io_; } -void DAGContext::initExchangeReceiverIfMPP(Context & context, size_t max_streams) -{ - if (isMPPTask()) - { - if (mpp_exchange_receiver_map_inited) - throw TiFlashException("Repeatedly initialize mpp_exchange_receiver_map", Errors::Coprocessor::Internal); - traverseExecutors(dag_request, [&](const tipb::Executor & executor) { - if (executor.tp() == tipb::ExecType::TypeExchangeReceiver) - { - assert(executor.has_executor_id()); - const auto & executor_id = executor.executor_id(); - // In order to distinguish different exchange receivers. - auto exchange_receiver = std::make_shared( - std::make_shared( - executor.exchange_receiver(), - getMPPTaskMeta(), - context.getTMTContext().getKVCluster(), - context.getTMTContext().getMPPTaskManager(), - context.getSettingsRef().enable_local_tunnel, - context.getSettingsRef().enable_async_grpc_client), - executor.exchange_receiver().encoded_task_meta_size(), - max_streams, - log->identifier(), - executor_id); - mpp_exchange_receiver_map[executor_id] = exchange_receiver; - new_thread_count_of_exchange_receiver += exchange_receiver->computeNewThreadCount(); - } - return true; - }); - mpp_exchange_receiver_map_inited = true; - } -} - const std::unordered_map> & DAGContext::getMPPExchangeReceiverMap() const { if (!isMPPTask()) throw TiFlashException("mpp_exchange_receiver_map is used in mpp only", Errors::Coprocessor::Internal); - if (!mpp_exchange_receiver_map_inited) - throw TiFlashException("mpp_exchange_receiver_map has not been initialized", Errors::Coprocessor::Internal); - return mpp_exchange_receiver_map; -} - -void DAGContext::cancelAllExchangeReceiver() -{ - for (auto & it : mpp_exchange_receiver_map) - { - it.second->cancel(); - } -} - -int DAGContext::getNewThreadCountOfExchangeReceiver() const -{ - return new_thread_count_of_exchange_receiver; + RUNTIME_ASSERT(mpp_exchange_receiver_map != nullptr, log, "MPPTask without exchange receiver map"); + return *mpp_exchange_receiver_map; } bool DAGContext::containsRegionsInfoForTable(Int64 table_id) const diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index c20eb3a367e..07b65b2d8fe 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -37,6 +37,8 @@ namespace DB class Context; class MPPTunnelSet; class ExchangeReceiver; +using ExchangeReceiverMap = std::unordered_map>; +using ExchangeReceiverMapPtr = std::shared_ptr>>; class Join; using JoinPtr = std::shared_ptr; @@ -254,7 +256,6 @@ class DAGContext return io; } - int getNewThreadCountOfExchangeReceiver() const; UInt64 getFlags() const { return flags; @@ -303,10 +304,11 @@ class DAGContext bool columnsForTestEmpty() { return columns_for_test_map.empty(); } - void cancelAllExchangeReceiver(); - - void initExchangeReceiverIfMPP(Context & context, size_t max_streams); const std::unordered_map> & getMPPExchangeReceiverMap() const; + void setMPPExchangeReceiverMap(ExchangeReceiverMapPtr & exchange_receiver_map) + { + mpp_exchange_receiver_map = exchange_receiver_map; + } void addSubquery(const String & subquery_id, SubqueryForSet && subquery); bool hasSubquery() const { return !subqueries.empty(); } @@ -367,10 +369,8 @@ class DAGContext ConcurrentBoundedQueue warnings; /// warning_count is the actual warning count during the entire execution std::atomic warning_count; - int new_thread_count_of_exchange_receiver = 0; /// key: executor_id of ExchangeReceiver nodes in dag. - std::unordered_map> mpp_exchange_receiver_map; - bool mpp_exchange_receiver_map_inited = false; + ExchangeReceiverMapPtr mpp_exchange_receiver_map; /// vector of SubqueriesForSets(such as join build subquery). /// The order of the vector is also the order of the subquery. std::vector subqueries; diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index bea26fe9f99..9ffa29cd14d 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -1432,6 +1432,7 @@ tipb::EncodeType analyzeDAGEncodeType(DAGContext & dag_context) return tipb::EncodeType::TypeDefault; return encode_type; } + tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name) { static std::unordered_map func_name_sig_map = getFuncNameToSigMap(); diff --git a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp index a67ebf20aa5..0e767d65d77 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterDAG.cpp @@ -24,19 +24,8 @@ namespace DB InterpreterDAG::InterpreterDAG(Context & context_, const DAGQuerySource & dag_) : context(context_) , dag(dag_) + , max_streams(context.getMaxStreams()) { - const Settings & settings = context.getSettingsRef(); - if (dagContext().isBatchCop() || (dagContext().isMPPTask() && !dagContext().isTest())) - max_streams = settings.max_threads; - else if (dagContext().isTest()) - max_streams = dagContext().initialize_concurrency; - else - max_streams = 1; - - if (max_streams > 1) - { - max_streams *= settings.max_streams_to_max_threads_ratio; - } } void setRestorePipelineConcurrency(DAGQueryBlock & query_block) @@ -75,10 +64,6 @@ BlockInputStreams InterpreterDAG::executeQueryBlock(DAGQueryBlock & query_block) BlockIO InterpreterDAG::execute() { - /// Due to learner read, DAGQueryBlockInterpreter may take a long time to build - /// the query plan, so we init mpp exchange receiver before executeQueryBlock - dagContext().initExchangeReceiverIfMPP(context, max_streams); - BlockInputStreams streams = executeQueryBlock(*dag.getRootQueryBlock()); DAGPipeline pipeline; pipeline.streams = streams; diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 8f9ca8e55e5..40f03ff79ba 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -22,11 +22,14 @@ #include #include #include +#include +#include #include #include #include #include #include +#include #include #include #include @@ -94,13 +97,73 @@ void MPPTask::run() newThreadManager()->scheduleThenDetach(true, "MPPTask", [self = shared_from_this()] { self->runImpl(); }); } -void MPPTask::registerTunnel(const MPPTaskId & task_id, MPPTunnelPtr tunnel) +void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) { - if (status == CANCELLED) - throw Exception("the tunnel " + tunnel->id() + " can not been registered, because the task is cancelled"); + tunnel_set = std::make_shared(log->identifier()); + std::chrono::seconds timeout(task_request.timeout()); + const auto & exchange_sender = dag_req.root_executor().exchange_sender(); - RUNTIME_ASSERT(tunnel_set != nullptr, log, "mpp task without tunnel set"); - tunnel_set->registerTunnel(task_id, tunnel); + for (int i = 0; i < exchange_sender.encoded_task_meta_size(); ++i) + { + // exchange sender will register the tunnels and wait receiver to found a connection. + mpp::TaskMeta task_meta; + if (unlikely(!task_meta.ParseFromString(exchange_sender.encoded_task_meta(i)))) + throw TiFlashException("Failed to decode task meta info in ExchangeSender", Errors::Coprocessor::BadRequest); + bool is_local = context->getSettingsRef().enable_local_tunnel && meta.address() == task_meta.address(); + bool is_async = !is_local && context->getSettingsRef().enable_async_server; + MPPTunnelPtr tunnel = std::make_shared(task_meta, task_request.meta(), timeout, context->getSettingsRef().max_threads, is_local, is_async, log->identifier()); + LOG_FMT_DEBUG(log, "begin to register the tunnel {}", tunnel->id()); + if (status != INITIALIZING) + throw Exception(fmt::format("The tunnel {} can not be registered, because the task is not in initializing state", tunnel->id())); + tunnel_set->registerTunnel(MPPTaskId{task_meta.start_ts(), task_meta.task_id()}, tunnel); + if (!dag_context->isRootMPPTask()) + { + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_mpp_register_tunnel_for_non_root_mpp_task); + } + } +} + +void MPPTask::initExchangeReceivers() +{ + mpp_exchange_receiver_map = std::make_shared(); + traverseExecutors(&dag_req, [&](const tipb::Executor & executor) { + if (executor.tp() == tipb::ExecType::TypeExchangeReceiver) + { + assert(executor.has_executor_id()); + const auto & executor_id = executor.executor_id(); + // In order to distinguish different exchange receivers. + auto exchange_receiver = std::make_shared( + std::make_shared( + executor.exchange_receiver(), + dag_context->getMPPTaskMeta(), + context->getTMTContext().getKVCluster(), + context->getTMTContext().getMPPTaskManager(), + context->getSettingsRef().enable_local_tunnel, + context->getSettingsRef().enable_async_grpc_client), + executor.exchange_receiver().encoded_task_meta_size(), + context->getMaxStreams(), + log->identifier(), + executor_id); + if (status != RUNNING) + throw Exception("exchange receiver map can not be initialized, because the task is not in running state"); + + (*mpp_exchange_receiver_map)[executor_id] = exchange_receiver; + new_thread_count_of_exchange_receiver += exchange_receiver->computeNewThreadCount(); + } + return true; + }); + dag_context->setMPPExchangeReceiverMap(mpp_exchange_receiver_map); +} + +void MPPTask::cancelAllExchangeReceivers() +{ + if (likely(mpp_exchange_receiver_map != nullptr)) + { + for (auto & it : *mpp_exchange_receiver_map) + { + it.second->cancel(); + } + } } std::pair MPPTask::getTunnel(const ::mpp::EstablishMPPConnectionRequest * request) @@ -116,7 +179,7 @@ std::pair MPPTask::getTunnel(const ::mpp::EstablishMPPConn MPPTaskId receiver_id{request->receiver_meta().start_ts(), request->receiver_meta().task_id()}; RUNTIME_ASSERT(tunnel_set != nullptr, log, "mpp task without tunnel set"); - auto tunnel_ptr = tunnel_set->getTunnelById(receiver_id); + auto tunnel_ptr = tunnel_set->getTunnelByReceiverTaskId(receiver_id); if (tunnel_ptr == nullptr) { auto err_msg = fmt::format( @@ -207,25 +270,8 @@ void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) } // register tunnels - tunnel_set = std::make_shared(log->identifier()); - std::chrono::seconds timeout(task_request.timeout()); + registerTunnels(task_request); - for (int i = 0; i < exchange_sender.encoded_task_meta_size(); i++) - { - // exchange sender will register the tunnels and wait receiver to found a connection. - mpp::TaskMeta task_meta; - if (!task_meta.ParseFromString(exchange_sender.encoded_task_meta(i))) - throw TiFlashException("Failed to decode task meta info in ExchangeSender", Errors::Coprocessor::BadRequest); - bool is_local = context->getSettingsRef().enable_local_tunnel && meta.address() == task_meta.address(); - bool is_async = !is_local && context->getSettingsRef().enable_async_server; - MPPTunnelPtr tunnel = std::make_shared(task_meta, task_request.meta(), timeout, context->getSettingsRef().max_threads, is_local, is_async, log->identifier()); - LOG_FMT_DEBUG(log, "begin to register the tunnel {}", tunnel->id()); - registerTunnel(MPPTaskId{task_meta.start_ts(), task_meta.task_id()}, tunnel); - if (!dag_context->isRootMPPTask()) - { - FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::exception_during_mpp_register_tunnel_for_non_root_mpp_task); - } - } dag_context->tunnel_set = tunnel_set; // register task. auto task_manager = tmt_context.getMPPTaskManager(); @@ -251,6 +297,7 @@ void MPPTask::prepare(const mpp::DispatchTaskRequest & task_request) void MPPTask::preprocess() { auto start_time = Clock::now(); + initExchangeReceivers(); DAGQuerySource dag(*context); executeQuery(dag, *context, false, QueryProcessingStage::Complete); auto end_time = Clock::now(); @@ -280,7 +327,7 @@ void MPPTask::runImpl() LOG_FMT_INFO(log, "task starts preprocessing"); preprocess(); needed_threads = estimateCountOfNewThreads(); - LOG_FMT_DEBUG(log, "Estimate new thread count of query :{} including tunnel_threads: {} , receiver_threads: {}", needed_threads, dag_context->tunnel_set->getRemoteTunnelCnt(), dag_context->getNewThreadCountOfExchangeReceiver()); + LOG_FMT_DEBUG(log, "Estimate new thread count of query :{} including tunnel_threads: {} , receiver_threads: {}", needed_threads, dag_context->tunnel_set->getRemoteTunnelCnt(), new_thread_count_of_exchange_receiver); scheduleOrWait(); @@ -346,8 +393,7 @@ void MPPTask::runImpl() else { context->getProcessList().sendCancelToQuery(context->getCurrentQueryId(), context->getClientInfo().current_user, true); - if (dag_context) - dag_context->cancelAllExchangeReceiver(); + cancelAllExchangeReceivers(); writeErrToAllTunnels(err_msg); } LOG_FMT_INFO(log, "task ends, time cost is {} ms.", stopwatch.elapsedMilliseconds()); diff --git a/dbms/src/Flash/Mpp/MPPTask.h b/dbms/src/Flash/Mpp/MPPTask.h index ee434a2f2ff..c8423ac484c 100644 --- a/dbms/src/Flash/Mpp/MPPTask.h +++ b/dbms/src/Flash/Mpp/MPPTask.h @@ -62,8 +62,6 @@ class MPPTask : public std::enable_shared_from_this void run(); - void registerTunnel(const MPPTaskId & id, MPPTunnelPtr tunnel); - int getNeededThreads(); enum class ScheduleState @@ -107,6 +105,12 @@ class MPPTask : public std::enable_shared_from_this int estimateCountOfNewThreads(); + void registerTunnels(const mpp::DispatchTaskRequest & task_request); + + void initExchangeReceivers(); + + void cancelAllExchangeReceivers(); + tipb::DAGRequest dag_req; ContextPtr context; @@ -122,6 +126,10 @@ class MPPTask : public std::enable_shared_from_this MPPTaskId id; MPPTunnelSetPtr tunnel_set; + /// key: executor_id of ExchangeReceiver nodes in dag. + ExchangeReceiverMapPtr mpp_exchange_receiver_map; + + int new_thread_count_of_exchange_receiver = 0; MPPTaskManager * manager = nullptr; diff --git a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp index 500e9501b08..8d709bb7d38 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSet.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnelSet.cpp @@ -133,12 +133,12 @@ void MPPTunnelSetBase::writeError(const String & msg) } template -void MPPTunnelSetBase::registerTunnel(const MPPTaskId & id, const TunnelPtr & tunnel) +void MPPTunnelSetBase::registerTunnel(const MPPTaskId & receiver_task_id, const TunnelPtr & tunnel) { - if (id_to_index_map.find(id) != id_to_index_map.end()) - throw Exception("the tunnel " + tunnel->id() + " has been registered"); + if (receiver_task_id_to_index_map.find(receiver_task_id) != receiver_task_id_to_index_map.end()) + throw Exception(fmt::format("the tunnel {} has been registered", tunnel->id())); - id_to_index_map[id] = tunnels.size(); + receiver_task_id_to_index_map[receiver_task_id] = tunnels.size(); tunnels.push_back(tunnel); if (!tunnel->isLocal()) { @@ -163,10 +163,10 @@ void MPPTunnelSetBase::finishWrite() } template -typename MPPTunnelSetBase::TunnelPtr MPPTunnelSetBase::getTunnelById(const MPPTaskId & id) +typename MPPTunnelSetBase::TunnelPtr MPPTunnelSetBase::getTunnelByReceiverTaskId(const MPPTaskId & id) { - auto it = id_to_index_map.find(id); - if (it == id_to_index_map.end()) + auto it = receiver_task_id_to_index_map.find(id); + if (it == receiver_task_id_to_index_map.end()) { return nullptr; } diff --git a/dbms/src/Flash/Mpp/MPPTunnelSet.h b/dbms/src/Flash/Mpp/MPPTunnelSet.h index 021c609f516..e4123db1be5 100644 --- a/dbms/src/Flash/Mpp/MPPTunnelSet.h +++ b/dbms/src/Flash/Mpp/MPPTunnelSet.h @@ -59,7 +59,7 @@ class MPPTunnelSetBase : private boost::noncopyable void finishWrite(); void registerTunnel(const MPPTaskId & id, const TunnelPtr & tunnel); - TunnelPtr getTunnelById(const MPPTaskId & id); + TunnelPtr getTunnelByReceiverTaskId(const MPPTaskId & id); uint16_t getPartitionNum() const { return tunnels.size(); } @@ -72,7 +72,7 @@ class MPPTunnelSetBase : private boost::noncopyable private: std::vector tunnels; - std::unordered_map id_to_index_map; + std::unordered_map receiver_task_id_to_index_map; const LoggerPtr log; int remote_tunnel_cnt = 0; diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index a0adef5b50d..3beedbd3601 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -28,6 +28,7 @@ #include #include #include +#include #include #include #include @@ -1879,6 +1880,30 @@ SharedQueriesPtr Context::getSharedQueries() return shared->shared_queries; } +size_t Context::getMaxStreams() const +{ + size_t max_streams = settings.max_threads; + bool is_cop_request = false; + if (dag_context != nullptr) + { + if (dag_context->isTest()) + max_streams = dag_context->initialize_concurrency; + else if (!dag_context->isBatchCop() && !dag_context->isMPPTask()) + { + is_cop_request = true; + max_streams = 1; + } + } + if (max_streams > 1) + max_streams *= settings.max_streams_to_max_threads_ratio; + if (max_streams == 0) + max_streams = 1; + if (unlikely(max_streams != 1 && is_cop_request)) + /// for cop request, the max_streams should be 1 + throw Exception("Cop request only support running with max_streams = 1"); + return max_streams; +} + SessionCleaner::~SessionCleaner() { try diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 5d5c39263c6..b6e759e364b 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -459,6 +459,8 @@ class Context void reloadDeltaTreeConfig(const Poco::Util::AbstractConfiguration & config); + size_t getMaxStreams() const; + private: /** Check if the current client has access to the specified database. * If access is denied, throw an exception. From a79ad91e8b3b8fe8da6b447f4ab46206e94a3971 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger <60744015+Lloyd-Pottiger@users.noreply.github.com> Date: Wed, 15 Jun 2022 14:18:33 +0800 Subject: [PATCH 042/104] Revise default background threads size (#4723) close pingcap/tiflash#4702 --- dbms/src/Core/Defines.h | 1 - dbms/src/Interpreters/Context.cpp | 24 ++++++++++---- dbms/src/Interpreters/Context.h | 4 ++- dbms/src/Interpreters/Settings.h | 6 ++-- dbms/src/Server/Server.cpp | 33 +++++++++++-------- .../src/Storages/BackgroundProcessingPool.cpp | 3 ++ dbms/src/Storages/BackgroundProcessingPool.h | 4 +-- dbms/src/TestUtils/TiFlashTestEnv.cpp | 6 ++++ 8 files changed, 54 insertions(+), 27 deletions(-) diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index 33d116dae33..75f6f16bb25 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -78,7 +78,6 @@ /// too short a period can cause errors to disappear immediately after creation. #define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD (2 * DBMS_DEFAULT_SEND_TIMEOUT_SEC) #define DEFAULT_QUERIES_QUEUE_WAIT_TIME_MS 5000 /// Maximum waiting time in the request queue. -#define DBMS_DEFAULT_BACKGROUND_POOL_SIZE 16 #define DBMS_MIN_REVISION_WITH_CLIENT_INFO 54032 #define DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE 54058 diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 3beedbd3601..a0345daaa75 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -68,10 +68,8 @@ #include #include -#include #include -#include - +#include namespace ProfileEvents { @@ -1443,19 +1441,33 @@ void Context::dropCaches() const } BackgroundProcessingPool & Context::getBackgroundPool() +{ + // Note: shared->background_pool should be initialized first. + auto lock = getLock(); + return *shared->background_pool; +} + +BackgroundProcessingPool & Context::initializeBackgroundPool(UInt16 pool_size) { auto lock = getLock(); if (!shared->background_pool) - shared->background_pool = std::make_shared(settings.background_pool_size); + shared->background_pool = std::make_shared(pool_size); return *shared->background_pool; } BackgroundProcessingPool & Context::getBlockableBackgroundPool() { - // TODO: choose a better thread pool size and maybe a better name for the pool + // TODO: maybe a better name for the pool + // Note: shared->blockable_background_pool should be initialized first. + auto lock = getLock(); + return *shared->blockable_background_pool; +} + +BackgroundProcessingPool & Context::initializeBlockableBackgroundPool(UInt16 pool_size) +{ auto lock = getLock(); if (!shared->blockable_background_pool) - shared->blockable_background_pool = std::make_shared(settings.background_pool_size); + shared->blockable_background_pool = std::make_shared(pool_size); return *shared->blockable_background_pool; } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index b6e759e364b..434179e1ab8 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -380,7 +380,9 @@ class Context bool useL0Opt() const; BackgroundProcessingPool & getBackgroundPool(); + BackgroundProcessingPool & initializeBackgroundPool(UInt16 pool_size); BackgroundProcessingPool & getBlockableBackgroundPool(); + BackgroundProcessingPool & initializeBlockableBackgroundPool(UInt16 pool_size); void createTMTContext(const TiFlashRaftConfig & raft_config, pingcap::ClusterConfig && cluster_config); @@ -505,7 +507,7 @@ class DDLGuard class SessionCleaner { public: - SessionCleaner(Context & context_) + explicit SessionCleaner(Context & context_) : context{context_} {} ~SessionCleaner(); diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 9361e0525d2..f2b3bbbd7fe 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -80,8 +80,8 @@ struct Settings M(SettingBool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.") \ M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.") \ M(SettingBool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.") \ - M(SettingUInt64, background_pool_size, DBMS_DEFAULT_BACKGROUND_POOL_SIZE, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server " \ - "startup.") \ + M(SettingUInt64, background_pool_size, 0, "Number of threads performing background work for tables (for example, merging in merge tree). Only effective at server startup. " \ + "0 means a quarter of the number of logical CPU cores of the machine.") \ \ M(SettingBool, optimize_move_to_prewhere, true, "Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree.") \ \ @@ -356,7 +356,7 @@ struct Settings M(SettingUInt64, elastic_threadpool_shrink_period_ms, 300000, "The shrink period(ms) of elastic thread pool.") \ M(SettingBool, enable_local_tunnel, true, "Enable local data transfer between local MPP tasks.") \ M(SettingBool, enable_async_grpc_client, true, "Enable async grpc in MPP.") \ - M(SettingUInt64, grpc_completion_queue_pool_size, 0, "The size of gRPC completion queue pool. 0 means using hardware_concurrency.")\ + M(SettingUInt64, grpc_completion_queue_pool_size, 0, "The size of gRPC completion queue pool. 0 means the number of logical CPU cores. Only effective at server startup")\ M(SettingBool, enable_async_server, true, "Enable async rpc server.") \ M(SettingUInt64, async_pollers_per_cq, 200, "grpc async pollers per cqs") \ M(SettingUInt64, async_cqs, 1, "grpc async cqs") \ diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 3e2c29de76c..95c1d5d3f2a 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -53,10 +53,15 @@ #include #include #include +#include +#include +#include #include #include #include +#include #include +#include #include #include #include @@ -81,12 +86,6 @@ #include #include -#include "HTTPHandlerFactory.h" -#include "MetricsPrometheus.h" -#include "MetricsTransmitter.h" -#include "StatusFile.h" -#include "TCPHandlerFactory.h" - #if Poco_NetSSL_FOUND #include #include @@ -1135,6 +1134,19 @@ int Server::main(const std::vector & /*args*/) global_context->getPathCapacity(), global_context->getFileProvider()); + /// if default value of background_pool_size is 0 + /// set it to the a quarter of the number of logical CPU cores of machine. + Settings & settings = global_context->getSettingsRef(); + if (settings.background_pool_size == 0) + { + global_context->setSetting("background_pool_size", std::to_string(server_info.cpu_info.logical_cores / 4)); + } + LOG_FMT_INFO(log, "Background & Blockable Background pool size: {}", settings.background_pool_size); + + /// Initialize the background & blockable background thread pool. + auto & bg_pool = global_context->initializeBackgroundPool(settings.background_pool_size); + auto & blockable_bg_pool = global_context->initializeBlockableBackgroundPool(settings.background_pool_size); + global_context->initializePageStorageMode(global_context->getPathPool(), STORAGE_FORMAT_CURRENT.page); global_context->initializeGlobalStoragePoolIfNeed(global_context->getPathPool()); LOG_FMT_INFO(log, "Global PageStorage run mode is {}", static_cast(global_context->getPageStorageRunMode())); @@ -1251,13 +1263,6 @@ int Server::main(const std::vector & /*args*/) /// Load global settings from default_profile and system_profile. /// It internally depends on UserConfig::parseSettings. global_context->setDefaultProfiles(config()); - Settings & settings = global_context->getSettingsRef(); - - /// Initialize the background thread pool. - /// It internally depends on settings.background_pool_size, - /// so must be called after settings has been load. - auto & bg_pool = global_context->getBackgroundPool(); - auto & blockable_bg_pool = global_context->getBlockableBackgroundPool(); /// Initialize RateLimiter. global_context->initializeRateLimiter(config(), bg_pool, blockable_bg_pool); @@ -1409,7 +1414,7 @@ int Server::main(const std::vector & /*args*/) { auto size = settings.grpc_completion_queue_pool_size; if (size == 0) - size = std::thread::hardware_concurrency(); + size = server_info.cpu_info.logical_cores; GRPCCompletionQueuePool::global_instance = std::make_unique(size); } diff --git a/dbms/src/Storages/BackgroundProcessingPool.cpp b/dbms/src/Storages/BackgroundProcessingPool.cpp index 96c2c6cc622..9fb4271ea38 100644 --- a/dbms/src/Storages/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/BackgroundProcessingPool.cpp @@ -87,6 +87,9 @@ BackgroundProcessingPool::BackgroundProcessingPool(int size_) : size(size_) , thread_ids_counter(size_) { + if (size <= 0) + throw Exception("BackgroundProcessingPool size must be greater than 0", ErrorCodes::LOGICAL_ERROR); + LOG_FMT_INFO(&Poco::Logger::get("BackgroundProcessingPool"), "Create BackgroundProcessingPool with {} threads", size); threads.resize(size); diff --git a/dbms/src/Storages/BackgroundProcessingPool.h b/dbms/src/Storages/BackgroundProcessingPool.h index 1ba6c4efcf8..49a01b3a397 100644 --- a/dbms/src/Storages/BackgroundProcessingPool.h +++ b/dbms/src/Storages/BackgroundProcessingPool.h @@ -81,7 +81,7 @@ class BackgroundProcessingPool using TaskHandle = std::shared_ptr; - BackgroundProcessingPool(int size_); + explicit BackgroundProcessingPool(int size_); size_t getNumberOfThreads() const { return size; } @@ -96,7 +96,7 @@ class BackgroundProcessingPool /// 2. thread B also get the same task /// 3. thread A finish the execution of the task quickly, release the task and try to update the next schedule time of the task /// 4. thread B find the task is not occupied and execute the task again almost immediately - TaskHandle addTask(const Task & task, const bool multi = true, const size_t interval_ms = 0); + TaskHandle addTask(const Task & task, bool multi = true, size_t interval_ms = 0); void removeTask(const TaskHandle & task); ~BackgroundProcessingPool(); diff --git a/dbms/src/TestUtils/TiFlashTestEnv.cpp b/dbms/src/TestUtils/TiFlashTestEnv.cpp index cbd42b57550..a7bcfe43d7a 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.cpp +++ b/dbms/src/TestUtils/TiFlashTestEnv.cpp @@ -24,6 +24,8 @@ #include #include +#include + namespace DB::tests { std::unique_ptr TiFlashTestEnv::global_context = nullptr; @@ -39,6 +41,10 @@ void TiFlashTestEnv::initializeGlobalContext(Strings testdata_path, PageStorageR KeyManagerPtr key_manager = std::make_shared(false); global_context->initializeFileProvider(key_manager, false); + // initialize background & blockable background thread pool + global_context->initializeBackgroundPool(std::thread::hardware_concurrency() / 4); + global_context->initializeBlockableBackgroundPool(std::thread::hardware_concurrency() / 4); + // Theses global variables should be initialized by the following order // 1. capacity // 2. path pool From 617fe546febdf939c1a40aeefacbd22592ef757c Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Thu, 16 Jun 2022 11:26:34 +0800 Subject: [PATCH 043/104] add microbenchmark for exchange and window function (#5137) close pingcap/tiflash#4276, close pingcap/tiflash#5138 --- dbms/CMakeLists.txt | 1 + dbms/src/Debug/astToExecutor.h | 2 +- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 2 + .../Coprocessor/DAGQueryBlockInterpreter.h | 2 + dbms/src/Flash/tests/CMakeLists.txt | 11 - dbms/src/Flash/tests/WindowTestUtil.h | 81 ++ dbms/src/Flash/tests/bench_exchange.cpp | 407 ++++++++++ dbms/src/Flash/tests/bench_exchange.h | 291 ++++++++ dbms/src/Flash/tests/bench_window.cpp | 107 +++ dbms/src/Flash/tests/exchange_perftest.cpp | 699 ------------------ dbms/src/TestUtils/mockExecutor.cpp | 3 +- dbms/src/TestUtils/mockExecutor.h | 2 +- libs/libcommon/include/common/types.h | 1 + 13 files changed, 896 insertions(+), 713 deletions(-) create mode 100644 dbms/src/Flash/tests/WindowTestUtil.h create mode 100644 dbms/src/Flash/tests/bench_exchange.cpp create mode 100644 dbms/src/Flash/tests/bench_exchange.h create mode 100644 dbms/src/Flash/tests/bench_window.cpp delete mode 100644 dbms/src/Flash/tests/exchange_perftest.cpp diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index e1e52fab73b..0df79f89a84 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -316,6 +316,7 @@ if (ENABLE_TESTS) ${TiFlash_SOURCE_DIR}/dbms/src/AggregateFunctions/AggregateFunctionSum.cpp ) target_include_directories(bench_dbms BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR} ${benchmark_SOURCE_DIR}/include) + target_compile_definitions(bench_dbms PUBLIC DBMS_PUBLIC_GTEST) target_link_libraries(bench_dbms gtest dbms test_util_bench_main benchmark clickhouse_functions) if (ENABLE_TIFLASH_DTWORKLOAD) diff --git a/dbms/src/Debug/astToExecutor.h b/dbms/src/Debug/astToExecutor.h index cbd2e5ade3a..4d87c0db77e 100644 --- a/dbms/src/Debug/astToExecutor.h +++ b/dbms/src/Debug/astToExecutor.h @@ -350,4 +350,4 @@ ExecutorPtr compileWindow(ExecutorPtr input, size_t & executor_index, ASTPtr fun ExecutorPtr compileSort(ExecutorPtr input, size_t & executor_index, ASTPtr order_by_expr_list, bool is_partial_sort); void literalFieldToTiPBExpr(const ColumnInfo & ci, const Field & field, tipb::Expr * expr, Int32 collator_id); -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 9f201006a88..046088ab2b2 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -153,7 +153,9 @@ class DAGExpressionAnalyzer : private boost::noncopyable const tipb::Window & window, size_t window_columns_start_index); +#ifndef DBMS_PUBLIC_GTEST private: +#endif NamesAndTypes buildOrderColumns( const ExpressionActionsPtr & actions, const ::google::protobuf::RepeatedPtrField & order_by); diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 0b3b2db9623..e68c4f91cee 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -54,7 +54,9 @@ class DAGQueryBlockInterpreter BlockInputStreams execute(); +#ifndef DBMS_PUBLIC_GTEST private: +#endif void executeImpl(DAGPipeline & pipeline); void handleMockTableScan(const TiDBTableScan & table_scan, DAGPipeline & pipeline); void handleTableScan(const TiDBTableScan & table_scan, DAGPipeline & pipeline); diff --git a/dbms/src/Flash/tests/CMakeLists.txt b/dbms/src/Flash/tests/CMakeLists.txt index a34e4b23432..944908dcb25 100644 --- a/dbms/src/Flash/tests/CMakeLists.txt +++ b/dbms/src/Flash/tests/CMakeLists.txt @@ -13,14 +13,3 @@ # limitations under the License. include_directories (${CMAKE_CURRENT_BINARY_DIR}) - -add_executable (exchange_perftest - exchange_perftest.cpp - ${TiFlash_SOURCE_DIR}/dbms/src/Server/StorageConfigParser.cpp - ${TiFlash_SOURCE_DIR}/dbms/src/Functions/FunctionsConversion.cpp) -target_link_libraries (exchange_perftest - gtest_main - dbms - clickhouse_functions - clickhouse_aggregate_functions - tiflash-dttool-lib) diff --git a/dbms/src/Flash/tests/WindowTestUtil.h b/dbms/src/Flash/tests/WindowTestUtil.h new file mode 100644 index 00000000000..3f4cb7d595f --- /dev/null +++ b/dbms/src/Flash/tests/WindowTestUtil.h @@ -0,0 +1,81 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include + +namespace DB +{ +namespace tests +{ + +inline std::shared_ptr mockInterpreter(Context & context, const std::vector & source_columns, int concurrency) +{ + std::vector mock_input_streams_vec = {}; + DAGQueryBlock mock_query_block(0, static_cast>(nullptr)); + std::vector mock_subqueries_for_sets = {}; + std::shared_ptr mock_interpreter = std::make_shared(context, + mock_input_streams_vec, + mock_query_block, + concurrency); + mock_interpreter->analyzer = std::make_unique(std::move(source_columns), context); + return mock_interpreter; +} + +inline void mockExecuteProject(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, NamesWithAliases & final_project) +{ + mock_interpreter->executeProject(pipeline, final_project); +} + +inline void mockExecuteWindowOrder(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, const tipb::Sort & sort) +{ + mock_interpreter->handleWindowOrder(pipeline, sort); + mock_interpreter->input_streams_vec[0] = pipeline.streams; + NamesWithAliases final_project; + for (const auto & column : (*mock_interpreter->analyzer).source_columns) + { + final_project.push_back({column.name, ""}); + } + mockExecuteProject(mock_interpreter, pipeline, final_project); +} + +inline void mockExecuteWindowOrder(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, const String & sort_json) +{ + tipb::Sort sort; + ::google::protobuf::util::JsonStringToMessage(sort_json, &sort); + mockExecuteWindowOrder(mock_interpreter, pipeline, sort); +} + +inline void mockExecuteWindow(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, const tipb::Window & window) +{ + mock_interpreter->handleWindow(pipeline, window); + mock_interpreter->input_streams_vec[0] = pipeline.streams; + NamesWithAliases final_project; + for (const auto & column : (*mock_interpreter->analyzer).source_columns) + { + final_project.push_back({column.name, ""}); + } + mockExecuteProject(mock_interpreter, pipeline, final_project); +} + +inline void mockExecuteWindow(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, std::string window_json_str) +{ + tipb::Window window; + google::protobuf::util::JsonStringToMessage(window_json_str, &window); + mockExecuteWindow(mock_interpreter, pipeline, window); +} + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Flash/tests/bench_exchange.cpp b/dbms/src/Flash/tests/bench_exchange.cpp new file mode 100644 index 00000000000..fbb53bfd4a4 --- /dev/null +++ b/dbms/src/Flash/tests/bench_exchange.cpp @@ -0,0 +1,407 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +#include // to include the implementation of StreamingDAGResponseWriter +#include // to include the implementation of ExchangeReceiver +#include // to include the implementation of MPPTunnel +#include // to include the implementation of MPPTunnelSet +#include +#include + + +namespace DB +{ +namespace tests +{ + +std::random_device rd; + +MockBlockInputStream::MockBlockInputStream(const std::vector & blocks_, StopFlag & stop_flag_) + : blocks(blocks_) + , header(blocks[0].cloneEmpty()) + , mt(rd()) + , dist(0, blocks.size() - 1) + , stop_flag(stop_flag_) +{} + +MockFixedRowsBlockInputStream::MockFixedRowsBlockInputStream(size_t total_rows_, const std::vector & blocks_) + : header(blocks_[0].cloneEmpty()) + , mt(rd()) + , dist(0, blocks_.size() - 1) + , current_rows(0) + , total_rows(total_rows_) + , blocks(blocks_) +{} + +Block makeBlock(int row_num) +{ + std::mt19937 mt(rd()); + std::uniform_int_distribution int64_dist; + std::uniform_int_distribution len_dist(10, 20); + std::uniform_int_distribution char_dist; + + InferredDataVector> int64_vec; + InferredDataVector> int64_vec2; + for (int i = 0; i < row_num; ++i) + { + int64_vec.emplace_back(int64_dist(mt)); + int64_vec2.emplace_back(int64_dist(mt)); + } + + InferredDataVector> string_vec; + for (int i = 0; i < row_num; ++i) + { + int len = len_dist(mt); + String s; + for (int j = 0; j < len; ++j) + s.push_back(char_dist(mt)); + string_vec.push_back(std::move(s)); + } + + auto int64_data_type = makeDataType>(); + ColumnWithTypeAndName int64_column(makeColumn>(int64_data_type, int64_vec), int64_data_type, "int64_1"); + ColumnWithTypeAndName int64_column2(makeColumn>(int64_data_type, int64_vec2), int64_data_type, "int64_2"); + + auto string_data_type = makeDataType>(); + ColumnWithTypeAndName string_column(makeColumn>(string_data_type, string_vec), string_data_type, "string"); + + return Block({int64_column, string_column, int64_column2}); +} + +std::vector makeBlocks(int block_num, int row_num) +{ + std::vector blocks; + for (int i = 0; i < block_num; ++i) + blocks.push_back(makeBlock(row_num)); + return blocks; +} + +mpp::MPPDataPacket makePacket(ChunkCodecStream & codec, int row_num) +{ + auto block = makeBlock(row_num); + codec.encode(block, 0, row_num); + + mpp::MPPDataPacket packet; + packet.add_chunks(codec.getString()); + codec.clear(); + + return packet; +} + +std::vector makePackets(ChunkCodecStream & codec, int packet_num, int row_num) +{ + std::vector packets; + for (int i = 0; i < packet_num; ++i) + packets.push_back(std::make_shared(makePacket(codec, row_num))); + return packets; +} + +std::vector makePacketQueues(int source_num, int queue_size) +{ + std::vector queues(source_num); + for (int i = 0; i < source_num; ++i) + queues[i] = std::make_shared(queue_size); + return queues; +} + +std::vector makeFields() +{ + std::vector fields(3); + fields[0].set_tp(TiDB::TypeLongLong); + fields[1].set_tp(TiDB::TypeString); + fields[2].set_tp(TiDB::TypeLongLong); + return fields; +} + +void printException(const Exception & e) +{ + std::string text = e.displayText(); + + auto embedded_stack_trace_pos = text.find("Stack trace"); + std::cerr << "Code: " << e.code() << ". " << text << std::endl + << std::endl; + if (std::string::npos == embedded_stack_trace_pos) + std::cerr << "Stack trace:" << std::endl + << e.getStackTrace().toString() << std::endl; +} + +void sendPacket(const std::vector & packets, const PacketQueuePtr & queue, StopFlag & stop_flag) +{ + std::mt19937 mt(rd()); + std::uniform_int_distribution dist(0, packets.size() - 1); + + while (!stop_flag.load()) + { + int i = dist(mt); + queue->tryPush(packets[i], std::chrono::milliseconds(10)); + } + queue->finish(); +} + +void receivePacket(const PacketQueuePtr & queue) +{ + while (true) + { + PacketPtr packet; + if (!queue->pop(packet)) + break; + } +} + +ReceiverHelper::ReceiverHelper(int concurrency_, int source_num_) + : concurrency(concurrency_) + , source_num(source_num_) +{ + pb_exchange_receiver.set_tp(tipb::Hash); + for (int i = 0; i < source_num; ++i) + { + mpp::TaskMeta task; + task.set_start_ts(0); + task.set_task_id(i); + task.set_partition_id(i); + task.set_address(""); + + String encoded_task; + task.SerializeToString(&encoded_task); + + pb_exchange_receiver.add_encoded_task_meta(encoded_task); + } + + fields = makeFields(); + *pb_exchange_receiver.add_field_types() = fields[0]; + *pb_exchange_receiver.add_field_types() = fields[1]; + *pb_exchange_receiver.add_field_types() = fields[2]; + + task_meta.set_task_id(100); + + queues = makePacketQueues(source_num, 10); +} + +MockExchangeReceiverPtr ReceiverHelper::buildReceiver() +{ + return std::make_shared( + std::make_shared(queues, fields), + source_num, + concurrency, + "mock_req_id", + "mock_exchange_receiver_id"); +} + +std::vector ReceiverHelper::buildExchangeReceiverStream() +{ + auto receiver = buildReceiver(); + std::vector streams(concurrency); + for (int i = 0; i < concurrency; ++i) + { + streams[i] = std::make_shared(receiver, "mock_req_id", "mock_executor_id" + std::to_string(i)); + } + return streams; +} + +BlockInputStreamPtr ReceiverHelper::buildUnionStream() +{ + auto streams = buildExchangeReceiverStream(); + return std::make_shared>(streams, nullptr, concurrency, /*req_id=*/""); +} + +void ReceiverHelper::finish() +{ + if (join_ptr) + { + join_ptr->setBuildTableState(Join::BuildTableState::SUCCEED); + std::cout << fmt::format("Hash table size: {} bytes", join_ptr->getTotalByteCount()) << std::endl; + } +} + +SenderHelper::SenderHelper( + int source_num_, + int concurrency_, + const std::vector & queues_, + const std::vector & fields) + : source_num(source_num_) + , concurrency(concurrency_) + , queues(queues_) +{ + mpp::TaskMeta task_meta; + tunnel_set = std::make_shared("mock_req_id"); + for (int i = 0; i < source_num; ++i) + { + auto writer = std::make_shared(queues[i]); + mock_writers.push_back(writer); + + auto tunnel = std::make_shared( + task_meta, + task_meta, + std::chrono::seconds(60), + concurrency, + false, + false, + "mock_req_id"); + tunnel->connect(writer.get()); + tunnels.push_back(tunnel); + MPPTaskId id(0, i); + tunnel_set->registerTunnel(id, tunnel); + } + + tipb::DAGRequest dag_request; + tipb::Executor root_executor; + root_executor.set_executor_id("ExchangeSender_100"); + *dag_request.mutable_root_executor() = root_executor; + + dag_context = std::make_unique(dag_request); + dag_context->is_mpp_task = true; + dag_context->is_root_mpp_task = false; + dag_context->encode_type = tipb::EncodeType::TypeCHBlock; + dag_context->result_field_types = fields; +} + +BlockInputStreamPtr SenderHelper::buildUnionStream( + StopFlag & stop_flag, + const std::vector & blocks) +{ + std::vector send_streams; + for (int i = 0; i < concurrency; ++i) + { + BlockInputStreamPtr stream = std::make_shared(blocks, stop_flag); + std::unique_ptr response_writer( + new StreamingDAGResponseWriter( + tunnel_set, + {0, 1, 2}, + TiDB::TiDBCollators(3), + tipb::Hash, + -1, + -1, + true, + *dag_context)); + send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); + } + + return std::make_shared>(send_streams, nullptr, concurrency, /*req_id=*/""); +} + +BlockInputStreamPtr SenderHelper::buildUnionStream(size_t total_rows, const std::vector & blocks) +{ + std::vector send_streams; + for (int i = 0; i < concurrency; ++i) + { + BlockInputStreamPtr stream = std::make_shared(total_rows / concurrency, blocks); + std::unique_ptr response_writer( + new StreamingDAGResponseWriter( + tunnel_set, + {0, 1, 2}, + TiDB::TiDBCollators(3), + tipb::Hash, + -1, + -1, + true, + *dag_context)); + send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); + } + + return std::make_shared>(send_streams, nullptr, concurrency, /*req_id=*/""); +} + +void SenderHelper::finish() +{ + for (size_t i = 0; i < tunnels.size(); ++i) + { + tunnels[i]->writeDone(); + tunnels[i]->waitForFinish(); + mock_writers[i]->finish(); + } +} + +void ExchangeBench::SetUp(const benchmark::State &) +{ + Poco::Logger::root().setLevel("error"); + + DynamicThreadPool::global_instance = std::make_unique( + /*fixed_thread_num=*/300, + std::chrono::milliseconds(100000)); + + input_blocks = makeBlocks(/*block_num=*/100, /*row_num=*/1024); + + try + { + DB::registerWindowFunctions(); + DB::registerFunctions(); + } + catch (DB::Exception &) + { + // Maybe another test has already registered, ignore exception here. + } +} + +void ExchangeBench::TearDown(const benchmark::State &) +{ + input_blocks.clear(); + // NOTE: Must reset here, otherwise DynamicThreadPool::fixedWork() may core because metrics already destroyed. + DynamicThreadPool::global_instance.reset(); +} + +void ExchangeBench::runAndWait(std::shared_ptr receiver_helper, + BlockInputStreamPtr receiver_stream, + std::shared_ptr & sender_helper, + BlockInputStreamPtr sender_stream) +{ + std::future sender_future = DynamicThreadPool::global_instance->schedule(/*memory_tracker=*/false, + [sender_stream, sender_helper] { + sender_stream->readPrefix(); + while (const auto & block = sender_stream->read()) {} + sender_stream->readSuffix(); + sender_helper->finish(); + }); + std::future receiver_future = DynamicThreadPool::global_instance->schedule(/*memory_tracker=*/false, + [receiver_stream, receiver_helper] { + receiver_stream->readPrefix(); + while (const auto & block = receiver_stream->read()) {} + receiver_stream->readSuffix(); + receiver_helper->finish(); + }); + sender_future.get(); + receiver_future.get(); +} + +BENCHMARK_DEFINE_F(ExchangeBench, basic_send_receive) +(benchmark::State & state) +try +{ + const int concurrency = state.range(0); + const int source_num = state.range(1); + const int total_rows = state.range(2); + Context context = TiFlashTestEnv::getContext(); + + for (auto _ : state) + { + std::shared_ptr receiver_helper = std::make_shared(concurrency, source_num); + BlockInputStreamPtr receiver_stream = receiver_helper->buildUnionStream(); + + std::shared_ptr sender_helper = std::make_shared(source_num, + concurrency, + receiver_helper->queues, + receiver_helper->fields); + BlockInputStreamPtr sender_stream = sender_helper->buildUnionStream(total_rows, input_blocks); + + runAndWait(receiver_helper, receiver_stream, sender_helper, sender_stream); + } +} +CATCH +BENCHMARK_REGISTER_F(ExchangeBench, basic_send_receive) + ->Args({8, 1, 1024 * 1000}); + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Flash/tests/bench_exchange.h b/dbms/src/Flash/tests/bench_exchange.h new file mode 100644 index 00000000000..6b09e319613 --- /dev/null +++ b/dbms/src/Flash/tests/bench_exchange.h @@ -0,0 +1,291 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +namespace tests +{ + + +using Packet = mpp::MPPDataPacket; +using PacketPtr = std::shared_ptr; +using PacketQueue = MPMCQueue; +using PacketQueuePtr = std::shared_ptr; +using StopFlag = std::atomic; + +// NOLINTBEGIN(readability-convert-member-functions-to-static) +struct MockReceiverContext +{ + using Status = ::grpc::Status; + struct Request + { + String debugString() const + { + return "{Request}"; + } + + int source_index = 0; + int send_task_id = 0; + int recv_task_id = -1; + }; + + struct Reader + { + explicit Reader(const PacketQueuePtr & queue_) + : queue(queue_) + {} + + void initialize() const {} + + bool read(PacketPtr & packet [[maybe_unused]]) const + { + PacketPtr res; + if (queue->pop(res)) + { + *packet = *res; // avoid change shared packets + return true; + } + return false; + } + + Status finish() const + { + return ::grpc::Status(); + } + + PacketQueuePtr queue; + }; + + struct MockAsyncGrpcExchangePacketReader + { + // Not implement benchmark for Async GRPC for now. + void init(UnaryCallback *) { assert(0); } + void read(MPPDataPacketPtr &, UnaryCallback *) { assert(0); } + void finish(::grpc::Status &, UnaryCallback *) { assert(0); } + }; + + using AsyncReader = MockAsyncGrpcExchangePacketReader; + + MockReceiverContext( + const std::vector & queues_, + const std::vector & field_types_) + : queues(queues_) + , field_types(field_types_) + {} + + void fillSchema(DAGSchema & schema) const + { + schema.clear(); + for (size_t i = 0; i < field_types.size(); ++i) + { + String name = "exchange_receiver_" + std::to_string(i); + ColumnInfo info = TiDB::fieldTypeToColumnInfo(field_types[i]); + schema.emplace_back(std::move(name), std::move(info)); + } + } + + Request makeRequest(int index) const + { + return {index, index, -1}; + } + + std::shared_ptr makeReader(const Request & request) + { + return std::make_shared(queues[request.send_task_id]); + } + + static Status getStatusOK() + { + return ::grpc::Status(); + } + + bool supportAsync(const Request &) const { return false; } + void makeAsyncReader( + const Request &, + std::shared_ptr &, + UnaryCallback *) const {} + + std::vector queues; + std::vector field_types; +}; +// NOLINTEND(readability-convert-member-functions-to-static) + +using MockExchangeReceiver = ExchangeReceiverBase; +using MockExchangeReceiverPtr = std::shared_ptr; +using MockExchangeReceiverInputStream = TiRemoteBlockInputStream; + +struct MockWriter : public PacketWriter +{ + explicit MockWriter(PacketQueuePtr queue_) + : queue(std::move(queue_)) + {} + + bool write(const Packet & packet) override + { + queue->push(std::make_shared(packet)); + return true; + } + + void finish() + { + queue->finish(); + } + + PacketQueuePtr queue; +}; + +using MockWriterPtr = std::shared_ptr; +using MockTunnel = MPPTunnelBase; +using MockTunnelPtr = std::shared_ptr; +using MockTunnelSet = MPPTunnelSetBase; +using MockTunnelSetPtr = std::shared_ptr; + +struct MockBlockInputStream : public IProfilingBlockInputStream +{ + const std::vector & blocks; + Block header; + std::mt19937 mt; + std::uniform_int_distribution dist; + StopFlag & stop_flag; + + MockBlockInputStream(const std::vector & blocks_, StopFlag & stop_flag_); + + String getName() const override { return "MockBlockInputStream"; } + Block getHeader() const override { return header; } + + Block readImpl() override + { + if (stop_flag.load(std::memory_order_relaxed)) + return Block{}; + return blocks[dist(mt)]; + } +}; + +// Similar to MockBlockInputStream, but return fixed count of rows. +struct MockFixedRowsBlockInputStream : public IProfilingBlockInputStream +{ + Block header; + std::mt19937 mt; + std::uniform_int_distribution dist; + size_t current_rows; + size_t total_rows; + const std::vector & blocks; + + MockFixedRowsBlockInputStream(size_t total_rows_, const std::vector & blocks_); + + String getName() const override { return "MockBlockInputStream"; } + Block getHeader() const override { return header; } + + Block readImpl() override + { + if (current_rows >= total_rows) + return Block{}; + Block res = blocks[dist(mt)]; + current_rows += res.rows(); + return res; + } +}; + +Block makeBlock(int row_num); +std::vector makeBlocks(int block_num, int row_num); +mpp::MPPDataPacket makePacket(ChunkCodecStream & codec, int row_num); +std::vector makePackets(ChunkCodecStream & codec, int packet_num, int row_num); +std::vector makePacketQueues(int source_num, int queue_size); +std::vector makeFields(); +void printException(const Exception & e); +void sendPacket(const std::vector & packets, const PacketQueuePtr & queue, StopFlag & stop_flag); +void receivePacket(const PacketQueuePtr & queue); + +struct ReceiverHelper +{ + const int concurrency; + const int source_num; + tipb::ExchangeReceiver pb_exchange_receiver; + std::vector fields; + mpp::TaskMeta task_meta; + std::vector queues; + std::shared_ptr join_ptr; + + explicit ReceiverHelper(int concurrency_, int source_num_); + MockExchangeReceiverPtr buildReceiver(); + std::vector buildExchangeReceiverStream(); + BlockInputStreamPtr buildUnionStream(); + BlockInputStreamPtr buildUnionStreamWithHashJoinBuildStream(); + void finish(); +}; + +struct SenderHelper +{ + const int source_num; + const int concurrency; + + std::vector queues; + std::vector mock_writers; + std::vector tunnels; + MockTunnelSetPtr tunnel_set; + std::unique_ptr dag_context; + + SenderHelper( + int source_num_, + int concurrency_, + const std::vector & queues_, + const std::vector & fields); + + // Using MockBlockInputStream to build streams. + BlockInputStreamPtr buildUnionStream(StopFlag & stop_flag, const std::vector & blocks); + // Using MockFixedRowsBlockInputStream to build streams. + BlockInputStreamPtr buildUnionStream(size_t total_rows, const std::vector & blocks); + + void finish(); +}; + +class ExchangeBench : public benchmark::Fixture +{ +public: + void SetUp(const benchmark::State &) override; + void TearDown(const benchmark::State &) override; + void runAndWait(std::shared_ptr receiver_helper, + BlockInputStreamPtr receiver_stream, + std::shared_ptr & sender_helper, + BlockInputStreamPtr sender_stream); + + std::vector input_blocks; +}; + + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Flash/tests/bench_window.cpp b/dbms/src/Flash/tests/bench_window.cpp new file mode 100644 index 00000000000..da9df20fdf3 --- /dev/null +++ b/dbms/src/Flash/tests/bench_window.cpp @@ -0,0 +1,107 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +namespace DB +{ +namespace tests +{ +class WindowFunctionBench : public ExchangeBench +{ +public: + void SetUp(const benchmark::State & state) override + { + // build tipb::Window and tipb::Sort. + // select row_number() over w1 from t1 window w1 as (partition by c1, c2, c3 order by c1, c2, c3); + ExchangeBench::SetUp(state); + MockColumnInfos columns{ + {"c1", TiDB::TP::TypeLongLong}, + {"c2", TiDB::TP::TypeString}, + {"c3", TiDB::TP::TypeLongLong}, + }; + size_t executor_index = 0; + DAGRequestBuilder builder(executor_index); + builder + .mockTable("test", "t1", columns) + .sort({{"c1", false}, {"c2", false}, {"c3", false}}, true) + .window(RowNumber(), + {{"c1", false}, {"c2", false}, {"c3", false}}, + {{"c1", false}, {"c2", false}, {"c3", false}}, + buildDefaultRowsFrame()); + tipb::DAGRequest req; + MPPInfo mpp_info(0, -1, -1, {}, std::unordered_map>{}); + builder.getRoot()->toTiPBExecutor(req.mutable_root_executor(), /*collator_id=*/0, mpp_info, TiFlashTestEnv::getContext()); + assert(req.root_executor().tp() == tipb::TypeWindow); + window = req.root_executor().window(); + assert(window.child().tp() == tipb::TypeSort); + sort = window.child().sort(); + } + + void prepareWindowStream(Context & context, int concurrency, int source_num, int total_rows, const std::vector & blocks, BlockInputStreamPtr & sender_stream, BlockInputStreamPtr & receiver_stream, std::shared_ptr & sender_helper, std::shared_ptr & receiver_helper) const + { + DAGPipeline pipeline; + receiver_helper = std::make_shared(concurrency, source_num); + pipeline.streams = receiver_helper->buildExchangeReceiverStream(); + + sender_helper = std::make_shared(source_num, concurrency, receiver_helper->queues, receiver_helper->fields); + sender_stream = sender_helper->buildUnionStream(total_rows, blocks); + + context.setDAGContext(sender_helper->dag_context.get()); + std::vector source_columns{ + NameAndTypePair("c1", makeNullable(std::make_shared())), + NameAndTypePair("c2", makeNullable(std::make_shared())), + NameAndTypePair("c3", makeNullable(std::make_shared()))}; + auto mock_interpreter = mockInterpreter(context, source_columns, concurrency); + mock_interpreter->input_streams_vec.push_back(pipeline.streams); + mockExecuteWindowOrder(mock_interpreter, pipeline, sort); + mockExecuteWindow(mock_interpreter, pipeline, window); + pipeline.transform([&](auto & stream) { + stream = std::make_shared(stream, 8192, 0, "mock_executor_id_squashing"); + }); + receiver_stream = std::make_shared>(pipeline.streams, nullptr, concurrency, /*req_id=*/""); + } + + tipb::Window window; + tipb::Sort sort; +}; + +BENCHMARK_DEFINE_F(WindowFunctionBench, basic_row_number) +(benchmark::State & state) +try +{ + const int concurrency = state.range(0); + const int source_num = state.range(1); + const int total_rows = state.range(2); + Context context = TiFlashTestEnv::getContext(); + + for (auto _ : state) + { + std::shared_ptr sender_helper; + std::shared_ptr receiver_helper; + BlockInputStreamPtr sender_stream; + BlockInputStreamPtr receiver_stream; + + prepareWindowStream(context, concurrency, source_num, total_rows, input_blocks, sender_stream, receiver_stream, sender_helper, receiver_helper); + + runAndWait(receiver_helper, receiver_stream, sender_helper, sender_stream); + } +} +CATCH +BENCHMARK_REGISTER_F(WindowFunctionBench, basic_row_number) + ->Args({8, 1, 1024 * 1000}); + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Flash/tests/exchange_perftest.cpp b/dbms/src/Flash/tests/exchange_perftest.cpp deleted file mode 100644 index c2e047bec62..00000000000 --- a/dbms/src/Flash/tests/exchange_perftest.cpp +++ /dev/null @@ -1,699 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include // to include the implementation of StreamingDAGResponseWriter -#include // to include the implementation of ExchangeReceiver -#include // to include the implementation of MPPTunnel -#include // to include the implementation of MPPTunnelSet -#include -#include -#include - -namespace DB::tests -{ -namespace -{ -std::random_device rd; - -using Packet = mpp::MPPDataPacket; -using PacketPtr = std::shared_ptr; -using PacketQueue = MPMCQueue; -using PacketQueuePtr = std::shared_ptr; -using StopFlag = std::atomic; - -std::atomic received_data_size{0}; - -struct MockReceiverContext -{ - struct Status - { - int status_code = 0; - String error_msg; - - bool ok() const - { - return status_code == 0; - } - - const String & error_message() const - { - return error_msg; - } - - int error_code() const - { - return status_code; - } - }; - - struct Request - { - String debugString() const - { - return "{Request}"; - } - - int source_index = 0; - int send_task_id = 0; - int recv_task_id = -1; - }; - - struct Reader - { - explicit Reader(const PacketQueuePtr & queue_) - : queue(queue_) - {} - - void initialize() const - { - } - - bool read(PacketPtr & packet [[maybe_unused]]) const - { - PacketPtr res; - if (queue->pop(res)) - { - received_data_size.fetch_add(res->ByteSizeLong()); - *packet = *res; // avoid change shared packets - return true; - } - return false; - } - - Status finish() const - { - return {0, ""}; - } - - PacketQueuePtr queue; - }; - - MockReceiverContext( - const std::vector & queues_, - const std::vector & field_types_) - : queues(queues_) - , field_types(field_types_) - { - } - - void fillSchema(DAGSchema & schema) const - { - schema.clear(); - for (size_t i = 0; i < field_types.size(); ++i) - { - String name = "exchange_receiver_" + std::to_string(i); - ColumnInfo info = TiDB::fieldTypeToColumnInfo(field_types[i]); - schema.emplace_back(std::move(name), std::move(info)); - } - } - - Request makeRequest(int index) const - { - return {index, index, -1}; - } - - std::shared_ptr makeReader(const Request & request) - { - return std::make_shared(queues[request.send_task_id]); - } - - static Status getStatusOK() - { - return {0, ""}; - } - - std::vector queues; - std::vector field_types; -}; - -using MockExchangeReceiver = ExchangeReceiverBase; -using MockExchangeReceiverPtr = std::shared_ptr; -using MockExchangeReceiverInputStream = TiRemoteBlockInputStream; - -struct MockWriter -{ - explicit MockWriter(PacketQueuePtr queue_) - : queue(std::move(queue_)) - {} - - bool Write(const Packet & packet) - { - queue->push(std::make_shared(packet)); - return true; - } - - void finish() - { - queue->finish(); - } - - PacketQueuePtr queue; -}; - -using MockWriterPtr = std::shared_ptr; -using MockTunnel = MPPTunnelBase; -using MockTunnelPtr = std::shared_ptr; -using MockTunnelSet = MPPTunnelSetBase; -using MockTunnelSetPtr = std::shared_ptr; - -struct MockBlockInputStream : public IProfilingBlockInputStream -{ - const std::vector & blocks; - Block header; - std::mt19937 mt; - std::uniform_int_distribution dist; - StopFlag & stop_flag; - - MockBlockInputStream(const std::vector & blocks_, StopFlag & stop_flag_) - : blocks(blocks_) - , header(blocks[0].cloneEmpty()) - , mt(rd()) - , dist(0, blocks.size() - 1) - , stop_flag(stop_flag_) - {} - - String getName() const override { return "MockBlockInputStream"; } - Block getHeader() const override { return header; } - - Block readImpl() override - { - if (stop_flag.load(std::memory_order_relaxed)) - return Block{}; - return blocks[dist(mt)]; - } -}; - -Block makeBlock(int row_num) -{ - std::mt19937 mt(rd()); - std::uniform_int_distribution int64_dist; - std::uniform_int_distribution len_dist(10, 20); - std::uniform_int_distribution char_dist; - - InferredDataVector> int64_vec; - InferredDataVector> int64_vec2; - for (int i = 0; i < row_num; ++i) - { - int64_vec.emplace_back(int64_dist(mt)); - int64_vec2.emplace_back(int64_dist(mt)); - } - - InferredDataVector> string_vec; - for (int i = 0; i < row_num; ++i) - { - int len = len_dist(mt); - String s; - for (int j = 0; j < len; ++j) - s.push_back(char_dist(mt)); - string_vec.push_back(std::move(s)); - } - - auto int64_data_type = makeDataType>(); - ColumnWithTypeAndName int64_column(makeColumn>(int64_data_type, int64_vec), int64_data_type, "int64_1"); - ColumnWithTypeAndName int64_column2(makeColumn>(int64_data_type, int64_vec2), int64_data_type, "int64_2"); - - auto string_data_type = makeDataType>(); - ColumnWithTypeAndName string_column(makeColumn>(string_data_type, string_vec), string_data_type, "string"); - - return Block({int64_column, string_column, int64_column2}); -} - -std::vector makeBlocks(int block_num, int row_num) -{ - std::vector blocks; - for (int i = 0; i < block_num; ++i) - blocks.push_back(makeBlock(row_num)); - return blocks; -} - -mpp::MPPDataPacket makePacket(ChunkCodecStream & codec, int row_num) -{ - auto block = makeBlock(row_num); - codec.encode(block, 0, row_num); - - mpp::MPPDataPacket packet; - packet.add_chunks(codec.getString()); - codec.clear(); - - return packet; -} - -std::vector makePackets(ChunkCodecStream & codec, int packet_num, int row_num) -{ - std::vector packets; - for (int i = 0; i < packet_num; ++i) - packets.push_back(std::make_shared(makePacket(codec, row_num))); - return packets; -} - -std::vector makePacketQueues(int source_num, int queue_size) -{ - std::vector queues; - for (int i = 0; i < source_num; ++i) - queues.push_back(std::make_shared(queue_size)); - return queues; -} - -std::vector makeFields() -{ - std::vector fields(3); - fields[0].set_tp(TiDB::TypeLongLong); - fields[1].set_tp(TiDB::TypeString); - fields[2].set_tp(TiDB::TypeLongLong); - return fields; -} - -void printException(const Exception & e) -{ - std::string text = e.displayText(); - - auto embedded_stack_trace_pos = text.find("Stack trace"); - std::cerr << "Code: " << e.code() << ". " << text << std::endl - << std::endl; - if (std::string::npos == embedded_stack_trace_pos) - std::cerr << "Stack trace:" << std::endl - << e.getStackTrace().toString() << std::endl; -} - -void sendPacket(const std::vector & packets, const PacketQueuePtr & queue, StopFlag & stop_flag) -{ - std::mt19937 mt(rd()); - std::uniform_int_distribution dist(0, packets.size() - 1); - - while (!stop_flag.load()) - { - int i = dist(mt); - queue->tryPush(packets[i], std::chrono::milliseconds(10)); - } - queue->finish(); -} - -void receivePacket(const PacketQueuePtr & queue) -{ - while (true) - { - PacketPtr packet; - if (queue->pop(packet)) - received_data_size.fetch_add(packet->ByteSizeLong()); - else - break; - } -} - -template -void readBlock(BlockInputStreamPtr stream) -{ - [[maybe_unused]] auto get_rate = [](auto count, auto duration) { - return count * 1000 / duration.count(); - }; - - [[maybe_unused]] auto get_mib = [](auto v) { - return v / 1024 / 1024; - }; - - [[maybe_unused]] auto start = std::chrono::high_resolution_clock::now(); - [[maybe_unused]] auto second_ago = start; - [[maybe_unused]] Int64 block_count = 0; - [[maybe_unused]] Int64 last_block_count = 0; - [[maybe_unused]] Int64 last_data_size = received_data_size.load(); - try - { - stream->readPrefix(); - while (auto block = stream->read()) - { - if constexpr (print_progress) - { - ++block_count; - auto cur = std::chrono::high_resolution_clock::now(); - auto duration = std::chrono::duration_cast(cur - second_ago); - if (duration.count() >= 1000) - { - Int64 data_size = received_data_size.load(); - std::cout - << fmt::format( - "Blocks: {:<10} Data(MiB): {:<8} Block/s: {:<6} Data/s(MiB): {:<6}", - block_count, - get_mib(data_size), - get_rate(block_count - last_block_count, duration), - get_mib(get_rate(data_size - last_data_size, duration))) - << std::endl; - second_ago = cur; - last_block_count = block_count; - last_data_size = data_size; - } - } - } - stream->readSuffix(); - - if constexpr (print_progress) - { - auto cur = std::chrono::high_resolution_clock::now(); - auto duration = std::chrono::duration_cast(cur - start); - Int64 data_size = received_data_size.load(); - std::cout - << fmt::format( - "End. Blocks: {:<10} Data(MiB): {:<8} Block/s: {:<6} Data/s(MiB): {:<6}", - block_count, - get_mib(data_size), - get_rate(block_count, duration), - get_mib(get_rate(data_size, duration))) - << std::endl; - } - } - catch (const Exception & e) - { - printException(e); - throw; - } -} - -struct ReceiverHelper -{ - const int source_num; - tipb::ExchangeReceiver pb_exchange_receiver; - std::vector fields; - mpp::TaskMeta task_meta; - std::vector queues; - std::shared_ptr join_ptr; - - explicit ReceiverHelper(int source_num_) - : source_num(source_num_) - { - pb_exchange_receiver.set_tp(tipb::Hash); - for (int i = 0; i < source_num; ++i) - { - mpp::TaskMeta task; - task.set_start_ts(0); - task.set_task_id(i); - task.set_partition_id(i); - task.set_address(""); - - String encoded_task; - task.SerializeToString(&encoded_task); - - pb_exchange_receiver.add_encoded_task_meta(encoded_task); - } - - fields = makeFields(); - *pb_exchange_receiver.add_field_types() = fields[0]; - *pb_exchange_receiver.add_field_types() = fields[1]; - *pb_exchange_receiver.add_field_types() = fields[2]; - - task_meta.set_task_id(100); - - queues = makePacketQueues(source_num, 10); - } - - MockExchangeReceiverPtr buildReceiver() - { - return std::make_shared( - std::make_shared(queues, fields), - source_num, - source_num * 5, - nullptr); - } - - BlockInputStreamPtr buildUnionStream(int concurrency) - { - auto receiver = buildReceiver(); - std::vector streams; - for (int i = 0; i < concurrency; ++i) - streams.push_back(std::make_shared(receiver, nullptr)); - return std::make_shared>(streams, nullptr, concurrency, /*req_id=*/""); - } - - BlockInputStreamPtr buildUnionStreamWithHashJoinBuildStream(int concurrency) - { - auto receiver = buildReceiver(); - std::vector streams; - for (int i = 0; i < concurrency; ++i) - streams.push_back(std::make_shared(receiver, nullptr)); - - auto receiver_header = streams.front()->getHeader(); - auto key_name = receiver_header.getByPosition(0).name; - - join_ptr = std::make_shared( - Names{key_name}, - Names{key_name}, - true, - SizeLimits(0, 0, OverflowMode::THROW), - ASTTableJoin::Kind::Inner, - ASTTableJoin::Strictness::All, - /*req_id=*/"", - TiDB::TiDBCollators{nullptr}, - "", - "", - "", - "", - nullptr, - 65536); - - join_ptr->init(receiver_header, concurrency); - - for (int i = 0; i < concurrency; ++i) - streams[i] = std::make_shared(streams[i], join_ptr, i, /*req_id=*/""); - - return std::make_shared>(streams, nullptr, concurrency, /*req_id=*/""); - } - - void finish() - { - if (join_ptr) - { - join_ptr->setBuildTableState(Join::BuildTableState::SUCCEED); - std::cout << fmt::format("Hash table size: {} bytes", join_ptr->getTotalByteCount()) << std::endl; - } - } -}; - -struct SenderHelper -{ - const int source_num; - const int concurrency; - - std::vector queues; - std::vector mock_writers; - std::vector tunnels; - MockTunnelSetPtr tunnel_set; - std::unique_ptr dag_context; - - SenderHelper( - int source_num_, - int concurrency_, - const std::vector & queues_, - const std::vector & fields) - : source_num(source_num_) - , concurrency(concurrency_) - , queues(queues_) - { - mpp::TaskMeta task_meta; - tunnel_set = std::make_shared(); - for (int i = 0; i < source_num; ++i) - { - auto writer = std::make_shared(queues[i]); - mock_writers.push_back(writer); - - auto tunnel = std::make_shared( - task_meta, - task_meta, - std::chrono::seconds(60), - concurrency, - false); - tunnel->connect(writer.get()); - tunnels.push_back(tunnel); - tunnel_set->addTunnel(tunnel); - } - - tipb::DAGRequest dag_request; - tipb::Executor root_executor; - root_executor.set_executor_id("ExchangeSender_100"); - *dag_request.mutable_root_executor() = root_executor; - - dag_context = std::make_unique(dag_request); - dag_context->is_mpp_task = true; - dag_context->is_root_mpp_task = false; - dag_context->encode_type = tipb::EncodeType::TypeCHBlock; - dag_context->result_field_types = fields; - } - - BlockInputStreamPtr buildUnionStream( - StopFlag & stop_flag, - const std::vector & blocks) - { - std::vector send_streams; - for (int i = 0; i < concurrency; ++i) - { - BlockInputStreamPtr stream = std::make_shared(blocks, stop_flag); - std::unique_ptr response_writer( - new StreamingDAGResponseWriter( - tunnel_set, - {0, 1, 2}, - TiDB::TiDBCollators(3), - tipb::Hash, - -1, - -1, - true, - *dag_context)); - send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); - } - - return std::make_shared>(send_streams, nullptr, concurrency, /*req_id=*/""); - } - - void finish() - { - for (size_t i = 0; i < tunnels.size(); ++i) - { - tunnels[i]->writeDone(); - tunnels[i]->waitForFinish(); - mock_writers[i]->finish(); - } - } -}; - -void testOnlyReceiver(int concurrency, int source_num, int block_rows, int seconds) -{ - ReceiverHelper receiver_helper(source_num); - auto union_input_stream = receiver_helper.buildUnionStream(concurrency); - - auto chunk_codec_stream = CHBlockChunkCodec().newCodecStream(receiver_helper.fields); - auto packets = makePackets(*chunk_codec_stream, 100, block_rows); - - StopFlag stop_flag(false); - - std::vector threads; - for (const auto & queue : receiver_helper.queues) - threads.emplace_back(sendPacket, std::cref(packets), queue, std::ref(stop_flag)); - threads.emplace_back(readBlock, union_input_stream); - - std::this_thread::sleep_for(std::chrono::seconds(seconds)); - stop_flag.store(true); - for (auto & thread : threads) - thread.join(); - - receiver_helper.finish(); -} - -template -void testSenderReceiver(int concurrency, int source_num, int block_rows, int seconds) -{ - ReceiverHelper receiver_helper(source_num); - BlockInputStreamPtr union_receive_stream; - if constexpr (with_join) - union_receive_stream = receiver_helper.buildUnionStreamWithHashJoinBuildStream(concurrency); - else - union_receive_stream = receiver_helper.buildUnionStream(concurrency); - - StopFlag stop_flag(false); - auto blocks = makeBlocks(100, block_rows); - - SenderHelper sender_helper(source_num, concurrency, receiver_helper.queues, receiver_helper.fields); - auto union_send_stream = sender_helper.buildUnionStream(stop_flag, blocks); - - auto write_thread = std::thread(readBlock, union_send_stream); - auto read_thread = std::thread(readBlock, union_receive_stream); - - std::this_thread::sleep_for(std::chrono::seconds(seconds)); - stop_flag.store(true); - - write_thread.join(); - sender_helper.finish(); - - read_thread.join(); - receiver_helper.finish(); -} - -void testOnlySender(int concurrency, int source_num, int block_rows, int seconds) -{ - auto queues = makePacketQueues(source_num, 10); - auto fields = makeFields(); - - StopFlag stop_flag(false); - auto blocks = makeBlocks(100, block_rows); - - SenderHelper sender_helper(source_num, concurrency, queues, fields); - auto union_send_stream = sender_helper.buildUnionStream(stop_flag, blocks); - - auto write_thread = std::thread(readBlock, union_send_stream); - std::vector read_threads; - for (int i = 0; i < source_num; ++i) - read_threads.emplace_back(receivePacket, queues[i]); - - std::this_thread::sleep_for(std::chrono::seconds(seconds)); - stop_flag.store(true); - - write_thread.join(); - sender_helper.finish(); - - for (auto & t : read_threads) - t.join(); -} - -} // namespace -} // namespace DB::tests - -int main(int argc [[maybe_unused]], char ** argv [[maybe_unused]]) -{ - if (argc < 2 || argc > 6) - { - std::cerr << fmt::format("Usage: {} [receiver|sender|sender_receiver|sender_receiver_join] ", argv[0]) << std::endl; - exit(1); - } - - String method = argv[1]; - int concurrency = argc >= 3 ? atoi(argv[2]) : 5; - int source_num = argc >= 4 ? atoi(argv[3]) : 2; - int block_rows = argc >= 5 ? atoi(argv[4]) : 5000; - int seconds = argc >= 6 ? atoi(argv[5]) : 10; - - using TestHandler = std::function; - std::unordered_map handlers = { - {"receiver", DB::tests::testOnlyReceiver}, - {"sender", DB::tests::testOnlySender}, - {"sender_receiver", DB::tests::testSenderReceiver}, - {"sender_receiver_join", DB::tests::testSenderReceiver}, - }; - - auto it = handlers.find(method); - if (it != handlers.end()) - { - std::cout - << fmt::format( - "{}. concurrency = {}. source_num = {}. block_rows = {}. seconds = {}", - method, - concurrency, - source_num, - block_rows, - seconds) - << std::endl; - it->second(concurrency, source_num, block_rows, seconds); - } - else - { - std::cerr << "Unknown method: " << method << std::endl; - exit(1); - } -} diff --git a/dbms/src/TestUtils/mockExecutor.cpp b/dbms/src/TestUtils/mockExecutor.cpp index e1ccbdbb010..2cf8a939b58 100644 --- a/dbms/src/TestUtils/mockExecutor.cpp +++ b/dbms/src/TestUtils/mockExecutor.cpp @@ -440,4 +440,5 @@ DAGRequestBuilder MockDAGRequestContext::receive(String exchange_name) } return builder; } -} // namespace DB::tests \ No newline at end of file + +} // namespace DB::tests diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index d52b5ec674a..c11635ac93e 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -188,4 +188,4 @@ MockWindowFrame buildDefaultRowsFrame(); #define Rank() makeASTFunction("Rank") #define DenseRank() makeASTFunction("DenseRank") -} // namespace DB::tests \ No newline at end of file +} // namespace DB::tests diff --git a/libs/libcommon/include/common/types.h b/libs/libcommon/include/common/types.h index 139fc10e980..87c7215d91f 100644 --- a/libs/libcommon/include/common/types.h +++ b/libs/libcommon/include/common/types.h @@ -25,6 +25,7 @@ #if defined(__clang__) #pragma GCC diagnostic ignored "-Wunknown-warning-option" #pragma GCC diagnostic ignored "-Wdeprecated-copy" +#pragma GCC diagnostic ignored "-Wdeprecated-declarations" #pragma GCC diagnostic ignored "-Wtautological-constant-out-of-range-compare" #endif #pragma GCC diagnostic ignored "-Wmaybe-uninitialized" From a9a32b9c751fbc6ebb2aeb6bc5f8254d04c6ff73 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Fri, 17 Jun 2022 08:26:35 +0800 Subject: [PATCH 044/104] Fix the rename_table_across_databases.test to make it can run successfully multiple times. (#5157) close pingcap/tiflash#5145 --- tests/fullstack-test2/ddl/rename_table_across_databases.test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/fullstack-test2/ddl/rename_table_across_databases.test b/tests/fullstack-test2/ddl/rename_table_across_databases.test index c78c27138a0..bc27668bd0c 100644 --- a/tests/fullstack-test2/ddl/rename_table_across_databases.test +++ b/tests/fullstack-test2/ddl/rename_table_across_databases.test @@ -52,7 +52,7 @@ mysql> set session tidb_isolation_read_engines='tiflash'; select * from test_new +------+------+ # check if table info updated. ->> select tidb_database,tidb_name from system.tables where is_tombstone = 0 and (tidb_database = 'test' and tidb_name='t') or (tidb_database='test_new' and tidb_name='t2') +>> select tidb_database,tidb_name from system.tables where is_tombstone = 0 and ((tidb_database = 'test' and tidb_name='t') or (tidb_database='test_new' and tidb_name='t2')) ┌─tidb_database─┬─tidb_name─┐ │ test_new │ t2 │ └───────────────┴───────────┘ From ecd615f205fc6b137d25168a8757b41228dd5efa Mon Sep 17 00:00:00 2001 From: jiaqizho Date: Fri, 17 Jun 2022 10:44:35 +0800 Subject: [PATCH 045/104] Add more test in mix mode (#5017) ref pingcap/tiflash#3594 --- .../V3/tests/gtest_page_storage_mix_mode.cpp | 301 +++++++++++++++++- 1 file changed, 298 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp b/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp index 078daa3e5b4..74e56c929d8 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp @@ -85,6 +85,16 @@ class PageStorageMixedTest : public DB::base::TiFlashStorageTestBasic return run_mode; } + PageReader newMixedPageReader(PageStorage::SnapshotPtr & snapshot) + { + return storage_pool_mix->newLogReader(nullptr, snapshot); + } + + PageReader newMixedPageReader() + { + return storage_pool_mix->newLogReader(nullptr, true, "PageStorageMixedTest"); + } + void reloadV2StoragePool() { db_context->setPageStorageRunMode(PageStorageRunMode::ONLY_V2); @@ -1035,7 +1045,7 @@ try // Thread A create snapshot for read auto snapshot_mix_before_merge_delta = page_reader_mix->getSnapshot("ReadWithSnapshotAfterMergeDelta"); { - auto page_reader_mix_with_snap = storage_pool_mix->newLogReader(nullptr, snapshot_mix_before_merge_delta); + auto page_reader_mix_with_snap = newMixedPageReader(snapshot_mix_before_merge_delta); const auto & page1 = page_reader_mix_with_snap.read(1); const auto & page2 = page_reader_mix_with_snap.read(2); const auto & page3 = page_reader_mix_with_snap.read(3); @@ -1044,7 +1054,7 @@ try ASSERT_PAGE_EQ(c_buff2, buf_sz2, page3, 3); } { - auto page_reader_mix_with_snap = storage_pool_mix->newLogReader(nullptr, true, "ReadWithSnapshotAfterMergeDelta"); + auto page_reader_mix_with_snap = newMixedPageReader(); const auto & page1 = page_reader_mix_with_snap.read(1); const auto & page2 = page_reader_mix_with_snap.read(2); const auto & page3 = page_reader_mix_with_snap.read(3); @@ -1063,7 +1073,7 @@ try } // Thread A continue to read 1, 3 { - auto page_reader_mix_with_snap = storage_pool_mix->newLogReader(nullptr, snapshot_mix_before_merge_delta); + auto page_reader_mix_with_snap = newMixedPageReader(snapshot_mix_before_merge_delta); // read 1, 3 with snapshot, should be success const auto & page1 = page_reader_mix_with_snap.read(1); const auto & page3 = page_reader_mix_with_snap.read(3); @@ -1071,6 +1081,7 @@ try ASSERT_PAGE_EQ(c_buff2, buf_sz2, page3, 3); ASSERT_THROW(page_reader_mix_with_snap.read(4), DB::Exception); } + { // Revert v3 WriteBatch batch; @@ -1081,6 +1092,290 @@ try } CATCH +TEST_F(PageStorageMixedTest, refWithSnapshot2) +try +{ + UInt64 tag = 0; + const size_t buf_sz = 1024; + char c_buff[buf_sz]; + for (size_t i = 0; i < buf_sz; ++i) + { + c_buff[i] = i % 0xff; + } + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, tag, buff, buf_sz); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + batch.putRefPage(2, 1); + page_writer_v2->write(std::move(batch), nullptr); + } + + // Change to mix mode here + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::MIX_MODE); + + auto snapshot_mix = page_reader_mix->getSnapshot(""); + { + WriteBatch batch; + batch.delPage(1); + batch.delPage(2); + page_writer_mix->write(std::move(batch), nullptr); + } + + { + auto page_maps = newMixedPageReader(snapshot_mix).read({1, 2}); + ASSERT_EQ(page_maps.size(), 2); + + ASSERT_PAGE_EQ(c_buff, buf_sz, page_maps[1], 1); + ASSERT_PAGE_EQ(c_buff, buf_sz, page_maps[2], 2); + } +} +CATCH + +TEST_F(PageStorageMixedTest, refWithSnapshot3) +try +{ + UInt64 tag = 0; + const size_t buf_sz = 1024; + char c_buff[buf_sz]; + for (size_t i = 0; i < buf_sz; ++i) + { + c_buff[i] = i % 0xff; + } + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, tag, buff, buf_sz); + // to keep mix mode + batch.putExternal(10, 1); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + batch.putRefPage(2, 1); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + batch.delPage(1); + batch.delPage(2); + page_writer_v2->write(std::move(batch), nullptr); + } + + // Change to mix mode here + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::MIX_MODE); + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, tag, buff, buf_sz); + page_writer_mix->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + batch.putRefPage(2, 1); + page_writer_mix->write(std::move(batch), nullptr); + } + + auto snapshot_mix = page_reader_mix->getSnapshot(""); + { + WriteBatch batch; + batch.delPage(1); + batch.delPage(2); + page_writer_mix->write(std::move(batch), nullptr); + } + + { + auto page_maps = newMixedPageReader(snapshot_mix).read({1, 2}); + ASSERT_EQ(page_maps.size(), 2); + + ASSERT_PAGE_EQ(c_buff, buf_sz, page_maps[1], 1); + ASSERT_PAGE_EQ(c_buff, buf_sz, page_maps[2], 2); + } +} +CATCH + +TEST_F(PageStorageMixedTest, refWithSnapshot4) +try +{ + UInt64 tag = 0; + const size_t buf_sz = 1024; + char c_buff[buf_sz]; + for (size_t i = 0; i < buf_sz; ++i) + { + c_buff[i] = i % 0xff; + } + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, tag, buff, buf_sz); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + batch.putRefPage(2, 1); + page_writer_v2->write(std::move(batch), nullptr); + } + + // Change to mix mode here + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::MIX_MODE); + + { + WriteBatch batch; + batch.delPage(2); + page_writer_mix->write(std::move(batch), nullptr); + } + + { + auto page1 = page_reader_mix->read(1); + + ASSERT_PAGE_EQ(c_buff, buf_sz, page1, 1); + } +} +CATCH + +TEST_F(PageStorageMixedTest, refWithSnapshot5) +try +{ + UInt64 tag = 0; + const size_t buf_sz = 1024; + char c_buff[buf_sz]; + for (size_t i = 0; i < buf_sz; ++i) + { + c_buff[i] = i % 0xff; + } + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, tag, buff, buf_sz); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + batch.putRefPage(2, 1); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + batch.delPage(1); + page_writer_v2->write(std::move(batch), nullptr); + } + + // Change to mix mode here + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::MIX_MODE); + + { + auto page1 = page_reader_mix->read(2); + + ASSERT_PAGE_EQ(c_buff, buf_sz, page1, 2); + } +} +CATCH + +TEST_F(PageStorageMixedTest, refWithSnapshot6) +try +{ + UInt64 tag = 0; + const size_t buf_sz = 1024; + char c_buff[buf_sz]; + for (size_t i = 0; i < buf_sz; ++i) + { + c_buff[i] = i % 0xff; + } + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff, sizeof(c_buff)); + batch.putPage(1, tag, buff, buf_sz); + page_writer_v2->write(std::move(batch), nullptr); + } + + { + WriteBatch batch; + batch.putRefPage(2, 1); + page_writer_v2->write(std::move(batch), nullptr); + } + + // Change to mix mode here + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::MIX_MODE); + + { + WriteBatch batch; + batch.delPage(1); + page_writer_mix->write(std::move(batch), nullptr); + } + + { + auto page1 = page_reader_mix->read(2); + + ASSERT_PAGE_EQ(c_buff, buf_sz, page1, 2); + } +} +CATCH + +TEST_F(PageStorageMixedTest, ReadWithSnapshot2) +try +{ + UInt64 tag = 0; + const size_t buf_sz = 1; + char c_buff1[buf_sz]; + c_buff1[0] = 1; + + char c_buff2[buf_sz]; + c_buff2[0] = 2; + + { + WriteBatch batch; + ReadBufferPtr buff = std::make_shared(c_buff1, buf_sz); + batch.putPage(1, tag, buff, buf_sz); + page_writer_v2->write(std::move(batch), nullptr); + } + + // Change to mix mode here + ASSERT_EQ(reloadMixedStoragePool(), PageStorageRunMode::MIX_MODE); + + auto snapshot_mix = page_reader_mix->getSnapshot(""); + { + WriteBatch batch; + batch.delPage(1); + ReadBufferPtr buff = std::make_shared(c_buff2, buf_sz); + batch.putPage(1, tag, buff, buf_sz); + page_writer_mix->write(std::move(batch), nullptr); + } + + { + auto page1 = newMixedPageReader(snapshot_mix).read(1); + ASSERT_PAGE_EQ(c_buff1, buf_sz, page1, 1); + } + + { + auto page1 = page_reader_mix->read(1); + ASSERT_PAGE_EQ(c_buff2, buf_sz, page1, 1); + } + + { + // Revert v3 + WriteBatch batch; + batch.delPage(1); + page_writer_mix->write(std::move(batch), nullptr); + } +} +CATCH + } // namespace PS::V3::tests } // namespace DB From 164eda5296fe0714ba3e7a30f0ffd7a562b378be Mon Sep 17 00:00:00 2001 From: JaySon Date: Sat, 18 Jun 2022 17:16:35 +0800 Subject: [PATCH 046/104] *: Add some comments about decoding (#5158) ref pingcap/tiflash#4862 --- dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp | 46 +++---- .../Page/V3/tests/gtest_blob_store.cpp | 4 + .../DecodingStorageSchemaSnapshot.h | 4 +- .../Storages/Transaction/PartitionStreams.cpp | 1 + .../Transaction/RegionBlockReader.cpp | 1 + .../Storages/Transaction/RegionBlockReader.h | 2 +- dbms/src/Storages/Transaction/RowCodec.cpp | 43 ++++--- .../Storages/Transaction/TiKVRecordFormat.h | 119 ++++++++++++------ .../Transaction/tests/gtest_kvstore.cpp | 10 +- .../Transaction/tests/region_helper.h | 21 +++- tests/fullstack-test/mpp/mpp_fail.test | 1 + 11 files changed, 161 insertions(+), 91 deletions(-) diff --git a/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp b/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp index 7eecbbdf6f7..9d5b848ddea 100644 --- a/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp +++ b/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp @@ -60,7 +60,7 @@ RegionPtr GenDbgRegionSnapshotWithData(Context & context, const ASTs & args) { const String & database_name = typeid_cast(*args[0]).name; const String & table_name = typeid_cast(*args[1]).name; - RegionID region_id = static_cast(safeGet(typeid_cast(*args[2]).value)); + auto region_id = static_cast(safeGet(typeid_cast(*args[2]).value)); TableID table_id = RegionBench::getTableID(context, database_name, table_name, ""); MockTiDB::TablePtr table = MockTiDB::instance().getTableByName(database_name, table_name); auto & table_info = table->table_info; @@ -70,8 +70,8 @@ RegionPtr GenDbgRegionSnapshotWithData(Context & context, const ASTs & args) if (!is_common_handle) { - HandleID start = static_cast(safeGet(typeid_cast(*args[3]).value)); - HandleID end = static_cast(safeGet(typeid_cast(*args[4]).value)); + auto start = static_cast(safeGet(typeid_cast(*args[3]).value)); + auto end = static_cast(safeGet(typeid_cast(*args[4]).value)); region = RegionBench::createRegion(table_id, region_id, start, end); } else @@ -105,8 +105,8 @@ RegionPtr GenDbgRegionSnapshotWithData(Context & context, const ASTs & args) for (auto it = args_begin; it != args_end; it += len) { HandleID handle_id = is_common_handle ? 0 : static_cast(safeGet(typeid_cast(*it[0]).value)); - Timestamp tso = static_cast(safeGet(typeid_cast(*it[1]).value)); - UInt8 del = static_cast(safeGet(typeid_cast(*it[2]).value)); + auto tso = static_cast(safeGet(typeid_cast(*it[1]).value)); + auto del = static_cast(safeGet(typeid_cast(*it[2]).value)); { std::vector fields; @@ -168,7 +168,7 @@ void MockRaftCommand::dbgFuncRegionSnapshotWithData(Context & context, const AST // DBGInvoke region_snapshot(region-id, start-key, end-key, database-name, table-name[, partition-id]) void MockRaftCommand::dbgFuncRegionSnapshot(Context & context, const ASTs & args, DBGInvoker::Printer output) { - RegionID region_id = static_cast(safeGet(typeid_cast(*args[0]).value)); + auto region_id = static_cast(safeGet(typeid_cast(*args[0]).value)); bool has_partition_id = false; size_t args_size = args.size(); if (dynamic_cast(args[args_size - 1].get()) != nullptr) @@ -214,15 +214,15 @@ void MockRaftCommand::dbgFuncRegionSnapshot(Context & context, const ASTs & args } else { - HandleID start = static_cast(safeGet(typeid_cast(*args[1]).value)); - HandleID end = static_cast(safeGet(typeid_cast(*args[2]).value)); + auto start = static_cast(safeGet(typeid_cast(*args[1]).value)); + auto end = static_cast(safeGet(typeid_cast(*args[2]).value)); start_key = RecordKVFormat::genKey(table_id, start); end_key = RecordKVFormat::genKey(table_id, end); } region_info.set_start_key(start_key.toString()); region_info.set_end_key(end_key.toString()); - *region_info.add_peers() = createPeer(1, true); - *region_info.add_peers() = createPeer(2, true); + *region_info.add_peers() = tests::createPeer(1, true); + *region_info.add_peers() = tests::createPeer(2, true); auto peer_id = 1; auto start_decoded_key = RecordKVFormat::decodeTiKVKey(start_key); auto end_decoded_key = RecordKVFormat::decodeTiKVKey(end_key); @@ -432,9 +432,9 @@ void MockRaftCommand::dbgFuncIngestSST(Context & context, const ASTs & args, DBG { const String & database_name = typeid_cast(*args[0]).name; const String & table_name = typeid_cast(*args[1]).name; - RegionID region_id = static_cast(safeGet(typeid_cast(*args[2]).value)); - RegionID start_handle = static_cast(safeGet(typeid_cast(*args[3]).value)); - RegionID end_handle = static_cast(safeGet(typeid_cast(*args[4]).value)); + auto region_id = static_cast(safeGet(typeid_cast(*args[2]).value)); + auto start_handle = static_cast(safeGet(typeid_cast(*args[3]).value)); + auto end_handle = static_cast(safeGet(typeid_cast(*args[4]).value)); MockTiDB::TablePtr table = MockTiDB::instance().getTableByName(database_name, table_name); const auto & table_info = RegionBench::getTableInfo(context, database_name, table_name); @@ -555,7 +555,7 @@ void MockRaftCommand::dbgFuncRegionSnapshotApplyBlock(Context & context, const A throw Exception("Args not matched, should be: region-id", ErrorCodes::BAD_ARGUMENTS); } - RegionID region_id = static_cast(safeGet(typeid_cast(*args.front()).value)); + auto region_id = static_cast(safeGet(typeid_cast(*args.front()).value)); auto [region, block_cache] = GLOBAL_REGION_MAP.popRegionCache("__snap_" + std::to_string(region_id)); auto & tmt = context.getTMTContext(); context.getTMTContext().getKVStore()->checkAndApplySnapshot({region, std::move(block_cache)}, tmt); @@ -577,12 +577,12 @@ void MockRaftCommand::dbgFuncRegionSnapshotPreHandleDTFiles(Context & context, c const String & database_name = typeid_cast(*args[0]).name; const String & table_name = typeid_cast(*args[1]).name; - RegionID region_id = static_cast(safeGet(typeid_cast(*args[2]).value)); - RegionID start_handle = static_cast(safeGet(typeid_cast(*args[3]).value)); - RegionID end_handle = static_cast(safeGet(typeid_cast(*args[4]).value)); + auto region_id = static_cast(safeGet(typeid_cast(*args[2]).value)); + auto start_handle = static_cast(safeGet(typeid_cast(*args[3]).value)); + auto end_handle = static_cast(safeGet(typeid_cast(*args[4]).value)); - const String schema_str = safeGet(typeid_cast(*args[5]).value); - String handle_pk_name = safeGet(typeid_cast(*args[6]).value); + const auto schema_str = safeGet(typeid_cast(*args[5]).value); + auto handle_pk_name = safeGet(typeid_cast(*args[6]).value); UInt64 test_fields = 1; if (args.size() > 7) @@ -677,10 +677,10 @@ void MockRaftCommand::dbgFuncRegionSnapshotPreHandleDTFilesWithHandles(Context & const String & database_name = typeid_cast(*args[0]).name; const String & table_name = typeid_cast(*args[1]).name; - RegionID region_id = static_cast(safeGet(typeid_cast(*args[2]).value)); + auto region_id = static_cast(safeGet(typeid_cast(*args[2]).value)); - const String schema_str = safeGet(typeid_cast(*args[3]).value); - String handle_pk_name = safeGet(typeid_cast(*args[4]).value); + const auto schema_str = safeGet(typeid_cast(*args[3]).value); + auto handle_pk_name = safeGet(typeid_cast(*args[4]).value); std::vector handles; for (size_t i = 5; i < args.size(); ++i) @@ -770,7 +770,7 @@ void MockRaftCommand::dbgFuncRegionSnapshotApplyDTFiles(Context & context, const if (args.size() != 1) throw Exception("Args not matched, should be: region-id", ErrorCodes::BAD_ARGUMENTS); - RegionID region_id = static_cast(safeGet(typeid_cast(*args.front()).value)); + auto region_id = static_cast(safeGet(typeid_cast(*args.front()).value)); const auto region_name = "__snap_snap_" + std::to_string(region_id); auto [new_region, ingest_ids] = GLOBAL_REGION_MAP.popRegionSnap(region_name); auto & tmt = context.getTMTContext(); diff --git a/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp b/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp index fdd08c7cb8e..f9daacc4cce 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_blob_store.cpp @@ -306,6 +306,7 @@ try blob_store.blob_stats.restoreByEntry(PageEntryV3{ .file_id = file_id1, .size = 128, + .padded_size = 0, .tag = 0, .offset = 1024, .checksum = 0x4567, @@ -313,6 +314,7 @@ try blob_store.blob_stats.restoreByEntry(PageEntryV3{ .file_id = file_id1, .size = 512, + .padded_size = 0, .tag = 0, .offset = 2048, .checksum = 0x4567, @@ -320,6 +322,7 @@ try blob_store.blob_stats.restoreByEntry(PageEntryV3{ .file_id = file_id2, .size = 512, + .padded_size = 0, .tag = 0, .offset = 2048, .checksum = 0x4567, @@ -402,6 +405,7 @@ try blob_store.blob_stats.restoreByEntry(PageEntryV3{ .file_id = id, .size = 1024, + .padded_size = 0, .tag = 0, .offset = 0, .checksum = 0x4567, diff --git a/dbms/src/Storages/Transaction/DecodingStorageSchemaSnapshot.h b/dbms/src/Storages/Transaction/DecodingStorageSchemaSnapshot.h index c636d9e60ab..e8e0610326c 100644 --- a/dbms/src/Storages/Transaction/DecodingStorageSchemaSnapshot.h +++ b/dbms/src/Storages/Transaction/DecodingStorageSchemaSnapshot.h @@ -125,11 +125,11 @@ struct DecodingStorageSchemaSnapshot { auto pk_pos_iter = pk_pos_map.begin(); size_t column_pos_in_block = 0; - for (auto iter = sorted_column_id_with_pos.begin(); iter != sorted_column_id_with_pos.end(); iter++) + for (auto & column_id_with_pos : sorted_column_id_with_pos) { if (pk_pos_iter == pk_pos_map.end()) break; - if (pk_pos_iter->first == iter->first) + if (pk_pos_iter->first == column_id_with_pos.first) { pk_pos_iter->second = column_pos_in_block; pk_pos_iter++; diff --git a/dbms/src/Storages/Transaction/PartitionStreams.cpp b/dbms/src/Storages/Transaction/PartitionStreams.cpp index 4b2ca6c07a8..456f067fe5e 100644 --- a/dbms/src/Storages/Transaction/PartitionStreams.cpp +++ b/dbms/src/Storages/Transaction/PartitionStreams.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include diff --git a/dbms/src/Storages/Transaction/RegionBlockReader.cpp b/dbms/src/Storages/Transaction/RegionBlockReader.cpp index af351f4a6b0..a9384e4a14d 100644 --- a/dbms/src/Storages/Transaction/RegionBlockReader.cpp +++ b/dbms/src/Storages/Transaction/RegionBlockReader.cpp @@ -186,6 +186,7 @@ bool RegionBlockReader::readImpl(Block & block, const RegionDataReadInfoList & d } else { + // For common handle, sometimes we need to decode the value from encoded key instead of encoded value auto * raw_extra_column = const_cast((block.getByPosition(extra_handle_column_pos)).column.get()); raw_extra_column->insertData(pk->data(), pk->size()); /// decode key and insert pk columns if needed diff --git a/dbms/src/Storages/Transaction/RegionBlockReader.h b/dbms/src/Storages/Transaction/RegionBlockReader.h index ec633e805c0..004d9f40447 100644 --- a/dbms/src/Storages/Transaction/RegionBlockReader.h +++ b/dbms/src/Storages/Transaction/RegionBlockReader.h @@ -41,7 +41,7 @@ class Block; class RegionBlockReader : private boost::noncopyable { public: - RegionBlockReader(DecodingStorageSchemaSnapshotConstPtr schema_snapshot_); + explicit RegionBlockReader(DecodingStorageSchemaSnapshotConstPtr schema_snapshot_); /// Read `data_list` as a block. /// diff --git a/dbms/src/Storages/Transaction/RowCodec.cpp b/dbms/src/Storages/Transaction/RowCodec.cpp index 427544a0467..ea7f6b7c2da 100644 --- a/dbms/src/Storages/Transaction/RowCodec.cpp +++ b/dbms/src/Storages/Transaction/RowCodec.cpp @@ -314,7 +314,7 @@ bool appendRowV2ToBlock( ColumnID pk_handle_id, bool force_decode) { - UInt8 row_flag = readLittleEndian(&raw_value[1]); + auto row_flag = readLittleEndian(&raw_value[1]); bool is_big = row_flag & RowV2::BigRowMask; return is_big ? appendRowV2ToBlockImpl(raw_value, column_ids_iter, column_ids_iter_end, block, block_column_pos, column_infos, pk_handle_id, force_decode) : appendRowV2ToBlockImpl(raw_value, column_ids_iter, column_ids_iter_end, block, block_column_pos, column_infos, pk_handle_id, force_decode); @@ -360,9 +360,10 @@ bool appendRowV2ToBlockImpl( decodeUInts::ColumnIDType>(cursor, raw_value, num_null_columns, null_column_ids); decodeUInts::ValueOffsetType>(cursor, raw_value, num_not_null_columns, value_offsets); size_t values_start_pos = cursor; - size_t id_not_null = 0, id_null = 0; + size_t idx_not_null = 0; + size_t idx_null = 0; // Merge ordered not null/null columns to keep order. - while (id_not_null < not_null_column_ids.size() || id_null < null_column_ids.size()) + while (idx_not_null < not_null_column_ids.size() || idx_null < null_column_ids.size()) { if (column_ids_iter == column_ids_iter_end) { @@ -371,24 +372,31 @@ bool appendRowV2ToBlockImpl( } bool is_null; - if (id_not_null < not_null_column_ids.size() && id_null < null_column_ids.size()) - is_null = not_null_column_ids[id_not_null] > null_column_ids[id_null]; + if (idx_not_null < not_null_column_ids.size() && idx_null < null_column_ids.size()) + is_null = not_null_column_ids[idx_not_null] > null_column_ids[idx_null]; else - is_null = id_null < null_column_ids.size(); + is_null = idx_null < null_column_ids.size(); - auto next_datum_column_id = is_null ? null_column_ids[id_null] : not_null_column_ids[id_not_null]; + auto next_datum_column_id = is_null ? null_column_ids[idx_null] : not_null_column_ids[idx_not_null]; if (column_ids_iter->first > next_datum_column_id) { - // extra column + // The next column id to read is bigger than the column id of next datum in encoded row. + // It means this is the datum of extra column. May happen when reading after dropping + // a column. if (!force_decode) return false; + // Ignore the extra column and continue to parse other datum if (is_null) - id_null++; + idx_null++; else - id_not_null++; + idx_not_null++; } else if (column_ids_iter->first < next_datum_column_id) { + // The next column id to read is less than the column id of next datum in encoded row. + // It means this is the datum of missing column. May happen when reading after adding + // a column. + // Fill with default value and continue to read data for next column id. const auto & column_info = column_infos[column_ids_iter->second]; if (!addDefaultValueToColumnIfPossible(column_info, block, block_column_pos, force_decode)) return false; @@ -397,7 +405,7 @@ bool appendRowV2ToBlockImpl( } else { - // if pk_handle_id is a valid column id, then it means the table's pk_is_handle is true + // If pk_handle_id is a valid column id, then it means the table's pk_is_handle is true // we can just ignore the pk value encoded in value part if (unlikely(column_ids_iter->first == pk_handle_id)) { @@ -405,15 +413,16 @@ bool appendRowV2ToBlockImpl( block_column_pos++; if (is_null) { - id_null++; + idx_null++; } else { - id_not_null++; + idx_not_null++; } continue; } + // Parse the datum. auto * raw_column = const_cast((block.getByPosition(block_column_pos)).column.get()); const auto & column_info = column_infos[column_ids_iter->second]; if (is_null) @@ -432,15 +441,15 @@ bool appendRowV2ToBlockImpl( } // ColumnNullable::insertDefault just insert a null value raw_column->insertDefault(); - id_null++; + idx_null++; } else { - size_t start = id_not_null ? value_offsets[id_not_null - 1] : 0; - size_t length = value_offsets[id_not_null] - start; + size_t start = idx_not_null ? value_offsets[idx_not_null - 1] : 0; + size_t length = value_offsets[idx_not_null] - start; if (!raw_column->decodeTiDBRowV2Datum(values_start_pos + start, raw_value, length, force_decode)) return false; - id_not_null++; + idx_not_null++; } column_ids_iter++; block_column_pos++; diff --git a/dbms/src/Storages/Transaction/TiKVRecordFormat.h b/dbms/src/Storages/Transaction/TiKVRecordFormat.h index 4a25b6d9292..c507616f6e9 100644 --- a/dbms/src/Storages/Transaction/TiKVRecordFormat.h +++ b/dbms/src/Storages/Transaction/TiKVRecordFormat.h @@ -30,7 +30,6 @@ namespace DB { - namespace ErrorCodes { extern const int LOGICAL_ERROR; @@ -38,7 +37,6 @@ extern const int LOGICAL_ERROR; namespace RecordKVFormat { - enum CFModifyFlag : UInt8 { PutFlag = 'P', @@ -83,17 +81,35 @@ inline TiKVKey encodeAsTiKVKey(const String & ori_str) return TiKVKey(ss.releaseStr()); } -inline UInt64 encodeUInt64(const UInt64 x) { return toBigEndian(x); } +inline UInt64 encodeUInt64(const UInt64 x) +{ + return toBigEndian(x); +} -inline UInt64 encodeInt64(const Int64 x) { return encodeUInt64(static_cast(x) ^ SIGN_MASK); } +inline UInt64 encodeInt64(const Int64 x) +{ + return encodeUInt64(static_cast(x) ^ SIGN_MASK); +} -inline UInt64 encodeUInt64Desc(const UInt64 x) { return encodeUInt64(~x); } +inline UInt64 encodeUInt64Desc(const UInt64 x) +{ + return encodeUInt64(~x); +} -inline UInt64 decodeUInt64(const UInt64 x) { return toBigEndian(x); } +inline UInt64 decodeUInt64(const UInt64 x) +{ + return toBigEndian(x); +} -inline UInt64 decodeUInt64Desc(const UInt64 x) { return ~decodeUInt64(x); } +inline UInt64 decodeUInt64Desc(const UInt64 x) +{ + return ~decodeUInt64(x); +} -inline Int64 decodeInt64(const UInt64 x) { return static_cast(decodeUInt64(x) ^ SIGN_MASK); } +inline Int64 decodeInt64(const UInt64 x) +{ + return static_cast(decodeUInt64(x) ^ SIGN_MASK); +} inline void encodeInt64(const Int64 x, WriteBuffer & ss) { @@ -125,7 +141,10 @@ inline DecodedTiKVKey genRawKey(const TableID tableId, const HandleID handleId) return key; } -inline TiKVKey genKey(const TableID tableId, const HandleID handleId) { return encodeAsTiKVKey(genRawKey(tableId, handleId)); } +inline TiKVKey genKey(const TableID tableId, const HandleID handleId) +{ + return encodeAsTiKVKey(genRawKey(tableId, handleId)); +} inline TiKVKey genKey(const TiDB::TableInfo & table_info, std::vector keys) { @@ -176,29 +195,50 @@ inline std::tuple decodeTiKVKeyFull(const TiKVKey & key) } } -inline DecodedTiKVKey decodeTiKVKey(const TiKVKey & key) { return std::get<0>(decodeTiKVKeyFull(key)); } +inline DecodedTiKVKey decodeTiKVKey(const TiKVKey & key) +{ + return std::get<0>(decodeTiKVKeyFull(key)); +} -inline Timestamp getTs(const TiKVKey & key) { return decodeUInt64Desc(read(key.data() + key.dataSize() - 8)); } +inline Timestamp getTs(const TiKVKey & key) +{ + return decodeUInt64Desc(read(key.data() + key.dataSize() - 8)); +} -inline TableID getTableId(const DecodedTiKVKey & key) { return decodeInt64(read(key.data() + 1)); } +inline TableID getTableId(const DecodedTiKVKey & key) +{ + return decodeInt64(read(key.data() + 1)); +} -inline HandleID getHandle(const DecodedTiKVKey & key) { return decodeInt64(read(key.data() + RAW_KEY_NO_HANDLE_SIZE)); } +inline HandleID getHandle(const DecodedTiKVKey & key) +{ + return decodeInt64(read(key.data() + RAW_KEY_NO_HANDLE_SIZE)); +} inline RawTiDBPK getRawTiDBPK(const DecodedTiKVKey & key) { return std::make_shared(key.begin() + RAW_KEY_NO_HANDLE_SIZE, key.end()); } -inline TableID getTableId(const TiKVKey & key) { return getTableId(decodeTiKVKey(key)); } +inline TableID getTableId(const TiKVKey & key) +{ + return getTableId(decodeTiKVKey(key)); +} -inline HandleID getHandle(const TiKVKey & key) { return getHandle(decodeTiKVKey(key)); } +inline HandleID getHandle(const TiKVKey & key) +{ + return getHandle(decodeTiKVKey(key)); +} inline bool isRecord(const DecodedTiKVKey & raw_key) { return raw_key.size() >= RAW_KEY_SIZE && raw_key[0] == TABLE_PREFIX && memcmp(raw_key.data() + 9, RECORD_PREFIX_SEP, 2) == 0; } -inline TiKVKey truncateTs(const TiKVKey & key) { return TiKVKey(String(key.data(), key.dataSize() - sizeof(Timestamp))); } +inline TiKVKey truncateTs(const TiKVKey & key) +{ + return TiKVKey(String(key.data(), key.dataSize() - sizeof(Timestamp))); +} inline TiKVKey appendTs(const TiKVKey & key, Timestamp ts) { @@ -215,7 +255,12 @@ inline TiKVKey genKey(TableID tableId, HandleID handleId, Timestamp ts) } inline TiKVValue encodeLockCfValue( - UInt8 lock_type, const String & primary, Timestamp ts, UInt64 ttl, const String * short_value = nullptr, Timestamp min_commit_ts = 0) + UInt8 lock_type, + const String & primary, + Timestamp ts, + UInt64 ttl, + const String * short_value = nullptr, + Timestamp min_commit_ts = 0) { WriteBufferFromOwnString res; res.write(lock_type); @@ -275,7 +320,10 @@ inline R readVarInt(const char *& data, size_t & len) return res; } -inline UInt64 readVarUInt(const char *& data, size_t & len) { return readVarInt(data, len); } +inline UInt64 readVarUInt(const char *& data, size_t & len) +{ + return readVarInt(data, len); +} inline UInt8 readUInt8(const char *& data, size_t & len) { @@ -347,30 +395,29 @@ inline DecodedWriteCFValue decodeWriteCfValue(const TiKVValue & value) auto flag = RecordKVFormat::readUInt8(data, len); switch (flag) { - case RecordKVFormat::SHORT_VALUE_PREFIX: - { - size_t slen = RecordKVFormat::readUInt8(data, len); - if (slen > len) - throw Exception("content len not equal to short value len", ErrorCodes::LOGICAL_ERROR); - short_value = RecordKVFormat::readRawString(data, len, slen); - break; - } - case RecordKVFormat::FLAG_OVERLAPPED_ROLLBACK: - // ignore - break; - case RecordKVFormat::GC_FENCE_PREFIX: - /** + case RecordKVFormat::SHORT_VALUE_PREFIX: + { + size_t slen = RecordKVFormat::readUInt8(data, len); + if (slen > len) + throw Exception("content len not equal to short value len", ErrorCodes::LOGICAL_ERROR); + short_value = RecordKVFormat::readRawString(data, len, slen); + break; + } + case RecordKVFormat::FLAG_OVERLAPPED_ROLLBACK: + // ignore + break; + case RecordKVFormat::GC_FENCE_PREFIX: + /** * according to https://github.com/tikv/tikv/pull/9207, when meet `GC fence` flag, it is definitely a * rewriting record and there must be a complete row written to tikv, just ignore it in tiflash. */ - return std::nullopt; - default: - throw Exception("invalid flag " + std::to_string(flag) + " in write cf", ErrorCodes::LOGICAL_ERROR); + return std::nullopt; + default: + throw Exception("invalid flag " + std::to_string(flag) + " in write cf", ErrorCodes::LOGICAL_ERROR); } } - return InnerDecodedWriteCFValue{write_type, prewrite_ts, - short_value.empty() ? nullptr : std::make_shared(short_value.data(), short_value.length())}; + return InnerDecodedWriteCFValue{write_type, prewrite_ts, short_value.empty() ? nullptr : std::make_shared(short_value.data(), short_value.length())}; } inline TiKVValue encodeWriteCfValue(UInt8 write_type, Timestamp ts, std::string_view short_value = {}, bool gc_fence = false) diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp index f0cafce3914..36a91522bb6 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp @@ -18,10 +18,9 @@ #include #include #include +#include #include -#include "region_helper.h" - namespace DB { namespace RegionBench @@ -37,13 +36,6 @@ extern void ChangeRegionStateRange(RegionState & region_state, bool source_at_le namespace tests { -RegionPtr makeRegion(UInt64 id, const std::string start_key, const std::string end_key, const TiFlashRaftProxyHelper * proxy_helper = nullptr) -{ - return std::make_shared( - RegionMeta(createPeer(2, true), createRegionInfo(id, std::move(start_key), std::move(end_key)), initialApplyState()), - proxy_helper); -} - class RegionKVStoreTest : public ::testing::Test { public: diff --git a/dbms/src/Storages/Transaction/tests/region_helper.h b/dbms/src/Storages/Transaction/tests/region_helper.h index 2808ace0ecb..39bae2669ab 100644 --- a/dbms/src/Storages/Transaction/tests/region_helper.h +++ b/dbms/src/Storages/Transaction/tests/region_helper.h @@ -18,8 +18,10 @@ #include -using namespace DB; - +namespace DB +{ +namespace tests +{ #define ASSERT_CHECK(cond, res) \ do \ { \ @@ -37,7 +39,7 @@ using namespace DB; #define ASSERT_CHECK_EQUAL(a, b, res) \ do \ { \ - if (!(a == b)) \ + if (!((a) == (b))) \ { \ std::cerr << __FILE__ << ":" << __LINE__ << ":" \ << " Assertion " << #a << " == " << #b << " failed.\n"; \ @@ -76,3 +78,16 @@ inline RegionMeta createRegionMeta(UInt64 id, DB::TableID table_id, std::optiona /*region=*/createRegionInfo(id, RecordKVFormat::genKey(table_id, 0), RecordKVFormat::genKey(table_id, 300)), /*apply_state_=*/apply_state.value_or(initialApplyState())); } + +inline RegionPtr makeRegion(UInt64 id, const std::string start_key, const std::string end_key, const TiFlashRaftProxyHelper * proxy_helper = nullptr) +{ + return std::make_shared( + RegionMeta( + createPeer(2, true), + createRegionInfo(id, std::move(start_key), std::move(end_key)), + initialApplyState()), + proxy_helper); +} + +} // namespace tests +} // namespace DB diff --git a/tests/fullstack-test/mpp/mpp_fail.test b/tests/fullstack-test/mpp/mpp_fail.test index 02259a90681..7af5fef3f89 100644 --- a/tests/fullstack-test/mpp/mpp_fail.test +++ b/tests/fullstack-test/mpp/mpp_fail.test @@ -21,6 +21,7 @@ mysql> insert into test.t values(1,'a'),(2,'b'),(3,'c') mysql> alter table test.t set tiflash replica 1 func> wait_table test t +mysql> analyze table test.t # Data. From 604b0de112fdb8a80b4c0157de03c8db3498444f Mon Sep 17 00:00:00 2001 From: hehechen Date: Mon, 20 Jun 2022 17:20:36 +0800 Subject: [PATCH 047/104] MinMax Index Supports Nullable DataType (#5153) close pingcap/tiflash#4787 --- dbms/src/DataTypes/IDataType.h | 1 - .../Storages/DeltaMerge/File/DMFileWriter.cpp | 13 +- .../Storages/DeltaMerge/Index/MinMaxIndex.cpp | 191 +++++++++++++++++- .../Storages/DeltaMerge/Index/MinMaxIndex.h | 3 + .../tests/gtest_dm_minmax_index.cpp | 82 +++++++- 5 files changed, 268 insertions(+), 22 deletions(-) diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index 120d0b1ba30..71fda0615e4 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -471,7 +471,6 @@ class IDataType : private boost::noncopyable virtual bool isEnum() const { return false; }; virtual bool isNullable() const { return false; } - /** Is this type can represent only NULL value? (It also implies isNullable) */ virtual bool onlyNull() const { return false; } diff --git a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp index 3bff05ef19f..272d548eee1 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFileWriter.cpp @@ -72,10 +72,9 @@ DMFileWriter::DMFileWriter(const DMFilePtr & dmfile_, for (auto & cd : write_columns) { // TODO: currently we only generate index for Integers, Date, DateTime types, and this should be configurable by user. - // TODO: If column type is nullable, we won't generate index for it /// for handle column always generate index - bool do_index = cd.id == EXTRA_HANDLE_COLUMN_ID || cd.type->isInteger() || cd.type->isDateOrDateTime(); - + auto type = removeNullable(cd.type); + bool do_index = cd.id == EXTRA_HANDLE_COLUMN_ID || type->isInteger() || type->isDateOrDateTime(); if (options.flags.isSingleFile()) { if (do_index) @@ -122,7 +121,7 @@ void DMFileWriter::addStreams(ColId col_id, DataTypePtr type, bool do_index) void DMFileWriter::write(const Block & block, const BlockProperty & block_property) { is_empty_file = false; - DMFile::PackStat stat; + DMFile::PackStat stat{}; stat.rows = block.rows(); stat.not_clean = block_property.not_clean_rows; stat.bytes = block.bytes(); // This is bytes of pack data in memory. @@ -219,7 +218,7 @@ void DMFileWriter::writeColumn(ColId col_id, const IDataType & type, const IColu "Type shouldn be nullable when substream_path's type is NullMap.", Errors::DeltaTree::Internal); - const ColumnNullable & col = static_cast(column); + const auto & col = static_cast(column); col.checkConsistency(); DataTypeUInt8().serializeBinaryBulk(col.getNullMapColumn(), single_file_stream->original_layer, 0, rows); } @@ -230,8 +229,8 @@ void DMFileWriter::writeColumn(ColId col_id, const IDataType & type, const IColu "Type shouldn be nullable when substream_path's type is NullableElements.", Errors::DeltaTree::Internal); - const DataTypeNullable & nullable_type = static_cast(type); - const ColumnNullable & col = static_cast(column); + const auto & nullable_type = static_cast(type); + const auto & col = static_cast(column); nullable_type.getNestedType()->serializeBinaryBulk(col.getNestedColumn(), single_file_stream->original_layer, 0, rows); } else diff --git a/dbms/src/Storages/DeltaMerge/Index/MinMaxIndex.cpp b/dbms/src/Storages/DeltaMerge/Index/MinMaxIndex.cpp index 2681284948c..6229d54c169 100644 --- a/dbms/src/Storages/DeltaMerge/Index/MinMaxIndex.cpp +++ b/dbms/src/Storages/DeltaMerge/Index/MinMaxIndex.cpp @@ -61,7 +61,6 @@ inline std::pair minmax(const IColumn & column, const ColumnVect void MinMaxIndex::addPack(const IColumn & column, const ColumnVector * del_mark) { - const IColumn * column_ptr = &column; auto size = column.size(); bool has_null = false; if (column.isColumnNullable()) @@ -70,7 +69,6 @@ void MinMaxIndex::addPack(const IColumn & column, const ColumnVector * de const auto & nullable_column = static_cast(column); const auto & null_mark_data = nullable_column.getNullMapColumn().getData(); - column_ptr = &nullable_column.getNestedColumn(); for (size_t i = 0; i < size; ++i) { @@ -82,14 +80,13 @@ void MinMaxIndex::addPack(const IColumn & column, const ColumnVector * de } } - const IColumn & updated_column = *column_ptr; - auto [min_index, max_index] = details::minmax(updated_column, del_mark, 0, updated_column.size()); + auto [min_index, max_index] = details::minmax(column, del_mark, 0, column.size()); if (min_index != NONE_EXIST) { has_null_marks->push_back(has_null); has_value_marks->push_back(1); - minmaxes->insertFrom(updated_column, min_index); - minmaxes->insertFrom(updated_column, max_index); + minmaxes->insertFrom(column, min_index); + minmaxes->insertFrom(column, max_index); } else { @@ -158,6 +155,62 @@ std::pair MinMaxIndex::getUInt64MinMax(size_t pack_index) return {minmaxes->get64(pack_index * 2), minmaxes->get64(pack_index * 2 + 1)}; } +RSResult MinMaxIndex::checkNullableEqual(size_t pack_index, const Field & value, const DataTypePtr & type) +{ + const auto & column_nullable = static_cast(*minmaxes); + + const auto * raw_type = type.get(); + +#define DISPATCH(TYPE) \ + if (typeid_cast(raw_type)) \ + { \ + auto & minmaxes_data = toColumnVectorData(column_nullable.getNestedColumnPtr()); \ + auto min = minmaxes_data[pack_index * 2]; \ + auto max = minmaxes_data[pack_index * 2 + 1]; \ + return RoughCheck::checkEqual(value, type, min, max); \ + } + FOR_NUMERIC_TYPES(DISPATCH) +#undef DISPATCH + if (typeid_cast(raw_type)) + { + const auto & minmaxes_data = toColumnVectorData(column_nullable.getNestedColumnPtr()); + auto min = minmaxes_data[pack_index * 2]; + auto max = minmaxes_data[pack_index * 2 + 1]; + return RoughCheck::checkEqual(value, type, min, max); + } + if (typeid_cast(raw_type)) + { + const auto & minmaxes_data = toColumnVectorData(column_nullable.getNestedColumnPtr()); + auto min = minmaxes_data[pack_index * 2]; + auto max = minmaxes_data[pack_index * 2 + 1]; + return RoughCheck::checkEqual(value, type, min, max); + } + if (typeid_cast(raw_type) || typeid_cast(raw_type)) + { + // For DataTypeMyDateTime / DataTypeMyDate, simply compare them as comparing UInt64 is OK. + // Check `struct MyTimeBase` for more details. + const auto & minmaxes_data = toColumnVectorData(column_nullable.getNestedColumnPtr()); + auto min = minmaxes_data[pack_index * 2]; + auto max = minmaxes_data[pack_index * 2 + 1]; + return RoughCheck::checkEqual(value, type, min, max); + } + if (typeid_cast(raw_type)) + { + const auto * string_column = checkAndGetColumn(column_nullable.getNestedColumnPtr().get()); + const auto & chars = string_column->getChars(); + const auto & offsets = string_column->getOffsets(); + size_t pos = pack_index * 2; + size_t prev_offset = pos == 0 ? 0 : offsets[pos - 1]; + // todo use StringRef instead of String + auto min = String(chars[prev_offset], offsets[pos] - prev_offset - 1); + pos = pack_index * 2 + 1; + prev_offset = offsets[pos - 1]; + auto max = String(chars[prev_offset], offsets[pos] - prev_offset - 1); + return RoughCheck::checkEqual(value, type, min, max); + } + return RSResult::Some; +} + RSResult MinMaxIndex::checkEqual(size_t pack_index, const Field & value, const DataTypePtr & type) { if ((*has_null_marks)[pack_index] || value.isNull()) @@ -165,7 +218,13 @@ RSResult MinMaxIndex::checkEqual(size_t pack_index, const Field & value, const D if (!(*has_value_marks)[pack_index]) return RSResult::None; + // if minmaxes_data has null value, the value of minmaxes_data[i] is meaningless and maybe just some random value. + // But we have checked the has_null_marks above and ensured that there is no null value in MinMax Indexes. const auto * raw_type = type.get(); + if (typeid_cast(raw_type)) + { + return checkNullableEqual(pack_index, value, removeNullable(type)); + } #define DISPATCH(TYPE) \ if (typeid_cast(raw_type)) \ { \ @@ -215,6 +274,62 @@ RSResult MinMaxIndex::checkEqual(size_t pack_index, const Field & value, const D } return RSResult::Some; } + +RSResult MinMaxIndex::checkNullableGreater(size_t pack_index, const Field & value, const DataTypePtr & type) +{ + const auto & column_nullable = static_cast(*minmaxes); + const auto * raw_type = type.get(); + +#define DISPATCH(TYPE) \ + if (typeid_cast(raw_type)) \ + { \ + auto & minmaxes_data = toColumnVectorData(column_nullable.getNestedColumnPtr()); \ + auto min = minmaxes_data[pack_index * 2]; \ + auto max = minmaxes_data[pack_index * 2 + 1]; \ + return RoughCheck::checkGreater(value, type, min, max); \ + } + FOR_NUMERIC_TYPES(DISPATCH) +#undef DISPATCH + if (typeid_cast(raw_type)) + { + const auto & minmaxes_data = toColumnVectorData(column_nullable.getNestedColumnPtr()); + auto min = minmaxes_data[pack_index * 2]; + auto max = minmaxes_data[pack_index * 2 + 1]; + return RoughCheck::checkGreater(value, type, min, max); + } + if (typeid_cast(raw_type)) + { + const auto & minmaxes_data = toColumnVectorData(column_nullable.getNestedColumnPtr()); + auto min = minmaxes_data[pack_index * 2]; + auto max = minmaxes_data[pack_index * 2 + 1]; + return RoughCheck::checkGreater(value, type, min, max); + } + if (typeid_cast(raw_type) || typeid_cast(raw_type)) + { + // For DataTypeMyDateTime / DataTypeMyDate, simply compare them as comparing UInt64 is OK. + // Check `struct MyTimeBase` for more details. + const auto & minmaxes_data = toColumnVectorData(column_nullable.getNestedColumnPtr()); + auto min = minmaxes_data[pack_index * 2]; + auto max = minmaxes_data[pack_index * 2 + 1]; + return RoughCheck::checkGreater(value, type, min, max); + } + if (typeid_cast(raw_type)) + { + const auto * string_column = checkAndGetColumn(column_nullable.getNestedColumnPtr().get()); + const auto & chars = string_column->getChars(); + const auto & offsets = string_column->getOffsets(); + size_t pos = pack_index * 2; + size_t prev_offset = pos == 0 ? 0 : offsets[pos - 1]; + // todo use StringRef instead of String + auto min = String(chars[prev_offset], offsets[pos] - prev_offset - 1); + pos = pack_index * 2 + 1; + prev_offset = offsets[pos - 1]; + auto max = String(chars[prev_offset], offsets[pos] - prev_offset - 1); + return RoughCheck::checkGreater(value, type, min, max); + } + return RSResult::Some; +} + RSResult MinMaxIndex::checkGreater(size_t pack_index, const Field & value, const DataTypePtr & type, int /*nan_direction_hint*/) { if ((*has_null_marks)[pack_index] || value.isNull()) @@ -223,6 +338,10 @@ RSResult MinMaxIndex::checkGreater(size_t pack_index, const Field & value, const return RSResult::None; const auto * raw_type = type.get(); + if (typeid_cast(raw_type)) + { + return checkNullableGreater(pack_index, value, removeNullable(type)); + } #define DISPATCH(TYPE) \ if (typeid_cast(raw_type)) \ { \ @@ -272,6 +391,62 @@ RSResult MinMaxIndex::checkGreater(size_t pack_index, const Field & value, const } return RSResult::Some; } + +RSResult MinMaxIndex::checkNullableGreaterEqual(size_t pack_index, const Field & value, const DataTypePtr & type) +{ + const auto & column_nullable = static_cast(*minmaxes); + + const auto * raw_type = type.get(); +#define DISPATCH(TYPE) \ + if (typeid_cast(raw_type)) \ + { \ + auto & minmaxes_data = toColumnVectorData(column_nullable.getNestedColumnPtr()); \ + auto min = minmaxes_data[pack_index * 2]; \ + auto max = minmaxes_data[pack_index * 2 + 1]; \ + return RoughCheck::checkGreaterEqual(value, type, min, max); \ + } + FOR_NUMERIC_TYPES(DISPATCH) +#undef DISPATCH + if (typeid_cast(raw_type)) + { + const auto & minmaxes_data = toColumnVectorData(column_nullable.getNestedColumnPtr()); + auto min = minmaxes_data[pack_index * 2]; + auto max = minmaxes_data[pack_index * 2 + 1]; + return RoughCheck::checkGreaterEqual(value, type, min, max); + } + if (typeid_cast(raw_type)) + { + const auto & minmaxes_data = toColumnVectorData(column_nullable.getNestedColumnPtr()); + auto min = minmaxes_data[pack_index * 2]; + auto max = minmaxes_data[pack_index * 2 + 1]; + return RoughCheck::checkGreaterEqual(value, type, min, max); + } + if (typeid_cast(raw_type) || typeid_cast(raw_type)) + { + // For DataTypeMyDateTime / DataTypeMyDate, simply compare them as comparing UInt64 is OK. + // Check `struct MyTimeBase` for more details. + const auto & minmaxes_data = toColumnVectorData(column_nullable.getNestedColumnPtr()); + auto min = minmaxes_data[pack_index * 2]; + auto max = minmaxes_data[pack_index * 2 + 1]; + return RoughCheck::checkGreaterEqual(value, type, min, max); + } + if (typeid_cast(raw_type)) + { + const auto * string_column = checkAndGetColumn(column_nullable.getNestedColumnPtr().get()); + const auto & chars = string_column->getChars(); + const auto & offsets = string_column->getOffsets(); + size_t pos = pack_index * 2; + size_t prev_offset = pos == 0 ? 0 : offsets[pos - 1]; + // todo use StringRef instead of String + auto min = String(reinterpret_cast(&chars[prev_offset]), offsets[pos] - prev_offset - 1); + pos = pack_index * 2 + 1; + prev_offset = offsets[pos - 1]; + auto max = String(reinterpret_cast(&chars[prev_offset]), offsets[pos] - prev_offset - 1); + return RoughCheck::checkGreaterEqual(value, type, min, max); + } + return RSResult::Some; +} + RSResult MinMaxIndex::checkGreaterEqual(size_t pack_index, const Field & value, const DataTypePtr & type, int /*nan_direction_hint*/) { if ((*has_null_marks)[pack_index] || value.isNull()) @@ -280,6 +455,10 @@ RSResult MinMaxIndex::checkGreaterEqual(size_t pack_index, const Field & value, return RSResult::None; const auto * raw_type = type.get(); + if (typeid_cast(raw_type)) + { + return checkNullableGreaterEqual(pack_index, value, removeNullable(type)); + } #define DISPATCH(TYPE) \ if (typeid_cast(raw_type)) \ { \ diff --git a/dbms/src/Storages/DeltaMerge/Index/MinMaxIndex.h b/dbms/src/Storages/DeltaMerge/Index/MinMaxIndex.h index 7efd37fafa4..73284333c73 100644 --- a/dbms/src/Storages/DeltaMerge/Index/MinMaxIndex.h +++ b/dbms/src/Storages/DeltaMerge/Index/MinMaxIndex.h @@ -81,6 +81,9 @@ class MinMaxIndex RSResult checkGreaterEqual(size_t pack_index, const Field & value, const DataTypePtr & type, int nan_direction); static String toString(); + RSResult checkNullableEqual(size_t pack_index, const Field & value, const DataTypePtr & type); + RSResult checkNullableGreater(size_t pack_index, const Field & value, const DataTypePtr & type); + RSResult checkNullableGreaterEqual(size_t pack_index, const Field & value, const DataTypePtr & type); }; diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp index 96c0070b73b..bb31b687186 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_minmax_index.cpp @@ -214,14 +214,6 @@ try ASSERT_EQ(true, checkMatch(case_name, *context, "MyDateTime", "2020-09-27", createLessEqual(attr("MyDateTime"), parseMyDateTime("2020-09-27"), 0))); ASSERT_EQ(false, checkMatch(case_name, *context, "MyDateTime", "2020-09-27", createLessEqual(attr("MyDateTime"), parseMyDateTime("2020-09-26"), 0))); - /// Currently we don't do filtering for null values. i.e. if a pack contains any null values, then the pack will pass the filter. - ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "100"}, {"1", "1", "0", "\\N"}}, createEqual(attr("Nullable(Int64)"), Field((Int64)101)))); - ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "100"}, {"1", "1", "0", "\\N"}}, createIn(attr("Nullable(Int64)"), {Field((Int64)101)}))); - ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "100"}, {"1", "1", "0", "\\N"}}, createGreater(attr("Nullable(Int64)"), Field((Int64)100), 0))); - ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "100"}, {"1", "1", "0", "\\N"}}, createGreaterEqual(attr("Nullable(Int64)"), Field((Int64)101), 0))); - ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "100"}, {"1", "1", "0", "\\N"}}, createLess(attr("Nullable(Int64)"), Field((Int64)100), 0))); - ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "100"}, {"1", "1", "0", "\\N"}}, createLessEqual(attr("Nullable(Int64)"), Field((Int64)99), 0))); - ASSERT_EQ(false, checkDelMatch(case_name, *context, "Int64", "100", createEqual(attr("Int64"), Field((Int64)100)))); ASSERT_EQ(true, checkPkMatch(case_name, *context, "Int64", "100", createEqual(pkAttr(), Field((Int64)100)), true)); ASSERT_EQ(true, checkPkMatch(case_name, *context, "Int64", "100", createGreater(pkAttr(), Field((Int64)99), 0), true)); @@ -236,6 +228,80 @@ try } CATCH +TEST_F(DMMinMaxIndexTest, NullableToNullable) +try +{ + const auto * case_name = ::testing::UnitTest::GetInstance()->current_test_info()->name(); + // clang-format off + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(Int64)", "100", createEqual(attr("Nullable(Int64)"), Field((Int64)101)))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", "100", createEqual(attr("Nullable(Int64)"), Field((Int64)100)))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", "100", createIn(attr("Nullable(Int64)"), {Field((Int64)100)}))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(Int64)", "100", createIn(attr("Nullable(Int64)"), {Field((Int64)101)}))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", "100", createGreater(attr("Nullable(Int64)"), Field((Int64)99), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(Int64)", "100", createGreater(attr("Nullable(Int64)"), Field((Int64)100), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", "100", createGreaterEqual(attr("Nullable(Int64)"), Field((Int64)100), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(Int64)", "100", createGreaterEqual(attr("Nullable(Int64)"), Field((Int64)101), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", "100", createLess(attr("Nullable(Int64)"), Field((Int64)101), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(Int64)", "100", createLess(attr("Nullable(Int64)"), Field((Int64)100), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", "100", createLessEqual(attr("Nullable(Int64)"), Field((Int64)100), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(Int64)", "100", createLessEqual(attr("Nullable(Int64)"), Field((Int64)99), 0))); + + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Date)", "2020-09-27", createEqual(attr("Nullable(Date)"), Field((String) "2020-09-27")))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(Date)", "2020-09-27", createEqual(attr("Nullable(Date)"), Field((String) "2020-09-28")))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Date)", "2020-09-27", createIn(attr("Nullable(Date)"), {Field((String) "2020-09-27")}))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(Date)", "2020-09-27", createIn(attr("Nullable(Date)"), {Field((String) "2020-09-28")}))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Date)", "2020-09-27", createGreater(attr("Nullable(Date)"), Field((String) "2020-09-26"), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(Date)", "2020-09-27", createGreater(attr("Nullable(Date)"), Field((String) "2020-09-27"), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Date)", "2020-09-27", createGreaterEqual(attr("Nullable(Date)"), Field((String) "2020-09-27"), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(Date)", "2020-09-27", createGreaterEqual(attr("Nullable(Date)"), Field((String) "2020-09-28"), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Date)", "2020-09-27", createLess(attr("Nullable(Date)"), Field((String) "2020-09-28"), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(Date)", "2020-09-27", createLess(attr("Nullable(Date)"), Field((String) "2020-09-27"), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Date)", "2020-09-27", createLessEqual(attr("Nullable(Date)"), Field((String) "2020-09-27"), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(Date)", "2020-09-27", createLessEqual(attr("Nullable(Date)"), Field((String) "2020-09-26"), 0))); + + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(DateTime)", "2020-01-01 05:00:01", createEqual(attr("Nullable(DateTime)"), Field((String) "2020-01-01 05:00:01")))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(DateTime)", "2020-01-01 05:00:01", createEqual(attr("Nullable(DateTime)"), Field((String) "2020-01-01 05:00:02")))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(DateTime)", "2020-01-01 05:00:01", createIn(attr("Nullable(DateTime)"), {Field((String) "2020-01-01 05:00:01")}))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(DateTime)", "2020-01-01 05:00:01", createIn(attr("Nullable(DateTime)"), {Field((String) "2020-01-01 05:00:02")}))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(DateTime)", "2020-01-01 05:00:01", createGreater(attr("Nullable(DateTime)"), Field((String) "2020-01-01 05:00:00"), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(DateTime)", "2020-01-01 05:00:01", createGreater(attr("Nullable(DateTime)"), Field((String) "2020-01-01 05:00:01"), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(DateTime)", "2020-01-01 05:00:01", createGreaterEqual(attr("Nullable(DateTime)"), Field((String) "2020-01-01 05:00:01"), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(DateTime)", "2020-01-01 05:00:01", createGreaterEqual(attr("Nullable(DateTime)"), Field((String) "2020-01-01 05:00:02"), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(DateTime)", "2020-01-01 05:00:01", createLess(attr("Nullable(DateTime)"), Field((String) "2020-01-01 05:00:02"), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(DateTime)", "2020-01-01 05:00:01", createLess(attr("Nullable(DateTime)"), Field((String) "2020-01-01 05:00:01"), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(DateTime)", "2020-01-01 05:00:01", createLessEqual(attr("Nullable(DateTime)"), Field((String) "2020-01-01 05:00:01"), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(DateTime)", "2020-01-01 05:00:01", createLessEqual(attr("Nullable(DateTime)"), Field((String) "2020-01-01 05:00:00"), 0))); + + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(MyDateTime)", "2020-09-27", createEqual(attr("Nullable(MyDateTime)"), parseMyDateTime("2020-09-27")))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(MyDateTime)", "2020-09-27", createEqual(attr("Nullable(MyDateTime)"), parseMyDateTime("2020-09-28")))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(MyDateTime)", "2020-09-27", createIn(attr("Nullable(MyDateTime)"), {parseMyDateTime("2020-09-27")}))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(MyDateTime)", "2020-09-27", createIn(attr("Nullable(MyDateTime)"), {parseMyDateTime("2020-09-28")}))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(MyDateTime)", "2020-09-27", createGreater(attr("Nullable(MyDateTime)"), parseMyDateTime("2020-09-26"), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(MyDateTime)", "2020-09-27", createGreater(attr("Nullable(MyDateTime)"), parseMyDateTime("2020-09-27"), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(MyDateTime)", "2020-09-27", createGreaterEqual(attr("Nullable(MyDateTime)"), parseMyDateTime("2020-09-27"), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(MyDateTime)", "2020-09-27", createGreaterEqual(attr("Nullable(MyDateTime)"), parseMyDateTime("2020-09-28"), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(MyDateTime)", "2020-09-27", createLess(attr("Nullable(MyDateTime)"), parseMyDateTime("2020-09-28"), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(MyDateTime)", "2020-09-27", createLess(attr("Nullable(MyDateTime)"), parseMyDateTime("2020-09-27"), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(MyDateTime)", "2020-09-27", createLessEqual(attr("Nullable(MyDateTime)"), parseMyDateTime("2020-09-27"), 0))); + ASSERT_EQ(false, checkMatch(case_name, *context, "Nullable(MyDateTime)", "2020-09-27", createLessEqual(attr("Nullable(MyDateTime)"), parseMyDateTime("2020-09-26"), 0))); + + // has null + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "100"}, {"1", "1", "0", "\\N"}}, createEqual(attr("Nullable(Int64)"), Field((Int64)101)))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "100"}, {"1", "1", "0", "\\N"}}, createIn(attr("Nullable(Int64)"), {Field((Int64)101)}))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "100"}, {"1", "1", "0", "\\N"}}, createGreater(attr("Nullable(Int64)"), Field((Int64)100), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "100"}, {"1", "1", "0", "\\N"}}, createGreaterEqual(attr("Nullable(Int64)"), Field((Int64)101), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "100"}, {"1", "1", "0", "\\N"}}, createLess(attr("Nullable(Int64)"), Field((Int64)100), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "100"}, {"1", "1", "0", "\\N"}}, createLessEqual(attr("Nullable(Int64)"), Field((Int64)99), 0))); + + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "\\N"}}, createEqual(attr("Nullable(Int64)"), Field((Int64)101)))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "\\N"}}, createIn(attr("Nullable(Int64)"), {Field((Int64)101)}))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "\\N"}}, createGreater(attr("Nullable(Int64)"), Field((Int64)100), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "\\N"}}, createGreaterEqual(attr("Nullable(Int64)"), Field((Int64)101), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "\\N"}}, createLess(attr("Nullable(Int64)"), Field((Int64)100), 0))); + ASSERT_EQ(true, checkMatch(case_name, *context, "Nullable(Int64)", {{"0", "0", "0", "\\N"}}, createLessEqual(attr("Nullable(Int64)"), Field((Int64)99), 0))); +} +CATCH + TEST_F(DMMinMaxIndexTest, Logical) try { From 40baecabe6563aef92b210018d1893f8236416c5 Mon Sep 17 00:00:00 2001 From: Meng Xin Date: Mon, 20 Jun 2022 18:02:37 +0800 Subject: [PATCH 048/104] Reduce some unnecessary prometheus metrics. (#5006) close pingcap/tiflash#5080 --- dbms/src/Client/Connection.cpp | 8 - .../src/Client/ConnectionPoolWithFailover.cpp | 11 +- dbms/src/Common/Arena.h | 10 - dbms/src/Common/CurrentMetrics.cpp | 25 - dbms/src/Common/PoolWithFailoverBase.h | 8 - dbms/src/Common/ProfileEvents.cpp | 246 +++----- dbms/src/Common/TiFlashMetrics.h | 71 +-- .../AggregatingBlockInputStream.cpp | 8 - .../AsynchronousBlockInputStream.h | 8 - .../DataStreams/CountingBlockOutputStream.cpp | 16 +- .../MergeSortingBlockInputStream.cpp | 9 - ...regatedMemoryEfficientBlockInputStream.cpp | 13 +- .../ParallelAggregatingBlockInputStream.cpp | 9 - .../src/DataStreams/ParallelInputsProcessor.h | 7 - dbms/src/Dictionaries/CacheDictionary.cpp | 596 +++++++++--------- .../ComplexKeyCacheDictionary.cpp | 148 ++--- .../Dictionaries/ComplexKeyCacheDictionary.h | 178 +++--- .../WriteBufferFromFileProvider.cpp | 4 +- ...teReadBufferFromFileBaseByFileProvider.cpp | 6 - ...eWriteBufferFromFileBaseByFileProvider.cpp | 7 - dbms/src/Functions/FunctionsGeo.cpp | 44 +- dbms/src/Functions/Regexps.h | 8 - dbms/src/IO/BufferWithOwnMemory.h | 11 - dbms/src/IO/ChecksumBuffer.h | 4 - dbms/src/IO/CompressedReadBufferBase.cpp | 13 - dbms/src/IO/ReadBufferFromFileDescriptor.cpp | 6 +- dbms/src/IO/WriteBufferFromFileDescriptor.cpp | 2 - dbms/src/IO/createReadBufferFromFileBase.cpp | 9 - dbms/src/IO/createWriteBufferFromFileBase.cpp | 9 - dbms/src/Interpreters/Aggregator.cpp | 23 +- dbms/src/Interpreters/Context.cpp | 6 +- dbms/src/Interpreters/ExpressionActions.cpp | 7 - .../Interpreters/InterpreterInsertQuery.cpp | 11 +- .../Interpreters/InterpreterSelectQuery.cpp | 2 - dbms/src/Interpreters/ProcessList.h | 8 - dbms/src/Interpreters/QueryPriorities.h | 8 - dbms/src/Server/HTTPHandler.h | 20 +- dbms/src/Server/Server.cpp | 7 - dbms/src/Server/TCPHandler.h | 7 - .../src/Storages/BackgroundProcessingPool.cpp | 3 - dbms/src/Storages/MarkCache.h | 18 +- dbms/src/Storages/StorageBuffer.cpp | 37 -- dbms/src/TableFunctions/ITableFunction.cpp | 7 - 43 files changed, 612 insertions(+), 1046 deletions(-) diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 61a2843ac59..e21bde19a47 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -38,12 +38,6 @@ #include #endif - -namespace CurrentMetrics -{ -extern const Metric SendExternalTables; -} - namespace DB { namespace ErrorCodes @@ -434,8 +428,6 @@ void Connection::sendExternalTablesData(ExternalTablesData & data) size_t maybe_compressed_out_bytes = maybe_compressed_out ? maybe_compressed_out->count() : 0; size_t rows = 0; - CurrentMetrics::Increment metric_increment{CurrentMetrics::SendExternalTables}; - for (auto & elem : data) { elem.first->readPrefix(); diff --git a/dbms/src/Client/ConnectionPoolWithFailover.cpp b/dbms/src/Client/ConnectionPoolWithFailover.cpp index a9b6825a3fe..179b2d92c0e 100644 --- a/dbms/src/Client/ConnectionPoolWithFailover.cpp +++ b/dbms/src/Client/ConnectionPoolWithFailover.cpp @@ -20,13 +20,6 @@ #include #include - -namespace ProfileEvents -{ -extern const Event DistributedConnectionMissingTable; -extern const Event DistributedConnectionStaleReplica; -} // namespace ProfileEvents - namespace DB { namespace ErrorCodes @@ -50,7 +43,7 @@ ConnectionPoolWithFailover::ConnectionPoolWithFailover( hostname_differences.resize(nested_pools.size()); for (size_t i = 0; i < nested_pools.size(); ++i) { - ConnectionPool & connection_pool = dynamic_cast(*nested_pools[i]); + auto & connection_pool = dynamic_cast(*nested_pools[i]); hostname_differences[i] = getHostNameDifference(local_hostname, connection_pool.getHost()); } } @@ -187,7 +180,6 @@ ConnectionPoolWithFailover::tryGetEntry( fail_message = "There is no table " + table_to_check->database + "." + table_to_check->table + " on server: " + result.entry->getDescription(); LOG_WARNING(log, fail_message); - ProfileEvents::increment(ProfileEvents::DistributedConnectionMissingTable); return result; } @@ -217,7 +209,6 @@ ConnectionPoolWithFailover::tryGetEntry( table_to_check->database, table_to_check->table, delay); - ProfileEvents::increment(ProfileEvents::DistributedConnectionStaleReplica); } } catch (const Exception & e) diff --git a/dbms/src/Common/Arena.h b/dbms/src/Common/Arena.h index c61ebfca8aa..ebaaf607a6d 100644 --- a/dbms/src/Common/Arena.h +++ b/dbms/src/Common/Arena.h @@ -24,13 +24,6 @@ #include #include - -namespace ProfileEvents -{ -extern const Event ArenaAllocChunks; -extern const Event ArenaAllocBytes; -} // namespace ProfileEvents - namespace DB { /** Memory pool to append something. For example, short strings. @@ -55,9 +48,6 @@ class Arena : private boost::noncopyable Chunk(size_t size_, Chunk * prev_) { - ProfileEvents::increment(ProfileEvents::ArenaAllocChunks); - ProfileEvents::increment(ProfileEvents::ArenaAllocBytes, size_); - begin = reinterpret_cast(Allocator::alloc(size_)); pos = begin; end = begin + size_; diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index 8a2f111d882..b7ce9fd1e89 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -17,36 +17,11 @@ /// Available metrics. Add something here as you wish. #define APPLY_FOR_METRICS(M) \ - M(Query) \ - M(Merge) \ - M(ReplicatedFetch) \ - M(ReplicatedSend) \ - M(ReplicatedChecks) \ - M(BackgroundPoolTask) \ - M(DiskSpaceReservedForMerge) \ - M(DistributedSend) \ - M(QueryPreempted) \ - M(TCPConnection) \ - M(HTTPConnection) \ - M(InterserverConnection) \ M(OpenFileForRead) \ M(OpenFileForWrite) \ M(OpenFileForReadWrite) \ - M(SendExternalTables) \ - M(QueryThread) \ - M(ReadonlyReplica) \ - M(LeaderReplica) \ M(MemoryTracking) \ M(MemoryTrackingInBackgroundProcessingPool) \ - M(MemoryTrackingForMerges) \ - M(LeaderElection) \ - M(EphemeralNode) \ - M(DelayedInserts) \ - M(ContextLockWait) \ - M(StorageBufferRows) \ - M(StorageBufferBytes) \ - M(DictCacheRequests) \ - M(Revision) \ M(PSMVCCNumSnapshots) \ M(PSMVCCSnapshotsList) \ M(RWLockWaitingReaders) \ diff --git a/dbms/src/Common/PoolWithFailoverBase.h b/dbms/src/Common/PoolWithFailoverBase.h index a5483587e3c..04e6474c0fe 100644 --- a/dbms/src/Common/PoolWithFailoverBase.h +++ b/dbms/src/Common/PoolWithFailoverBase.h @@ -40,12 +40,6 @@ extern const int LOGICAL_ERROR; } // namespace ErrorCodes } // namespace DB -namespace ProfileEvents -{ -extern const Event DistributedConnectionFailTry; -extern const Event DistributedConnectionFailAtAll; -} // namespace ProfileEvents - /// This class provides a pool with fault tolerance. It is used for pooling of connections to replicated DB. /// Initialized by several PoolBase objects. /// When a connection is requested, tries to create or choose an alive connection from one of the nested pools. @@ -254,14 +248,12 @@ PoolWithFailoverBase::getMany( else { LOG_FMT_WARNING(log, "Connection failed at try No.{}, reason: {}", shuffled_pool.error_count + 1, fail_message); - ProfileEvents::increment(ProfileEvents::DistributedConnectionFailTry); ++shuffled_pool.error_count; if (shuffled_pool.error_count >= max_tries) { ++failed_pools_count; - ProfileEvents::increment(ProfileEvents::DistributedConnectionFailAtAll); } } } diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index 0ec1ce438a6..7507ff0b1f8 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -16,160 +16,98 @@ /// Available events. Add something here as you wish. -#define APPLY_FOR_EVENTS(M) \ - M(Query) \ - M(SelectQuery) \ - M(InsertQuery) \ - M(DeleteQuery) \ - M(FileOpen) \ - M(FileOpenFailed) \ - M(Seek) \ - M(ReadBufferFromFileDescriptorRead) \ - M(ReadBufferFromFileDescriptorReadFailed) \ - M(ReadBufferFromFileDescriptorReadBytes) \ - M(WriteBufferFromFileDescriptorWrite) \ - M(WriteBufferFromFileDescriptorWriteFailed) \ - M(WriteBufferFromFileDescriptorWriteBytes) \ - M(ReadBufferAIORead) \ - M(ReadBufferAIOReadBytes) \ - M(WriteBufferAIOWrite) \ - M(WriteBufferAIOWriteBytes) \ - M(ReadCompressedBytes) \ - M(CompressedReadBufferBlocks) \ - M(CompressedReadBufferBytes) \ - M(UncompressedCacheHits) \ - M(UncompressedCacheMisses) \ - M(UncompressedCacheWeightLost) \ - M(IOBufferAllocs) \ - M(IOBufferAllocBytes) \ - M(ArenaAllocChunks) \ - M(ArenaAllocBytes) \ - M(FunctionExecute) \ - M(TableFunctionExecute) \ - M(MarkCacheHits) \ - M(MarkCacheMisses) \ - M(CreatedReadBufferOrdinary) \ - M(CreatedReadBufferAIO) \ - M(CreatedWriteBufferOrdinary) \ - M(CreatedWriteBufferAIO) \ - \ - M(InsertedRows) \ - M(InsertedBytes) \ - M(DelayedInserts) \ - M(RejectedInserts) \ - M(DelayedInsertsMilliseconds) \ - M(DuplicatedInsertedBlocks) \ - \ - M(DistributedConnectionFailTry) \ - M(DistributedConnectionMissingTable) \ - M(DistributedConnectionStaleReplica) \ - M(DistributedConnectionFailAtAll) \ - \ - M(CompileAttempt) \ - M(CompileSuccess) \ - \ - M(ExternalSortWritePart) \ - M(ExternalSortMerge) \ - M(ExternalAggregationWritePart) \ - M(ExternalAggregationMerge) \ - M(ExternalAggregationCompressedBytes) \ - M(ExternalAggregationUncompressedBytes) \ - \ - M(SlowRead) \ - M(ReadBackoff) \ - \ - M(RegexpCreated) \ - M(ContextLock) \ - \ - M(StorageBufferFlush) \ - M(StorageBufferErrorOnFlush) \ - M(StorageBufferPassedAllMinThresholds) \ - M(StorageBufferPassedTimeMaxThreshold) \ - M(StorageBufferPassedRowsMaxThreshold) \ - M(StorageBufferPassedBytesMaxThreshold) \ - \ - M(DictCacheKeysRequested) \ - M(DictCacheKeysRequestedMiss) \ - M(DictCacheKeysRequestedFound) \ - M(DictCacheKeysExpired) \ - M(DictCacheKeysNotFound) \ - M(DictCacheKeysHit) \ - M(DictCacheRequestTimeNs) \ - M(DictCacheRequests) \ - M(DictCacheLockWriteNs) \ - M(DictCacheLockReadNs) \ - \ - M(DistributedSyncInsertionTimeoutExceeded) \ - M(DataAfterMergeDiffersFromReplica) \ - M(PolygonsAddedToPool) \ - M(PolygonsInPoolAllocatedBytes) \ - M(RWLockAcquiredReadLocks) \ - M(RWLockAcquiredWriteLocks) \ - M(RWLockReadersWaitMilliseconds) \ - M(RWLockWritersWaitMilliseconds) \ - \ - M(PSMWritePages) \ - M(PSMWriteIOCalls) \ - M(PSV3MBlobExpansion) \ - M(PSV3MBlobReused) \ - M(PSMWriteBytes) \ - M(PSMBackgroundWriteBytes) \ - M(PSMReadPages) \ - M(PSMBackgroundReadBytes) \ - \ - M(PSMReadIOCalls) \ - M(PSMReadBytes) \ - M(PSMWriteFailed) \ - M(PSMReadFailed) \ - \ - M(PSMVCCApplyOnCurrentBase) \ - M(PSMVCCApplyOnCurrentDelta) \ - M(PSMVCCApplyOnNewDelta) \ - M(PSMVCCCompactOnDelta) \ - M(PSMVCCCompactOnDeltaRebaseRejected) \ - M(PSMVCCCompactOnBase) \ - \ - M(DMWriteBytes) \ - M(DMWriteBlock) \ - M(DMWriteBlockNS) \ - M(DMWriteFile) \ - M(DMWriteFileNS) \ - M(DMDeleteRange) \ - M(DMDeleteRangeNS) \ - M(DMAppendDeltaPrepare) \ - M(DMAppendDeltaPrepareNS) \ - M(DMAppendDeltaCommitMemory) \ - M(DMAppendDeltaCommitMemoryNS) \ - M(DMAppendDeltaCommitDisk) \ - M(DMAppendDeltaCommitDiskNS) \ - M(DMAppendDeltaCleanUp) \ - M(DMAppendDeltaCleanUpNS) \ - M(DMPlace) \ - M(DMPlaceNS) \ - M(DMPlaceUpsert) \ - M(DMPlaceUpsertNS) \ - M(DMPlaceDeleteRange) \ - M(DMPlaceDeleteRangeNS) \ - M(DMDeltaMerge) \ - M(DMDeltaMergeNS) \ - M(DMSegmentSplit) \ - M(DMSegmentSplitNS) \ - M(DMSegmentGetSplitPoint) \ - M(DMSegmentGetSplitPointNS) \ - M(DMSegmentMerge) \ - M(DMSegmentMergeNS) \ - M(DMFlushDeltaCache) \ - M(DMFlushDeltaCacheNS) \ - M(DMCleanReadRows) \ - \ - M(FileFSync) \ - \ - M(DMFileFilterNoFilter) \ - M(DMFileFilterAftPKAndPackSet) \ - M(DMFileFilterAftRoughSet) \ - \ - M(ChecksumDigestBytes) \ - \ +#define APPLY_FOR_EVENTS(M) \ + M(Query) \ + M(FileOpen) \ + M(FileOpenFailed) \ + M(ReadBufferFromFileDescriptorRead) \ + M(ReadBufferFromFileDescriptorReadFailed) \ + M(ReadBufferFromFileDescriptorReadBytes) \ + M(WriteBufferFromFileDescriptorWrite) \ + M(WriteBufferFromFileDescriptorWriteBytes) \ + M(ReadBufferAIORead) \ + M(ReadBufferAIOReadBytes) \ + M(WriteBufferAIOWrite) \ + M(WriteBufferAIOWriteBytes) \ + \ + M(UncompressedCacheHits) \ + M(UncompressedCacheMisses) \ + M(UncompressedCacheWeightLost) \ + M(MarkCacheHits) \ + M(MarkCacheMisses) \ + \ + M(ExternalAggregationCompressedBytes) \ + M(ExternalAggregationUncompressedBytes) \ + \ + M(ContextLock) \ + \ + M(RWLockAcquiredReadLocks) \ + M(RWLockAcquiredWriteLocks) \ + M(RWLockReadersWaitMilliseconds) \ + M(RWLockWritersWaitMilliseconds) \ + \ + M(PSMWritePages) \ + M(PSMWriteIOCalls) \ + M(PSV3MBlobExpansion) \ + M(PSV3MBlobReused) \ + M(PSMWriteBytes) \ + M(PSMBackgroundWriteBytes) \ + M(PSMReadPages) \ + M(PSMBackgroundReadBytes) \ + \ + M(PSMReadIOCalls) \ + M(PSMReadBytes) \ + M(PSMWriteFailed) \ + M(PSMReadFailed) \ + \ + M(PSMVCCApplyOnCurrentBase) \ + M(PSMVCCApplyOnCurrentDelta) \ + M(PSMVCCApplyOnNewDelta) \ + M(PSMVCCCompactOnDelta) \ + M(PSMVCCCompactOnDeltaRebaseRejected) \ + M(PSMVCCCompactOnBase) \ + \ + M(DMWriteBytes) \ + M(DMWriteBlock) \ + M(DMWriteBlockNS) \ + M(DMWriteFile) \ + M(DMWriteFileNS) \ + M(DMDeleteRange) \ + M(DMDeleteRangeNS) \ + M(DMAppendDeltaPrepare) \ + M(DMAppendDeltaPrepareNS) \ + M(DMAppendDeltaCommitMemory) \ + M(DMAppendDeltaCommitMemoryNS) \ + M(DMAppendDeltaCommitDisk) \ + M(DMAppendDeltaCommitDiskNS) \ + M(DMAppendDeltaCleanUp) \ + M(DMAppendDeltaCleanUpNS) \ + M(DMPlace) \ + M(DMPlaceNS) \ + M(DMPlaceUpsert) \ + M(DMPlaceUpsertNS) \ + M(DMPlaceDeleteRange) \ + M(DMPlaceDeleteRangeNS) \ + M(DMDeltaMerge) \ + M(DMDeltaMergeNS) \ + M(DMSegmentSplit) \ + M(DMSegmentSplitNS) \ + M(DMSegmentGetSplitPoint) \ + M(DMSegmentGetSplitPointNS) \ + M(DMSegmentMerge) \ + M(DMSegmentMergeNS) \ + M(DMFlushDeltaCache) \ + M(DMFlushDeltaCacheNS) \ + M(DMCleanReadRows) \ + \ + M(FileFSync) \ + \ + M(DMFileFilterNoFilter) \ + M(DMFileFilterAftPKAndPackSet) \ + M(DMFileFilterAftRoughSet) \ + \ + M(ChecksumDigestBytes) \ + \ M(RaftWaitIndexTimeout) namespace ProfileEvents diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 9aa826e0e30..c0ce60af01e 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -60,27 +60,27 @@ namespace DB F(type_partition_ts, {"type", "partition_table_scan"}), \ F(type_window, {"type", "window"}), F(type_window_sort, {"type", "window_sort"})) \ M(tiflash_coprocessor_request_duration_seconds, "Bucketed histogram of request duration", Histogram, \ - F(type_batch, {{"type", "batch"}}, ExpBuckets{0.0005, 2, 30}), F(type_cop, {{"type", "cop"}}, ExpBuckets{0.0005, 2, 30}), \ - F(type_super_batch, {{"type", "super_batch"}}, ExpBuckets{0.0005, 2, 30}), \ - F(type_dispatch_mpp_task, {{"type", "dispatch_mpp_task"}}, ExpBuckets{0.0005, 2, 30}), \ - F(type_mpp_establish_conn, {{"type", "mpp_establish_conn"}}, ExpBuckets{0.0005, 2, 30}), \ - F(type_cancel_mpp_task, {{"type", "cancel_mpp_task"}}, ExpBuckets{0.0005, 2, 30}), \ - F(type_run_mpp_task, {{"type", "run_mpp_task"}}, ExpBuckets{0.0005, 2, 30})) \ + F(type_batch, {{"type", "batch"}}, ExpBuckets{0.001, 2, 20}), F(type_cop, {{"type", "cop"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_super_batch, {{"type", "super_batch"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_dispatch_mpp_task, {{"type", "dispatch_mpp_task"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_mpp_establish_conn, {{"type", "mpp_establish_conn"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_cancel_mpp_task, {{"type", "cancel_mpp_task"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_run_mpp_task, {{"type", "run_mpp_task"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_coprocessor_request_memory_usage, "Bucketed histogram of request memory usage", Histogram, \ F(type_cop, {{"type", "cop"}}, ExpBuckets{1024 * 1024, 2, 16}), \ - F(type_super_batch, {{"type", "super_batch"}}, ExpBuckets{1024 * 1024, 2, 16}), \ - F(type_run_mpp_task, {{"type", "run_mpp_task"}}, ExpBuckets{1024 * 1024, 2, 16})) \ + F(type_super_batch, {{"type", "super_batch"}}, ExpBuckets{1024 * 1024, 2, 20}), \ + F(type_run_mpp_task, {{"type", "run_mpp_task"}}, ExpBuckets{1024 * 1024, 2, 20})) \ M(tiflash_coprocessor_request_error, "Total number of request error", Counter, F(reason_meet_lock, {"reason", "meet_lock"}), \ F(reason_region_not_found, {"reason", "region_not_found"}), F(reason_epoch_not_match, {"reason", "epoch_not_match"}), \ F(reason_kv_client_error, {"reason", "kv_client_error"}), F(reason_internal_error, {"reason", "internal_error"}), \ F(reason_other_error, {"reason", "other_error"})) \ M(tiflash_coprocessor_request_handle_seconds, "Bucketed histogram of request handle duration", Histogram, \ - F(type_batch, {{"type", "batch"}}, ExpBuckets{0.0005, 2, 30}), F(type_cop, {{"type", "cop"}}, ExpBuckets{0.0005, 2, 30}), \ - F(type_super_batch, {{"type", "super_batch"}}, ExpBuckets{0.0005, 2, 30}), \ - F(type_dispatch_mpp_task, {{"type", "dispatch_mpp_task"}}, ExpBuckets{0.0005, 2, 30}), \ - F(type_mpp_establish_conn, {{"type", "mpp_establish_conn"}}, ExpBuckets{0.0005, 2, 30}), \ - F(type_cancel_mpp_task, {{"type", "cancel_mpp_task"}}, ExpBuckets{0.0005, 2, 30}), \ - F(type_run_mpp_task, {{"type", "run_mpp_task"}}, ExpBuckets{0.0005, 2, 30})) \ + F(type_batch, {{"type", "batch"}}, ExpBuckets{0.001, 2, 20}), F(type_cop, {{"type", "cop"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_super_batch, {{"type", "super_batch"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_dispatch_mpp_task, {{"type", "dispatch_mpp_task"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_mpp_establish_conn, {{"type", "mpp_establish_conn"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_cancel_mpp_task, {{"type", "cancel_mpp_task"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_run_mpp_task, {{"type", "run_mpp_task"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_coprocessor_response_bytes, "Total bytes of response body", Counter) \ M(tiflash_schema_version, "Current version of tiflash cached schema", Gauge) \ M(tiflash_schema_applying, "Whether the schema is applying or not (holding lock)", Gauge) \ @@ -95,21 +95,14 @@ namespace DB F(type_alter_column_tp, {"type", "alter_column_type"}), F(type_rename_column, {"type", "rename_column"}), \ F(type_exchange_partition, {"type", "exchange_partition"})) \ M(tiflash_schema_apply_duration_seconds, "Bucketed histogram of ddl apply duration", Histogram, \ - F(type_ddl_apply_duration, {{"req", "ddl_apply_duration"}}, ExpBuckets{0.0005, 2, 20})) \ - M(tiflash_tmt_merge_count, "Total number of TMT engine merge", Counter) \ - M(tiflash_tmt_merge_duration_seconds, "Bucketed histogram of TMT engine merge duration", Histogram, \ - F(type_tmt_merge_duration, {{"type", "tmt_merge_duration"}}, ExpBuckets{0.0005, 2, 20})) \ - M(tiflash_tmt_write_parts_count, "Total number of TMT engine write parts", Counter) \ - M(tiflash_tmt_write_parts_duration_seconds, "Bucketed histogram of TMT engine write parts duration", Histogram, \ - F(type_tmt_write_duration, {{"type", "tmt_write_parts_duration"}}, ExpBuckets{0.0005, 2, 20})) \ - M(tiflash_tmt_read_parts_count, "Total number of TMT engine read parts", Gauge) \ + F(type_ddl_apply_duration, {{"req", "ddl_apply_duration"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_raft_read_index_count, "Total number of raft read index", Counter) \ M(tiflash_raft_read_index_duration_seconds, "Bucketed histogram of raft read index duration", Histogram, \ - F(type_raft_read_index_duration, {{"type", "tmt_raft_read_index_duration"}}, ExpBuckets{0.0005, 2, 20})) \ + F(type_raft_read_index_duration, {{"type", "tmt_raft_read_index_duration"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_raft_wait_index_duration_seconds, "Bucketed histogram of raft wait index duration", Histogram, \ - F(type_raft_wait_index_duration, {{"type", "tmt_raft_wait_index_duration"}}, ExpBuckets{0.0005, 2, 20})) \ + F(type_raft_wait_index_duration, {{"type", "tmt_raft_wait_index_duration"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_syncing_data_freshness, "The freshness of tiflash data with tikv data", Histogram, \ - F(type_syncing_data_freshness, {{"type", "data_freshness"}}, ExpBuckets{0.0005, 2, 20})) \ + F(type_syncing_data_freshness, {{"type", "data_freshness"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_storage_write_amplification, "The data write amplification in storage engine", Gauge) \ M(tiflash_storage_read_tasks_count, "Total number of storage engine read tasks", Counter) \ M(tiflash_storage_command_count, "Total number of storage's command, such as delete range / shutdown /startup", Counter, \ @@ -122,16 +115,16 @@ namespace DB F(type_seg_split, {"type", "seg_split"}), F(type_seg_split_fg, {"type", "seg_split_fg"}), \ F(type_seg_merge, {"type", "seg_merge"}), F(type_place_index_update, {"type", "place_index_update"})) \ M(tiflash_storage_subtask_duration_seconds, "Bucketed histogram of storage's sub task duration", Histogram, \ - F(type_delta_merge, {{"type", "delta_merge"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_delta_merge_fg, {{"type", "delta_merge_fg"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_delta_merge_fg_rpc, {{"type", "delta_merge_fg_rpc"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_delta_merge_bg_gc, {{"type", "delta_merge_bg_gc"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_delta_compact, {{"type", "delta_compact"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_delta_flush, {{"type", "delta_flush"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_seg_split, {{"type", "seg_split"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_seg_split_fg, {{"type", "seg_split_fg"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_seg_merge, {{"type", "seg_merge"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_place_index_update, {{"type", "place_index_update"}}, ExpBuckets{0.0005, 2, 20})) \ + F(type_delta_merge, {{"type", "delta_merge"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_delta_merge_fg, {{"type", "delta_merge_fg"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_delta_merge_fg_rpc, {{"type", "delta_merge_fg_rpc"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_delta_merge_bg_gc, {{"type", "delta_merge_bg_gc"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_delta_compact, {{"type", "delta_compact"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_delta_flush, {{"type", "delta_flush"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_seg_split, {{"type", "seg_split"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_seg_split_fg, {{"type", "seg_split_fg"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_seg_merge, {{"type", "seg_merge"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_place_index_update, {{"type", "place_index_update"}}, ExpBuckets{0.001, 2, 20})) \ M(tiflash_storage_throughput_bytes, "Calculate the throughput of tasks of storage in bytes", Gauge, /**/ \ F(type_write, {"type", "write"}), /**/ \ F(type_ingest, {"type", "ingest"}), /**/ \ @@ -145,8 +138,8 @@ namespace DB F(type_split, {"type", "split"}), /**/ \ F(type_merge, {"type", "merge"})) /**/ \ M(tiflash_storage_write_stall_duration_seconds, "The write stall duration of storage, in seconds", Histogram, /**/ \ - F(type_write, {{"type", "write"}}, ExpBuckets{0.0005, 2, 20}), /**/ \ - F(type_delete_range, {{"type", "delete_range"}}, ExpBuckets{0.0005, 2, 20})) /**/ \ + F(type_write, {{"type", "write"}}, ExpBuckets{0.001, 2, 20}), /**/ \ + F(type_delete_range, {{"type", "delete_range"}}, ExpBuckets{0.001, 2, 20})) /**/ \ M(tiflash_storage_page_gc_count, "Total number of page's gc execution.", Counter, \ F(type_exec, {"type", "exec"}), \ F(type_low_write, {"type", "low_write"}), \ @@ -170,7 +163,7 @@ namespace DB Histogram, /* these command usually cost servel seconds, increase the start bucket to 50ms */ \ F(type_ingest_sst, {{"type", "ingest_sst"}}, ExpBuckets{0.05, 2, 10}), \ F(type_apply_snapshot_predecode, {{"type", "snapshot_predecode"}}, ExpBuckets{0.05, 2, 10}), \ - F(type_apply_snapshot_predecode_sst2dt, {{"type", "snapshot_predecode_sst2dt"}}, ExpBuckets{0.05, 2, 10}), \ + F(type_apply_snapshot_predecode_sst2dt, {{"type", "snapshot_predecode_sst2dt"}}, ExpBuckets{0.05, 2, 10}), \ F(type_apply_snapshot_flush, {{"type", "snapshot_flush"}}, ExpBuckets{0.05, 2, 10})) \ M(tiflash_raft_process_keys, "Total number of keys processed in some types of Raft commands", Counter, \ F(type_apply_snapshot, {"type", "apply_snapshot"}), F(type_ingest_sst, {"type", "ingest_sst"})) \ @@ -212,7 +205,7 @@ namespace DB F(type_thread_hard_limit, {"type", "thread_hard_limit"}), \ F(type_hard_limit_exceeded_count, {"type", "hard_limit_exceeded_count"})) \ M(tiflash_task_scheduler_waiting_duration_seconds, "Bucketed histogram of task waiting for scheduling duration", Histogram, \ - F(type_task_scheduler_waiting_duration, {{"type", "task_waiting_duration"}}, ExpBuckets{0.0005, 2, 20})) + F(type_task_scheduler_waiting_duration, {{"type", "task_waiting_duration"}}, ExpBuckets{0.001, 2, 20})) // clang-format on diff --git a/dbms/src/DataStreams/AggregatingBlockInputStream.cpp b/dbms/src/DataStreams/AggregatingBlockInputStream.cpp index 0d9e907c5f4..4cd09d1ea63 100644 --- a/dbms/src/DataStreams/AggregatingBlockInputStream.cpp +++ b/dbms/src/DataStreams/AggregatingBlockInputStream.cpp @@ -17,12 +17,6 @@ #include #include - -namespace ProfileEvents -{ -extern const Event ExternalAggregationMerge; -} - namespace DB { Block AggregatingBlockInputStream::getHeader() const @@ -56,8 +50,6 @@ Block AggregatingBlockInputStream::readImpl() * then read and merge them, spending the minimum amount of memory. */ - ProfileEvents::increment(ProfileEvents::ExternalAggregationMerge); - if (!isCancelled()) { /// Flush data in the RAM to disk also. It's easier than merging on-disk and RAM data. diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.h b/dbms/src/DataStreams/AsynchronousBlockInputStream.h index e75d1603648..5b373c26e95 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.h @@ -22,12 +22,6 @@ #include #include - -namespace CurrentMetrics -{ -extern const Metric QueryThread; -} - namespace DB { /** Executes another BlockInputStream in a separate thread. @@ -141,8 +135,6 @@ class AsynchronousBlockInputStream : public IProfilingBlockInputStream /// Calculations that can be performed in a separate thread void calculate() { - CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; - try { if (first) diff --git a/dbms/src/DataStreams/CountingBlockOutputStream.cpp b/dbms/src/DataStreams/CountingBlockOutputStream.cpp index 26bc5a4566f..52dc6b598b9 100644 --- a/dbms/src/DataStreams/CountingBlockOutputStream.cpp +++ b/dbms/src/DataStreams/CountingBlockOutputStream.cpp @@ -12,20 +12,11 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include - - -namespace ProfileEvents -{ - extern const Event InsertedRows; - extern const Event InsertedBytes; -} - +#include namespace DB { - void CountingBlockOutputStream::write(const Block & block) { stream->write(block); @@ -33,9 +24,6 @@ void CountingBlockOutputStream::write(const Block & block) Progress local_progress(block.rows(), block.bytes(), 0); progress.incrementPiecewiseAtomically(local_progress); - ProfileEvents::increment(ProfileEvents::InsertedRows, local_progress.rows); - ProfileEvents::increment(ProfileEvents::InsertedBytes, local_progress.bytes); - if (process_elem) process_elem->updateProgressOut(local_progress); @@ -43,4 +31,4 @@ void CountingBlockOutputStream::write(const Block & block) progress_callback(local_progress); } -} +} // namespace DB diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp index e79426f686e..cf8db3f8711 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp @@ -19,13 +19,6 @@ #include #include - -namespace ProfileEvents -{ -extern const Event ExternalSortWritePart; -extern const Event ExternalSortMerge; -} // namespace ProfileEvents - namespace DB { /** Remove constant columns from block. @@ -136,7 +129,6 @@ Block MergeSortingBlockInputStream::readImpl() MergeSortingBlocksBlockInputStream block_in(blocks, description, log->identifier(), max_merged_block_size, limit); LOG_FMT_INFO(log, "Sorting and writing part of data into temporary file {}", path); - ProfileEvents::increment(ProfileEvents::ExternalSortWritePart); copyData(block_in, block_out, &is_cancelled); /// NOTE. Possibly limit disk usage. LOG_FMT_INFO(log, "Done writing part of data into temporary file {}", path); @@ -155,7 +147,6 @@ Block MergeSortingBlockInputStream::readImpl() else { /// If there was temporary files. - ProfileEvents::increment(ProfileEvents::ExternalSortMerge); LOG_FMT_INFO(log, "There are {} temporary sorted parts to merge.", temporary_files.size()); diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp index 5d0b677b792..3a1cc1eed31 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp @@ -19,13 +19,6 @@ #include - -namespace CurrentMetrics -{ -extern const Metric QueryThread; -} - - namespace DB { /** Scheme of operation: @@ -156,7 +149,7 @@ void MergingAggregatedMemoryEfficientBlockInputStream::cancel(bool kill) for (auto & input : inputs) { - if (IProfilingBlockInputStream * child = dynamic_cast(input.stream.get())) + if (auto * child = dynamic_cast(input.stream.get())) { try { @@ -198,7 +191,6 @@ void MergingAggregatedMemoryEfficientBlockInputStream::start() reading_pool->schedule( wrapInvocable(true, [&child] { - CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; child->readPrefix(); })); } @@ -309,8 +301,6 @@ void MergingAggregatedMemoryEfficientBlockInputStream::finalize() void MergingAggregatedMemoryEfficientBlockInputStream::mergeThread() { - CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; - try { while (!parallel_merge_data->finish) @@ -490,7 +480,6 @@ MergingAggregatedMemoryEfficientBlockInputStream::BlocksToMerge MergingAggregate if (need_that_input(input)) { reading_pool->schedule(wrapInvocable(true, [&input, &read_from_input] { - CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; read_from_input(input); })); } diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp index 1a59b979c29..f4f8dfc1338 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp @@ -20,13 +20,6 @@ #include #include - -namespace ProfileEvents -{ -extern const Event ExternalAggregationMerge; -} - - namespace DB { ParallelAggregatingBlockInputStream::ParallelAggregatingBlockInputStream( @@ -101,8 +94,6 @@ Block ParallelAggregatingBlockInputStream::readImpl() * then read and merge them, spending the minimum amount of memory. */ - ProfileEvents::increment(ProfileEvents::ExternalAggregationMerge); - const auto & files = aggregator.getTemporaryFiles(); BlockInputStreams input_streams; for (const auto & file : files.files) diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index 0e839093cd7..34c70a7085e 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -46,11 +46,6 @@ * then read block from source and then put source back to queue of available sources. */ -namespace CurrentMetrics -{ -extern const Metric QueryThread; -} - namespace DB { /** Union mode. @@ -208,8 +203,6 @@ class ParallelInputsProcessor { std::exception_ptr exception; - CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; - try { while (!finish) diff --git a/dbms/src/Dictionaries/CacheDictionary.cpp b/dbms/src/Dictionaries/CacheDictionary.cpp index 8573bdad6bd..0d7243ede8f 100644 --- a/dbms/src/Dictionaries/CacheDictionary.cpp +++ b/dbms/src/Dictionaries/CacheDictionary.cpp @@ -12,57 +12,36 @@ // See the License for the specific language governing permissions and // limitations under the License. -#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 ProfileEvents -{ - extern const Event DictCacheKeysRequested; - extern const Event DictCacheKeysRequestedMiss; - extern const Event DictCacheKeysRequestedFound; - extern const Event DictCacheKeysExpired; - extern const Event DictCacheKeysNotFound; - extern const Event DictCacheKeysHit; - extern const Event DictCacheRequestTimeNs; - extern const Event DictCacheRequests; - extern const Event DictCacheLockWriteNs; - extern const Event DictCacheLockReadNs; -} - -namespace CurrentMetrics -{ - extern const Metric DictCacheRequests; -} +#include +#include +#include +#include +#include +#include namespace DB { - namespace ErrorCodes { - extern const int TYPE_MISMATCH; - extern const int BAD_ARGUMENTS; - extern const int UNSUPPORTED_METHOD; - extern const int LOGICAL_ERROR; -} +extern const int TYPE_MISMATCH; +extern const int BAD_ARGUMENTS; +extern const int UNSUPPORTED_METHOD; +extern const int LOGICAL_ERROR; +} // namespace ErrorCodes inline size_t CacheDictionary::getCellIdx(const Key id) const @@ -73,15 +52,15 @@ inline size_t CacheDictionary::getCellIdx(const Key id) const } -CacheDictionary::CacheDictionary(const std::string & name, const DictionaryStructure & dict_struct, - DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime, - const size_t size) - : name{name}, dict_struct(dict_struct), - source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime), - size{roundUpToPowerOfTwoOrZero(std::max(size, size_t(max_collision_length)))}, - size_overlap_mask{this->size - 1}, - cells{this->size}, - rnd_engine(randomSeed()) +CacheDictionary::CacheDictionary(const std::string & name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime, const size_t size) + : name{name} + , dict_struct(dict_struct) + , source_ptr{std::move(source_ptr)} + , dict_lifetime(dict_lifetime) + , size{roundUpToPowerOfTwoOrZero(std::max(size, size_t(max_collision_length)))} + , size_overlap_mask{this->size - 1} + , cells{this->size} + , rnd_engine(randomSeed()) { if (!this->source_ptr->supportsSelectiveLoad()) throw Exception{ @@ -100,13 +79,19 @@ void CacheDictionary::toParent(const PaddedPODArray & ids, PaddedPODArray(hierarchical_attribute->null_values); - getItemsNumber(*hierarchical_attribute, ids, out, [&] (const size_t) { return null_value; }); + getItemsNumber(*hierarchical_attribute, ids, out, [&](const size_t) { return null_value; }); } /// Allow to use single value in same way as array. -static inline CacheDictionary::Key getAt(const PaddedPODArray & arr, const size_t idx) { return arr[idx]; } -static inline CacheDictionary::Key getAt(const CacheDictionary::Key & value, const size_t) { return value; } +static inline CacheDictionary::Key getAt(const PaddedPODArray & arr, const size_t idx) +{ + return arr[idx]; +} +static inline CacheDictionary::Key getAt(const CacheDictionary::Key & value, const size_t) +{ + return value; +} template @@ -118,7 +103,7 @@ void CacheDictionary::isInImpl( /// Transform all children to parents until ancestor id or null_value will be reached. size_t size = out.size(); - memset(out.data(), 0xFF, size); /// 0xFF means "not calculated" + memset(out.data(), 0xFF, size); /// 0xFF means "not calculated" const auto null_value = std::get(hierarchical_attribute->null_values); @@ -224,19 +209,19 @@ void CacheDictionary::isInConstantVector( } -#define DECLARE(TYPE)\ -void CacheDictionary::get##TYPE(const std::string & attribute_name, const PaddedPODArray & ids, PaddedPODArray & out) const\ -{\ - auto & attribute = getAttribute(attribute_name);\ - if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE))\ - throw Exception{\ - name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\ - ErrorCodes::TYPE_MISMATCH};\ - \ - const auto null_value = std::get(attribute.null_values);\ - \ - getItemsNumber(attribute, ids, out, [&] (const size_t) { return null_value; });\ -} +#define DECLARE(TYPE) \ + void CacheDictionary::get##TYPE(const std::string & attribute_name, const PaddedPODArray & ids, PaddedPODArray & out) const \ + { \ + auto & attribute = getAttribute(attribute_name); \ + if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE)) \ + throw Exception{ \ + name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \ + ErrorCodes::TYPE_MISMATCH}; \ + \ + const auto null_value = std::get(attribute.null_values); \ + \ + getItemsNumber(attribute, ids, out, [&](const size_t) { return null_value; }); \ + } DECLARE(UInt8) DECLARE(UInt16) DECLARE(UInt32) @@ -260,22 +245,24 @@ void CacheDictionary::getString(const std::string & attribute_name, const Padded const auto null_value = StringRef{std::get(attribute.null_values)}; - getItemsString(attribute, ids, out, [&] (const size_t) { return null_value; }); + getItemsString(attribute, ids, out, [&](const size_t) { return null_value; }); } -#define DECLARE(TYPE)\ -void CacheDictionary::get##TYPE(\ - const std::string & attribute_name, const PaddedPODArray & ids, const PaddedPODArray & def,\ - PaddedPODArray & out) const\ -{\ - auto & attribute = getAttribute(attribute_name);\ - if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE))\ - throw Exception{\ - name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\ - ErrorCodes::TYPE_MISMATCH};\ - \ - getItemsNumber(attribute, ids, out, [&] (const size_t row) { return def[row]; });\ -} +#define DECLARE(TYPE) \ + void CacheDictionary::get##TYPE( \ + const std::string & attribute_name, \ + const PaddedPODArray & ids, \ + const PaddedPODArray & def, \ + PaddedPODArray & out) const \ + { \ + auto & attribute = getAttribute(attribute_name); \ + if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE)) \ + throw Exception{ \ + name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \ + ErrorCodes::TYPE_MISMATCH}; \ + \ + getItemsNumber(attribute, ids, out, [&](const size_t row) { return def[row]; }); \ + } DECLARE(UInt8) DECLARE(UInt16) DECLARE(UInt32) @@ -290,7 +277,9 @@ DECLARE(Float64) #undef DECLARE void CacheDictionary::getString( - const std::string & attribute_name, const PaddedPODArray & ids, const ColumnString * const def, + const std::string & attribute_name, + const PaddedPODArray & ids, + const ColumnString * const def, ColumnString * const out) const { auto & attribute = getAttribute(attribute_name); @@ -299,21 +288,24 @@ void CacheDictionary::getString( name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH}; - getItemsString(attribute, ids, out, [&] (const size_t row) { return def->getDataAt(row); }); + getItemsString(attribute, ids, out, [&](const size_t row) { return def->getDataAt(row); }); } -#define DECLARE(TYPE)\ -void CacheDictionary::get##TYPE(\ - const std::string & attribute_name, const PaddedPODArray & ids, const TYPE def, PaddedPODArray & out) const\ -{\ - auto & attribute = getAttribute(attribute_name);\ - if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE))\ - throw Exception{\ - name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type),\ - ErrorCodes::TYPE_MISMATCH};\ - \ - getItemsNumber(attribute, ids, out, [&] (const size_t) { return def; });\ -} +#define DECLARE(TYPE) \ + void CacheDictionary::get##TYPE( \ + const std::string & attribute_name, \ + const PaddedPODArray & ids, \ + const TYPE def, \ + PaddedPODArray & out) const \ + { \ + auto & attribute = getAttribute(attribute_name); \ + if (!isAttributeTypeConvertibleTo(attribute.type, AttributeUnderlyingType::TYPE)) \ + throw Exception{ \ + name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), \ + ErrorCodes::TYPE_MISMATCH}; \ + \ + getItemsNumber(attribute, ids, out, [&](const size_t) { return def; }); \ + } DECLARE(UInt8) DECLARE(UInt16) DECLARE(UInt32) @@ -328,7 +320,9 @@ DECLARE(Float64) #undef DECLARE void CacheDictionary::getString( - const std::string & attribute_name, const PaddedPODArray & ids, const String & def, + const std::string & attribute_name, + const PaddedPODArray & ids, + const String & def, ColumnString * const out) const { auto & attribute = getAttribute(attribute_name); @@ -337,7 +331,7 @@ void CacheDictionary::getString( name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH}; - getItemsString(attribute, ids, out, [&] (const size_t) { return StringRef{def}; }); + getItemsString(attribute, ids, out, [&](const size_t) { return StringRef{def}; }); } @@ -390,8 +384,6 @@ void CacheDictionary::has(const PaddedPODArray & ids, PaddedPODArray const auto rows = ext::size(ids); { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - const auto now = std::chrono::system_clock::now(); /// fetch up-to-date values, decide which ones require update for (const auto row : ext::range(0, rows)) @@ -416,10 +408,6 @@ void CacheDictionary::has(const PaddedPODArray & ids, PaddedPODArray } } - ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); - ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); - ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); - query_count.fetch_add(rows, std::memory_order_relaxed); hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release); @@ -427,21 +415,19 @@ void CacheDictionary::has(const PaddedPODArray & ids, PaddedPODArray return; std::vector required_ids(outdated_ids.size()); - std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), - [] (auto & pair) { return pair.first; }); + std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), [](auto & pair) { return pair.first; }); /// request new values - update(required_ids, - [&] (const auto id, const auto) - { - for (const auto row : outdated_ids[id]) - out[row] = true; - }, - [&] (const auto id, const auto) - { - for (const auto row : outdated_ids[id]) - out[row] = false; - }); + update( + required_ids, + [&](const auto id, const auto) { + for (const auto row : outdated_ids[id]) + out[row] = true; + }, + [&](const auto id, const auto) { + for (const auto row : outdated_ids[id]) + out[row] = false; + }); } @@ -476,68 +462,68 @@ CacheDictionary::Attribute CacheDictionary::createAttributeWithType(const Attrib switch (type) { - case AttributeUnderlyingType::UInt8: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); - bytes_allocated += size * sizeof(UInt8); - break; - case AttributeUnderlyingType::UInt16: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); - bytes_allocated += size * sizeof(UInt16); - break; - case AttributeUnderlyingType::UInt32: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); - bytes_allocated += size * sizeof(UInt32); - break; - case AttributeUnderlyingType::UInt64: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); - bytes_allocated += size * sizeof(UInt64); - break; - case AttributeUnderlyingType::UInt128: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); - bytes_allocated += size * sizeof(UInt128); - break; - case AttributeUnderlyingType::Int8: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); - bytes_allocated += size * sizeof(Int8); - break; - case AttributeUnderlyingType::Int16: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); - bytes_allocated += size * sizeof(Int16); - break; - case AttributeUnderlyingType::Int32: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); - bytes_allocated += size * sizeof(Int32); - break; - case AttributeUnderlyingType::Int64: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); - bytes_allocated += size * sizeof(Int64); - break; - case AttributeUnderlyingType::Float32: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); - bytes_allocated += size * sizeof(Float32); - break; - case AttributeUnderlyingType::Float64: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); - bytes_allocated += size * sizeof(Float64); - break; - case AttributeUnderlyingType::String: - std::get(attr.null_values) = null_value.get(); - std::get>(attr.arrays) = std::make_unique>(size); - bytes_allocated += size * sizeof(StringRef); - if (!string_arena) - string_arena = std::make_unique(); - break; + case AttributeUnderlyingType::UInt8: + std::get(attr.null_values) = null_value.get(); + std::get>(attr.arrays) = std::make_unique>(size); + bytes_allocated += size * sizeof(UInt8); + break; + case AttributeUnderlyingType::UInt16: + std::get(attr.null_values) = null_value.get(); + std::get>(attr.arrays) = std::make_unique>(size); + bytes_allocated += size * sizeof(UInt16); + break; + case AttributeUnderlyingType::UInt32: + std::get(attr.null_values) = null_value.get(); + std::get>(attr.arrays) = std::make_unique>(size); + bytes_allocated += size * sizeof(UInt32); + break; + case AttributeUnderlyingType::UInt64: + std::get(attr.null_values) = null_value.get(); + std::get>(attr.arrays) = std::make_unique>(size); + bytes_allocated += size * sizeof(UInt64); + break; + case AttributeUnderlyingType::UInt128: + std::get(attr.null_values) = null_value.get(); + std::get>(attr.arrays) = std::make_unique>(size); + bytes_allocated += size * sizeof(UInt128); + break; + case AttributeUnderlyingType::Int8: + std::get(attr.null_values) = null_value.get(); + std::get>(attr.arrays) = std::make_unique>(size); + bytes_allocated += size * sizeof(Int8); + break; + case AttributeUnderlyingType::Int16: + std::get(attr.null_values) = null_value.get(); + std::get>(attr.arrays) = std::make_unique>(size); + bytes_allocated += size * sizeof(Int16); + break; + case AttributeUnderlyingType::Int32: + std::get(attr.null_values) = null_value.get(); + std::get>(attr.arrays) = std::make_unique>(size); + bytes_allocated += size * sizeof(Int32); + break; + case AttributeUnderlyingType::Int64: + std::get(attr.null_values) = null_value.get(); + std::get>(attr.arrays) = std::make_unique>(size); + bytes_allocated += size * sizeof(Int64); + break; + case AttributeUnderlyingType::Float32: + std::get(attr.null_values) = null_value.get(); + std::get>(attr.arrays) = std::make_unique>(size); + bytes_allocated += size * sizeof(Float32); + break; + case AttributeUnderlyingType::Float64: + std::get(attr.null_values) = null_value.get(); + std::get>(attr.arrays) = std::make_unique>(size); + bytes_allocated += size * sizeof(Float64); + break; + case AttributeUnderlyingType::String: + std::get(attr.null_values) = null_value.get(); + std::get>(attr.arrays) = std::make_unique>(size); + bytes_allocated += size * sizeof(StringRef); + if (!string_arena) + string_arena = std::make_unique(); + break; } return attr; @@ -551,8 +537,8 @@ void CacheDictionary::getItemsNumber( PaddedPODArray & out, DefaultGetter && get_default) const { - if (false) {} -#define DISPATCH(TYPE) \ + if (false) {} // NOLINT +#define DISPATCH(TYPE) \ else if (attribute.type == AttributeUnderlyingType::TYPE) \ getItemsNumberImpl(attribute, ids, out, std::forward(get_default)); DISPATCH(UInt8) @@ -567,8 +553,7 @@ void CacheDictionary::getItemsNumber( DISPATCH(Float32) DISPATCH(Float64) #undef DISPATCH - else - throw Exception("Unexpected type of attribute: " + toString(attribute.type), ErrorCodes::LOGICAL_ERROR); + else throw Exception("Unexpected type of attribute: " + toString(attribute.type), ErrorCodes::LOGICAL_ERROR); } template @@ -586,8 +571,6 @@ void CacheDictionary::getItemsNumberImpl( size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0; { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - const auto now = std::chrono::system_clock::now(); /// fetch up-to-date values, decide which ones require update for (const auto row : ext::range(0, rows)) @@ -618,10 +601,6 @@ void CacheDictionary::getItemsNumberImpl( } } - ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); - ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); - ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); - query_count.fetch_add(rows, std::memory_order_relaxed); hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release); @@ -629,23 +608,21 @@ void CacheDictionary::getItemsNumberImpl( return; std::vector required_ids(outdated_ids.size()); - std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), - [] (auto & pair) { return pair.first; }); + std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), [](auto & pair) { return pair.first; }); /// request new values - update(required_ids, - [&] (const auto id, const auto cell_idx) - { - const auto attribute_value = attribute_array[cell_idx]; + update( + required_ids, + [&](const auto id, const auto cell_idx) { + const auto attribute_value = attribute_array[cell_idx]; - for (const auto row : outdated_ids[id]) - out[row] = static_cast(attribute_value); - }, - [&] (const auto id, const auto) - { - for (const auto row : outdated_ids[id]) - out[row] = get_default(row); - }); + for (const auto row : outdated_ids[id]) + out[row] = static_cast(attribute_value); // NOLINT + }, + [&](const auto id, const auto) { + for (const auto row : outdated_ids[id]) + out[row] = get_default(row); + }); } template @@ -666,8 +643,6 @@ void CacheDictionary::getItemsString( /// perform optimistic version, fallback to pessimistic if failed { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - const auto now = std::chrono::system_clock::now(); /// fetch up-to-date values, discard on fail for (const auto row : ext::range(0, rows)) @@ -710,8 +685,6 @@ void CacheDictionary::getItemsString( size_t total_length = 0; size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0; { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - const auto now = std::chrono::system_clock::now(); for (const auto row : ext::range(0, ids.size())) { @@ -741,10 +714,6 @@ void CacheDictionary::getItemsString( } } - ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); - ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); - ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); - query_count.fetch_add(rows, std::memory_order_relaxed); hit_count.fetch_add(rows - outdated_ids.size(), std::memory_order_release); @@ -752,22 +721,20 @@ void CacheDictionary::getItemsString( if (!outdated_ids.empty()) { std::vector required_ids(outdated_ids.size()); - std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), - [] (auto & pair) { return pair.first; }); - - update(required_ids, - [&] (const auto id, const auto cell_idx) - { - const auto attribute_value = attribute_array[cell_idx]; - - map[id] = String{attribute_value}; - total_length += (attribute_value.size + 1) * outdated_ids[id].size(); - }, - [&] (const auto id, const auto) - { - for (const auto row : outdated_ids[id]) - total_length += get_default(row).size + 1; - }); + std::transform(std::begin(outdated_ids), std::end(outdated_ids), std::begin(required_ids), [](auto & pair) { return pair.first; }); + + update( + required_ids, + [&](const auto id, const auto cell_idx) { + const auto attribute_value = attribute_array[cell_idx]; + + map[id] = String{attribute_value}; + total_length += (attribute_value.size + 1) * outdated_ids[id].size(); + }, + [&](const auto id, const auto) { + for (const auto row : outdated_ids[id]) + total_length += get_default(row).size + 1; + }); } out->getChars().reserve(total_length); @@ -790,18 +757,13 @@ void CacheDictionary::update( { std::unordered_map remaining_ids{requested_ids.size()}; for (const auto id : requested_ids) - remaining_ids.insert({ id, 0 }); + remaining_ids.insert({id, 0}); - std::uniform_int_distribution distribution - { + std::uniform_int_distribution distribution{ dict_lifetime.min_sec, - dict_lifetime.max_sec - }; - - const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; + dict_lifetime.max_sec}; { - CurrentMetrics::Increment metric_increment{CurrentMetrics::DictCacheRequests}; Stopwatch watch; auto stream = source_ptr->loadIds(requested_ids); stream->readPrefix(); @@ -810,7 +772,7 @@ void CacheDictionary::update( while (const auto block = stream->read()) { - const auto id_column = typeid_cast(block.safeGetByPosition(0).column.get()); + const auto * id_column = typeid_cast(block.safeGetByPosition(0).column.get()); if (!id_column) throw Exception{ name + ": id column has type different from UInt64.", @@ -819,8 +781,7 @@ void CacheDictionary::update( const auto & ids = id_column->getData(); /// cache column pointers - const auto column_ptrs = ext::map(ext::range(0, attributes.size()), [&block] (size_t i) - { + const auto column_ptrs = ext::map(ext::range(0, attributes.size()), [&block](size_t i) { return block.safeGetByPosition(i + 1).column.get(); }); @@ -859,9 +820,6 @@ void CacheDictionary::update( } stream->readSuffix(); - - ProfileEvents::increment(ProfileEvents::DictCacheKeysRequested, requested_ids.size()); - ProfileEvents::increment(ProfileEvents::DictCacheRequestTimeNs, watch.elapsed()); } size_t not_found_num = 0, found_num = 0; @@ -903,10 +861,6 @@ void CacheDictionary::update( /// inform caller that the cell has not been found on_id_not_found(id, cell_idx); } - - ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedMiss, not_found_num); - ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedFound, found_num); - ProfileEvents::increment(ProfileEvents::DictCacheRequests); } @@ -914,32 +868,54 @@ void CacheDictionary::setDefaultAttributeValue(Attribute & attribute, const Key { switch (attribute.type) { - case AttributeUnderlyingType::UInt8: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::UInt16: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::UInt32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::UInt64: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::UInt128: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Int8: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Int16: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Int32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Int64: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Float32: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::Float64: std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); break; - case AttributeUnderlyingType::String: - { - const auto & null_value_ref = std::get(attribute.null_values); - auto & string_ref = std::get>(attribute.arrays)[idx]; - - if (string_ref.data != null_value_ref.data()) - { - if (string_ref.data) - string_arena->free(const_cast(string_ref.data), string_ref.size); + case AttributeUnderlyingType::UInt8: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::UInt16: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::UInt32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::UInt64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::UInt128: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Int8: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Int16: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Int32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Int64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Float32: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::Float64: + std::get>(attribute.arrays)[idx] = std::get(attribute.null_values); + break; + case AttributeUnderlyingType::String: + { + const auto & null_value_ref = std::get(attribute.null_values); + auto & string_ref = std::get>(attribute.arrays)[idx]; - string_ref = StringRef{null_value_ref}; - } + if (string_ref.data != null_value_ref.data()) + { + if (string_ref.data) + string_arena->free(const_cast(string_ref.data), string_ref.size); - break; + string_ref = StringRef{null_value_ref}; } + + break; + } } } @@ -947,39 +923,61 @@ void CacheDictionary::setAttributeValue(Attribute & attribute, const Key idx, co { switch (attribute.type) { - case AttributeUnderlyingType::UInt8: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::UInt16: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::UInt32: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::UInt64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::UInt128: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Int8: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Int16: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Int32: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Int64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Float32: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::Float64: std::get>(attribute.arrays)[idx] = value.get(); break; - case AttributeUnderlyingType::String: - { - const auto & string = value.get(); - auto & string_ref = std::get>(attribute.arrays)[idx]; - const auto & null_value_ref = std::get(attribute.null_values); - - /// free memory unless it points to a null_value - if (string_ref.data && string_ref.data != null_value_ref.data()) - string_arena->free(const_cast(string_ref.data), string_ref.size); + case AttributeUnderlyingType::UInt8: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::UInt16: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::UInt32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::UInt64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::UInt128: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Int8: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Int16: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Int32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Int64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Float32: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::Float64: + std::get>(attribute.arrays)[idx] = value.get(); + break; + case AttributeUnderlyingType::String: + { + const auto & string = value.get(); + auto & string_ref = std::get>(attribute.arrays)[idx]; + const auto & null_value_ref = std::get(attribute.null_values); - const auto size = string.size(); - if (size != 0) - { - auto string_ptr = string_arena->alloc(size + 1); - std::copy(string.data(), string.data() + size + 1, string_ptr); - string_ref = StringRef{string_ptr, size}; - } - else - string_ref = {}; + /// free memory unless it points to a null_value + if (string_ref.data && string_ref.data != null_value_ref.data()) + string_arena->free(const_cast(string_ref.data), string_ref.size); - break; + const auto size = string.size(); + if (size != 0) + { + auto * string_ptr = string_arena->alloc(size + 1); + std::copy(string.data(), string.data() + size + 1, string_ptr); + string_ref = StringRef{string_ptr, size}; } + else + string_ref = {}; + + break; + } } } @@ -989,22 +987,18 @@ CacheDictionary::Attribute & CacheDictionary::getAttribute(const std::string & a if (it == std::end(attribute_index_by_name)) throw Exception{ name + ": no such attribute '" + attribute_name + "'", - ErrorCodes::BAD_ARGUMENTS - }; + ErrorCodes::BAD_ARGUMENTS}; return attributes[it->second]; } bool CacheDictionary::isEmptyCell(const UInt64 idx) const { - return (idx != zero_cell_idx && cells[idx].id == 0) || (cells[idx].data - == ext::safe_bit_cast(CellMetadata::time_point_t())); + return (idx != zero_cell_idx && cells[idx].id == 0) || (cells[idx].data == ext::safe_bit_cast(CellMetadata::time_point_t())); } PaddedPODArray CacheDictionary::getCachedIds() const { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - PaddedPODArray array; for (size_t idx = 0; idx < cells.size(); ++idx) { @@ -1024,4 +1018,4 @@ BlockInputStreamPtr CacheDictionary::getBlockInputStream(const Names & column_na } -} +} // namespace DB diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp index 330ee036136..fb9a94b29a0 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.cpp @@ -12,48 +12,27 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include -#include #include #include -#include -#include -#include -#include #include -#include -#include - - -namespace ProfileEvents -{ - - extern const Event DictCacheKeysRequested; - extern const Event DictCacheKeysRequestedMiss; - extern const Event DictCacheKeysRequestedFound; - extern const Event DictCacheKeysExpired; - extern const Event DictCacheKeysNotFound; - extern const Event DictCacheKeysHit; - extern const Event DictCacheRequestTimeNs; - extern const Event DictCacheLockWriteNs; - extern const Event DictCacheLockReadNs; -} - -namespace CurrentMetrics -{ - extern const Metric DictCacheRequests; -} +#include +#include +#include +#include +#include +#include +#include +#include namespace DB { - namespace ErrorCodes { - extern const int TYPE_MISMATCH; - extern const int BAD_ARGUMENTS; - extern const int UNSUPPORTED_METHOD; -} +extern const int TYPE_MISMATCH; +extern const int BAD_ARGUMENTS; +extern const int UNSUPPORTED_METHOD; +} // namespace ErrorCodes inline UInt64 ComplexKeyCacheDictionary::getCellIdx(const StringRef key) const @@ -64,13 +43,14 @@ inline UInt64 ComplexKeyCacheDictionary::getCellIdx(const StringRef key) const } -ComplexKeyCacheDictionary::ComplexKeyCacheDictionary(const std::string & name, const DictionaryStructure & dict_struct, - DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime, - const size_t size) - : name{name}, dict_struct(dict_struct), source_ptr{std::move(source_ptr)}, dict_lifetime(dict_lifetime), - size{roundUpToPowerOfTwoOrZero(std::max(size, size_t(max_collision_length)))}, - size_overlap_mask{this->size - 1}, - rnd_engine(randomSeed()) +ComplexKeyCacheDictionary::ComplexKeyCacheDictionary(const std::string & name, const DictionaryStructure & dict_struct, DictionarySourcePtr source_ptr, const DictionaryLifetime dict_lifetime, const size_t size) + : name{name} + , dict_struct(dict_struct) + , source_ptr{std::move(source_ptr)} + , dict_lifetime(dict_lifetime) + , size{roundUpToPowerOfTwoOrZero(std::max(size, size_t(max_collision_length)))} + , size_overlap_mask{this->size - 1} + , rnd_engine(randomSeed()) { if (!this->source_ptr->supportsSelectiveLoad()) throw Exception{ @@ -85,7 +65,9 @@ ComplexKeyCacheDictionary::ComplexKeyCacheDictionary(const ComplexKeyCacheDictio {} void ComplexKeyCacheDictionary::getString( - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, + const std::string & attribute_name, + const Columns & key_columns, + const DataTypes & key_types, ColumnString * out) const { dict_struct.validateKeyTypes(key_types); @@ -98,12 +80,15 @@ void ComplexKeyCacheDictionary::getString( const auto null_value = StringRef{std::get(attribute.null_values)}; - getItemsString(attribute, key_columns, out, [&] (const size_t) { return null_value; }); + getItemsString(attribute, key_columns, out, [&](const size_t) { return null_value; }); } void ComplexKeyCacheDictionary::getString( - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, - const ColumnString * const def, ColumnString * const out) const + const std::string & attribute_name, + const Columns & key_columns, + const DataTypes & key_types, + const ColumnString * const def, + ColumnString * const out) const { dict_struct.validateKeyTypes(key_types); @@ -113,12 +98,15 @@ void ComplexKeyCacheDictionary::getString( name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH}; - getItemsString(attribute, key_columns, out, [&] (const size_t row) { return def->getDataAt(row); }); + getItemsString(attribute, key_columns, out, [&](const size_t row) { return def->getDataAt(row); }); } void ComplexKeyCacheDictionary::getString( - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, - const String & def, ColumnString * const out) const + const std::string & attribute_name, + const Columns & key_columns, + const DataTypes & key_types, + const String & def, + ColumnString * const out) const { dict_struct.validateKeyTypes(key_types); @@ -128,7 +116,7 @@ void ComplexKeyCacheDictionary::getString( name + ": type mismatch: attribute " + attribute_name + " has type " + toString(attribute.type), ErrorCodes::TYPE_MISMATCH}; - getItemsString(attribute, key_columns, out, [&] (const size_t) { return StringRef{def}; }); + getItemsString(attribute, key_columns, out, [&](const size_t) { return StringRef{def}; }); } /// returns cell_idx (always valid for replacing), 'cell is valid' flag, 'cell is outdated' flag, @@ -190,8 +178,6 @@ void ComplexKeyCacheDictionary::has(const Columns & key_columns, const DataTypes size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0; { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - const auto now = std::chrono::system_clock::now(); /// fetch up-to-date values, decide which ones require update for (const auto row : ext::range(0, rows_num)) @@ -220,9 +206,6 @@ void ComplexKeyCacheDictionary::has(const Columns & key_columns, const DataTypes } } } - ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); - ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); - ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); query_count.fetch_add(rows_num, std::memory_order_relaxed); hit_count.fetch_add(rows_num - outdated_keys.size(), std::memory_order_release); @@ -231,18 +214,18 @@ void ComplexKeyCacheDictionary::has(const Columns & key_columns, const DataTypes return; std::vector required_rows(outdated_keys.size()); - std::transform(std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), - [] (auto & pair) { return pair.getMapped().front(); }); + std::transform(std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair) { return pair.getMapped().front(); }); /// request new values - update(key_columns, keys_array, required_rows, - [&] (const StringRef key, const auto) - { + update( + key_columns, + keys_array, + required_rows, + [&](const StringRef key, const auto) { for (const auto out_idx : outdated_keys[key]) out[out_idx] = true; }, - [&] (const StringRef key, const auto) - { + [&](const StringRef key, const auto) { for (const auto out_idx : outdated_keys[key]) out[out_idx] = false; }); @@ -297,8 +280,11 @@ void ComplexKeyCacheDictionary::freeKey(const StringRef key) const template StringRef ComplexKeyCacheDictionary::placeKeysInPool( - const size_t row, const Columns & key_columns, StringRefs & keys, - const std::vector & key_attributes, Pool & pool) + const size_t row, + const Columns & key_columns, + StringRefs & keys, + const std::vector & key_attributes, + Pool & pool) { const auto keys_size = key_columns.size(); size_t sum_keys_size{}; @@ -337,25 +323,32 @@ StringRef ComplexKeyCacheDictionary::placeKeysInPool( } } - return { place, sum_keys_size }; + return {place, sum_keys_size}; } /// Explicit instantiations. template StringRef ComplexKeyCacheDictionary::placeKeysInPool( - const size_t row, const Columns & key_columns, StringRefs & keys, - const std::vector & key_attributes, Arena & pool); + const size_t row, + const Columns & key_columns, + StringRefs & keys, + const std::vector & key_attributes, + Arena & pool); template StringRef ComplexKeyCacheDictionary::placeKeysInPool( - const size_t row, const Columns & key_columns, StringRefs & keys, - const std::vector & key_attributes, ArenaWithFreeLists & pool); + const size_t row, + const Columns & key_columns, + StringRefs & keys, + const std::vector & key_attributes, + ArenaWithFreeLists & pool); StringRef ComplexKeyCacheDictionary::placeKeysInFixedSizePool( - const size_t row, const Columns & key_columns) const + const size_t row, + const Columns & key_columns) const { - const auto res = fixed_size_keys_pool->alloc(); - auto place = res; + auto * const res = fixed_size_keys_pool->alloc(); + auto * place = res; for (const auto & key_column : key_columns) { @@ -364,36 +357,33 @@ StringRef ComplexKeyCacheDictionary::placeKeysInFixedSizePool( place += key.size; } - return { res, key_size }; + return {res, key_size}; } StringRef ComplexKeyCacheDictionary::copyIntoArena(StringRef src, Arena & arena) { char * allocated = arena.alloc(src.size); memcpy(allocated, src.data, src.size); - return { allocated, src.size }; + return {allocated, src.size}; } StringRef ComplexKeyCacheDictionary::copyKey(const StringRef key) const { - const auto res = key_size_is_fixed ? fixed_size_keys_pool->alloc() : keys_pool->alloc(key.size); + auto * const res = key_size_is_fixed ? fixed_size_keys_pool->alloc() : keys_pool->alloc(key.size); memcpy(res, key.data, key.size); - return { res, key.size }; + return {res, key.size}; } bool ComplexKeyCacheDictionary::isEmptyCell(const UInt64 idx) const { - return (cells[idx].key == StringRef{} && (idx != zero_cell_idx - || cells[idx].data == ext::safe_bit_cast(CellMetadata::time_point_t()))); + return (cells[idx].key == StringRef{} && (idx != zero_cell_idx || cells[idx].data == ext::safe_bit_cast(CellMetadata::time_point_t()))); } BlockInputStreamPtr ComplexKeyCacheDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const { std::vector keys; { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - for (auto idx : ext::range(0, cells.size())) if (!isEmptyCell(idx) && !cells[idx].isDefault()) @@ -404,4 +394,4 @@ BlockInputStreamPtr ComplexKeyCacheDictionary::getBlockInputStream(const Names & return std::make_shared(shared_from_this(), max_block_size, keys, column_names); } -} +} // namespace DB diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h index feb61261f1d..19fe5214fef 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary.h @@ -14,12 +14,6 @@ #pragma once -#include -#include -#include -#include -#include -#include #include #include #include @@ -29,24 +23,17 @@ #include #include #include + +#include +#include #include #include #include +#include #include - - -namespace ProfileEvents -{ -extern const Event DictCacheKeysRequested; -extern const Event DictCacheKeysRequestedMiss; -extern const Event DictCacheKeysRequestedFound; -extern const Event DictCacheKeysExpired; -extern const Event DictCacheKeysNotFound; -extern const Event DictCacheKeysHit; -extern const Event DictCacheRequestTimeNs; -extern const Event DictCacheLockWriteNs; -extern const Event DictCacheLockReadNs; -} +#include +#include +#include namespace DB { @@ -54,10 +41,10 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase { public: ComplexKeyCacheDictionary(const std::string & name, - const DictionaryStructure & dict_struct, - DictionarySourcePtr source_ptr, - const DictionaryLifetime dict_lifetime, - const size_t size); + const DictionaryStructure & dict_struct, + DictionarySourcePtr source_ptr, + const DictionaryLifetime dict_lifetime, + const size_t size); ComplexKeyCacheDictionary(const ComplexKeyCacheDictionary & other); @@ -144,9 +131,12 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase /// In all functions below, key_columns must be full (non-constant) columns. /// See the requirement in IDataType.h for text-serialization functions. -#define DECLARE(TYPE) \ - void get##TYPE( \ - const std::string & attribute_name, const Columns & key_columns, const DataTypes & key_types, PaddedPODArray & out) const; +#define DECLARE(TYPE) \ + void get##TYPE( \ + const std::string & attribute_name, \ + const Columns & key_columns, \ + const DataTypes & key_types, \ + PaddedPODArray & out) const; DECLARE(UInt8) DECLARE(UInt16) DECLARE(UInt32) @@ -164,10 +154,10 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase #define DECLARE(TYPE) \ void get##TYPE(const std::string & attribute_name, \ - const Columns & key_columns, \ - const DataTypes & key_types, \ - const PaddedPODArray & def, \ - PaddedPODArray & out) const; + const Columns & key_columns, \ + const DataTypes & key_types, \ + const PaddedPODArray & def, \ + PaddedPODArray & out) const; DECLARE(UInt8) DECLARE(UInt16) DECLARE(UInt32) @@ -182,17 +172,17 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase #undef DECLARE void getString(const std::string & attribute_name, - const Columns & key_columns, - const DataTypes & key_types, - const ColumnString * const def, - ColumnString * const out) const; + const Columns & key_columns, + const DataTypes & key_types, + const ColumnString * const def, + ColumnString * const out) const; #define DECLARE(TYPE) \ void get##TYPE(const std::string & attribute_name, \ - const Columns & key_columns, \ - const DataTypes & key_types, \ - const TYPE def, \ - PaddedPODArray & out) const; + const Columns & key_columns, \ + const DataTypes & key_types, \ + const TYPE def, \ + PaddedPODArray & out) const; DECLARE(UInt8) DECLARE(UInt16) DECLARE(UInt32) @@ -207,10 +197,10 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase #undef DECLARE void getString(const std::string & attribute_name, - const Columns & key_columns, - const DataTypes & key_types, - const String & def, - ColumnString * const out) const; + const Columns & key_columns, + const DataTypes & key_types, + const String & def, + ColumnString * const out) const; void has(const Columns & key_columns, const DataTypes & key_types, PaddedPODArray & out) const; @@ -263,17 +253,17 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase AttributeUnderlyingType type; std::tuple null_values; std::tuple, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType, - ContainerPtrType> + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType, + ContainerPtrType> arrays; }; @@ -283,7 +273,10 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase template void getItemsNumber( - Attribute & attribute, const Columns & key_columns, PaddedPODArray & out, DefaultGetter && get_default) const + Attribute & attribute, + const Columns & key_columns, + PaddedPODArray & out, + DefaultGetter && get_default) const { if (false) { @@ -308,7 +301,10 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase template void getItemsNumberImpl( - Attribute & attribute, const Columns & key_columns, PaddedPODArray & out, DefaultGetter && get_default) const + Attribute & attribute, + const Columns & key_columns, + PaddedPODArray & out, + DefaultGetter && get_default) const { /// Mapping: -> { all indices `i` of `key_columns` such that `key_columns[i]` = } MapType> outdated_keys; @@ -322,8 +318,6 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0; { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - const auto now = std::chrono::system_clock::now(); /// fetch up-to-date values, decide which ones require update for (const auto row : ext::range(0, rows_num)) @@ -354,9 +348,6 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase } } } - ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); - ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); - ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); query_count.fetch_add(rows_num, std::memory_order_relaxed); hit_count.fetch_add(rows_num - outdated_keys.size(), std::memory_order_release); @@ -365,19 +356,21 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase std::vector required_rows(outdated_keys.size()); std::transform( - std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair) { return pair.getMapped().front(); }); + std::begin(outdated_keys), + std::end(outdated_keys), + std::begin(required_rows), + [](auto & pair) { return pair.getMapped().front(); }); /// request new values - update(key_columns, + update( + key_columns, keys_array, required_rows, - [&](const StringRef key, const size_t cell_idx) - { + [&](const StringRef key, const size_t cell_idx) { for (const auto row : outdated_keys[key]) out[row] = static_cast(attribute_array[cell_idx]); }, - [&](const StringRef key, const size_t) - { + [&](const StringRef key, const size_t) { for (const auto row : outdated_keys[key]) out[row] = get_default(row); }); @@ -400,8 +393,6 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase /// perform optimistic version, fallback to pessimistic if failed { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - const auto now = std::chrono::system_clock::now(); /// fetch up-to-date values, discard on fail for (const auto row : ext::range(0, rows_num)) @@ -446,8 +437,6 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase size_t total_length = 0; size_t cache_expired = 0, cache_not_found = 0, cache_hit = 0; { - const ProfilingScopedReadRWLock read_lock{rw_lock, ProfileEvents::DictCacheLockReadNs}; - const auto now = std::chrono::system_clock::now(); for (const auto row : ext::range(0, rows_num)) { @@ -477,9 +466,6 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase } } } - ProfileEvents::increment(ProfileEvents::DictCacheKeysExpired, cache_expired); - ProfileEvents::increment(ProfileEvents::DictCacheKeysNotFound, cache_not_found); - ProfileEvents::increment(ProfileEvents::DictCacheKeysHit, cache_hit); query_count.fetch_add(rows_num, std::memory_order_relaxed); hit_count.fetch_add(rows_num - outdated_keys.size(), std::memory_order_release); @@ -488,16 +474,15 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase if (!outdated_keys.empty()) { std::vector required_rows(outdated_keys.size()); - std::transform(std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair) - { + std::transform(std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair) { return pair.getMapped().front(); }); - update(key_columns, + update( + key_columns, keys_array, required_rows, - [&](const StringRef key, const size_t cell_idx) - { + [&](const StringRef key, const size_t cell_idx) { const StringRef attribute_value = attribute_array[cell_idx]; /// We must copy key and value to own memory, because it may be replaced with another @@ -508,8 +493,7 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase map[copied_key] = copied_value; total_length += (attribute_value.size + 1) * outdated_keys[key].size(); }, - [&](const StringRef key, const size_t) - { + [&](const StringRef key, const size_t) { for (const auto row : outdated_keys[key]) total_length += get_default(row).size + 1; }); @@ -521,17 +505,17 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase { const StringRef key = keys_array[row]; const auto it = map.find(key); - const auto string_ref = it != std::end(map) ? it->getMapped(): get_default(row); + const auto string_ref = it != std::end(map) ? it->getMapped() : get_default(row); out->insertData(string_ref.data, string_ref.size); } }; template void update(const Columns & in_key_columns, - const PODArray & in_keys, - const std::vector & in_requested_rows, - PresentKeyHandler && on_cell_updated, - AbsentKeyHandler && on_key_not_found) const + const PODArray & in_keys, + const std::vector & in_requested_rows, + PresentKeyHandler && on_cell_updated, + AbsentKeyHandler && on_key_not_found) const { MapType remaining_keys{in_requested_rows.size()}; for (const auto row : in_requested_rows) @@ -539,7 +523,6 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase std::uniform_int_distribution distribution(dict_lifetime.min_sec, dict_lifetime.max_sec); - const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; { Stopwatch watch; auto stream = source_ptr->loadKeys(in_key_columns, in_requested_rows); @@ -555,10 +538,11 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase { /// cache column pointers const auto key_columns = ext::map( - ext::range(0, keys_size), [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx).column; }); + ext::range(0, keys_size), + [&](const size_t attribute_idx) { return block.safeGetByPosition(attribute_idx).column; }); const auto attribute_columns = ext::map(ext::range(0, attributes_size), - [&](const size_t attribute_idx) { return block.safeGetByPosition(keys_size + attribute_idx).column; }); + [&](const size_t attribute_idx) { return block.safeGetByPosition(keys_size + attribute_idx).column; }); const auto rows_num = block.rows(); @@ -612,9 +596,6 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase } stream->readSuffix(); - - ProfileEvents::increment(ProfileEvents::DictCacheKeysRequested, in_requested_rows.size()); - ProfileEvents::increment(ProfileEvents::DictCacheRequestTimeNs, watch.elapsed()); } size_t found_num = 0; @@ -671,9 +652,6 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase /// inform caller that the cell has not been found on_key_not_found(key, cell_idx); } - - ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedMiss, found_num); - ProfileEvents::increment(ProfileEvents::DictCacheKeysRequestedMiss, not_found_num); }; UInt64 getCellIdx(const StringRef key) const; @@ -690,10 +668,10 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase template static StringRef placeKeysInPool(const size_t row, - const Columns & key_columns, - StringRefs & keys, - const std::vector & key_attributes, - Arena & pool); + const Columns & key_columns, + StringRefs & keys, + const std::vector & key_attributes, + Arena & pool); StringRef placeKeysInFixedSizePool(const size_t row, const Columns & key_columns) const; @@ -752,4 +730,4 @@ class ComplexKeyCacheDictionary final : public IDictionaryBase const std::chrono::time_point creation_time = std::chrono::system_clock::now(); }; -} +} // namespace DB diff --git a/dbms/src/Encryption/WriteBufferFromFileProvider.cpp b/dbms/src/Encryption/WriteBufferFromFileProvider.cpp index 4c99b8e24b1..a17dd85d379 100644 --- a/dbms/src/Encryption/WriteBufferFromFileProvider.cpp +++ b/dbms/src/Encryption/WriteBufferFromFileProvider.cpp @@ -19,7 +19,6 @@ namespace ProfileEvents { extern const Event WriteBufferFromFileDescriptorWrite; -extern const Event WriteBufferFromFileDescriptorWriteFailed; extern const Event WriteBufferFromFileDescriptorWriteBytes; } // namespace ProfileEvents @@ -72,8 +71,7 @@ void WriteBufferFromFileProvider::nextImpl() if ((-1 == res || 0 == res) && errno != EINTR) { - ProfileEvents::increment(ProfileEvents::WriteBufferFromFileDescriptorWriteFailed); - throwFromErrno("Cannot write to file " + getFileName(), ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR); + throwFromErrno("Cannot write to file " + getFileName(), ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR); // NOLINT } if (res > 0) diff --git a/dbms/src/Encryption/createReadBufferFromFileBaseByFileProvider.cpp b/dbms/src/Encryption/createReadBufferFromFileBaseByFileProvider.cpp index b76d58c20cd..1858d474d60 100644 --- a/dbms/src/Encryption/createReadBufferFromFileBaseByFileProvider.cpp +++ b/dbms/src/Encryption/createReadBufferFromFileBaseByFileProvider.cpp @@ -20,10 +20,6 @@ #endif #include #include -namespace ProfileEvents -{ -extern const Event CreatedReadBufferOrdinary; -} namespace DB { @@ -46,7 +42,6 @@ std::unique_ptr createReadBufferFromFileBaseByFileProvid { if ((aio_threshold == 0) || (estimated_size < aio_threshold)) { - ProfileEvents::increment(ProfileEvents::CreatedReadBufferOrdinary); return std::make_unique( file_provider, filename_, @@ -75,7 +70,6 @@ createReadBufferFromFileBaseByFileProvider( size_t checksum_frame_size, int flags_) { - ProfileEvents::increment(ProfileEvents::CreatedReadBufferOrdinary); auto file = file_provider->newRandomAccessFile(filename_, encryption_path_, read_limiter, flags_); auto allocation_size = std::min(estimated_size, checksum_frame_size); switch (checksum_algorithm) diff --git a/dbms/src/Encryption/createWriteBufferFromFileBaseByFileProvider.cpp b/dbms/src/Encryption/createWriteBufferFromFileBaseByFileProvider.cpp index 2f1a2cbaeb8..5e8a6940598 100644 --- a/dbms/src/Encryption/createWriteBufferFromFileBaseByFileProvider.cpp +++ b/dbms/src/Encryption/createWriteBufferFromFileBaseByFileProvider.cpp @@ -20,11 +20,6 @@ #include #include -namespace ProfileEvents -{ -extern const Event CreatedWriteBufferOrdinary; -} - namespace DB { namespace ErrorCodes @@ -49,7 +44,6 @@ createWriteBufferFromFileBaseByFileProvider( { if ((aio_threshold == 0) || (estimated_size < aio_threshold)) { - ProfileEvents::increment(ProfileEvents::CreatedWriteBufferOrdinary); return std::make_unique( file_provider, filename_, @@ -81,7 +75,6 @@ createWriteBufferFromFileBaseByFileProvider( int flags_, mode_t mode) { - ProfileEvents::increment(ProfileEvents::CreatedWriteBufferOrdinary); auto file_ptr = file_provider->newWritableFile(filename_, encryption_path_, true, create_new_encryption_info_, write_limiter_, flags_, mode); switch (checksum_algorithm) diff --git a/dbms/src/Functions/FunctionsGeo.cpp b/dbms/src/Functions/FunctionsGeo.cpp index a6fd2ff522e..02e11b66d77 100644 --- a/dbms/src/Functions/FunctionsGeo.cpp +++ b/dbms/src/Functions/FunctionsGeo.cpp @@ -28,13 +28,6 @@ #include #include - -namespace ProfileEvents -{ -extern const Event PolygonsAddedToPool; -extern const Event PolygonsInPoolAllocatedBytes; -} // namespace ProfileEvents - namespace DB { namespace ErrorCodes @@ -60,9 +53,6 @@ ColumnPtr callPointInPolygonImplWithPool(const IColumn & x, const IColumn & y, P /// To allocate memory. ptr->init(); - ProfileEvents::increment(ProfileEvents::PolygonsAddedToPool); - ProfileEvents::increment(ProfileEvents::PolygonsInPoolAllocatedBytes, ptr->getAllocatedBytes()); - return ptr.release(); }; @@ -121,30 +111,30 @@ class FunctionPointInPolygon : public IFunction throw Exception("Too few arguments", ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION); } - auto getMsgPrefix = [this](size_t i) { + auto get_msg_prefix = [this](size_t i) { return "Argument " + toString(i + 1) + " for function " + getName(); }; for (size_t i = 1; i < arguments.size(); ++i) { - auto * array = checkAndGetDataType(arguments[i].get()); + const auto * array = checkAndGetDataType(arguments[i].get()); if (array == nullptr && i != 1) - throw Exception(getMsgPrefix(i) + " must be array of tuples.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(get_msg_prefix(i) + " must be array of tuples.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - auto * tuple = checkAndGetDataType(array ? array->getNestedType().get() : arguments[i].get()); + const auto * tuple = checkAndGetDataType(array ? array->getNestedType().get() : arguments[i].get()); if (tuple == nullptr) - throw Exception(getMsgPrefix(i) + " must contains tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(get_msg_prefix(i) + " must contains tuple.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); const DataTypes & elements = tuple->getElements(); if (elements.size() != 2) - throw Exception(getMsgPrefix(i) + " must have exactly two elements.", ErrorCodes::BAD_ARGUMENTS); + throw Exception(get_msg_prefix(i) + " must have exactly two elements.", ErrorCodes::BAD_ARGUMENTS); for (auto j : ext::range(0, elements.size())) { if (!elements[j]->isNumber()) { - throw Exception(getMsgPrefix(i) + " must contains numeric tuple at position " + toString(j + 1), + throw Exception(get_msg_prefix(i) + " must contains numeric tuple at position " + toString(j + 1), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } } @@ -156,10 +146,10 @@ class FunctionPointInPolygon : public IFunction void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override { const IColumn * point_col = block.getByPosition(arguments[0]).column.get(); - auto const_tuple_col = checkAndGetColumn(point_col); + const auto * const_tuple_col = checkAndGetColumn(point_col); if (const_tuple_col) point_col = &const_tuple_col->getDataColumn(); - auto tuple_col = checkAndGetColumn(point_col); + const auto * tuple_col = checkAndGetColumn(point_col); if (!tuple_col) { @@ -207,18 +197,18 @@ class FunctionPointInPolygon : public IFunction { Polygon polygon; - auto getMsgPrefix = [this](size_t i) { + auto get_msg_prefix = [this](size_t i) { return "Argument " + toString(i + 1) + " for function " + getName(); }; for (size_t i = 1; i < arguments.size(); ++i) { - auto const_col = checkAndGetColumn(block.getByPosition(arguments[i]).column.get()); - auto array_col = const_col ? checkAndGetColumn(&const_col->getDataColumn()) : nullptr; - auto tuple_col = array_col ? checkAndGetColumn(&array_col->getData()) : nullptr; + const auto * const_col = checkAndGetColumn(block.getByPosition(arguments[i]).column.get()); + const auto * array_col = const_col ? checkAndGetColumn(&const_col->getDataColumn()) : nullptr; + const auto * tuple_col = array_col ? checkAndGetColumn(&array_col->getData()) : nullptr; if (!tuple_col) - throw Exception(getMsgPrefix(i) + " must be constant array of tuples.", ErrorCodes::ILLEGAL_COLUMN); + throw Exception(get_msg_prefix(i) + " must be constant array of tuples.", ErrorCodes::ILLEGAL_COLUMN); const auto & tuple_columns = tuple_col->getColumns(); const auto & column_x = tuple_columns[0]; @@ -232,7 +222,7 @@ class FunctionPointInPolygon : public IFunction auto size = column_x->size(); if (size == 0) - throw Exception(getMsgPrefix(i) + " shouldn't be empty.", ErrorCodes::ILLEGAL_COLUMN); + throw Exception(get_msg_prefix(i) + " shouldn't be empty.", ErrorCodes::ILLEGAL_COLUMN); for (auto j : ext::range(0, size)) { @@ -246,11 +236,11 @@ class FunctionPointInPolygon : public IFunction container.push_back(container.front()); } - auto callImpl = use_object_pool + auto call_impl = use_object_pool ? FunctionPointInPolygonDetail::callPointInPolygonImplWithPool, PointInPolygonImpl> : FunctionPointInPolygonDetail::callPointInPolygonImpl, PointInPolygonImpl>; - return callImpl(x, y, polygon); + return call_impl(x, y, polygon); } }; diff --git a/dbms/src/Functions/Regexps.h b/dbms/src/Functions/Regexps.h index 119169be8b5..3eddd383cfb 100644 --- a/dbms/src/Functions/Regexps.h +++ b/dbms/src/Functions/Regexps.h @@ -18,13 +18,6 @@ #include #include - -namespace ProfileEvents -{ -extern const Event RegexpCreated; -} - - namespace DB { namespace Regexps @@ -54,7 +47,6 @@ inline Pool::Pointer get(const std::string & pattern, int flags) if (no_capture) flags |= OptimizedRegularExpression::RE_NO_CAPTURE; - ProfileEvents::increment(ProfileEvents::RegexpCreated); return new Regexp{createRegexp(pattern, flags)}; }); } diff --git a/dbms/src/IO/BufferWithOwnMemory.h b/dbms/src/IO/BufferWithOwnMemory.h index 272f4fc5c01..babe2541b33 100644 --- a/dbms/src/IO/BufferWithOwnMemory.h +++ b/dbms/src/IO/BufferWithOwnMemory.h @@ -21,14 +21,6 @@ #include - -namespace ProfileEvents -{ -extern const Event IOBufferAllocs; -extern const Event IOBufferAllocBytes; -} // namespace ProfileEvents - - namespace DB { /** Replacement for std::vector to use in buffers. @@ -119,9 +111,6 @@ struct Memory return; } - ProfileEvents::increment(ProfileEvents::IOBufferAllocs); - ProfileEvents::increment(ProfileEvents::IOBufferAllocBytes, m_capacity); - size_t new_capacity = align(m_capacity, alignment); m_data = static_cast(Allocator::alloc(new_capacity, alignment)); m_capacity = new_capacity; diff --git a/dbms/src/IO/ChecksumBuffer.h b/dbms/src/IO/ChecksumBuffer.h index f6d60677a12..b095545ea6e 100644 --- a/dbms/src/IO/ChecksumBuffer.h +++ b/dbms/src/IO/ChecksumBuffer.h @@ -27,7 +27,6 @@ namespace ProfileEvents { // no need to update sync, since write buffers inherit that directly from `WriteBufferFromFileDescriptor` extern const Event WriteBufferFromFileDescriptorWrite; -extern const Event WriteBufferFromFileDescriptorWriteFailed; extern const Event WriteBufferFromFileDescriptorWriteBytes; extern const Event ReadBufferFromFileDescriptorRead; extern const Event ReadBufferFromFileDescriptorReadBytes; @@ -107,7 +106,6 @@ class FramedChecksumWriteBuffer : public WriteBufferFromFileDescriptor } if (unlikely(count == -1)) { - ProfileEvents::increment(ProfileEvents::WriteBufferFromFileDescriptorWriteFailed); if (errno == EINTR) continue; else @@ -386,8 +384,6 @@ class FramedChecksumReadBuffer : public ReadBufferFromFileDescriptor off_t doSeek(off_t offset, int whence) override { - ProfileEvents::increment(ProfileEvents::Seek); - auto & frame = reinterpret_cast &>( *(this->working_buffer.begin() - sizeof(ChecksumFrame))); // align should not fail diff --git a/dbms/src/IO/CompressedReadBufferBase.cpp b/dbms/src/IO/CompressedReadBufferBase.cpp index dd54c1b47a8..58bf47a9298 100644 --- a/dbms/src/IO/CompressedReadBufferBase.cpp +++ b/dbms/src/IO/CompressedReadBufferBase.cpp @@ -28,14 +28,6 @@ #include - -namespace ProfileEvents -{ -extern const Event ReadCompressedBytes; -extern const Event CompressedReadBufferBlocks; -extern const Event CompressedReadBufferBytes; -} // namespace ProfileEvents - namespace DB { namespace ErrorCodes @@ -83,8 +75,6 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_ if (size_compressed > DBMS_MAX_COMPRESSED_SIZE) throw Exception("Too large size_compressed. Most likely corrupted data.", ErrorCodes::TOO_LARGE_SIZE_COMPRESSED); - ProfileEvents::increment(ProfileEvents::ReadCompressedBytes, size_compressed + sizeof(checksum)); - /// Is whole compressed block located in 'compressed_in' buffer? if (compressed_in->offset() >= COMPRESSED_BLOCK_HEADER_SIZE && compressed_in->position() + size_compressed - COMPRESSED_BLOCK_HEADER_SIZE <= compressed_in->buffer().end()) @@ -115,9 +105,6 @@ size_t CompressedReadBufferBase::readCompressedData(size_t & size_ template void CompressedReadBufferBase::decompress(char * to, size_t size_decompressed, size_t size_compressed_without_checksum) { - ProfileEvents::increment(ProfileEvents::CompressedReadBufferBlocks); - ProfileEvents::increment(ProfileEvents::CompressedReadBufferBytes, size_decompressed); - UInt8 method = compressed_buffer[0]; /// See CompressedWriteBuffer.h if (method == static_cast(CompressionMethodByte::LZ4)) diff --git a/dbms/src/IO/ReadBufferFromFileDescriptor.cpp b/dbms/src/IO/ReadBufferFromFileDescriptor.cpp index 90cc6e3ca76..4b3d52f3741 100644 --- a/dbms/src/IO/ReadBufferFromFileDescriptor.cpp +++ b/dbms/src/IO/ReadBufferFromFileDescriptor.cpp @@ -77,7 +77,7 @@ bool ReadBufferFromFileDescriptor::nextImpl() if (profile_callback) { - ProfileInfo info; + ProfileInfo info; // NOLINT info.bytes_requested = internal_buffer.size(); info.bytes_read = res; info.nanoseconds = watch->elapsed(); @@ -120,8 +120,6 @@ off_t ReadBufferFromFileDescriptor::doSeek(off_t offset, int whence) } else { - ProfileEvents::increment(ProfileEvents::Seek); - pos = working_buffer.end(); off_t res = doSeekInFile(new_pos, SEEK_SET); if (-1 == res) @@ -145,7 +143,7 @@ bool ReadBufferFromFileDescriptor::poll(size_t timeout_microseconds) FD_SET(fd, &fds); timeval timeout = {time_t(timeout_microseconds / 1000000), suseconds_t(timeout_microseconds % 1000000)}; - int res = select(1, &fds, 0, 0, &timeout); + int res = select(1, &fds, nullptr, nullptr, &timeout); if (-1 == res) throwFromErrno("Cannot select", ErrorCodes::CANNOT_SELECT); diff --git a/dbms/src/IO/WriteBufferFromFileDescriptor.cpp b/dbms/src/IO/WriteBufferFromFileDescriptor.cpp index c18337497b7..49b6d871870 100644 --- a/dbms/src/IO/WriteBufferFromFileDescriptor.cpp +++ b/dbms/src/IO/WriteBufferFromFileDescriptor.cpp @@ -24,7 +24,6 @@ namespace ProfileEvents { extern const Event FileFSync; extern const Event WriteBufferFromFileDescriptorWrite; -extern const Event WriteBufferFromFileDescriptorWriteFailed; extern const Event WriteBufferFromFileDescriptorWriteBytes; } // namespace ProfileEvents @@ -57,7 +56,6 @@ void WriteBufferFromFileDescriptor::nextImpl() if ((-1 == res || 0 == res) && errno != EINTR) { - ProfileEvents::increment(ProfileEvents::WriteBufferFromFileDescriptorWriteFailed); throwFromErrno("Cannot write to file " + getFileName(), ErrorCodes::CANNOT_WRITE_TO_FILE_DESCRIPTOR); } diff --git a/dbms/src/IO/createReadBufferFromFileBase.cpp b/dbms/src/IO/createReadBufferFromFileBase.cpp index 24c9dfb204c..0d129d03a1a 100644 --- a/dbms/src/IO/createReadBufferFromFileBase.cpp +++ b/dbms/src/IO/createReadBufferFromFileBase.cpp @@ -20,13 +20,6 @@ #endif #include - -namespace ProfileEvents -{ -extern const Event CreatedReadBufferOrdinary; -extern const Event CreatedReadBufferAIO; -} // namespace ProfileEvents - namespace DB { namespace ErrorCodes @@ -46,13 +39,11 @@ createReadBufferFromFileBase( { if ((aio_threshold == 0) || (estimated_size < aio_threshold)) { - ProfileEvents::increment(ProfileEvents::CreatedReadBufferOrdinary); return std::make_unique(filename_, buffer_size_, flags_, existing_memory_, alignment); } else { #if !defined(__APPLE__) && !defined(__FreeBSD__) && !defined(_MSC_VER) - ProfileEvents::increment(ProfileEvents::CreatedReadBufferAIO); return std::make_unique(filename_, buffer_size_, flags_, existing_memory_); #else throw Exception("AIO is not implemented yet on MacOS X", ErrorCodes::NOT_IMPLEMENTED); diff --git a/dbms/src/IO/createWriteBufferFromFileBase.cpp b/dbms/src/IO/createWriteBufferFromFileBase.cpp index 96bf3e65558..0e741eb3e5d 100644 --- a/dbms/src/IO/createWriteBufferFromFileBase.cpp +++ b/dbms/src/IO/createWriteBufferFromFileBase.cpp @@ -19,13 +19,6 @@ #endif #include - -namespace ProfileEvents -{ -extern const Event CreatedWriteBufferOrdinary; -extern const Event CreatedWriteBufferAIO; -} // namespace ProfileEvents - namespace DB { namespace ErrorCodes @@ -45,13 +38,11 @@ WriteBufferFromFileBase * createWriteBufferFromFileBase( { if ((aio_threshold == 0) || (estimated_size < aio_threshold)) { - ProfileEvents::increment(ProfileEvents::CreatedWriteBufferOrdinary); return new WriteBufferFromFile(filename_, buffer_size_, flags_, mode, existing_memory_, alignment); } else { #if !defined(__APPLE__) && !defined(__FreeBSD__) && !defined(_MSC_VER) - ProfileEvents::increment(ProfileEvents::CreatedWriteBufferAIO); return new WriteBufferAIO(filename_, buffer_size_, flags_, mode, existing_memory_); #else throw Exception("AIO is not implemented yet on MacOS X", ErrorCodes::NOT_IMPLEMENTED); diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 6e067b88d81..6a39bc333a8 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -37,19 +37,6 @@ #include #include - -namespace ProfileEvents -{ -extern const Event ExternalAggregationWritePart; -extern const Event ExternalAggregationCompressedBytes; -extern const Event ExternalAggregationUncompressedBytes; -} // namespace ProfileEvents - -namespace CurrentMetrics -{ -extern const Metric QueryThread; -} - namespace DB { namespace ErrorCodes @@ -658,7 +645,6 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, co NativeBlockOutputStream block_out(compressed_buf, ClickHouseRevision::get(), getHeader(false)); LOG_FMT_DEBUG(log, "Writing part of aggregation data into temporary file {}.", path); - ProfileEvents::increment(ProfileEvents::ExternalAggregationWritePart); /// Flush only two-level data and possibly overflow data. @@ -694,9 +680,6 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants, co temporary_files.sum_size_compressed += compressed_bytes; } - ProfileEvents::increment(ProfileEvents::ExternalAggregationCompressedBytes, compressed_bytes); - ProfileEvents::increment(ProfileEvents::ExternalAggregationUncompressedBytes, uncompressed_bytes); - LOG_FMT_TRACE( log, "Written part in {:.3f} sec., {} rows, " @@ -1016,7 +999,7 @@ Block Aggregator::prepareBlockAndFill( aggregate_columns[i] = header.getByName(aggregate_column_name).type->createColumn(); /// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states. - ColumnAggregateFunction & column_aggregate_func = assert_cast(*aggregate_columns[i]); + auto & column_aggregate_func = assert_cast(*aggregate_columns[i]); for (auto & pool : data_variants.aggregates_pools) column_aggregate_func.addArena(pool); @@ -1502,7 +1485,7 @@ class MergingAndConvertingBlockInputStream : public IProfilingBlockInputStream Block getHeader() const override { return aggregator.getHeader(final); } - ~MergingAndConvertingBlockInputStream() + ~MergingAndConvertingBlockInputStream() override { LOG_FMT_TRACE(&Poco::Logger::get(__PRETTY_FUNCTION__), "Waiting for threads to finish"); @@ -1636,8 +1619,6 @@ class MergingAndConvertingBlockInputStream : public IProfilingBlockInputStream void thread(Int32 bucket_num) { - CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; - try { /// TODO: add no_more_keys support maybe diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index a0345daaa75..2dbd495d2c4 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -76,10 +76,10 @@ namespace ProfileEvents extern const Event ContextLock; } +#include + namespace CurrentMetrics { -extern const Metric ContextLockWait; -extern const Metric MemoryTrackingForMerges; extern const Metric GlobalStorageRunMode; } // namespace CurrentMetrics @@ -307,8 +307,6 @@ Context::~Context() std::unique_lock Context::getLock() const { - ProfileEvents::increment(ProfileEvents::ContextLock); - CurrentMetrics::Increment increment{CurrentMetrics::ContextLockWait}; return std::unique_lock(shared->mutex); } diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 8e75a64427c..0ab8519e4d0 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -27,12 +27,6 @@ #include #include - -namespace ProfileEvents -{ -extern const Event FunctionExecute; -} - namespace DB { namespace ErrorCodes @@ -339,7 +333,6 @@ void ExpressionAction::execute(Block & block) const size_t num_columns_without_result = block.columns(); block.insert({nullptr, result_type, result_name}); - ProfileEvents::increment(ProfileEvents::FunctionExecute); function->execute(block, arguments, num_columns_without_result); break; diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index aa64cf8ca94..782a254925a 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -30,12 +30,6 @@ #include #include - -namespace ProfileEvents -{ -extern const Event InsertQuery; -} - namespace DB { namespace ErrorCodes @@ -54,7 +48,6 @@ InterpreterInsertQuery::InterpreterInsertQuery( , context(context_) , allow_materialized(allow_materialized_) { - ProfileEvents::increment(ProfileEvents::InsertQuery); } @@ -62,7 +55,7 @@ StoragePtr InterpreterInsertQuery::getTable(const ASTInsertQuery & query) { if (query.table_function) { - auto table_function = typeid_cast(query.table_function.get()); + const auto * table_function = typeid_cast(query.table_function.get()); const auto & factory = TableFunctionFactory::instance(); return factory.get(table_function->name, context)->execute(query.table_function, context); } @@ -71,7 +64,7 @@ StoragePtr InterpreterInsertQuery::getTable(const ASTInsertQuery & query) return context.getTable(query.database, query.table); } -Block InterpreterInsertQuery::getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table) +Block InterpreterInsertQuery::getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table) // NOLINT { Block table_sample_non_materialized; if (query.is_import) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 51b55f65bd4..01e8625f943 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -131,8 +131,6 @@ InterpreterSelectQuery::~InterpreterSelectQuery() = default; void InterpreterSelectQuery::init(const Names & required_result_column_names) { - ProfileEvents::increment(ProfileEvents::SelectQuery); - if (!context.hasQueryContext()) context.setQueryContext(context); diff --git a/dbms/src/Interpreters/ProcessList.h b/dbms/src/Interpreters/ProcessList.h index fdc009237aa..5ed586c263d 100644 --- a/dbms/src/Interpreters/ProcessList.h +++ b/dbms/src/Interpreters/ProcessList.h @@ -31,12 +31,6 @@ #include #include - -namespace CurrentMetrics -{ -extern const Metric Query; -} - namespace DB { class IStorage; @@ -90,8 +84,6 @@ class ProcessListElement QueryPriorities::Handle priority_handle; - CurrentMetrics::Increment num_queries{CurrentMetrics::Query}; - std::atomic is_killed{false}; /// Be careful using it. For example, queries field could be modified concurrently. diff --git a/dbms/src/Interpreters/QueryPriorities.h b/dbms/src/Interpreters/QueryPriorities.h index 5f34ae616c7..ca01e4f0a6c 100644 --- a/dbms/src/Interpreters/QueryPriorities.h +++ b/dbms/src/Interpreters/QueryPriorities.h @@ -23,13 +23,6 @@ #include #include - -namespace CurrentMetrics -{ -extern const Metric QueryPreempted; -} - - namespace DB { /** Implements query priorities in very primitive way. @@ -95,7 +88,6 @@ class QueryPriorities if (!found) return true; - CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryPreempted}; if (std::cv_status::timeout == condvar.wait_for(lock, cur_timeout)) return false; else diff --git a/dbms/src/Server/HTTPHandler.h b/dbms/src/Server/HTTPHandler.h index bd06d56bd4e..74b5cc4b4c7 100644 --- a/dbms/src/Server/HTTPHandler.h +++ b/dbms/src/Server/HTTPHandler.h @@ -14,24 +14,20 @@ #pragma once -#include "IServer.h" - -#include - #include #include +#include +#include "IServer.h" -namespace CurrentMetrics + +namespace Poco { - extern const Metric HTTPConnection; +class Logger; } -namespace Poco { class Logger; } - namespace DB { - class WriteBufferFromHTTPServerResponse; @@ -69,11 +65,9 @@ class HTTPHandler : public Poco::Net::HTTPRequestHandler IServer & server; Poco::Logger * log; - /// It is the name of the server that will be sent in an http-header X-ClickHouse-Server-Display-Name. + /// It is the name of the server that will be sent in an http-header X-ClickHouse-Server-Display-Name. String server_display_name; - CurrentMetrics::Increment metric_increment{CurrentMetrics::HTTPConnection}; - /// Also initializes 'used_output'. void processQuery( Poco::Net::HTTPServerRequest & request, @@ -91,4 +85,4 @@ class HTTPHandler : public Poco::Net::HTTPRequestHandler void pushDelayedResults(Output & used_output); }; -} +} // namespace DB diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 95c1d5d3f2a..186ab0889d8 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -175,11 +175,6 @@ namespace } } // namespace -namespace CurrentMetrics -{ -extern const Metric Revision; -} - namespace DB { namespace ErrorCodes @@ -1064,8 +1059,6 @@ int Server::main(const std::vector & /*args*/) LOG_FMT_INFO(log, "TiFlashRaftProxyHelper is null, failed to get server info"); } - CurrentMetrics::set(CurrentMetrics::Revision, ClickHouseRevision::get()); - // print necessary grpc log. grpc_log = &Poco::Logger::get("grpc"); gpr_set_log_verbosity(GPR_LOG_SEVERITY_DEBUG); diff --git a/dbms/src/Server/TCPHandler.h b/dbms/src/Server/TCPHandler.h index ed0af52dc98..2fde0b11d9b 100644 --- a/dbms/src/Server/TCPHandler.h +++ b/dbms/src/Server/TCPHandler.h @@ -29,11 +29,6 @@ #include "IServer.h" -namespace CurrentMetrics -{ -extern const Metric TCPConnection; -} - namespace Poco { class Logger; @@ -131,8 +126,6 @@ class TCPHandler : public Poco::Net::TCPServerConnection /// At the moment, only one ongoing query in the connection is supported at a time. QueryState state; - CurrentMetrics::Increment metric_increment{CurrentMetrics::TCPConnection}; - /// It is the name of the server that will be sent to the client. String server_display_name; diff --git a/dbms/src/Storages/BackgroundProcessingPool.cpp b/dbms/src/Storages/BackgroundProcessingPool.cpp index 9fb4271ea38..15740fa2875 100644 --- a/dbms/src/Storages/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/BackgroundProcessingPool.cpp @@ -42,7 +42,6 @@ inline static pid_t getTid() namespace CurrentMetrics { -extern const Metric BackgroundPoolTask; extern const Metric MemoryTrackingInBackgroundProcessingPool; } // namespace CurrentMetrics @@ -218,8 +217,6 @@ void BackgroundProcessingPool::threadFunction() continue; { - CurrentMetrics::Increment metric_increment{CurrentMetrics::BackgroundPoolTask}; - bool done_work = false; if (!task->multi) { diff --git a/dbms/src/Storages/MarkCache.h b/dbms/src/Storages/MarkCache.h index 5816b0c1bba..728f830e0d0 100644 --- a/dbms/src/Storages/MarkCache.h +++ b/dbms/src/Storages/MarkCache.h @@ -14,24 +14,23 @@ #pragma once -#include - #include #include #include -#include #include +#include + +#include namespace ProfileEvents { - extern const Event MarkCacheHits; - extern const Event MarkCacheMisses; -} +extern const Event MarkCacheHits; +extern const Event MarkCacheMisses; +} // namespace ProfileEvents namespace DB { - /// Estimate of number of bytes in cache for marks. struct MarksWeightFunction { @@ -53,7 +52,8 @@ class MarkCache : public LRUCache MappedPtr getOrSet(const Key & key, LoadFunc && load) @@ -70,4 +70,4 @@ class MarkCache : public LRUCache; -} +} // namespace DB diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index 0dc05674696..1d7c0ace57f 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -34,24 +34,6 @@ #include - -namespace ProfileEvents -{ -extern const Event StorageBufferFlush; -extern const Event StorageBufferErrorOnFlush; -extern const Event StorageBufferPassedAllMinThresholds; -extern const Event StorageBufferPassedTimeMaxThreshold; -extern const Event StorageBufferPassedRowsMaxThreshold; -extern const Event StorageBufferPassedBytesMaxThreshold; -} // namespace ProfileEvents - -namespace CurrentMetrics -{ -extern const Metric StorageBufferRows; -extern const Metric StorageBufferBytes; -} // namespace CurrentMetrics - - namespace DB { namespace ErrorCodes @@ -170,10 +152,6 @@ static void appendBlock(const Block & from, Block & to) to.checkNumberOfRows(); size_t rows = from.rows(); - size_t bytes = from.bytes(); - - CurrentMetrics::add(CurrentMetrics::StorageBufferRows, rows); - CurrentMetrics::add(CurrentMetrics::StorageBufferBytes, bytes); size_t old_rows = to.rows(); @@ -430,25 +408,21 @@ bool StorageBuffer::checkThresholdsImpl(size_t rows, size_t bytes, time_t time_p { if (time_passed > min_thresholds.time && rows > min_thresholds.rows && bytes > min_thresholds.bytes) { - ProfileEvents::increment(ProfileEvents::StorageBufferPassedAllMinThresholds); return true; } if (time_passed > max_thresholds.time) { - ProfileEvents::increment(ProfileEvents::StorageBufferPassedTimeMaxThreshold); return true; } if (rows > max_thresholds.rows) { - ProfileEvents::increment(ProfileEvents::StorageBufferPassedRowsMaxThreshold); return true; } if (bytes > max_thresholds.bytes) { - ProfileEvents::increment(ProfileEvents::StorageBufferPassedBytesMaxThreshold); return true; } @@ -495,11 +469,6 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds) buffer.data.swap(block_to_write); buffer.first_write_time = 0; - CurrentMetrics::sub(CurrentMetrics::StorageBufferRows, block_to_write.rows()); - CurrentMetrics::sub(CurrentMetrics::StorageBufferBytes, block_to_write.bytes()); - - ProfileEvents::increment(ProfileEvents::StorageBufferFlush); - LOG_FMT_TRACE(log, "Flushing buffer with {} rows, {} bytes, age {} seconds.", rows, bytes, time_passed); if (no_destination) @@ -517,13 +486,7 @@ void StorageBuffer::flushBuffer(Buffer & buffer, bool check_thresholds) } catch (...) { - ProfileEvents::increment(ProfileEvents::StorageBufferErrorOnFlush); - /// Return the block to its place in the buffer. - - CurrentMetrics::add(CurrentMetrics::StorageBufferRows, block_to_write.rows()); - CurrentMetrics::add(CurrentMetrics::StorageBufferBytes, block_to_write.bytes()); - buffer.data.swap(block_to_write); if (!buffer.first_write_time) diff --git a/dbms/src/TableFunctions/ITableFunction.cpp b/dbms/src/TableFunctions/ITableFunction.cpp index ca05075cac0..d262a5637f7 100644 --- a/dbms/src/TableFunctions/ITableFunction.cpp +++ b/dbms/src/TableFunctions/ITableFunction.cpp @@ -15,17 +15,10 @@ #include #include - -namespace ProfileEvents -{ -extern const Event TableFunctionExecute; -} - namespace DB { StoragePtr ITableFunction::execute(const ASTPtr & ast_function, const Context & context) const { - ProfileEvents::increment(ProfileEvents::TableFunctionExecute); return executeImpl(ast_function, context); } From ebb27d1eb5537728fee499b8f1c415aee1658472 Mon Sep 17 00:00:00 2001 From: hehechen Date: Mon, 20 Jun 2022 20:24:37 +0800 Subject: [PATCH 049/104] fix pageworklaod (#5165) close pingcap/tiflash#5164 --- .../src/Storages/Page/workload/CMakeLists.txt | 3 +- ...moryCostInGC.cpp => HeavyMemoryCostInGC.h} | 2 - .../workload/{HeavyRead.cpp => HeavyRead.h} | 2 - ...SkewWriteRead.cpp => HeavySkewWriteRead.h} | 2 - .../workload/{HeavyWrite.cpp => HeavyWrite.h} | 2 - ...alidBigFileGC.cpp => HighValidBigFileGC.h} | 2 - ...tsLongTime.cpp => HoldSnapshotsLongTime.h} | 2 - dbms/src/Storages/Page/workload/MainEntry.cpp | 48 +++++++------------ .../Page/workload/{Normal.cpp => Normal.h} | 2 - dbms/src/Storages/Page/workload/PSWorkload.h | 20 ++++---- ...city.cpp => PageStorageInMemoryCapacity.h} | 2 - ...usandsOfOffset.cpp => ThousandsOfOffset.h} | 2 - 12 files changed, 29 insertions(+), 60 deletions(-) rename dbms/src/Storages/Page/workload/{HeavyMemoryCostInGC.cpp => HeavyMemoryCostInGC.h} (98%) rename dbms/src/Storages/Page/workload/{HeavyRead.cpp => HeavyRead.h} (98%) rename dbms/src/Storages/Page/workload/{HeavySkewWriteRead.cpp => HeavySkewWriteRead.h} (98%) rename dbms/src/Storages/Page/workload/{HeavyWrite.cpp => HeavyWrite.h} (98%) rename dbms/src/Storages/Page/workload/{HighValidBigFileGC.cpp => HighValidBigFileGC.h} (98%) rename dbms/src/Storages/Page/workload/{HoldSnapshotsLongTime.cpp => HoldSnapshotsLongTime.h} (98%) rename dbms/src/Storages/Page/workload/{Normal.cpp => Normal.h} (98%) rename dbms/src/Storages/Page/workload/{PageStorageInMemoryCapacity.cpp => PageStorageInMemoryCapacity.h} (99%) rename dbms/src/Storages/Page/workload/{ThousandsOfOffset.cpp => ThousandsOfOffset.h} (99%) diff --git a/dbms/src/Storages/Page/workload/CMakeLists.txt b/dbms/src/Storages/Page/workload/CMakeLists.txt index 5c8ecb34d97..adf94c75f11 100644 --- a/dbms/src/Storages/Page/workload/CMakeLists.txt +++ b/dbms/src/Storages/Page/workload/CMakeLists.txt @@ -14,8 +14,7 @@ include_directories (${CMAKE_CURRENT_BINARY_DIR}) -set (page-workload-src HeavyMemoryCostInGC.cpp HeavyRead.cpp HeavySkewWriteRead.cpp HeavyWrite.cpp HighValidBigFileGC.cpp HoldSnapshotsLongTime.cpp Normal.cpp - PageStorageInMemoryCapacity.cpp ThousandsOfOffset.cpp MainEntry.cpp Normal.cpp PageStorageInMemoryCapacity.cpp PSBackground.cpp PSRunnable.cpp PSStressEnv.cpp PSWorkload.cpp) +set (page-workload-src MainEntry.cpp PSBackground.cpp PSRunnable.cpp PSStressEnv.cpp PSWorkload.cpp) add_library (page-workload-lib ${page-workload-src}) target_link_libraries (page-workload-lib dbms clickhouse_functions clickhouse-server-lib) diff --git a/dbms/src/Storages/Page/workload/HeavyMemoryCostInGC.cpp b/dbms/src/Storages/Page/workload/HeavyMemoryCostInGC.h similarity index 98% rename from dbms/src/Storages/Page/workload/HeavyMemoryCostInGC.cpp rename to dbms/src/Storages/Page/workload/HeavyMemoryCostInGC.h index 7e745e29fc2..3daaf10ffb3 100644 --- a/dbms/src/Storages/Page/workload/HeavyMemoryCostInGC.cpp +++ b/dbms/src/Storages/Page/workload/HeavyMemoryCostInGC.h @@ -81,6 +81,4 @@ class HeavyMemoryCostInGC fmt::format("Memory Peak is {} , it should not bigger than {} ", metrics_dumper->getMemoryPeak(), 5 * 1024 * 1024)); } }; - -REGISTER_WORKLOAD(HeavyMemoryCostInGC) } // namespace DB::PS::tests diff --git a/dbms/src/Storages/Page/workload/HeavyRead.cpp b/dbms/src/Storages/Page/workload/HeavyRead.h similarity index 98% rename from dbms/src/Storages/Page/workload/HeavyRead.cpp rename to dbms/src/Storages/Page/workload/HeavyRead.h index a67c435e84c..80023f95988 100644 --- a/dbms/src/Storages/Page/workload/HeavyRead.cpp +++ b/dbms/src/Storages/Page/workload/HeavyRead.h @@ -70,6 +70,4 @@ class HeavyRead : public StressWorkload } } }; - -REGISTER_WORKLOAD(HeavyRead) } // namespace DB::PS::tests \ No newline at end of file diff --git a/dbms/src/Storages/Page/workload/HeavySkewWriteRead.cpp b/dbms/src/Storages/Page/workload/HeavySkewWriteRead.h similarity index 98% rename from dbms/src/Storages/Page/workload/HeavySkewWriteRead.cpp rename to dbms/src/Storages/Page/workload/HeavySkewWriteRead.h index 805bf105358..0e75bc0d3e5 100644 --- a/dbms/src/Storages/Page/workload/HeavySkewWriteRead.cpp +++ b/dbms/src/Storages/Page/workload/HeavySkewWriteRead.h @@ -85,6 +85,4 @@ class HeavySkewWriteRead : public StressWorkload return true; } }; - -REGISTER_WORKLOAD(HeavySkewWriteRead) } // namespace DB::PS::tests \ No newline at end of file diff --git a/dbms/src/Storages/Page/workload/HeavyWrite.cpp b/dbms/src/Storages/Page/workload/HeavyWrite.h similarity index 98% rename from dbms/src/Storages/Page/workload/HeavyWrite.cpp rename to dbms/src/Storages/Page/workload/HeavyWrite.h index 8dfd7f810f7..54b7585ee20 100644 --- a/dbms/src/Storages/Page/workload/HeavyWrite.cpp +++ b/dbms/src/Storages/Page/workload/HeavyWrite.h @@ -72,6 +72,4 @@ class HeavyWrite : public StressWorkload return true; } }; - -REGISTER_WORKLOAD(HeavyWrite) } // namespace DB::PS::tests \ No newline at end of file diff --git a/dbms/src/Storages/Page/workload/HighValidBigFileGC.cpp b/dbms/src/Storages/Page/workload/HighValidBigFileGC.h similarity index 98% rename from dbms/src/Storages/Page/workload/HighValidBigFileGC.cpp rename to dbms/src/Storages/Page/workload/HighValidBigFileGC.h index a9af6aebb76..cc3b5b45135 100644 --- a/dbms/src/Storages/Page/workload/HighValidBigFileGC.cpp +++ b/dbms/src/Storages/Page/workload/HighValidBigFileGC.h @@ -129,6 +129,4 @@ class HighValidBigFileGCWorkload private: UInt64 gc_time_ms = 0; }; - -REGISTER_WORKLOAD(HighValidBigFileGCWorkload) } // namespace DB::PS::tests \ No newline at end of file diff --git a/dbms/src/Storages/Page/workload/HoldSnapshotsLongTime.cpp b/dbms/src/Storages/Page/workload/HoldSnapshotsLongTime.h similarity index 98% rename from dbms/src/Storages/Page/workload/HoldSnapshotsLongTime.cpp rename to dbms/src/Storages/Page/workload/HoldSnapshotsLongTime.h index f02fbf65bcd..071a104010c 100644 --- a/dbms/src/Storages/Page/workload/HoldSnapshotsLongTime.cpp +++ b/dbms/src/Storages/Page/workload/HoldSnapshotsLongTime.h @@ -94,6 +94,4 @@ class HoldSnapshotsLongTime : public StressWorkload return true; } }; - -REGISTER_WORKLOAD(HoldSnapshotsLongTime) } // namespace DB::PS::tests \ No newline at end of file diff --git a/dbms/src/Storages/Page/workload/MainEntry.cpp b/dbms/src/Storages/Page/workload/MainEntry.cpp index ac82e1ea4bc..18e42106c90 100644 --- a/dbms/src/Storages/Page/workload/MainEntry.cpp +++ b/dbms/src/Storages/Page/workload/MainEntry.cpp @@ -11,44 +11,32 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. +#include +#include +#include +#include +#include +#include +#include #include #include +#include +#include using namespace DB::PS::tests; int StressWorkload::mainEntry(int argc, char ** argv) { { - // maybe due to sequence of linking, REGISTER_WORKLOAD is not visible to main function in dbms/src/Server/main.cpp - // cause that REGISTER_WORKLOAD will not be triggered before mainEntry - // we do this to trigger REGISTER_WORKLOAD explicitly. - void _work_load_register_named_HeavyMemoryCostInGC(); - void (*f)() = _work_load_register_named_HeavyMemoryCostInGC; - (void)f; - void _work_load_register_named_HeavyRead(); - f = _work_load_register_named_HeavyRead; - (void)f; - void _work_load_register_named_HeavySkewWriteRead(); - f = _work_load_register_named_HeavySkewWriteRead; - (void)f; - void _work_load_register_named_HeavyWrite(); - f = _work_load_register_named_HeavyWrite; - (void)f; - void _work_load_register_named_HighValidBigFileGCWorkload(); - f = _work_load_register_named_HighValidBigFileGCWorkload; - (void)f; - void _work_load_register_named_HoldSnapshotsLongTime(); - f = _work_load_register_named_HoldSnapshotsLongTime; - (void)f; - void _work_load_register_named_PageStorageInMemoryCapacity(); - f = _work_load_register_named_PageStorageInMemoryCapacity; - (void)f; - void _work_load_register_named_NormalWorkload(); - f = _work_load_register_named_NormalWorkload; - (void)f; - void _work_load_register_named_ThousandsOfOffset(); - f = _work_load_register_named_ThousandsOfOffset; - (void)f; + work_load_register(); + work_load_register(); + work_load_register(); + work_load_register(); + work_load_register(); + work_load_register(); + work_load_register(); + work_load_register(); + work_load_register(); } try { diff --git a/dbms/src/Storages/Page/workload/Normal.cpp b/dbms/src/Storages/Page/workload/Normal.h similarity index 98% rename from dbms/src/Storages/Page/workload/Normal.cpp rename to dbms/src/Storages/Page/workload/Normal.h index 57229395809..164f17b9d61 100644 --- a/dbms/src/Storages/Page/workload/Normal.cpp +++ b/dbms/src/Storages/Page/workload/Normal.h @@ -77,6 +77,4 @@ class NormalWorkload stop_watch.stop(); } }; - -REGISTER_WORKLOAD(NormalWorkload) } // namespace DB::PS::tests diff --git a/dbms/src/Storages/Page/workload/PSWorkload.h b/dbms/src/Storages/Page/workload/PSWorkload.h index eaaaf4eba5b..26a9c24d6da 100644 --- a/dbms/src/Storages/Page/workload/PSWorkload.h +++ b/dbms/src/Storages/Page/workload/PSWorkload.h @@ -193,15 +193,15 @@ class StressWorkloadManger StressEnv options; }; -#define REGISTER_WORKLOAD(WORKLOAD) \ - void __attribute__((constructor)) _work_load_register_named_##WORKLOAD(void) \ - { \ - StressWorkloadManger::getInstance().reg( \ - WORKLOAD::nameFunc(), \ - WORKLOAD::maskFunc(), \ - [](const StressEnv & opts) -> std::shared_ptr { \ - return std::make_shared(opts); \ - }); \ - } +template +void work_load_register() +{ + StressWorkloadManger::getInstance().reg( + Workload::nameFunc(), + Workload::maskFunc(), + [](const StressEnv & opts) -> std::shared_ptr { + return std::make_shared(opts); + }); +} } // namespace DB::PS::tests diff --git a/dbms/src/Storages/Page/workload/PageStorageInMemoryCapacity.cpp b/dbms/src/Storages/Page/workload/PageStorageInMemoryCapacity.h similarity index 99% rename from dbms/src/Storages/Page/workload/PageStorageInMemoryCapacity.cpp rename to dbms/src/Storages/Page/workload/PageStorageInMemoryCapacity.h index 6ab321d1a10..337c732e6f7 100644 --- a/dbms/src/Storages/Page/workload/PageStorageInMemoryCapacity.cpp +++ b/dbms/src/Storages/Page/workload/PageStorageInMemoryCapacity.h @@ -174,6 +174,4 @@ class PageStorageInMemoryCapacity : public StressWorkload std::round(resident_used) ? (total_mem / ((double)resident_used / page_writen)) : 0)); } }; - -REGISTER_WORKLOAD(PageStorageInMemoryCapacity) } // namespace DB::PS::tests \ No newline at end of file diff --git a/dbms/src/Storages/Page/workload/ThousandsOfOffset.cpp b/dbms/src/Storages/Page/workload/ThousandsOfOffset.h similarity index 99% rename from dbms/src/Storages/Page/workload/ThousandsOfOffset.cpp rename to dbms/src/Storages/Page/workload/ThousandsOfOffset.h index 5a02ef48d68..0232ea235f1 100644 --- a/dbms/src/Storages/Page/workload/ThousandsOfOffset.cpp +++ b/dbms/src/Storages/Page/workload/ThousandsOfOffset.h @@ -169,6 +169,4 @@ class ThousandsOfOffset : public StressWorkload return true; } }; - -REGISTER_WORKLOAD(ThousandsOfOffset) } // namespace DB::PS::tests \ No newline at end of file From f3f37ae8fe551cd00f558a7728716adad7b2f8fe Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 22 Jun 2022 10:16:36 +0800 Subject: [PATCH 050/104] Enhancement: Add how to run integration tests and microbenchmark tests in README (#5182) close pingcap/tiflash#5172, ref pingcap/tiflash#5178 --- README.md | 25 ++++++++++++++++++++++++- 1 file changed, 24 insertions(+), 1 deletion(-) diff --git a/README.md b/README.md index 02af727105b..aa64e39d5ba 100644 --- a/README.md +++ b/README.md @@ -242,7 +242,30 @@ LSAN_OPTIONS=suppressions=$WORKSPACE/tiflash/test/sanitize/asan.suppression ## Run Integration Tests -TBD. +1. Build your own tiflash binary in $BUILD with `-DCMAKE_BUILD_TYPE=DEBUG`. +``` +cd $BUILD +cmake $WORKSPACE/tiflash -GNinja -DCMAKE_BUILD_TYPE=DEBUG +ninja tiflash +``` +2. Run tidb cluster locally using tiup playgroud or other tools. +``` +tiup playground nightly --tiflash.binpath $BUILD/dbms/src/Server/tiflash +``` +3. Check $WORKSPACE/tests/_env.sh to make the port and build dir right. +4. Run your integration tests using commands like "./run-test.sh fullstack-test2/ddl" under $WORKSPACE dir + +## Run MicroBenchmark Tests + +To run micro benchmark tests, you need to build with -DCMAKE_BUILD_TYPE=RELEASE -DENABLE_TESTS=ON: + +```shell +cd $BUILD +cmake $WORKSPACE/tiflash -GNinja -DCMAKE_BUILD_TYPE=DEBUG -DENABLE_TESTS=ON +ninja bench_dbms +``` + +And the microbenchmark-test executables are at `$BUILD/dbms/bench_dbms`, you can run it with `./bench_dbms` or `./bench_dbms --benchmark_filter=xxx` . More usage please check with `./bench_dbms --help`. ## Generate LLVM Coverage Report From 649462ab275ff4a7a36b9998f4aa909d31442b92 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 22 Jun 2022 10:44:37 +0800 Subject: [PATCH 051/104] Enhancement: add a integrated test on DDL module (#5130) ref pingcap/tiflash#5129 --- dbms/src/Common/FailPoint.cpp | 46 +- .../Interpreters/InterpreterSelectQuery.cpp | 16 + .../Storages/Transaction/PartitionStreams.cpp | 15 + .../ddl/multi_alter_with_write.test | 880 ++++++++++++++++++ 4 files changed, 936 insertions(+), 21 deletions(-) create mode 100644 tests/fullstack-test2/ddl/multi_alter_with_write.test diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index c6c3caa44ad..10d0a558a50 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -85,33 +85,37 @@ std::unordered_map> FailPointHelper::f M(force_remote_read_for_batch_cop) \ M(force_context_path) \ M(force_slow_page_storage_snapshot_release) \ - M(force_change_all_blobs_to_read_only) - -#define APPLY_FOR_FAILPOINTS_ONCE_WITH_CHANNEL(M) \ - M(pause_with_alter_locks_acquired) \ - M(hang_in_execution) \ - M(pause_before_dt_background_delta_merge) \ - M(pause_until_dt_background_delta_merge) \ - M(pause_before_apply_raft_cmd) \ - M(pause_before_apply_raft_snapshot) \ - M(pause_until_apply_raft_snapshot) \ + M(force_change_all_blobs_to_read_only) \ + M(unblock_query_init_after_write) + + +#define APPLY_FOR_PAUSEABLE_FAILPOINTS_ONCE(M) \ + M(pause_with_alter_locks_acquired) \ + M(hang_in_execution) \ + M(pause_before_dt_background_delta_merge) \ + M(pause_until_dt_background_delta_merge) \ + M(pause_before_apply_raft_cmd) \ + M(pause_before_apply_raft_snapshot) \ + M(pause_until_apply_raft_snapshot) \ M(pause_after_copr_streams_acquired_once) -#define APPLY_FOR_FAILPOINTS_WITH_CHANNEL(M) \ - M(pause_when_reading_from_dt_stream) \ - M(pause_when_writing_to_dt_store) \ - M(pause_when_ingesting_to_dt_store) \ - M(pause_when_altering_dt_store) \ - M(pause_after_copr_streams_acquired) \ - M(pause_before_server_merge_one_delta) +#define APPLY_FOR_PAUSEABLE_FAILPOINTS(M) \ + M(pause_when_reading_from_dt_stream) \ + M(pause_when_writing_to_dt_store) \ + M(pause_when_ingesting_to_dt_store) \ + M(pause_when_altering_dt_store) \ + M(pause_after_copr_streams_acquired) \ + M(pause_before_server_merge_one_delta) \ + M(pause_query_init) + namespace FailPoints { #define M(NAME) extern const char(NAME)[] = #NAME ""; APPLY_FOR_FAILPOINTS_ONCE(M) APPLY_FOR_FAILPOINTS(M) -APPLY_FOR_FAILPOINTS_ONCE_WITH_CHANNEL(M) -APPLY_FOR_FAILPOINTS_WITH_CHANNEL(M) +APPLY_FOR_PAUSEABLE_FAILPOINTS_ONCE(M) +APPLY_FOR_PAUSEABLE_FAILPOINTS(M) #undef M } // namespace FailPoints @@ -167,11 +171,11 @@ void FailPointHelper::enableFailPoint(const String & fail_point_name) } #define M(NAME) SUB_M(NAME, FIU_ONETIME) - APPLY_FOR_FAILPOINTS_ONCE_WITH_CHANNEL(M) + APPLY_FOR_PAUSEABLE_FAILPOINTS_ONCE(M) #undef M #define M(NAME) SUB_M(NAME, 0) - APPLY_FOR_FAILPOINTS_WITH_CHANNEL(M) + APPLY_FOR_PAUSEABLE_FAILPOINTS(M) #undef M #undef SUB_M diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 01e8625f943..fe8f04427a0 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include @@ -93,6 +94,12 @@ extern const int SCHEMA_VERSION_ERROR; extern const int UNKNOWN_EXCEPTION; } // namespace ErrorCodes + +namespace FailPoints +{ +extern const char pause_query_init[]; +} // namespace FailPoints + InterpreterSelectQuery::InterpreterSelectQuery( const ASTPtr & query_ptr_, const Context & context_, @@ -131,6 +138,15 @@ InterpreterSelectQuery::~InterpreterSelectQuery() = default; void InterpreterSelectQuery::init(const Names & required_result_column_names) { + /// the failpoint pause_query_init should use with the failpoint unblock_query_init_after_write, + /// to fulfill that the select query action will be blocked before init state to wait the write action finished. + /// In using, we need enable unblock_query_init_after_write in our test code, + /// and before each write statement take effect, we need enable pause_query_init. + /// When the write action finished, the pause_query_init will be disabled automatically, + /// and then the select query could be continued. + /// you can refer multi_alter_with_write.test for an example. + FAIL_POINT_PAUSE(FailPoints::pause_query_init); + if (!context.hasQueryContext()) context.setQueryContext(context); diff --git a/dbms/src/Storages/Transaction/PartitionStreams.cpp b/dbms/src/Storages/Transaction/PartitionStreams.cpp index 456f067fe5e..cf151c4270d 100644 --- a/dbms/src/Storages/Transaction/PartitionStreams.cpp +++ b/dbms/src/Storages/Transaction/PartitionStreams.cpp @@ -40,6 +40,8 @@ namespace FailPoints extern const char pause_before_apply_raft_cmd[]; extern const char pause_before_apply_raft_snapshot[]; extern const char force_set_safepoint_when_decode_block[]; +extern const char unblock_query_init_after_write[]; +extern const char pause_query_init[]; } // namespace FailPoints namespace ErrorCodes @@ -151,6 +153,7 @@ static void writeRegionDataToStorage( default: throw Exception("Unknown StorageEngine: " + toString(static_cast(storage->engineType())), ErrorCodes::LOGICAL_ERROR); } + write_part_cost = watch.elapsedMilliseconds(); GET_METRIC(tiflash_raft_write_data_to_storage_duration_seconds, type_write).Observe(write_part_cost / 1000.0); if (need_decode) @@ -165,10 +168,20 @@ static void writeRegionDataToStorage( /// decoding data. Check the test case for more details. FAIL_POINT_PAUSE(FailPoints::pause_before_apply_raft_cmd); + /// disable pause_query_init when the write action finish, to make the query action continue. + /// the usage of unblock_query_init_after_write and pause_query_init can refer to InterpreterSelectQuery::init + SCOPE_EXIT({ + fiu_do_on(FailPoints::unblock_query_init_after_write, { + FailPointHelper::disableFailPoint(FailPoints::pause_query_init); + }); + }); + /// Try read then write once. { if (atomic_read_write(false)) + { return; + } } /// If first try failed, sync schema and force read then write. @@ -177,10 +190,12 @@ static void writeRegionDataToStorage( tmt.getSchemaSyncer()->syncSchemas(context); if (!atomic_read_write(true)) + { // Failure won't be tolerated this time. // TODO: Enrich exception message. throw Exception("Write region " + std::to_string(region->id()) + " to table " + std::to_string(table_id) + " failed", ErrorCodes::LOGICAL_ERROR); + } } } diff --git a/tests/fullstack-test2/ddl/multi_alter_with_write.test b/tests/fullstack-test2/ddl/multi_alter_with_write.test new file mode 100644 index 00000000000..3284511d775 --- /dev/null +++ b/tests/fullstack-test2/ddl/multi_alter_with_write.test @@ -0,0 +1,880 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# this test focus on the case when multi DDL actions happen closely +#( and these DDL actions will be fetched in the same regular sync schema duration.) +# and there are some corresponding insert(write) actions between these DDL actions. +# Considering that these write actions and these schema change will arrive at +# tiflash in a different order, we simulate these different order situation to check +# that our schema module was working correctly. + +# TiDB Timeline : write cmd 1 | alter cmd 1 | write cmd 2 | alter cmd 2 | write cmd 3 + +# stop regular schema sync +=> DBGInvoke __enable_schema_sync_service('false') + +# Enable the failpoint and make it pause before applying the raft cmd to write a row +>> DBGInvoke __init_fail_point() +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# Enable the failpoint to make our query only start when the write action finished +>> DBGInvoke __enable_fail_point(unblock_query_init_after_write) + +# ----------------------------------------------------------------------------- +# Order 1 : write cmd 1 | alter cmd 1 | write cmd 2 | alter cmd 2 | write cmd 3 +# ----------------------------------------------------------------------------- + +mysql> drop table if exists test.t +mysql> create table test.t(a int primary key, b decimal(5,2) not NULL, c varchar(10), d int default 0); + +mysql> alter table test.t set tiflash replica 1; + +func> wait_table test t + +# write cmd 1 +mysql> insert into test.t (a, b, c) values (1, 4.5, 'abc'); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 1 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┐ +│ 1 │ 4.50 │ abc │ 0 │ +└─────┴──────┴───────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# alter cmd 1 +mysql> alter table test.t add column e decimal(6,1) NULL; + +# make alter cmd 1 take effect +>> DBGInvoke __refresh_schemas() + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┬─e───┐ +│ 1 │ 4.50 │ abc │ 0 │ \N │ +└─────┴──────┴───────┴─────┴─────┘ + +# write cmd 2 +mysql> insert into test.t values (3, 0.2, 'ccc', 3, 0.1); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 2 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┬─e───┐ +│ 1 │ 4.50 │ abc │ 0 │ \N │ +│ 3 │ 0.20 │ ccc │ 3 │ 0.1 │ +└─────┴──────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# alter cmd 2 +mysql> alter table test.t drop column b; + +# make alter cmd 2 take effect +>> DBGInvoke __refresh_schemas() + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +└─────┴───────┴─────┴─────┘ + +# write cmd 3 +mysql> insert into test.t values (4, 'abcd', 10, 0.2); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 3 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +│ 4 │ abcd │ 10 │ 0.2 │ +└─────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# --------------------------------------------------------------------------------------------- +# Order 2 : write cmd 1 | alter cmd 1 | write cmd 2 | write cmd 3 --> sync schema(alter cmd 2) +# --------------------------------------------------------------------------------------------- + +mysql> drop table if exists test.t +mysql> create table test.t(a int primary key, b decimal(5,2) not NULL, c varchar(10), d int default 0); + +mysql> alter table test.t set tiflash replica 1; + +func> wait_table test t + +# write cmd 1 +mysql> insert into test.t (a, b, c) values (1, 4.5, 'abc'); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 1 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┐ +│ 1 │ 4.50 │ abc │ 0 │ +└─────┴──────┴───────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# alter cmd 1 +mysql> alter table test.t add column e decimal(6,1) NULL; + +# make alter cmd 1 take effect +>> DBGInvoke __refresh_schemas() + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┬─e───┐ +│ 1 │ 4.50 │ abc │ 0 │ \N │ +└─────┴──────┴───────┴─────┴─────┘ + +# write cmd 2 +mysql> insert into test.t values (3, 0.2, 'ccc', 3, 0.1); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 2 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┬─e───┐ +│ 1 │ 4.50 │ abc │ 0 │ \N │ +│ 3 │ 0.20 │ ccc │ 3 │ 0.1 │ +└─────┴──────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# alter cmd 2 +mysql> alter table test.t drop column b; + +# write cmd 3 +mysql> insert into test.t values (4, 'abcd', 10, 0.2); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 3 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +# check what happen after write cmd 3 --> call sync schema and get alter cmd 2 happen +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +│ 4 │ abcd │ 10 │ 0.2 │ +└─────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# ----------------------------------------------------------------------------------------------- +# Order 3 : write cmd 1 | alter cmd 1 | alter cmd 2 | write cmd 2 -->sync schema() | write cmd 3 +# ----------------------------------------------------------------------------------------------- + +mysql> drop table if exists test.t +mysql> create table test.t(a int primary key, b decimal(5,2) not NULL, c varchar(10), d int default 0); + +mysql> alter table test.t set tiflash replica 1; + +func> wait_table test t + +# write cmd 1 +mysql> insert into test.t (a, b, c) values (1, 4.5, 'abc'); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 1 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┐ +│ 1 │ 4.50 │ abc │ 0 │ +└─────┴──────┴───────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# alter cmd 1 +mysql> alter table test.t add column e decimal(6,1) NULL; + +# make alter cmd 1 take effect +>> DBGInvoke __refresh_schemas() + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┬─e───┐ +│ 1 │ 4.50 │ abc │ 0 │ \N │ +└─────┴──────┴───────┴─────┴─────┘ + +# write cmd 2 +mysql> insert into test.t values (3, 0.2, 'ccc', 3, 0.1); + +# alter cmd 2 +mysql> alter table test.t drop column b; + +# make alter cmd 2 take effect +>> DBGInvoke __refresh_schemas() + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 2 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +└─────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# write cmd 3 +mysql> insert into test.t values (4, 'abcd', 10, 0.2); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 3 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +│ 4 │ abcd │ 10 │ 0.2 │ +└─────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# ----------------------------------------------------------------------------------------------- +# Order 4 : write cmd 1 | write cmd 2 --> sync schema(alter cmd 1) | alter cmd 2 | write cmd 3 +# ----------------------------------------------------------------------------------------------- + +mysql> drop table if exists test.t +mysql> create table test.t(a int primary key, b decimal(5,2) not NULL, c varchar(10), d int default 0); + +mysql> alter table test.t set tiflash replica 1; + +func> wait_table test t + +# write cmd 1 +mysql> insert into test.t (a, b, c) values (1, 4.5, 'abc'); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 1 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┐ +│ 1 │ 4.50 │ abc │ 0 │ +└─────┴──────┴───────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# alter cmd 1 +mysql> alter table test.t add column e decimal(6,1) NULL; + +# check no schema change before write cmd 2 take effect +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┐ +│ 1 │ 4.50 │ abc │ 0 │ +└─────┴──────┴───────┴─────┘ + +# write cmd 2 +mysql> insert into test.t values (3, 0.2, 'ccc', 3, 0.1); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 2 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +# check what happen after write cmd 2 --> should call sync schema, get the alter cmd 1 happened. +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┬─e───┐ +│ 1 │ 4.50 │ abc │ 0 │ \N │ +│ 3 │ 0.20 │ ccc │ 3 │ 0.1 │ +└─────┴──────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# alter cmd 2 +mysql> alter table test.t drop column b; + +# make alter cmd 2 take effect +>> DBGInvoke __refresh_schemas() + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +└─────┴───────┴─────┴─────┘ + +# write cmd 3 +mysql> insert into test.t values (4, 'abcd', 10, 0.2); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 3 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +│ 4 │ abcd │ 10 │ 0.2 │ +└─────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# --------------------------------------------------------------------------------------------------------------------- +# Order 5 : write cmd 1 | write cmd 2 --> sync schema(alter cmd 1) | write cmd 3 --> sync schema(alter cmd 2) +# ---------------------------------------------------------------------------------------------------------------------- + +mysql> drop table if exists test.t +mysql> create table test.t(a int primary key, b decimal(5,2) not NULL, c varchar(10), d int default 0); + +mysql> alter table test.t set tiflash replica 1; + +func> wait_table test t + +# write cmd 1 +mysql> insert into test.t (a, b, c) values (1, 4.5, 'abc'); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 1 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┐ +│ 1 │ 4.50 │ abc │ 0 │ +└─────┴──────┴───────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# alter cmd 1 +mysql> alter table test.t add column e decimal(6,1) NULL; + +# write cmd 2 +mysql> insert into test.t values (3, 0.2, 'ccc', 3, 0.1); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 2 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +# check what happen after write cmd 2 --> should call sync schema, get the alter cmd 1 happened. +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┬─e───┐ +│ 1 │ 4.50 │ abc │ 0 │ \N │ +│ 3 │ 0.20 │ ccc │ 3 │ 0.1 │ +└─────┴──────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# alter cmd 2 +mysql> alter table test.t drop column b; + +# write cmd 3 +mysql> insert into test.t values (4, 'abcd', 10, 0.2); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 3 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +# check what happen after write cmd 3 --> should call sync schema, get the alter cmd 2 happened. +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +│ 4 │ abcd │ 10 │ 0.2 │ +└─────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# ----------------------------------------------------------------------------------------------- +# Order 6 : write cmd 1 | write cmd 2 --> sync schema(alter cmd 1 alter cmd 2) | write cmd 3 +# ----------------------------------------------------------------------------------------------- + +mysql> drop table if exists test.t +mysql> create table test.t(a int primary key, b decimal(5,2) not NULL, c varchar(10), d int default 0); + +mysql> alter table test.t set tiflash replica 1; + +func> wait_table test t + +# write cmd 1 +mysql> insert into test.t (a, b, c) values (1, 4.5, 'abc'); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 1 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┐ +│ 1 │ 4.50 │ abc │ 0 │ +└─────┴──────┴───────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# alter cmd 1 +mysql> alter table test.t add column e decimal(6,1) NULL; + +# write cmd 2 +mysql> insert into test.t values (3, 0.2, 'ccc', 3, 0.1); + +# alter cmd 2 +mysql> alter table test.t drop column b; + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 2 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +# check what happen after write cmd 2 --> should call sync schema, get the alter cmd 1 && alter cmd 2 happened. +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +└─────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# write cmd 3 +mysql> insert into test.t values (4, 'abcd', 10, 0.2); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 3 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +│ 4 │ abcd │ 10 │ 0.2 │ +└─────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# ------------------------------------------------------------------------------- +# Order 7 : alter cmd 1 | write cmd 1 | write cmd 2 | alter cmd 2 | write cmd 3 +# ------------------------------------------------------------------------------- + +mysql> drop table if exists test.t +mysql> create table test.t(a int primary key, b decimal(5,2) not NULL, c varchar(10), d int default 0); + +mysql> alter table test.t set tiflash replica 1; + +func> wait_table test t + +# add a new pre write to make check the alter cmd 1 more convenient. +mysql> insert into test.t (a, b, c) values (0, 0, ' '); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┐ +│ 0 │ 0.00 │ │ 0 │ +└─────┴──────┴───────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + + +# alter cmd 1 +mysql> alter table test.t add column e decimal(6,1) NULL; + +# make alter cmd 1 take effect +>> DBGInvoke __refresh_schemas() + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┬─e───┐ +│ 0 │ 0.00 │ │ 0 │ \N │ +└─────┴──────┴───────┴─────┴─────┘ + +# write cmd 1 +mysql> insert into test.t (a, b, c) values (1, 4.5, 'abc'); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 1 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┬─e───┐ +│ 0 │ 0.00 │ │ 0 │ \N │ +│ 1 │ 4.50 │ abc │ 0 │ \N │ +└─────┴──────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# write cmd 2 +mysql> insert into test.t values (3, 0.2, 'ccc', 3, 0.1); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 2 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┬─e───┐ +│ 0 │ 0.00 │ │ 0 │ \N │ +│ 1 │ 4.50 │ abc │ 0 │ \N │ +│ 3 │ 0.20 │ ccc │ 3 │ 0.1 │ +└─────┴──────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# alter cmd 2 +mysql> alter table test.t drop column b; + +# make alter cmd 2 take effect +>> DBGInvoke __refresh_schemas() + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 0 │ │ 0 │ \N │ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +└─────┴───────┴─────┴─────┘ + +# write cmd 3 +mysql> insert into test.t values (4, 'abcd', 10, 0.2); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 3 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 0 │ │ 0 │ \N │ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +│ 4 │ abcd │ 10 │ 0.2 │ +└─────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# -------------------------------------------------------------------------------------------------- +# Order 8 : alter cmd 1 | write cmd 1 | write cmd 2 | write cmd 3 --> sync schema(alter cmd 2) +# -------------------------------------------------------------------------------------------------- + +mysql> drop table if exists test.t +mysql> create table test.t(a int primary key, b decimal(5,2) not NULL, c varchar(10), d int default 0); + +mysql> alter table test.t set tiflash replica 1; + +func> wait_table test t + +# add a new pre write to make check the alter cmd 1 more convenient. +mysql> insert into test.t (a, b, c) values (0, 0, ' '); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┐ +│ 0 │ 0.00 │ │ 0 │ +└─────┴──────┴───────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# alter cmd 1 +mysql> alter table test.t add column e decimal(6,1) NULL; + +# make alter cmd 1 take effect +>> DBGInvoke __refresh_schemas() + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┬─e───┐ +│ 0 │ 0.00 │ │ 0 │ \N │ +└─────┴──────┴───────┴─────┴─────┘ + +# write cmd 1 +mysql> insert into test.t (a, b, c) values (1, 4.5, 'abc'); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 1 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┬─e───┐ +│ 0 │ 0.00 │ │ 0 │ \N │ +│ 1 │ 4.50 │ abc │ 0 │ \N │ +└─────┴──────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# write cmd 2 +mysql> insert into test.t values (3, 0.2, 'ccc', 3, 0.1); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 2 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┬─e───┐ +│ 0 │ 0.00 │ │ 0 │ \N │ +│ 1 │ 4.50 │ abc │ 0 │ \N │ +│ 3 │ 0.20 │ ccc │ 3 │ 0.1 │ +└─────┴──────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# alter cmd 2 +mysql> alter table test.t drop column b; + +# write cmd 3 +mysql> insert into test.t values (4, 'abcd', 10, 0.2); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 3 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +# check what happen after write cmd 3 --> should call sync schema, get the alter cmd 2 happened. +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 0 │ │ 0 │ \N │ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +│ 4 │ abcd │ 10 │ 0.2 │ +└─────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# -------------------------------------------------------------------------------------------------- +# Order 9 : alter cmd 1 | write cmd 1 | alter cmd 2 | write cmd 2 -->sync schema() | write cmd 3 +# -------------------------------------------------------------------------------------------------- + +mysql> drop table if exists test.t +mysql> create table test.t(a int primary key, b decimal(5,2) not NULL, c varchar(10), d int default 0); + +mysql> alter table test.t set tiflash replica 1; + +func> wait_table test t + +# add a new pre write to make check the alter cmd 1 more convenient. +mysql> insert into test.t (a, b, c) values (0, 0, ' '); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┐ +│ 0 │ 0.00 │ │ 0 │ +└─────┴──────┴───────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# alter cmd 1 +mysql> alter table test.t add column e decimal(6,1) NULL; + +# make alter cmd 1 take effect +>> DBGInvoke __refresh_schemas() + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┬─e───┐ +│ 0 │ 0.00 │ │ 0 │ \N │ +└─────┴──────┴───────┴─────┴─────┘ + +# write cmd 1 +mysql> insert into test.t (a, b, c) values (1, 4.5, 'abc'); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 1 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┬─e───┐ +│ 0 │ 0.00 │ │ 0 │ \N │ +│ 1 │ 4.50 │ abc │ 0 │ \N │ +└─────┴──────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# write cmd 2 +mysql> insert into test.t values (3, 0.2, 'ccc', 3, 0.1); + +# alter cmd 2 +mysql> alter table test.t drop column b; + +# make alter cmd 2 take effect +>> DBGInvoke __refresh_schemas() + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 2 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 0 │ │ 0 │ \N │ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +└─────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# write cmd 3 +mysql> insert into test.t values (4, 'abcd', 10, 0.2); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 3 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 0 │ │ 0 │ \N │ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +│ 4 │ abcd │ 10 │ 0.2 │ +└─────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# ------------------------------------------------------------------------------------------------------------------ +# Order 10 : alter cmd 1 | alter cmd 2 | write cmd 1 -->sync schema() | write cmd 2 -->sync schema() | write cmd 3 +# ------------------------------------------------------------------------------------------------------------------ + +mysql> drop table if exists test.t +mysql> create table test.t(a int primary key, b decimal(5,2) not NULL, c varchar(10), d int default 0); + +mysql> alter table test.t set tiflash replica 1; + +func> wait_table test t + +# add a new pre write to make check the alter cmd 1 more convenient. +mysql> insert into test.t (a, b, c) values (0, 0, ' '); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─b────┬─c─────┬─d───┐ +│ 0 │ 0.00 │ │ 0 │ +└─────┴──────┴───────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# write cmd 1 +mysql> insert into test.t (a, b, c) values (1, 4.5, 'abc'); + +# alter cmd 1 +mysql> alter table test.t add column e decimal(6,1) NULL; + +# make alter cmd 1 take effect +>> DBGInvoke __refresh_schemas() + +# write cmd 2 +mysql> insert into test.t values (3, 0.2, 'ccc', 3, 0.1); + +# alter cmd 2 +mysql> alter table test.t drop column b; + +# make alter cmd 2 take effect +>> DBGInvoke __refresh_schemas() + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 1 and write cmd 2 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 0 │ │ 0 │ \N │ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +└─────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + +# write cmd 3 +mysql> insert into test.t values (4, 'abcd', 10, 0.2); + +# enable pause_query_init make query start until write cmd finish +>> DBGInvoke __enable_fail_point(pause_query_init) + +# make write cmd 3 take effect +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) + +=> DBGInvoke query_mapped('select * from \$d.\$t', test, t) +┌─a───┬─c─────┬─d───┬─e───┐ +│ 0 │ │ 0 │ \N │ +│ 1 │ abc │ 0 │ \N │ +│ 3 │ ccc │ 3 │ 0.1 │ +│ 4 │ abcd │ 10 │ 0.2 │ +└─────┴───────┴─────┴─────┘ + +>> DBGInvoke __enable_fail_point(pause_before_apply_raft_cmd) + + +## + +=> DBGInvoke __enable_schema_sync_service('true') +>> DBGInvoke __disable_fail_point(pause_before_apply_raft_cmd) +>> DBGInvoke __disable_fail_point(unblock_query_init_after_write) +>> DBGInvoke __disable_fail_point(pause_query_init) \ No newline at end of file From 45bc5a45f680dd9d1846616c2dd9ecf3be89dcd6 Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger <60744015+Lloyd-Pottiger@users.noreply.github.com> Date: Wed, 22 Jun 2022 11:22:36 +0800 Subject: [PATCH 052/104] Revert "Revise default background threads size" (#5176) close pingcap/tiflash#5177 --- dbms/src/Core/Defines.h | 1 + dbms/src/Interpreters/Context.cpp | 24 ++++---------- dbms/src/Interpreters/Context.h | 4 +-- dbms/src/Interpreters/Settings.h | 6 ++-- dbms/src/Server/Server.cpp | 33 ++++++++----------- .../src/Storages/BackgroundProcessingPool.cpp | 3 -- dbms/src/Storages/BackgroundProcessingPool.h | 4 +-- dbms/src/TestUtils/TiFlashTestEnv.cpp | 6 ---- 8 files changed, 27 insertions(+), 54 deletions(-) diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index 75f6f16bb25..33d116dae33 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -78,6 +78,7 @@ /// too short a period can cause errors to disappear immediately after creation. #define DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD (2 * DBMS_DEFAULT_SEND_TIMEOUT_SEC) #define DEFAULT_QUERIES_QUEUE_WAIT_TIME_MS 5000 /// Maximum waiting time in the request queue. +#define DBMS_DEFAULT_BACKGROUND_POOL_SIZE 16 #define DBMS_MIN_REVISION_WITH_CLIENT_INFO 54032 #define DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE 54058 diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 2dbd495d2c4..44699a324f4 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -68,8 +68,10 @@ #include #include +#include #include -#include +#include + namespace ProfileEvents { @@ -1439,33 +1441,19 @@ void Context::dropCaches() const } BackgroundProcessingPool & Context::getBackgroundPool() -{ - // Note: shared->background_pool should be initialized first. - auto lock = getLock(); - return *shared->background_pool; -} - -BackgroundProcessingPool & Context::initializeBackgroundPool(UInt16 pool_size) { auto lock = getLock(); if (!shared->background_pool) - shared->background_pool = std::make_shared(pool_size); + shared->background_pool = std::make_shared(settings.background_pool_size); return *shared->background_pool; } BackgroundProcessingPool & Context::getBlockableBackgroundPool() { - // TODO: maybe a better name for the pool - // Note: shared->blockable_background_pool should be initialized first. - auto lock = getLock(); - return *shared->blockable_background_pool; -} - -BackgroundProcessingPool & Context::initializeBlockableBackgroundPool(UInt16 pool_size) -{ + // TODO: choose a better thread pool size and maybe a better name for the pool auto lock = getLock(); if (!shared->blockable_background_pool) - shared->blockable_background_pool = std::make_shared(pool_size); + shared->blockable_background_pool = std::make_shared(settings.background_pool_size); return *shared->blockable_background_pool; } diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 434179e1ab8..b6e759e364b 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -380,9 +380,7 @@ class Context bool useL0Opt() const; BackgroundProcessingPool & getBackgroundPool(); - BackgroundProcessingPool & initializeBackgroundPool(UInt16 pool_size); BackgroundProcessingPool & getBlockableBackgroundPool(); - BackgroundProcessingPool & initializeBlockableBackgroundPool(UInt16 pool_size); void createTMTContext(const TiFlashRaftConfig & raft_config, pingcap::ClusterConfig && cluster_config); @@ -507,7 +505,7 @@ class DDLGuard class SessionCleaner { public: - explicit SessionCleaner(Context & context_) + SessionCleaner(Context & context_) : context{context_} {} ~SessionCleaner(); diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index f2b3bbbd7fe..9361e0525d2 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -80,8 +80,8 @@ struct Settings M(SettingBool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.") \ M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.") \ M(SettingBool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.") \ - M(SettingUInt64, background_pool_size, 0, "Number of threads performing background work for tables (for example, merging in merge tree). Only effective at server startup. " \ - "0 means a quarter of the number of logical CPU cores of the machine.") \ + M(SettingUInt64, background_pool_size, DBMS_DEFAULT_BACKGROUND_POOL_SIZE, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server " \ + "startup.") \ \ M(SettingBool, optimize_move_to_prewhere, true, "Allows disabling WHERE to PREWHERE optimization in SELECT queries from MergeTree.") \ \ @@ -356,7 +356,7 @@ struct Settings M(SettingUInt64, elastic_threadpool_shrink_period_ms, 300000, "The shrink period(ms) of elastic thread pool.") \ M(SettingBool, enable_local_tunnel, true, "Enable local data transfer between local MPP tasks.") \ M(SettingBool, enable_async_grpc_client, true, "Enable async grpc in MPP.") \ - M(SettingUInt64, grpc_completion_queue_pool_size, 0, "The size of gRPC completion queue pool. 0 means the number of logical CPU cores. Only effective at server startup")\ + M(SettingUInt64, grpc_completion_queue_pool_size, 0, "The size of gRPC completion queue pool. 0 means using hardware_concurrency.")\ M(SettingBool, enable_async_server, true, "Enable async rpc server.") \ M(SettingUInt64, async_pollers_per_cq, 200, "grpc async pollers per cqs") \ M(SettingUInt64, async_cqs, 1, "grpc async cqs") \ diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 186ab0889d8..1bb35e51866 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -53,15 +53,10 @@ #include #include #include -#include -#include -#include #include #include #include -#include #include -#include #include #include #include @@ -86,6 +81,12 @@ #include #include +#include "HTTPHandlerFactory.h" +#include "MetricsPrometheus.h" +#include "MetricsTransmitter.h" +#include "StatusFile.h" +#include "TCPHandlerFactory.h" + #if Poco_NetSSL_FOUND #include #include @@ -1127,19 +1128,6 @@ int Server::main(const std::vector & /*args*/) global_context->getPathCapacity(), global_context->getFileProvider()); - /// if default value of background_pool_size is 0 - /// set it to the a quarter of the number of logical CPU cores of machine. - Settings & settings = global_context->getSettingsRef(); - if (settings.background_pool_size == 0) - { - global_context->setSetting("background_pool_size", std::to_string(server_info.cpu_info.logical_cores / 4)); - } - LOG_FMT_INFO(log, "Background & Blockable Background pool size: {}", settings.background_pool_size); - - /// Initialize the background & blockable background thread pool. - auto & bg_pool = global_context->initializeBackgroundPool(settings.background_pool_size); - auto & blockable_bg_pool = global_context->initializeBlockableBackgroundPool(settings.background_pool_size); - global_context->initializePageStorageMode(global_context->getPathPool(), STORAGE_FORMAT_CURRENT.page); global_context->initializeGlobalStoragePoolIfNeed(global_context->getPathPool()); LOG_FMT_INFO(log, "Global PageStorage run mode is {}", static_cast(global_context->getPageStorageRunMode())); @@ -1256,6 +1244,13 @@ int Server::main(const std::vector & /*args*/) /// Load global settings from default_profile and system_profile. /// It internally depends on UserConfig::parseSettings. global_context->setDefaultProfiles(config()); + Settings & settings = global_context->getSettingsRef(); + + /// Initialize the background thread pool. + /// It internally depends on settings.background_pool_size, + /// so must be called after settings has been load. + auto & bg_pool = global_context->getBackgroundPool(); + auto & blockable_bg_pool = global_context->getBlockableBackgroundPool(); /// Initialize RateLimiter. global_context->initializeRateLimiter(config(), bg_pool, blockable_bg_pool); @@ -1407,7 +1402,7 @@ int Server::main(const std::vector & /*args*/) { auto size = settings.grpc_completion_queue_pool_size; if (size == 0) - size = server_info.cpu_info.logical_cores; + size = std::thread::hardware_concurrency(); GRPCCompletionQueuePool::global_instance = std::make_unique(size); } diff --git a/dbms/src/Storages/BackgroundProcessingPool.cpp b/dbms/src/Storages/BackgroundProcessingPool.cpp index 15740fa2875..45ba032bf53 100644 --- a/dbms/src/Storages/BackgroundProcessingPool.cpp +++ b/dbms/src/Storages/BackgroundProcessingPool.cpp @@ -86,9 +86,6 @@ BackgroundProcessingPool::BackgroundProcessingPool(int size_) : size(size_) , thread_ids_counter(size_) { - if (size <= 0) - throw Exception("BackgroundProcessingPool size must be greater than 0", ErrorCodes::LOGICAL_ERROR); - LOG_FMT_INFO(&Poco::Logger::get("BackgroundProcessingPool"), "Create BackgroundProcessingPool with {} threads", size); threads.resize(size); diff --git a/dbms/src/Storages/BackgroundProcessingPool.h b/dbms/src/Storages/BackgroundProcessingPool.h index 49a01b3a397..1ba6c4efcf8 100644 --- a/dbms/src/Storages/BackgroundProcessingPool.h +++ b/dbms/src/Storages/BackgroundProcessingPool.h @@ -81,7 +81,7 @@ class BackgroundProcessingPool using TaskHandle = std::shared_ptr; - explicit BackgroundProcessingPool(int size_); + BackgroundProcessingPool(int size_); size_t getNumberOfThreads() const { return size; } @@ -96,7 +96,7 @@ class BackgroundProcessingPool /// 2. thread B also get the same task /// 3. thread A finish the execution of the task quickly, release the task and try to update the next schedule time of the task /// 4. thread B find the task is not occupied and execute the task again almost immediately - TaskHandle addTask(const Task & task, bool multi = true, size_t interval_ms = 0); + TaskHandle addTask(const Task & task, const bool multi = true, const size_t interval_ms = 0); void removeTask(const TaskHandle & task); ~BackgroundProcessingPool(); diff --git a/dbms/src/TestUtils/TiFlashTestEnv.cpp b/dbms/src/TestUtils/TiFlashTestEnv.cpp index a7bcfe43d7a..cbd42b57550 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.cpp +++ b/dbms/src/TestUtils/TiFlashTestEnv.cpp @@ -24,8 +24,6 @@ #include #include -#include - namespace DB::tests { std::unique_ptr TiFlashTestEnv::global_context = nullptr; @@ -41,10 +39,6 @@ void TiFlashTestEnv::initializeGlobalContext(Strings testdata_path, PageStorageR KeyManagerPtr key_manager = std::make_shared(false); global_context->initializeFileProvider(key_manager, false); - // initialize background & blockable background thread pool - global_context->initializeBackgroundPool(std::thread::hardware_concurrency() / 4); - global_context->initializeBlockableBackgroundPool(std::thread::hardware_concurrency() / 4); - // Theses global variables should be initialized by the following order // 1. capacity // 2. path pool From bfceb28168082e4107ccf3828ee720ca3139e22d Mon Sep 17 00:00:00 2001 From: Schrodinger ZHU Yifan Date: Wed, 22 Jun 2022 14:42:37 +0800 Subject: [PATCH 053/104] chore: remove extra dyn cast (#5186) close pingcap/tiflash#5185 --- dbms/src/Flash/Coprocessor/ArrowColCodec.cpp | 75 ++++++++++---------- 1 file changed, 37 insertions(+), 38 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/ArrowColCodec.cpp b/dbms/src/Flash/Coprocessor/ArrowColCodec.cpp index a1c6061948a..1609c83b029 100644 --- a/dbms/src/Flash/Coprocessor/ArrowColCodec.cpp +++ b/dbms/src/Flash/Coprocessor/ArrowColCodec.cpp @@ -20,7 +20,6 @@ #include #include #include -#include #include #include #include @@ -41,7 +40,7 @@ extern const int NOT_IMPLEMENTED; const IColumn * getNestedCol(const IColumn * flash_col) { if (flash_col->isColumnNullable()) - return dynamic_cast(flash_col)->getNestedColumnPtr().get(); + return static_cast(flash_col)->getNestedColumnPtr().get(); else return flash_col; } @@ -75,8 +74,8 @@ bool flashDecimalColToArrowColInternal( const IColumn * nested_col = getNestedCol(flash_col_untyped); if (checkColumn>(nested_col) && checkDataType>(data_type)) { - const ColumnDecimal * flash_col = checkAndGetColumn>(nested_col); - const DataTypeDecimal * type = checkAndGetDataType>(data_type); + const auto * flash_col = checkAndGetColumn>(nested_col); + const auto * type = checkAndGetDataType>(data_type); UInt32 scale = type->getScale(); for (size_t i = start_index; i < end_index; i++) { @@ -92,8 +91,8 @@ bool flashDecimalColToArrowColInternal( std::vector digits; digits.reserve(type->getPrec()); decimalToVector(dec.value, digits, scale); - TiDBDecimal tiDecimal(scale, digits, dec.value < 0); - dag_column.append(tiDecimal); + TiDBDecimal ti_decimal(scale, digits, dec.value < 0); + dag_column.append(ti_decimal); } return true; } @@ -121,7 +120,7 @@ template bool flashIntegerColToArrowColInternal(TiDBColumn & dag_column, const IColumn * flash_col_untyped, size_t start_index, size_t end_index) { const IColumn * nested_col = getNestedCol(flash_col_untyped); - if (const ColumnVector * flash_col = checkAndGetColumn>(nested_col)) + if (const auto * flash_col = checkAndGetColumn>(nested_col)) { constexpr bool is_unsigned = std::is_unsigned_v; for (size_t i = start_index; i < end_index; i++) @@ -135,9 +134,9 @@ bool flashIntegerColToArrowColInternal(TiDBColumn & dag_column, const IColumn * } } if constexpr (is_unsigned) - dag_column.append((UInt64)flash_col->getElement(i)); + dag_column.append(static_cast(flash_col->getElement(i))); else - dag_column.append((Int64)flash_col->getElement(i)); + dag_column.append(static_cast(flash_col->getElement(i))); } return true; } @@ -148,7 +147,7 @@ template void flashDoubleColToArrowCol(TiDBColumn & dag_column, const IColumn * flash_col_untyped, size_t start_index, size_t end_index) { const IColumn * nested_col = getNestedCol(flash_col_untyped); - if (const ColumnVector * flash_col = checkAndGetColumn>(nested_col)) + if (const auto * flash_col = checkAndGetColumn>(nested_col)) { for (size_t i = start_index; i < end_index; i++) { @@ -160,7 +159,7 @@ void flashDoubleColToArrowCol(TiDBColumn & dag_column, const IColumn * flash_col continue; } } - dag_column.append((T)flash_col->getElement(i)); + dag_column.append(static_cast(flash_col->getElement(i))); } return; } @@ -196,7 +195,7 @@ void flashDateOrDateTimeColToArrowCol( { const IColumn * nested_col = getNestedCol(flash_col_untyped); using DateFieldType = DataTypeMyTimeBase::FieldType; - auto * flash_col = checkAndGetColumn>(nested_col); + const auto * flash_col = checkAndGetColumn>(nested_col); for (size_t i = start_index; i < end_index; i++) { if constexpr (is_nullable) @@ -217,7 +216,7 @@ void flashStringColToArrowCol(TiDBColumn & dag_column, const IColumn * flash_col { const IColumn * nested_col = getNestedCol(flash_col_untyped); // columnFixedString is not used so do not check it - auto * flash_col = checkAndGetColumn(nested_col); + const auto * flash_col = checkAndGetColumn(nested_col); for (size_t i = start_index; i < end_index; i++) { // todo check if we can convert flash_col to DAG col directly since the internal representation is almost the same @@ -242,7 +241,7 @@ void flashBitColToArrowCol( const tipb::FieldType & field_type) { const IColumn * nested_col = getNestedCol(flash_col_untyped); - auto * flash_col = checkAndGetColumn>(nested_col); + const auto * flash_col = checkAndGetColumn>(nested_col); for (size_t i = start_index; i < end_index; i++) { if constexpr (is_nullable) @@ -267,7 +266,7 @@ void flashEnumColToArrowCol( const IDataType * data_type) { const IColumn * nested_col = getNestedCol(flash_col_untyped); - auto * flash_col = checkAndGetColumn>(nested_col); + const auto * flash_col = checkAndGetColumn>(nested_col); const auto * enum_type = checkAndGetDataType(data_type); size_t enum_value_size = enum_type->getValues().size(); for (size_t i = start_index; i < end_index; i++) @@ -280,10 +279,10 @@ void flashEnumColToArrowCol( continue; } } - auto enum_value = (UInt64)flash_col->getElement(i); + auto enum_value = static_cast(flash_col->getElement(i)); if (enum_value == 0 || enum_value > enum_value_size) throw TiFlashException("number of enum overflow enum boundary", Errors::Coprocessor::Internal); - TiDBEnum ti_enum(enum_value, enum_type->getNameForValue((const DataTypeEnum16::FieldType)enum_value)); + TiDBEnum ti_enum(enum_value, enum_type->getNameForValue(static_cast(enum_value))); dag_column.append(ti_enum); } } @@ -300,7 +299,7 @@ void flashColToArrowCol(TiDBColumn & dag_column, const ColumnWithTypeAndName & f throw TiFlashException("Flash column and TiDB column has different not null flag", Errors::Coprocessor::Internal); } if (type->isNullable()) - type = dynamic_cast(type)->getNestedType().get(); + type = static_cast(type)->getNestedType().get(); switch (tidb_column_info.tp) { @@ -457,7 +456,7 @@ const char * arrowEnumColToFlashCol( { if (checkNull(i, null_count, null_bitmap, col)) continue; - const auto enum_value = (Int64)toLittleEndian(*(reinterpret_cast(pos + offsets[i]))); + const auto enum_value = static_cast(toLittleEndian(*(reinterpret_cast(pos + offsets[i])))); col.column->assumeMutable()->insert(Field(enum_value)); } return pos + offsets[length]; @@ -479,11 +478,11 @@ const char * arrowBitColToFlashCol( continue; const String value = String(pos + offsets[i], pos + offsets[i + 1]); if (value.length() == 0) - col.column->assumeMutable()->insert(Field(UInt64(0))); + col.column->assumeMutable()->insert(Field(static_cast(0))); UInt64 result = 0; - for (auto & c : value) + for (const auto & c : value) { - result = (result << 8u) | (UInt8)c; + result = (result << 8u) | static_cast(c); } col.column->assumeMutable()->insert(Field(result)); } @@ -500,7 +499,7 @@ T toCHDecimal(UInt8 digits_int, UInt8 digits_frac, bool negative, const Int32 * UInt8 tailing_digit = digits_frac % DIGITS_PER_WORD; typename T::NativeType value = 0; - const int word_max = int(1e9); + const int word_max = static_cast(1e9); for (int i = 0; i < word_int; i++) { value = value * word_max + word_buf[i]; @@ -552,28 +551,28 @@ const char * arrowDecimalColToFlashCol( pos += 1; Int32 word_buf[MAX_WORD_BUF_LEN]; const DataTypePtr decimal_type - = col.type->isNullable() ? dynamic_cast(col.type.get())->getNestedType() : col.type; - for (int j = 0; j < MAX_WORD_BUF_LEN; j++) + = col.type->isNullable() ? static_cast(col.type.get())->getNestedType() : col.type; + for (int & j : word_buf) { - word_buf[j] = toLittleEndian(*(reinterpret_cast(pos))); + j = toLittleEndian(*(reinterpret_cast(pos))); pos += 4; } - if (auto * type32 = checkDecimal(*decimal_type)) + if (const auto * type32 = checkDecimal(*decimal_type)) { auto res = toCHDecimal(digits_int, digits_frac, negative, word_buf); col.column->assumeMutable()->insert(DecimalField(res, type32->getScale())); } - else if (auto * type64 = checkDecimal(*decimal_type)) + else if (const auto * type64 = checkDecimal(*decimal_type)) { auto res = toCHDecimal(digits_int, digits_frac, negative, word_buf); col.column->assumeMutable()->insert(DecimalField(res, type64->getScale())); } - else if (auto * type128 = checkDecimal(*decimal_type)) + else if (const auto * type128 = checkDecimal(*decimal_type)) { auto res = toCHDecimal(digits_int, digits_frac, negative, word_buf); col.column->assumeMutable()->insert(DecimalField(res, type128->getScale())); } - else if (auto * type256 = checkDecimal(*decimal_type)) + else if (const auto * type256 = checkDecimal(*decimal_type)) { auto res = toCHDecimal(digits_int, digits_frac, negative, word_buf); col.column->assumeMutable()->insert(DecimalField(res, type256->getScale())); @@ -600,13 +599,13 @@ const char * arrowDateColToFlashCol( continue; } UInt64 chunk_time = toLittleEndian(*(reinterpret_cast(pos))); - UInt16 year = (UInt16)((chunk_time & MyTimeBase::YEAR_BIT_FIELD_MASK) >> MyTimeBase::YEAR_BIT_FIELD_OFFSET); - UInt8 month = (UInt8)((chunk_time & MyTimeBase::MONTH_BIT_FIELD_MASK) >> MyTimeBase::MONTH_BIT_FIELD_OFFSET); - UInt8 day = (UInt8)((chunk_time & MyTimeBase::DAY_BIT_FIELD_MASK) >> MyTimeBase::DAY_BIT_FIELD_OFFSET); - UInt16 hour = (UInt16)((chunk_time & MyTimeBase::HOUR_BIT_FIELD_MASK) >> MyTimeBase::HOUR_BIT_FIELD_OFFSET); - UInt8 minute = (UInt8)((chunk_time & MyTimeBase::MINUTE_BIT_FIELD_MASK) >> MyTimeBase::MINUTE_BIT_FIELD_OFFSET); - UInt8 second = (UInt8)((chunk_time & MyTimeBase::SECOND_BIT_FIELD_MASK) >> MyTimeBase::SECOND_BIT_FIELD_OFFSET); - UInt32 micro_second = (UInt32)((chunk_time & MyTimeBase::MICROSECOND_BIT_FIELD_MASK) >> MyTimeBase::MICROSECOND_BIT_FIELD_OFFSET); + auto year = static_cast((chunk_time & MyTimeBase::YEAR_BIT_FIELD_MASK) >> MyTimeBase::YEAR_BIT_FIELD_OFFSET); + auto month = static_cast((chunk_time & MyTimeBase::MONTH_BIT_FIELD_MASK) >> MyTimeBase::MONTH_BIT_FIELD_OFFSET); + auto day = static_cast((chunk_time & MyTimeBase::DAY_BIT_FIELD_MASK) >> MyTimeBase::DAY_BIT_FIELD_OFFSET); + auto hour = static_cast((chunk_time & MyTimeBase::HOUR_BIT_FIELD_MASK) >> MyTimeBase::HOUR_BIT_FIELD_OFFSET); + auto minute = static_cast((chunk_time & MyTimeBase::MINUTE_BIT_FIELD_MASK) >> MyTimeBase::MINUTE_BIT_FIELD_OFFSET); + auto second = static_cast((chunk_time & MyTimeBase::SECOND_BIT_FIELD_MASK) >> MyTimeBase::SECOND_BIT_FIELD_OFFSET); + auto micro_second = static_cast((chunk_time & MyTimeBase::MICROSECOND_BIT_FIELD_MASK) >> MyTimeBase::MICROSECOND_BIT_FIELD_OFFSET); MyDateTime mt(year, month, day, hour, minute, second, micro_second); pos += field_length; col.column->assumeMutable()->insert(Field(mt.toPackedUInt())); @@ -659,7 +658,7 @@ const char * arrowNumColToFlashCol( case TiDB::TypeFloat: u32 = toLittleEndian(*(reinterpret_cast(pos))); std::memcpy(&f32, &u32, sizeof(Float32)); - col.column->assumeMutable()->insert(Field((Float64)f32)); + col.column->assumeMutable()->insert(Field(static_cast(f32))); break; case TiDB::TypeDouble: u64 = toLittleEndian(*(reinterpret_cast(pos))); From e14c677700f66ae5df2e43eabd43e00ef95625f0 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 22 Jun 2022 15:46:37 +0800 Subject: [PATCH 054/104] Add MPPReceiverSet, which includes ExchangeReceiver and CoprocessorReader (#5175) ref pingcap/tiflash#5095 --- dbms/src/Flash/Coprocessor/DAGContext.cpp | 14 ++++-- dbms/src/Flash/Coprocessor/DAGContext.h | 20 +++++--- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 6 +-- .../Coprocessor/DAGStorageInterpreter.cpp | 1 + dbms/src/Flash/Mpp/MPPReceiverSet.cpp | 48 +++++++++++++++++++ dbms/src/Flash/Mpp/MPPReceiverSet.h | 44 +++++++++++++++++ dbms/src/Flash/Mpp/MPPTask.cpp | 17 +++---- dbms/src/Flash/Mpp/MPPTask.h | 7 +-- 8 files changed, 131 insertions(+), 26 deletions(-) create mode 100644 dbms/src/Flash/Mpp/MPPReceiverSet.cpp create mode 100644 dbms/src/Flash/Mpp/MPPReceiverSet.h diff --git a/dbms/src/Flash/Coprocessor/DAGContext.cpp b/dbms/src/Flash/Coprocessor/DAGContext.cpp index 1ef7338a589..ec0544c6ee4 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.cpp +++ b/dbms/src/Flash/Coprocessor/DAGContext.cpp @@ -206,12 +206,20 @@ void DAGContext::attachBlockIO(const BlockIO & io_) io = io_; } -const std::unordered_map> & DAGContext::getMPPExchangeReceiverMap() const +ExchangeReceiverPtr DAGContext::getMPPExchangeReceiver(const String & executor_id) const { if (!isMPPTask()) throw TiFlashException("mpp_exchange_receiver_map is used in mpp only", Errors::Coprocessor::Internal); - RUNTIME_ASSERT(mpp_exchange_receiver_map != nullptr, log, "MPPTask without exchange receiver map"); - return *mpp_exchange_receiver_map; + RUNTIME_ASSERT(mpp_receiver_set != nullptr, log, "MPPTask without receiver set"); + return mpp_receiver_set->getExchangeReceiver(executor_id); +} + +void DAGContext::addCoprocessorReader(const CoprocessorReaderPtr & coprocessor_reader) +{ + if (!isMPPTask()) + return; + RUNTIME_ASSERT(mpp_receiver_set != nullptr, log, "MPPTask without receiver set"); + return mpp_receiver_set->addCoprocessorReader(coprocessor_reader); } bool DAGContext::containsRegionsInfoForTable(Int64 table_id) const diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 07b65b2d8fe..8b94d4637a8 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -37,8 +37,13 @@ namespace DB class Context; class MPPTunnelSet; class ExchangeReceiver; -using ExchangeReceiverMap = std::unordered_map>; -using ExchangeReceiverMapPtr = std::shared_ptr>>; +using ExchangeReceiverPtr = std::shared_ptr; +/// key: executor_id of ExchangeReceiver nodes in dag. +using ExchangeReceiverMap = std::unordered_map; +class MPPReceiverSet; +using MPPReceiverSetPtr = std::shared_ptr; +class CoprocessorReader; +using CoprocessorReaderPtr = std::shared_ptr; class Join; using JoinPtr = std::shared_ptr; @@ -304,11 +309,12 @@ class DAGContext bool columnsForTestEmpty() { return columns_for_test_map.empty(); } - const std::unordered_map> & getMPPExchangeReceiverMap() const; - void setMPPExchangeReceiverMap(ExchangeReceiverMapPtr & exchange_receiver_map) + ExchangeReceiverPtr getMPPExchangeReceiver(const String & executor_id) const; + void setMPPReceiverSet(const MPPReceiverSetPtr & receiver_set) { - mpp_exchange_receiver_map = exchange_receiver_map; + mpp_receiver_set = receiver_set; } + void addCoprocessorReader(const CoprocessorReaderPtr & coprocessor_reader); void addSubquery(const String & subquery_id, SubqueryForSet && subquery); bool hasSubquery() const { return !subqueries.empty(); } @@ -369,8 +375,8 @@ class DAGContext ConcurrentBoundedQueue warnings; /// warning_count is the actual warning count during the entire execution std::atomic warning_count; - /// key: executor_id of ExchangeReceiver nodes in dag. - ExchangeReceiverMapPtr mpp_exchange_receiver_map; + + MPPReceiverSetPtr mpp_receiver_set; /// vector of SubqueriesForSets(such as join build subquery). /// The order of the vector is also the order of the subquery. std::vector subqueries; diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 86d6428c92a..e322a830744 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -481,14 +481,14 @@ void DAGQueryBlockInterpreter::recordProfileStreams(DAGPipeline & pipeline, cons void DAGQueryBlockInterpreter::handleExchangeReceiver(DAGPipeline & pipeline) { - auto it = dagContext().getMPPExchangeReceiverMap().find(query_block.source_name); - if (unlikely(it == dagContext().getMPPExchangeReceiverMap().end())) + auto exchange_receiver = dagContext().getMPPExchangeReceiver(query_block.source_name); + if (unlikely(exchange_receiver == nullptr)) throw Exception("Can not find exchange receiver for " + query_block.source_name, ErrorCodes::LOGICAL_ERROR); // todo choose a more reasonable stream number auto & exchange_receiver_io_input_streams = dagContext().getInBoundIOInputStreamsMap()[query_block.source_name]; for (size_t i = 0; i < max_streams; ++i) { - BlockInputStreamPtr stream = std::make_shared(it->second, log->identifier(), query_block.source_name); + BlockInputStreamPtr stream = std::make_shared(exchange_receiver, log->identifier(), query_block.source_name); exchange_receiver_io_input_streams.push_back(stream); stream = std::make_shared(stream, 8192, 0, log->identifier()); stream->setExtraInfo("squashing after exchange receiver"); diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index 14cddd94730..ad2de7217e0 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -486,6 +486,7 @@ void DAGStorageInterpreter::buildRemoteStreams(std::vector && rem std::vector tasks(all_tasks.begin() + task_start, all_tasks.begin() + task_end); auto coprocessor_reader = std::make_shared(schema, cluster, tasks, has_enforce_encode_type, 1); + context.getDAGContext()->addCoprocessorReader(coprocessor_reader); BlockInputStreamPtr input = std::make_shared(coprocessor_reader, log->identifier(), table_scan.getTableScanExecutorID()); pipeline.streams.push_back(input); task_start = task_end; diff --git a/dbms/src/Flash/Mpp/MPPReceiverSet.cpp b/dbms/src/Flash/Mpp/MPPReceiverSet.cpp new file mode 100644 index 00000000000..60cca308c18 --- /dev/null +++ b/dbms/src/Flash/Mpp/MPPReceiverSet.cpp @@ -0,0 +1,48 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +namespace DB +{ +void MPPReceiverSet::addExchangeReceiver(const String & executor_id, const ExchangeReceiverPtr & exchange_receiver) +{ + RUNTIME_ASSERT(exchange_receiver_map.find(executor_id) == exchange_receiver_map.end(), log, "Duplicate executor_id: {} in DAGRequest", executor_id); + exchange_receiver_map[executor_id] = exchange_receiver; +} + +void MPPReceiverSet::addCoprocessorReader(const CoprocessorReaderPtr & coprocessor_reader) +{ + coprocessor_readers.push_back(coprocessor_reader); +} + +ExchangeReceiverPtr MPPReceiverSet::getExchangeReceiver(const String & executor_id) const +{ + auto it = exchange_receiver_map.find(executor_id); + if (unlikely(it == exchange_receiver_map.end())) + return nullptr; + return it->second; +} + +void MPPReceiverSet::cancel() +{ + for (auto & it : exchange_receiver_map) + { + it.second->cancel(); + } + for (auto & cop_reader : coprocessor_readers) + cop_reader->cancel(); +} +} // namespace DB diff --git a/dbms/src/Flash/Mpp/MPPReceiverSet.h b/dbms/src/Flash/Mpp/MPPReceiverSet.h new file mode 100644 index 00000000000..44274cb3ce8 --- /dev/null +++ b/dbms/src/Flash/Mpp/MPPReceiverSet.h @@ -0,0 +1,44 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include + +namespace DB +{ +class MPPReceiverSet +{ +public: + explicit MPPReceiverSet(const String & req_id) + : log(Logger::get("MPPReceiverSet", req_id)) + {} + void addExchangeReceiver(const String & executor_id, const ExchangeReceiverPtr & exchange_receiver); + void addCoprocessorReader(const CoprocessorReaderPtr & coprocessor_reader); + ExchangeReceiverPtr getExchangeReceiver(const String & executor_id) const; + void cancel(); + +private: + /// two kinds of receiver in MPP + /// ExchangeReceiver: receiver data from other MPPTask + /// CoprocessorReader: used in remote read + ExchangeReceiverMap exchange_receiver_map; + std::vector coprocessor_readers; + const LoggerPtr log; +}; + +using MPPReceiverSetPtr = std::shared_ptr; + +} // namespace DB diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 40f03ff79ba..0381bbdfa04 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -125,7 +125,7 @@ void MPPTask::registerTunnels(const mpp::DispatchTaskRequest & task_request) void MPPTask::initExchangeReceivers() { - mpp_exchange_receiver_map = std::make_shared(); + receiver_set = std::make_shared(log->identifier()); traverseExecutors(&dag_req, [&](const tipb::Executor & executor) { if (executor.tp() == tipb::ExecType::TypeExchangeReceiver) { @@ -147,22 +147,19 @@ void MPPTask::initExchangeReceivers() if (status != RUNNING) throw Exception("exchange receiver map can not be initialized, because the task is not in running state"); - (*mpp_exchange_receiver_map)[executor_id] = exchange_receiver; + receiver_set->addExchangeReceiver(executor_id, exchange_receiver); new_thread_count_of_exchange_receiver += exchange_receiver->computeNewThreadCount(); } return true; }); - dag_context->setMPPExchangeReceiverMap(mpp_exchange_receiver_map); + dag_context->setMPPReceiverSet(receiver_set); } -void MPPTask::cancelAllExchangeReceivers() +void MPPTask::cancelAllReceivers() { - if (likely(mpp_exchange_receiver_map != nullptr)) + if (likely(receiver_set != nullptr)) { - for (auto & it : *mpp_exchange_receiver_map) - { - it.second->cancel(); - } + receiver_set->cancel(); } } @@ -393,7 +390,7 @@ void MPPTask::runImpl() else { context->getProcessList().sendCancelToQuery(context->getCurrentQueryId(), context->getClientInfo().current_user, true); - cancelAllExchangeReceivers(); + cancelAllReceivers(); writeErrToAllTunnels(err_msg); } LOG_FMT_INFO(log, "task ends, time cost is {} ms.", stopwatch.elapsedMilliseconds()); diff --git a/dbms/src/Flash/Mpp/MPPTask.h b/dbms/src/Flash/Mpp/MPPTask.h index c8423ac484c..d7e5ed169de 100644 --- a/dbms/src/Flash/Mpp/MPPTask.h +++ b/dbms/src/Flash/Mpp/MPPTask.h @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -109,7 +110,7 @@ class MPPTask : public std::enable_shared_from_this void initExchangeReceivers(); - void cancelAllExchangeReceivers(); + void cancelAllReceivers(); tipb::DAGRequest dag_req; @@ -126,8 +127,8 @@ class MPPTask : public std::enable_shared_from_this MPPTaskId id; MPPTunnelSetPtr tunnel_set; - /// key: executor_id of ExchangeReceiver nodes in dag. - ExchangeReceiverMapPtr mpp_exchange_receiver_map; + + MPPReceiverSetPtr receiver_set; int new_thread_count_of_exchange_receiver = 0; From 18325f9eb4dec584f3551e0ef7fd4593590a2a77 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Wed, 22 Jun 2022 17:14:37 +0800 Subject: [PATCH 055/104] DDL: Use Column Name Instead of Offset to Find the common handle cluster index (#5166) close pingcap/tiflash#5154 --- dbms/src/Core/Block.cpp | 14 +- dbms/src/Debug/MockTiDB.cpp | 5 +- dbms/src/Debug/dbgFuncMockRaftCommand.cpp | 42 +++-- dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp | 24 ++- dbms/src/Debug/dbgFuncRegion.cpp | 8 +- dbms/src/Debug/dbgTools.cpp | 15 +- .../DecodingStorageSchemaSnapshot.h | 14 +- .../Transaction/RegionBlockReader.cpp | 2 + dbms/src/Storages/Transaction/TiDB.cpp | 4 +- dbms/src/Storages/Transaction/TiDB.h | 21 ++- .../Storages/Transaction/TiKVRecordFormat.h | 9 +- .../Transaction/tests/RowCodecTestUtils.h | 8 +- .../tests/bench_region_block_reader.cpp | 171 ++++++++++++++++++ ...gtest_decoding_storage_schema_snapshot.cpp | 65 +++++++ .../tests/gtest_region_block_reader.cpp | 68 +++---- .../clustered_index/ddl.test | 86 +++++++++ 16 files changed, 477 insertions(+), 79 deletions(-) create mode 100644 dbms/src/Storages/Transaction/tests/bench_region_block_reader.cpp create mode 100644 dbms/src/Storages/Transaction/tests/gtest_decoding_storage_schema_snapshot.cpp diff --git a/dbms/src/Core/Block.cpp b/dbms/src/Core/Block.cpp index 28db7af82e1..971e8f36e2a 100644 --- a/dbms/src/Core/Block.cpp +++ b/dbms/src/Core/Block.cpp @@ -238,10 +238,18 @@ void Block::checkNumberOfRows() const if (rows == -1) rows = size; else if (rows != size) - throw Exception("Sizes of columns doesn't match: " - + data.front().name + ": " + toString(rows) - + ", " + elem.name + ": " + toString(size), + { + auto first_col = data.front(); + throw Exception(fmt::format( + "Sizes of columns doesn't match: {}(id={}): {}, {}(id={}): {}", + first_col.name, + first_col.column_id, + rows, + elem.name, + elem.column_id, + size), ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); + } } } diff --git a/dbms/src/Debug/MockTiDB.cpp b/dbms/src/Debug/MockTiDB.cpp index 42ab56a97c1..7b3bdb0948f 100644 --- a/dbms/src/Debug/MockTiDB.cpp +++ b/dbms/src/Debug/MockTiDB.cpp @@ -221,7 +221,6 @@ TiDB::TableInfoPtr MockTiDB::parseColumns( { String & name = string_tokens[index]; index_info.idx_cols[index].name = name; - index_info.idx_cols[index].offset = pk_column_pos_map[name]; index_info.idx_cols[index].length = -1; } } @@ -302,7 +301,7 @@ int MockTiDB::newTables( tables_by_id.emplace(table->table_info.id, table); tables_by_name.emplace(qualified_name, table); - AffectedOption opt; + AffectedOption opt{}; opt.schema_id = table->database_id; opt.table_id = table->id(); opt.old_schema_id = table->database_id; @@ -571,7 +570,7 @@ void MockTiDB::renameTables(const std::vectordatabase_id; opt.table_id = new_table->id(); opt.old_schema_id = table->database_id; diff --git a/dbms/src/Debug/dbgFuncMockRaftCommand.cpp b/dbms/src/Debug/dbgFuncMockRaftCommand.cpp index df93ee1c78d..3626041f428 100644 --- a/dbms/src/Debug/dbgFuncMockRaftCommand.cpp +++ b/dbms/src/Debug/dbgFuncMockRaftCommand.cpp @@ -40,7 +40,7 @@ void MockRaftCommand::dbgFuncRegionBatchSplit(Context & context, const ASTs & ar auto & tmt = context.getTMTContext(); auto & kvstore = tmt.getKVStore(); - RegionID region_id = (RegionID)safeGet(typeid_cast(*args[0]).value); + auto region_id = static_cast(safeGet(typeid_cast(*args[0]).value)); const String & database_name = typeid_cast(*args[1]).name; const String & table_name = typeid_cast(*args[2]).name; auto table = MockTiDB::instance().getTableByName(database_name, table_name); @@ -49,7 +49,7 @@ void MockRaftCommand::dbgFuncRegionBatchSplit(Context & context, const ASTs & ar if (4 + handle_column_size * 4 != args.size()) throw Exception("Args not matched, should be: region-id1, database-name, table-name, start1, end1, start2, end2, region-id2", ErrorCodes::BAD_ARGUMENTS); - RegionID region_id2 = (RegionID)safeGet(typeid_cast(*args[args.size() - 1]).value); + auto region_id2 = static_cast(safeGet(typeid_cast(*args[args.size() - 1]).value)); auto table_id = table->id(); TiKVKey start_key1, start_key2, end_key1, end_key2; @@ -59,9 +59,17 @@ void MockRaftCommand::dbgFuncRegionBatchSplit(Context & context, const ASTs & ar std::vector start_keys2; std::vector end_keys1; std::vector end_keys2; + + std::unordered_map column_name_columns_index_map; + for (size_t i = 0; i < table_info.columns.size(); i++) + { + column_name_columns_index_map.emplace(table_info.columns[i].name, i); + } + for (size_t i = 0; i < handle_column_size; i++) { - auto & column_info = table_info.columns[table_info.getPrimaryIndexInfo().idx_cols[i].offset]; + auto idx = column_name_columns_index_map[table_info.getPrimaryIndexInfo().idx_cols[i].name]; + auto & column_info = table_info.columns[idx]; auto start_field1 = RegionBench::convertField(column_info, typeid_cast(*args[3 + i]).value); TiDB::DatumBumpy start_datum1 = TiDB::DatumBumpy(start_field1, column_info.tp); @@ -88,10 +96,10 @@ void MockRaftCommand::dbgFuncRegionBatchSplit(Context & context, const ASTs & ar } else { - HandleID start1 = (HandleID)safeGet(typeid_cast(*args[3]).value); - HandleID end1 = (HandleID)safeGet(typeid_cast(*args[4]).value); - HandleID start2 = (HandleID)safeGet(typeid_cast(*args[5]).value); - HandleID end2 = (HandleID)safeGet(typeid_cast(*args[6]).value); + auto start1 = static_cast(safeGet(typeid_cast(*args[3]).value)); + auto end1 = static_cast(safeGet(typeid_cast(*args[4]).value)); + auto start2 = static_cast(safeGet(typeid_cast(*args[5]).value)); + auto end2 = static_cast(safeGet(typeid_cast(*args[6]).value)); start_key1 = RecordKVFormat::genKey(table_id, start1); start_key2 = RecordKVFormat::genKey(table_id, start2); end_key1 = RecordKVFormat::genKey(table_id, end1); @@ -110,7 +118,7 @@ void MockRaftCommand::dbgFuncRegionBatchSplit(Context & context, const ASTs & ar request.set_cmd_type(raft_cmdpb::AdminCmdType::BatchSplit); raft_cmdpb::BatchSplitResponse * splits = response.mutable_splits(); { - auto region = splits->add_regions(); + auto * region = splits->add_regions(); region->set_id(region_id); region->set_start_key(start_key1); region->set_end_key(end_key1); @@ -118,7 +126,7 @@ void MockRaftCommand::dbgFuncRegionBatchSplit(Context & context, const ASTs & ar *region->mutable_region_epoch() = new_epoch; } { - auto region = splits->add_regions(); + auto * region = splits->add_regions(); region->set_id(region_id2); region->set_start_key(start_key2); region->set_end_key(end_key2); @@ -144,8 +152,8 @@ void MockRaftCommand::dbgFuncPrepareMerge(Context & context, const ASTs & args, throw Exception("Args not matched, should be: source-id1, target-id2", ErrorCodes::BAD_ARGUMENTS); } - RegionID region_id = (RegionID)safeGet(typeid_cast(*args[0]).value); - RegionID target_id = (RegionID)safeGet(typeid_cast(*args[1]).value); + auto region_id = static_cast(safeGet(typeid_cast(*args[0]).value)); + auto target_id = static_cast(safeGet(typeid_cast(*args[1]).value)); auto & tmt = context.getTMTContext(); auto & kvstore = tmt.getKVStore(); @@ -157,7 +165,7 @@ void MockRaftCommand::dbgFuncPrepareMerge(Context & context, const ASTs & args, { request.set_cmd_type(raft_cmdpb::AdminCmdType::PrepareMerge); - auto prepare_merge = request.mutable_prepare_merge(); + auto * prepare_merge = request.mutable_prepare_merge(); { auto min_index = region->appliedIndex(); prepare_merge->set_min_index(min_index); @@ -184,8 +192,8 @@ void MockRaftCommand::dbgFuncCommitMerge(Context & context, const ASTs & args, D throw Exception("Args not matched, should be: source-id1, current-id2", ErrorCodes::BAD_ARGUMENTS); } - RegionID source_id = (RegionID)safeGet(typeid_cast(*args[0]).value); - RegionID current_id = (RegionID)safeGet(typeid_cast(*args[1]).value); + auto source_id = static_cast(safeGet(typeid_cast(*args[0]).value)); + auto current_id = static_cast(safeGet(typeid_cast(*args[1]).value)); auto & tmt = context.getTMTContext(); auto & kvstore = tmt.getKVStore(); @@ -196,7 +204,7 @@ void MockRaftCommand::dbgFuncCommitMerge(Context & context, const ASTs & args, D { request.set_cmd_type(raft_cmdpb::AdminCmdType::CommitMerge); - auto commit_merge = request.mutable_commit_merge(); + auto * commit_merge = request.mutable_commit_merge(); { commit_merge->set_commit(source_region->appliedIndex()); *commit_merge->mutable_source() = source_region->getMetaRegion(); @@ -220,7 +228,7 @@ void MockRaftCommand::dbgFuncRollbackMerge(Context & context, const ASTs & args, throw Exception("Args not matched, should be: region-id", ErrorCodes::BAD_ARGUMENTS); } - RegionID region_id = (RegionID)safeGet(typeid_cast(*args[0]).value); + auto region_id = static_cast(safeGet(typeid_cast(*args[0]).value)); auto & tmt = context.getTMTContext(); auto & kvstore = tmt.getKVStore(); @@ -231,7 +239,7 @@ void MockRaftCommand::dbgFuncRollbackMerge(Context & context, const ASTs & args, { request.set_cmd_type(raft_cmdpb::AdminCmdType::RollbackMerge); - auto rollback_merge = request.mutable_rollback_merge(); + auto * rollback_merge = request.mutable_rollback_merge(); { auto merge_state = region->getMergeState(); rollback_merge->set_commit(merge_state.commit()); diff --git a/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp b/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp index 9d5b848ddea..b5d3f252d0a 100644 --- a/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp +++ b/dbms/src/Debug/dbgFuncMockRaftSnapshot.cpp @@ -68,6 +68,12 @@ RegionPtr GenDbgRegionSnapshotWithData(Context & context, const ASTs & args) size_t handle_column_size = is_common_handle ? table_info.getPrimaryIndexInfo().idx_cols.size() : 1; RegionPtr region; + std::unordered_map column_name_columns_index_map; + for (size_t i = 0; i < table_info.columns.size(); i++) + { + column_name_columns_index_map.emplace(table_info.columns[i].name, i); + } + if (!is_common_handle) { auto start = static_cast(safeGet(typeid_cast(*args[3]).value)); @@ -81,7 +87,8 @@ RegionPtr GenDbgRegionSnapshotWithData(Context & context, const ASTs & args) std::vector end_keys; for (size_t i = 0; i < handle_column_size; i++) { - auto & column_info = table_info.columns[table_info.getPrimaryIndexInfo().idx_cols[i].offset]; + auto idx = column_name_columns_index_map[table_info.getPrimaryIndexInfo().idx_cols[i].name]; + auto & column_info = table_info.columns[idx]; auto start_field = RegionBench::convertField(column_info, typeid_cast(*args[3 + i]).value); TiDB::DatumBumpy start_datum = TiDB::DatumBumpy(start_field, column_info.tp); start_keys.emplace_back(start_datum.field()); @@ -122,9 +129,9 @@ RegionPtr GenDbgRegionSnapshotWithData(Context & context, const ASTs & args) std::vector keys; // handle key for (size_t i = 0; i < table_info.getPrimaryIndexInfo().idx_cols.size(); i++) { - auto & idx_col = table_info.getPrimaryIndexInfo().idx_cols[i]; - auto & column_info = table_info.columns[idx_col.offset]; - auto start_field = RegionBench::convertField(column_info, fields[idx_col.offset]); + auto idx = column_name_columns_index_map[table_info.getPrimaryIndexInfo().idx_cols[i].name]; + auto & column_info = table_info.columns[idx]; + auto start_field = RegionBench::convertField(column_info, fields[idx]); TiDB::DatumBumpy start_datum = TiDB::DatumBumpy(start_field, column_info.tp); keys.emplace_back(start_datum.field()); } @@ -198,9 +205,16 @@ void MockRaftCommand::dbgFuncRegionSnapshot(Context & context, const ASTs & args // Get start key and end key form multiple column if it is clustered_index. std::vector start_keys; std::vector end_keys; + + std::unordered_map column_name_columns_index_map; + for (size_t i = 0; i < table_info.columns.size(); i++) + { + column_name_columns_index_map.emplace(table_info.columns[i].name, i); + } for (size_t i = 0; i < handle_column_size; i++) { - const auto & column_info = table_info.columns[table_info.getPrimaryIndexInfo().idx_cols[i].offset]; + auto idx = column_name_columns_index_map[table_info.getPrimaryIndexInfo().idx_cols[i].name]; + const auto & column_info = table_info.columns[idx]; auto start_field = RegionBench::convertField(column_info, typeid_cast(*args[1 + i]).value); TiDB::DatumBumpy start_datum = TiDB::DatumBumpy(start_field, column_info.tp); start_keys.emplace_back(start_datum.field()); diff --git a/dbms/src/Debug/dbgFuncRegion.cpp b/dbms/src/Debug/dbgFuncRegion.cpp index b2024eac1d8..f65a18b8fd0 100644 --- a/dbms/src/Debug/dbgFuncRegion.cpp +++ b/dbms/src/Debug/dbgFuncRegion.cpp @@ -61,9 +61,15 @@ void dbgFuncPutRegion(Context & context, const ASTs & args, DBGInvoker::Printer { std::vector start_keys; std::vector end_keys; + std::unordered_map column_name_columns_index_map; + for (size_t i = 0; i < table_info.columns.size(); i++) + { + column_name_columns_index_map.emplace(table_info.columns[i].name, i); + } for (size_t i = 0; i < handle_column_size; i++) { - const auto & column_info = table_info.columns[table_info.getPrimaryIndexInfo().idx_cols[i].offset]; + auto idx = column_name_columns_index_map[table_info.getPrimaryIndexInfo().idx_cols[i].name]; + const auto & column_info = table_info.columns[idx]; auto start_field = RegionBench::convertField(column_info, typeid_cast(*args[1 + i]).value); TiDB::DatumBumpy start_datum = TiDB::DatumBumpy(start_field, column_info.tp); start_keys.emplace_back(start_datum.field()); diff --git a/dbms/src/Debug/dbgTools.cpp b/dbms/src/Debug/dbgTools.cpp index 685b2563a3b..854d8a18bd5 100644 --- a/dbms/src/Debug/dbgTools.cpp +++ b/dbms/src/Debug/dbgTools.cpp @@ -310,7 +310,7 @@ void insert( // // Parse the fields in the inserted row std::vector fields; { - for (ASTs::const_iterator it = values_begin; it != values_end; ++it) + for (auto it = values_begin; it != values_end; ++it) { auto field = typeid_cast((*it).get())->value; fields.emplace_back(field); @@ -330,11 +330,18 @@ void insert( // if (table_info.is_common_handle) { std::vector keys; + + std::unordered_map column_name_columns_index_map; + for (size_t i = 0; i < table_info.columns.size(); i++) + { + column_name_columns_index_map.emplace(table_info.columns[i].name, i); + } + for (size_t i = 0; i < table_info.getPrimaryIndexInfo().idx_cols.size(); i++) { - const auto & idx_col = table_info.getPrimaryIndexInfo().idx_cols[i]; - const auto & column_info = table_info.columns[idx_col.offset]; - auto start_field = RegionBench::convertField(column_info, fields[idx_col.offset]); + const auto & col_idx = column_name_columns_index_map[table_info.getPrimaryIndexInfo().idx_cols[i].name]; + const auto & column_info = table_info.columns[col_idx]; + auto start_field = RegionBench::convertField(column_info, fields[col_idx]); TiDB::DatumBumpy start_datum = TiDB::DatumBumpy(start_field, column_info.tp); keys.emplace_back(start_datum.field()); } diff --git a/dbms/src/Storages/Transaction/DecodingStorageSchemaSnapshot.h b/dbms/src/Storages/Transaction/DecodingStorageSchemaSnapshot.h index e8e0610326c..b0cacefe6f4 100644 --- a/dbms/src/Storages/Transaction/DecodingStorageSchemaSnapshot.h +++ b/dbms/src/Storages/Transaction/DecodingStorageSchemaSnapshot.h @@ -77,10 +77,12 @@ struct DecodingStorageSchemaSnapshot , decoding_schema_version{decoding_schema_version_} { std::unordered_map column_lut; + std::unordered_map column_name_id_map; for (size_t i = 0; i < table_info_.columns.size(); i++) { const auto & ci = table_info_.columns[i]; column_lut.emplace(ci.id, i); + column_name_id_map.emplace(ci.name, ci.id); } for (size_t i = 0; i < column_defines->size(); i++) { @@ -88,7 +90,7 @@ struct DecodingStorageSchemaSnapshot sorted_column_id_with_pos.insert({cd.id, i}); if (cd.id != TiDBPkColumnID && cd.id != VersionColumnID && cd.id != DelMarkColumnID) { - auto & columns = table_info_.columns; + const auto & columns = table_info_.columns; column_infos.push_back(columns[column_lut.at(cd.id)]); } else @@ -100,10 +102,14 @@ struct DecodingStorageSchemaSnapshot // create pk related metadata if needed if (is_common_handle) { - const auto & primary_index_info = table_info_.getPrimaryIndexInfo(); - for (size_t i = 0; i < primary_index_info.idx_cols.size(); i++) + /// we will not update the IndexInfo except Rename DDL. + /// When the add column / drop column action happenes, the offset of each column may change + /// Thus, we should not use offset to get the column we want, + /// but use to compare the column name to get the column id. + const auto & primary_index_cols = table_info_.getPrimaryIndexInfo().idx_cols; + for (const auto & col : primary_index_cols) { - auto pk_column_id = table_info_.columns[primary_index_info.idx_cols[i].offset].id; + auto pk_column_id = column_name_id_map[col.name]; pk_column_ids.emplace_back(pk_column_id); pk_pos_map.emplace(pk_column_id, reinterpret_cast(std::numeric_limits::max())); } diff --git a/dbms/src/Storages/Transaction/RegionBlockReader.cpp b/dbms/src/Storages/Transaction/RegionBlockReader.cpp index a9384e4a14d..2ec690c467b 100644 --- a/dbms/src/Storages/Transaction/RegionBlockReader.cpp +++ b/dbms/src/Storages/Transaction/RegionBlockReader.cpp @@ -208,6 +208,8 @@ bool RegionBlockReader::readImpl(Block & block, const RegionDataReadInfoList & d } index++; } + block.checkNumberOfRows(); + return true; } diff --git a/dbms/src/Storages/Transaction/TiDB.cpp b/dbms/src/Storages/Transaction/TiDB.cpp index 15bf2a3fb58..dc7f1f3e348 100644 --- a/dbms/src/Storages/Transaction/TiDB.cpp +++ b/dbms/src/Storages/Transaction/TiDB.cpp @@ -631,8 +631,8 @@ catch (const Poco::Exception & e) /////////////////////// IndexColumnInfo::IndexColumnInfo(Poco::JSON::Object::Ptr json) - : offset(0) - , length(0) + : length(0) + , offset(0) { deserialize(json); } diff --git a/dbms/src/Storages/Transaction/TiDB.h b/dbms/src/Storages/Transaction/TiDB.h index f67bfb332c7..4c28a614857 100644 --- a/dbms/src/Storages/Transaction/TiDB.h +++ b/dbms/src/Storages/Transaction/TiDB.h @@ -179,7 +179,6 @@ struct ColumnInfo ColumnID id = -1; String name; - Int32 offset = -1; Poco::Dynamic::Var origin_default_value; Poco::Dynamic::Var default_value; Poco::Dynamic::Var default_bit_value; @@ -212,6 +211,12 @@ struct ColumnInfo static Int64 getTimeValue(const String &); static Int64 getYearValue(const String &); static UInt64 getBitValue(const String &); + +private: + /// please be very careful when you have to use offset, + /// because we never update offset when DDL action changes. + /// Thus, our offset will not exactly correspond the order of columns. + Int32 offset = -1; }; enum PartitionType @@ -298,8 +303,13 @@ struct IndexColumnInfo void deserialize(Poco::JSON::Object::Ptr json); String name; - Int32 offset; Int32 length; + +private: + /// please be very careful when you have to use offset, + /// because we never update offset when DDL action changes. + /// Thus, our offset will not exactly correspond the order of columns. + Int32 offset; }; struct IndexInfo { @@ -385,7 +395,12 @@ struct TableInfo bool isLogicalPartitionTable() const { return is_partition_table && belonging_table_id == DB::InvalidTableID && partition.enable; } - /// should not be called if is_common_handle = false + /// should not be called if is_common_handle = false. + /// when use IndexInfo, please avoid to use the offset info + /// the offset value may be wrong in some cases, + /// due to we will not update IndexInfo except RENAME DDL action, + /// but DDL like add column / drop column may change the offset of columns + /// Thus, please be very careful when you must have to use offset information !!!!! const IndexInfo & getPrimaryIndexInfo() const { return index_infos[0]; } IndexInfo & getPrimaryIndexInfo() { return index_infos[0]; } diff --git a/dbms/src/Storages/Transaction/TiKVRecordFormat.h b/dbms/src/Storages/Transaction/TiKVRecordFormat.h index c507616f6e9..10a7f7220e9 100644 --- a/dbms/src/Storages/Transaction/TiKVRecordFormat.h +++ b/dbms/src/Storages/Transaction/TiKVRecordFormat.h @@ -154,9 +154,16 @@ inline TiKVKey genKey(const TiDB::TableInfo & table_info, std::vector key memcpy(key.data() + 1, reinterpret_cast(&big_endian_table_id), 8); memcpy(key.data() + 1 + 8, RecordKVFormat::RECORD_PREFIX_SEP, 2); WriteBufferFromOwnString ss; + + std::unordered_map column_name_columns_index_map; + for (size_t i = 0; i < table_info.columns.size(); i++) + { + column_name_columns_index_map.emplace(table_info.columns[i].name, i); + } for (size_t i = 0; i < keys.size(); i++) { - DB::EncodeDatum(keys[i], table_info.columns[table_info.getPrimaryIndexInfo().idx_cols[i].offset].getCodecFlag(), ss); + auto idx = column_name_columns_index_map[table_info.getPrimaryIndexInfo().idx_cols[i].name]; + DB::EncodeDatum(keys[i], table_info.columns[idx].getCodecFlag(), ss); } return encodeAsTiKVKey(key + ss.releaseStr()); } diff --git a/dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h b/dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h index 20b395a9952..34e0d3d4104 100644 --- a/dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h +++ b/dbms/src/Storages/Transaction/tests/RowCodecTestUtils.h @@ -237,14 +237,14 @@ std::pair> getTableInfoAndFields(ColumnIDs handle_ { table_info.is_common_handle = true; TiDB::IndexInfo index_info; - for (size_t i = 0; i < handle_ids.size(); i++) + for (auto handle_id : handle_ids) { TiDB::IndexColumnInfo index_column_info; - for (size_t pos = 0; pos < table_info.columns.size(); pos++) + for (auto & column : table_info.columns) { - if (table_info.columns[pos].id == handle_ids[i]) + if (column.id == handle_id) { - index_column_info.offset = pos; + index_column_info.name = column.name; break; } } diff --git a/dbms/src/Storages/Transaction/tests/bench_region_block_reader.cpp b/dbms/src/Storages/Transaction/tests/bench_region_block_reader.cpp new file mode 100644 index 00000000000..05ab637de7f --- /dev/null +++ b/dbms/src/Storages/Transaction/tests/bench_region_block_reader.cpp @@ -0,0 +1,171 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include + +#include "RowCodecTestUtils.h" + +using TableInfo = TiDB::TableInfo; +namespace DB::tests +{ +using ColumnIDs = std::vector; +class RegionBlockReaderBenchTest : public benchmark::Fixture +{ +protected: + Int64 handle_value = 100; + UInt8 del_mark_value = 0; + UInt64 version_value = 100; + + RegionDataReadInfoList data_list_read; + std::unordered_map fields_map; + + enum RowEncodeVersion + { + RowV1, + RowV2 + }; + +protected: + void SetUp(const benchmark::State & /*state*/) override + { + data_list_read.clear(); + fields_map.clear(); + } + + void encodeColumns(TableInfo & table_info, std::vector & fields, RowEncodeVersion row_version, size_t num_rows) + { + // for later check + std::unordered_map column_name_columns_index_map; + for (size_t i = 0; i < table_info.columns.size(); i++) + { + fields_map.emplace(table_info.columns[i].id, fields[i]); + column_name_columns_index_map.emplace(table_info.columns[i].name, i); + } + + std::vector value_fields; + std::vector pk_fields; + for (size_t i = 0; i < table_info.columns.size(); i++) + { + if (!table_info.columns[i].hasPriKeyFlag()) + value_fields.emplace_back(fields[i]); + else + pk_fields.emplace_back(fields[i]); + } + + // create PK + WriteBufferFromOwnString pk_buf; + if (table_info.is_common_handle) + { + auto & primary_index_info = table_info.getPrimaryIndexInfo(); + for (size_t i = 0; i < primary_index_info.idx_cols.size(); i++) + { + auto idx = column_name_columns_index_map[primary_index_info.idx_cols[i].name]; + EncodeDatum(pk_fields[i], table_info.columns[idx].getCodecFlag(), pk_buf); + } + } + else + { + DB::EncodeInt64(handle_value, pk_buf); + } + RawTiDBPK pk{std::make_shared(pk_buf.releaseStr())}; + // create value + WriteBufferFromOwnString value_buf; + if (row_version == RowEncodeVersion::RowV1) + { + encodeRowV1(table_info, value_fields, value_buf); + } + else if (row_version == RowEncodeVersion::RowV2) + { + encodeRowV2(table_info, value_fields, value_buf); + } + else + { + throw Exception("Unknown row format " + std::to_string(row_version), ErrorCodes::LOGICAL_ERROR); + } + auto row_value = std::make_shared(std::move(value_buf.str())); + for (size_t i = 0; i < num_rows; i++) + data_list_read.emplace_back(pk, del_mark_value, version_value, row_value); + } + + bool decodeColumns(DecodingStorageSchemaSnapshotConstPtr decoding_schema, bool force_decode) const + { + RegionBlockReader reader{decoding_schema}; + Block block = createBlockSortByColumnID(decoding_schema); + return reader.read(block, data_list_read, force_decode); + } + + std::pair> getNormalTableInfoFields(const ColumnIDs & handle_ids, bool is_common_handle) const + { + return getTableInfoAndFields( + handle_ids, + is_common_handle, + ColumnIDValue(2, handle_value), + ColumnIDValue(3, std::numeric_limits::max()), + ColumnIDValue(4, std::numeric_limits::min()), + ColumnIDValue(9, String("aaa")), + ColumnIDValue(10, DecimalField(ToDecimal(12345678910ULL, 4), 4)), + ColumnIDValueNull(11)); + } +}; + +BENCHMARK_DEFINE_F(RegionBlockReaderBenchTest, CommonHandle) +(benchmark::State & state) +{ + size_t num_rows = state.range(0); + auto [table_info, fields] = getNormalTableInfoFields({2, 3, 4}, true); + encodeColumns(table_info, fields, RowEncodeVersion::RowV2, num_rows); + auto decoding_schema = getDecodingStorageSchemaSnapshot(table_info); + for (auto _ : state) + { + decodeColumns(decoding_schema, true); + } +} + + +BENCHMARK_DEFINE_F(RegionBlockReaderBenchTest, PKIsNotHandle) +(benchmark::State & state) +{ + size_t num_rows = state.range(0); + auto [table_info, fields] = getNormalTableInfoFields({EXTRA_HANDLE_COLUMN_ID}, false); + encodeColumns(table_info, fields, RowEncodeVersion::RowV2, num_rows); + auto decoding_schema = getDecodingStorageSchemaSnapshot(table_info); + for (auto _ : state) + { + decodeColumns(decoding_schema, true); + } +} + +BENCHMARK_DEFINE_F(RegionBlockReaderBenchTest, PKIsHandle) +(benchmark::State & state) +{ + size_t num_rows = state.range(0); + auto [table_info, fields] = getNormalTableInfoFields({2}, false); + encodeColumns(table_info, fields, RowEncodeVersion::RowV2, num_rows); + auto decoding_schema = getDecodingStorageSchemaSnapshot(table_info); + for (auto _ : state) + { + decodeColumns(decoding_schema, true); + } +} + +constexpr size_t num_iterations_test = 1000; + +BENCHMARK_REGISTER_F(RegionBlockReaderBenchTest, PKIsHandle)->Iterations(num_iterations_test)->Arg(1)->Arg(10)->Arg(100); +BENCHMARK_REGISTER_F(RegionBlockReaderBenchTest, CommonHandle)->Iterations(num_iterations_test)->Arg(1)->Arg(10)->Arg(100); +BENCHMARK_REGISTER_F(RegionBlockReaderBenchTest, PKIsNotHandle)->Iterations(num_iterations_test)->Arg(1)->Arg(10)->Arg(100); + +} // namespace DB::tests diff --git a/dbms/src/Storages/Transaction/tests/gtest_decoding_storage_schema_snapshot.cpp b/dbms/src/Storages/Transaction/tests/gtest_decoding_storage_schema_snapshot.cpp new file mode 100644 index 00000000000..1de9809ecad --- /dev/null +++ b/dbms/src/Storages/Transaction/tests/gtest_decoding_storage_schema_snapshot.cpp @@ -0,0 +1,65 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include + +#include "RowCodecTestUtils.h" + +namespace DB::tests +{ +static TableInfo getTableInfoByJson(const String & json_table_info) +{ + return TableInfo(json_table_info); +} +TEST(DecodingStorageSchemaSnapshotTest, CheckPKInfosUnderClusteredIndex) +{ + // table with column [A,B,C,D], primary keys [A,C] + const String json_table_info = R"json({"id":75,"name":{"O":"test","L":"test"},"charset":"utf8mb4","collate":"utf8mb4_bin","cols":[{"id":1,"name":{"O":"A","L":"a"},"offset":0,"origin_default":null,"origin_default_bit":null,"default":null,"default_bit":null,"default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":4099,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2},{"id":2,"name":{"O":"B","L":"b"},"offset":1,"origin_default":null,"origin_default_bit":null,"default":null,"default_bit":null,"default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":15,"Flag":0,"Flen":20,"Decimal":0,"Charset":"utf8mb4","Collate":"utf8mb4_bin","Elems":null},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2},{"id":3,"name":{"O":"C","L":"c"},"offset":2,"origin_default":null,"origin_default_bit":null,"default":null,"default_bit":null,"default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":4099,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2},{"id":4,"name":{"O":"D","L":"d"},"offset":3,"origin_default":null,"origin_default_bit":null,"default":null,"default_bit":null,"default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2}],"index_info":[{"id":1,"idx_name":{"O":"PRIMARY","L":"primary"},"tbl_name":{"O":"","L":""},"idx_cols":[{"name":{"O":"A","L":"a"},"offset":0,"length":-1},{"name":{"O":"C","L":"c"},"offset":2,"length":-1}],"state":5,"comment":"","index_type":1,"is_unique":true,"is_primary":true,"is_invisible":false,"is_global":false}],"constraint_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"is_common_handle":true,"common_handle_version":1,"comment":"","auto_inc_id":0,"auto_id_cache":0,"auto_rand_id":0,"max_col_id":4,"max_idx_id":1,"max_cst_id":0,"update_timestamp":434039123413303302,"ShardRowIDBits":0,"max_shard_row_id_bits":0,"auto_random_bits":0,"pre_split_regions":0,"partition":null,"compression":"","view":null,"sequence":null,"Lock":null,"version":4,"tiflash_replica":{"Count":1,"LocationLabels":[],"Available":false,"AvailablePartitionIDs":null},"is_columnar":false,"temp_table_type":0,"cache_table_status":0,"policy_ref_info":null,"stats_options":null})json"; + auto table_info = getTableInfoByJson(json_table_info); + auto decoding_schema = getDecodingStorageSchemaSnapshot(table_info); + + //check decoding_schema->pk_column_ids infos + ASSERT_EQ(decoding_schema->pk_column_ids.size(), 2); + ASSERT_EQ(decoding_schema->pk_column_ids[0], 1); + ASSERT_EQ(decoding_schema->pk_column_ids[1], 3); + + //check decoding_schema->pk_pos_map infos + ASSERT_EQ(decoding_schema->pk_column_ids.size(), decoding_schema->pk_pos_map.size()); + // there are three hidden column in the decoded block, so the position of A,C is 3,5 + ASSERT_EQ(decoding_schema->pk_pos_map.at(decoding_schema->pk_column_ids[0]), 3); + ASSERT_EQ(decoding_schema->pk_pos_map.at(decoding_schema->pk_column_ids[1]), 5); +} + +TEST(DecodingStorageSchemaSnapshotTest, CheckPKInfosUnderClusteredIndexAfterDropColumn) +{ + // drop column B for [A,B,C,D]; table with column [A,C,D], primary keys [A,C] + const String json_table_info = R"json({"id":75,"name":{"O":"test","L":"test"},"charset":"utf8mb4","collate":"utf8mb4_bin","cols":[{"id":1,"name":{"O":"A","L":"a"},"offset":0,"origin_default":null,"origin_default_bit":null,"default":null,"default_bit":null,"default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":4099,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2},{"id":3,"name":{"O":"C","L":"c"},"offset":2,"origin_default":null,"origin_default_bit":null,"default":null,"default_bit":null,"default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":4099,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2},{"id":4,"name":{"O":"D","L":"d"},"offset":3,"origin_default":null,"origin_default_bit":null,"default":null,"default_bit":null,"default_is_expr":false,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","hidden":false,"change_state_info":null,"version":2}],"index_info":[{"id":1,"idx_name":{"O":"PRIMARY","L":"primary"},"tbl_name":{"O":"","L":""},"idx_cols":[{"name":{"O":"A","L":"a"},"offset":0,"length":-1},{"name":{"O":"C","L":"c"},"offset":2,"length":-1}],"state":5,"comment":"","index_type":1,"is_unique":true,"is_primary":true,"is_invisible":false,"is_global":false}],"constraint_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"is_common_handle":true,"common_handle_version":1,"comment":"","auto_inc_id":0,"auto_id_cache":0,"auto_rand_id":0,"max_col_id":4,"max_idx_id":1,"max_cst_id":0,"update_timestamp":434039123413303302,"ShardRowIDBits":0,"max_shard_row_id_bits":0,"auto_random_bits":0,"pre_split_regions":0,"partition":null,"compression":"","view":null,"sequence":null,"Lock":null,"version":4,"tiflash_replica":{"Count":1,"LocationLabels":[],"Available":false,"AvailablePartitionIDs":null},"is_columnar":false,"temp_table_type":0,"cache_table_status":0,"policy_ref_info":null,"stats_options":null})json"; + auto table_info = getTableInfoByJson(json_table_info); + auto decoding_schema = getDecodingStorageSchemaSnapshot(table_info); + + //check decoding_schema->pk_column_ids infos + ASSERT_EQ(decoding_schema->pk_column_ids.size(), 2); + ASSERT_EQ(decoding_schema->pk_column_ids[0], 1); + ASSERT_EQ(decoding_schema->pk_column_ids[1], 3); + + //check decoding_schema->pk_pos_map infos + ASSERT_EQ(decoding_schema->pk_column_ids.size(), decoding_schema->pk_pos_map.size()); + // there are three hidden column in the decoded block, so the position of A,C is 3,4 + ASSERT_EQ(decoding_schema->pk_pos_map.at(decoding_schema->pk_column_ids[0]), 3); + ASSERT_EQ(decoding_schema->pk_pos_map.at(decoding_schema->pk_column_ids[1]), 4); +} + +} // namespace DB::tests diff --git a/dbms/src/Storages/Transaction/tests/gtest_region_block_reader.cpp b/dbms/src/Storages/Transaction/tests/gtest_region_block_reader.cpp index 6a883230854..d08b4dd3738 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_region_block_reader.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_region_block_reader.cpp @@ -26,13 +26,13 @@ using ColumnIDs = std::vector; class RegionBlockReaderTestFixture : public ::testing::Test { protected: - Int64 handle_value_ = 100; - UInt8 del_mark_value_ = 0; - UInt64 version_value_ = 100; - size_t rows_ = 3; + Int64 handle_value = 100; + UInt8 del_mark_value = 0; + UInt64 version_value = 100; + size_t rows = 3; - RegionDataReadInfoList data_list_read_; - std::unordered_map fields_map_; + RegionDataReadInfoList data_list_read; + std::unordered_map fields_map; enum RowEncodeVersion { @@ -43,8 +43,8 @@ class RegionBlockReaderTestFixture : public ::testing::Test protected: void SetUp() override { - data_list_read_.clear(); - fields_map_.clear(); + data_list_read.clear(); + fields_map.clear(); } void TearDown() override {} @@ -52,8 +52,12 @@ class RegionBlockReaderTestFixture : public ::testing::Test void encodeColumns(TableInfo & table_info, std::vector & fields, RowEncodeVersion row_version) { // for later check + std::unordered_map column_name_columns_index_map; for (size_t i = 0; i < table_info.columns.size(); i++) - fields_map_.emplace(table_info.columns[i].id, fields[i]); + { + fields_map.emplace(table_info.columns[i].id, fields[i]); + column_name_columns_index_map.emplace(table_info.columns[i].name, i); + } std::vector value_fields; std::vector pk_fields; @@ -72,13 +76,13 @@ class RegionBlockReaderTestFixture : public ::testing::Test auto & primary_index_info = table_info.getPrimaryIndexInfo(); for (size_t i = 0; i < primary_index_info.idx_cols.size(); i++) { - size_t pk_offset = primary_index_info.idx_cols[i].offset; - EncodeDatum(pk_fields[i], table_info.columns[pk_offset].getCodecFlag(), pk_buf); + auto idx = column_name_columns_index_map[primary_index_info.idx_cols[i].name]; + EncodeDatum(pk_fields[i], table_info.columns[idx].getCodecFlag(), pk_buf); } } else { - DB::EncodeInt64(handle_value_, pk_buf); + DB::EncodeInt64(handle_value, pk_buf); } RawTiDBPK pk{std::make_shared(pk_buf.releaseStr())}; // create value @@ -96,44 +100,44 @@ class RegionBlockReaderTestFixture : public ::testing::Test throw Exception("Unknown row format " + std::to_string(row_version), ErrorCodes::LOGICAL_ERROR); } auto row_value = std::make_shared(std::move(value_buf.str())); - for (size_t i = 0; i < rows_; i++) - data_list_read_.emplace_back(pk, del_mark_value_, version_value_, row_value); + for (size_t i = 0; i < rows; i++) + data_list_read.emplace_back(pk, del_mark_value, version_value, row_value); } void checkBlock(DecodingStorageSchemaSnapshotConstPtr decoding_schema, const Block & block) const { ASSERT_EQ(block.columns(), decoding_schema->column_defines->size()); - for (size_t row = 0; row < rows_; row++) + for (size_t row = 0; row < rows; row++) { for (size_t pos = 0; pos < block.columns(); pos++) { - auto & column_element = block.getByPosition(pos); + const auto & column_element = block.getByPosition(pos); if (row == 0) { - ASSERT_EQ(column_element.column->size(), rows_); + ASSERT_EQ(column_element.column->size(), rows); } if (column_element.name == EXTRA_HANDLE_COLUMN_NAME) { if (decoding_schema->is_common_handle) { - ASSERT_EQ((*column_element.column)[row], Field(*std::get<0>(data_list_read_[row]))); + ASSERT_EQ((*column_element.column)[row], Field(*std::get<0>(data_list_read[row]))); } else { - ASSERT_EQ((*column_element.column)[row], Field(handle_value_)); + ASSERT_EQ((*column_element.column)[row], Field(handle_value)); } } else if (column_element.name == VERSION_COLUMN_NAME) { - ASSERT_EQ((*column_element.column)[row], Field(version_value_)); + ASSERT_EQ((*column_element.column)[row], Field(version_value)); } else if (column_element.name == TAG_COLUMN_NAME) { - ASSERT_EQ((*column_element.column)[row], Field(NearestFieldType::Type(del_mark_value_))); + ASSERT_EQ((*column_element.column)[row], Field(NearestFieldType::Type(del_mark_value))); } else { - ASSERT_EQ((*column_element.column)[row], fields_map_.at(column_element.column_id)); + ASSERT_EQ((*column_element.column)[row], fields_map.at(column_element.column_id)); } } } @@ -143,7 +147,7 @@ class RegionBlockReaderTestFixture : public ::testing::Test { RegionBlockReader reader{decoding_schema}; Block block = createBlockSortByColumnID(decoding_schema); - if (!reader.read(block, data_list_read_, force_decode)) + if (!reader.read(block, data_list_read, force_decode)) return false; checkBlock(decoding_schema, block); @@ -155,7 +159,7 @@ class RegionBlockReaderTestFixture : public ::testing::Test return getTableInfoAndFields( handle_ids, is_common_handle, - ColumnIDValue(2, handle_value_), + ColumnIDValue(2, handle_value), ColumnIDValue(3, std::numeric_limits::max()), ColumnIDValue(4, std::numeric_limits::min()), ColumnIDValue(9, String("aaa")), @@ -170,7 +174,7 @@ class RegionBlockReaderTestFixture : public ::testing::Test handle_ids, is_common_handle, ColumnIDValue(1, String("")), - ColumnIDValue(2, handle_value_), + ColumnIDValue(2, handle_value), ColumnIDValue(3, std::numeric_limits::max()), ColumnIDValue(4, std::numeric_limits::min()), ColumnIDValue(8, String("")), @@ -182,12 +186,12 @@ class RegionBlockReaderTestFixture : public ::testing::Test // add default value for missing column std::vector missing_column_ids{1, 8, 13}; String missing_column_default_value = String("default"); - for (size_t i = 0; i < table_info.columns.size(); i++) + for (auto & column : table_info.columns) { - if (std::find(missing_column_ids.begin(), missing_column_ids.end(), table_info.columns[i].id) != missing_column_ids.end()) + if (std::find(missing_column_ids.begin(), missing_column_ids.end(), column.id) != missing_column_ids.end()) { - table_info.columns[i].origin_default_value = missing_column_default_value; - fields_map_.emplace(table_info.columns[i].id, Field(missing_column_default_value)); + column.origin_default_value = missing_column_default_value; + fields_map.emplace(column.id, Field(missing_column_default_value)); } } return table_info; @@ -199,7 +203,7 @@ class RegionBlockReaderTestFixture : public ::testing::Test std::tie(table_info, std::ignore) = getTableInfoAndFields( handle_ids, is_common_handle, - ColumnIDValue(2, handle_value_), + ColumnIDValue(2, handle_value), ColumnIDValue(4, std::numeric_limits::min()), ColumnIDValue(9, String("aaa")), ColumnIDValue(10, DecimalField(ToDecimal(12345678910ULL, 4), 4))); @@ -212,7 +216,7 @@ class RegionBlockReaderTestFixture : public ::testing::Test std::tie(table_info, std::ignore) = getTableInfoAndFields( handle_ids, is_common_handle, - ColumnIDValue(2, handle_value_), + ColumnIDValue(2, handle_value), ColumnIDValue(3, std::numeric_limits::max()), ColumnIDValue(4, std::numeric_limits::min()), ColumnIDValue(9, String("aaa")), @@ -227,7 +231,7 @@ class RegionBlockReaderTestFixture : public ::testing::Test std::tie(table_info, std::ignore) = getTableInfoAndFields( handle_ids, is_common_handle, - ColumnIDValue(2, handle_value_), + ColumnIDValue(2, handle_value), ColumnIDValue(3, std::numeric_limits::max()), ColumnIDValue(4, std::numeric_limits::min()), ColumnIDValue(9, String("aaa")), diff --git a/tests/fullstack-test-dt/clustered_index/ddl.test b/tests/fullstack-test-dt/clustered_index/ddl.test index 8abe450c11a..6c4925c9619 100644 --- a/tests/fullstack-test-dt/clustered_index/ddl.test +++ b/tests/fullstack-test-dt/clustered_index/ddl.test @@ -66,3 +66,89 @@ mysql> set session tidb_isolation_read_engines='tiflash'; select * from test.t_2 mysql> drop table test.t_1; mysql> drop table test.t_2; + +### about issue 5154 to check whether add column/drop column will effect the cluster index decode +### drop the column between two columns that are cluster index columns + +mysql> drop table if exists test.t_3; +mysql> create table test.t_3 (A int, B varchar(20), C int, D int, PRIMARY KEY(A,C) CLUSTERED); +mysql> insert into test.t_3 values (1,'1',1,1),(2,'2',2,2); + +mysql> alter table test.t_3 set tiflash replica 1; + +func> wait_table test t_3 + +mysql> set session tidb_isolation_read_engines='tiflash';select * from test.t_3; ++---+---+---+---+ +| A | B | C | D | ++---+---+---+---+ +| 1 | 1 | 1 | 1 | +| 2 | 2 | 2 | 2 | ++---+---+---+---+ + +mysql> alter table test.t_3 drop column B; + +mysql> set session tidb_isolation_read_engines='tiflash';select * from test.t_3; ++---+---+---+ +| A | C | D | ++---+---+---+ +| 1 | 1 | 1 | +| 2 | 2 | 2 | ++---+---+---+ + +# insert some rows +mysql> insert into test.t_3 values (3,3,3),(4,4,4); + +mysql> set session tidb_isolation_read_engines='tiflash';select * from test.t_3; ++---+---+---+ +| A | C | D | ++---+---+---+ +| 1 | 1 | 1 | +| 2 | 2 | 2 | +| 3 | 3 | 3 | +| 4 | 4 | 4 | ++---+---+---+ + +mysql> drop table test.t_3; + +### add the column between two columns that are cluster index columns +mysql> drop table if exists test.t_4 +mysql> create table test.t_4 (A int, B varchar(20), C int, D int, PRIMARY KEY(A,C) CLUSTERED); + +mysql> insert into test.t_4 values (1,'1',1,1),(2,'2',2,2); + +mysql> alter table test.t_4 set tiflash replica 1; + +func> wait_table test t_4 + +mysql> set session tidb_isolation_read_engines='tiflash';select * from test.t_4; ++---+---+---+---+ +| A | B | C | D | ++---+---+---+---+ +| 1 | 1 | 1 | 1 | +| 2 | 2 | 2 | 2 | ++---+---+---+---+ + +mysql> alter table test.t_4 Add column E int after B; + +mysql> set session tidb_isolation_read_engines='tiflash';select * from test.t_4; ++---+---+------+---+---+ +| A | B | E | C | D | ++---+---+------+---+---+ +| 1 | 1 | NULL | 1 | 1 | +| 2 | 2 | NULL | 2 | 2 | ++---+---+------+---+---+ + +mysql> insert into test.t_4 values (3,'3',3,3,3),(4,'4',4,4,4); + +mysql> set session tidb_isolation_read_engines='tiflash';select * from test.t_4; ++---+---+------+------+------+ +| A | B | E | C | D | ++---+---+------+------+------+ +| 1 | 1 | NULL | 1 | 1 | +| 2 | 2 | NULL | 2 | 2 | +| 3 | 3 | 3 | 3 | 3 | +| 4 | 4 | 4 | 4 | 4 | ++---+---+------+------+------+ + +mysql> drop table test.t_4; \ No newline at end of file From 8a5dc2963af649fc02d15a4034106a50aa9c093e Mon Sep 17 00:00:00 2001 From: yibin Date: Wed, 22 Jun 2022 17:50:37 +0800 Subject: [PATCH 056/104] Add random failpoint in critical paths (#4876) close pingcap/tiflash#4807 --- dbms/src/Common/FailPoint.cpp | 65 ++++++++++++++++++- dbms/src/Common/FailPoint.h | 20 +++++- dbms/src/Common/wrapInvocable.h | 1 - .../DataStreams/SharedQueryBlockInputStream.h | 7 ++ dbms/src/DataStreams/SizeLimits.cpp | 23 +++++-- dbms/src/Flash/EstablishCall.cpp | 7 ++ dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 15 ++++- dbms/src/Flash/Mpp/MPPTask.cpp | 11 +++- dbms/src/Flash/Mpp/MPPTaskManager.cpp | 7 ++ dbms/src/Flash/Mpp/MPPTunnel.cpp | 2 + dbms/src/Flash/Mpp/MinTSOScheduler.cpp | 10 ++- dbms/src/Interpreters/Aggregator.cpp | 9 +++ dbms/src/Interpreters/Join.cpp | 12 +++- dbms/src/Interpreters/executeQuery.cpp | 7 +- dbms/src/Server/Server.cpp | 2 + 15 files changed, 181 insertions(+), 17 deletions(-) diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index 10d0a558a50..1dff46c273b 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -12,7 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include +#include +#include +#include +#include +#include #include #include @@ -21,7 +27,6 @@ namespace DB { std::unordered_map> FailPointHelper::fail_point_wait_channels; - #define APPLY_FOR_FAILPOINTS_ONCE(M) \ M(exception_between_drop_meta_and_data) \ M(exception_between_alter_data_and_meta) \ @@ -109,6 +114,22 @@ std::unordered_map> FailPointHelper::f M(pause_query_init) +#define APPLY_FOR_RANDOM_FAILPOINTS(M) \ + M(random_tunnel_wait_timeout_failpoint) \ + M(random_tunnel_init_rpc_failure_failpoint) \ + M(random_receiver_sync_msg_push_failure_failpoint) \ + M(random_receiver_async_msg_push_failure_failpoint) \ + M(random_limit_check_failpoint) \ + M(random_join_build_failpoint) \ + M(random_join_prob_failpoint) \ + M(random_aggregate_create_state_failpoint) \ + M(random_aggregate_merge_failpoint) \ + M(random_sharedquery_failpoint) \ + M(random_interpreter_failpoint) \ + M(random_task_lifecycle_failpoint) \ + M(random_task_manager_find_task_failure_failpoint) \ + M(random_min_tso_scheduler_failpoint) + namespace FailPoints { #define M(NAME) extern const char(NAME)[] = #NAME ""; @@ -116,6 +137,7 @@ APPLY_FOR_FAILPOINTS_ONCE(M) APPLY_FOR_FAILPOINTS(M) APPLY_FOR_PAUSEABLE_FAILPOINTS_ONCE(M) APPLY_FOR_PAUSEABLE_FAILPOINTS(M) +APPLY_FOR_RANDOM_FAILPOINTS(M) #undef M } // namespace FailPoints @@ -179,7 +201,7 @@ void FailPointHelper::enableFailPoint(const String & fail_point_name) #undef M #undef SUB_M - throw Exception("Cannot find fail point " + fail_point_name, ErrorCodes::FAIL_POINT_ERROR); + throw Exception(fmt::format("Cannot find fail point {}", fail_point_name), ErrorCodes::FAIL_POINT_ERROR); } void FailPointHelper::disableFailPoint(const String & fail_point_name) @@ -204,6 +226,41 @@ void FailPointHelper::wait(const String & fail_point_name) ptr->wait(); } } + +void FailPointHelper::initRandomFailPoints(Poco::Util::LayeredConfiguration & config, Poco::Logger * log) +{ + String random_fail_point_cfg = config.getString("flash.random_fail_points", ""); + if (random_fail_point_cfg.empty()) + return; + + Poco::StringTokenizer string_tokens(random_fail_point_cfg, ","); + for (const auto & string_token : string_tokens) + { + Poco::StringTokenizer pair_tokens(string_token, "-"); + RUNTIME_ASSERT((pair_tokens.count() == 2), log, "RandomFailPoints config should be FailPointA-RatioA,FailPointB-RatioB,... format"); + double rate = atof(pair_tokens[1].c_str()); //NOLINT(cert-err34-c): check conversion error manually + RUNTIME_ASSERT((0 <= rate && rate <= 1.0), log, "RandomFailPoint trigger rate should in [0,1], while {}", rate); + enableRandomFailPoint(pair_tokens[0], rate); + } + LOG_FMT_INFO(log, "Enable RandomFailPoints: {}", random_fail_point_cfg); +} + +void FailPointHelper::enableRandomFailPoint(const String & fail_point_name, double rate) +{ +#define SUB_M(NAME) \ + if (fail_point_name == FailPoints::NAME) \ + { \ + fiu_enable_random(FailPoints::NAME, 1, nullptr, 0, rate); \ + return; \ + } + +#define M(NAME) SUB_M(NAME) + APPLY_FOR_RANDOM_FAILPOINTS(M) +#undef M +#undef SUB_M + + throw Exception(fmt::format("Cannot find fail point {}", fail_point_name), ErrorCodes::FAIL_POINT_ERROR); +} #else class FailPointChannel { @@ -214,6 +271,10 @@ void FailPointHelper::enableFailPoint(const String &) {} void FailPointHelper::disableFailPoint(const String &) {} void FailPointHelper::wait(const String &) {} + +void FailPointHelper::initRandomFailPoints(Poco::Util::LayeredConfiguration & config, Poco::Logger * log) {} + +void FailPointHelper::enableRandomFailPoint(const String & fail_point_name, double rate) {} #endif } // namespace DB diff --git a/dbms/src/Common/FailPoint.h b/dbms/src/Common/FailPoint.h index 2cf40ad55e4..31df2dbdcd2 100644 --- a/dbms/src/Common/FailPoint.h +++ b/dbms/src/Common/FailPoint.h @@ -21,6 +21,15 @@ #include +namespace Poco +{ +class Logger; +namespace Util +{ +class LayeredConfiguration; +} +} // namespace Poco + namespace DB { namespace ErrorCodes @@ -35,7 +44,6 @@ extern const int FAIL_POINT_ERROR; // When `fail_point` is enabled, wait till it is disabled #define FAIL_POINT_PAUSE(fail_point) fiu_do_on(fail_point, FailPointHelper::wait(fail_point);) - class FailPointChannel; class FailPointHelper { @@ -46,6 +54,16 @@ class FailPointHelper static void wait(const String & fail_point_name); + /* + * For Server RandomFailPoint test usage. When FIU_ENABLE is defined, this function does the following work: + * 1. Return if TiFlash config has empty flash.random_fail_points cfg + * 2. Parse flash.random_fail_points, which expect to has "FailPointA-RatioA,FailPointB-RatioB,..." format + * 3. Call enableRandomFailPoint method with parsed FailPointName and Rate + */ + static void initRandomFailPoints(Poco::Util::LayeredConfiguration & config, Poco::Logger * log); + + static void enableRandomFailPoint(const String & fail_point_name, double rate); + private: static std::unordered_map> fail_point_wait_channels; }; diff --git a/dbms/src/Common/wrapInvocable.h b/dbms/src/Common/wrapInvocable.h index d6cee519835..1c93bb3e782 100644 --- a/dbms/src/Common/wrapInvocable.h +++ b/dbms/src/Common/wrapInvocable.h @@ -35,7 +35,6 @@ inline auto wrapInvocable(bool propagate_memory_tracker, Func && func, Args &&.. // run the task with the parameters provided return std::apply(std::move(func), std::move(args)); }; - return capture; } } // namespace DB diff --git a/dbms/src/DataStreams/SharedQueryBlockInputStream.h b/dbms/src/DataStreams/SharedQueryBlockInputStream.h index e7cece67f0b..d7c0707b5aa 100644 --- a/dbms/src/DataStreams/SharedQueryBlockInputStream.h +++ b/dbms/src/DataStreams/SharedQueryBlockInputStream.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include #include @@ -24,6 +25,11 @@ namespace DB { +namespace FailPoints +{ +extern const char random_sharedquery_failpoint[]; +} // namespace FailPoints + /** This block input stream is used by SharedQuery. * It enable multiple threads read from one stream. */ @@ -136,6 +142,7 @@ class SharedQueryBlockInputStream : public IProfilingBlockInputStream in->readPrefix(); while (true) { + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_sharedquery_failpoint); Block block = in->read(); // in is finished or queue is canceled if (!block || !queue.push(block)) diff --git a/dbms/src/DataStreams/SizeLimits.cpp b/dbms/src/DataStreams/SizeLimits.cpp index 7dd5e1524ba..4d1bfaae997 100644 --- a/dbms/src/DataStreams/SizeLimits.cpp +++ b/dbms/src/DataStreams/SizeLimits.cpp @@ -12,22 +12,30 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include -#include #include -#include +#include +#include +#include +#include namespace DB { +namespace FailPoints +{ +extern const char random_limit_check_failpoint[]; +} // namespace FailPoints bool SizeLimits::check(UInt64 rows, UInt64 bytes, const char * what, int exception_code) const { - if (max_rows && rows > max_rows) + bool rows_exceed_limit = max_rows && rows > max_rows; + fiu_do_on(FailPoints::random_limit_check_failpoint, rows_exceed_limit = true;); + if (rows_exceed_limit) { if (overflow_mode == OverflowMode::THROW) throw Exception("Limit for " + std::string(what) + " exceeded, max rows: " + formatReadableQuantity(max_rows) - + ", current rows: " + formatReadableQuantity(rows), exception_code); + + ", current rows: " + formatReadableQuantity(rows), + exception_code); else return false; } @@ -36,7 +44,8 @@ bool SizeLimits::check(UInt64 rows, UInt64 bytes, const char * what, int excepti { if (overflow_mode == OverflowMode::THROW) throw Exception("Limit for " + std::string(what) + " exceeded, max bytes: " + formatReadableSizeWithBinarySuffix(max_bytes) - + ", current bytes: " + formatReadableSizeWithBinarySuffix(bytes), exception_code); + + ", current bytes: " + formatReadableSizeWithBinarySuffix(bytes), + exception_code); else return false; } @@ -44,4 +53,4 @@ bool SizeLimits::check(UInt64 rows, UInt64 bytes, const char * what, int excepti return true; } -} +} // namespace DB diff --git a/dbms/src/Flash/EstablishCall.cpp b/dbms/src/Flash/EstablishCall.cpp index 8af81e30962..89857a2407e 100644 --- a/dbms/src/Flash/EstablishCall.cpp +++ b/dbms/src/Flash/EstablishCall.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -19,6 +20,11 @@ namespace DB { +namespace FailPoints +{ +extern const char random_tunnel_init_rpc_failure_failpoint[]; +} // namespace FailPoints + EstablishCallData::EstablishCallData(AsyncFlashService * service, grpc::ServerCompletionQueue * cq, grpc::ServerCompletionQueue * notify_cq, const std::shared_ptr> & is_shutdown) : service(service) , cq(cq) @@ -71,6 +77,7 @@ void EstablishCallData::initRpc() std::exception_ptr eptr = nullptr; try { + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_tunnel_init_rpc_failure_failpoint); service->establishMPPConnectionSyncOrAsync(&ctx, &request, nullptr, this); } catch (...) diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index f194afee31f..ec8bde51469 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include @@ -22,6 +23,12 @@ namespace DB { +namespace FailPoints +{ +extern const char random_receiver_sync_msg_push_failure_failpoint[]; +extern const char random_receiver_async_msg_push_failure_failpoint[]; +} // namespace FailPoints + namespace { String getReceiverStateStr(const ExchangeReceiverState & s) @@ -257,7 +264,9 @@ class AsyncRequestHandler : public UnaryCallback recv_msg->packet = std::move(packet); recv_msg->source_index = request->source_index; recv_msg->req_info = req_info; - if (!msg_channel->push(std::move(recv_msg))) + bool push_success = msg_channel->push(std::move(recv_msg)); + fiu_do_on(FailPoints::random_receiver_async_msg_push_failure_failpoint, push_success = false;); + if (!push_success) return false; // can't reuse packet since it is sent to readers. packet = std::make_shared(); @@ -483,7 +492,9 @@ void ExchangeReceiverBase::readLoop(const Request & req) if (recv_msg->packet->has_error()) throw Exception("Exchange receiver meet error : " + recv_msg->packet->error().msg()); - if (!msg_channel.push(std::move(recv_msg))) + bool push_success = msg_channel.push(std::move(recv_msg)); + fiu_do_on(FailPoints::random_receiver_sync_msg_push_failure_failpoint, push_success = false;); + if (!push_success) { meet_error = true; auto local_state = getState(); diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 0381bbdfa04..ac084ba4550 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -51,6 +51,7 @@ extern const char exception_before_mpp_register_tunnel_for_root_mpp_task[]; extern const char exception_during_mpp_register_tunnel_for_non_root_mpp_task[]; extern const char exception_during_mpp_write_err_to_tunnel[]; extern const char force_no_local_region_for_mpp_task[]; +extern const char random_task_lifecycle_failpoint[]; } // namespace FailPoints MPPTask::MPPTask(const mpp::TaskMeta & meta_, const ContextPtr & context_) @@ -394,7 +395,15 @@ void MPPTask::runImpl() writeErrToAllTunnels(err_msg); } LOG_FMT_INFO(log, "task ends, time cost is {} ms.", stopwatch.elapsedMilliseconds()); - unregisterTask(); + // unregister flag is only for FailPoint usage, to produce the situation that MPPTask is destructed + // by grpc CancelMPPTask thread; + bool unregister = true; + fiu_do_on(FailPoints::random_task_lifecycle_failpoint, { + if (!err_msg.empty()) + unregister = false; + }); + if (unregister) + unregisterTask(); if (switchStatus(RUNNING, FINISHED)) LOG_INFO(log, "finish task"); diff --git a/dbms/src/Flash/Mpp/MPPTaskManager.cpp b/dbms/src/Flash/Mpp/MPPTaskManager.cpp index 531f8f7a10d..3df4af5de5f 100644 --- a/dbms/src/Flash/Mpp/MPPTaskManager.cpp +++ b/dbms/src/Flash/Mpp/MPPTaskManager.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -22,6 +23,11 @@ namespace DB { +namespace FailPoints +{ +extern const char random_task_manager_find_task_failure_failpoint[]; +} // namespace FailPoints + MPPTaskManager::MPPTaskManager(MPPTaskSchedulerPtr scheduler_) : scheduler(std::move(scheduler_)) , log(&Poco::Logger::get("TaskManager")) @@ -50,6 +56,7 @@ MPPTaskPtr MPPTaskManager::findTaskWithTimeout(const mpp::TaskMeta & meta, std:: it = query_it->second->task_map.find(id); return it != query_it->second->task_map.end(); }); + fiu_do_on(FailPoints::random_task_manager_find_task_failure_failpoint, ret = false;); if (cancelled) { errMsg = fmt::format("Task [{},{}] has been cancelled.", meta.start_ts(), meta.task_id()); diff --git a/dbms/src/Flash/Mpp/MPPTunnel.cpp b/dbms/src/Flash/Mpp/MPPTunnel.cpp index 826e7fea88a..13a7eaad95e 100644 --- a/dbms/src/Flash/Mpp/MPPTunnel.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnel.cpp @@ -25,6 +25,7 @@ namespace DB namespace FailPoints { extern const char exception_during_mpp_close_tunnel[]; +extern const char random_tunnel_wait_timeout_failpoint[]; } // namespace FailPoints template @@ -322,6 +323,7 @@ void MPPTunnelBase::waitUntilConnectedOrFinished(std::unique_lock #include #include #include namespace DB { +namespace FailPoints +{ +extern const char random_min_tso_scheduler_failpoint[]; +} // namespace FailPoints + constexpr UInt64 MAX_UINT64 = std::numeric_limits::max(); constexpr UInt64 OS_THREAD_SOFT_LIMIT = 100000; @@ -193,7 +199,9 @@ bool MinTSOScheduler::scheduleImp(const UInt64 tso, const MPPQueryTaskSetPtr & q } else { - if (tso <= min_tso) /// the min_tso query should fully run, otherwise throw errors here. + bool is_tso_min = tso <= min_tso; + fiu_do_on(FailPoints::random_min_tso_scheduler_failpoint, is_tso_min = true;); + if (is_tso_min) /// the min_tso query should fully run, otherwise throw errors here. { has_error = true; auto msg = fmt::format("threads are unavailable for the query {} ({} min_tso {}) {}, need {}, but used {} of the thread hard limit {}, {} active and {} waiting queries.", tso, tso == min_tso ? "is" : "is newer than", min_tso, isWaiting ? "from the waiting set" : "when directly schedule it", needed_threads, estimated_thread_usage, thread_hard_limit, active_set.size(), waiting_set.size()); diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 6a39bc333a8..6cb947a1bfa 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -48,6 +49,11 @@ extern const int CANNOT_MERGE_DIFFERENT_AGGREGATED_DATA_VARIANTS; extern const int LOGICAL_ERROR; } // namespace ErrorCodes +namespace FailPoints +{ +extern const char random_aggregate_create_state_failpoint[]; +extern const char random_aggregate_merge_failpoint[]; +} // namespace FailPoints AggregatedDataVariants::~AggregatedDataVariants() { @@ -317,6 +323,7 @@ void Aggregator::createAggregateStates(AggregateDataPtr & aggregate_data) const * In order that then everything is properly destroyed, we "roll back" some of the created states. * The code is not very convenient. */ + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_aggregate_create_state_failpoint); aggregate_functions[j]->create(aggregate_data + offsets_of_aggregate_states[j]); } catch (...) @@ -1504,6 +1511,8 @@ class MergingAndConvertingBlockInputStream : public IProfilingBlockInputStream if (current_bucket_num >= NUM_BUCKETS) return {}; + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_aggregate_merge_failpoint); + AggregatedDataVariantsPtr & first = data[0]; if (current_bucket_num == -1) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 820618a6e8b..181ebcaaa64 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -26,9 +27,17 @@ #include #include #include +#include + namespace DB { +namespace FailPoints +{ +extern const char random_join_build_failpoint[]; +extern const char random_join_prob_failpoint[]; +} // namespace FailPoints + namespace ErrorCodes { extern const int UNKNOWN_SET_DATA_VARIANT; @@ -621,6 +630,7 @@ void NO_INLINE insertFromBlockImplTypeCaseWithLock( } for (size_t insert_index = 0; insert_index < segment_index_info.size(); insert_index++) { + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_join_build_failpoint); size_t segment_index = (insert_index + stream_index) % segment_index_info.size(); if (segment_index == segment_size) { @@ -1513,7 +1523,7 @@ void Join::joinBlockImpl(Block & block, const Maps & maps) const default: throw Exception("Unknown JOIN keys variant.", ErrorCodes::UNKNOWN_SET_DATA_VARIANT); } - + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_join_prob_failpoint); for (size_t i = 0; i < num_columns_to_add; ++i) { const ColumnWithTypeAndName & sample_col = sample_block_with_columns_to_add.getByPosition(i); diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 96cfc0a58ae..78ad4b41ce6 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -53,7 +54,10 @@ extern const int LOGICAL_ERROR; extern const int QUERY_IS_TOO_LARGE; extern const int INTO_OUTFILE_NOT_ALLOWED; } // namespace ErrorCodes - +namespace FailPoints +{ +extern const char random_interpreter_failpoint[]; +} // namespace FailPoints namespace { void checkASTSizeLimits(const IAST & ast, const Settings & settings) @@ -226,6 +230,7 @@ std::tuple executeQueryImpl( context.setProcessListElement(&process_list_entry->get()); } + FAIL_POINT_TRIGGER_EXCEPTION(FailPoints::random_interpreter_failpoint); auto interpreter = query_src.interpreter(context, stage); res = interpreter->execute(); diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 1bb35e51866..571ba8fe3a5 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -977,6 +978,7 @@ int Server::main(const std::vector & /*args*/) Poco::Logger * log = &logger(); #ifdef FIU_ENABLE fiu_init(0); // init failpoint + FailPointHelper::initRandomFailPoints(config(), log); #endif UpdateMallocConfig(log); From 7c19a375044169dcc48bb365d78b32480975ec7c Mon Sep 17 00:00:00 2001 From: hehechen Date: Wed, 22 Jun 2022 18:28:37 +0800 Subject: [PATCH 057/104] Segment test framework (#5150) close pingcap/tiflash#5151 --- .../src/Storages/DeltaMerge/tests/DMTestEnv.h | 5 +- .../DeltaMerge/tests/gtest_segment.cpp | 86 ++++ .../tests/gtest_segment_test_basic.cpp | 430 ++++++++++++++++++ .../tests/gtest_segment_test_basic.h | 123 +++++ 4 files changed, 642 insertions(+), 2 deletions(-) create mode 100644 dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp create mode 100644 dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp create mode 100644 dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h diff --git a/dbms/src/Storages/DeltaMerge/tests/DMTestEnv.h b/dbms/src/Storages/DeltaMerge/tests/DMTestEnv.h index b35dae0cbe2..84fafbc46ef 100644 --- a/dbms/src/Storages/DeltaMerge/tests/DMTestEnv.h +++ b/dbms/src/Storages/DeltaMerge/tests/DMTestEnv.h @@ -273,7 +273,8 @@ class DMTestEnv DataTypePtr pk_type = EXTRA_HANDLE_COLUMN_INT_TYPE, bool is_common_handle = false, size_t rowkey_column_size = 1, - bool with_internal_columns = true) + bool with_internal_columns = true, + bool is_deleted = false) { Block block; const size_t num_rows = (end - beg); @@ -324,7 +325,7 @@ class DMTestEnv VERSION_COLUMN_ID)); // tag_col block.insert(DB::tests::createColumn( - std::vector(num_rows, 0), + std::vector(num_rows, is_deleted), TAG_COLUMN_NAME, TAG_COLUMN_ID)); } diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp new file mode 100644 index 00000000000..1c68ba3bb2a --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp @@ -0,0 +1,86 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace DM +{ +namespace tests +{ +class SegmentOperationTest : public SegmentTestBasic +{ +protected: + static void SetUpTestCase() {} +}; + +TEST_F(SegmentOperationTest, Issue4956) +try +{ + SegmentTestOptions options; + reloadWithOptions(options); + + // flush data, make the segment can be split. + writeSegment(DELTA_MERGE_FIRST_SEGMENT_ID); + flushSegmentCache(DELTA_MERGE_FIRST_SEGMENT_ID); + // write data to cache, reproduce the https://github.com/pingcap/tiflash/issues/4956 + writeSegment(DELTA_MERGE_FIRST_SEGMENT_ID); + deleteRangeSegment(DELTA_MERGE_FIRST_SEGMENT_ID); + auto segment_id = splitSegment(DELTA_MERGE_FIRST_SEGMENT_ID); + ASSERT_TRUE(segment_id.has_value()); + + mergeSegment(DELTA_MERGE_FIRST_SEGMENT_ID, *segment_id); +} +CATCH + +TEST_F(SegmentOperationTest, TestSegment) +try +{ + SegmentTestOptions options; + reloadWithOptions(options); + writeSegment(DELTA_MERGE_FIRST_SEGMENT_ID); + flushSegmentCache(DELTA_MERGE_FIRST_SEGMENT_ID); + mergeSegmentDelta(DELTA_MERGE_FIRST_SEGMENT_ID); + auto segment_id = splitSegment(DELTA_MERGE_FIRST_SEGMENT_ID); + ASSERT_TRUE(segment_id.has_value()); + + size_t origin_rows = getSegmentRowNum(DELTA_MERGE_FIRST_SEGMENT_ID); + + writeSegment(*segment_id); + flushSegmentCache(*segment_id); + deleteRangeSegment(*segment_id); + writeSegmentWithDeletedPack(*segment_id); + mergeSegment(DELTA_MERGE_FIRST_SEGMENT_ID, *segment_id); + + EXPECT_EQ(getSegmentRowNum(DELTA_MERGE_FIRST_SEGMENT_ID), origin_rows); +} +CATCH + +TEST_F(SegmentOperationTest, TestSegmentRandom) +try +{ + SegmentTestOptions options; + options.is_common_handle = true; + reloadWithOptions(options); + randomSegmentTest(100); +} +CATCH +} // namespace tests +} // namespace DM +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp new file mode 100644 index 00000000000..c676f2e08d5 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.cpp @@ -0,0 +1,430 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace DM +{ +namespace tests +{ +void SegmentTestBasic::reloadWithOptions(SegmentTestOptions config) +{ + TiFlashStorageTestBasic::SetUp(); + options = config; + table_columns = std::make_shared(); + + root_segment = reload(config.is_common_handle); + ASSERT_EQ(root_segment->segmentId(), DELTA_MERGE_FIRST_SEGMENT_ID); + segments.clear(); + segments[DELTA_MERGE_FIRST_SEGMENT_ID] = root_segment; +} + +PageId SegmentTestBasic::createNewSegmentWithSomeData() +{ + SegmentPtr new_segment; + std::tie(root_segment, new_segment) = root_segment->split(dmContext(), tableColumns()); + + const size_t num_rows_write_per_batch = 100; + { + // write to segment and flush + Block block = DMTestEnv::prepareSimpleWriteBlock(0, num_rows_write_per_batch, false); + new_segment->write(dmContext(), std::move(block), true); + } + { + // write to segment and don't flush + Block block = DMTestEnv::prepareSimpleWriteBlock(num_rows_write_per_batch, 2 * num_rows_write_per_batch, false); + new_segment->write(dmContext(), std::move(block), false); + } + return new_segment->segmentId(); +} + +size_t SegmentTestBasic::getSegmentRowNumWithoutMVCC(PageId segment_id) +{ + auto segment = segments[segment_id]; + auto in = segment->getInputStreamRaw(dmContext(), *tableColumns()); + + size_t num_rows_read = 0; + in->readPrefix(); + while (Block block = in->read()) + { + num_rows_read += block.rows(); + } + in->readSuffix(); + return num_rows_read; +} + +size_t SegmentTestBasic::getSegmentRowNum(PageId segment_id) +{ + auto segment = segments[segment_id]; + auto in = segment->getInputStream(dmContext(), *tableColumns(), {segment->getRowKeyRange()}); + + size_t num_rows_read = 0; + in->readPrefix(); + while (Block block = in->read()) + { + num_rows_read += block.rows(); + } + in->readSuffix(); + return num_rows_read; +} + +void SegmentTestBasic::checkSegmentRow(PageId segment_id, size_t expected_row_num) +{ + auto segment = segments[segment_id]; + // read written data + auto in = segment->getInputStream(dmContext(), *tableColumns(), {segment->getRowKeyRange()}); + + size_t num_rows_read = 0; + in->readPrefix(); + while (Block block = in->read()) + { + num_rows_read += block.rows(); + } + in->readSuffix(); + ASSERT_EQ(num_rows_read, expected_row_num); +} + +std::optional SegmentTestBasic::splitSegment(PageId segment_id) +{ + auto origin_segment = segments[segment_id]; + size_t origin_segment_row_num = getSegmentRowNum(segment_id); + SegmentPtr segment, new_segment; + std::tie(segment, new_segment) = origin_segment->split(dmContext(), tableColumns()); + if (new_segment) + { + segments[new_segment->segmentId()] = new_segment; + segments[segment_id] = segment; + + EXPECT_EQ(origin_segment_row_num, getSegmentRowNum(segment_id) + getSegmentRowNum(new_segment->segmentId())); + return new_segment->segmentId(); + } + return std::nullopt; +} + +void SegmentTestBasic::mergeSegment(PageId left_segment_id, PageId right_segment_id) +{ + auto left_segment = segments[left_segment_id]; + auto right_segment = segments[right_segment_id]; + + size_t left_segment_row_num = getSegmentRowNum(left_segment_id); + size_t right_segment_row_num = getSegmentRowNum(right_segment_id); + LOG_FMT_TRACE(&Poco::Logger::root(), "merge in segment:{}:{} and {}:{}", left_segment->segmentId(), left_segment_row_num, right_segment->segmentId(), right_segment_row_num); + + SegmentPtr merged_segment = Segment::merge(dmContext(), tableColumns(), left_segment, right_segment); + segments[merged_segment->segmentId()] = merged_segment; + auto it = segments.find(right_segment->segmentId()); + if (it != segments.end()) + { + segments.erase(it); + } + EXPECT_EQ(getSegmentRowNum(merged_segment->segmentId()), left_segment_row_num + right_segment_row_num); +} + +void SegmentTestBasic::mergeSegmentDelta(PageId segment_id) +{ + auto segment = segments[segment_id]; + size_t segment_row_num = getSegmentRowNum(segment_id); + SegmentPtr merged_segment = segment->mergeDelta(dmContext(), tableColumns()); + segments[merged_segment->segmentId()] = merged_segment; + EXPECT_EQ(getSegmentRowNum(merged_segment->segmentId()), segment_row_num); +} + +void SegmentTestBasic::flushSegmentCache(PageId segment_id) +{ + auto segment = segments[segment_id]; + size_t segment_row_num = getSegmentRowNum(segment_id); + segment->flushCache(dmContext()); + EXPECT_EQ(getSegmentRowNum(segment_id), segment_row_num); +} + +std::pair SegmentTestBasic::getSegmentKeyRange(SegmentPtr segment) +{ + Int64 start_key, end_key; + if (!options.is_common_handle) + { + start_key = segment->getRowKeyRange().getStart().int_value; + end_key = segment->getRowKeyRange().getEnd().int_value; + return {start_key, end_key}; + } + EXPECT_EQ(segment->getRowKeyRange().getStart().data[0], TiDB::CodecFlagInt); + EXPECT_EQ(segment->getRowKeyRange().getEnd().data[0], TiDB::CodecFlagInt); + { + size_t cursor = 1; + start_key = DecodeInt64(cursor, String(segment->getRowKeyRange().getStart().data, segment->getRowKeyRange().getStart().size)); + } + { + size_t cursor = 1; + end_key = DecodeInt64(cursor, String(segment->getRowKeyRange().getEnd().data, segment->getRowKeyRange().getEnd().size)); + } + return {start_key, end_key}; +} + +void SegmentTestBasic::writeSegment(PageId segment_id, UInt64 write_rows) +{ + if (write_rows == 0) + { + return; + } + auto segment = segments[segment_id]; + size_t segment_row_num = getSegmentRowNumWithoutMVCC(segment_id); + std::pair keys = getSegmentKeyRange(segment); + Int64 start_key = keys.first; + Int64 end_key = keys.second; + UInt64 remain_row_num = 0; + if (static_cast(end_key - start_key) > write_rows) + { + end_key = start_key + write_rows; + } + else + { + remain_row_num = write_rows - static_cast(end_key - start_key); + } + { + // write to segment and not flush + Block block = DMTestEnv::prepareSimpleWriteBlock(start_key, end_key, false, version, DMTestEnv::pk_name, EXTRA_HANDLE_COLUMN_ID, options.is_common_handle ? EXTRA_HANDLE_COLUMN_STRING_TYPE : EXTRA_HANDLE_COLUMN_INT_TYPE, options.is_common_handle); + segment->write(dmContext(), std::move(block), false); + LOG_FMT_TRACE(&Poco::Logger::root(), "write key range [{}, {})", start_key, end_key); + version++; + } + while (remain_row_num > 0) + { + UInt64 write_num = std::min(remain_row_num, static_cast(end_key - start_key)); + Block block = DMTestEnv::prepareSimpleWriteBlock(start_key, write_num + start_key, false, version, DMTestEnv::pk_name, EXTRA_HANDLE_COLUMN_ID, options.is_common_handle ? EXTRA_HANDLE_COLUMN_STRING_TYPE : EXTRA_HANDLE_COLUMN_INT_TYPE, options.is_common_handle); + segment->write(dmContext(), std::move(block), false); + remain_row_num -= write_num; + LOG_FMT_TRACE(&Poco::Logger::root(), "write key range [{}, {})", start_key, write_num + start_key); + version++; + } + EXPECT_EQ(getSegmentRowNumWithoutMVCC(segment_id), segment_row_num + write_rows); +} + +void SegmentTestBasic::writeSegmentWithDeletedPack(PageId segment_id) +{ + UInt64 write_rows = DEFAULT_MERGE_BLOCK_SIZE; + auto segment = segments[segment_id]; + size_t segment_row_num = getSegmentRowNumWithoutMVCC(segment_id); + std::pair keys = getSegmentKeyRange(segment); + Int64 start_key = keys.first; + Int64 end_key = keys.second; + UInt64 remain_row_num = 0; + if (static_cast(end_key - start_key) > write_rows) + { + end_key = start_key + write_rows; + } + else + { + remain_row_num = write_rows - static_cast(end_key - start_key); + } + { + // write to segment and not flush + Block block = DMTestEnv::prepareSimpleWriteBlock(start_key, end_key, false, version, DMTestEnv::pk_name, EXTRA_HANDLE_COLUMN_ID, options.is_common_handle ? EXTRA_HANDLE_COLUMN_STRING_TYPE : EXTRA_HANDLE_COLUMN_INT_TYPE, options.is_common_handle, 1, true, true); + segment->write(dmContext(), std::move(block), true); + LOG_FMT_TRACE(&Poco::Logger::root(), "write key range [{}, {})", start_key, end_key); + version++; + } + while (remain_row_num > 0) + { + UInt64 write_num = std::min(remain_row_num, static_cast(end_key - start_key)); + Block block = DMTestEnv::prepareSimpleWriteBlock(start_key, write_num + start_key, false, version, DMTestEnv::pk_name, EXTRA_HANDLE_COLUMN_ID, options.is_common_handle ? EXTRA_HANDLE_COLUMN_STRING_TYPE : EXTRA_HANDLE_COLUMN_INT_TYPE, options.is_common_handle, 1, true, true); + segment->write(dmContext(), std::move(block), true); + remain_row_num -= write_num; + LOG_FMT_TRACE(&Poco::Logger::root(), "write key range [{}, {})", start_key, write_num + start_key); + version++; + } + EXPECT_EQ(getSegmentRowNumWithoutMVCC(segment_id), segment_row_num + write_rows); +} + +void SegmentTestBasic::deleteRangeSegment(PageId segment_id) +{ + auto segment = segments[segment_id]; + segment->write(dmContext(), /*delete_range*/ segment->getRowKeyRange()); + EXPECT_EQ(getSegmentRowNum(segment_id), 0); +} + +void SegmentTestBasic::writeRandomSegment() +{ + if (segments.empty()) + { + return; + } + PageId random_segment_id = getRandomSegmentId(); + LOG_FMT_TRACE(&Poco::Logger::root(), "start write segment:{}", random_segment_id); + writeSegment(random_segment_id); +} +void SegmentTestBasic::writeRandomSegmentWithDeletedPack() +{ + if (segments.empty()) + { + return; + } + PageId random_segment_id = getRandomSegmentId(); + LOG_FMT_TRACE(&Poco::Logger::root(), "start write segment with deleted pack:{}", random_segment_id); + writeSegmentWithDeletedPack(random_segment_id); +} + +void SegmentTestBasic::deleteRangeRandomSegment() +{ + if (segments.empty()) + { + return; + } + PageId random_segment_id = getRandomSegmentId(); + LOG_FMT_TRACE(&Poco::Logger::root(), "start delete range segment:{}", random_segment_id); + deleteRangeSegment(random_segment_id); +} + +void SegmentTestBasic::splitRandomSegment() +{ + if (segments.empty()) + { + return; + } + PageId random_segment_id = getRandomSegmentId(); + LOG_FMT_TRACE(&Poco::Logger::root(), "start split segment:{}", random_segment_id); + splitSegment(random_segment_id); +} + +void SegmentTestBasic::mergeRandomSegment() +{ + if (segments.empty() || segments.size() == 1) + { + return; + } + std::pair segment_pair; + segment_pair = getRandomMergeablePair(); + LOG_FMT_TRACE(&Poco::Logger::root(), "start merge segment:{} and {}", segment_pair.first, segment_pair.second); + mergeSegment(segment_pair.first, segment_pair.second); +} + +void SegmentTestBasic::mergeDeltaRandomSegment() +{ + if (segments.empty()) + { + return; + } + PageId random_segment_id = getRandomSegmentId(); + LOG_FMT_TRACE(&Poco::Logger::root(), "start merge delta in segment:{}", random_segment_id); + mergeSegmentDelta(random_segment_id); +} + +void SegmentTestBasic::flushCacheRandomSegment() +{ + if (segments.empty()) + { + return; + } + PageId random_segment_id = getRandomSegmentId(); + LOG_FMT_TRACE(&Poco::Logger::root(), "start flush cache in segment:{}", random_segment_id); + flushSegmentCache(random_segment_id); +} + +void SegmentTestBasic::randomSegmentTest(size_t operator_count) +{ + for (size_t i = 0; i < operator_count; i++) + { + auto op = static_cast(random() % SegmentOperaterMax); + segment_operator_entries[op](); + } +} + +PageId SegmentTestBasic::getRandomSegmentId() +{ + auto max_segment_id = segments.rbegin()->first; + PageId random_segment_id = random() % (max_segment_id + 1); + auto it = segments.find(random_segment_id); + while (it == segments.end()) + { + random_segment_id = random() % (max_segment_id + 1); + it = segments.find(random_segment_id); + } + return random_segment_id; +} + +std::pair SegmentTestBasic::getRandomMergeablePair() +{ + while (true) + { + PageId random_left_segment_id = getRandomSegmentId(); + PageId random_right_segment_id = random_left_segment_id; + while (random_right_segment_id == random_left_segment_id) + { + random_right_segment_id = getRandomSegmentId(); + } + auto left_segment = segments[random_left_segment_id]; + auto right_segment = segments[random_right_segment_id]; + if (compare(left_segment->getRowKeyRange().getEnd(), right_segment->getRowKeyRange().getStart()) != 0 || left_segment->nextSegmentId() != right_segment->segmentId()) + { + continue; + } + return {random_left_segment_id, random_right_segment_id}; + } +} + +RowKeyRange SegmentTestBasic::commanHandleKeyRange() +{ + String start_key, end_key; + { + WriteBufferFromOwnString ss; + ::DB::EncodeUInt(static_cast(TiDB::CodecFlagInt), ss); + ::DB::EncodeInt64(std::numeric_limits::min(), ss); + start_key = ss.releaseStr(); + } + { + WriteBufferFromOwnString ss; + ::DB::EncodeUInt(static_cast(TiDB::CodecFlagInt), ss); + ::DB::EncodeInt64(std::numeric_limits::max(), ss); + end_key = ss.releaseStr(); + } + return RowKeyRange(RowKeyValue(true, std::make_shared(start_key), 0), RowKeyValue(true, std::make_shared(end_key), 0), true, 1); +} + +SegmentPtr SegmentTestBasic::reload(bool is_common_handle, const ColumnDefinesPtr & pre_define_columns, DB::Settings && db_settings) +{ + TiFlashStorageTestBasic::reload(std::move(db_settings)); + storage_path_pool = std::make_unique(db_context->getPathPool().withTable("test", "t1", false)); + storage_pool = std::make_unique(*db_context, /*ns_id*/ 100, *storage_path_pool, "test.t1"); + storage_pool->restore(); + ColumnDefinesPtr cols = (!pre_define_columns) ? DMTestEnv::getDefaultColumns(is_common_handle ? DMTestEnv::PkType::CommonHandle : DMTestEnv::PkType::HiddenTiDBRowID) : pre_define_columns; + setColumns(cols); + + return Segment::newSegment(*dm_context, table_columns, is_common_handle ? commanHandleKeyRange() : RowKeyRange::newAll(is_common_handle, 1), storage_pool->newMetaPageId(), 0); +} + +void SegmentTestBasic::setColumns(const ColumnDefinesPtr & columns) +{ + *table_columns = *columns; + + dm_context = std::make_unique(*db_context, + *storage_path_pool, + *storage_pool, + 0, + /*min_version_*/ 0, + settings.not_compress_columns, + options.is_common_handle, + 1, + db_context->getSettingsRef()); +} +} // namespace tests +} // namespace DM +} // namespace DB diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h new file mode 100644 index 00000000000..ab0c7d6d0be --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment_test_basic.h @@ -0,0 +1,123 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +#pragma once + +#include +#include +#include +#include +#include + +#include + +namespace DB +{ +namespace DM +{ +namespace tests +{ +class SegmentTestBasic : public DB::base::TiFlashStorageTestBasic +{ +public: + struct SegmentTestOptions + { + bool is_common_handle = false; + }; + +public: + void reloadWithOptions(SegmentTestOptions config); + + std::optional splitSegment(PageId segment_id); + void mergeSegment(PageId left_segment_id, PageId right_segment_id); + void mergeSegmentDelta(PageId segment_id); + void flushSegmentCache(PageId segment_id); + void writeSegment(PageId segment_id, UInt64 write_rows = 100); + void writeSegmentWithDeletedPack(PageId segment_id); + void deleteRangeSegment(PageId segment_id); + + + void writeRandomSegment(); + void writeRandomSegmentWithDeletedPack(); + void deleteRangeRandomSegment(); + void splitRandomSegment(); + void mergeRandomSegment(); + void mergeDeltaRandomSegment(); + void flushCacheRandomSegment(); + + void randomSegmentTest(size_t operator_count); + + PageId createNewSegmentWithSomeData(); + size_t getSegmentRowNumWithoutMVCC(PageId segment_id); + size_t getSegmentRowNum(PageId segment_id); + void checkSegmentRow(PageId segment_id, size_t expected_row_num); + std::pair getSegmentKeyRange(SegmentPtr segment); + +protected: + // + std::map segments; + + enum SegmentOperaterType + { + Write = 0, + DeleteRange, + Split, + Merge, + MergeDelta, + FlushCache, + WriteDeletedPack, + SegmentOperaterMax + }; + + const std::vector> segment_operator_entries = { + [this] { writeRandomSegment(); }, + [this] { deleteRangeRandomSegment(); }, + [this] { splitRandomSegment(); }, + [this] { mergeRandomSegment(); }, + [this] { mergeDeltaRandomSegment(); }, + [this] { flushCacheRandomSegment(); }, + [this] { + writeRandomSegmentWithDeletedPack(); + }}; + + PageId getRandomSegmentId(); + + std::pair getRandomMergeablePair(); + + RowKeyRange commanHandleKeyRange(); + + SegmentPtr reload(bool is_common_handle, const ColumnDefinesPtr & pre_define_columns = {}, DB::Settings && db_settings = DB::Settings()); + + // setColumns should update dm_context at the same time + void setColumns(const ColumnDefinesPtr & columns); + + const ColumnDefinesPtr & tableColumns() const { return table_columns; } + + DMContext & dmContext() { return *dm_context; } + +protected: + /// all these var lives as ref in dm_context + std::unique_ptr storage_path_pool; + std::unique_ptr storage_pool; + /// dm_context + std::unique_ptr dm_context; + ColumnDefinesPtr table_columns; + DM::DeltaMergeStore::Settings settings; + + SegmentPtr root_segment; + UInt64 version = 0; + SegmentTestOptions options; +}; +} // namespace tests +} // namespace DM +} // namespace DB \ No newline at end of file From 640c1033cdfe29264df722c0a512ecd4b32c4de6 Mon Sep 17 00:00:00 2001 From: hehechen Date: Wed, 22 Jun 2022 18:58:37 +0800 Subject: [PATCH 058/104] optimize ps v3 restore (#5163) ref pingcap/tiflash#4914 --- dbms/src/Storages/Page/V3/PageDirectory.cpp | 20 ++++++++++++------- dbms/src/Storages/Page/V3/PageDirectory.h | 8 +++++--- .../Storages/Page/V3/PageDirectoryFactory.cpp | 6 ++++-- .../Page/V3/tests/gtest_page_directory.cpp | 4 ++-- 4 files changed, 24 insertions(+), 14 deletions(-) diff --git a/dbms/src/Storages/Page/V3/PageDirectory.cpp b/dbms/src/Storages/Page/V3/PageDirectory.cpp index 5eb275f5af5..951da42de1c 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory.cpp +++ b/dbms/src/Storages/Page/V3/PageDirectory.cpp @@ -478,7 +478,7 @@ PageSize VersionedPageEntries::getEntriesByBlobIds( bool VersionedPageEntries::cleanOutdatedEntries( UInt64 lowest_seq, std::map> * normal_entries_to_deref, - PageEntriesV3 & entries_removed, + PageEntriesV3 * entries_removed, const PageLock & /*page_lock*/) { if (type == EditRecordType::VAR_EXTERNAL) @@ -541,7 +541,10 @@ bool VersionedPageEntries::cleanOutdatedEntries( { if (iter->second.being_ref_count == 1) { - entries_removed.emplace_back(iter->second.entry); + if (entries_removed) + { + entries_removed->emplace_back(iter->second.entry); + } iter = entries.erase(iter); } // The `being_ref_count` for this version is valid. While for older versions, @@ -551,7 +554,10 @@ bool VersionedPageEntries::cleanOutdatedEntries( else { // else there are newer "entry" in the version list, the outdated entries should be removed - entries_removed.emplace_back(iter->second.entry); + if (entries_removed) + { + entries_removed->emplace_back(iter->second.entry); + } iter = entries.erase(iter); } } @@ -564,7 +570,7 @@ bool VersionedPageEntries::cleanOutdatedEntries( return entries.empty() || (entries.size() == 1 && entries.begin()->second.isDelete()); } -bool VersionedPageEntries::derefAndClean(UInt64 lowest_seq, PageIdV3Internal page_id, const PageVersion & deref_ver, const Int64 deref_count, PageEntriesV3 & entries_removed) +bool VersionedPageEntries::derefAndClean(UInt64 lowest_seq, PageIdV3Internal page_id, const PageVersion & deref_ver, const Int64 deref_count, PageEntriesV3 * entries_removed) { auto page_lock = acquireLock(); if (type == EditRecordType::VAR_EXTERNAL) @@ -1239,7 +1245,7 @@ bool PageDirectory::tryDumpSnapshot(const ReadLimiterPtr & read_limiter, const W return done_any_io; } -PageEntriesV3 PageDirectory::gcInMemEntries() +PageEntriesV3 PageDirectory::gcInMemEntries(bool return_removed_entries) { UInt64 lowest_seq = sequence.load(); @@ -1303,7 +1309,7 @@ PageEntriesV3 PageDirectory::gcInMemEntries() const bool all_deleted = iter->second->cleanOutdatedEntries( lowest_seq, &normal_entries_to_deref, - all_del_entries, + return_removed_entries ? &all_del_entries : nullptr, iter->second->acquireLock()); { @@ -1342,7 +1348,7 @@ PageEntriesV3 PageDirectory::gcInMemEntries() page_id, /*deref_ver=*/deref_counter.first, /*deref_count=*/deref_counter.second, - all_del_entries); + return_removed_entries ? &all_del_entries : nullptr); if (all_deleted) { diff --git a/dbms/src/Storages/Page/V3/PageDirectory.h b/dbms/src/Storages/Page/V3/PageDirectory.h index bd7c433022f..2f0f09f4e42 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory.h +++ b/dbms/src/Storages/Page/V3/PageDirectory.h @@ -223,14 +223,14 @@ class VersionedPageEntries bool cleanOutdatedEntries( UInt64 lowest_seq, std::map> * normal_entries_to_deref, - PageEntriesV3 & entries_removed, + PageEntriesV3 * entries_removed, const PageLock & page_lock); bool derefAndClean( UInt64 lowest_seq, PageIdV3Internal page_id, const PageVersion & deref_ver, Int64 deref_count, - PageEntriesV3 & entries_removed); + PageEntriesV3 * entries_removed); void collapseTo(UInt64 seq, PageIdV3Internal page_id, PageEntriesEdit & edit); @@ -360,7 +360,9 @@ class PageDirectory bool tryDumpSnapshot(const ReadLimiterPtr & read_limiter = nullptr, const WriteLimiterPtr & write_limiter = nullptr); - PageEntriesV3 gcInMemEntries(); + // Perform a GC for in-memory entries and return the removed entries. + // If `return_removed_entries` is false, then just return an empty set. + PageEntriesV3 gcInMemEntries(bool return_removed_entries = true); std::set getAliveExternalIds(NamespaceId ns_id) const; diff --git a/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp b/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp index 483c5073ab5..968049a3273 100644 --- a/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp +++ b/dbms/src/Storages/Page/V3/PageDirectoryFactory.cpp @@ -44,7 +44,8 @@ PageDirectoryPtr PageDirectoryFactory::createFromReader(String storage_name, WAL // After restoring from the disk, we need cleanup all invalid entries in memory, or it will // try to run GC again on some entries that are already marked as invalid in BlobStore. - dir->gcInMemEntries(); + // It's no need to remove the expired entries in BlobStore, so skip filling removed_entries to imporve performance. + dir->gcInMemEntries(/*return_removed_entries=*/false); LOG_FMT_INFO(DB::Logger::get("PageDirectoryFactory", storage_name), "PageDirectory restored [max_page_id={}] [max_applied_ver={}]", dir->getMaxId(), dir->sequence); if (blob_stats) @@ -84,7 +85,8 @@ PageDirectoryPtr PageDirectoryFactory::createFromEdit(String storage_name, FileP // After restoring from the disk, we need cleanup all invalid entries in memory, or it will // try to run GC again on some entries that are already marked as invalid in BlobStore. - dir->gcInMemEntries(); + // It's no need to remove the expired entries in BlobStore when restore, so no need to fill removed_entries. + dir->gcInMemEntries(/*return_removed_entries=*/false); if (blob_stats) { diff --git a/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp b/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp index 83e07f75d37..6d6ef41630f 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_directory.cpp @@ -644,14 +644,14 @@ class VersionedEntriesTest : public ::testing::Test { DerefCounter deref_counter; PageEntriesV3 removed_entries; - bool all_removed = entries.cleanOutdatedEntries(seq, &deref_counter, removed_entries, entries.acquireLock()); + bool all_removed = entries.cleanOutdatedEntries(seq, &deref_counter, &removed_entries, entries.acquireLock()); return {all_removed, removed_entries, deref_counter}; } std::tuple runDeref(UInt64 seq, PageVersion ver, Int64 decrease_num) { PageEntriesV3 removed_entries; - bool all_removed = entries.derefAndClean(seq, buildV3Id(TEST_NAMESPACE_ID, page_id), ver, decrease_num, removed_entries); + bool all_removed = entries.derefAndClean(seq, buildV3Id(TEST_NAMESPACE_ID, page_id), ver, decrease_num, &removed_entries); return {all_removed, removed_entries}; } From 69cbfdf8a6bfb1d98ac76dea6e70d87ab3a1ed84 Mon Sep 17 00:00:00 2001 From: Shenghui Wu <793703860@qq.com> Date: Thu, 23 Jun 2022 09:52:36 +0800 Subject: [PATCH 059/104] Fix build failed (#5196) close pingcap/tiflash#5195 --- dbms/src/Common/FailPoint.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index 1dff46c273b..ad5010d7826 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -272,9 +272,9 @@ void FailPointHelper::disableFailPoint(const String &) {} void FailPointHelper::wait(const String &) {} -void FailPointHelper::initRandomFailPoints(Poco::Util::LayeredConfiguration & config, Poco::Logger * log) {} +void FailPointHelper::initRandomFailPoints(Poco::Util::LayeredConfiguration &, Poco::Logger *) {} -void FailPointHelper::enableRandomFailPoint(const String & fail_point_name, double rate) {} +void FailPointHelper::enableRandomFailPoint(const String &, double) {} #endif } // namespace DB From dab31a5e786fce7e2a064977df411a024ea55d6e Mon Sep 17 00:00:00 2001 From: Schrodinger ZHU Yifan Date: Fri, 24 Jun 2022 20:32:37 +0800 Subject: [PATCH 060/104] feat: delta tree dispatching (#5199) close pingcap/tiflash#5200 --- dbms/src/Storages/DeltaMerge/DeltaTree.h | 223 ++++++++------------- dbms/src/Storages/DeltaMerge/DeltaTree.ipp | 165 +++++++++++++++ 2 files changed, 248 insertions(+), 140 deletions(-) create mode 100644 dbms/src/Storages/DeltaMerge/DeltaTree.ipp diff --git a/dbms/src/Storages/DeltaMerge/DeltaTree.h b/dbms/src/Storages/DeltaMerge/DeltaTree.h index 47674ab2cfc..29e127fe35f 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaTree.h +++ b/dbms/src/Storages/DeltaMerge/DeltaTree.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include #include @@ -810,6 +811,20 @@ class DeltaTree template InternPtr afterNodeUpdated(T * node); +#ifdef __x86_64__ + template + InternPtr afterNodeUpdatedGeneric(T * node); + + template + InternPtr afterNodeUpdatedAVX512(T * node); + + template + InternPtr afterNodeUpdatedAVX(T * node); + + template + InternPtr afterNodeUpdatedSSE4(T * node); +#endif + inline void afterLeafUpdated(LeafPtr leaf) { if (leaf->count == 0 && isRootOnly()) @@ -1348,158 +1363,86 @@ typename DT_CLASS::InterAndSid DT_CLASS::submitMinSid(T * node, UInt64 subtree_m } } -DT_TEMPLATE -template -typename DT_CLASS::InternPtr DT_CLASS::afterNodeUpdated(T * node) +#ifndef __x86_64__ +#define TIFLASH_DT_IMPL_NAME afterNodeUpdated +#include "DeltaTree.ipp" +#undef TIFLASH_DT_IMPL_NAME +#else + +// generic implementation +#define TIFLASH_DT_IMPL_NAME afterNodeUpdatedGeneric +#include "DeltaTree.ipp" +#undef TIFLASH_DT_IMPL_NAME + +// avx512 implementation +TIFLASH_BEGIN_AVX512_SPECIFIC_CODE +#define TIFLASH_DT_IMPL_NAME afterNodeUpdatedAVX512 +#include "DeltaTree.ipp" +#undef TIFLASH_DT_IMPL_NAME +TIFLASH_END_TARGET_SPECIFIC_CODE + +// avx implementation +TIFLASH_BEGIN_AVX_SPECIFIC_CODE +#define TIFLASH_DT_IMPL_NAME afterNodeUpdatedAVX +#include "DeltaTree.ipp" +#undef TIFLASH_DT_IMPL_NAME +TIFLASH_END_TARGET_SPECIFIC_CODE + +// sse4 implementation +TIFLASH_BEGIN_SSE4_SPECIFIC_CODE +#define TIFLASH_DT_IMPL_NAME afterNodeUpdatedSSE4 +#include "DeltaTree.ipp" +#undef TIFLASH_DT_IMPL_NAME +TIFLASH_END_TARGET_SPECIFIC_CODE + +namespace Impl { - if (!node) - return {}; - - constexpr bool is_leaf = std::is_same::value; +enum class DeltaTreeVariant +{ + Generic, + SSE4, + AVX, + AVX512 +}; - if (root == asNode(node) && !isLeaf(root) && node->count == 1) +static inline DeltaTreeVariant resolveDeltaTreeVariant() +{ + if (DB::TargetSpecific::AVX512Checker::runtimeSupport()) { - /// Decrease tree height. - root = as(Intern, root)->children[0]; - - --(node->count); - freeNode(node); - - if (isLeaf(root)) - as(Leaf, root)->parent = nullptr; - else - as(Intern, root)->parent = nullptr; - --height; - - LOG_FMT_TRACE(log, "height {} -> {}", (height + 1), height); - - return {}; + return DeltaTreeVariant::AVX512; } - - auto parent = node->parent; - bool parent_updated = false; - - if (T::overflow(node->count)) // split + if (DB::TargetSpecific::AVXChecker::runtimeSupport()) { - if (!parent) - { - /// Increase tree height. - parent = createNode(); - root = asNode(parent); - - parent->deltas[0] = checkDelta(node->getDelta()); - parent->children[0] = asNode(node); - ++(parent->count); - parent->refreshChildParent(); - - ++height; - - LOG_FMT_TRACE(log, "height {} -> {}", (height - 1), height); - } - - auto pos = parent->searchChild(asNode(node)); - - T * next_n = createNode(); - - UInt64 sep_sid = node->split(next_n); - - // handle parent update - parent->shiftEntries(pos + 1, 1); - // for current node - parent->deltas[pos] = checkDelta(node->getDelta()); - // for next node - parent->sids[pos] = sep_sid; - parent->deltas[pos + 1] = checkDelta(next_n->getDelta()); - parent->children[pos + 1] = asNode(next_n); - - ++(parent->count); - - if constexpr (is_leaf) - { - if (as(Leaf, node) == right_leaf) - right_leaf = as(Leaf, next_n); - } - - parent_updated = true; + return DeltaTreeVariant::AVX; } - else if (T::underflow(node->count) && root != asNode(node)) // adopt or merge + if (DB::TargetSpecific::SSE4Checker::runtimeSupport()) { - auto pos = parent->searchChild(asNode(node)); - - // currently we always adopt from the right one if possible - bool is_sibling_left; - size_t sibling_pos; - T * sibling; - - if (unlikely(parent->count <= 1)) - throw Exception("Unexpected parent entry count: " + DB::toString(parent->count)); - - if (pos == parent->count - 1) - { - is_sibling_left = true; - sibling_pos = pos - 1; - sibling = as(T, parent->children[sibling_pos]); - } - else - { - is_sibling_left = false; - sibling_pos = pos + 1; - sibling = as(T, parent->children[sibling_pos]); - } - - if (unlikely(sibling->parent != node->parent)) - throw Exception("parent not the same"); - - auto after_adopt = (node->count + sibling->count) / 2; - if (T::underflow(after_adopt)) - { - // Do merge. - // adoption won't work because the sibling doesn't have enough entries. - - node->merge(sibling, is_sibling_left, pos); - freeNode(sibling); - - pos = std::min(pos, sibling_pos); - parent->deltas[pos] = checkDelta(node->getDelta()); - parent->children[pos] = asNode(node); - parent->shiftEntries(pos + 2, -1); - - if constexpr (is_leaf) - { - if (is_sibling_left && (as(Leaf, sibling) == left_leaf)) - left_leaf = as(Leaf, node); - else if (!is_sibling_left && as(Leaf, sibling) == right_leaf) - right_leaf = as(Leaf, node); - } - --(parent->count); - } - else - { - // Do adoption. - - auto adopt_count = after_adopt - node->count; - auto new_sep_sid = node->adopt(sibling, is_sibling_left, adopt_count, pos); + return DeltaTreeVariant::SSE4; + } + return DeltaTreeVariant::Generic; +} - parent->sids[std::min(pos, sibling_pos)] = new_sep_sid; - parent->deltas[pos] = checkDelta(node->getDelta()); - parent->deltas[sibling_pos] = checkDelta(sibling->getDelta()); - } +static inline DeltaTreeVariant DELTA_TREE_VARIANT = resolveDeltaTreeVariant(); +} // namespace Impl - parent_updated = true; - } - else if (parent) +DT_TEMPLATE +template +typename DT_CLASS::InternPtr DT_CLASS::afterNodeUpdated(T * node) +{ + switch (Impl::DELTA_TREE_VARIANT) { - auto pos = parent->searchChild(asNode(node)); - auto delta = node->getDelta(); - parent_updated = parent->deltas[pos] != delta; - parent->deltas[pos] = checkDelta(delta); + case Impl::DeltaTreeVariant::Generic: + return afterNodeUpdatedGeneric(node); + case Impl::DeltaTreeVariant::SSE4: + return afterNodeUpdatedSSE4(node); + case Impl::DeltaTreeVariant::AVX: + return afterNodeUpdatedAVX(node); + case Impl::DeltaTreeVariant::AVX512: + return afterNodeUpdatedAVX512(node); } - - if (parent_updated) - return parent; - else - return {}; } +#endif + #undef as #undef asNode diff --git a/dbms/src/Storages/DeltaMerge/DeltaTree.ipp b/dbms/src/Storages/DeltaMerge/DeltaTree.ipp new file mode 100644 index 00000000000..27b8a3b96f1 --- /dev/null +++ b/dbms/src/Storages/DeltaMerge/DeltaTree.ipp @@ -0,0 +1,165 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. +DT_TEMPLATE +template +__attribute__((noinline, flatten)) typename DT_CLASS::InternPtr DT_CLASS::TIFLASH_DT_IMPL_NAME(T * node) +{ + if (!node) + return {}; + + constexpr bool is_leaf = std::is_same::value; + + if (root == asNode(node) && !isLeaf(root) && node->count == 1) + { + /// Decrease tree height. + root = as(Intern, root)->children[0]; + + --(node->count); + freeNode(node); + + if (isLeaf(root)) + as(Leaf, root)->parent = nullptr; + else + as(Intern, root)->parent = nullptr; + --height; + + LOG_FMT_TRACE(log, "height {} -> {}", (height + 1), height); + + return {}; + } + + auto parent = node->parent; + bool parent_updated = false; + + if (T::overflow(node->count)) // split + { + if (!parent) + { + /// Increase tree height. + parent = createNode(); + root = asNode(parent); + + parent->deltas[0] = checkDelta(node->getDelta()); + parent->children[0] = asNode(node); + ++(parent->count); + parent->refreshChildParent(); + + ++height; + + LOG_FMT_TRACE(log, "height {} -> {}", (height - 1), height); + } + + auto pos = parent->searchChild(asNode(node)); + + T * next_n = createNode(); + + UInt64 sep_sid = node->split(next_n); + + // handle parent update + parent->shiftEntries(pos + 1, 1); + // for current node + parent->deltas[pos] = checkDelta(node->getDelta()); + // for next node + parent->sids[pos] = sep_sid; + parent->deltas[pos + 1] = checkDelta(next_n->getDelta()); + parent->children[pos + 1] = asNode(next_n); + + ++(parent->count); + + if constexpr (is_leaf) + { + if (as(Leaf, node) == right_leaf) + right_leaf = as(Leaf, next_n); + } + + parent_updated = true; + } + else if (T::underflow(node->count) && root != asNode(node)) // adopt or merge + { + auto pos = parent->searchChild(asNode(node)); + + // currently we always adopt from the right one if possible + bool is_sibling_left; + size_t sibling_pos; + T * sibling; + + if (unlikely(parent->count <= 1)) + throw Exception("Unexpected parent entry count: " + DB::toString(parent->count)); + + if (pos == parent->count - 1) + { + is_sibling_left = true; + sibling_pos = pos - 1; + sibling = as(T, parent->children[sibling_pos]); + } + else + { + is_sibling_left = false; + sibling_pos = pos + 1; + sibling = as(T, parent->children[sibling_pos]); + } + + if (unlikely(sibling->parent != node->parent)) + throw Exception("parent not the same"); + + auto after_adopt = (node->count + sibling->count) / 2; + if (T::underflow(after_adopt)) + { + // Do merge. + // adoption won't work because the sibling doesn't have enough entries. + + node->merge(sibling, is_sibling_left, pos); + freeNode(sibling); + + pos = std::min(pos, sibling_pos); + parent->deltas[pos] = checkDelta(node->getDelta()); + parent->children[pos] = asNode(node); + parent->shiftEntries(pos + 2, -1); + + if constexpr (is_leaf) + { + if (is_sibling_left && (as(Leaf, sibling) == left_leaf)) + left_leaf = as(Leaf, node); + else if (!is_sibling_left && as(Leaf, sibling) == right_leaf) + right_leaf = as(Leaf, node); + } + --(parent->count); + } + else + { + // Do adoption. + + auto adopt_count = after_adopt - node->count; + auto new_sep_sid = node->adopt(sibling, is_sibling_left, adopt_count, pos); + + parent->sids[std::min(pos, sibling_pos)] = new_sep_sid; + parent->deltas[pos] = checkDelta(node->getDelta()); + parent->deltas[sibling_pos] = checkDelta(sibling->getDelta()); + } + + parent_updated = true; + } + else if (parent) + { + auto pos = parent->searchChild(asNode(node)); + auto delta = node->getDelta(); + parent_updated = parent->deltas[pos] != delta; + parent->deltas[pos] = checkDelta(delta); + } + + if (parent_updated) + return parent; + else + return {}; +} \ No newline at end of file From 73e708cd22b935ca240a236a87e261aabddd770e Mon Sep 17 00:00:00 2001 From: Schrodinger ZHU Yifan Date: Fri, 24 Jun 2022 21:12:37 +0800 Subject: [PATCH 061/104] feat: introduce specialized API to write fixed length data rapidly (#5181) close pingcap/tiflash#5183 --- dbms/src/Flash/Coprocessor/TiDBColumn.cpp | 8 ++++---- dbms/src/IO/WriteBuffer.h | 18 ++++++++++++++++++ 2 files changed, 22 insertions(+), 4 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/TiDBColumn.cpp b/dbms/src/Flash/Coprocessor/TiDBColumn.cpp index 7183374a5c1..eef89696d3a 100644 --- a/dbms/src/Flash/Coprocessor/TiDBColumn.cpp +++ b/dbms/src/Flash/Coprocessor/TiDBColumn.cpp @@ -28,7 +28,7 @@ template void encodeLittleEndian(const T & value, WriteBuffer & ss) { auto v = toLittleEndian(value); - ss.write(reinterpret_cast(&v), sizeof(v)); + ss.template writeFixed(&v); } TiDBColumn::TiDBColumn(Int8 element_len_) @@ -141,10 +141,10 @@ void TiDBColumn::append(const TiDBDecimal & decimal) encodeLittleEndian(decimal.digits_int, *data); encodeLittleEndian(decimal.digits_frac, *data); encodeLittleEndian(decimal.result_frac, *data); - encodeLittleEndian((UInt8)decimal.negative, *data); - for (int i = 0; i < MAX_WORD_BUF_LEN; i++) + encodeLittleEndian(static_cast(decimal.negative), *data); + for (int i : decimal.word_buf) { - encodeLittleEndian(decimal.word_buf[i], *data); + encodeLittleEndian(i, *data); } finishAppendFixed(); } diff --git a/dbms/src/IO/WriteBuffer.h b/dbms/src/IO/WriteBuffer.h index 361081d1176..0c0fa2cb545 100644 --- a/dbms/src/IO/WriteBuffer.h +++ b/dbms/src/IO/WriteBuffer.h @@ -96,6 +96,24 @@ class WriteBuffer : public BufferBase } } + template + __attribute__((always_inline)) void writeFixed(const T * __restrict from) + { + if (likely(working_buffer.end() - pos >= static_cast(sizeof(T)))) + { + tiflash_compiler_builtin_memcpy(pos, from, sizeof(T)); + pos += sizeof(T); + } + else + { + [&]() __attribute__((noinline)) + { + write(reinterpret_cast(from), sizeof(T)); + } + (); + } + } + inline void write(char x) { From f84d7e37e7c850891048ec3efb2cf80e5a32adb3 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Tue, 28 Jun 2022 12:18:39 +0800 Subject: [PATCH 062/104] Add gtest for Limit, TopN, Projection (#5187) (#5188) close pingcap/tiflash#5187 --- dbms/src/Debug/astToExecutor.cpp | 1 - dbms/src/Flash/tests/gtest_executor.cpp | 2 +- dbms/src/Flash/tests/gtest_limit_executor.cpp | 77 ++++++ .../Flash/tests/gtest_projection_executor.cpp | 225 ++++++++++++++++++ dbms/src/Flash/tests/gtest_topn_executor.cpp | 221 +++++++++++++++++ dbms/src/TestUtils/FunctionTestUtils.cpp | 5 +- dbms/src/TestUtils/mockExecutor.cpp | 5 + dbms/src/TestUtils/mockExecutor.h | 6 +- 8 files changed, 536 insertions(+), 6 deletions(-) create mode 100644 dbms/src/Flash/tests/gtest_limit_executor.cpp create mode 100644 dbms/src/Flash/tests/gtest_projection_executor.cpp create mode 100644 dbms/src/Flash/tests/gtest_topn_executor.cpp diff --git a/dbms/src/Debug/astToExecutor.cpp b/dbms/src/Debug/astToExecutor.cpp index fec76d7a085..7d1f3bc7209 100644 --- a/dbms/src/Debug/astToExecutor.cpp +++ b/dbms/src/Debug/astToExecutor.cpp @@ -1629,7 +1629,6 @@ ExecutorPtr compileProject(ExecutorPtr input, size_t & executor_index, ASTPtr se } } } - auto project = std::make_shared(executor_index, output_schema, std::move(exprs)); project->children.push_back(input); return project; diff --git a/dbms/src/Flash/tests/gtest_executor.cpp b/dbms/src/Flash/tests/gtest_executor.cpp index 64c60f14bb6..b4ba1a75563 100644 --- a/dbms/src/Flash/tests/gtest_executor.cpp +++ b/dbms/src/Flash/tests/gtest_executor.cpp @@ -227,4 +227,4 @@ try CATCH } // namespace tests -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/tests/gtest_limit_executor.cpp b/dbms/src/Flash/tests/gtest_limit_executor.cpp new file mode 100644 index 00000000000..e4a3aa5db5e --- /dev/null +++ b/dbms/src/Flash/tests/gtest_limit_executor.cpp @@ -0,0 +1,77 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +namespace DB +{ +namespace tests +{ + +class ExecutorLimitTestRunner : public DB::tests::ExecutorTest +{ +public: + using ColDataType = std::optional::FieldType>; + using ColumnWithData = std::vector; + + void initializeContext() override + { + ExecutorTest::initializeContext(); + + context.addMockTable({db_name, table_name}, + {{col_name, TiDB::TP::TypeString}}, + {toNullableVec(col_name, col0)}); + } + + std::shared_ptr buildDAGRequest(size_t limit_num) + { + return context.scan(db_name, table_name).limit(limit_num).build(context); + } + + /// Prepare some names + const String db_name{"test_db"}; + const String table_name{"projection_test_table"}; + const String col_name{"limit_col"}; + const ColumnWithData col0{"col0-0", {}, "col0-2", "col0-3", {}, "col0-5", "col0-6", "col0-7"}; +}; + +TEST_F(ExecutorLimitTestRunner, Limit) +try +{ + std::shared_ptr request; + ColumnsWithTypeAndName expect_cols; + + /// Check limit result with various parameters + const size_t col_data_num = col0.size(); + for (size_t limit_num = 0; limit_num <= col_data_num + 3; ++limit_num) + { + if (limit_num == col_data_num + 3) + limit_num = INT_MAX; + request = buildDAGRequest(limit_num); + + if (limit_num == 0) + expect_cols = {}; + else if (limit_num > col_data_num) + expect_cols = {toNullableVec(col_name, ColumnWithData(col0.begin(), col0.end()))}; + else + expect_cols = {toNullableVec(col_name, ColumnWithData(col0.begin(), col0.begin() + limit_num))}; + + executeStreams(request, expect_cols); + } +} +CATCH + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Flash/tests/gtest_projection_executor.cpp b/dbms/src/Flash/tests/gtest_projection_executor.cpp new file mode 100644 index 00000000000..4f6401eb483 --- /dev/null +++ b/dbms/src/Flash/tests/gtest_projection_executor.cpp @@ -0,0 +1,225 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +namespace DB +{ +namespace tests +{ + +class ExecutorProjectionTestRunner : public DB::tests::ExecutorTest +{ +public: + using ColDataString = std::vector::FieldType>>; + using ColDataInt32 = std::vector::FieldType>>; + + void initializeContext() override + { + ExecutorTest::initializeContext(); + + context.addMockTable({db_name, table_name}, + {{col_names[0], TiDB::TP::TypeString}, + {col_names[1], TiDB::TP::TypeString}, + {col_names[2], TiDB::TP::TypeString}, + {col_names[3], TiDB::TP::TypeLong}, + {col_names[4], TiDB::TP::TypeLong}}, + {toNullableVec(col_names[0], col0), + toNullableVec(col_names[1], col1), + toNullableVec(col_names[2], col2), + toNullableVec(col_names[3], col3), + toNullableVec(col_names[4], col4)}); + } + + template + std::shared_ptr buildDAGRequest(T param, const String & sort_col) + { + /// topN is introduced, so that we can get stable results in concurrency environment. + return context.scan(db_name, table_name).project(param).topN(sort_col, false, 100).build(context); + }; + + void executeWithConcurrency(const std::shared_ptr & request, const ColumnsWithTypeAndName & expect_columns) + { + for (size_t i = 1; i < 10; i += 2) + { + executeStreams(request, expect_columns, i); + } + } + + /// Prepare column data + const ColDataString col0{"col0-0", "col0-1", "", "col0-2", {}, "col0-3", ""}; + const ColDataString col1{"col1-0", {}, "", "col1-1", "", "col1-2", "col1-3"}; + const ColDataString col2{"", "col2-0", "col2-1", {}, "col2-3", {}, "col2-4"}; + const ColDataInt32 col3{1, {}, 0, -111111, {}, 0, 9999}; + + /** Each value in col4 should be different from each other so that topn + * could sort the columns into an unique result, or multi-results could + * be right. + */ + const ColDataInt32 col4{0, 5, -123, -234, {}, 24353, 9999}; + + /// Results after sorted by col4 + const ColDataString col0_sorted_asc{{}, "col0-2", "", "col0-0", "col0-1", "", "col0-3"}; + const ColDataString col1_sorted_asc{"", "col1-1", "", "col1-0", {}, "col1-3", "col1-2"}; + const ColDataString col2_sorted_asc{"col2-3", {}, "col2-1", "", "col2-0", "col2-4", {}}; + const ColDataInt32 col3_sorted_asc{{}, -111111, 0, 1, {}, 9999, 0}; + const ColDataInt32 col4_sorted_asc{{}, -234, -123, 0, 5, 9999, 24353}; + + /// Prepare some names + std::vector col_names{"col0", "col1", "col2", "col3", "col4"}; + const String db_name{"test_db"}; + const String table_name{"projection_test_table"}; +}; + +TEST_F(ExecutorProjectionTestRunner, Projection) +try +{ + /// Check single column + auto request = buildDAGRequest({col_names[4]}, col_names[4]); + executeWithConcurrency(request, {toNullableVec(col_names[4], col4_sorted_asc)}); + + /// Check multi columns + request = buildDAGRequest({col_names[0], col_names[4]}, col_names[4]); + executeWithConcurrency(request, + { + toNullableVec(col_names[0], col0_sorted_asc), + toNullableVec(col_names[4], col4_sorted_asc), + }); + + /// Check multi columns + request = buildDAGRequest({col_names[0], col_names[1], col_names[4]}, col_names[4]); + executeWithConcurrency(request, + {toNullableVec(col_names[0], col0_sorted_asc), + toNullableVec(col_names[1], col1_sorted_asc), + toNullableVec(col_names[4], col4_sorted_asc)}); + + /// Check duplicate columns + request = buildDAGRequest({col_names[4], col_names[4], col_names[4]}, col_names[4]); + executeWithConcurrency(request, + {toNullableVec(col_names[4], col4_sorted_asc), + toNullableVec(col_names[4], col4_sorted_asc), + toNullableVec(col_names[4], col4_sorted_asc)}); + + { + /// Check large number of columns + const size_t col_num = 100; + MockColumnNamesVec projection_input; + ColumnsWithTypeAndName columns; + auto expect_column = toNullableVec(col_names[4], col4_sorted_asc); + + for (size_t i = 0; i < col_num; ++i) + { + projection_input.push_back(col_names[4]); + columns.push_back(expect_column); + } + + request = buildDAGRequest(projection_input, col_names[4]); + executeWithConcurrency(request, columns); + } +} +CATCH + +TEST_F(ExecutorProjectionTestRunner, ProjectionFunction) +try +{ + std::shared_ptr request; + + /// Test "equal" function + + /// Data type: TypeString + request = buildDAGRequest({eq(col(col_names[0]), col(col_names[0])), col(col_names[4])}, col_names[4]); + executeWithConcurrency(request, + {toNullableVec({{}, 1, 1, 1, 1, 1, 1}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + request = buildDAGRequest({eq(col(col_names[0]), col(col_names[1])), col(col_names[4])}, col_names[4]); + executeWithConcurrency(request, + {toNullableVec({{}, 0, 1, 0, {}, 0, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + /// Data type: TypeLong + request = buildDAGRequest({eq(col(col_names[3]), col(col_names[4])), col(col_names[4])}, col_names[4]); + executeWithConcurrency(request, + {toNullableVec({{}, 0, 0, 0, {}, 1, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + + /// Test "greater" function + + /// Data type: TypeString + request = buildDAGRequest({gt(col(col_names[0]), col(col_names[1])), col(col_names[4])}, col_names[4]); + executeWithConcurrency(request, + {toNullableVec({{}, 0, 0, 0, {}, 0, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + request = buildDAGRequest({gt(col(col_names[1]), col(col_names[0])), col(col_names[4])}, col_names[4]); + executeWithConcurrency(request, + {toNullableVec({{}, 1, 0, 1, {}, 1, 1}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + /// Data type: TypeLong + request = buildDAGRequest({gt(col(col_names[3]), col(col_names[4])), col(col_names[4])}, col_names[4]); + executeWithConcurrency(request, + {toNullableVec({{}, 0, 1, 1, {}, 0, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + request = buildDAGRequest({gt(col(col_names[4]), col(col_names[3])), col(col_names[4])}, col_names[4]); + executeWithConcurrency(request, + {toNullableVec({{}, 1, 0, 0, {}, 0, 1}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + + /// Test "and" function + + /// Data type: TypeString + request = buildDAGRequest({And(col(col_names[0]), col(col_names[0])), col(col_names[4])}, col_names[4]); + executeWithConcurrency(request, + {toNullableVec({{}, 0, 0, 0, 0, 0, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + request = buildDAGRequest({And(col(col_names[0]), col(col_names[1])), col(col_names[4])}, col_names[4]); + executeWithConcurrency(request, + {toNullableVec({0, 0, 0, 0, 0, 0, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + /// Data type: TypeLong + request = buildDAGRequest({And(col(col_names[3]), col(col_names[4])), col(col_names[4])}, col_names[4]); + executeWithConcurrency(request, + {toNullableVec({{}, 1, 0, 0, {}, 1, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + /// Test "not" function + + /// Data type: TypeString + request = buildDAGRequest({NOT(col(col_names[0])), NOT(col(col_names[1])), NOT(col(col_names[2])), col(col_names[4])}, col_names[4]); + executeWithConcurrency(request, + {toNullableVec({{}, 1, 1, 1, 1, 1, 1}), + toNullableVec({1, 1, 1, 1, {}, 1, 1}), + toNullableVec({1, {}, 1, 1, 1, 1, {}}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + /// Data type: TypeLong + request = buildDAGRequest({NOT(col(col_names[3])), NOT(col(col_names[4])), col(col_names[4])}, col_names[4]); + executeWithConcurrency(request, + {toNullableVec({{}, 0, 1, 0, {}, 0, 1}), + toNullableVec({{}, 0, 0, 1, 0, 0, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + /// TODO more functions... +} +CATCH + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Flash/tests/gtest_topn_executor.cpp b/dbms/src/Flash/tests/gtest_topn_executor.cpp new file mode 100644 index 00000000000..0e55702795d --- /dev/null +++ b/dbms/src/Flash/tests/gtest_topn_executor.cpp @@ -0,0 +1,221 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +namespace DB +{ +namespace tests +{ + +class ExecutorTopNTestRunner : public DB::tests::ExecutorTest +{ +public: + using ColStringType = std::optional::FieldType>; + using ColInt32Type = std::optional::FieldType>; + using ColumnWithString = std::vector; + using ColumnWithInt32 = std::vector; + + void initializeContext() override + { + ExecutorTest::initializeContext(); + + context.addMockTable({db_name, table_single_name}, + {{single_col_name, TiDB::TP::TypeString}}, + {toNullableVec(single_col_name, col0)}); + + context.addMockTable({db_name, table_name}, + {{col_name[0], TiDB::TP::TypeLong}, + {col_name[1], TiDB::TP::TypeString}, + {col_name[2], TiDB::TP::TypeString}, + {col_name[3], TiDB::TP::TypeLong}}, + {toNullableVec(col_name[0], col_age), + toNullableVec(col_name[1], col_gender), + toNullableVec(col_name[2], col_country), + toNullableVec(col_name[3], c0l_salary)}); + } + + std::shared_ptr buildDAGRequest(const String & table_name, const String & col_name, bool is_desc, int limit_num) + { + return context.scan(db_name, table_name).topN(col_name, is_desc, limit_num).build(context); + } + + std::shared_ptr buildDAGRequest(const String & table_name, MockOrderByItems order_by_items, int limit, MockAsts func_proj_ast = {}, MockColumnNames out_proj_ast = {}) + { + if (func_proj_ast.size() == 0) + return context.scan(db_name, table_name).topN(order_by_items, limit).build(context); + else + return context.scan(db_name, table_name).project(func_proj_ast).topN(order_by_items, limit).project(out_proj_ast).build(context); + } + + /// Prepare some names + const String db_name{"test_db"}; + + const String table_single_name{"topn_single_table"}; /// For single column test + const String single_col_name{"single_col"}; + ColumnWithString col0{"col0-0", "col0-1", "col0-2", {}, "col0-4", {}, "col0-6", "col0-7"}; + + const String table_name{"clerk"}; + const std::vector col_name{"age", "gender", "country", "salary"}; + ColumnWithInt32 col_age{{}, 27, 32, 36, {}, 34}; + ColumnWithString col_gender{"female", "female", "male", "female", "male", "male"}; + ColumnWithString col_country{"korea", "usa", "usa", "china", "china", "china"}; + ColumnWithInt32 c0l_salary{1300, 0, {}, 900, {}, -300}; +}; + +TEST_F(ExecutorTopNTestRunner, TopN) +try +{ + std::shared_ptr request; + std::vector expect_cols; + + { + /// Test single column + size_t col_data_num = col0.size(); + for (size_t i = 1; i <= 1; ++i) + { + bool is_desc; + is_desc = static_cast(i); /// Set descent or ascent + if (is_desc) + sort(col0.begin(), col0.end(), std::greater()); /// Sort col0 for the following comparison + else + sort(col0.begin(), col0.end()); + + for (size_t limit_num = 0; limit_num <= col_data_num + 5; ++limit_num) + { + request = buildDAGRequest(table_single_name, single_col_name, is_desc, limit_num); + + expect_cols.clear(); + if (limit_num == 0 || limit_num > col_data_num) + expect_cols.push_back({toNullableVec(single_col_name, ColumnWithString(col0.begin(), col0.end()))}); + else + expect_cols.push_back({toNullableVec(single_col_name, ColumnWithString(col0.begin(), col0.begin() + limit_num))}); + + executeStreams(request, expect_cols[0]); + executeStreams(request, expect_cols[0], 2); + executeStreams(request, expect_cols[0], 4); + executeStreams(request, expect_cols[0], 8); + } + } + } + + { + /// Test multi-columns + expect_cols = {{toNullableVec(col_name[0], ColumnWithInt32{36, 34, 32, 27, {}, {}}), + toNullableVec(col_name[1], ColumnWithString{"female", "male", "male", "female", "male", "female"}), + toNullableVec(col_name[2], ColumnWithString{"china", "china", "usa", "usa", "china", "korea"}), + toNullableVec(col_name[3], ColumnWithInt32{900, -300, {}, 0, {}, 1300})}, + {toNullableVec(col_name[0], ColumnWithInt32{32, {}, 34, 27, 36, {}}), + toNullableVec(col_name[1], ColumnWithString{"male", "male", "male", "female", "female", "female"}), + toNullableVec(col_name[2], ColumnWithString{"usa", "china", "china", "usa", "china", "korea"}), + toNullableVec(col_name[3], ColumnWithInt32{{}, {}, -300, 0, 900, 1300})}, + {toNullableVec(col_name[0], ColumnWithInt32{34, {}, 32, 36, {}, 27}), + toNullableVec(col_name[1], ColumnWithString{"male", "male", "male", "female", "female", "female"}), + toNullableVec(col_name[2], ColumnWithString{"china", "china", "usa", "china", "korea", "usa"}), + toNullableVec(col_name[3], ColumnWithInt32{-300, {}, {}, 900, 1300, 0})}}; + + std::vector order_by_items{ + /// select * from clerk order by age DESC, gender DESC; + {MockOrderByItem(col_name[0], true), MockOrderByItem(col_name[1], true)}, + /// select * from clerk order by gender DESC, salary ASC; + {MockOrderByItem(col_name[1], true), MockOrderByItem(col_name[3], false)}, + /// select * from clerk order by gender DESC, country ASC, salary DESC; + {MockOrderByItem(col_name[1], true), MockOrderByItem(col_name[2], false), MockOrderByItem(col_name[3], true)}}; + + size_t test_num = expect_cols.size(); + + for (size_t i = 0; i < test_num; ++i) + { + request = buildDAGRequest(table_name, order_by_items[i], 100); + executeStreams(request, expect_cols[i]); + } + } +} +CATCH + +TEST_F(ExecutorTopNTestRunner, TopNFunction) +try +{ + std::shared_ptr request; + std::vector expect_cols; + MockColumnNames output_projection{col_name[0], col_name[1], col_name[2], col_name[3]}; + MockAsts func_projection; // Do function operation for topn + MockOrderByItems order_by_items; + ASTPtr col0_ast = col(col_name[0]); + ASTPtr col1_ast = col(col_name[1]); + ASTPtr col2_ast = col(col_name[2]); + ASTPtr col3_ast = col(col_name[3]); + ASTPtr func_ast; + + { + /// "and" function + expect_cols = {{toNullableVec(col_name[0], ColumnWithInt32{{}, {}, 32, 27, 36, 34}), + toNullableVec(col_name[1], ColumnWithString{"female", "male", "male", "female", "female", "male"}), + toNullableVec(col_name[2], ColumnWithString{"korea", "china", "usa", "usa", "china", "china"}), + toNullableVec(col_name[3], ColumnWithInt32{1300, {}, {}, 0, 900, -300})}}; + + { + /// select * from clerk order by age and salary ASC limit 100; + order_by_items = {MockOrderByItem("and(age, salary)", false)}; + func_ast = And(col(col_name[0]), col(col_name[3])); + func_projection = {col0_ast, col1_ast, col2_ast, col3_ast, func_ast}; + + request = buildDAGRequest(table_name, order_by_items, 100, func_projection, output_projection); + executeStreams(request, expect_cols[0]); + } + } + + { + /// "equal" function + expect_cols = {{toNullableVec(col_name[0], ColumnWithInt32{27, 36, 34, 32, {}, {}}), + toNullableVec(col_name[1], ColumnWithString{"female", "female", "male", "male", "female", "male"}), + toNullableVec(col_name[2], ColumnWithString{"usa", "china", "china", "usa", "korea", "china"}), + toNullableVec(col_name[3], ColumnWithInt32{0, 900, -300, {}, 1300, {}})}}; + + { + /// select age, salary from clerk order by age = salary DESC limit 100; + order_by_items = {MockOrderByItem("equals(age, salary)", true)}; + func_ast = eq(col(col_name[0]), col(col_name[3])); + func_projection = {col0_ast, col1_ast, col2_ast, col3_ast, func_ast}; + + request = buildDAGRequest(table_name, order_by_items, 100, func_projection, output_projection); + executeStreams(request, expect_cols[0]); + } + } + + { + /// "greater" function + expect_cols = {{toNullableVec(col_name[0], ColumnWithInt32{{}, 32, {}, 36, 27, 34}), + toNullableVec(col_name[1], ColumnWithString{"female", "male", "male", "female", "female", "male"}), + toNullableVec(col_name[2], ColumnWithString{"korea", "usa", "china", "china", "usa", "china"}), + toNullableVec(col_name[3], ColumnWithInt32{1300, {}, {}, 900, 0, -300})}}; + + { + /// select age, gender, country, salary from clerk order by age > salary ASC limit 100; + order_by_items = {MockOrderByItem("greater(age, salary)", false)}; + func_ast = gt(col(col_name[0]), col(col_name[3])); + func_projection = {col0_ast, col1_ast, col2_ast, col3_ast, func_ast}; + + request = buildDAGRequest(table_name, order_by_items, 100, func_projection, output_projection); + executeStreams(request, expect_cols[0]); + } + } + + /// TODO more functions... +} +CATCH + +} // namespace tests +} // namespace DB diff --git a/dbms/src/TestUtils/FunctionTestUtils.cpp b/dbms/src/TestUtils/FunctionTestUtils.cpp index 637fbf51c00..7fb526aeb01 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.cpp +++ b/dbms/src/TestUtils/FunctionTestUtils.cpp @@ -108,14 +108,15 @@ void blockEqual( const Block & actual) { size_t columns = actual.columns(); + size_t expected_columns = expected.columns(); - ASSERT_TRUE(expected.columns() == columns); + ASSERT_EQ(expected_columns, columns); for (size_t i = 0; i < columns; ++i) { const auto & expected_col = expected.getByPosition(i); const auto & actual_col = actual.getByPosition(i); - ASSERT_TRUE(actual_col.type->getName() == expected_col.type->getName()); + ASSERT_EQ(actual_col.type->getName(), expected_col.type->getName()); ASSERT_COLUMN_EQ(expected_col.column, actual_col.column); } } diff --git a/dbms/src/TestUtils/mockExecutor.cpp b/dbms/src/TestUtils/mockExecutor.cpp index 2cf8a939b58..9a6e92dd9c1 100644 --- a/dbms/src/TestUtils/mockExecutor.cpp +++ b/dbms/src/TestUtils/mockExecutor.cpp @@ -219,6 +219,11 @@ DAGRequestBuilder & DAGRequestBuilder::project(MockAsts exprs) } DAGRequestBuilder & DAGRequestBuilder::project(MockColumnNames col_names) +{ + return project(MockColumnNamesVec(col_names)); +} + +DAGRequestBuilder & DAGRequestBuilder::project(MockColumnNamesVec col_names) { assert(root); auto exp_list = std::make_shared(); diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index c11635ac93e..bad92c4226d 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -31,6 +31,7 @@ using MockOrderByItems = std::initializer_list; using MockPartitionByItem = std::pair; using MockPartitionByItems = std::initializer_list; using MockColumnNames = std::initializer_list; +using MockColumnNamesVec = std::vector; using MockAsts = std::initializer_list; using MockWindowFrame = mock::MockWindowFrame; @@ -84,6 +85,7 @@ class DAGRequestBuilder DAGRequestBuilder & project(const String & col_name); DAGRequestBuilder & project(MockAsts expr); DAGRequestBuilder & project(MockColumnNames col_names); + DAGRequestBuilder & project(MockColumnNamesVec col_names); DAGRequestBuilder & exchangeSender(tipb::ExchangeType exchange_type); @@ -181,8 +183,8 @@ MockWindowFrame buildDefaultRowsFrame(); #define gt(expr1, expr2) makeASTFunction("greater", (expr1), (expr2)) #define And(expr1, expr2) makeASTFunction("and", (expr1), (expr2)) #define Or(expr1, expr2) makeASTFunction("or", (expr1), (expr2)) -#define NOT(expr) makeASTFunction("not", (expr1), (expr2)) -#define Max(expr) makeASTFunction("max", expr) +#define NOT(expr) makeASTFunction("not", (expr)) +#define Max(expr) makeASTFunction("max", (expr)) /// Window functions #define RowNumber() makeASTFunction("RowNumber") #define Rank() makeASTFunction("Rank") From 7a203394e0ac9f6fece645ef78644d1b5083b2f1 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 29 Jun 2022 13:52:39 +0800 Subject: [PATCH 063/104] add `MPPTask::handleError()` (#5202) ref pingcap/tiflash#5095 --- .../ParallelAggregatingBlockInputStream.cpp | 4 +- dbms/src/DataStreams/UnionBlockInputStream.h | 4 +- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 2 +- dbms/src/Flash/Mpp/MPPHandler.cpp | 2 +- dbms/src/Flash/Mpp/MPPTask.cpp | 156 +++++++++++------- dbms/src/Flash/Mpp/MPPTask.h | 21 ++- dbms/src/Flash/Mpp/MPPTaskManager.cpp | 11 ++ dbms/src/Flash/Mpp/MPPTaskManager.h | 2 + dbms/src/Flash/Mpp/TaskStatus.cpp | 2 + dbms/src/Flash/Mpp/TaskStatus.h | 1 + tests/fullstack-test/mpp/issue_2471.test | 10 +- tests/fullstack-test/mpp/mpp_fail.test | 50 +++++- tests/run-test.py | 39 +++-- 13 files changed, 215 insertions(+), 89 deletions(-) diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp index f4f8dfc1338..f983de91b37 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp @@ -198,8 +198,8 @@ void ParallelAggregatingBlockInputStream::Handler::onException(std::exception_pt /// can not cancel parent inputStream or the exception might be lost if (!parent.executed) - /// kill the processor so ExchangeReceiver will be closed - parent.processor.cancel(true); + /// use cancel instead of kill to avoid too many useless error message + parent.processor.cancel(false); } diff --git a/dbms/src/DataStreams/UnionBlockInputStream.h b/dbms/src/DataStreams/UnionBlockInputStream.h index 251d0663e14..a782c3dd087 100644 --- a/dbms/src/DataStreams/UnionBlockInputStream.h +++ b/dbms/src/DataStreams/UnionBlockInputStream.h @@ -293,8 +293,8 @@ class UnionBlockInputStream final : public IProfilingBlockInputStream /// and the exception is lost. output_queue.emplace(exception); /// can not cancel itself or the exception might be lost - /// kill the processor so ExchangeReceiver will be closed - processor.cancel(true); + /// use cancel instead of kill to avoid too many useless error message + processor.cancel(false); } struct Handler diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index ec8bde51469..3b36adf2c40 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -358,7 +358,7 @@ template void ExchangeReceiverBase::cancel() { setEndState(ExchangeReceiverState::CANCELED); - msg_channel.finish(); + msg_channel.cancel(); } template diff --git a/dbms/src/Flash/Mpp/MPPHandler.cpp b/dbms/src/Flash/Mpp/MPPHandler.cpp index a3096aaa644..7f97a1dd698 100644 --- a/dbms/src/Flash/Mpp/MPPHandler.cpp +++ b/dbms/src/Flash/Mpp/MPPHandler.cpp @@ -31,7 +31,7 @@ void MPPHandler::handleError(const MPPTaskPtr & task, String error) try { if (task) - task->cancel(error); + task->handleError(error); } catch (...) { diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index ac084ba4550..c2d5e6f49f8 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -81,6 +81,34 @@ MPPTask::~MPPTask() LOG_FMT_DEBUG(log, "finish MPPTask: {}", id.toString()); } +void MPPTask::abortTunnels(const String & message, AbortType abort_type) +{ + if (abort_type == AbortType::ONCANCELLATION) + { + closeAllTunnels(message); + } + else + { + RUNTIME_ASSERT(tunnel_set != nullptr, log, "mpp task without tunnel set"); + tunnel_set->writeError(message); + } +} + +void MPPTask::abortReceivers() +{ + if (likely(receiver_set != nullptr)) + { + receiver_set->cancel(); + } +} + +void MPPTask::abortDataStreams(AbortType abort_type) +{ + /// When abort type is ONERROR, it means MPPTask already known it meet error, so let the remaining task stop silently to avoid too many useless error message + bool is_kill = abort_type == AbortType::ONCANCELLATION; + context->getProcessList().sendCancelToQuery(context->getCurrentQueryId(), context->getClientInfo().current_user, is_kill); +} + void MPPTask::closeAllTunnels(const String & reason) { if (likely(tunnel_set)) @@ -156,14 +184,6 @@ void MPPTask::initExchangeReceivers() dag_context->setMPPReceiverSet(receiver_set); } -void MPPTask::cancelAllReceivers() -{ - if (likely(receiver_set != nullptr)) - { - receiver_set->cancel(); - } -} - std::pair MPPTask::getTunnel(const ::mpp::EstablishMPPConnectionRequest * request) { if (status == CANCELLED) @@ -357,42 +377,43 @@ void MPPTask::runImpl() return_statistics.blocks, return_statistics.bytes); } - catch (Exception & e) - { - err_msg = e.displayText(); - LOG_FMT_ERROR(log, "task running meets error: {} Stack Trace : {}", err_msg, e.getStackTrace().toString()); - } - catch (pingcap::Exception & e) - { - err_msg = e.message(); - LOG_FMT_ERROR(log, "task running meets error: {}", err_msg); - } - catch (std::exception & e) - { - err_msg = e.what(); - LOG_FMT_ERROR(log, "task running meets error: {}", err_msg); - } catch (...) { - err_msg = "unrecovered error"; - LOG_FMT_ERROR(log, "task running meets error: {}", err_msg); + err_msg = getCurrentExceptionMessage(true); } + if (err_msg.empty()) { - // todo when error happens, should try to update the metrics if it is available - auto throughput = dag_context->getTableScanThroughput(); - if (throughput.first) - GET_METRIC(tiflash_storage_logical_throughput_bytes).Observe(throughput.second); - auto process_info = context->getProcessListElement()->getInfo(); - auto peak_memory = process_info.peak_memory_usage > 0 ? process_info.peak_memory_usage : 0; - GET_METRIC(tiflash_coprocessor_request_memory_usage, type_run_mpp_task).Observe(peak_memory); - mpp_task_statistics.setMemoryPeak(peak_memory); + if (switchStatus(RUNNING, FINISHED)) + LOG_INFO(log, "finish task"); + else + LOG_FMT_WARNING(log, "finish task which is in {} state", taskStatusToString(status)); + if (status == FINISHED) + { + // todo when error happens, should try to update the metrics if it is available + auto throughput = dag_context->getTableScanThroughput(); + if (throughput.first) + GET_METRIC(tiflash_storage_logical_throughput_bytes).Observe(throughput.second); + auto process_info = context->getProcessListElement()->getInfo(); + auto peak_memory = process_info.peak_memory_usage > 0 ? process_info.peak_memory_usage : 0; + GET_METRIC(tiflash_coprocessor_request_memory_usage, type_run_mpp_task).Observe(peak_memory); + mpp_task_statistics.setMemoryPeak(peak_memory); + } } else { - context->getProcessList().sendCancelToQuery(context->getCurrentQueryId(), context->getClientInfo().current_user, true); - cancelAllReceivers(); - writeErrToAllTunnels(err_msg); + if (status == RUNNING) + { + LOG_FMT_ERROR(log, "task running meets error: {}", err_msg); + try + { + handleError(err_msg); + } + catch (...) + { + tryLogCurrentException(log, "Meet error while try to handle error in MPPTask"); + } + } } LOG_FMT_INFO(log, "task ends, time cost is {} ms.", stopwatch.elapsedMilliseconds()); // unregister flag is only for FailPoint usage, to produce the situation that MPPTask is destructed @@ -405,52 +426,73 @@ void MPPTask::runImpl() if (unregister) unregisterTask(); - if (switchStatus(RUNNING, FINISHED)) - LOG_INFO(log, "finish task"); - else - LOG_WARNING(log, "finish task which was cancelled before"); - - mpp_task_statistics.end(status.load(), err_msg); + mpp_task_statistics.end(status.load(), err_string); mpp_task_statistics.logTracingJson(); } -void MPPTask::writeErrToAllTunnels(const String & e) +void MPPTask::handleError(const String & error_msg) { - RUNTIME_ASSERT(tunnel_set != nullptr, log, "mpp task without tunnel set"); - tunnel_set->writeError(e); + if (manager == nullptr || !manager->isTaskToBeCancelled(id)) + abort(error_msg, AbortType::ONERROR); } -void MPPTask::cancel(const String & reason) +void MPPTask::abort(const String & message, AbortType abort_type) { - CPUAffinityManager::getInstance().bindSelfQueryThread(); - LOG_FMT_WARNING(log, "Begin cancel task: {}", id.toString()); + String abort_type_string; + TaskStatus next_task_status; + switch (abort_type) + { + case AbortType::ONCANCELLATION: + abort_type_string = "ONCANCELLATION"; + next_task_status = CANCELLED; + break; + case AbortType::ONERROR: + abort_type_string = "ONERROR"; + next_task_status = FAILED; + break; + } + LOG_FMT_WARNING(log, "Begin abort task: {}, abort type: {}", id.toString(), abort_type_string); while (true) { auto previous_status = status.load(); - if (previous_status == FINISHED || previous_status == CANCELLED) + if (previous_status == FINISHED || previous_status == CANCELLED || previous_status == FAILED) { - LOG_FMT_WARNING(log, "task already {}", (previous_status == FINISHED ? "finished" : "cancelled")); + LOG_FMT_WARNING(log, "task already in {} state", taskStatusToString(previous_status)); return; } - else if (previous_status == INITIALIZING && switchStatus(INITIALIZING, CANCELLED)) + else if (previous_status == INITIALIZING && switchStatus(INITIALIZING, next_task_status)) { - closeAllTunnels(reason); + err_string = message; + /// if the task is in initializing state, mpp task can return error to TiDB directly, + /// so just close all tunnels here + closeAllTunnels(message); unregisterTask(); - LOG_WARNING(log, "Finish cancel task from uninitialized"); + LOG_WARNING(log, "Finish abort task from uninitialized"); return; } - else if (previous_status == RUNNING && switchStatus(RUNNING, CANCELLED)) + else if (previous_status == RUNNING && switchStatus(RUNNING, next_task_status)) { + /// abort the components from top to bottom because if bottom components are aborted + /// first, the top components may see an error caused by the abort, which is not + /// the original error + err_string = message; + abortTunnels(message, abort_type); + abortDataStreams(abort_type); + abortReceivers(); scheduleThisTask(ScheduleState::FAILED); - context->getProcessList().sendCancelToQuery(context->getCurrentQueryId(), context->getClientInfo().current_user, true); - closeAllTunnels(reason); /// runImpl is running, leave remaining work to runImpl - LOG_WARNING(log, "Finish cancel task from running"); + LOG_WARNING(log, "Finish abort task from running"); return; } } } +void MPPTask::cancel(const String & reason) +{ + CPUAffinityManager::getInstance().bindSelfQueryThread(); + abort(reason, AbortType::ONCANCELLATION); +} + bool MPPTask::switchStatus(TaskStatus from, TaskStatus to) { return status.compare_exchange_strong(from, to); diff --git a/dbms/src/Flash/Mpp/MPPTask.h b/dbms/src/Flash/Mpp/MPPTask.h index d7e5ed169de..a30150b26e8 100644 --- a/dbms/src/Flash/Mpp/MPPTask.h +++ b/dbms/src/Flash/Mpp/MPPTask.h @@ -59,6 +59,8 @@ class MPPTask : public std::enable_shared_from_this void cancel(const String & reason); + void handleError(const String & error_msg); + void prepare(const mpp::DispatchTaskRequest & task_request); void run(); @@ -90,12 +92,22 @@ class MPPTask : public std::enable_shared_from_this void unregisterTask(); - void writeErrToAllTunnels(const String & e); - /// Similar to `writeErrToAllTunnels`, but it just try to write the error message to tunnel /// without waiting the tunnel to be connected void closeAllTunnels(const String & reason); + enum class AbortType + { + /// todo add ONKILL to distinguish between silent cancellation and kill + ONCANCELLATION, + ONERROR, + }; + void abort(const String & message, AbortType abort_type); + + void abortTunnels(const String & message, AbortType abort_type); + void abortReceivers(); + void abortDataStreams(AbortType abort_type); + void finishWrite(); bool switchStatus(TaskStatus from, TaskStatus to); @@ -110,8 +122,6 @@ class MPPTask : public std::enable_shared_from_this void initExchangeReceivers(); - void cancelAllReceivers(); - tipb::DAGRequest dag_req; ContextPtr context; @@ -121,6 +131,7 @@ class MPPTask : public std::enable_shared_from_this MemoryTracker * memory_tracker = nullptr; std::atomic status{INITIALIZING}; + String err_string; mpp::TaskMeta meta; @@ -138,8 +149,6 @@ class MPPTask : public std::enable_shared_from_this MPPTaskStatistics mpp_task_statistics; - Exception err; - friend class MPPTaskManager; int needed_threads; diff --git a/dbms/src/Flash/Mpp/MPPTaskManager.cpp b/dbms/src/Flash/Mpp/MPPTaskManager.cpp index 3df4af5de5f..c5499eda89d 100644 --- a/dbms/src/Flash/Mpp/MPPTaskManager.cpp +++ b/dbms/src/Flash/Mpp/MPPTaskManager.cpp @@ -147,6 +147,17 @@ bool MPPTaskManager::registerTask(MPPTaskPtr task) return true; } +bool MPPTaskManager::isTaskToBeCancelled(const MPPTaskId & task_id) +{ + std::unique_lock lock(mu); + auto it = mpp_query_map.find(task_id.start_ts); + if (it != mpp_query_map.end() && it->second->to_be_cancelled) + { + return it->second->task_map.find(task_id) != it->second->task_map.end(); + } + return false; +} + void MPPTaskManager::unregisterTask(MPPTask * task) { std::unique_lock lock(mu); diff --git a/dbms/src/Flash/Mpp/MPPTaskManager.h b/dbms/src/Flash/Mpp/MPPTaskManager.h index d7047804aca..770acea3853 100644 --- a/dbms/src/Flash/Mpp/MPPTaskManager.h +++ b/dbms/src/Flash/Mpp/MPPTaskManager.h @@ -73,6 +73,8 @@ class MPPTaskManager : private boost::noncopyable void unregisterTask(MPPTask * task); + bool isTaskToBeCancelled(const MPPTaskId & task_id); + bool tryToScheduleTask(const MPPTaskPtr & task); void releaseThreadsFromScheduler(const int needed_threads); diff --git a/dbms/src/Flash/Mpp/TaskStatus.cpp b/dbms/src/Flash/Mpp/TaskStatus.cpp index 423b768faea..c87ae2b8eb4 100644 --- a/dbms/src/Flash/Mpp/TaskStatus.cpp +++ b/dbms/src/Flash/Mpp/TaskStatus.cpp @@ -29,6 +29,8 @@ StringRef taskStatusToString(const TaskStatus & status) return "FINISHED"; case CANCELLED: return "CANCELLED"; + case FAILED: + return "FAILED"; default: throw Exception("Unknown TaskStatus"); } diff --git a/dbms/src/Flash/Mpp/TaskStatus.h b/dbms/src/Flash/Mpp/TaskStatus.h index 999e30790bf..0997c8adc52 100644 --- a/dbms/src/Flash/Mpp/TaskStatus.h +++ b/dbms/src/Flash/Mpp/TaskStatus.h @@ -24,6 +24,7 @@ enum TaskStatus RUNNING, FINISHED, CANCELLED, + FAILED, }; StringRef taskStatusToString(const TaskStatus & status); diff --git a/tests/fullstack-test/mpp/issue_2471.test b/tests/fullstack-test/mpp/issue_2471.test index 4a1528595e8..497ce605893 100644 --- a/tests/fullstack-test/mpp/issue_2471.test +++ b/tests/fullstack-test/mpp/issue_2471.test @@ -35,7 +35,15 @@ mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_opt_bro => DBGInvoke __enable_fail_point(exception_in_creating_set_input_stream) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_opt_broadcast_cartesian_join=2; select * from a as t1 left join a as t2 on t1.id = t2.id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: DB::Exception: Fail point FailPoints::exception_in_creating_set_input_stream is triggered. +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_in_creating_set_input_stream is triggered., e.what() = DB::Exception, Stack trace: +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} => DBGInvoke __disable_fail_point(exception_in_creating_set_input_stream) diff --git a/tests/fullstack-test/mpp/mpp_fail.test b/tests/fullstack-test/mpp/mpp_fail.test index 7af5fef3f89..e03c6150be6 100644 --- a/tests/fullstack-test/mpp/mpp_fail.test +++ b/tests/fullstack-test/mpp/mpp_fail.test @@ -71,20 +71,44 @@ ERROR 1105 (HY000) at line 1: DB::Exception: Fail point FailPoints::exception_be ## exception during mpp run non root task => DBGInvoke __enable_fail_point(exception_during_mpp_non_root_task_run) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: DB::Exception: Exchange receiver meet error : DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered. +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered., e.what() = DB::Exception, Stack trace: +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} => DBGInvoke __disable_fail_point(exception_during_mpp_non_root_task_run) ## exception during mpp run root task => DBGInvoke __enable_fail_point(exception_during_mpp_root_task_run) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: DB::Exception: Fail point FailPoints::exception_during_mpp_root_task_run is triggered. +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_root_task_run is triggered., e.what() = DB::Exception, Stack trace: +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} => DBGInvoke __disable_fail_point(exception_during_mpp_root_task_run) ## exception during mpp write err to tunnel => DBGInvoke __enable_fail_point(exception_during_mpp_non_root_task_run) => DBGInvoke __enable_fail_point(exception_during_mpp_write_err_to_tunnel) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: DB::Exception: Exchange receiver meet error : Failed to write error msg to tunnel +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Failed to write error msg to tunnel, e.what() = DB::Exception, Stack trace: +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} => DBGInvoke __disable_fail_point(exception_during_mpp_non_root_task_run) => DBGInvoke __disable_fail_point(exception_during_mpp_write_err_to_tunnel) @@ -92,7 +116,14 @@ ERROR 1105 (HY000) at line 1: other error for mpp stream: DB::Exception: Exchang => DBGInvoke __enable_fail_point(exception_during_mpp_non_root_task_run) => DBGInvoke __enable_fail_point(exception_during_mpp_close_tunnel) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: DB::Exception: Exchange receiver meet error : DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered. +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered., e.what() = DB::Exception, Stack trace: +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} => DBGInvoke __disable_fail_point(exception_during_mpp_non_root_task_run) => DBGInvoke __disable_fail_point(exception_during_mpp_close_tunnel) @@ -125,7 +156,16 @@ ERROR 1105 (HY000) at line 1: other error for mpp stream: DB::Exception: Exchang ## ensure build1, build2-probe1, probe2 in the CreatingSets, test the bug where build1 throw exception but not change the build state, thus block the build2-probe1, at last this query hangs. => DBGInvoke __enable_fail_point(exception_mpp_hash_build) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; set @@tidb_broadcast_join_threshold_count=0; set @@tidb_broadcast_join_threshold_size=0; select t1.id from test.t t1 join test.t t2 on t1.id = t2.id and t1.id <2 join (select id from test.t group by id) t3 on t2.id=t3.id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: DB::Exception: Fail point FailPoints::exception_mpp_hash_build is triggered. +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_mpp_hash_build is triggered., e.what() = DB::Exception, Stack trace: +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} +{#LINE} => DBGInvoke __disable_fail_point(exception_mpp_hash_build) # Clean up. diff --git a/tests/run-test.py b/tests/run-test.py index 843fe7c79b4..a2bcee0ce99 100644 --- a/tests/run-test.py +++ b/tests/run-test.py @@ -29,6 +29,7 @@ UNFINISHED_1_PREFIX = '\t' UNFINISHED_2_PREFIX = ' ' WORD_PH = '{#WORD}' +LINE_PH = '{#LINE}' CURL_TIDB_STATUS_PREFIX = 'curl_tidb> ' verbose = False @@ -138,18 +139,22 @@ def match_ph_word(line): # TODO: Support more place holders, eg: {#NUMBER} def compare_line(line, template): - while True: - i = template.find(WORD_PH) - if i < 0: - return line == template - else: - if line[:i] != template[:i]: - return False - j = match_ph_word(line[i:]) - if j == 0: - return False - template = template[i + len(WORD_PH):] - line = line[i + j:] + l = template.find(LINE_PH) + if l >= 0: + return True + else: + while True: + i = template.find(WORD_PH) + if i < 0: + return line == template + else: + if line[:i] != template[:i]: + return False + j = match_ph_word(line[i:]) + if j == 0: + return False + template = template[i + len(WORD_PH):] + line = line[i + j:] class MySQLCompare: @@ -194,11 +199,14 @@ def matched(outputs, matches): b = MySQLCompare.parse_excepted_outputs(matches) return a == b else: - if len(outputs) != len(matches): + if len(outputs) > len(matches): return False for i in range(0, len(outputs)): if not compare_line(outputs[i], matches[i]): return False + for i in range(len(outputs), len(matches)): + if not compare_line("", matches[i]): + return False return True @@ -212,11 +220,14 @@ def matched(outputs, matches, fuzz): b = parse_table_parts(matches, fuzz) return a == b else: - if len(outputs) != len(matches): + if len(outputs) > len(matches): return False for i in range(0, len(outputs)): if not compare_line(outputs[i], matches[i]): return False + for i in range(len(outputs), len(matches)): + if not compare_line("", matches[i]): + return False return True From 31a9611b6f6abf8004c20f616cfbd121fd68d747 Mon Sep 17 00:00:00 2001 From: hehechen Date: Wed, 29 Jun 2022 20:00:40 +0800 Subject: [PATCH 064/104] Check result of starting grpc server (#5257) close pingcap/tiflash#5255 --- dbms/src/Server/Server.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 571ba8fe3a5..477844cd509 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -185,6 +185,7 @@ extern const int NO_ELEMENTS_IN_CONFIG; extern const int SUPPORT_IS_DISABLED; extern const int ARGUMENT_OUT_OF_BOUND; extern const int INVALID_CONFIG_PARAMETER; +extern const int IP_ADDRESS_NOT_ALLOWED; } // namespace ErrorCodes namespace Debug @@ -622,6 +623,10 @@ class Server::FlashGrpcServerHolder } } flash_grpc_server = builder.BuildAndStart(); + if (!flash_grpc_server) + { + throw Exception("Exception happens when start grpc server, the flash.service_addr may be invalid, flash.service_addr is " + raft_config.flash_server_addr, ErrorCodes::IP_ADDRESS_NOT_ALLOWED); + } LOG_FMT_INFO(log, "Flash grpc server listening on [{}]", raft_config.flash_server_addr); Debug::setServiceAddr(raft_config.flash_server_addr); if (enable_async_server) From 1ff3b38633149dfe9cd0dff63afe89f75d15eed2 Mon Sep 17 00:00:00 2001 From: Schrodinger ZHU Yifan Date: Wed, 29 Jun 2022 20:26:40 +0800 Subject: [PATCH 065/104] feat: add optimized routines for aarch64 (#5231) close pingcap/tiflash#5240 --- .gitmodules | 3 + contrib/CMakeLists.txt | 4 + contrib/arm-optimized-routines | 1 + .../CMakeLists.txt | 40 ++++++ .../arm-optimized-routines-cmake/src/aor.c | 115 ++++++++++++++++++ dbms/src/Server/Server.cpp | 7 +- libs/libcommon/CMakeLists.txt | 4 + 7 files changed, 172 insertions(+), 2 deletions(-) create mode 160000 contrib/arm-optimized-routines create mode 100644 contrib/arm-optimized-routines-cmake/CMakeLists.txt create mode 100644 contrib/arm-optimized-routines-cmake/src/aor.c diff --git a/.gitmodules b/.gitmodules index 8472d78404e..335e1dbd9c8 100644 --- a/.gitmodules +++ b/.gitmodules @@ -82,3 +82,6 @@ [submodule "contrib/cpu_features"] path = contrib/cpu_features url = https://github.com/google/cpu_features +[submodule "contrib/arm-optimized-routines"] + path = contrib/arm-optimized-routines + url = https://github.com/ARM-software/optimized-routines diff --git a/contrib/CMakeLists.txt b/contrib/CMakeLists.txt index 71f81ae3ee5..4520d1cb176 100644 --- a/contrib/CMakeLists.txt +++ b/contrib/CMakeLists.txt @@ -165,3 +165,7 @@ add_subdirectory(benchmark) set (BUILD_TESTING OFF CACHE BOOL "Disable cpu-features testing" FORCE) add_subdirectory(cpu_features) + +if (ARCH_AARCH64 AND ARCH_LINUX) + add_subdirectory(arm-optimized-routines-cmake) +endif () diff --git a/contrib/arm-optimized-routines b/contrib/arm-optimized-routines new file mode 160000 index 00000000000..e373f659523 --- /dev/null +++ b/contrib/arm-optimized-routines @@ -0,0 +1 @@ +Subproject commit e373f6595230087a8ddea449bfb14b47150b4059 diff --git a/contrib/arm-optimized-routines-cmake/CMakeLists.txt b/contrib/arm-optimized-routines-cmake/CMakeLists.txt new file mode 100644 index 00000000000..b08a761fe62 --- /dev/null +++ b/contrib/arm-optimized-routines-cmake/CMakeLists.txt @@ -0,0 +1,40 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# This library is to override performance-critical routines for aarch64 targets. +# The implementations are imported from official ARM repo. +# To reduce dispatching cost, indirect function technique is utilized. Therefore, +# this library should only be enabled with ELF targets. + +# Considerations: +# - By Jun, 2022, most enterprise OSs (CentOS 7, CentOS Stream 8 and RHEL 8) still +# use relatively old glibc on ARM64, where ASIMD, MTE, DC ZVA and SVE are not +# fully utilized. However, it is becoming increasingly common to use ARM64 instances +# in cloud-native situations. +# - `optimized-routines` repo is actively maintained by ARM officials. Therefore, +# the qualities can be ensured while using it also enables us to keep sync with latest +# acceleration techniques. + +ENABLE_LANGUAGE(C) +ENABLE_LANGUAGE(ASM) +set(TIFLASH_AOR_DIR ../arm-optimized-routines) + +file(GLOB TIFLASH_AARCH64_STRING_FILES ${TIFLASH_AOR_DIR}/string/aarch64/*.S) +add_library(tiflash-aarch64-string STATIC ${TIFLASH_AARCH64_STRING_FILES} src/aor.c) +target_compile_options(tiflash-aarch64-string PRIVATE -march=armv8-a+sve) +target_include_directories(tiflash-aarch64-string PRIVATE ${TIFLASH_AOR_DIR}/string/include) + +file(GLOB TIFLASH_AARCH64_MATH_FILES ${TIFLASH_AOR_DIR}/math/*.c) +add_library(tiflash-aarch64-math STATIC ${TIFLASH_AARCH64_MATH_FILES}) +target_include_directories(tiflash-aarch64-math PRIVATE ${TIFLASH_AOR_DIR}/math/include) \ No newline at end of file diff --git a/contrib/arm-optimized-routines-cmake/src/aor.c b/contrib/arm-optimized-routines-cmake/src/aor.c new file mode 100644 index 00000000000..daff1df3c4b --- /dev/null +++ b/contrib/arm-optimized-routines-cmake/src/aor.c @@ -0,0 +1,115 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include + +// Provide default macro definitions in case that they are not defined on current linux distro. +// For example, TiFlash compiled on older linux kernels may also be used in newer ones. +// These values should be stable for Linux: only false negative is expected when running on +// older kernels, but it is acceptable as `google/cpu_features` is also doing so. +#ifndef HWCAP2_MTE +#define HWCAP2_MTE (1 << 18) +#endif + +#ifndef HWCAP_SVE +#define HWCAP_SVE (1 << 22) +#endif + +#ifndef AT_HWCAP2 +#define AT_HWCAP2 26 +#endif + +#ifndef AT_HWCAP +#define AT_HWCAP 16 +#endif + +/// check if MTE is supported in current environment +static inline bool mte_supported(void) +{ + return (getauxval(AT_HWCAP2) & HWCAP2_MTE) != 0; +} + +/// check if SVE is supported in current environment +static inline bool sve_supported(void) +{ + return (getauxval(AT_HWCAP) & HWCAP_SVE) != 0; +} + +#define STRINGIFY_IMPL(X) #X +#define STRINGIFY(X) STRINGIFY_IMPL(X) +/** + * \brief + * Symbol is defined as hidden visibility. Therefore, implementations here are only to override routines with TiFlash + * binary itself. This is because dependencies like `ld.so`, `libgcc_s.so`, etc will need essential routines like + * `memcpy` to finish the early loading procedure. Therefore, declare such symbols as visible indirect function will + * create cyclic dependency. It shall be good enough to override symbols within TiFlash, as most heavy computation works + * are happening in the main binary. + * \param NAME: exported symbol name + * \param SVE: preferred implementation when SVE is available + * \param MTE: preferred implementation when MTE is available + * \param ASIMD: preferred implementation for generic aarch64 targets (ASIMD is required by default for Armv8 and above) + */ +#define DISPATCH(NAME, SVE, MTE, ASIMD) \ + extern typeof(ASIMD) __tiflash_##NAME __attribute__((ifunc(STRINGIFY(__tiflash_##NAME##_resolver)))); \ + extern typeof(ASIMD) NAME __attribute__((visibility("hidden"), alias(STRINGIFY(__tiflash_##NAME)))); \ + _Pragma("GCC diagnostic push") \ + _Pragma("GCC diagnostic ignored \"-Wunused-function\"") static typeof(ASIMD) * __tiflash_##NAME##_resolver(void) \ + { \ + if (sve_supported()) \ + { \ + return SVE; \ + } \ + if (mte_supported()) \ + { \ + return MTE; \ + } \ + return ASIMD; \ + } \ + _Pragma("GCC diagnostic pop") +#undef memcpy +#undef memmove +#undef memset +#undef memchr +#undef memrchr +#undef memcmp +#undef strcpy +#undef stpcpy +#undef strcmp +#undef strchr +#undef strrchr +#undef strchrnul +#undef strlen +#undef strnlen +#undef strncmp + +DISPATCH(memcpy, __memcpy_aarch64_sve, __memcpy_aarch64_simd, __memcpy_aarch64_simd) +DISPATCH(memmove, __memmove_aarch64_sve, __memmove_aarch64_simd, __memmove_aarch64_simd) +DISPATCH(memset, __memset_aarch64, __memset_aarch64, __memset_aarch64) +DISPATCH(memchr, __memchr_aarch64_sve, __memchr_aarch64_mte, __memchr_aarch64) +DISPATCH(memrchr, __memrchr_aarch64, __memrchr_aarch64, __memrchr_aarch64) +DISPATCH(memcmp, __memcmp_aarch64_sve, __memcmp_aarch64, __memcmp_aarch64) +DISPATCH(strcpy, __strcpy_aarch64_sve, __strcpy_aarch64, __strcpy_aarch64) +DISPATCH(stpcpy, __stpcpy_aarch64_sve, __stpcpy_aarch64, __stpcpy_aarch64) +DISPATCH(strcmp, __strcmp_aarch64_sve, __strcmp_aarch64, __strcmp_aarch64) +DISPATCH(strchr, __strchr_aarch64_sve, __strchr_aarch64_mte, __strchr_aarch64) +DISPATCH(strrchr, __strrchr_aarch64_sve, __strrchr_aarch64_mte, __strrchr_aarch64) +DISPATCH(strchrnul, __strchrnul_aarch64_sve, __strchrnul_aarch64_mte, __strchrnul_aarch64) +DISPATCH(strlen, __strlen_aarch64_sve, __strlen_aarch64_mte, __strlen_aarch64) +DISPATCH(strnlen, __strnlen_aarch64_sve, __strnlen_aarch64, __strnlen_aarch64) +DISPATCH(strncmp, __strncmp_aarch64_sve, __strncmp_aarch64, __strncmp_aarch64) \ No newline at end of file diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 477844cd509..3358ae2a60e 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -152,6 +152,7 @@ void loadMiConfig(Logger * log) } #undef TRY_LOAD_CONF #endif + namespace { [[maybe_unused]] void tryLoadBoolConfigFromEnv(Poco::Logger * log, bool & target, const char * name) @@ -967,7 +968,10 @@ class Server::TcpHttpServersHolder LOG_DEBUG(log, debug_msg); } - const std::vector> & getServers() const { return servers; } + const std::vector> & getServers() const + { + return servers; + } private: Server & server; @@ -1003,7 +1007,6 @@ int Server::main(const std::vector & /*args*/) #ifdef TIFLASH_ENABLE_SVE_SUPPORT tryLoadBoolConfigFromEnv(log, simd_option::ENABLE_SVE, "TIFLASH_ENABLE_SVE"); #endif - registerFunctions(); registerAggregateFunctions(); registerWindowFunctions(); diff --git a/libs/libcommon/CMakeLists.txt b/libs/libcommon/CMakeLists.txt index 5fd25c5d238..2bedb312d07 100644 --- a/libs/libcommon/CMakeLists.txt +++ b/libs/libcommon/CMakeLists.txt @@ -198,3 +198,7 @@ if (ARCH_AMD64) src/crc64_sse2_asimd.cpp APPEND COMPILE_FLAGS "-mpclmul") endif() + +if (ARCH_AARCH64 AND ARCH_LINUX) + target_link_libraries (common PUBLIC tiflash-aarch64-string tiflash-aarch64-math) +endif() From cbe6ab5593926ba82d15f8b2ac398fd98dff56bc Mon Sep 17 00:00:00 2001 From: Schrodinger ZHU Yifan Date: Thu, 30 Jun 2022 11:22:39 +0800 Subject: [PATCH 066/104] fix: aarch64-quick-fix (#5259) close pingcap/tiflash#5260 --- contrib/arm-optimized-routines-cmake/CMakeLists.txt | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/contrib/arm-optimized-routines-cmake/CMakeLists.txt b/contrib/arm-optimized-routines-cmake/CMakeLists.txt index b08a761fe62..89baa7222f3 100644 --- a/contrib/arm-optimized-routines-cmake/CMakeLists.txt +++ b/contrib/arm-optimized-routines-cmake/CMakeLists.txt @@ -26,6 +26,7 @@ # the qualities can be ensured while using it also enables us to keep sync with latest # acceleration techniques. +set(CMAKE_C_FLAGS "") ENABLE_LANGUAGE(C) ENABLE_LANGUAGE(ASM) set(TIFLASH_AOR_DIR ../arm-optimized-routines) @@ -37,4 +38,8 @@ target_include_directories(tiflash-aarch64-string PRIVATE ${TIFLASH_AOR_DIR}/str file(GLOB TIFLASH_AARCH64_MATH_FILES ${TIFLASH_AOR_DIR}/math/*.c) add_library(tiflash-aarch64-math STATIC ${TIFLASH_AARCH64_MATH_FILES}) -target_include_directories(tiflash-aarch64-math PRIVATE ${TIFLASH_AOR_DIR}/math/include) \ No newline at end of file +target_include_directories(tiflash-aarch64-math PRIVATE ${TIFLASH_AOR_DIR}/math/include) + +# it is reasonable to keep these libraries optimized +target_compile_options(tiflash-aarch64-string PRIVATE -O3 -g3 -fno-omit-frame-pointer -ffunction-sections -fdata-sections) +target_compile_options(tiflash-aarch64-math PRIVATE -O3 -g3 -fno-omit-frame-pointer -ffunction-sections -fdata-sections) From 045d24fe854193ff7fe9e56f0ed1ff0e4aa1cddc Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Fri, 1 Jul 2022 15:58:40 +0800 Subject: [PATCH 067/104] Update client-c to support ipv6 (#5270) close pingcap/tiflash#5247 --- contrib/client-c | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/client-c b/contrib/client-c index 36e05cb0f24..034d1e782cb 160000 --- a/contrib/client-c +++ b/contrib/client-c @@ -1 +1 @@ -Subproject commit 36e05cb0f24c085785abf367176dac2a45bfd67b +Subproject commit 034d1e782cb4697f99b09b679c00dade00f19dd5 From 38b37e079bf499ade14fff2eccd9dfec4963c9cc Mon Sep 17 00:00:00 2001 From: YangKeao Date: Sun, 3 Jul 2022 22:43:00 +0800 Subject: [PATCH 068/104] upgrade prometheus-cpp to v1.0.1 (#5279) ref pingcap/tiflash#2103, close pingcap/tiflash#5278 --- contrib/prometheus-cpp | 2 +- contrib/prometheus-cpp-cmake/pull/CMakeLists.txt | 9 +++++++++ contrib/prometheus-cpp-cmake/push/CMakeLists.txt | 2 ++ 3 files changed, 12 insertions(+), 1 deletion(-) diff --git a/contrib/prometheus-cpp b/contrib/prometheus-cpp index ca1f3463e74..76470b3ec02 160000 --- a/contrib/prometheus-cpp +++ b/contrib/prometheus-cpp @@ -1 +1 @@ -Subproject commit ca1f3463e74d957d1cccddd4a1a29e3e5d34bd83 +Subproject commit 76470b3ec024c8214e1f4253fb1f4c0b28d3df94 diff --git a/contrib/prometheus-cpp-cmake/pull/CMakeLists.txt b/contrib/prometheus-cpp-cmake/pull/CMakeLists.txt index daebd1b7c5a..993618e16ac 100644 --- a/contrib/prometheus-cpp-cmake/pull/CMakeLists.txt +++ b/contrib/prometheus-cpp-cmake/pull/CMakeLists.txt @@ -12,9 +12,18 @@ if(ENABLE_COMPRESSION) endif() add_library(pull + ${PROMETHEUS_SRC_DIR}/pull/src/basic_auth.cc + ${PROMETHEUS_SRC_DIR}/pull/src/basic_auth.h + ${PROMETHEUS_SRC_DIR}/pull/src/endpoint.cc + ${PROMETHEUS_SRC_DIR}/pull/src/endpoint.h ${PROMETHEUS_SRC_DIR}/pull/src/exposer.cc ${PROMETHEUS_SRC_DIR}/pull/src/handler.cc ${PROMETHEUS_SRC_DIR}/pull/src/handler.h + ${PROMETHEUS_SRC_DIR}/pull/src/metrics_collector.cc + ${PROMETHEUS_SRC_DIR}/pull/src/metrics_collector.h + + ${PROMETHEUS_SRC_DIR}/pull/src/detail/base64.h + $<$:$> ) diff --git a/contrib/prometheus-cpp-cmake/push/CMakeLists.txt b/contrib/prometheus-cpp-cmake/push/CMakeLists.txt index 71dad9fb812..b776d17bdaf 100644 --- a/contrib/prometheus-cpp-cmake/push/CMakeLists.txt +++ b/contrib/prometheus-cpp-cmake/push/CMakeLists.txt @@ -3,6 +3,8 @@ if(NOT CURL_FOUND) endif() add_library(push + ${PROMETHEUS_SRC_DIR}/push/src/curl_wrapper.cc + ${PROMETHEUS_SRC_DIR}/push/src/curl_wrapper.h ${PROMETHEUS_SRC_DIR}/push/src/gateway.cc ) From 4ce641b2706d3ffcb41d6abfb019292e1b8ee550 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Sun, 3 Jul 2022 23:08:59 +0800 Subject: [PATCH 069/104] Fix README type error (#5273) ref pingcap/tiflash#5178 --- README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/README.md b/README.md index aa64e39d5ba..ab996b6f3d6 100644 --- a/README.md +++ b/README.md @@ -253,7 +253,7 @@ ninja tiflash tiup playground nightly --tiflash.binpath $BUILD/dbms/src/Server/tiflash ``` 3. Check $WORKSPACE/tests/_env.sh to make the port and build dir right. -4. Run your integration tests using commands like "./run-test.sh fullstack-test2/ddl" under $WORKSPACE dir +4. Run your integration tests using commands like "./run-test.sh fullstack-test2/ddl" under $WORKSPACE/tests dir ## Run MicroBenchmark Tests @@ -261,7 +261,7 @@ To run micro benchmark tests, you need to build with -DCMAKE_BUILD_TYPE=RELEASE ```shell cd $BUILD -cmake $WORKSPACE/tiflash -GNinja -DCMAKE_BUILD_TYPE=DEBUG -DENABLE_TESTS=ON +cmake $WORKSPACE/tiflash -GNinja -DCMAKE_BUILD_TYPE=RELEASE -DENABLE_TESTS=ON ninja bench_dbms ``` From 19dfdd792215c1c593983d66ee0f4c5865fc394c Mon Sep 17 00:00:00 2001 From: Schrodinger ZHU Yifan Date: Mon, 4 Jul 2022 11:15:00 +0800 Subject: [PATCH 070/104] fix(cmake): make sure libc++ is utilized by tiflash-proxy (#5281) close pingcap/tiflash#5282 --- contrib/tiflash-proxy-cmake/CMakeLists.txt | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/contrib/tiflash-proxy-cmake/CMakeLists.txt b/contrib/tiflash-proxy-cmake/CMakeLists.txt index e243ecba37c..e3e2df379a1 100644 --- a/contrib/tiflash-proxy-cmake/CMakeLists.txt +++ b/contrib/tiflash-proxy-cmake/CMakeLists.txt @@ -4,7 +4,11 @@ file(GLOB_RECURSE _TIFLASH_PROXY_SRCS "${_TIFLASH_PROXY_SOURCE_DIR}/*.rs") list(FILTER _TIFLASH_PROXY_SRCS EXCLUDE REGEX ${_TIFLASH_PROXY_SOURCE_DIR}/target/.*) # use `CFLAGS=-w CXXFLAGS=-w` to inhibit warning messages. -set(TIFLASH_RUST_ENV CMAKE=${CMAKE_COMMAND} CFLAGS=-w CXXFLAGS=-w) +if (TIFLASH_LLVM_TOOLCHAIN) + set(TIFLASH_RUST_ENV CMAKE=${CMAKE_COMMAND} "CFLAGS=-w -fuse-ld=lld" "CXXFLAGS=-w -fuse-ld=lld -stdlib=libc++") +else() + set(TIFLASH_RUST_ENV CMAKE=${CMAKE_COMMAND} CFLAGS=-w CXXFLAGS=-w) +endif() if(TIFLASH_LLVM_TOOLCHAIN AND USE_LIBCXX) set(TIFLASH_RUST_LINKER ${CMAKE_CURRENT_BINARY_DIR}/tiflash-linker) From 09402e36aefb84f82bb10a12a3c5e37d83d01733 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 4 Jul 2022 11:47:00 +0800 Subject: [PATCH 071/104] fix the wrong order of execution summary for list based executors (#5242) close pingcap/tiflash#5241 --- dbms/src/Flash/Coprocessor/DAGContext.h | 6 +++- dbms/src/Flash/Coprocessor/DAGQuerySource.cpp | 23 +++++++++++++ .../Flash/Coprocessor/DAGResponseWriter.cpp | 33 ++++++++++++++----- 3 files changed, 53 insertions(+), 9 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 8b94d4637a8..a50a4d4007b 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -349,6 +349,10 @@ class DAGContext std::vector output_field_types; std::vector output_offsets; + /// Hold the order of list based executors. + /// It is used to ensure that the order of Execution summary of list based executors is the same as the order of list based executors. + std::vector list_based_executors_order; + private: void initExecutorIdToJoinIdMap(); void initOutputInfo(); @@ -356,7 +360,7 @@ class DAGContext private: /// Hold io for correcting the destruction order. BlockIO io; - /// profile_streams_map is a map that maps from executor_id to profile BlockInputStreams + /// profile_streams_map is a map that maps from executor_id to profile BlockInputStreams. std::unordered_map profile_streams_map; /// executor_id_to_join_id_map is a map that maps executor id to all the join executor id of itself and all its children. std::unordered_map> executor_id_to_join_id_map; diff --git a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp index 882699e1599..d68a7b17aaa 100644 --- a/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQuerySource.cpp @@ -20,6 +20,26 @@ namespace DB { +namespace +{ +void fillOrderForListBasedExecutors(DAGContext & dag_context, const DAGQueryBlock & query_block) +{ + assert(query_block.source); + auto & list_based_executors_order = dag_context.list_based_executors_order; + list_based_executors_order.push_back(query_block.source_name); + if (query_block.selection) + list_based_executors_order.push_back(query_block.selection_name); + if (query_block.aggregation) + list_based_executors_order.push_back(query_block.aggregation_name); + if (query_block.having) + list_based_executors_order.push_back(query_block.having_name); + if (query_block.limit_or_topn) + list_based_executors_order.push_back(query_block.limit_or_topn_name); + if (query_block.exchange_sender) + dag_context.list_based_executors_order.push_back(query_block.exchange_sender_name); +} +} // namespace + DAGQuerySource::DAGQuerySource(Context & context_) : context(context_) { @@ -32,6 +52,9 @@ DAGQuerySource::DAGQuerySource(Context & context_) else { root_query_block = std::make_shared(1, dag_request.executors()); + auto & dag_context = getDAGContext(); + if (!dag_context.return_executor_id) + fillOrderForListBasedExecutors(dag_context, *root_query_block); } } diff --git a/dbms/src/Flash/Coprocessor/DAGResponseWriter.cpp b/dbms/src/Flash/Coprocessor/DAGResponseWriter.cpp index 53bebc91da8..33f6d99f9d8 100644 --- a/dbms/src/Flash/Coprocessor/DAGResponseWriter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGResponseWriter.cpp @@ -89,12 +89,10 @@ void DAGResponseWriter::addExecuteSummaries(tipb::SelectResponse & response, boo } } - /// add execution_summary for local executor - for (auto & p : dag_context.getProfileStreamsMap()) - { + auto fill_execution_summary = [&](const String & executor_id, const BlockInputStreams & streams) { ExecutionSummary current; /// part 1: local execution info - for (auto & stream_ptr : p.second) + for (const auto & stream_ptr : streams) { if (auto * p_stream = dynamic_cast(stream_ptr.get())) { @@ -105,16 +103,16 @@ void DAGResponseWriter::addExecuteSummaries(tipb::SelectResponse & response, boo current.concurrency++; } /// part 2: remote execution info - if (merged_remote_execution_summaries.find(p.first) != merged_remote_execution_summaries.end()) + if (merged_remote_execution_summaries.find(executor_id) != merged_remote_execution_summaries.end()) { - for (auto & remote : merged_remote_execution_summaries[p.first]) + for (auto & remote : merged_remote_execution_summaries[executor_id]) current.merge(remote, false); } /// part 3: for join need to add the build time /// In TiFlash, a hash join's build side is finished before probe side starts, /// so the join probe side's running time does not include hash table's build time, /// when construct ExecSummaries, we need add the build cost to probe executor - auto all_join_id_it = dag_context.getExecutorIdToJoinIdMap().find(p.first); + auto all_join_id_it = dag_context.getExecutorIdToJoinIdMap().find(executor_id); if (all_join_id_it != dag_context.getExecutorIdToJoinIdMap().end()) { for (const auto & join_executor_id : all_join_id_it->second) @@ -138,8 +136,27 @@ void DAGResponseWriter::addExecuteSummaries(tipb::SelectResponse & response, boo } current.time_processed_ns += dag_context.compile_time_ns; - fillTiExecutionSummary(response.add_execution_summaries(), current, p.first, delta_mode); + fillTiExecutionSummary(response.add_execution_summaries(), current, executor_id, delta_mode); + }; + + /// add execution_summary for local executor + if (dag_context.return_executor_id) + { + for (auto & p : dag_context.getProfileStreamsMap()) + fill_execution_summary(p.first, p.second); + } + else + { + const auto & profile_streams_map = dag_context.getProfileStreamsMap(); + assert(profile_streams_map.size() == dag_context.list_based_executors_order.size()); + for (const auto & executor_id : dag_context.list_based_executors_order) + { + auto it = profile_streams_map.find(executor_id); + assert(it != profile_streams_map.end()); + fill_execution_summary(executor_id, it->second); + } } + for (auto & p : merged_remote_execution_summaries) { if (local_executors.find(p.first) == local_executors.end()) From a89222abd6ad1934c54838531532a1253bf4f66c Mon Sep 17 00:00:00 2001 From: jiaqizho Date: Mon, 4 Jul 2022 12:27:00 +0800 Subject: [PATCH 072/104] Schema: allow loading empty schema diff when the version grows up. (#5245) close pingcap/tiflash#5244 --- dbms/src/Debug/MockSchemaGetter.h | 13 ++- dbms/src/Debug/MockTiDB.cpp | 7 +- dbms/src/Debug/MockTiDB.h | 4 +- .../Storages/Transaction/ReadIndexWorker.cpp | 2 +- dbms/src/TiDB/Schema/SchemaGetter.cpp | 13 ++- dbms/src/TiDB/Schema/SchemaGetter.h | 6 +- dbms/src/TiDB/Schema/TiDBSchemaSyncer.h | 79 ++++++++++++++----- 7 files changed, 96 insertions(+), 28 deletions(-) diff --git a/dbms/src/Debug/MockSchemaGetter.h b/dbms/src/Debug/MockSchemaGetter.h index f02699866ce..11c5d97f036 100644 --- a/dbms/src/Debug/MockSchemaGetter.h +++ b/dbms/src/Debug/MockSchemaGetter.h @@ -17,16 +17,25 @@ #include #include +#include + namespace DB { - struct MockSchemaGetter { TiDB::DBInfoPtr getDatabase(DatabaseID db_id) { return MockTiDB::instance().getDBInfoByID(db_id); } Int64 getVersion() { return MockTiDB::instance().getVersion(); } - SchemaDiff getSchemaDiff(Int64 version) { return MockTiDB::instance().getSchemaDiff(version); } + std::optional getSchemaDiff(Int64 version) + { + return MockTiDB::instance().getSchemaDiff(version); + } + + bool checkSchemaDiffExists(Int64 version) + { + return MockTiDB::instance().checkSchemaDiffExists(version); + } TiDB::TableInfoPtr getTableInfo(DatabaseID, TableID table_id) { return MockTiDB::instance().getTableInfoByID(table_id); } diff --git a/dbms/src/Debug/MockTiDB.cpp b/dbms/src/Debug/MockTiDB.cpp index 7b3bdb0948f..99d9625461b 100644 --- a/dbms/src/Debug/MockTiDB.cpp +++ b/dbms/src/Debug/MockTiDB.cpp @@ -668,9 +668,14 @@ std::pair MockTiDB::getDBIDByName(const String & database_name return std::make_pair(false, -1); } -SchemaDiff MockTiDB::getSchemaDiff(Int64 version_) +std::optional MockTiDB::getSchemaDiff(Int64 version_) { return version_diff[version_]; } +bool MockTiDB::checkSchemaDiffExists(Int64 version) +{ + return version_diff.find(version) != version_diff.end(); +} + } // namespace DB diff --git a/dbms/src/Debug/MockTiDB.h b/dbms/src/Debug/MockTiDB.h index 36d2af90859..261e547b13a 100644 --- a/dbms/src/Debug/MockTiDB.h +++ b/dbms/src/Debug/MockTiDB.h @@ -127,7 +127,9 @@ class MockTiDB : public ext::Singleton std::pair getDBIDByName(const String & database_name); - SchemaDiff getSchemaDiff(Int64 version); + bool checkSchemaDiffExists(Int64 version); + + std::optional getSchemaDiff(Int64 version); std::unordered_map getDatabases() { return databases; } diff --git a/dbms/src/Storages/Transaction/ReadIndexWorker.cpp b/dbms/src/Storages/Transaction/ReadIndexWorker.cpp index 3223c815989..7de79dd5c6d 100644 --- a/dbms/src/Storages/Transaction/ReadIndexWorker.cpp +++ b/dbms/src/Storages/Transaction/ReadIndexWorker.cpp @@ -880,7 +880,7 @@ BatchReadIndexRes ReadIndexWorkerManager::batchReadIndex( } } { // if meet timeout, which means part of regions can not get response from leader, try to poll rest tasks - TEST_LOG_FMT("rest {}, poll rest tasks onece", tasks.size()); + TEST_LOG_FMT("rest {}, poll rest tasks once", tasks.size()); while (!tasks.empty()) { diff --git a/dbms/src/TiDB/Schema/SchemaGetter.cpp b/dbms/src/TiDB/Schema/SchemaGetter.cpp index 7f52f9301b1..6e333d6ba87 100644 --- a/dbms/src/TiDB/Schema/SchemaGetter.cpp +++ b/dbms/src/TiDB/Schema/SchemaGetter.cpp @@ -19,7 +19,6 @@ namespace DB { - namespace ErrorCodes { extern const int SCHEMA_SYNC_ERROR; @@ -188,18 +187,26 @@ Int64 SchemaGetter::getVersion() return std::stoll(ver); } +bool SchemaGetter::checkSchemaDiffExists(Int64 ver) +{ + String key = getSchemaDiffKey(ver); + String data = TxnStructure::get(snap, key); + return !data.empty(); +} + String SchemaGetter::getSchemaDiffKey(Int64 ver) { return std::string(schemaDiffPrefix) + ":" + std::to_string(ver); } -SchemaDiff SchemaGetter::getSchemaDiff(Int64 ver) +std::optional SchemaGetter::getSchemaDiff(Int64 ver) { String key = getSchemaDiffKey(ver); String data = TxnStructure::get(snap, key); if (data.empty()) { - throw TiFlashException("cannot find schema diff for version: " + std::to_string(ver), Errors::Table::SyncError); + LOG_FMT_WARNING(log, "The schema diff for version {}, key {} is empty.", ver, key); + return std::nullopt; } SchemaDiff diff; diff.deserialize(data); diff --git a/dbms/src/TiDB/Schema/SchemaGetter.h b/dbms/src/TiDB/Schema/SchemaGetter.h index 02d2f7a7c88..fe0ecd59af0 100644 --- a/dbms/src/TiDB/Schema/SchemaGetter.h +++ b/dbms/src/TiDB/Schema/SchemaGetter.h @@ -26,6 +26,8 @@ #include +#include + namespace DB { // The enum results are completely the same as the DDL Action listed in the "parser/model/ddl.go" of TiDB codebase, which must be keeping in sync. @@ -138,7 +140,9 @@ struct SchemaGetter Int64 getVersion(); - SchemaDiff getSchemaDiff(Int64 ver); + bool checkSchemaDiffExists(Int64 ver); + + std::optional getSchemaDiff(Int64 ver); static String getSchemaDiffKey(Int64 ver); diff --git a/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h b/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h index 4fdba195acb..a23aeab139f 100644 --- a/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h +++ b/dbms/src/TiDB/Schema/TiDBSchemaSyncer.h @@ -106,21 +106,31 @@ struct TiDBSchemaSyncer : public SchemaSyncer Stopwatch watch; SCOPE_EXIT({ GET_METRIC(tiflash_schema_apply_duration_seconds).Observe(watch.elapsedSeconds()); }); - LOG_FMT_INFO(log, "start to sync schemas. current version is: {} and try to sync schema version to: {}", cur_version, version); + LOG_FMT_INFO(log, "Start to sync schemas. current version is: {} and try to sync schema version to: {}", cur_version, version); // Show whether the schema mutex is held for a long time or not. GET_METRIC(tiflash_schema_applying).Set(1.0); SCOPE_EXIT({ GET_METRIC(tiflash_schema_applying).Set(0.0); }); GET_METRIC(tiflash_schema_apply_count, type_diff).Increment(); - if (!tryLoadSchemaDiffs(getter, version, context)) + // After the feature concurrent DDL, TiDB does `update schema version` before `set schema diff`, and they are done in separate transactions. + // So TiFlash may see a schema version X but no schema diff X, meaning that the transaction of schema diff X has not been committed or has + // been aborted. + // However, TiDB makes sure that if we get a schema version X, then the schema diff X-1 must exist. Otherwise the transaction of schema diff + // X-1 is aborted and we can safely ignore it. + // Since TiDB can not make sure the schema diff of the latest schema version X is not empty, under this situation we should set the `cur_version` + // to X-1 and try to fetch the schema diff X next time. + Int64 version_after_load_diff = 0; + if (version_after_load_diff = tryLoadSchemaDiffs(getter, version, context); version_after_load_diff == -1) { GET_METRIC(tiflash_schema_apply_count, type_full).Increment(); loadAllSchema(getter, version, context); + // After loadAllSchema, we need update `version_after_load_diff` by last diff value exist or not + version_after_load_diff = getter.checkSchemaDiffExists(version) ? version : version - 1; } - cur_version = version; + cur_version = version_after_load_diff; GET_METRIC(tiflash_schema_version).Set(cur_version); - LOG_FMT_INFO(log, "end sync schema, version has been updated to {}", cur_version); + LOG_FMT_INFO(log, "End sync schema, version has been updated to {}{}", cur_version, cur_version == version ? "" : "(latest diff is empty)"); return true; } @@ -144,30 +154,60 @@ struct TiDBSchemaSyncer : public SchemaSyncer return it->second; } - bool tryLoadSchemaDiffs(Getter & getter, Int64 version, Context & context) + // Return Values + // - if latest schema diff is not empty, return the (latest_version) + // - if latest schema diff is empty, return the (latest_version - 1) + // - if error happend, return (-1) + Int64 tryLoadSchemaDiffs(Getter & getter, Int64 latest_version, Context & context) { - if (isTooOldSchema(cur_version, version)) + if (isTooOldSchema(cur_version, latest_version)) { - return false; + return -1; } - LOG_FMT_DEBUG(log, "try load schema diffs."); + LOG_FMT_DEBUG(log, "Try load schema diffs."); - SchemaBuilder builder(getter, context, databases, version); + SchemaBuilder builder(getter, context, databases, latest_version); Int64 used_version = cur_version; - std::vector diffs; - while (used_version < version) + // First get all schema diff from `cur_version` to `latest_version`. Only apply the schema diff(s) if we fetch all + // schema diff without any exception. + std::vector> diffs; + while (used_version < latest_version) { used_version++; diffs.push_back(getter.getSchemaDiff(used_version)); } - LOG_FMT_DEBUG(log, "end load schema diffs with total {} entries.", diffs.size()); + LOG_FMT_DEBUG(log, "End load schema diffs with total {} entries.", diffs.size()); + try { - for (const auto & diff : diffs) + for (size_t diff_index = 0; diff_index < diffs.size(); ++diff_index) { - builder.applyDiff(diff); + const auto & schema_diff = diffs[diff_index]; + + if (!schema_diff) + { + // If `schema diff` from `latest_version` got empty `schema diff` + // Then we won't apply to `latest_version`, but we will apply to `latest_version - 1` + // If `schema diff` from [`cur_version`, `latest_version - 1`] got empty `schema diff` + // Then we should just skip it. + // + // example: + // - `cur_version` is 1, `latest_version` is 10 + // - The schema diff of schema version [2,4,6] is empty, Then we just skip it. + // - The schema diff of schema version 10 is empty, Then we should just apply version into 9 + if (diff_index != diffs.size() - 1) + { + LOG_FMT_WARNING(log, "Skip the schema diff from version {}. ", cur_version + diff_index + 1); + continue; + } + + // if diff_index == diffs.size() - 1, return used_version - 1; + return used_version - 1; + } + + builder.applyDiff(*schema_diff); } } catch (TiFlashException & e) @@ -177,7 +217,7 @@ struct TiDBSchemaSyncer : public SchemaSyncer GET_METRIC(tiflash_schema_apply_count, type_failed).Increment(); } LOG_FMT_WARNING(log, "apply diff meets exception : {} \n stack is {}", e.displayText(), e.getStackTrace().toString()); - return false; + return -1; } catch (Exception & e) { @@ -187,21 +227,22 @@ struct TiDBSchemaSyncer : public SchemaSyncer } GET_METRIC(tiflash_schema_apply_count, type_failed).Increment(); LOG_FMT_WARNING(log, "apply diff meets exception : {} \n stack is {}", e.displayText(), e.getStackTrace().toString()); - return false; + return -1; } catch (Poco::Exception & e) { GET_METRIC(tiflash_schema_apply_count, type_failed).Increment(); LOG_FMT_WARNING(log, "apply diff meets exception : {}", e.displayText()); - return false; + return -1; } catch (std::exception & e) { GET_METRIC(tiflash_schema_apply_count, type_failed).Increment(); LOG_FMT_WARNING(log, "apply diff meets exception : {}", e.what()); - return false; + return -1; } - return true; + + return used_version; } void loadAllSchema(Getter & getter, Int64 version, Context & context) From 6da631c99c918bfffcf183128306a5e6bd35c7f7 Mon Sep 17 00:00:00 2001 From: lidezhu <47731263+lidezhu@users.noreply.github.com> Date: Mon, 4 Jul 2022 16:57:01 +0800 Subject: [PATCH 073/104] Optimize apply speed under heavy write pressure (#4883) ref pingcap/tiflash#4728 --- .../DeltaMerge/Delta/DeltaValueSpace.cpp | 13 +++++++ .../DeltaMerge/Delta/DeltaValueSpace.h | 7 ++++ .../Storages/DeltaMerge/DeltaMergeStore.cpp | 35 +++++++++++++++---- .../src/Storages/DeltaMerge/DeltaMergeStore.h | 8 ++--- dbms/src/Storages/DeltaMerge/Segment.h | 2 ++ dbms/src/Storages/IManageableStorage.h | 2 +- dbms/src/Storages/StorageDeltaMerge.cpp | 6 ++-- dbms/src/Storages/StorageDeltaMerge.h | 2 +- dbms/src/Storages/Transaction/KVStore.cpp | 27 ++++++++------ dbms/src/Storages/Transaction/KVStore.h | 2 +- dbms/src/Storages/Transaction/RegionTable.cpp | 2 +- 11 files changed, 79 insertions(+), 27 deletions(-) diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp index 132732d6989..8a69b7573e2 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.cpp @@ -141,6 +141,19 @@ bool DeltaValueSpace::ingestColumnFiles(DMContext & /*context*/, const RowKeyRan bool DeltaValueSpace::flush(DMContext & context) { + bool v = false; + if (!is_flushing.compare_exchange_strong(v, true)) + { + // other thread is flushing, just return. + LOG_FMT_DEBUG(log, "{}, Flush stop because other thread is flushing", simpleInfo()); + return false; + } + SCOPE_EXIT({ + bool v = true; + if (!is_flushing.compare_exchange_strong(v, false)) + throw Exception(simpleInfo() + " is expected to be flushing", ErrorCodes::LOGICAL_ERROR); + }); + LOG_FMT_DEBUG(log, "{}, Flush start", info()); /// We have two types of data needed to flush to disk: diff --git a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h index 8f14682caa8..04fb97b3004 100644 --- a/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h +++ b/dbms/src/Storages/DeltaMerge/Delta/DeltaValueSpace.h @@ -77,6 +77,11 @@ class DeltaValueSpace /// Note that those things can not be done at the same time. std::atomic_bool is_updating = false; + /// Note that it's safe to do multiple flush concurrently but only one of them can succeed, + /// and other thread's work is just a waste of resource. + /// So we only allow one flush task running at any time to aviod waste resource. + std::atomic_bool is_flushing = false; + std::atomic last_try_flush_rows = 0; std::atomic last_try_flush_bytes = 0; std::atomic last_try_compact_column_files = 0; @@ -159,6 +164,8 @@ class DeltaValueSpace size_t getTotalCacheBytes() const; size_t getValidCacheRows() const; + bool isFlushing() const { return is_flushing; } + bool isUpdating() const { return is_updating; } bool tryLockUpdating() diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 195ed5c53c2..09f290e311c 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -980,14 +980,14 @@ void DeltaMergeStore::deleteRange(const Context & db_context, const DB::Settings checkSegmentUpdate(dm_context, segment, ThreadType::Write); } -void DeltaMergeStore::flushCache(const DMContextPtr & dm_context, const RowKeyRange & range) +bool DeltaMergeStore::flushCache(const DMContextPtr & dm_context, const RowKeyRange & range, bool try_until_succeed) { RowKeyRange cur_range = range; while (!cur_range.none()) { RowKeyRange segment_range; - // Keep trying until succeeded. + // Keep trying until succeeded if needed. while (true) { SegmentPtr segment; @@ -1010,10 +1010,15 @@ void DeltaMergeStore::flushCache(const DMContextPtr & dm_context, const RowKeyRa { break; } + else if (!try_until_succeed) + { + return false; + } } cur_range.setStart(segment_range.end); } + return true; } void DeltaMergeStore::mergeDeltaAll(const Context & context) @@ -1347,6 +1352,12 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const && (delta_rows - delta_last_try_flush_rows >= delta_cache_limit_rows || delta_bytes - delta_last_try_flush_bytes >= delta_cache_limit_bytes); bool should_foreground_flush = unsaved_rows >= delta_cache_limit_rows * 3 || unsaved_bytes >= delta_cache_limit_bytes * 3; + /// For write thread, we want to avoid foreground flush to block the process of apply raft command. + /// So we increase the threshold of foreground flush for write thread. + if (thread_type == ThreadType::Write) + { + should_foreground_flush = unsaved_rows >= delta_cache_limit_rows * 10 || unsaved_bytes >= delta_cache_limit_bytes * 10; + } bool should_background_merge_delta = ((delta_check_rows >= delta_limit_rows || delta_check_bytes >= delta_limit_bytes) // && (delta_rows - delta_last_try_merge_delta_rows >= delta_cache_limit_rows @@ -1404,9 +1415,16 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const } else if (should_background_flush) { - delta_last_try_flush_rows = delta_rows; - delta_last_try_flush_bytes = delta_bytes; - try_add_background_task(BackgroundTask{TaskType::Flush, dm_context, segment, {}}); + /// It's meaningless to add more flush tasks if the segment is flushing. + /// Because only one flush task can proceed at any time. + /// And after the current flush task finished, + /// it will call `checkSegmentUpdate` again to check whether there is more flush task to do. + if (!segment->isFlushing()) + { + delta_last_try_flush_rows = delta_rows; + delta_last_try_flush_bytes = delta_bytes; + try_add_background_task(BackgroundTask{TaskType::Flush, dm_context, segment, {}}); + } } } @@ -1502,7 +1520,12 @@ void DeltaMergeStore::checkSegmentUpdate(const DMContextPtr & dm_context, const return false; }; auto try_bg_compact = [&]() { - if (should_compact) + /// Compact task should be a really low priority task. + /// And if the segment is flushing, + /// we should avoid adding background compact task to reduce lock contention on the segment and save disk throughput. + /// And after the current flush task complete, + /// it will call `checkSegmentUpdate` again to check whether there is other kinds of task to do. + if (should_compact && !segment->isFlushing()) { delta_last_try_compact_column_files = column_file_count; try_add_background_task(BackgroundTask{TaskType::Compact, dm_context, segment, {}}); diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h index 705481ca107..57c2a42b807 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.h @@ -367,14 +367,14 @@ class DeltaMergeStore : private boost::noncopyable const SegmentIdSet & read_segments = {}, size_t extra_table_id_index = InvalidColumnID); - /// Force flush all data to disk. - void flushCache(const Context & context, const RowKeyRange & range) + /// Try flush all data in `range` to disk and return whether the task succeed. + bool flushCache(const Context & context, const RowKeyRange & range, bool try_until_succeed = true) { auto dm_context = newDMContext(context, context.getSettingsRef()); - flushCache(dm_context, range); + return flushCache(dm_context, range, try_until_succeed); } - void flushCache(const DMContextPtr & dm_context, const RowKeyRange & range); + bool flushCache(const DMContextPtr & dm_context, const RowKeyRange & range, bool try_until_succeed = true); /// Merge delta into the stable layer for all segments. /// diff --git a/dbms/src/Storages/DeltaMerge/Segment.h b/dbms/src/Storages/DeltaMerge/Segment.h index cccfc5091b9..8058329ae91 100644 --- a/dbms/src/Storages/DeltaMerge/Segment.h +++ b/dbms/src/Storages/DeltaMerge/Segment.h @@ -300,6 +300,8 @@ class Segment : private boost::noncopyable void drop(const FileProviderPtr & file_provider, WriteBatches & wbs); + bool isFlushing() const { return delta->isFlushing(); } + RowsAndBytes getRowsAndBytesInRange( DMContext & dm_context, const SegmentSnapshotPtr & segment_snap, diff --git a/dbms/src/Storages/IManageableStorage.h b/dbms/src/Storages/IManageableStorage.h index ebf84c592e4..2ff766a9c6d 100644 --- a/dbms/src/Storages/IManageableStorage.h +++ b/dbms/src/Storages/IManageableStorage.h @@ -68,7 +68,7 @@ class IManageableStorage : public IStorage virtual void flushCache(const Context & /*context*/) {} - virtual void flushCache(const Context & /*context*/, const DM::RowKeyRange & /*range_to_flush*/) {} + virtual bool flushCache(const Context & /*context*/, const DM::RowKeyRange & /*range_to_flush*/, [[maybe_unused]] bool try_until_succeed = true) { return true; } virtual BlockInputStreamPtr status() { return {}; } diff --git a/dbms/src/Storages/StorageDeltaMerge.cpp b/dbms/src/Storages/StorageDeltaMerge.cpp index 67d32c73a05..a6de4efb3ac 100644 --- a/dbms/src/Storages/StorageDeltaMerge.cpp +++ b/dbms/src/Storages/StorageDeltaMerge.cpp @@ -775,12 +775,12 @@ void StorageDeltaMerge::checkStatus(const Context & context) void StorageDeltaMerge::flushCache(const Context & context) { - flushCache(context, DM::RowKeyRange::newAll(is_common_handle, rowkey_column_size)); + flushCache(context, DM::RowKeyRange::newAll(is_common_handle, rowkey_column_size), /* try_until_succeed */ true); } -void StorageDeltaMerge::flushCache(const Context & context, const DM::RowKeyRange & range_to_flush) +bool StorageDeltaMerge::flushCache(const Context & context, const DM::RowKeyRange & range_to_flush, bool try_until_succeed) { - getAndMaybeInitStore()->flushCache(context, range_to_flush); + return getAndMaybeInitStore()->flushCache(context, range_to_flush, try_until_succeed); } void StorageDeltaMerge::mergeDelta(const Context & context) diff --git a/dbms/src/Storages/StorageDeltaMerge.h b/dbms/src/Storages/StorageDeltaMerge.h index 79ee225d237..9e4ab12ad4f 100644 --- a/dbms/src/Storages/StorageDeltaMerge.h +++ b/dbms/src/Storages/StorageDeltaMerge.h @@ -73,7 +73,7 @@ class StorageDeltaMerge void flushCache(const Context & context) override; - void flushCache(const Context & context, const DM::RowKeyRange & range_to_flush) override; + bool flushCache(const Context & context, const DM::RowKeyRange & range_to_flush, bool try_until_succeed) override; /// Merge delta into the stable layer for all segments. /// diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index 318a04c6ed9..f9d6d01955e 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -129,7 +129,7 @@ void KVStore::traverseRegions(std::function & callback(region.first, region.second); } -void KVStore::tryFlushRegionCacheInStorage(TMTContext & tmt, const Region & region, Poco::Logger * log) +bool KVStore::tryFlushRegionCacheInStorage(TMTContext & tmt, const Region & region, Poco::Logger * log, bool try_until_succeed) { auto table_id = region.getMappedTableID(); auto storage = tmt.getStorages().get(table_id); @@ -139,7 +139,7 @@ void KVStore::tryFlushRegionCacheInStorage(TMTContext & tmt, const Region & regi "tryFlushRegionCacheInStorage can not get table for region {} with table id {}, ignored", region.toString(), table_id); - return; + return true; } try @@ -151,7 +151,7 @@ void KVStore::tryFlushRegionCacheInStorage(TMTContext & tmt, const Region & regi region.getRange()->getMappedTableID(), storage->isCommonHandle(), storage->getRowKeyColumnSize()); - storage->flushCache(tmt.getContext(), rowkey_range); + return storage->flushCache(tmt.getContext(), rowkey_range, try_until_succeed); } catch (DB::Exception & e) { @@ -159,6 +159,7 @@ void KVStore::tryFlushRegionCacheInStorage(TMTContext & tmt, const Region & regi if (e.code() != ErrorCodes::TABLE_IS_DROPPED) throw; } + return true; } void KVStore::tryPersist(RegionID region_id) @@ -366,12 +367,12 @@ EngineStoreApplyRes KVStore::handleUselessAdminRaftCmd( if (rows >= region_compact_log_min_rows.load(std::memory_order_relaxed) || size_bytes >= region_compact_log_min_bytes.load(std::memory_order_relaxed)) { - // if rows or bytes more than threshold, flush cache and perist mem data. + // if rows or bytes more than threshold, try to flush cache and persist mem data. return true; } else { - // if thhere is little data in mem, wait until time interval reached threshold. + // if there is little data in mem, wait until time interval reached threshold. // use random period so that lots of regions will not be persisted at same time. auto compact_log_period = std::rand() % region_compact_log_period.load(std::memory_order_relaxed); // NOLINT return !(curr_region.lastCompactLogTime() + Seconds{compact_log_period} > Clock::now()); @@ -381,11 +382,17 @@ EngineStoreApplyRes KVStore::handleUselessAdminRaftCmd( if (check_sync_log()) { - tryFlushRegionCacheInStorage(tmt, curr_region, log); - persistRegion(curr_region, region_task_lock, "compact raft log"); - curr_region.markCompactLog(); - curr_region.cleanApproxMemCacheInfo(); - return EngineStoreApplyRes::Persist; + if (tryFlushRegionCacheInStorage(tmt, curr_region, log, /* try_until_succeed */ false)) + { + persistRegion(curr_region, region_task_lock, "compact raft log"); + curr_region.markCompactLog(); + curr_region.cleanApproxMemCacheInfo(); + return EngineStoreApplyRes::Persist; + } + else + { + return EngineStoreApplyRes::None; + } } return EngineStoreApplyRes::None; } diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index bb45e65d18b..66e2fe32b75 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -91,7 +91,7 @@ class KVStore final : private boost::noncopyable void tryPersist(RegionID region_id); - static void tryFlushRegionCacheInStorage(TMTContext & tmt, const Region & region, Poco::Logger * log); + static bool tryFlushRegionCacheInStorage(TMTContext & tmt, const Region & region, Poco::Logger * log, bool try_until_succeed = true); size_t regionSize() const; EngineStoreApplyRes handleAdminRaftCmd(raft_cmdpb::AdminRequest && request, diff --git a/dbms/src/Storages/Transaction/RegionTable.cpp b/dbms/src/Storages/Transaction/RegionTable.cpp index c855d5b3226..5ae36a4bd64 100644 --- a/dbms/src/Storages/Transaction/RegionTable.cpp +++ b/dbms/src/Storages/Transaction/RegionTable.cpp @@ -230,7 +230,7 @@ void removeObsoleteDataInStorage( auto rowkey_range = DM::RowKeyRange::fromRegionRange(handle_range, table_id, table_id, storage->isCommonHandle(), storage->getRowKeyColumnSize()); dm_storage->deleteRange(rowkey_range, context->getSettingsRef()); - dm_storage->flushCache(*context, rowkey_range); // flush to disk + dm_storage->flushCache(*context, rowkey_range, /*try_until_succeed*/ true); // flush to disk } catch (DB::Exception & e) { From a0ecce02725e10c32e8484e7be79396fda5bfdf6 Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Tue, 5 Jul 2022 14:49:00 +0800 Subject: [PATCH 074/104] update proxy to raftstore-proxy-6.2 (#5287) ref pingcap/tiflash#4982 --- contrib/tiflash-proxy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index ca2f51f94e5..6ea4d608b1c 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit ca2f51f94e55bdd23749dcc02ab4afb94eeb5ae5 +Subproject commit 6ea4d608b1c03fab89d17f54a2e399602231e27c From a2287043f8d551a4c2e841819db1efa5cc8f91ef Mon Sep 17 00:00:00 2001 From: Wenxuan Date: Wed, 6 Jul 2022 14:25:02 +0800 Subject: [PATCH 075/104] Flush segment cache when doing the compaction (#5284) close pingcap/tiflash#5179 --- .../Storages/DeltaMerge/DeltaMergeStore.cpp | 7 +++ .../tests/gtest_dm_delta_merge_store.cpp | 49 +++++++++++++++++++ 2 files changed, 56 insertions(+) diff --git a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp index 09f290e311c..73ad22d6d1f 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp +++ b/dbms/src/Storages/DeltaMerge/DeltaMergeStore.cpp @@ -1060,6 +1060,13 @@ std::optional DeltaMergeStore::mergeDeltaBySegment(const Contex segment = segment_it->second; } + if (!segment->flushCache(*dm_context)) + { + // If the flush failed, it means there are parallel updates to the segment in the background. + // In this case, we try again. + continue; + } + const auto new_segment = segmentMergeDelta(*dm_context, segment, run_thread); if (new_segment) { diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp index d46e1b7aa36..b7913c44a2c 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_dm_delta_merge_store.cpp @@ -3762,6 +3762,55 @@ try CATCH +// Verify that unflushed data will also be compacted. +TEST_P(DeltaMergeStoreMergeDeltaBySegmentTest, Flush) +try +{ + { + // Write data to first 3 segments and flush. + auto newly_written_rows = helper->rows_by_segments[0] + helper->rows_by_segments[1] + helper->rows_by_segments[2]; + Block block = DMTestEnv::prepareSimpleWriteBlock(0, newly_written_rows, false, pk_type, 5 /* new tso */); + store->write(*db_context, db_context->getSettingsRef(), block); + store->flushCache(dm_context, RowKeyRange::newAll(store->isCommonHandle(), store->getRowKeyColumnSize())); + + helper->expected_delta_rows[0] += helper->rows_by_segments[0]; + helper->expected_delta_rows[1] += helper->rows_by_segments[1]; + helper->expected_delta_rows[2] += helper->rows_by_segments[2]; + helper->verifyExpectedRowsForAllSegments(); + + auto segment1 = std::next(store->segments.begin())->second; + ASSERT_EQ(segment1->getDelta()->getUnsavedRows(), 0); + } + { + // Write new data to segment[1] without flush. + auto newly_written_rows = helper->rows_by_segments[1]; + Block block = DMTestEnv::prepareSimpleWriteBlock(helper->rows_by_segments[0], helper->rows_by_segments[0] + newly_written_rows, false, pk_type, 10 /* new tso */); + store->write(*db_context, db_context->getSettingsRef(), block); + + helper->expected_delta_rows[1] += helper->rows_by_segments[1]; + helper->verifyExpectedRowsForAllSegments(); + + auto segment1 = std::next(store->segments.begin())->second; + ASSERT_GT(segment1->getDelta()->getUnsavedRows(), 0); + } + { + auto segment1 = std::next(store->segments.begin())->second; + auto result = store->mergeDeltaBySegment(*db_context, segment1->getRowKeyRange().start, DeltaMergeStore::TaskRunThread::Foreground); + ASSERT_NE(result, std::nullopt); + + segment1 = std::next(store->segments.begin())->second; + ASSERT_EQ(*result, segment1->getRowKeyRange()); + + helper->expected_stable_rows[1] += helper->expected_delta_rows[1]; + helper->expected_delta_rows[1] = 0; + helper->verifyExpectedRowsForAllSegments(); + + ASSERT_EQ(segment1->getDelta()->getUnsavedRows(), 0); + } +} +CATCH + + } // namespace tests } // namespace DM } // namespace DB From 029926dd5ac14867680a06dbea6f2ee85b462462 Mon Sep 17 00:00:00 2001 From: Wenxuan Date: Wed, 6 Jul 2022 14:51:02 +0800 Subject: [PATCH 076/104] metrics: Fix incorrect metrics for delta_merge tasks (#5061) close pingcap/tiflash#5055 --- metrics/grafana/tiflash_summary.json | 298 +++++++++++++++++++++------ 1 file changed, 232 insertions(+), 66 deletions(-) diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index f899a47ed10..8e18357031d 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -52,7 +52,7 @@ "gnetId": null, "graphTooltip": 1, "id": null, - "iteration": 1653635389238, + "iteration": 1654217728945, "links": [], "panels": [ { @@ -3878,7 +3878,7 @@ "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 5, "w": 12, "x": 0, "y": 21 @@ -3893,6 +3893,7 @@ "min": false, "rightSide": true, "show": true, + "sideWidth": null, "total": false, "values": false }, @@ -3908,38 +3909,27 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ - { - "alias": "/(delta_merge)|(seg_)/", - "yaxis": 2 - } - ], + "seriesOverrides": [], "spaceLength": 10, "stack": false, - "steppedLine": false, + "steppedLine": true, "targets": [ { - "expr": "sum(rate(tiflash_storage_subtask_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"delta_merge|delta_merge_fg|delta_merge_bg_gc|seg_merge|seg_split|seg_split_fg\"}[1m])) by (type)", + "exemplar": true, + "expr": "sum(rate(tiflash_storage_subtask_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (type)", "format": "time_series", "hide": false, - "intervalFactor": 1, + "interval": "", + "intervalFactor": 2, "legendFormat": "{{type}}", "refId": "A" - }, - { - "expr": "sum(increase(tiflash_storage_subtask_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"delta_merge|delta_merge_fg|delta_merge_bg_gc|seg_merge|seg_split|seg_split_fg\"}[1m])) by (type)", - "format": "time_series", - "hide": false, - "intervalFactor": 1, - "legendFormat": "{{type}}", - "refId": "B" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Internal Tasks OPS", + "title": "Small Internal Tasks OPS", "tooltip": { "shared": true, "sort": 0, @@ -3955,7 +3945,7 @@ }, "yaxes": [ { - "decimals": null, + "decimals": 1, "format": "ops", "label": null, "logBase": 1, @@ -3969,7 +3959,7 @@ "logBase": 1, "max": null, "min": "0", - "show": true + "show": false } ], "yaxis": { @@ -3988,10 +3978,10 @@ "defaults": {}, "overrides": [] }, - "fill": 1, + "fill": 0, "fillGradient": 0, "gridPos": { - "h": 8, + "h": 5, "w": 12, "x": 12, "y": 21 @@ -4023,58 +4013,233 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [ + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { - "alias": "/^.*-delta_merge/", - "yaxis": 2 + "exemplar": false, + "expr": "histogram_quantile(1, sum(rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type!~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (le,type))", + "format": "time_series", + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "max-{{type}}", + "refId": "A" + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Small Internal Tasks Duration", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ + { + "decimals": 1, + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true }, { - "alias": "/^.*-seg_split/", - "yaxis": 2 + "format": "s", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": false } ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Total number of storage's internal sub tasks", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 5, + "w": 12, + "x": 0, + "y": 26 + }, + "hiddenSeries": false, + "id": 130, + "legend": { + "alignAsTable": true, + "avg": false, + "current": false, + "max": false, + "min": false, + "rightSide": true, + "show": true, + "total": false, + "values": false + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], "spaceLength": 10, "stack": false, - "steppedLine": false, + "steppedLine": true, "targets": [ { - "expr": "histogram_quantile(1, sum(rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le,type))", + "exemplar": true, + "expr": "sum(rate(tiflash_storage_subtask_count{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (type)", "format": "time_series", "hide": false, - "intervalFactor": 1, - "legendFormat": "max-{{type}}", + "interval": "", + "intervalFactor": 2, + "legendFormat": "{{type}}", "refId": "A" - }, + } + ], + "thresholds": [], + "timeFrom": null, + "timeRegions": [], + "timeShift": null, + "title": "Large Internal Tasks OPS", + "tooltip": { + "shared": true, + "sort": 0, + "value_type": "individual" + }, + "type": "graph", + "xaxis": { + "buckets": null, + "mode": "time", + "name": null, + "show": true, + "values": [] + }, + "yaxes": [ { - "expr": "histogram_quantile(0.99, sum(rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le,type))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "99-{{type}}", - "refId": "B" + "decimals": 1, + "format": "ops", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": true }, { - "expr": "histogram_quantile(0.95, sum(rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le,type))", - "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "95-{{type}}", - "refId": "C" - }, + "format": "opm", + "label": null, + "logBase": 1, + "max": null, + "min": "0", + "show": false + } + ], + "yaxis": { + "align": false, + "alignLevel": null + } + }, + { + "aliasColors": {}, + "bars": false, + "dashLength": 10, + "dashes": false, + "datasource": "${DS_TEST-CLUSTER}", + "description": "Duration of storage's internal sub tasks", + "fieldConfig": { + "defaults": {}, + "overrides": [] + }, + "fill": 0, + "fillGradient": 0, + "gridPos": { + "h": 5, + "w": 12, + "x": 12, + "y": 26 + }, + "hiddenSeries": false, + "id": 131, + "legend": { + "alignAsTable": true, + "avg": false, + "current": true, + "max": true, + "min": false, + "rightSide": true, + "show": true, + "sort": null, + "sortDesc": null, + "total": false, + "values": true + }, + "lines": true, + "linewidth": 1, + "links": [], + "nullPointMode": "null as zero", + "options": { + "alertThreshold": true + }, + "percentage": false, + "pluginVersion": "7.5.11", + "pointradius": 5, + "points": false, + "renderer": "flot", + "seriesOverrides": [], + "spaceLength": 10, + "stack": false, + "steppedLine": false, + "targets": [ { - "expr": "histogram_quantile(0.80, sum(rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}[1m])) by (le,type))", + "exemplar": true, + "expr": "histogram_quantile(1, sum(rate(tiflash_storage_subtask_duration_seconds_bucket{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\", type=~\"(delta_merge|seg_merge|seg_split).*\"}[$__rate_interval])) by (le,type))", "format": "time_series", - "hide": true, - "intervalFactor": 1, - "legendFormat": "80-{{type}}", - "refId": "D" + "hide": false, + "interval": "", + "intervalFactor": 2, + "legendFormat": "max-{{type}}", + "refId": "A" } ], "thresholds": [], "timeFrom": null, "timeRegions": [], "timeShift": null, - "title": "Internal Tasks Duration", + "title": "Large Internal Tasks Duration", "tooltip": { "shared": true, "sort": 0, @@ -4090,6 +4255,7 @@ }, "yaxes": [ { + "decimals": 1, "format": "s", "label": null, "logBase": 1, @@ -4103,7 +4269,7 @@ "logBase": 1, "max": null, "min": "0", - "show": true + "show": false } ], "yaxis": { @@ -4128,7 +4294,7 @@ "h": 8, "w": 12, "x": 0, - "y": 29 + "y": 31 }, "hiddenSeries": false, "id": 43, @@ -4234,7 +4400,7 @@ "h": 8, "w": 12, "x": 12, - "y": 29 + "y": 31 }, "heatmap": {}, "hideZeroBuckets": true, @@ -4297,7 +4463,7 @@ "h": 8, "w": 12, "x": 0, - "y": 37 + "y": 39 }, "hiddenSeries": false, "id": 46, @@ -4420,7 +4586,7 @@ "h": 8, "w": 12, "x": 12, - "y": 37 + "y": 39 }, "hiddenSeries": false, "id": 47, @@ -4544,7 +4710,7 @@ "h": 8, "w": 12, "x": 0, - "y": 45 + "y": 47 }, "height": "", "hiddenSeries": false, @@ -4674,7 +4840,7 @@ "h": 8, "w": 12, "x": 12, - "y": 45 + "y": 47 }, "height": "", "hiddenSeries": false, @@ -4802,7 +4968,7 @@ "h": 8, "w": 12, "x": 0, - "y": 53 + "y": 55 }, "hiddenSeries": false, "id": 88, @@ -5002,7 +5168,7 @@ "h": 8, "w": 12, "x": 12, - "y": 53 + "y": 55 }, "hiddenSeries": false, "id": 67, @@ -5116,7 +5282,7 @@ "h": 8, "w": 12, "x": 0, - "y": 61 + "y": 63 }, "hiddenSeries": false, "id": 84, @@ -5216,7 +5382,7 @@ "h": 8, "w": 12, "x": 12, - "y": 61 + "y": 63 }, "hiddenSeries": false, "id": 86, @@ -8183,5 +8349,5 @@ "timezone": "", "title": "Test-Cluster-TiFlash-Summary", "uid": "SVbh2xUWk", - "version": 2 -} + "version": 1 +} \ No newline at end of file From 4f7e24acfd905ae9f8c65baf319cfdb173629e57 Mon Sep 17 00:00:00 2001 From: Schrodinger ZHU Yifan Date: Wed, 6 Jul 2022 15:25:02 +0800 Subject: [PATCH 077/104] dep: upgrade jemalloc (#5197) close pingcap/tiflash#5258 --- contrib/jemalloc | 2 +- contrib/jemalloc-cmake/CMakeLists.txt | 161 +++++-- .../jemalloc/internal/jemalloc_preamble.h | 115 +++-- .../include/jemalloc/jemalloc.h | 12 +- .../jemalloc/jemalloc_defs.h | 24 +- .../include/jemalloc/jemalloc_macros.h | 148 ++++++ .../include/jemalloc/jemalloc_protos.h | 86 ++++ .../include/jemalloc/jemalloc_protos_jet.h | 71 +++ .../include/jemalloc/jemalloc_rename.h | 46 +- .../jemalloc/jemalloc_typedefs.h | 45 +- .../internal/jemalloc_internal_defs.h.in | 425 +++++++++++++++++ .../internal/jemalloc_internal_defs.h.in | 425 +++++++++++++++++ .../internal/jemalloc_internal_defs.h.in | 427 +++++++++++++++++ .../internal/jemalloc_internal_defs.h.in | 427 +++++++++++++++++ .../include_linux_aarch64/README | 7 - ...nal_defs.h => jemalloc_internal_defs.h.in} | 143 +++--- .../jemalloc/jemalloc_defs.h | 43 -- .../jemalloc/jemalloc_macros.h | 129 ------ .../jemalloc/jemalloc_protos.h | 66 --- .../jemalloc/jemalloc_typedefs.h | 77 ---- .../internal/jemalloc_internal_defs.h.in | 427 +++++++++++++++++ .../internal/jemalloc_internal_defs.h.in | 427 +++++++++++++++++ .../include_linux_x86_64/README | 7 - ...nal_defs.h => jemalloc_internal_defs.h.in} | 152 ++++--- .../jemalloc/internal/jemalloc_preamble.h | 216 --------- .../jemalloc/jemalloc_macros.h | 129 ------ .../jemalloc/jemalloc_protos.h | 66 --- .../internal/jemalloc_internal_defs.h.in | 428 ++++++++++++++++++ 28 files changed, 3753 insertions(+), 978 deletions(-) rename contrib/jemalloc-cmake/{include_linux_aarch64 => include}/jemalloc/internal/jemalloc_preamble.h (69%) rename contrib/jemalloc-cmake/{include_linux_x86_64 => include}/jemalloc/jemalloc_defs.h (67%) create mode 100644 contrib/jemalloc-cmake/include/jemalloc/jemalloc_macros.h create mode 100644 contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h create mode 100644 contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos_jet.h rename contrib/jemalloc-cmake/{include_linux_x86_64 => include}/jemalloc/jemalloc_typedefs.h (57%) create mode 100644 contrib/jemalloc-cmake/include_darwin_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in create mode 100644 contrib/jemalloc-cmake/include_darwin_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in create mode 100644 contrib/jemalloc-cmake/include_freebsd_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in create mode 100644 contrib/jemalloc-cmake/include_freebsd_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in delete mode 100644 contrib/jemalloc-cmake/include_linux_aarch64/README rename contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/{jemalloc_internal_defs.h => jemalloc_internal_defs.h.in} (80%) delete mode 100644 contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_defs.h delete mode 100644 contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_macros.h delete mode 100644 contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_protos.h delete mode 100644 contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_typedefs.h create mode 100644 contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in create mode 100644 contrib/jemalloc-cmake/include_linux_riscv64/jemalloc/internal/jemalloc_internal_defs.h.in delete mode 100644 contrib/jemalloc-cmake/include_linux_x86_64/README rename contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/{jemalloc_internal_defs.h => jemalloc_internal_defs.h.in} (78%) delete mode 100644 contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_preamble.h delete mode 100644 contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_macros.h delete mode 100644 contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_protos.h create mode 100644 contrib/jemalloc-cmake/include_linux_x86_64_musl/jemalloc/internal/jemalloc_internal_defs.h.in diff --git a/contrib/jemalloc b/contrib/jemalloc index ea6b3e973b4..54eaed1d8b5 160000 --- a/contrib/jemalloc +++ b/contrib/jemalloc @@ -1 +1 @@ -Subproject commit ea6b3e973b477b8061e0076bb257dbd7f3faa756 +Subproject commit 54eaed1d8b56b1aa528be3bdd1877e59c56fa90c diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt index ef02fbabc81..91b17eb8ec7 100644 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ b/contrib/jemalloc-cmake/CMakeLists.txt @@ -4,65 +4,136 @@ endif() set(JEMALLOC_SOURCE_DIR ${TiFlash_SOURCE_DIR}/contrib/jemalloc) -set(SRCS -${JEMALLOC_SOURCE_DIR}/src/arena.c -${JEMALLOC_SOURCE_DIR}/src/background_thread.c -${JEMALLOC_SOURCE_DIR}/src/base.c -${JEMALLOC_SOURCE_DIR}/src/bin.c -${JEMALLOC_SOURCE_DIR}/src/bitmap.c -${JEMALLOC_SOURCE_DIR}/src/ckh.c -${JEMALLOC_SOURCE_DIR}/src/ctl.c -${JEMALLOC_SOURCE_DIR}/src/div.c -${JEMALLOC_SOURCE_DIR}/src/extent.c -${JEMALLOC_SOURCE_DIR}/src/extent_dss.c -${JEMALLOC_SOURCE_DIR}/src/extent_mmap.c -${JEMALLOC_SOURCE_DIR}/src/hash.c -${JEMALLOC_SOURCE_DIR}/src/hook.c -${JEMALLOC_SOURCE_DIR}/src/jemalloc.c -${JEMALLOC_SOURCE_DIR}/src/jemalloc_cpp.cpp -${JEMALLOC_SOURCE_DIR}/src/large.c -${JEMALLOC_SOURCE_DIR}/src/log.c -${JEMALLOC_SOURCE_DIR}/src/malloc_io.c -${JEMALLOC_SOURCE_DIR}/src/mutex.c -${JEMALLOC_SOURCE_DIR}/src/mutex_pool.c -${JEMALLOC_SOURCE_DIR}/src/nstime.c -${JEMALLOC_SOURCE_DIR}/src/pages.c -${JEMALLOC_SOURCE_DIR}/src/prng.c -${JEMALLOC_SOURCE_DIR}/src/prof.c -${JEMALLOC_SOURCE_DIR}/src/rtree.c -${JEMALLOC_SOURCE_DIR}/src/sc.c -${JEMALLOC_SOURCE_DIR}/src/stats.c -${JEMALLOC_SOURCE_DIR}/src/sz.c -${JEMALLOC_SOURCE_DIR}/src/tcache.c -${JEMALLOC_SOURCE_DIR}/src/test_hooks.c -${JEMALLOC_SOURCE_DIR}/src/ticker.c -${JEMALLOC_SOURCE_DIR}/src/tsd.c -${JEMALLOC_SOURCE_DIR}/src/witness.c -${JEMALLOC_SOURCE_DIR}/src/safety_check.c +set (SRCS + "${JEMALLOC_SOURCE_DIR}/src/arena.c" + "${JEMALLOC_SOURCE_DIR}/src/background_thread.c" + "${JEMALLOC_SOURCE_DIR}/src/base.c" + "${JEMALLOC_SOURCE_DIR}/src/bin.c" + "${JEMALLOC_SOURCE_DIR}/src/bin_info.c" + "${JEMALLOC_SOURCE_DIR}/src/bitmap.c" + "${JEMALLOC_SOURCE_DIR}/src/buf_writer.c" + "${JEMALLOC_SOURCE_DIR}/src/cache_bin.c" + "${JEMALLOC_SOURCE_DIR}/src/ckh.c" + "${JEMALLOC_SOURCE_DIR}/src/counter.c" + "${JEMALLOC_SOURCE_DIR}/src/ctl.c" + "${JEMALLOC_SOURCE_DIR}/src/decay.c" + "${JEMALLOC_SOURCE_DIR}/src/div.c" + "${JEMALLOC_SOURCE_DIR}/src/ecache.c" + "${JEMALLOC_SOURCE_DIR}/src/edata.c" + "${JEMALLOC_SOURCE_DIR}/src/edata_cache.c" + "${JEMALLOC_SOURCE_DIR}/src/ehooks.c" + "${JEMALLOC_SOURCE_DIR}/src/emap.c" + "${JEMALLOC_SOURCE_DIR}/src/eset.c" + "${JEMALLOC_SOURCE_DIR}/src/exp_grow.c" + "${JEMALLOC_SOURCE_DIR}/src/extent.c" + "${JEMALLOC_SOURCE_DIR}/src/extent_dss.c" + "${JEMALLOC_SOURCE_DIR}/src/extent_mmap.c" + "${JEMALLOC_SOURCE_DIR}/src/fxp.c" + "${JEMALLOC_SOURCE_DIR}/src/hook.c" + "${JEMALLOC_SOURCE_DIR}/src/hpa.c" + "${JEMALLOC_SOURCE_DIR}/src/hpa_hooks.c" + "${JEMALLOC_SOURCE_DIR}/src/hpdata.c" + "${JEMALLOC_SOURCE_DIR}/src/inspect.c" + "${JEMALLOC_SOURCE_DIR}/src/jemalloc.c" + "${JEMALLOC_SOURCE_DIR}/src/large.c" + "${JEMALLOC_SOURCE_DIR}/src/log.c" + "${JEMALLOC_SOURCE_DIR}/src/malloc_io.c" + "${JEMALLOC_SOURCE_DIR}/src/mutex.c" + "${JEMALLOC_SOURCE_DIR}/src/nstime.c" + "${JEMALLOC_SOURCE_DIR}/src/pa.c" + "${JEMALLOC_SOURCE_DIR}/src/pac.c" + "${JEMALLOC_SOURCE_DIR}/src/pa_extra.c" + "${JEMALLOC_SOURCE_DIR}/src/pages.c" + "${JEMALLOC_SOURCE_DIR}/src/pai.c" + "${JEMALLOC_SOURCE_DIR}/src/peak_event.c" + "${JEMALLOC_SOURCE_DIR}/src/prof.c" + "${JEMALLOC_SOURCE_DIR}/src/prof_data.c" + "${JEMALLOC_SOURCE_DIR}/src/prof_log.c" + "${JEMALLOC_SOURCE_DIR}/src/prof_recent.c" + "${JEMALLOC_SOURCE_DIR}/src/prof_stats.c" + "${JEMALLOC_SOURCE_DIR}/src/prof_sys.c" + "${JEMALLOC_SOURCE_DIR}/src/psset.c" + "${JEMALLOC_SOURCE_DIR}/src/rtree.c" + "${JEMALLOC_SOURCE_DIR}/src/safety_check.c" + "${JEMALLOC_SOURCE_DIR}/src/san_bump.c" + "${JEMALLOC_SOURCE_DIR}/src/san.c" + "${JEMALLOC_SOURCE_DIR}/src/sc.c" + "${JEMALLOC_SOURCE_DIR}/src/sec.c" + "${JEMALLOC_SOURCE_DIR}/src/stats.c" + "${JEMALLOC_SOURCE_DIR}/src/sz.c" + "${JEMALLOC_SOURCE_DIR}/src/tcache.c" + "${JEMALLOC_SOURCE_DIR}/src/test_hooks.c" + "${JEMALLOC_SOURCE_DIR}/src/thread_event.c" + "${JEMALLOC_SOURCE_DIR}/src/ticker.c" + "${JEMALLOC_SOURCE_DIR}/src/tsd.c" + "${JEMALLOC_SOURCE_DIR}/src/witness.c" ) if(CMAKE_SYSTEM_NAME MATCHES "Darwin") list(APPEND SRCS ${JEMALLOC_SOURCE_DIR}/src/zone.c) endif() +if (ARCH_LINUX) + # ThreadPool select job randomly, and there can be some threads that had been + # performed some memory heavy task before and will be inactive for some time, + # but until it will became active again, the memory will not be freed since by + # default each thread has it's own arena, but there should be not more then + # 4*CPU arenas (see opt.nareans description). + # + # By enabling percpu_arena number of arenas limited to number of CPUs and hence + # this problem should go away. + # + # muzzy_decay_ms -- use MADV_FREE when available on newer Linuxes, to + # avoid spurious latencies and additional work associated with + # MADV_DONTNEED. See + # https://github.com/ClickHouse/ClickHouse/issues/11121 for motivation. + set (JEMALLOC_CONFIG_MALLOC_CONF "percpu_arena:percpu,oversize_threshold:0,muzzy_decay_ms:5000,dirty_decay_ms:5000") +else() + set (JEMALLOC_CONFIG_MALLOC_CONF "oversize_threshold:0,muzzy_decay_ms:5000,dirty_decay_ms:5000") +endif() + +message (STATUS "jemalloc malloc_conf: ${JEMALLOC_CONFIG_MALLOC_CONF}") + if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -w") endif () add_library(jemalloc STATIC ${SRCS}) +set (JEMALLOC_INCLUDE_PREFIX) + +if (ARCH_LINUX) + set (JEMALLOC_INCLUDE_PREFIX "include_linux") + target_compile_definitions(jemalloc PRIVATE JEMALLOC_MADV_FREE=8) +elseif (ARCH_FREEBSD) + set (JEMALLOC_INCLUDE_PREFIX "include_freebsd") +elseif (APPLE) + set (JEMALLOC_INCLUDE_PREFIX "include_darwin") +else () + message (FATAL_ERROR "internal jemalloc: This OS is not supported") +endif () -if (ARCH_ARM) - target_include_directories(jemalloc PUBLIC - ${CMAKE_CURRENT_SOURCE_DIR}/include - ${CMAKE_CURRENT_SOURCE_DIR}/include_linux_aarch64) +if (ARCH_AMD64) + if (USE_MUSL) + set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_x86_64_musl") + else() + set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_x86_64") + endif() +elseif (ARCH_AARCH64) + set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_aarch64") +elseif (ARCH_PPC64LE) + set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_ppc64le") +elseif (ARCH_RISCV64) + set(JEMALLOC_INCLUDE_PREFIX "${JEMALLOC_INCLUDE_PREFIX}_riscv64") else () - target_include_directories(jemalloc PUBLIC - ${CMAKE_CURRENT_SOURCE_DIR}/include - ${CMAKE_CURRENT_SOURCE_DIR}/include_linux_x86_64) + message (FATAL_ERROR "internal jemalloc: This arch is not supported") endif () -target_include_directories(jemalloc PRIVATE - ${JEMALLOC_SOURCE_DIR}/include) +configure_file(${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal/jemalloc_internal_defs.h.in + ${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal/jemalloc_internal_defs.h) +target_include_directories(jemalloc SYSTEM PRIVATE + "${CMAKE_CURRENT_BINARY_DIR}/${JEMALLOC_INCLUDE_PREFIX}/jemalloc/internal") + +target_include_directories(jemalloc PUBLIC ${JEMALLOC_SOURCE_DIR}/include ${TiFlash_SOURCE_DIR}/contrib/jemalloc-cmake/include) target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_NO_PRIVATE_NAMESPACE) @@ -80,3 +151,5 @@ if (ENABLE_JEMALLOC_PROF) target_link_libraries (jemalloc PRIVATE ${UNWIND_LIBRARY}) endif () endif () + +target_compile_options(jemalloc PRIVATE -D_GNU_SOURCE) diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_preamble.h b/contrib/jemalloc-cmake/include/jemalloc/internal/jemalloc_preamble.h similarity index 69% rename from contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_preamble.h rename to contrib/jemalloc-cmake/include/jemalloc/internal/jemalloc_preamble.h index d79551e1f25..45f43a6cd02 100644 --- a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_preamble.h +++ b/contrib/jemalloc-cmake/include/jemalloc/internal/jemalloc_preamble.h @@ -1,27 +1,33 @@ #ifndef JEMALLOC_PREAMBLE_H #define JEMALLOC_PREAMBLE_H -#include "jemalloc_internal_defs.h" #include "jemalloc/internal/jemalloc_internal_decls.h" +#include "jemalloc_internal_defs.h" -#ifdef JEMALLOC_UTRACE +#if defined(JEMALLOC_UTRACE) || defined(JEMALLOC_UTRACE_LABEL) #include +#if defined(JEMALLOC_UTRACE) +#define UTRACE_CALL(p, l) utrace(p, l) +#else +#define UTRACE_CALL(p, l) utrace("jemalloc_process", p, l) +#define JEMALLOC_UTRACE +#endif #endif #define JEMALLOC_NO_DEMANGLE #ifdef JEMALLOC_JET -# undef JEMALLOC_IS_MALLOC -# define JEMALLOC_N(n) jet_##n -# include "jemalloc/internal/public_namespace.h" -# define JEMALLOC_NO_RENAME -# include "jemalloc/jemalloc.h" -# undef JEMALLOC_NO_RENAME +#undef JEMALLOC_IS_MALLOC +#define JEMALLOC_N(n) jet_##n +#include "jemalloc/internal/public_namespace.h" +#define JEMALLOC_NO_RENAME +#include "jemalloc/jemalloc.h" +#undef JEMALLOC_NO_RENAME #else -# define JEMALLOC_N(n) je_##n -# include "jemalloc/jemalloc.h" +#define JEMALLOC_N(n) je_##n +#include "jemalloc/jemalloc.h" #endif -#if (defined(JEMALLOC_OSATOMIC) || defined(JEMALLOC_OSSPIN)) +#if defined(JEMALLOC_OSATOMIC) #include #endif @@ -39,16 +45,16 @@ * possible. */ #ifndef JEMALLOC_NO_PRIVATE_NAMESPACE -# ifndef JEMALLOC_JET -# include "jemalloc/internal/private_namespace.h" -# else -# include "jemalloc/internal/private_namespace_jet.h" -# endif +#ifndef JEMALLOC_JET +#include "jemalloc/internal/private_namespace.h" +#else +#include "jemalloc/internal/private_namespace_jet.h" +#endif #endif #include "jemalloc/internal/test_hooks.h" #ifdef JEMALLOC_DEFINE_MADVISE_FREE -# define JEMALLOC_MADV_FREE 8 +#define JEMALLOC_MADV_FREE 8 #endif static const bool config_debug = @@ -161,7 +167,55 @@ static const bool config_log = false #endif ; -#ifdef JEMALLOC_HAVE_SCHED_GETCPU +/* + * Are extra safety checks enabled; things like checking the size of sized + * deallocations, double-frees, etc. + */ +static const bool config_opt_safety_checks = +#ifdef JEMALLOC_OPT_SAFETY_CHECKS + true +#elif defined(JEMALLOC_DEBUG) + /* + * This lets us only guard safety checks by one flag instead of two; fast + * checks can guard solely by config_opt_safety_checks and run in debug mode + * too. + */ + true +#else + false +#endif + ; + +/* + * Extra debugging of sized deallocations too onerous to be included in the + * general safety checks. + */ +static const bool config_opt_size_checks = +#if defined(JEMALLOC_OPT_SIZE_CHECKS) || defined(JEMALLOC_DEBUG) + true +#else + false +#endif + ; + +static const bool config_uaf_detection = +#if defined(JEMALLOC_UAF_DETECTION) || defined(JEMALLOC_DEBUG) + true +#else + false +#endif + ; + +/* Whether or not the C++ extensions are enabled. */ +static const bool config_enable_cxx = +#ifdef JEMALLOC_ENABLE_CXX + true +#else + false +#endif + ; + +#if defined(_WIN32) || defined(JEMALLOC_HAVE_SCHED_GETCPU) /* Currently percpu_arena depends on sched_getcpu. */ #define JEMALLOC_PERCPU_ARENA #endif @@ -190,23 +244,16 @@ static const bool have_background_thread = false #endif ; - -#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS 1 -#define JEMALLOC_GCC_U8_SYNC_ATOMICS 1 - -/* - * Are extra safety checks enabled; things like checking the size of sized - * deallocations, double-frees, etc. - */ -static const bool config_opt_safety_checks = -#ifdef JEMALLOC_OPT_SAFETY_CHECKS +static const bool config_high_res_timer = +#ifdef JEMALLOC_HAVE_CLOCK_REALTIME true -#elif defined(JEMALLOC_DEBUG) - /* - * This lets us only guard safety checks by one flag instead of two; fast - * checks can guard solely by config_opt_safety_checks and run in debug mode - * too. - */ +#else + false +#endif + ; + +static const bool have_memcntl = +#ifdef JEMALLOC_HAVE_MEMCNTL true #else false diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc.h index d06243c5239..e90fa892100 100644 --- a/contrib/jemalloc-cmake/include/jemalloc/jemalloc.h +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc.h @@ -4,13 +4,21 @@ extern "C" { #endif +#if !defined(__clang__) +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wredundant-decls" +#endif + #include -#include #include #include +#include #include +#if !defined(__clang__) +#pragma GCC diagnostic pop +#endif + #ifdef __cplusplus } #endif - diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_defs.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h similarity index 67% rename from contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_defs.h rename to contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h index d1389237a77..1fc77be57cf 100644 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_defs.h +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_defs.h @@ -5,15 +5,29 @@ /* Defined if alloc_size attribute is supported. */ #define JEMALLOC_HAVE_ATTR_ALLOC_SIZE +/* Defined if format_arg(...) attribute is supported. */ +#define JEMALLOC_HAVE_ATTR_FORMAT_ARG + +/* Defined if format(gnu_printf, ...) attribute is supported. */ +/* #undef JEMALLOC_HAVE_ATTR_FORMAT_GNU_PRINTF */ + /* Defined if format(printf, ...) attribute is supported. */ #define JEMALLOC_HAVE_ATTR_FORMAT_PRINTF +/* Defined if fallthrough attribute is supported. */ +#define JEMALLOC_HAVE_ATTR_FALLTHROUGH + +/* Defined if cold attribute is supported. */ +#define JEMALLOC_HAVE_ATTR_COLD + /* * Define overrides for non-standard allocator-related functions if they are * present on the system. */ +#if !defined(USE_MUSL) #define JEMALLOC_OVERRIDE_MEMALIGN #define JEMALLOC_OVERRIDE_VALLOC +#endif /* * At least Linux omits the "const" in: @@ -32,11 +46,11 @@ #define JEMALLOC_USE_CXX_THROW #ifdef _MSC_VER -# ifdef _WIN64 -# define LG_SIZEOF_PTR_WIN 3 -# else -# define LG_SIZEOF_PTR_WIN 2 -# endif +#ifdef _WIN64 +#define LG_SIZEOF_PTR_WIN 3 +#else +#define LG_SIZEOF_PTR_WIN 2 +#endif #endif /* sizeof(void *) == 2^LG_SIZEOF_PTR. */ diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_macros.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_macros.h new file mode 100644 index 00000000000..ccb22470e64 --- /dev/null +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_macros.h @@ -0,0 +1,148 @@ +#include +#include +#include +#include +#include + +#define JEMALLOC_VERSION "5.3-RC" +#define JEMALLOC_VERSION_MAJOR 5 +#define JEMALLOC_VERSION_MINOR 3 +#define JEMALLOC_VERSION_BUGFIX 0 +#define JEMALLOC_VERSION_NREV 0 +#define JEMALLOC_VERSION_GID "ca709c3139f77f4c00a903cdee46d71e9028f6c6" +#define JEMALLOC_VERSION_GID_IDENT ca709c3139f77f4c00a903cdee46d71e9028f6c6 + +#define MALLOCX_LG_ALIGN(la) ((int)(la)) +#if LG_SIZEOF_PTR == 2 +#define MALLOCX_ALIGN(a) ((int)(ffs((int)(a)) - 1)) +#else +#define MALLOCX_ALIGN(a) \ + ((int)(((size_t)(a) < (size_t)INT_MAX) ? ffs((int)(a)) - 1 : ffs((int)(((size_t)(a)) >> 32)) + 31)) +#endif +#define MALLOCX_ZERO ((int)0x40) +/* + * Bias tcache index bits so that 0 encodes "automatic tcache management", and 1 + * encodes MALLOCX_TCACHE_NONE. + */ +#define MALLOCX_TCACHE(tc) ((int)(((tc) + 2) << 8)) +#define MALLOCX_TCACHE_NONE MALLOCX_TCACHE(-1) +/* + * Bias arena index bits so that 0 encodes "use an automatically chosen arena". + */ +#define MALLOCX_ARENA(a) ((((int)(a)) + 1) << 20) + +/* + * Use as arena index in "arena..{purge,decay,dss}" and + * "stats.arenas..*" mallctl interfaces to select all arenas. This + * definition is intentionally specified in raw decimal format to support + * cpp-based string concatenation, e.g. + * + * #define STRINGIFY_HELPER(x) #x + * #define STRINGIFY(x) STRINGIFY_HELPER(x) + * + * mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", NULL, NULL, NULL, + * 0); + */ +#define MALLCTL_ARENAS_ALL 4096 +/* + * Use as arena index in "stats.arenas..*" mallctl interfaces to select + * destroyed arenas. + */ +#define MALLCTL_ARENAS_DESTROYED 4097 + +#if defined(__cplusplus) && defined(JEMALLOC_USE_CXX_THROW) +#define JEMALLOC_CXX_THROW throw() +#else +#define JEMALLOC_CXX_THROW +#endif + +#if defined(_MSC_VER) +#define JEMALLOC_ATTR(s) +#define JEMALLOC_ALIGNED(s) __declspec(align(s)) +#define JEMALLOC_ALLOC_SIZE(s) +#define JEMALLOC_ALLOC_SIZE2(s1, s2) +#ifndef JEMALLOC_EXPORT +#ifdef DLLEXPORT +#define JEMALLOC_EXPORT __declspec(dllexport) +#else +#define JEMALLOC_EXPORT __declspec(dllimport) +#endif +#endif +#define JEMALLOC_FORMAT_ARG(i) +#define JEMALLOC_FORMAT_PRINTF(s, i) +#define JEMALLOC_FALLTHROUGH +#define JEMALLOC_NOINLINE __declspec(noinline) +#ifdef __cplusplus +#define JEMALLOC_NOTHROW __declspec(nothrow) +#else +#define JEMALLOC_NOTHROW +#endif +#define JEMALLOC_SECTION(s) __declspec(allocate(s)) +#define JEMALLOC_RESTRICT_RETURN __declspec(restrict) +#if _MSC_VER >= 1900 && !defined(__EDG__) +#define JEMALLOC_ALLOCATOR __declspec(allocator) +#else +#define JEMALLOC_ALLOCATOR +#endif +#define JEMALLOC_COLD +#elif defined(JEMALLOC_HAVE_ATTR) +#define JEMALLOC_ATTR(s) __attribute__((s)) +#define JEMALLOC_ALIGNED(s) JEMALLOC_ATTR(aligned(s)) +#ifdef JEMALLOC_HAVE_ATTR_ALLOC_SIZE +#define JEMALLOC_ALLOC_SIZE(s) JEMALLOC_ATTR(alloc_size(s)) +#define JEMALLOC_ALLOC_SIZE2(s1, s2) JEMALLOC_ATTR(alloc_size(s1, s2)) +#else +#define JEMALLOC_ALLOC_SIZE(s) +#define JEMALLOC_ALLOC_SIZE2(s1, s2) +#endif +#ifndef JEMALLOC_EXPORT +#define JEMALLOC_EXPORT JEMALLOC_ATTR(visibility("default")) +#endif +#ifdef JEMALLOC_HAVE_ATTR_FORMAT_ARG +#define JEMALLOC_FORMAT_ARG(i) JEMALLOC_ATTR(__format_arg__(3)) +#else +#define JEMALLOC_FORMAT_ARG(i) +#endif +#ifdef JEMALLOC_HAVE_ATTR_FORMAT_GNU_PRINTF +#define JEMALLOC_FORMAT_PRINTF(s, i) JEMALLOC_ATTR(format(gnu_printf, s, i)) +#elif defined(JEMALLOC_HAVE_ATTR_FORMAT_PRINTF) +#define JEMALLOC_FORMAT_PRINTF(s, i) JEMALLOC_ATTR(format(printf, s, i)) +#else +#define JEMALLOC_FORMAT_PRINTF(s, i) +#endif +#ifdef JEMALLOC_HAVE_ATTR_FALLTHROUGH +#define JEMALLOC_FALLTHROUGH JEMALLOC_ATTR(fallthrough) +#else +#define JEMALLOC_FALLTHROUGH +#endif +#define JEMALLOC_NOINLINE JEMALLOC_ATTR(noinline) +#define JEMALLOC_NOTHROW JEMALLOC_ATTR(nothrow) +#define JEMALLOC_SECTION(s) JEMALLOC_ATTR(section(s)) +#define JEMALLOC_RESTRICT_RETURN +#define JEMALLOC_ALLOCATOR +#ifdef JEMALLOC_HAVE_ATTR_COLD +#define JEMALLOC_COLD JEMALLOC_ATTR(__cold__) +#else +#define JEMALLOC_COLD +#endif +#else +#define JEMALLOC_ATTR(s) +#define JEMALLOC_ALIGNED(s) +#define JEMALLOC_ALLOC_SIZE(s) +#define JEMALLOC_ALLOC_SIZE2(s1, s2) +#define JEMALLOC_EXPORT +#define JEMALLOC_FORMAT_PRINTF(s, i) +#define JEMALLOC_FALLTHROUGH +#define JEMALLOC_NOINLINE +#define JEMALLOC_NOTHROW +#define JEMALLOC_SECTION(s) +#define JEMALLOC_RESTRICT_RETURN +#define JEMALLOC_ALLOCATOR +#define JEMALLOC_COLD +#endif + +#if (defined(__APPLE__) || defined(__FreeBSD__)) && !defined(JEMALLOC_NO_RENAME) +#define JEMALLOC_SYS_NOTHROW +#else +#define JEMALLOC_SYS_NOTHROW JEMALLOC_NOTHROW +#endif diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h new file mode 100644 index 00000000000..31f72d3a2af --- /dev/null +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos.h @@ -0,0 +1,86 @@ +// OSX does not have this for system alloc functions, so you will get +// "exception specification in declaration" error. +#if defined(__APPLE__) || defined(__FreeBSD__) || defined(USE_MUSL) +#undef JEMALLOC_NOTHROW +#define JEMALLOC_NOTHROW + +#undef JEMALLOC_SYS_NOTHROW +#define JEMALLOC_SYS_NOTHROW + +#undef JEMALLOC_CXX_THROW +#define JEMALLOC_CXX_THROW +#endif + +#include "jemalloc_rename.h" + +/* + * The je_ prefix on the following public symbol declarations is an artifact + * of namespace management, and should be omitted in application code unless + * JEMALLOC_NO_DEMANGLE is defined (see jemalloc_mangle.h). + */ +extern JEMALLOC_EXPORT const char * je_malloc_conf; +extern JEMALLOC_EXPORT void (*je_malloc_message)(void * cbopaque, + const char * s); + +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_SYS_NOTHROW * je_malloc(size_t size) + JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE(1); +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_SYS_NOTHROW * je_calloc(size_t num, size_t size) + JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE2(1, 2); +JEMALLOC_EXPORT int JEMALLOC_SYS_NOTHROW je_posix_memalign( + void ** memptr, + size_t alignment, + size_t size) JEMALLOC_CXX_THROW + JEMALLOC_ATTR(nonnull(1)); +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_SYS_NOTHROW * je_aligned_alloc(size_t alignment, + size_t size) JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) + JEMALLOC_ALLOC_SIZE(2); +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_SYS_NOTHROW * je_realloc(void * ptr, size_t size) + JEMALLOC_CXX_THROW JEMALLOC_ALLOC_SIZE(2); +JEMALLOC_EXPORT void JEMALLOC_SYS_NOTHROW je_free(void * ptr) + JEMALLOC_CXX_THROW; + +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_NOTHROW * je_mallocx(size_t size, int flags) + JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE(1); +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_NOTHROW * je_rallocx(void * ptr, size_t size, int flags) JEMALLOC_ALLOC_SIZE(2); +JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_xallocx(void * ptr, size_t size, size_t extra, int flags); +JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_sallocx(const void * ptr, + int flags) JEMALLOC_ATTR(pure); +JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_dallocx(void * ptr, int flags); +JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_sdallocx(void * ptr, size_t size, int flags); +JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_nallocx(size_t size, int flags) + JEMALLOC_ATTR(pure); + +JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_mallctl(const char * name, + void * oldp, + size_t * oldlenp, + void * newp, + size_t newlen); +JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_mallctlnametomib(const char * name, + size_t * mibp, + size_t * miblenp); +JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_mallctlbymib(const size_t * mib, + size_t miblen, + void * oldp, + size_t * oldlenp, + void * newp, + size_t newlen); +JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_malloc_stats_print( + void (*write_cb)(void *, const char *), + void * je_cbopaque, + const char * opts); +JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_malloc_usable_size( + JEMALLOC_USABLE_SIZE_CONST void * ptr) JEMALLOC_CXX_THROW; +#ifdef JEMALLOC_HAVE_MALLOC_SIZE +JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_malloc_size( + const void * ptr); +#endif + +#ifdef JEMALLOC_OVERRIDE_MEMALIGN +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_SYS_NOTHROW * je_memalign(size_t alignment, size_t size) + JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc); +#endif + +#ifdef JEMALLOC_OVERRIDE_VALLOC +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_SYS_NOTHROW * je_valloc(size_t size) JEMALLOC_CXX_THROW + JEMALLOC_ATTR(malloc); +#endif diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos_jet.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos_jet.h new file mode 100644 index 00000000000..195d57e2997 --- /dev/null +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_protos_jet.h @@ -0,0 +1,71 @@ +/* + * The jet_ prefix on the following public symbol declarations is an artifact + * of namespace management, and should be omitted in application code unless + * JEMALLOC_NO_DEMANGLE is defined (see jemalloc_mangle@install_suffix@.h). + */ +extern JEMALLOC_EXPORT const char * jet_malloc_conf; +extern JEMALLOC_EXPORT void (*jet_malloc_message)(void * cbopaque, + const char * s); + +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_SYS_NOTHROW * jet_malloc(size_t size) + JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE(1); +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_SYS_NOTHROW * jet_calloc(size_t num, size_t size) + JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE2(1, 2); +JEMALLOC_EXPORT int JEMALLOC_SYS_NOTHROW jet_posix_memalign( + void ** memptr, + size_t alignment, + size_t size) JEMALLOC_CXX_THROW + JEMALLOC_ATTR(nonnull(1)); +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_SYS_NOTHROW * jet_aligned_alloc(size_t alignment, + size_t size) JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) + JEMALLOC_ALLOC_SIZE(2); +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_SYS_NOTHROW * jet_realloc(void * ptr, size_t size) + JEMALLOC_CXX_THROW JEMALLOC_ALLOC_SIZE(2); +JEMALLOC_EXPORT void JEMALLOC_SYS_NOTHROW jet_free(void * ptr) + JEMALLOC_CXX_THROW; + +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_NOTHROW * jet_mallocx(size_t size, int flags) + JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE(1); +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_NOTHROW * jet_rallocx(void * ptr, size_t size, int flags) JEMALLOC_ALLOC_SIZE(2); +JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW jet_xallocx(void * ptr, size_t size, size_t extra, int flags); +JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW jet_sallocx(const void * ptr, + int flags) JEMALLOC_ATTR(pure); +JEMALLOC_EXPORT void JEMALLOC_NOTHROW jet_dallocx(void * ptr, int flags); +JEMALLOC_EXPORT void JEMALLOC_NOTHROW jet_sdallocx(void * ptr, size_t size, int flags); +JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW jet_nallocx(size_t size, int flags) + JEMALLOC_ATTR(pure); + +JEMALLOC_EXPORT int JEMALLOC_NOTHROW jet_mallctl(const char * name, + void * oldp, + size_t * oldlenp, + void * newp, + size_t newlen); +JEMALLOC_EXPORT int JEMALLOC_NOTHROW jet_mallctlnametomib(const char * name, + size_t * mibp, + size_t * miblenp); +JEMALLOC_EXPORT int JEMALLOC_NOTHROW jet_mallctlbymib(const size_t * mib, + size_t miblen, + void * oldp, + size_t * oldlenp, + void * newp, + size_t newlen); +JEMALLOC_EXPORT void JEMALLOC_NOTHROW jet_malloc_stats_print( + void (*write_cb)(void *, const char *), + void * jet_cbopaque, + const char * opts); +JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW jet_malloc_usable_size( + JEMALLOC_USABLE_SIZE_CONST void * ptr) JEMALLOC_CXX_THROW; +#ifdef JEMALLOC_HAVE_MALLOC_SIZE +JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW jet_malloc_size( + const void * ptr); +#endif + +#ifdef JEMALLOC_OVERRIDE_MEMALIGN +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_SYS_NOTHROW * jet_memalign(size_t alignment, size_t size) + JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc); +#endif + +#ifdef JEMALLOC_OVERRIDE_VALLOC +JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN void JEMALLOC_SYS_NOTHROW * jet_valloc(size_t size) JEMALLOC_CXX_THROW + JEMALLOC_ATTR(malloc); +#endif diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_rename.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_rename.h index a2ea2dd3533..d032d46752d 100644 --- a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_rename.h +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_rename.h @@ -4,26 +4,28 @@ * these macro definitions. */ #ifndef JEMALLOC_NO_RENAME -# define je_aligned_alloc aligned_alloc -# define je_calloc calloc -# define je_dallocx dallocx -# define je_free free -# define je_mallctl mallctl -# define je_mallctlbymib mallctlbymib -# define je_mallctlnametomib mallctlnametomib -# define je_malloc malloc -# define je_malloc_conf malloc_conf -# define je_malloc_message malloc_message -# define je_malloc_stats_print malloc_stats_print -# define je_malloc_usable_size malloc_usable_size -# define je_mallocx mallocx -# define je_nallocx nallocx -# define je_posix_memalign posix_memalign -# define je_rallocx rallocx -# define je_realloc realloc -# define je_sallocx sallocx -# define je_sdallocx sdallocx -# define je_xallocx xallocx -# define je_memalign memalign -# define je_valloc valloc +#define je_aligned_alloc aligned_alloc +#define je_calloc calloc +#define je_dallocx dallocx +#define je_free free +#define je_mallctl mallctl +#define je_mallctlbymib mallctlbymib +#define je_mallctlnametomib mallctlnametomib +#define je_malloc malloc +#define je_malloc_conf malloc_conf +#define je_malloc_conf_2_conf_harder malloc_conf_2_conf_harder +#define je_malloc_message malloc_message +#define je_malloc_stats_print malloc_stats_print +#define je_malloc_usable_size malloc_usable_size +#define je_mallocx mallocx +#define je_smallocx_ca709c3139f77f4c00a903cdee46d71e9028f6c6 smallocx_ca709c3139f77f4c00a903cdee46d71e9028f6c6 +#define je_nallocx nallocx +#define je_posix_memalign posix_memalign +#define je_rallocx rallocx +#define je_realloc realloc +#define je_sallocx sallocx +#define je_sdallocx sdallocx +#define je_xallocx xallocx +#define je_memalign memalign +#define je_valloc valloc #endif diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_typedefs.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_typedefs.h similarity index 57% rename from contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_typedefs.h rename to contrib/jemalloc-cmake/include/jemalloc/jemalloc_typedefs.h index 1a58874306e..eeaf7a6760e 100644 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_typedefs.h +++ b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_typedefs.h @@ -5,73 +5,66 @@ typedef struct extent_hooks_s extent_hooks_t; * extent_alloc(extent_hooks_t *extent_hooks, void *new_addr, size_t size, * size_t alignment, bool *zero, bool *commit, unsigned arena_ind); */ -typedef void *(extent_alloc_t)(extent_hooks_t *, void *, size_t, size_t, bool *, - bool *, unsigned); +typedef void *(extent_alloc_t)(extent_hooks_t *, void *, size_t, size_t, bool *, bool *, unsigned); /* * bool * extent_dalloc(extent_hooks_t *extent_hooks, void *addr, size_t size, * bool committed, unsigned arena_ind); */ -typedef bool (extent_dalloc_t)(extent_hooks_t *, void *, size_t, bool, - unsigned); +typedef bool(extent_dalloc_t)(extent_hooks_t *, void *, size_t, bool, unsigned); /* * void * extent_destroy(extent_hooks_t *extent_hooks, void *addr, size_t size, * bool committed, unsigned arena_ind); */ -typedef void (extent_destroy_t)(extent_hooks_t *, void *, size_t, bool, - unsigned); +typedef void(extent_destroy_t)(extent_hooks_t *, void *, size_t, bool, unsigned); /* * bool * extent_commit(extent_hooks_t *extent_hooks, void *addr, size_t size, * size_t offset, size_t length, unsigned arena_ind); */ -typedef bool (extent_commit_t)(extent_hooks_t *, void *, size_t, size_t, size_t, - unsigned); +typedef bool(extent_commit_t)(extent_hooks_t *, void *, size_t, size_t, size_t, unsigned); /* * bool * extent_decommit(extent_hooks_t *extent_hooks, void *addr, size_t size, * size_t offset, size_t length, unsigned arena_ind); */ -typedef bool (extent_decommit_t)(extent_hooks_t *, void *, size_t, size_t, - size_t, unsigned); +typedef bool(extent_decommit_t)(extent_hooks_t *, void *, size_t, size_t, size_t, unsigned); /* * bool * extent_purge(extent_hooks_t *extent_hooks, void *addr, size_t size, * size_t offset, size_t length, unsigned arena_ind); */ -typedef bool (extent_purge_t)(extent_hooks_t *, void *, size_t, size_t, size_t, - unsigned); +typedef bool(extent_purge_t)(extent_hooks_t *, void *, size_t, size_t, size_t, unsigned); /* * bool * extent_split(extent_hooks_t *extent_hooks, void *addr, size_t size, * size_t size_a, size_t size_b, bool committed, unsigned arena_ind); */ -typedef bool (extent_split_t)(extent_hooks_t *, void *, size_t, size_t, size_t, - bool, unsigned); +typedef bool(extent_split_t)(extent_hooks_t *, void *, size_t, size_t, size_t, bool, unsigned); /* * bool * extent_merge(extent_hooks_t *extent_hooks, void *addr_a, size_t size_a, * void *addr_b, size_t size_b, bool committed, unsigned arena_ind); */ -typedef bool (extent_merge_t)(extent_hooks_t *, void *, size_t, void *, size_t, - bool, unsigned); +typedef bool(extent_merge_t)(extent_hooks_t *, void *, size_t, void *, size_t, bool, unsigned); -struct extent_hooks_s { - extent_alloc_t *alloc; - extent_dalloc_t *dalloc; - extent_destroy_t *destroy; - extent_commit_t *commit; - extent_decommit_t *decommit; - extent_purge_t *purge_lazy; - extent_purge_t *purge_forced; - extent_split_t *split; - extent_merge_t *merge; +struct extent_hooks_s +{ + extent_alloc_t * alloc; + extent_dalloc_t * dalloc; + extent_destroy_t * destroy; + extent_commit_t * commit; + extent_decommit_t * decommit; + extent_purge_t * purge_lazy; + extent_purge_t * purge_forced; + extent_split_t * split; + extent_merge_t * merge; }; diff --git a/contrib/jemalloc-cmake/include_darwin_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_darwin_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in new file mode 100644 index 00000000000..8ad95c51560 --- /dev/null +++ b/contrib/jemalloc-cmake/include_darwin_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -0,0 +1,425 @@ +/* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ +#ifndef JEMALLOC_INTERNAL_DEFS_H_ +#define JEMALLOC_INTERNAL_DEFS_H_ +/* + * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all + * public APIs to be prefixed. This makes it possible, with some care, to use + * multiple allocators simultaneously. + */ +#define JEMALLOC_PREFIX "je_" +#define JEMALLOC_CPREFIX "JE_" + +/* + * Define overrides for non-standard allocator-related functions if they are + * present on the system. + */ +/* #undef JEMALLOC_OVERRIDE___LIBC_CALLOC */ +/* #undef JEMALLOC_OVERRIDE___LIBC_FREE */ +/* #undef JEMALLOC_OVERRIDE___LIBC_MALLOC */ +/* #undef JEMALLOC_OVERRIDE___LIBC_MEMALIGN */ +/* #undef JEMALLOC_OVERRIDE___LIBC_REALLOC */ +/* #undef JEMALLOC_OVERRIDE___LIBC_VALLOC */ +/* #undef JEMALLOC_OVERRIDE___POSIX_MEMALIGN */ + +/* + * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. + * For shared libraries, symbol visibility mechanisms prevent these symbols + * from being exported, but for static libraries, naming collisions are a real + * possibility. + */ +#define JEMALLOC_PRIVATE_NAMESPACE je_ + +/* + * Hyper-threaded CPUs may need a special instruction inside spin loops in + * order to yield to another virtual CPU. + */ +#define CPU_SPINWAIT +/* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ +#define HAVE_CPU_SPINWAIT 0 + +/* + * Number of significant bits in virtual addresses. This may be less than the + * total number of bits in a pointer, e.g. on x64, for which the uppermost 16 + * bits are the same as bit 47. + */ +#define LG_VADDR 64 + +/* Defined if C11 atomics are available. */ +#define JEMALLOC_C11_ATOMICS + +/* Defined if GCC __atomic atomics are available. */ +#define JEMALLOC_GCC_ATOMIC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS + +/* Defined if GCC __sync atomics are available. */ +#define JEMALLOC_GCC_SYNC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_SYNC_ATOMICS + +/* + * Defined if __builtin_clz() and __builtin_clzl() are available. + */ +#define JEMALLOC_HAVE_BUILTIN_CLZ + +/* + * Defined if os_unfair_lock_*() functions are available, as provided by Darwin. + */ +#define JEMALLOC_OS_UNFAIR_LOCK + +/* Defined if syscall(2) is usable. */ +/* #undef JEMALLOC_USE_SYSCALL */ + +/* + * Defined if secure_getenv(3) is available. + */ +/* #undef JEMALLOC_HAVE_SECURE_GETENV */ + +/* + * Defined if issetugid(2) is available. + */ +#define JEMALLOC_HAVE_ISSETUGID + +/* Defined if pthread_atfork(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_ATFORK + +/* Defined if pthread_setname_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_SETNAME_NP */ + +/* Defined if pthread_getname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_GETNAME_NP + +/* Defined if pthread_get_name_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_GET_NAME_NP */ + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC_COARSE, ...) is available. + */ +/* #undef JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE */ + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC, ...) is available. + */ +/* #undef JEMALLOC_HAVE_CLOCK_MONOTONIC */ + +/* + * Defined if mach_absolute_time() is available. + */ +#define JEMALLOC_HAVE_MACH_ABSOLUTE_TIME + +/* + * Defined if clock_gettime(CLOCK_REALTIME, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_REALTIME + +/* + * Defined if _malloc_thread_cleanup() exists. At least in the case of + * FreeBSD, pthread_key_create() allocates, which if used during malloc + * bootstrapping will cause recursion into the pthreads library. Therefore, if + * _malloc_thread_cleanup() exists, use it as the basis for thread cleanup in + * malloc_tsd. + */ +/* #undef JEMALLOC_MALLOC_THREAD_CLEANUP */ + +/* + * Defined if threaded initialization is known to be safe on this platform. + * Among other things, it must be possible to initialize a mutex without + * triggering allocation in order for threaded allocation to be safe. + */ +/* #undef JEMALLOC_THREADED_INIT */ + +/* + * Defined if the pthreads implementation defines + * _pthread_mutex_init_calloc_cb(), in which case the function is used in order + * to avoid recursive allocation during mutex initialization. + */ +/* #undef JEMALLOC_MUTEX_INIT_CB */ + +/* Non-empty if the tls_model attribute is supported. */ +#define JEMALLOC_TLS_MODEL __attribute__((tls_model("initial-exec"))) + +/* + * JEMALLOC_DEBUG enables assertions and other sanity checks, and disables + * inline functions. + */ +/* #undef JEMALLOC_DEBUG */ + +/* JEMALLOC_STATS enables statistics calculation. */ +#define JEMALLOC_STATS + +/* JEMALLOC_EXPERIMENTAL_SMALLOCX_API enables experimental smallocx API. */ +/* #undef JEMALLOC_EXPERIMENTAL_SMALLOCX_API */ + +/* JEMALLOC_PROF enables allocation profiling. */ +/* #undef JEMALLOC_PROF */ + +/* Use libunwind for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBUNWIND */ + +/* Use libgcc for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBGCC */ + +/* Use gcc intrinsics for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_GCC */ + +/* + * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage + * segment (DSS). + */ +/* #undef JEMALLOC_DSS */ + +/* Support memory filling (junk/zero). */ +#define JEMALLOC_FILL + +/* Support utrace(2)-based tracing. */ +/* #undef JEMALLOC_UTRACE */ + +/* Support utrace(2)-based tracing (label based signature). */ +/* #undef JEMALLOC_UTRACE_LABEL */ + +/* Support optional abort() on OOM. */ +/* #undef JEMALLOC_XMALLOC */ + +/* Support lazy locking (avoid locking unless a second thread is launched). */ +/* #undef JEMALLOC_LAZY_LOCK */ + +/* + * Minimum allocation alignment is 2^LG_QUANTUM bytes (ignoring tiny size + * classes). + */ +/* #undef LG_QUANTUM */ + +/* One page is 2^LG_PAGE bytes. */ +#define LG_PAGE 14 + +/* Maximum number of regions in a slab. */ +/* #undef CONFIG_LG_SLAB_MAXREGS */ + +/* + * One huge page is 2^LG_HUGEPAGE bytes. Note that this is defined even if the + * system does not explicitly support huge pages; system calls that require + * explicit huge page support are separately configured. + */ +#define LG_HUGEPAGE 21 + +/* + * If defined, adjacent virtual memory mappings with identical attributes + * automatically coalesce, and they fragment when changes are made to subranges. + * This is the normal order of things for mmap()/munmap(), but on Windows + * VirtualAlloc()/VirtualFree() operations must be precisely matched, i.e. + * mappings do *not* coalesce/fragment. + */ +#define JEMALLOC_MAPS_COALESCE + +/* + * If defined, retain memory for later reuse by default rather than using e.g. + * munmap() to unmap freed extents. This is enabled on 64-bit Linux because + * common sequences of mmap()/munmap() calls will cause virtual memory map + * holes. + */ +/* #undef JEMALLOC_RETAIN */ + +/* TLS is used to map arenas and magazine caches to threads. */ +/* #undef JEMALLOC_TLS */ + +/* + * Used to mark unreachable code to quiet "end of non-void" compiler warnings. + * Don't use this directly; instead use unreachable() from util.h + */ +#define JEMALLOC_INTERNAL_UNREACHABLE __builtin_unreachable + +/* + * ffs*() functions to use for bitmapping. Don't use these directly; instead, + * use ffs_*() from util.h. + */ +#define JEMALLOC_INTERNAL_FFSLL __builtin_ffsll +#define JEMALLOC_INTERNAL_FFSL __builtin_ffsl +#define JEMALLOC_INTERNAL_FFS __builtin_ffs + +/* + * popcount*() functions to use for bitmapping. + */ +#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl +#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount + +/* + * If defined, explicitly attempt to more uniformly distribute large allocation + * pointer alignments across all cache indices. + */ +#define JEMALLOC_CACHE_OBLIVIOUS + +/* + * If defined, enable logging facilities. We make this a configure option to + * avoid taking extra branches everywhere. + */ +/* #undef JEMALLOC_LOG */ + +/* + * If defined, use readlinkat() (instead of readlink()) to follow + * /etc/malloc_conf. + */ +/* #undef JEMALLOC_READLINKAT */ + +/* + * Darwin (OS X) uses zones to work around Mach-O symbol override shortcomings. + */ +#define JEMALLOC_ZONE + +/* + * Methods for determining whether the OS overcommits. + * JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY: Linux's + * /proc/sys/vm.overcommit_memory file. + * JEMALLOC_SYSCTL_VM_OVERCOMMIT: FreeBSD's vm.overcommit sysctl. + */ +/* #undef JEMALLOC_SYSCTL_VM_OVERCOMMIT */ +/* #undef JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY */ + +/* Defined if madvise(2) is available. */ +#define JEMALLOC_HAVE_MADVISE + +/* + * Defined if transparent huge pages are supported via the MADV_[NO]HUGEPAGE + * arguments to madvise(2). + */ +/* #undef JEMALLOC_HAVE_MADVISE_HUGE */ + +/* + * Methods for purging unused pages differ between operating systems. + * + * madvise(..., MADV_FREE) : This marks pages as being unused, such that they + * will be discarded rather than swapped out. + * madvise(..., MADV_DONTNEED) : If JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS is + * defined, this immediately discards pages, + * such that new pages will be demand-zeroed if + * the address region is later touched; + * otherwise this behaves similarly to + * MADV_FREE, though typically with higher + * system overhead. + */ +#define JEMALLOC_PURGE_MADVISE_FREE +#define JEMALLOC_PURGE_MADVISE_DONTNEED +/* #undef JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS */ + +/* Defined if madvise(2) is available but MADV_FREE is not (x86 Linux only). */ +/* #undef JEMALLOC_DEFINE_MADVISE_FREE */ + +/* + * Defined if MADV_DO[NT]DUMP is supported as an argument to madvise. + */ +/* #undef JEMALLOC_MADVISE_DONTDUMP */ + +/* + * Defined if MADV_[NO]CORE is supported as an argument to madvise. + */ +/* #undef JEMALLOC_MADVISE_NOCORE */ + +/* Defined if mprotect(2) is available. */ +#define JEMALLOC_HAVE_MPROTECT + +/* + * Defined if transparent huge pages (THPs) are supported via the + * MADV_[NO]HUGEPAGE arguments to madvise(2), and THP support is enabled. + */ +/* #undef JEMALLOC_THP */ + +/* Defined if posix_madvise is available. */ +/* #undef JEMALLOC_HAVE_POSIX_MADVISE */ + +/* + * Method for purging unused pages using posix_madvise. + * + * posix_madvise(..., POSIX_MADV_DONTNEED) + */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED_ZEROS */ + +/* + * Defined if memcntl page admin call is supported + */ +/* #undef JEMALLOC_HAVE_MEMCNTL */ + +/* + * Defined if malloc_size is supported + */ +#define JEMALLOC_HAVE_MALLOC_SIZE + +/* Define if operating system has alloca.h header. */ +/* #undef JEMALLOC_HAS_ALLOCA_H */ + +/* C99 restrict keyword supported. */ +#define JEMALLOC_HAS_RESTRICT + +/* For use by hash code. */ +/* #undef JEMALLOC_BIG_ENDIAN */ + +/* sizeof(int) == 2^LG_SIZEOF_INT. */ +#define LG_SIZEOF_INT 2 + +/* sizeof(long) == 2^LG_SIZEOF_LONG. */ +#define LG_SIZEOF_LONG 3 + +/* sizeof(long long) == 2^LG_SIZEOF_LONG_LONG. */ +#define LG_SIZEOF_LONG_LONG 3 + +/* sizeof(intmax_t) == 2^LG_SIZEOF_INTMAX_T. */ +#define LG_SIZEOF_INTMAX_T 3 + +/* glibc malloc hooks (__malloc_hook, __realloc_hook, __free_hook). */ +/* #undef JEMALLOC_GLIBC_MALLOC_HOOK */ + +/* glibc memalign hook. */ +/* #undef JEMALLOC_GLIBC_MEMALIGN_HOOK */ + +/* pthread support */ +#define JEMALLOC_HAVE_PTHREAD + +/* dlsym() support */ +#define JEMALLOC_HAVE_DLSYM + +/* Adaptive mutex support in pthreads. */ +/* #undef JEMALLOC_HAVE_PTHREAD_MUTEX_ADAPTIVE_NP */ + +/* GNU specific sched_getcpu support */ +/* #undef JEMALLOC_HAVE_SCHED_GETCPU */ + +/* GNU specific sched_setaffinity support */ +/* #undef JEMALLOC_HAVE_SCHED_SETAFFINITY */ + +/* + * If defined, all the features necessary for background threads are present. + */ +/* #undef JEMALLOC_BACKGROUND_THREAD */ + +/* + * If defined, jemalloc symbols are not exported (doesn't work when + * JEMALLOC_PREFIX is not defined). + */ +/* #undef JEMALLOC_EXPORT */ + +/* config.malloc_conf options string. */ +#define JEMALLOC_CONFIG_MALLOC_CONF "@JEMALLOC_CONFIG_MALLOC_CONF@" + +/* If defined, jemalloc takes the malloc/free/etc. symbol names. */ +/* #undef JEMALLOC_IS_MALLOC */ + +/* + * Defined if strerror_r returns char * if _GNU_SOURCE is defined. + */ +/* #undef JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE */ + +/* Performs additional safety checks when defined. */ +/* #undef JEMALLOC_OPT_SAFETY_CHECKS */ + +/* Is C++ support being built? */ +/* #undef JEMALLOC_ENABLE_CXX */ + +/* Performs additional size checks when defined. */ +/* #undef JEMALLOC_OPT_SIZE_CHECKS */ + +/* Allows sampled junk and stash for checking use-after-free when defined. */ +/* #undef JEMALLOC_UAF_DETECTION */ + +/* Darwin VM_MAKE_TAG support */ +#define JEMALLOC_HAVE_VM_MAKE_TAG + +#endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_darwin_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_darwin_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in new file mode 100644 index 00000000000..8671da5db69 --- /dev/null +++ b/contrib/jemalloc-cmake/include_darwin_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -0,0 +1,425 @@ +/* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ +#ifndef JEMALLOC_INTERNAL_DEFS_H_ +#define JEMALLOC_INTERNAL_DEFS_H_ +/* + * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all + * public APIs to be prefixed. This makes it possible, with some care, to use + * multiple allocators simultaneously. + */ +#define JEMALLOC_PREFIX "je_" +#define JEMALLOC_CPREFIX "JE_" + +/* + * Define overrides for non-standard allocator-related functions if they are + * present on the system. + */ +/* #undef JEMALLOC_OVERRIDE___LIBC_CALLOC */ +/* #undef JEMALLOC_OVERRIDE___LIBC_FREE */ +/* #undef JEMALLOC_OVERRIDE___LIBC_MALLOC */ +/* #undef JEMALLOC_OVERRIDE___LIBC_MEMALIGN */ +/* #undef JEMALLOC_OVERRIDE___LIBC_REALLOC */ +/* #undef JEMALLOC_OVERRIDE___LIBC_VALLOC */ +/* #undef JEMALLOC_OVERRIDE___POSIX_MEMALIGN */ + +/* + * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. + * For shared libraries, symbol visibility mechanisms prevent these symbols + * from being exported, but for static libraries, naming collisions are a real + * possibility. + */ +#define JEMALLOC_PRIVATE_NAMESPACE je_ + +/* + * Hyper-threaded CPUs may need a special instruction inside spin loops in + * order to yield to another virtual CPU. + */ +#define CPU_SPINWAIT __asm__ volatile("pause") +/* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ +#define HAVE_CPU_SPINWAIT 1 + +/* + * Number of significant bits in virtual addresses. This may be less than the + * total number of bits in a pointer, e.g. on x64, for which the uppermost 16 + * bits are the same as bit 47. + */ +#define LG_VADDR 48 + +/* Defined if C11 atomics are available. */ +#define JEMALLOC_C11_ATOMICS + +/* Defined if GCC __atomic atomics are available. */ +#define JEMALLOC_GCC_ATOMIC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS + +/* Defined if GCC __sync atomics are available. */ +#define JEMALLOC_GCC_SYNC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_SYNC_ATOMICS + +/* + * Defined if __builtin_clz() and __builtin_clzl() are available. + */ +#define JEMALLOC_HAVE_BUILTIN_CLZ + +/* + * Defined if os_unfair_lock_*() functions are available, as provided by Darwin. + */ +#define JEMALLOC_OS_UNFAIR_LOCK + +/* Defined if syscall(2) is usable. */ +/* #undef JEMALLOC_USE_SYSCALL */ + +/* + * Defined if secure_getenv(3) is available. + */ +/* #undef JEMALLOC_HAVE_SECURE_GETENV */ + +/* + * Defined if issetugid(2) is available. + */ +#define JEMALLOC_HAVE_ISSETUGID + +/* Defined if pthread_atfork(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_ATFORK + +/* Defined if pthread_setname_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_SETNAME_NP */ + +/* Defined if pthread_getname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_GETNAME_NP + +/* Defined if pthread_get_name_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_GET_NAME_NP */ + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC_COARSE, ...) is available. + */ +/* #undef JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE */ + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC, ...) is available. + */ +/* #undef JEMALLOC_HAVE_CLOCK_MONOTONIC */ + +/* + * Defined if mach_absolute_time() is available. + */ +#define JEMALLOC_HAVE_MACH_ABSOLUTE_TIME + +/* + * Defined if clock_gettime(CLOCK_REALTIME, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_REALTIME + +/* + * Defined if _malloc_thread_cleanup() exists. At least in the case of + * FreeBSD, pthread_key_create() allocates, which if used during malloc + * bootstrapping will cause recursion into the pthreads library. Therefore, if + * _malloc_thread_cleanup() exists, use it as the basis for thread cleanup in + * malloc_tsd. + */ +/* #undef JEMALLOC_MALLOC_THREAD_CLEANUP */ + +/* + * Defined if threaded initialization is known to be safe on this platform. + * Among other things, it must be possible to initialize a mutex without + * triggering allocation in order for threaded allocation to be safe. + */ +/* #undef JEMALLOC_THREADED_INIT */ + +/* + * Defined if the pthreads implementation defines + * _pthread_mutex_init_calloc_cb(), in which case the function is used in order + * to avoid recursive allocation during mutex initialization. + */ +/* #undef JEMALLOC_MUTEX_INIT_CB */ + +/* Non-empty if the tls_model attribute is supported. */ +#define JEMALLOC_TLS_MODEL __attribute__((tls_model("initial-exec"))) + +/* + * JEMALLOC_DEBUG enables assertions and other sanity checks, and disables + * inline functions. + */ +/* #undef JEMALLOC_DEBUG */ + +/* JEMALLOC_STATS enables statistics calculation. */ +#define JEMALLOC_STATS + +/* JEMALLOC_EXPERIMENTAL_SMALLOCX_API enables experimental smallocx API. */ +/* #undef JEMALLOC_EXPERIMENTAL_SMALLOCX_API */ + +/* JEMALLOC_PROF enables allocation profiling. */ +/* #undef JEMALLOC_PROF */ + +/* Use libunwind for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBUNWIND */ + +/* Use libgcc for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBGCC */ + +/* Use gcc intrinsics for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_GCC */ + +/* + * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage + * segment (DSS). + */ +/* #undef JEMALLOC_DSS */ + +/* Support memory filling (junk/zero). */ +#define JEMALLOC_FILL + +/* Support utrace(2)-based tracing. */ +/* #undef JEMALLOC_UTRACE */ + +/* Support utrace(2)-based tracing (label based signature). */ +/* #undef JEMALLOC_UTRACE_LABEL */ + +/* Support optional abort() on OOM. */ +/* #undef JEMALLOC_XMALLOC */ + +/* Support lazy locking (avoid locking unless a second thread is launched). */ +/* #undef JEMALLOC_LAZY_LOCK */ + +/* + * Minimum allocation alignment is 2^LG_QUANTUM bytes (ignoring tiny size + * classes). + */ +/* #undef LG_QUANTUM */ + +/* One page is 2^LG_PAGE bytes. */ +#define LG_PAGE 12 + +/* Maximum number of regions in a slab. */ +/* #undef CONFIG_LG_SLAB_MAXREGS */ + +/* + * One huge page is 2^LG_HUGEPAGE bytes. Note that this is defined even if the + * system does not explicitly support huge pages; system calls that require + * explicit huge page support are separately configured. + */ +#define LG_HUGEPAGE 21 + +/* + * If defined, adjacent virtual memory mappings with identical attributes + * automatically coalesce, and they fragment when changes are made to subranges. + * This is the normal order of things for mmap()/munmap(), but on Windows + * VirtualAlloc()/VirtualFree() operations must be precisely matched, i.e. + * mappings do *not* coalesce/fragment. + */ +#define JEMALLOC_MAPS_COALESCE + +/* + * If defined, retain memory for later reuse by default rather than using e.g. + * munmap() to unmap freed extents. This is enabled on 64-bit Linux because + * common sequences of mmap()/munmap() calls will cause virtual memory map + * holes. + */ +/* #undef JEMALLOC_RETAIN */ + +/* TLS is used to map arenas and magazine caches to threads. */ +/* #undef JEMALLOC_TLS */ + +/* + * Used to mark unreachable code to quiet "end of non-void" compiler warnings. + * Don't use this directly; instead use unreachable() from util.h + */ +#define JEMALLOC_INTERNAL_UNREACHABLE __builtin_unreachable + +/* + * ffs*() functions to use for bitmapping. Don't use these directly; instead, + * use ffs_*() from util.h. + */ +#define JEMALLOC_INTERNAL_FFSLL __builtin_ffsll +#define JEMALLOC_INTERNAL_FFSL __builtin_ffsl +#define JEMALLOC_INTERNAL_FFS __builtin_ffs + +/* + * popcount*() functions to use for bitmapping. + */ +#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl +#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount + +/* + * If defined, explicitly attempt to more uniformly distribute large allocation + * pointer alignments across all cache indices. + */ +#define JEMALLOC_CACHE_OBLIVIOUS + +/* + * If defined, enable logging facilities. We make this a configure option to + * avoid taking extra branches everywhere. + */ +/* #undef JEMALLOC_LOG */ + +/* + * If defined, use readlinkat() (instead of readlink()) to follow + * /etc/malloc_conf. + */ +/* #undef JEMALLOC_READLINKAT */ + +/* + * Darwin (OS X) uses zones to work around Mach-O symbol override shortcomings. + */ +#define JEMALLOC_ZONE + +/* + * Methods for determining whether the OS overcommits. + * JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY: Linux's + * /proc/sys/vm.overcommit_memory file. + * JEMALLOC_SYSCTL_VM_OVERCOMMIT: FreeBSD's vm.overcommit sysctl. + */ +/* #undef JEMALLOC_SYSCTL_VM_OVERCOMMIT */ +/* #undef JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY */ + +/* Defined if madvise(2) is available. */ +#define JEMALLOC_HAVE_MADVISE + +/* + * Defined if transparent huge pages are supported via the MADV_[NO]HUGEPAGE + * arguments to madvise(2). + */ +/* #undef JEMALLOC_HAVE_MADVISE_HUGE */ + +/* + * Methods for purging unused pages differ between operating systems. + * + * madvise(..., MADV_FREE) : This marks pages as being unused, such that they + * will be discarded rather than swapped out. + * madvise(..., MADV_DONTNEED) : If JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS is + * defined, this immediately discards pages, + * such that new pages will be demand-zeroed if + * the address region is later touched; + * otherwise this behaves similarly to + * MADV_FREE, though typically with higher + * system overhead. + */ +#define JEMALLOC_PURGE_MADVISE_FREE +#define JEMALLOC_PURGE_MADVISE_DONTNEED +/* #undef JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS */ + +/* Defined if madvise(2) is available but MADV_FREE is not (x86 Linux only). */ +/* #undef JEMALLOC_DEFINE_MADVISE_FREE */ + +/* + * Defined if MADV_DO[NT]DUMP is supported as an argument to madvise. + */ +/* #undef JEMALLOC_MADVISE_DONTDUMP */ + +/* + * Defined if MADV_[NO]CORE is supported as an argument to madvise. + */ +/* #undef JEMALLOC_MADVISE_NOCORE */ + +/* Defined if mprotect(2) is available. */ +#define JEMALLOC_HAVE_MPROTECT + +/* + * Defined if transparent huge pages (THPs) are supported via the + * MADV_[NO]HUGEPAGE arguments to madvise(2), and THP support is enabled. + */ +/* #undef JEMALLOC_THP */ + +/* Defined if posix_madvise is available. */ +/* #undef JEMALLOC_HAVE_POSIX_MADVISE */ + +/* + * Method for purging unused pages using posix_madvise. + * + * posix_madvise(..., POSIX_MADV_DONTNEED) + */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED_ZEROS */ + +/* + * Defined if memcntl page admin call is supported + */ +/* #undef JEMALLOC_HAVE_MEMCNTL */ + +/* + * Defined if malloc_size is supported + */ +#define JEMALLOC_HAVE_MALLOC_SIZE + +/* Define if operating system has alloca.h header. */ +/* #undef JEMALLOC_HAS_ALLOCA_H */ + +/* C99 restrict keyword supported. */ +#define JEMALLOC_HAS_RESTRICT + +/* For use by hash code. */ +/* #undef JEMALLOC_BIG_ENDIAN */ + +/* sizeof(int) == 2^LG_SIZEOF_INT. */ +#define LG_SIZEOF_INT 2 + +/* sizeof(long) == 2^LG_SIZEOF_LONG. */ +#define LG_SIZEOF_LONG 3 + +/* sizeof(long long) == 2^LG_SIZEOF_LONG_LONG. */ +#define LG_SIZEOF_LONG_LONG 3 + +/* sizeof(intmax_t) == 2^LG_SIZEOF_INTMAX_T. */ +#define LG_SIZEOF_INTMAX_T 3 + +/* glibc malloc hooks (__malloc_hook, __realloc_hook, __free_hook). */ +/* #undef JEMALLOC_GLIBC_MALLOC_HOOK */ + +/* glibc memalign hook. */ +/* #undef JEMALLOC_GLIBC_MEMALIGN_HOOK */ + +/* pthread support */ +#define JEMALLOC_HAVE_PTHREAD + +/* dlsym() support */ +#define JEMALLOC_HAVE_DLSYM + +/* Adaptive mutex support in pthreads. */ +/* #undef JEMALLOC_HAVE_PTHREAD_MUTEX_ADAPTIVE_NP */ + +/* GNU specific sched_getcpu support */ +/* #undef JEMALLOC_HAVE_SCHED_GETCPU */ + +/* GNU specific sched_setaffinity support */ +/* #undef JEMALLOC_HAVE_SCHED_SETAFFINITY */ + +/* + * If defined, all the features necessary for background threads are present. + */ +/* #undef JEMALLOC_BACKGROUND_THREAD */ + +/* + * If defined, jemalloc symbols are not exported (doesn't work when + * JEMALLOC_PREFIX is not defined). + */ +/* #undef JEMALLOC_EXPORT */ + +/* config.malloc_conf options string. */ +#define JEMALLOC_CONFIG_MALLOC_CONF "@JEMALLOC_CONFIG_MALLOC_CONF@" + +/* If defined, jemalloc takes the malloc/free/etc. symbol names. */ +/* #undef JEMALLOC_IS_MALLOC */ + +/* + * Defined if strerror_r returns char * if _GNU_SOURCE is defined. + */ +/* #undef JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE */ + +/* Performs additional safety checks when defined. */ +/* #undef JEMALLOC_OPT_SAFETY_CHECKS */ + +/* Is C++ support being built? */ +/* #undef JEMALLOC_ENABLE_CXX */ + +/* Performs additional size checks when defined. */ +/* #undef JEMALLOC_OPT_SIZE_CHECKS */ + +/* Allows sampled junk and stash for checking use-after-free when defined. */ +/* #undef JEMALLOC_UAF_DETECTION */ + +/* Darwin VM_MAKE_TAG support */ +#define JEMALLOC_HAVE_VM_MAKE_TAG + +#endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_freebsd_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_freebsd_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in new file mode 100644 index 00000000000..0f61417d65f --- /dev/null +++ b/contrib/jemalloc-cmake/include_freebsd_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -0,0 +1,427 @@ +/* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ +#ifndef JEMALLOC_INTERNAL_DEFS_H_ +#define JEMALLOC_INTERNAL_DEFS_H_ +/* + * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all + * public APIs to be prefixed. This makes it possible, with some care, to use + * multiple allocators simultaneously. + */ +/* #undef JEMALLOC_PREFIX */ +/* #undef JEMALLOC_CPREFIX */ + +/* + * Define overrides for non-standard allocator-related functions if they are + * present on the system. + */ +/* #undef JEMALLOC_OVERRIDE___LIBC_CALLOC */ +/* #undef JEMALLOC_OVERRIDE___LIBC_FREE */ +/* #undef JEMALLOC_OVERRIDE___LIBC_MALLOC */ +/* #undef JEMALLOC_OVERRIDE___LIBC_MEMALIGN */ +/* #undef JEMALLOC_OVERRIDE___LIBC_REALLOC */ +/* #undef JEMALLOC_OVERRIDE___LIBC_VALLOC */ +#define JEMALLOC_OVERRIDE___POSIX_MEMALIGN + +/* + * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. + * For shared libraries, symbol visibility mechanisms prevent these symbols + * from being exported, but for static libraries, naming collisions are a real + * possibility. + */ +#define JEMALLOC_PRIVATE_NAMESPACE je_ + +/* + * Hyper-threaded CPUs may need a special instruction inside spin loops in + * order to yield to another virtual CPU. + */ +#define CPU_SPINWAIT +/* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ +#define HAVE_CPU_SPINWAIT 0 + +/* + * Number of significant bits in virtual addresses. This may be less than the + * total number of bits in a pointer, e.g. on x64, for which the uppermost 16 + * bits are the same as bit 47. + */ +#define LG_VADDR 48 + +/* Defined if C11 atomics are available. */ +#define JEMALLOC_C11_ATOMICS + +/* Defined if GCC __atomic atomics are available. */ +#define JEMALLOC_GCC_ATOMIC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS + +/* Defined if GCC __sync atomics are available. */ +#define JEMALLOC_GCC_SYNC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_SYNC_ATOMICS + +/* + * Defined if __builtin_clz() and __builtin_clzl() are available. + */ +#define JEMALLOC_HAVE_BUILTIN_CLZ + +/* + * Defined if os_unfair_lock_*() functions are available, as provided by Darwin. + */ +/* #undef JEMALLOC_OS_UNFAIR_LOCK */ + +/* Defined if syscall(2) is usable. */ +#define JEMALLOC_USE_SYSCALL + +/* + * Defined if secure_getenv(3) is available. + */ +/* #undef JEMALLOC_HAVE_SECURE_GETENV */ + +/* + * Defined if issetugid(2) is available. + */ +#define JEMALLOC_HAVE_ISSETUGID + +/* Defined if pthread_atfork(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_ATFORK + +/* Only since 12.1-STABLE */ +/* Defined if pthread_setname_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_SETNAME_NP */ + +/* Only since 12.1-STABLE */ +/* Defined if pthread_getname_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_GETNAME_NP */ + +/* Defined if pthread_get_name_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_GET_NAME_NP + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC_COARSE, ...) is available. + */ +/* #undef JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE */ + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC + +/* + * Defined if mach_absolute_time() is available. + */ +/* #undef JEMALLOC_HAVE_MACH_ABSOLUTE_TIME */ + +/* + * Defined if clock_gettime(CLOCK_REALTIME, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_REALTIME + +/* + * Defined if _malloc_thread_cleanup() exists. At least in the case of + * FreeBSD, pthread_key_create() allocates, which if used during malloc + * bootstrapping will cause recursion into the pthreads library. Therefore, if + * _malloc_thread_cleanup() exists, use it as the basis for thread cleanup in + * malloc_tsd. + */ +#define JEMALLOC_MALLOC_THREAD_CLEANUP + +/* + * Defined if threaded initialization is known to be safe on this platform. + * Among other things, it must be possible to initialize a mutex without + * triggering allocation in order for threaded allocation to be safe. + */ +/* #undef JEMALLOC_THREADED_INIT */ + +/* + * Defined if the pthreads implementation defines + * _pthread_mutex_init_calloc_cb(), in which case the function is used in order + * to avoid recursive allocation during mutex initialization. + */ +#define JEMALLOC_MUTEX_INIT_CB + +/* Non-empty if the tls_model attribute is supported. */ +#define JEMALLOC_TLS_MODEL __attribute__((tls_model("initial-exec"))) + +/* + * JEMALLOC_DEBUG enables assertions and other sanity checks, and disables + * inline functions. + */ +/* #undef JEMALLOC_DEBUG */ + +/* JEMALLOC_STATS enables statistics calculation. */ +#define JEMALLOC_STATS + +/* JEMALLOC_EXPERIMENTAL_SMALLOCX_API enables experimental smallocx API. */ +/* #undef JEMALLOC_EXPERIMENTAL_SMALLOCX_API */ + +/* JEMALLOC_PROF enables allocation profiling. */ +/* #undef JEMALLOC_PROF */ + +/* Use libunwind for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBUNWIND */ + +/* Use libgcc for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBGCC */ + +/* Use gcc intrinsics for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_GCC */ + +/* + * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage + * segment (DSS). + */ +#define JEMALLOC_DSS + +/* Support memory filling (junk/zero). */ +#define JEMALLOC_FILL + +/* Support utrace(2)-based tracing. */ +/* #undef JEMALLOC_UTRACE */ + +/* Support utrace(2)-based tracing (label based signature). */ +/* #undef JEMALLOC_UTRACE_LABEL */ + +/* Support optional abort() on OOM. */ +/* #undef JEMALLOC_XMALLOC */ + +/* Support lazy locking (avoid locking unless a second thread is launched). */ +#define JEMALLOC_LAZY_LOCK + +/* + * Minimum allocation alignment is 2^LG_QUANTUM bytes (ignoring tiny size + * classes). + */ +/* #undef LG_QUANTUM */ + +/* One page is 2^LG_PAGE bytes. */ +#define LG_PAGE 16 + +/* Maximum number of regions in a slab. */ +/* #undef CONFIG_LG_SLAB_MAXREGS */ + +/* + * One huge page is 2^LG_HUGEPAGE bytes. Note that this is defined even if the + * system does not explicitly support huge pages; system calls that require + * explicit huge page support are separately configured. + */ +#define LG_HUGEPAGE 29 + +/* + * If defined, adjacent virtual memory mappings with identical attributes + * automatically coalesce, and they fragment when changes are made to subranges. + * This is the normal order of things for mmap()/munmap(), but on Windows + * VirtualAlloc()/VirtualFree() operations must be precisely matched, i.e. + * mappings do *not* coalesce/fragment. + */ +#define JEMALLOC_MAPS_COALESCE + +/* + * If defined, retain memory for later reuse by default rather than using e.g. + * munmap() to unmap freed extents. This is enabled on 64-bit Linux because + * common sequences of mmap()/munmap() calls will cause virtual memory map + * holes. + */ +/* #undef JEMALLOC_RETAIN */ + +/* TLS is used to map arenas and magazine caches to threads. */ +#define JEMALLOC_TLS + +/* + * Used to mark unreachable code to quiet "end of non-void" compiler warnings. + * Don't use this directly; instead use unreachable() from util.h + */ +#define JEMALLOC_INTERNAL_UNREACHABLE __builtin_unreachable + +/* + * ffs*() functions to use for bitmapping. Don't use these directly; instead, + * use ffs_*() from util.h. + */ +#define JEMALLOC_INTERNAL_FFSLL __builtin_ffsll +#define JEMALLOC_INTERNAL_FFSL __builtin_ffsl +#define JEMALLOC_INTERNAL_FFS __builtin_ffs + +/* + * popcount*() functions to use for bitmapping. + */ +#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl +#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount + +/* + * If defined, explicitly attempt to more uniformly distribute large allocation + * pointer alignments across all cache indices. + */ +#define JEMALLOC_CACHE_OBLIVIOUS + +/* + * If defined, enable logging facilities. We make this a configure option to + * avoid taking extra branches everywhere. + */ +/* #undef JEMALLOC_LOG */ + +/* + * If defined, use readlinkat() (instead of readlink()) to follow + * /etc/malloc_conf. + */ +/* #undef JEMALLOC_READLINKAT */ + +/* + * Darwin (OS X) uses zones to work around Mach-O symbol override shortcomings. + */ +/* #undef JEMALLOC_ZONE */ + +/* + * Methods for determining whether the OS overcommits. + * JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY: Linux's + * /proc/sys/vm.overcommit_memory file. + * JEMALLOC_SYSCTL_VM_OVERCOMMIT: FreeBSD's vm.overcommit sysctl. + */ +#define JEMALLOC_SYSCTL_VM_OVERCOMMIT +/* #undef JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY */ + +/* Defined if madvise(2) is available. */ +#define JEMALLOC_HAVE_MADVISE + +/* + * Defined if transparent huge pages are supported via the MADV_[NO]HUGEPAGE + * arguments to madvise(2). + */ +/* #undef JEMALLOC_HAVE_MADVISE_HUGE */ + +/* + * Methods for purging unused pages differ between operating systems. + * + * madvise(..., MADV_FREE) : This marks pages as being unused, such that they + * will be discarded rather than swapped out. + * madvise(..., MADV_DONTNEED) : If JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS is + * defined, this immediately discards pages, + * such that new pages will be demand-zeroed if + * the address region is later touched; + * otherwise this behaves similarly to + * MADV_FREE, though typically with higher + * system overhead. + */ +#define JEMALLOC_PURGE_MADVISE_FREE +#define JEMALLOC_PURGE_MADVISE_DONTNEED +/* #undef JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS */ + +/* Defined if madvise(2) is available but MADV_FREE is not (x86 Linux only). */ +/* #undef JEMALLOC_DEFINE_MADVISE_FREE */ + +/* + * Defined if MADV_DO[NT]DUMP is supported as an argument to madvise. + */ +/* #undef JEMALLOC_MADVISE_DONTDUMP */ + +/* + * Defined if MADV_[NO]CORE is supported as an argument to madvise. + */ +#define JEMALLOC_MADVISE_NOCORE + +/* Defined if mprotect(2) is available. */ +#define JEMALLOC_HAVE_MPROTECT + +/* + * Defined if transparent huge pages (THPs) are supported via the + * MADV_[NO]HUGEPAGE arguments to madvise(2), and THP support is enabled. + */ +/* #undef JEMALLOC_THP */ + +/* Defined if posix_madvise is available. */ +/* #undef JEMALLOC_HAVE_POSIX_MADVISE */ + +/* + * Method for purging unused pages using posix_madvise. + * + * posix_madvise(..., POSIX_MADV_DONTNEED) + */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED_ZEROS */ + +/* + * Defined if memcntl page admin call is supported + */ +/* #undef JEMALLOC_HAVE_MEMCNTL */ + +/* + * Defined if malloc_size is supported + */ +/* #undef JEMALLOC_HAVE_MALLOC_SIZE */ + +/* Define if operating system has alloca.h header. */ +/* #undef JEMALLOC_HAS_ALLOCA_H */ + +/* C99 restrict keyword supported. */ +#define JEMALLOC_HAS_RESTRICT + +/* For use by hash code. */ +/* #undef JEMALLOC_BIG_ENDIAN */ + +/* sizeof(int) == 2^LG_SIZEOF_INT. */ +#define LG_SIZEOF_INT 2 + +/* sizeof(long) == 2^LG_SIZEOF_LONG. */ +#define LG_SIZEOF_LONG 3 + +/* sizeof(long long) == 2^LG_SIZEOF_LONG_LONG. */ +#define LG_SIZEOF_LONG_LONG 3 + +/* sizeof(intmax_t) == 2^LG_SIZEOF_INTMAX_T. */ +#define LG_SIZEOF_INTMAX_T 3 + +/* glibc malloc hooks (__malloc_hook, __realloc_hook, __free_hook). */ +/* #undef JEMALLOC_GLIBC_MALLOC_HOOK */ + +/* glibc memalign hook. */ +/* #undef JEMALLOC_GLIBC_MEMALIGN_HOOK */ + +/* pthread support */ +#define JEMALLOC_HAVE_PTHREAD + +/* dlsym() support */ +#define JEMALLOC_HAVE_DLSYM + +/* Adaptive mutex support in pthreads. */ +#define JEMALLOC_HAVE_PTHREAD_MUTEX_ADAPTIVE_NP + +/* GNU specific sched_getcpu support */ +/* #undef JEMALLOC_HAVE_SCHED_GETCPU */ + +/* GNU specific sched_setaffinity support */ +/* #undef JEMALLOC_HAVE_SCHED_SETAFFINITY */ + +/* + * If defined, all the features necessary for background threads are present. + */ +#define JEMALLOC_BACKGROUND_THREAD + +/* + * If defined, jemalloc symbols are not exported (doesn't work when + * JEMALLOC_PREFIX is not defined). + */ +/* #undef JEMALLOC_EXPORT */ + +/* config.malloc_conf options string. */ +#define JEMALLOC_CONFIG_MALLOC_CONF "@JEMALLOC_CONFIG_MALLOC_CONF@" + +/* If defined, jemalloc takes the malloc/free/etc. symbol names. */ +#define JEMALLOC_IS_MALLOC + +/* + * Defined if strerror_r returns char * if _GNU_SOURCE is defined. + */ +/* #undef JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE */ + +/* Performs additional safety checks when defined. */ +/* #undef JEMALLOC_OPT_SAFETY_CHECKS */ + +/* Is C++ support being built? */ +/* #undef JEMALLOC_ENABLE_CXX */ + +/* Performs additional size checks when defined. */ +/* #undef JEMALLOC_OPT_SIZE_CHECKS */ + +/* Allows sampled junk and stash for checking use-after-free when defined. */ +/* #undef JEMALLOC_UAF_DETECTION */ + +/* Darwin VM_MAKE_TAG support */ +/* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ + +#endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_freebsd_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_freebsd_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in new file mode 100644 index 00000000000..32cad025f5f --- /dev/null +++ b/contrib/jemalloc-cmake/include_freebsd_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -0,0 +1,427 @@ +/* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ +#ifndef JEMALLOC_INTERNAL_DEFS_H_ +#define JEMALLOC_INTERNAL_DEFS_H_ +/* + * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all + * public APIs to be prefixed. This makes it possible, with some care, to use + * multiple allocators simultaneously. + */ +/* #undef JEMALLOC_PREFIX */ +/* #undef JEMALLOC_CPREFIX */ + +/* + * Define overrides for non-standard allocator-related functions if they are + * present on the system. + */ +/* #undef JEMALLOC_OVERRIDE___LIBC_CALLOC */ +/* #undef JEMALLOC_OVERRIDE___LIBC_FREE */ +/* #undef JEMALLOC_OVERRIDE___LIBC_MALLOC */ +/* #undef JEMALLOC_OVERRIDE___LIBC_MEMALIGN */ +/* #undef JEMALLOC_OVERRIDE___LIBC_REALLOC */ +/* #undef JEMALLOC_OVERRIDE___LIBC_VALLOC */ +#define JEMALLOC_OVERRIDE___POSIX_MEMALIGN + +/* + * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. + * For shared libraries, symbol visibility mechanisms prevent these symbols + * from being exported, but for static libraries, naming collisions are a real + * possibility. + */ +#define JEMALLOC_PRIVATE_NAMESPACE je_ + +/* + * Hyper-threaded CPUs may need a special instruction inside spin loops in + * order to yield to another virtual CPU. + */ +#define CPU_SPINWAIT __asm__ volatile("pause") +/* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ +#define HAVE_CPU_SPINWAIT 1 + +/* + * Number of significant bits in virtual addresses. This may be less than the + * total number of bits in a pointer, e.g. on x64, for which the uppermost 16 + * bits are the same as bit 47. + */ +#define LG_VADDR 48 + +/* Defined if C11 atomics are available. */ +#define JEMALLOC_C11_ATOMICS + +/* Defined if GCC __atomic atomics are available. */ +#define JEMALLOC_GCC_ATOMIC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS + +/* Defined if GCC __sync atomics are available. */ +#define JEMALLOC_GCC_SYNC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_SYNC_ATOMICS + +/* + * Defined if __builtin_clz() and __builtin_clzl() are available. + */ +#define JEMALLOC_HAVE_BUILTIN_CLZ + +/* + * Defined if os_unfair_lock_*() functions are available, as provided by Darwin. + */ +/* #undef JEMALLOC_OS_UNFAIR_LOCK */ + +/* Defined if syscall(2) is usable. */ +#define JEMALLOC_USE_SYSCALL + +/* + * Defined if secure_getenv(3) is available. + */ +/* #undef JEMALLOC_HAVE_SECURE_GETENV */ + +/* + * Defined if issetugid(2) is available. + */ +#define JEMALLOC_HAVE_ISSETUGID + +/* Defined if pthread_atfork(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_ATFORK + +/* Only since 12.1-STABLE */ +/* Defined if pthread_setname_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_SETNAME_NP */ + +/* Only since 12.1-STABLE */ +/* Defined if pthread_getname_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_GETNAME_NP */ + +/* Defined if pthread_get_name_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_GET_NAME_NP + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC_COARSE, ...) is available. + */ +/* #undef JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE */ + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC + +/* + * Defined if mach_absolute_time() is available. + */ +/* #undef JEMALLOC_HAVE_MACH_ABSOLUTE_TIME */ + +/* + * Defined if clock_gettime(CLOCK_REALTIME, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_REALTIME + +/* + * Defined if _malloc_thread_cleanup() exists. At least in the case of + * FreeBSD, pthread_key_create() allocates, which if used during malloc + * bootstrapping will cause recursion into the pthreads library. Therefore, if + * _malloc_thread_cleanup() exists, use it as the basis for thread cleanup in + * malloc_tsd. + */ +#define JEMALLOC_MALLOC_THREAD_CLEANUP + +/* + * Defined if threaded initialization is known to be safe on this platform. + * Among other things, it must be possible to initialize a mutex without + * triggering allocation in order for threaded allocation to be safe. + */ +/* #undef JEMALLOC_THREADED_INIT */ + +/* + * Defined if the pthreads implementation defines + * _pthread_mutex_init_calloc_cb(), in which case the function is used in order + * to avoid recursive allocation during mutex initialization. + */ +#define JEMALLOC_MUTEX_INIT_CB + +/* Non-empty if the tls_model attribute is supported. */ +#define JEMALLOC_TLS_MODEL __attribute__((tls_model("initial-exec"))) + +/* + * JEMALLOC_DEBUG enables assertions and other sanity checks, and disables + * inline functions. + */ +/* #undef JEMALLOC_DEBUG */ + +/* JEMALLOC_STATS enables statistics calculation. */ +#define JEMALLOC_STATS + +/* JEMALLOC_EXPERIMENTAL_SMALLOCX_API enables experimental smallocx API. */ +/* #undef JEMALLOC_EXPERIMENTAL_SMALLOCX_API */ + +/* JEMALLOC_PROF enables allocation profiling. */ +/* #undef JEMALLOC_PROF */ + +/* Use libunwind for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBUNWIND */ + +/* Use libgcc for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBGCC */ + +/* Use gcc intrinsics for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_GCC */ + +/* + * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage + * segment (DSS). + */ +#define JEMALLOC_DSS + +/* Support memory filling (junk/zero). */ +#define JEMALLOC_FILL + +/* Support utrace(2)-based tracing. */ +/* #undef JEMALLOC_UTRACE */ + +/* Support utrace(2)-based tracing (label based signature). */ +/* #undef JEMALLOC_UTRACE_LABEL */ + +/* Support optional abort() on OOM. */ +/* #undef JEMALLOC_XMALLOC */ + +/* Support lazy locking (avoid locking unless a second thread is launched). */ +#define JEMALLOC_LAZY_LOCK + +/* + * Minimum allocation alignment is 2^LG_QUANTUM bytes (ignoring tiny size + * classes). + */ +/* #undef LG_QUANTUM */ + +/* One page is 2^LG_PAGE bytes. */ +#define LG_PAGE 12 + +/* Maximum number of regions in a slab. */ +/* #undef CONFIG_LG_SLAB_MAXREGS */ + +/* + * One huge page is 2^LG_HUGEPAGE bytes. Note that this is defined even if the + * system does not explicitly support huge pages; system calls that require + * explicit huge page support are separately configured. + */ +#define LG_HUGEPAGE 21 + +/* + * If defined, adjacent virtual memory mappings with identical attributes + * automatically coalesce, and they fragment when changes are made to subranges. + * This is the normal order of things for mmap()/munmap(), but on Windows + * VirtualAlloc()/VirtualFree() operations must be precisely matched, i.e. + * mappings do *not* coalesce/fragment. + */ +#define JEMALLOC_MAPS_COALESCE + +/* + * If defined, retain memory for later reuse by default rather than using e.g. + * munmap() to unmap freed extents. This is enabled on 64-bit Linux because + * common sequences of mmap()/munmap() calls will cause virtual memory map + * holes. + */ +/* #undef JEMALLOC_RETAIN */ + +/* TLS is used to map arenas and magazine caches to threads. */ +#define JEMALLOC_TLS + +/* + * Used to mark unreachable code to quiet "end of non-void" compiler warnings. + * Don't use this directly; instead use unreachable() from util.h + */ +#define JEMALLOC_INTERNAL_UNREACHABLE __builtin_unreachable + +/* + * ffs*() functions to use for bitmapping. Don't use these directly; instead, + * use ffs_*() from util.h. + */ +#define JEMALLOC_INTERNAL_FFSLL __builtin_ffsll +#define JEMALLOC_INTERNAL_FFSL __builtin_ffsl +#define JEMALLOC_INTERNAL_FFS __builtin_ffs + +/* + * popcount*() functions to use for bitmapping. + */ +#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl +#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount + +/* + * If defined, explicitly attempt to more uniformly distribute large allocation + * pointer alignments across all cache indices. + */ +#define JEMALLOC_CACHE_OBLIVIOUS + +/* + * If defined, enable logging facilities. We make this a configure option to + * avoid taking extra branches everywhere. + */ +/* #undef JEMALLOC_LOG */ + +/* + * If defined, use readlinkat() (instead of readlink()) to follow + * /etc/malloc_conf. + */ +/* #undef JEMALLOC_READLINKAT */ + +/* + * Darwin (OS X) uses zones to work around Mach-O symbol override shortcomings. + */ +/* #undef JEMALLOC_ZONE */ + +/* + * Methods for determining whether the OS overcommits. + * JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY: Linux's + * /proc/sys/vm.overcommit_memory file. + * JEMALLOC_SYSCTL_VM_OVERCOMMIT: FreeBSD's vm.overcommit sysctl. + */ +#define JEMALLOC_SYSCTL_VM_OVERCOMMIT +/* #undef JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY */ + +/* Defined if madvise(2) is available. */ +#define JEMALLOC_HAVE_MADVISE + +/* + * Defined if transparent huge pages are supported via the MADV_[NO]HUGEPAGE + * arguments to madvise(2). + */ +/* #undef JEMALLOC_HAVE_MADVISE_HUGE */ + +/* + * Methods for purging unused pages differ between operating systems. + * + * madvise(..., MADV_FREE) : This marks pages as being unused, such that they + * will be discarded rather than swapped out. + * madvise(..., MADV_DONTNEED) : If JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS is + * defined, this immediately discards pages, + * such that new pages will be demand-zeroed if + * the address region is later touched; + * otherwise this behaves similarly to + * MADV_FREE, though typically with higher + * system overhead. + */ +#define JEMALLOC_PURGE_MADVISE_FREE +#define JEMALLOC_PURGE_MADVISE_DONTNEED +/* #undef JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS */ + +/* Defined if madvise(2) is available but MADV_FREE is not (x86 Linux only). */ +/* #undef JEMALLOC_DEFINE_MADVISE_FREE */ + +/* + * Defined if MADV_DO[NT]DUMP is supported as an argument to madvise. + */ +/* #undef JEMALLOC_MADVISE_DONTDUMP */ + +/* + * Defined if MADV_[NO]CORE is supported as an argument to madvise. + */ +#define JEMALLOC_MADVISE_NOCORE + +/* Defined if mprotect(2) is available. */ +#define JEMALLOC_HAVE_MPROTECT + +/* + * Defined if transparent huge pages (THPs) are supported via the + * MADV_[NO]HUGEPAGE arguments to madvise(2), and THP support is enabled. + */ +/* #undef JEMALLOC_THP */ + +/* Defined if posix_madvise is available. */ +/* #undef JEMALLOC_HAVE_POSIX_MADVISE */ + +/* + * Method for purging unused pages using posix_madvise. + * + * posix_madvise(..., POSIX_MADV_DONTNEED) + */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED_ZEROS */ + +/* + * Defined if memcntl page admin call is supported + */ +/* #undef JEMALLOC_HAVE_MEMCNTL */ + +/* + * Defined if malloc_size is supported + */ +/* #undef JEMALLOC_HAVE_MALLOC_SIZE */ + +/* Define if operating system has alloca.h header. */ +/* #undef JEMALLOC_HAS_ALLOCA_H */ + +/* C99 restrict keyword supported. */ +#define JEMALLOC_HAS_RESTRICT + +/* For use by hash code. */ +/* #undef JEMALLOC_BIG_ENDIAN */ + +/* sizeof(int) == 2^LG_SIZEOF_INT. */ +#define LG_SIZEOF_INT 2 + +/* sizeof(long) == 2^LG_SIZEOF_LONG. */ +#define LG_SIZEOF_LONG 3 + +/* sizeof(long long) == 2^LG_SIZEOF_LONG_LONG. */ +#define LG_SIZEOF_LONG_LONG 3 + +/* sizeof(intmax_t) == 2^LG_SIZEOF_INTMAX_T. */ +#define LG_SIZEOF_INTMAX_T 3 + +/* glibc malloc hooks (__malloc_hook, __realloc_hook, __free_hook). */ +/* #undef JEMALLOC_GLIBC_MALLOC_HOOK */ + +/* glibc memalign hook. */ +/* #undef JEMALLOC_GLIBC_MEMALIGN_HOOK */ + +/* pthread support */ +#define JEMALLOC_HAVE_PTHREAD + +/* dlsym() support */ +#define JEMALLOC_HAVE_DLSYM + +/* Adaptive mutex support in pthreads. */ +#define JEMALLOC_HAVE_PTHREAD_MUTEX_ADAPTIVE_NP + +/* GNU specific sched_getcpu support */ +/* #undef JEMALLOC_HAVE_SCHED_GETCPU */ + +/* GNU specific sched_setaffinity support */ +/* #undef JEMALLOC_HAVE_SCHED_SETAFFINITY */ + +/* + * If defined, all the features necessary for background threads are present. + */ +#define JEMALLOC_BACKGROUND_THREAD + +/* + * If defined, jemalloc symbols are not exported (doesn't work when + * JEMALLOC_PREFIX is not defined). + */ +/* #undef JEMALLOC_EXPORT */ + +/* config.malloc_conf options string. */ +#define JEMALLOC_CONFIG_MALLOC_CONF "@JEMALLOC_CONFIG_MALLOC_CONF@" + +/* If defined, jemalloc takes the malloc/free/etc. symbol names. */ +#define JEMALLOC_IS_MALLOC + +/* + * Defined if strerror_r returns char * if _GNU_SOURCE is defined. + */ +/* #undef JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE */ + +/* Performs additional safety checks when defined. */ +/* #undef JEMALLOC_OPT_SAFETY_CHECKS */ + +/* Is C++ support being built? */ +/* #undef JEMALLOC_ENABLE_CXX */ + +/* Performs additional size checks when defined. */ +/* #undef JEMALLOC_OPT_SIZE_CHECKS */ + +/* Allows sampled junk and stash for checking use-after-free when defined. */ +/* #undef JEMALLOC_UAF_DETECTION */ + +/* Darwin VM_MAKE_TAG support */ +/* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ + +#endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/README b/contrib/jemalloc-cmake/include_linux_aarch64/README deleted file mode 100644 index 2ab582803a2..00000000000 --- a/contrib/jemalloc-cmake/include_linux_aarch64/README +++ /dev/null @@ -1,7 +0,0 @@ -Here are pre-generated files from jemalloc on Linux aarch64. -You can obtain these files by running ./autogen.sh inside jemalloc source directory. - -Added #define GNU_SOURCE -Added JEMALLOC_OVERRIDE___POSIX_MEMALIGN because why not. -Removed JEMALLOC_HAVE_ATTR_FORMAT_GNU_PRINTF because it's non standard. -Removed JEMALLOC_PURGE_MADVISE_FREE because it's available only from Linux 4.5. diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in similarity index 80% rename from contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h rename to contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in index 5e598348e72..ad535e6d773 100644 --- a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h +++ b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -1,12 +1,6 @@ /* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ #ifndef JEMALLOC_INTERNAL_DEFS_H_ #define JEMALLOC_INTERNAL_DEFS_H_ - - -#ifndef _GNU_SOURCE - #define _GNU_SOURCE -#endif - /* * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all * public APIs to be prefixed. This makes it possible, with some care, to use @@ -19,13 +13,15 @@ * Define overrides for non-standard allocator-related functions if they are * present on the system. */ -#define JEMALLOC_OVERRIDE___LIBC_CALLOC -#define JEMALLOC_OVERRIDE___LIBC_FREE -#define JEMALLOC_OVERRIDE___LIBC_MALLOC -#define JEMALLOC_OVERRIDE___LIBC_MEMALIGN -#define JEMALLOC_OVERRIDE___LIBC_REALLOC -#define JEMALLOC_OVERRIDE___LIBC_VALLOC -#define JEMALLOC_OVERRIDE___POSIX_MEMALIGN +#if !defined(USE_MUSL) + #define JEMALLOC_OVERRIDE___LIBC_CALLOC + #define JEMALLOC_OVERRIDE___LIBC_FREE + #define JEMALLOC_OVERRIDE___LIBC_MALLOC + #define JEMALLOC_OVERRIDE___LIBC_MEMALIGN + #define JEMALLOC_OVERRIDE___LIBC_REALLOC + #define JEMALLOC_OVERRIDE___LIBC_VALLOC +#endif +/* #undef JEMALLOC_OVERRIDE___POSIX_MEMALIGN */ /* * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. @@ -51,29 +47,17 @@ #define LG_VADDR 48 /* Defined if C11 atomics are available. */ -#define JEMALLOC_C11_ATOMICS 1 +#define JEMALLOC_C11_ATOMICS /* Defined if GCC __atomic atomics are available. */ -#define JEMALLOC_GCC_ATOMIC_ATOMICS 1 +#define JEMALLOC_GCC_ATOMIC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS /* Defined if GCC __sync atomics are available. */ -#define JEMALLOC_GCC_SYNC_ATOMICS 1 - -/* - * Defined if __sync_add_and_fetch(uint32_t *, uint32_t) and - * __sync_sub_and_fetch(uint32_t *, uint32_t) are available, despite - * __GCC_HAVE_SYNC_COMPARE_AND_SWAP_4 not being defined (which means the - * functions are defined in libgcc instead of being inlines). - */ -/* #undef JE_FORCE_SYNC_COMPARE_AND_SWAP_4 */ - -/* - * Defined if __sync_add_and_fetch(uint64_t *, uint64_t) and - * __sync_sub_and_fetch(uint64_t *, uint64_t) are available, despite - * __GCC_HAVE_SYNC_COMPARE_AND_SWAP_8 not being defined (which means the - * functions are defined in libgcc instead of being inlines). - */ -/* #undef JE_FORCE_SYNC_COMPARE_AND_SWAP_8 */ +#define JEMALLOC_GCC_SYNC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_SYNC_ATOMICS /* * Defined if __builtin_clz() and __builtin_clzl() are available. @@ -85,19 +69,13 @@ */ /* #undef JEMALLOC_OS_UNFAIR_LOCK */ -/* - * Defined if OSSpin*() functions are available, as provided by Darwin, and - * documented in the spinlock(3) manual page. - */ -/* #undef JEMALLOC_OSSPIN */ - /* Defined if syscall(2) is usable. */ #define JEMALLOC_USE_SYSCALL /* * Defined if secure_getenv(3) is available. */ -#define JEMALLOC_HAVE_SECURE_GETENV +/* #undef JEMALLOC_HAVE_SECURE_GETENV */ /* * Defined if issetugid(2) is available. @@ -110,21 +88,32 @@ /* Defined if pthread_setname_np(3) is available. */ #define JEMALLOC_HAVE_PTHREAD_SETNAME_NP +/* Defined if pthread_getname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_GETNAME_NP + +/* Defined if pthread_get_name_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_GET_NAME_NP */ + /* * Defined if clock_gettime(CLOCK_MONOTONIC_COARSE, ...) is available. */ -#define JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE 1 +#define JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE /* * Defined if clock_gettime(CLOCK_MONOTONIC, ...) is available. */ -#define JEMALLOC_HAVE_CLOCK_MONOTONIC 1 +#define JEMALLOC_HAVE_CLOCK_MONOTONIC /* * Defined if mach_absolute_time() is available. */ /* #undef JEMALLOC_HAVE_MACH_ABSOLUTE_TIME */ +/* + * Defined if clock_gettime(CLOCK_REALTIME, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_REALTIME + /* * Defined if _malloc_thread_cleanup() exists. At least in the case of * FreeBSD, pthread_key_create() allocates, which if used during malloc @@ -187,6 +176,9 @@ /* Support utrace(2)-based tracing. */ /* #undef JEMALLOC_UTRACE */ +/* Support utrace(2)-based tracing (label based signature). */ +/* #undef JEMALLOC_UTRACE_LABEL */ + /* Support optional abort() on OOM. */ /* #undef JEMALLOC_XMALLOC */ @@ -202,6 +194,9 @@ /* One page is 2^LG_PAGE bytes. */ #define LG_PAGE 16 +/* Maximum number of regions in a slab. */ +/* #undef CONFIG_LG_SLAB_MAXREGS */ + /* * One huge page is 2^LG_HUGEPAGE bytes. Note that this is defined even if the * system does not explicitly support huge pages; system calls that require @@ -243,6 +238,12 @@ #define JEMALLOC_INTERNAL_FFSL __builtin_ffsl #define JEMALLOC_INTERNAL_FFS __builtin_ffs +/* + * popcount*() functions to use for bitmapping. + */ +#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl +#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount + /* * If defined, explicitly attempt to more uniformly distribute large allocation * pointer alignments across all cache indices. @@ -297,7 +298,7 @@ * MADV_FREE, though typically with higher * system overhead. */ -// #define JEMALLOC_PURGE_MADVISE_FREE +#define JEMALLOC_PURGE_MADVISE_FREE #define JEMALLOC_PURGE_MADVISE_DONTNEED #define JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS @@ -309,17 +310,46 @@ */ #define JEMALLOC_MADVISE_DONTDUMP +/* + * Defined if MADV_[NO]CORE is supported as an argument to madvise. + */ +/* #undef JEMALLOC_MADVISE_NOCORE */ + +/* Defined if mprotect(2) is available. */ +#define JEMALLOC_HAVE_MPROTECT + /* * Defined if transparent huge pages (THPs) are supported via the * MADV_[NO]HUGEPAGE arguments to madvise(2), and THP support is enabled. */ /* #undef JEMALLOC_THP */ +/* Defined if posix_madvise is available. */ +/* #undef JEMALLOC_HAVE_POSIX_MADVISE */ + +/* + * Method for purging unused pages using posix_madvise. + * + * posix_madvise(..., POSIX_MADV_DONTNEED) + */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED_ZEROS */ + +/* + * Defined if memcntl page admin call is supported + */ +/* #undef JEMALLOC_HAVE_MEMCNTL */ + +/* + * Defined if malloc_size is supported + */ +/* #undef JEMALLOC_HAVE_MALLOC_SIZE */ + /* Define if operating system has alloca.h header. */ -#define JEMALLOC_HAS_ALLOCA_H 1 +#define JEMALLOC_HAS_ALLOCA_H /* C99 restrict keyword supported. */ -#define JEMALLOC_HAS_RESTRICT 1 +#define JEMALLOC_HAS_RESTRICT /* For use by hash code. */ /* #undef JEMALLOC_BIG_ENDIAN */ @@ -360,7 +390,7 @@ /* * If defined, all the features necessary for background threads are present. */ -#define JEMALLOC_BACKGROUND_THREAD 1 +#define JEMALLOC_BACKGROUND_THREAD /* * If defined, jemalloc symbols are not exported (doesn't work when @@ -369,20 +399,29 @@ /* #undef JEMALLOC_EXPORT */ /* config.malloc_conf options string. */ -#define JEMALLOC_CONFIG_MALLOC_CONF "" +#define JEMALLOC_CONFIG_MALLOC_CONF "@JEMALLOC_CONFIG_MALLOC_CONF@" /* If defined, jemalloc takes the malloc/free/etc. symbol names. */ -#define JEMALLOC_IS_MALLOC 1 +#define JEMALLOC_IS_MALLOC /* * Defined if strerror_r returns char * if _GNU_SOURCE is defined. */ #define JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE -/* - * popcount*() functions to use for bitmapping. - */ -#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl -#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount +/* Performs additional safety checks when defined. */ +/* #undef JEMALLOC_OPT_SAFETY_CHECKS */ + +/* Is C++ support being built? */ +/* #undef JEMALLOC_ENABLE_CXX */ + +/* Performs additional size checks when defined. */ +/* #undef JEMALLOC_OPT_SIZE_CHECKS */ + +/* Allows sampled junk and stash for checking use-after-free when defined. */ +/* #undef JEMALLOC_UAF_DETECTION */ + +/* Darwin VM_MAKE_TAG support */ +/* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ #endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_defs.h b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_defs.h deleted file mode 100644 index d1389237a77..00000000000 --- a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_defs.h +++ /dev/null @@ -1,43 +0,0 @@ -/* include/jemalloc/jemalloc_defs.h. Generated from jemalloc_defs.h.in by configure. */ -/* Defined if __attribute__((...)) syntax is supported. */ -#define JEMALLOC_HAVE_ATTR - -/* Defined if alloc_size attribute is supported. */ -#define JEMALLOC_HAVE_ATTR_ALLOC_SIZE - -/* Defined if format(printf, ...) attribute is supported. */ -#define JEMALLOC_HAVE_ATTR_FORMAT_PRINTF - -/* - * Define overrides for non-standard allocator-related functions if they are - * present on the system. - */ -#define JEMALLOC_OVERRIDE_MEMALIGN -#define JEMALLOC_OVERRIDE_VALLOC - -/* - * At least Linux omits the "const" in: - * - * size_t malloc_usable_size(const void *ptr); - * - * Match the operating system's prototype. - */ -#define JEMALLOC_USABLE_SIZE_CONST - -/* - * If defined, specify throw() for the public function prototypes when compiling - * with C++. The only justification for this is to match the prototypes that - * glibc defines. - */ -#define JEMALLOC_USE_CXX_THROW - -#ifdef _MSC_VER -# ifdef _WIN64 -# define LG_SIZEOF_PTR_WIN 3 -# else -# define LG_SIZEOF_PTR_WIN 2 -# endif -#endif - -/* sizeof(void *) == 2^LG_SIZEOF_PTR. */ -#define LG_SIZEOF_PTR 3 diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_macros.h b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_macros.h deleted file mode 100644 index 34235894285..00000000000 --- a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_macros.h +++ /dev/null @@ -1,129 +0,0 @@ -#include -#include -#include -#include -#include - -#define JEMALLOC_VERSION "5.2.1-0-gea6b3e973b477b8061e0076bb257dbd7f3faa756" -#define JEMALLOC_VERSION_MAJOR 5 -#define JEMALLOC_VERSION_MINOR 2 -#define JEMALLOC_VERSION_BUGFIX 1 -#define JEMALLOC_VERSION_NREV 0 -#define JEMALLOC_VERSION_GID "ea6b3e973b477b8061e0076bb257dbd7f3faa756" -#define JEMALLOC_VERSION_GID_IDENT ea6b3e973b477b8061e0076bb257dbd7f3faa756 - -#define MALLOCX_LG_ALIGN(la) ((int)(la)) -#if LG_SIZEOF_PTR == 2 -# define MALLOCX_ALIGN(a) ((int)(ffs((int)(a))-1)) -#else -# define MALLOCX_ALIGN(a) \ - ((int)(((size_t)(a) < (size_t)INT_MAX) ? ffs((int)(a))-1 : \ - ffs((int)(((size_t)(a))>>32))+31)) -#endif -#define MALLOCX_ZERO ((int)0x40) -/* - * Bias tcache index bits so that 0 encodes "automatic tcache management", and 1 - * encodes MALLOCX_TCACHE_NONE. - */ -#define MALLOCX_TCACHE(tc) ((int)(((tc)+2) << 8)) -#define MALLOCX_TCACHE_NONE MALLOCX_TCACHE(-1) -/* - * Bias arena index bits so that 0 encodes "use an automatically chosen arena". - */ -#define MALLOCX_ARENA(a) ((((int)(a))+1) << 20) - -/* - * Use as arena index in "arena..{purge,decay,dss}" and - * "stats.arenas..*" mallctl interfaces to select all arenas. This - * definition is intentionally specified in raw decimal format to support - * cpp-based string concatenation, e.g. - * - * #define STRINGIFY_HELPER(x) #x - * #define STRINGIFY(x) STRINGIFY_HELPER(x) - * - * mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", NULL, NULL, NULL, - * 0); - */ -#define MALLCTL_ARENAS_ALL 4096 -/* - * Use as arena index in "stats.arenas..*" mallctl interfaces to select - * destroyed arenas. - */ -#define MALLCTL_ARENAS_DESTROYED 4097 - -#if defined(__cplusplus) && defined(JEMALLOC_USE_CXX_THROW) -# define JEMALLOC_CXX_THROW throw() -#else -# define JEMALLOC_CXX_THROW -#endif - -#if defined(_MSC_VER) -# define JEMALLOC_ATTR(s) -# define JEMALLOC_ALIGNED(s) __declspec(align(s)) -# define JEMALLOC_ALLOC_SIZE(s) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) -# ifndef JEMALLOC_EXPORT -# ifdef DLLEXPORT -# define JEMALLOC_EXPORT __declspec(dllexport) -# else -# define JEMALLOC_EXPORT __declspec(dllimport) -# endif -# endif -# define JEMALLOC_FORMAT_ARG(i) -# define JEMALLOC_FORMAT_PRINTF(s, i) -# define JEMALLOC_NOINLINE __declspec(noinline) -# ifdef __cplusplus -# define JEMALLOC_NOTHROW __declspec(nothrow) -# else -# define JEMALLOC_NOTHROW -# endif -# define JEMALLOC_SECTION(s) __declspec(allocate(s)) -# define JEMALLOC_RESTRICT_RETURN __declspec(restrict) -# if _MSC_VER >= 1900 && !defined(__EDG__) -# define JEMALLOC_ALLOCATOR __declspec(allocator) -# else -# define JEMALLOC_ALLOCATOR -# endif -#elif defined(JEMALLOC_HAVE_ATTR) -# define JEMALLOC_ATTR(s) __attribute__((s)) -# define JEMALLOC_ALIGNED(s) JEMALLOC_ATTR(aligned(s)) -# ifdef JEMALLOC_HAVE_ATTR_ALLOC_SIZE -# define JEMALLOC_ALLOC_SIZE(s) JEMALLOC_ATTR(alloc_size(s)) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) JEMALLOC_ATTR(alloc_size(s1, s2)) -# else -# define JEMALLOC_ALLOC_SIZE(s) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) -# endif -# ifndef JEMALLOC_EXPORT -# define JEMALLOC_EXPORT JEMALLOC_ATTR(visibility("default")) -# endif -# ifdef JEMALLOC_HAVE_ATTR_FORMAT_ARG -# define JEMALLOC_FORMAT_ARG(i) JEMALLOC_ATTR(__format_arg__(3)) -# else -# define JEMALLOC_FORMAT_ARG(i) -# endif -# ifdef JEMALLOC_HAVE_ATTR_FORMAT_GNU_PRINTF -# define JEMALLOC_FORMAT_PRINTF(s, i) JEMALLOC_ATTR(format(gnu_printf, s, i)) -# elif defined(JEMALLOC_HAVE_ATTR_FORMAT_PRINTF) -# define JEMALLOC_FORMAT_PRINTF(s, i) JEMALLOC_ATTR(format(printf, s, i)) -# else -# define JEMALLOC_FORMAT_PRINTF(s, i) -# endif -# define JEMALLOC_NOINLINE JEMALLOC_ATTR(noinline) -# define JEMALLOC_NOTHROW JEMALLOC_ATTR(nothrow) -# define JEMALLOC_SECTION(s) JEMALLOC_ATTR(section(s)) -# define JEMALLOC_RESTRICT_RETURN -# define JEMALLOC_ALLOCATOR -#else -# define JEMALLOC_ATTR(s) -# define JEMALLOC_ALIGNED(s) -# define JEMALLOC_ALLOC_SIZE(s) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) -# define JEMALLOC_EXPORT -# define JEMALLOC_FORMAT_PRINTF(s, i) -# define JEMALLOC_NOINLINE -# define JEMALLOC_NOTHROW -# define JEMALLOC_SECTION(s) -# define JEMALLOC_RESTRICT_RETURN -# define JEMALLOC_ALLOCATOR -#endif diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_protos.h b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_protos.h deleted file mode 100644 index ff025e30fa7..00000000000 --- a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_protos.h +++ /dev/null @@ -1,66 +0,0 @@ -/* - * The je_ prefix on the following public symbol declarations is an artifact - * of namespace management, and should be omitted in application code unless - * JEMALLOC_NO_DEMANGLE is defined (see jemalloc_mangle.h). - */ -extern JEMALLOC_EXPORT const char *je_malloc_conf; -extern JEMALLOC_EXPORT void (*je_malloc_message)(void *cbopaque, - const char *s); - -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_malloc(size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE(1); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_calloc(size_t num, size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE2(1, 2); -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_posix_memalign(void **memptr, - size_t alignment, size_t size) JEMALLOC_CXX_THROW JEMALLOC_ATTR(nonnull(1)); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_aligned_alloc(size_t alignment, - size_t size) JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) - JEMALLOC_ALLOC_SIZE(2); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_realloc(void *ptr, size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ALLOC_SIZE(2); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_free(void *ptr) - JEMALLOC_CXX_THROW; - -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_mallocx(size_t size, int flags) - JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE(1); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_rallocx(void *ptr, size_t size, - int flags) JEMALLOC_ALLOC_SIZE(2); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_xallocx(void *ptr, size_t size, - size_t extra, int flags); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_sallocx(const void *ptr, - int flags) JEMALLOC_ATTR(pure); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_dallocx(void *ptr, int flags); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_sdallocx(void *ptr, size_t size, - int flags); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_nallocx(size_t size, int flags) - JEMALLOC_ATTR(pure); - -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_mallctl(const char *name, - void *oldp, size_t *oldlenp, void *newp, size_t newlen); -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_mallctlnametomib(const char *name, - size_t *mibp, size_t *miblenp); -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_mallctlbymib(const size_t *mib, - size_t miblen, void *oldp, size_t *oldlenp, void *newp, size_t newlen); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_malloc_stats_print( - void (*write_cb)(void *, const char *), void *je_cbopaque, - const char *opts); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_malloc_usable_size( - JEMALLOC_USABLE_SIZE_CONST void *ptr) JEMALLOC_CXX_THROW; - -#ifdef JEMALLOC_OVERRIDE_MEMALIGN -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_memalign(size_t alignment, size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc); -#endif - -#ifdef JEMALLOC_OVERRIDE_VALLOC -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_valloc(size_t size) JEMALLOC_CXX_THROW - JEMALLOC_ATTR(malloc); -#endif diff --git a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_typedefs.h b/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_typedefs.h deleted file mode 100644 index 1a58874306e..00000000000 --- a/contrib/jemalloc-cmake/include_linux_aarch64/jemalloc/jemalloc_typedefs.h +++ /dev/null @@ -1,77 +0,0 @@ -typedef struct extent_hooks_s extent_hooks_t; - -/* - * void * - * extent_alloc(extent_hooks_t *extent_hooks, void *new_addr, size_t size, - * size_t alignment, bool *zero, bool *commit, unsigned arena_ind); - */ -typedef void *(extent_alloc_t)(extent_hooks_t *, void *, size_t, size_t, bool *, - bool *, unsigned); - -/* - * bool - * extent_dalloc(extent_hooks_t *extent_hooks, void *addr, size_t size, - * bool committed, unsigned arena_ind); - */ -typedef bool (extent_dalloc_t)(extent_hooks_t *, void *, size_t, bool, - unsigned); - -/* - * void - * extent_destroy(extent_hooks_t *extent_hooks, void *addr, size_t size, - * bool committed, unsigned arena_ind); - */ -typedef void (extent_destroy_t)(extent_hooks_t *, void *, size_t, bool, - unsigned); - -/* - * bool - * extent_commit(extent_hooks_t *extent_hooks, void *addr, size_t size, - * size_t offset, size_t length, unsigned arena_ind); - */ -typedef bool (extent_commit_t)(extent_hooks_t *, void *, size_t, size_t, size_t, - unsigned); - -/* - * bool - * extent_decommit(extent_hooks_t *extent_hooks, void *addr, size_t size, - * size_t offset, size_t length, unsigned arena_ind); - */ -typedef bool (extent_decommit_t)(extent_hooks_t *, void *, size_t, size_t, - size_t, unsigned); - -/* - * bool - * extent_purge(extent_hooks_t *extent_hooks, void *addr, size_t size, - * size_t offset, size_t length, unsigned arena_ind); - */ -typedef bool (extent_purge_t)(extent_hooks_t *, void *, size_t, size_t, size_t, - unsigned); - -/* - * bool - * extent_split(extent_hooks_t *extent_hooks, void *addr, size_t size, - * size_t size_a, size_t size_b, bool committed, unsigned arena_ind); - */ -typedef bool (extent_split_t)(extent_hooks_t *, void *, size_t, size_t, size_t, - bool, unsigned); - -/* - * bool - * extent_merge(extent_hooks_t *extent_hooks, void *addr_a, size_t size_a, - * void *addr_b, size_t size_b, bool committed, unsigned arena_ind); - */ -typedef bool (extent_merge_t)(extent_hooks_t *, void *, size_t, void *, size_t, - bool, unsigned); - -struct extent_hooks_s { - extent_alloc_t *alloc; - extent_dalloc_t *dalloc; - extent_destroy_t *destroy; - extent_commit_t *commit; - extent_decommit_t *decommit; - extent_purge_t *purge_lazy; - extent_purge_t *purge_forced; - extent_split_t *split; - extent_merge_t *merge; -}; diff --git a/contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in new file mode 100644 index 00000000000..12890f80ef1 --- /dev/null +++ b/contrib/jemalloc-cmake/include_linux_ppc64le/jemalloc/internal/jemalloc_internal_defs.h.in @@ -0,0 +1,427 @@ +/* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ +#ifndef JEMALLOC_INTERNAL_DEFS_H_ +#define JEMALLOC_INTERNAL_DEFS_H_ +/* + * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all + * public APIs to be prefixed. This makes it possible, with some care, to use + * multiple allocators simultaneously. + */ +/* #undef JEMALLOC_PREFIX */ +/* #undef JEMALLOC_CPREFIX */ + +/* + * Define overrides for non-standard allocator-related functions if they are + * present on the system. + */ +#if !defined(USE_MUSL) + #define JEMALLOC_OVERRIDE___LIBC_CALLOC + #define JEMALLOC_OVERRIDE___LIBC_FREE + #define JEMALLOC_OVERRIDE___LIBC_MALLOC + #define JEMALLOC_OVERRIDE___LIBC_MEMALIGN + #define JEMALLOC_OVERRIDE___LIBC_REALLOC + #define JEMALLOC_OVERRIDE___LIBC_VALLOC +#endif +/* #undef JEMALLOC_OVERRIDE___POSIX_MEMALIGN */ + +/* + * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. + * For shared libraries, symbol visibility mechanisms prevent these symbols + * from being exported, but for static libraries, naming collisions are a real + * possibility. + */ +#define JEMALLOC_PRIVATE_NAMESPACE je_ + +/* + * Hyper-threaded CPUs may need a special instruction inside spin loops in + * order to yield to another virtual CPU. + */ +#define CPU_SPINWAIT +/* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ +#define HAVE_CPU_SPINWAIT 0 + +/* + * Number of significant bits in virtual addresses. This may be less than the + * total number of bits in a pointer, e.g. on x64, for which the uppermost 16 + * bits are the same as bit 47. + */ +#define LG_VADDR 64 + +/* Defined if C11 atomics are available. */ +#define JEMALLOC_C11_ATOMICS + +/* Defined if GCC __atomic atomics are available. */ +#define JEMALLOC_GCC_ATOMIC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS + +/* Defined if GCC __sync atomics are available. */ +#define JEMALLOC_GCC_SYNC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_SYNC_ATOMICS + +/* + * Defined if __builtin_clz() and __builtin_clzl() are available. + */ +#define JEMALLOC_HAVE_BUILTIN_CLZ + +/* + * Defined if os_unfair_lock_*() functions are available, as provided by Darwin. + */ +/* #undef JEMALLOC_OS_UNFAIR_LOCK */ + +/* Defined if syscall(2) is usable. */ +#define JEMALLOC_USE_SYSCALL + +/* + * Defined if secure_getenv(3) is available. + */ +/* #undef JEMALLOC_HAVE_SECURE_GETENV */ + +/* + * Defined if issetugid(2) is available. + */ +/* #undef JEMALLOC_HAVE_ISSETUGID */ + +/* Defined if pthread_atfork(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_ATFORK */ + +/* Defined if pthread_setname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_SETNAME_NP + +/* Defined if pthread_getname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_GETNAME_NP + +/* Defined if pthread_get_name_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_GET_NAME_NP */ + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC_COARSE, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC + +/* + * Defined if mach_absolute_time() is available. + */ +/* #undef JEMALLOC_HAVE_MACH_ABSOLUTE_TIME */ + +/* + * Defined if clock_gettime(CLOCK_REALTIME, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_REALTIME + +/* + * Defined if _malloc_thread_cleanup() exists. At least in the case of + * FreeBSD, pthread_key_create() allocates, which if used during malloc + * bootstrapping will cause recursion into the pthreads library. Therefore, if + * _malloc_thread_cleanup() exists, use it as the basis for thread cleanup in + * malloc_tsd. + */ +/* #undef JEMALLOC_MALLOC_THREAD_CLEANUP */ + +/* + * Defined if threaded initialization is known to be safe on this platform. + * Among other things, it must be possible to initialize a mutex without + * triggering allocation in order for threaded allocation to be safe. + */ +#define JEMALLOC_THREADED_INIT + +/* + * Defined if the pthreads implementation defines + * _pthread_mutex_init_calloc_cb(), in which case the function is used in order + * to avoid recursive allocation during mutex initialization. + */ +/* #undef JEMALLOC_MUTEX_INIT_CB */ + +/* Non-empty if the tls_model attribute is supported. */ +#define JEMALLOC_TLS_MODEL __attribute__((tls_model("initial-exec"))) + +/* + * JEMALLOC_DEBUG enables assertions and other sanity checks, and disables + * inline functions. + */ +/* #undef JEMALLOC_DEBUG */ + +/* JEMALLOC_STATS enables statistics calculation. */ +#define JEMALLOC_STATS + +/* JEMALLOC_EXPERIMENTAL_SMALLOCX_API enables experimental smallocx API. */ +/* #undef JEMALLOC_EXPERIMENTAL_SMALLOCX_API */ + +/* JEMALLOC_PROF enables allocation profiling. */ +/* #undef JEMALLOC_PROF */ + +/* Use libunwind for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBUNWIND */ + +/* Use libgcc for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBGCC */ + +/* Use gcc intrinsics for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_GCC */ + +/* + * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage + * segment (DSS). + */ +#define JEMALLOC_DSS + +/* Support memory filling (junk/zero). */ +#define JEMALLOC_FILL + +/* Support utrace(2)-based tracing. */ +/* #undef JEMALLOC_UTRACE */ + +/* Support utrace(2)-based tracing (label based signature). */ +/* #undef JEMALLOC_UTRACE_LABEL */ + +/* Support optional abort() on OOM. */ +/* #undef JEMALLOC_XMALLOC */ + +/* Support lazy locking (avoid locking unless a second thread is launched). */ +/* #undef JEMALLOC_LAZY_LOCK */ + +/* + * Minimum allocation alignment is 2^LG_QUANTUM bytes (ignoring tiny size + * classes). + */ +/* #undef LG_QUANTUM */ + +/* One page is 2^LG_PAGE bytes. */ +#define LG_PAGE 16 + +/* Maximum number of regions in a slab. */ +/* #undef CONFIG_LG_SLAB_MAXREGS */ + +/* + * One huge page is 2^LG_HUGEPAGE bytes. Note that this is defined even if the + * system does not explicitly support huge pages; system calls that require + * explicit huge page support are separately configured. + */ +#define LG_HUGEPAGE 21 + +/* + * If defined, adjacent virtual memory mappings with identical attributes + * automatically coalesce, and they fragment when changes are made to subranges. + * This is the normal order of things for mmap()/munmap(), but on Windows + * VirtualAlloc()/VirtualFree() operations must be precisely matched, i.e. + * mappings do *not* coalesce/fragment. + */ +#define JEMALLOC_MAPS_COALESCE + +/* + * If defined, retain memory for later reuse by default rather than using e.g. + * munmap() to unmap freed extents. This is enabled on 64-bit Linux because + * common sequences of mmap()/munmap() calls will cause virtual memory map + * holes. + */ +#define JEMALLOC_RETAIN + +/* TLS is used to map arenas and magazine caches to threads. */ +#define JEMALLOC_TLS + +/* + * Used to mark unreachable code to quiet "end of non-void" compiler warnings. + * Don't use this directly; instead use unreachable() from util.h + */ +#define JEMALLOC_INTERNAL_UNREACHABLE __builtin_unreachable + +/* + * ffs*() functions to use for bitmapping. Don't use these directly; instead, + * use ffs_*() from util.h. + */ +#define JEMALLOC_INTERNAL_FFSLL __builtin_ffsll +#define JEMALLOC_INTERNAL_FFSL __builtin_ffsl +#define JEMALLOC_INTERNAL_FFS __builtin_ffs + +/* + * popcount*() functions to use for bitmapping. + */ +#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl +#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount + +/* + * If defined, explicitly attempt to more uniformly distribute large allocation + * pointer alignments across all cache indices. + */ +#define JEMALLOC_CACHE_OBLIVIOUS + +/* + * If defined, enable logging facilities. We make this a configure option to + * avoid taking extra branches everywhere. + */ +/* #undef JEMALLOC_LOG */ + +/* + * If defined, use readlinkat() (instead of readlink()) to follow + * /etc/malloc_conf. + */ +/* #undef JEMALLOC_READLINKAT */ + +/* + * Darwin (OS X) uses zones to work around Mach-O symbol override shortcomings. + */ +/* #undef JEMALLOC_ZONE */ + +/* + * Methods for determining whether the OS overcommits. + * JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY: Linux's + * /proc/sys/vm.overcommit_memory file. + * JEMALLOC_SYSCTL_VM_OVERCOMMIT: FreeBSD's vm.overcommit sysctl. + */ +/* #undef JEMALLOC_SYSCTL_VM_OVERCOMMIT */ +#define JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY + +/* Defined if madvise(2) is available. */ +#define JEMALLOC_HAVE_MADVISE + +/* + * Defined if transparent huge pages are supported via the MADV_[NO]HUGEPAGE + * arguments to madvise(2). + */ +#define JEMALLOC_HAVE_MADVISE_HUGE + +/* + * Methods for purging unused pages differ between operating systems. + * + * madvise(..., MADV_FREE) : This marks pages as being unused, such that they + * will be discarded rather than swapped out. + * madvise(..., MADV_DONTNEED) : If JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS is + * defined, this immediately discards pages, + * such that new pages will be demand-zeroed if + * the address region is later touched; + * otherwise this behaves similarly to + * MADV_FREE, though typically with higher + * system overhead. + */ +#define JEMALLOC_PURGE_MADVISE_FREE +#define JEMALLOC_PURGE_MADVISE_DONTNEED +#define JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS + +/* Defined if madvise(2) is available but MADV_FREE is not (x86 Linux only). */ +#define JEMALLOC_DEFINE_MADVISE_FREE + +/* + * Defined if MADV_DO[NT]DUMP is supported as an argument to madvise. + */ +#define JEMALLOC_MADVISE_DONTDUMP + +/* + * Defined if MADV_[NO]CORE is supported as an argument to madvise. + */ +/* #undef JEMALLOC_MADVISE_NOCORE */ + +/* Defined if mprotect(2) is available. */ +#define JEMALLOC_HAVE_MPROTECT + +/* + * Defined if transparent huge pages (THPs) are supported via the + * MADV_[NO]HUGEPAGE arguments to madvise(2), and THP support is enabled. + */ +/* #undef JEMALLOC_THP */ + +/* Defined if posix_madvise is available. */ +/* #undef JEMALLOC_HAVE_POSIX_MADVISE */ + +/* + * Method for purging unused pages using posix_madvise. + * + * posix_madvise(..., POSIX_MADV_DONTNEED) + */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED_ZEROS */ + +/* + * Defined if memcntl page admin call is supported + */ +/* #undef JEMALLOC_HAVE_MEMCNTL */ + +/* + * Defined if malloc_size is supported + */ +/* #undef JEMALLOC_HAVE_MALLOC_SIZE */ + +/* Define if operating system has alloca.h header. */ +#define JEMALLOC_HAS_ALLOCA_H + +/* C99 restrict keyword supported. */ +#define JEMALLOC_HAS_RESTRICT + +/* For use by hash code. */ +/* #undef JEMALLOC_BIG_ENDIAN */ + +/* sizeof(int) == 2^LG_SIZEOF_INT. */ +#define LG_SIZEOF_INT 2 + +/* sizeof(long) == 2^LG_SIZEOF_LONG. */ +#define LG_SIZEOF_LONG 3 + +/* sizeof(long long) == 2^LG_SIZEOF_LONG_LONG. */ +#define LG_SIZEOF_LONG_LONG 3 + +/* sizeof(intmax_t) == 2^LG_SIZEOF_INTMAX_T. */ +#define LG_SIZEOF_INTMAX_T 3 + +/* glibc malloc hooks (__malloc_hook, __realloc_hook, __free_hook). */ +#define JEMALLOC_GLIBC_MALLOC_HOOK + +/* glibc memalign hook. */ +#define JEMALLOC_GLIBC_MEMALIGN_HOOK + +/* pthread support */ +#define JEMALLOC_HAVE_PTHREAD + +/* dlsym() support */ +#define JEMALLOC_HAVE_DLSYM + +/* Adaptive mutex support in pthreads. */ +#define JEMALLOC_HAVE_PTHREAD_MUTEX_ADAPTIVE_NP + +/* GNU specific sched_getcpu support */ +#define JEMALLOC_HAVE_SCHED_GETCPU + +/* GNU specific sched_setaffinity support */ +#define JEMALLOC_HAVE_SCHED_SETAFFINITY + +/* + * If defined, all the features necessary for background threads are present. + */ +#define JEMALLOC_BACKGROUND_THREAD + +/* + * If defined, jemalloc symbols are not exported (doesn't work when + * JEMALLOC_PREFIX is not defined). + */ +/* #undef JEMALLOC_EXPORT */ + +/* config.malloc_conf options string. */ +#define JEMALLOC_CONFIG_MALLOC_CONF "@JEMALLOC_CONFIG_MALLOC_CONF@" + +/* If defined, jemalloc takes the malloc/free/etc. symbol names. */ +#define JEMALLOC_IS_MALLOC + +/* + * Defined if strerror_r returns char * if _GNU_SOURCE is defined. + */ +#define JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE + +/* Performs additional safety checks when defined. */ +/* #undef JEMALLOC_OPT_SAFETY_CHECKS */ + +/* Is C++ support being built? */ +/* #undef JEMALLOC_ENABLE_CXX */ + +/* Performs additional size checks when defined. */ +/* #undef JEMALLOC_OPT_SIZE_CHECKS */ + +/* Allows sampled junk and stash for checking use-after-free when defined. */ +/* #undef JEMALLOC_UAF_DETECTION */ + +/* Darwin VM_MAKE_TAG support */ +/* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ + +#endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_linux_riscv64/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_riscv64/jemalloc/internal/jemalloc_internal_defs.h.in new file mode 100644 index 00000000000..ad535e6d773 --- /dev/null +++ b/contrib/jemalloc-cmake/include_linux_riscv64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -0,0 +1,427 @@ +/* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ +#ifndef JEMALLOC_INTERNAL_DEFS_H_ +#define JEMALLOC_INTERNAL_DEFS_H_ +/* + * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all + * public APIs to be prefixed. This makes it possible, with some care, to use + * multiple allocators simultaneously. + */ +/* #undef JEMALLOC_PREFIX */ +/* #undef JEMALLOC_CPREFIX */ + +/* + * Define overrides for non-standard allocator-related functions if they are + * present on the system. + */ +#if !defined(USE_MUSL) + #define JEMALLOC_OVERRIDE___LIBC_CALLOC + #define JEMALLOC_OVERRIDE___LIBC_FREE + #define JEMALLOC_OVERRIDE___LIBC_MALLOC + #define JEMALLOC_OVERRIDE___LIBC_MEMALIGN + #define JEMALLOC_OVERRIDE___LIBC_REALLOC + #define JEMALLOC_OVERRIDE___LIBC_VALLOC +#endif +/* #undef JEMALLOC_OVERRIDE___POSIX_MEMALIGN */ + +/* + * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. + * For shared libraries, symbol visibility mechanisms prevent these symbols + * from being exported, but for static libraries, naming collisions are a real + * possibility. + */ +#define JEMALLOC_PRIVATE_NAMESPACE je_ + +/* + * Hyper-threaded CPUs may need a special instruction inside spin loops in + * order to yield to another virtual CPU. + */ +#define CPU_SPINWAIT +/* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ +#define HAVE_CPU_SPINWAIT 0 + +/* + * Number of significant bits in virtual addresses. This may be less than the + * total number of bits in a pointer, e.g. on x64, for which the uppermost 16 + * bits are the same as bit 47. + */ +#define LG_VADDR 48 + +/* Defined if C11 atomics are available. */ +#define JEMALLOC_C11_ATOMICS + +/* Defined if GCC __atomic atomics are available. */ +#define JEMALLOC_GCC_ATOMIC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS + +/* Defined if GCC __sync atomics are available. */ +#define JEMALLOC_GCC_SYNC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_SYNC_ATOMICS + +/* + * Defined if __builtin_clz() and __builtin_clzl() are available. + */ +#define JEMALLOC_HAVE_BUILTIN_CLZ + +/* + * Defined if os_unfair_lock_*() functions are available, as provided by Darwin. + */ +/* #undef JEMALLOC_OS_UNFAIR_LOCK */ + +/* Defined if syscall(2) is usable. */ +#define JEMALLOC_USE_SYSCALL + +/* + * Defined if secure_getenv(3) is available. + */ +/* #undef JEMALLOC_HAVE_SECURE_GETENV */ + +/* + * Defined if issetugid(2) is available. + */ +/* #undef JEMALLOC_HAVE_ISSETUGID */ + +/* Defined if pthread_atfork(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_ATFORK + +/* Defined if pthread_setname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_SETNAME_NP + +/* Defined if pthread_getname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_GETNAME_NP + +/* Defined if pthread_get_name_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_GET_NAME_NP */ + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC_COARSE, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC + +/* + * Defined if mach_absolute_time() is available. + */ +/* #undef JEMALLOC_HAVE_MACH_ABSOLUTE_TIME */ + +/* + * Defined if clock_gettime(CLOCK_REALTIME, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_REALTIME + +/* + * Defined if _malloc_thread_cleanup() exists. At least in the case of + * FreeBSD, pthread_key_create() allocates, which if used during malloc + * bootstrapping will cause recursion into the pthreads library. Therefore, if + * _malloc_thread_cleanup() exists, use it as the basis for thread cleanup in + * malloc_tsd. + */ +/* #undef JEMALLOC_MALLOC_THREAD_CLEANUP */ + +/* + * Defined if threaded initialization is known to be safe on this platform. + * Among other things, it must be possible to initialize a mutex without + * triggering allocation in order for threaded allocation to be safe. + */ +#define JEMALLOC_THREADED_INIT + +/* + * Defined if the pthreads implementation defines + * _pthread_mutex_init_calloc_cb(), in which case the function is used in order + * to avoid recursive allocation during mutex initialization. + */ +/* #undef JEMALLOC_MUTEX_INIT_CB */ + +/* Non-empty if the tls_model attribute is supported. */ +#define JEMALLOC_TLS_MODEL __attribute__((tls_model("initial-exec"))) + +/* + * JEMALLOC_DEBUG enables assertions and other sanity checks, and disables + * inline functions. + */ +/* #undef JEMALLOC_DEBUG */ + +/* JEMALLOC_STATS enables statistics calculation. */ +#define JEMALLOC_STATS + +/* JEMALLOC_EXPERIMENTAL_SMALLOCX_API enables experimental smallocx API. */ +/* #undef JEMALLOC_EXPERIMENTAL_SMALLOCX_API */ + +/* JEMALLOC_PROF enables allocation profiling. */ +/* #undef JEMALLOC_PROF */ + +/* Use libunwind for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBUNWIND */ + +/* Use libgcc for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBGCC */ + +/* Use gcc intrinsics for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_GCC */ + +/* + * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage + * segment (DSS). + */ +#define JEMALLOC_DSS + +/* Support memory filling (junk/zero). */ +#define JEMALLOC_FILL + +/* Support utrace(2)-based tracing. */ +/* #undef JEMALLOC_UTRACE */ + +/* Support utrace(2)-based tracing (label based signature). */ +/* #undef JEMALLOC_UTRACE_LABEL */ + +/* Support optional abort() on OOM. */ +/* #undef JEMALLOC_XMALLOC */ + +/* Support lazy locking (avoid locking unless a second thread is launched). */ +/* #undef JEMALLOC_LAZY_LOCK */ + +/* + * Minimum allocation alignment is 2^LG_QUANTUM bytes (ignoring tiny size + * classes). + */ +/* #undef LG_QUANTUM */ + +/* One page is 2^LG_PAGE bytes. */ +#define LG_PAGE 16 + +/* Maximum number of regions in a slab. */ +/* #undef CONFIG_LG_SLAB_MAXREGS */ + +/* + * One huge page is 2^LG_HUGEPAGE bytes. Note that this is defined even if the + * system does not explicitly support huge pages; system calls that require + * explicit huge page support are separately configured. + */ +#define LG_HUGEPAGE 29 + +/* + * If defined, adjacent virtual memory mappings with identical attributes + * automatically coalesce, and they fragment when changes are made to subranges. + * This is the normal order of things for mmap()/munmap(), but on Windows + * VirtualAlloc()/VirtualFree() operations must be precisely matched, i.e. + * mappings do *not* coalesce/fragment. + */ +#define JEMALLOC_MAPS_COALESCE + +/* + * If defined, retain memory for later reuse by default rather than using e.g. + * munmap() to unmap freed extents. This is enabled on 64-bit Linux because + * common sequences of mmap()/munmap() calls will cause virtual memory map + * holes. + */ +#define JEMALLOC_RETAIN + +/* TLS is used to map arenas and magazine caches to threads. */ +#define JEMALLOC_TLS + +/* + * Used to mark unreachable code to quiet "end of non-void" compiler warnings. + * Don't use this directly; instead use unreachable() from util.h + */ +#define JEMALLOC_INTERNAL_UNREACHABLE __builtin_unreachable + +/* + * ffs*() functions to use for bitmapping. Don't use these directly; instead, + * use ffs_*() from util.h. + */ +#define JEMALLOC_INTERNAL_FFSLL __builtin_ffsll +#define JEMALLOC_INTERNAL_FFSL __builtin_ffsl +#define JEMALLOC_INTERNAL_FFS __builtin_ffs + +/* + * popcount*() functions to use for bitmapping. + */ +#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl +#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount + +/* + * If defined, explicitly attempt to more uniformly distribute large allocation + * pointer alignments across all cache indices. + */ +#define JEMALLOC_CACHE_OBLIVIOUS + +/* + * If defined, enable logging facilities. We make this a configure option to + * avoid taking extra branches everywhere. + */ +/* #undef JEMALLOC_LOG */ + +/* + * If defined, use readlinkat() (instead of readlink()) to follow + * /etc/malloc_conf. + */ +/* #undef JEMALLOC_READLINKAT */ + +/* + * Darwin (OS X) uses zones to work around Mach-O symbol override shortcomings. + */ +/* #undef JEMALLOC_ZONE */ + +/* + * Methods for determining whether the OS overcommits. + * JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY: Linux's + * /proc/sys/vm.overcommit_memory file. + * JEMALLOC_SYSCTL_VM_OVERCOMMIT: FreeBSD's vm.overcommit sysctl. + */ +/* #undef JEMALLOC_SYSCTL_VM_OVERCOMMIT */ +#define JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY + +/* Defined if madvise(2) is available. */ +#define JEMALLOC_HAVE_MADVISE + +/* + * Defined if transparent huge pages are supported via the MADV_[NO]HUGEPAGE + * arguments to madvise(2). + */ +#define JEMALLOC_HAVE_MADVISE_HUGE + +/* + * Methods for purging unused pages differ between operating systems. + * + * madvise(..., MADV_FREE) : This marks pages as being unused, such that they + * will be discarded rather than swapped out. + * madvise(..., MADV_DONTNEED) : If JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS is + * defined, this immediately discards pages, + * such that new pages will be demand-zeroed if + * the address region is later touched; + * otherwise this behaves similarly to + * MADV_FREE, though typically with higher + * system overhead. + */ +#define JEMALLOC_PURGE_MADVISE_FREE +#define JEMALLOC_PURGE_MADVISE_DONTNEED +#define JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS + +/* Defined if madvise(2) is available but MADV_FREE is not (x86 Linux only). */ +/* #undef JEMALLOC_DEFINE_MADVISE_FREE */ + +/* + * Defined if MADV_DO[NT]DUMP is supported as an argument to madvise. + */ +#define JEMALLOC_MADVISE_DONTDUMP + +/* + * Defined if MADV_[NO]CORE is supported as an argument to madvise. + */ +/* #undef JEMALLOC_MADVISE_NOCORE */ + +/* Defined if mprotect(2) is available. */ +#define JEMALLOC_HAVE_MPROTECT + +/* + * Defined if transparent huge pages (THPs) are supported via the + * MADV_[NO]HUGEPAGE arguments to madvise(2), and THP support is enabled. + */ +/* #undef JEMALLOC_THP */ + +/* Defined if posix_madvise is available. */ +/* #undef JEMALLOC_HAVE_POSIX_MADVISE */ + +/* + * Method for purging unused pages using posix_madvise. + * + * posix_madvise(..., POSIX_MADV_DONTNEED) + */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED_ZEROS */ + +/* + * Defined if memcntl page admin call is supported + */ +/* #undef JEMALLOC_HAVE_MEMCNTL */ + +/* + * Defined if malloc_size is supported + */ +/* #undef JEMALLOC_HAVE_MALLOC_SIZE */ + +/* Define if operating system has alloca.h header. */ +#define JEMALLOC_HAS_ALLOCA_H + +/* C99 restrict keyword supported. */ +#define JEMALLOC_HAS_RESTRICT + +/* For use by hash code. */ +/* #undef JEMALLOC_BIG_ENDIAN */ + +/* sizeof(int) == 2^LG_SIZEOF_INT. */ +#define LG_SIZEOF_INT 2 + +/* sizeof(long) == 2^LG_SIZEOF_LONG. */ +#define LG_SIZEOF_LONG 3 + +/* sizeof(long long) == 2^LG_SIZEOF_LONG_LONG. */ +#define LG_SIZEOF_LONG_LONG 3 + +/* sizeof(intmax_t) == 2^LG_SIZEOF_INTMAX_T. */ +#define LG_SIZEOF_INTMAX_T 3 + +/* glibc malloc hooks (__malloc_hook, __realloc_hook, __free_hook). */ +#define JEMALLOC_GLIBC_MALLOC_HOOK + +/* glibc memalign hook. */ +#define JEMALLOC_GLIBC_MEMALIGN_HOOK + +/* pthread support */ +#define JEMALLOC_HAVE_PTHREAD + +/* dlsym() support */ +#define JEMALLOC_HAVE_DLSYM + +/* Adaptive mutex support in pthreads. */ +#define JEMALLOC_HAVE_PTHREAD_MUTEX_ADAPTIVE_NP + +/* GNU specific sched_getcpu support */ +#define JEMALLOC_HAVE_SCHED_GETCPU + +/* GNU specific sched_setaffinity support */ +#define JEMALLOC_HAVE_SCHED_SETAFFINITY + +/* + * If defined, all the features necessary for background threads are present. + */ +#define JEMALLOC_BACKGROUND_THREAD + +/* + * If defined, jemalloc symbols are not exported (doesn't work when + * JEMALLOC_PREFIX is not defined). + */ +/* #undef JEMALLOC_EXPORT */ + +/* config.malloc_conf options string. */ +#define JEMALLOC_CONFIG_MALLOC_CONF "@JEMALLOC_CONFIG_MALLOC_CONF@" + +/* If defined, jemalloc takes the malloc/free/etc. symbol names. */ +#define JEMALLOC_IS_MALLOC + +/* + * Defined if strerror_r returns char * if _GNU_SOURCE is defined. + */ +#define JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE + +/* Performs additional safety checks when defined. */ +/* #undef JEMALLOC_OPT_SAFETY_CHECKS */ + +/* Is C++ support being built? */ +/* #undef JEMALLOC_ENABLE_CXX */ + +/* Performs additional size checks when defined. */ +/* #undef JEMALLOC_OPT_SIZE_CHECKS */ + +/* Allows sampled junk and stash for checking use-after-free when defined. */ +/* #undef JEMALLOC_UAF_DETECTION */ + +/* Darwin VM_MAKE_TAG support */ +/* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ + +#endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/README b/contrib/jemalloc-cmake/include_linux_x86_64/README deleted file mode 100644 index bf7663bda8d..00000000000 --- a/contrib/jemalloc-cmake/include_linux_x86_64/README +++ /dev/null @@ -1,7 +0,0 @@ -Here are pre-generated files from jemalloc on Linux x86_64. -You can obtain these files by running ./autogen.sh inside jemalloc source directory. - -Added #define GNU_SOURCE -Added JEMALLOC_OVERRIDE___POSIX_MEMALIGN because why not. -Removed JEMALLOC_HAVE_ATTR_FORMAT_GNU_PRINTF because it's non standard. -Removed JEMALLOC_PURGE_MADVISE_FREE because it's available only from Linux 4.5. diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in similarity index 78% rename from contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h rename to contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in index 7c21fa79397..99ab2d53ca9 100644 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h +++ b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h.in @@ -1,11 +1,6 @@ /* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ #ifndef JEMALLOC_INTERNAL_DEFS_H_ #define JEMALLOC_INTERNAL_DEFS_H_ - -#ifndef _GNU_SOURCE - #define _GNU_SOURCE -#endif - /* * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all * public APIs to be prefixed. This makes it possible, with some care, to use @@ -18,13 +13,15 @@ * Define overrides for non-standard allocator-related functions if they are * present on the system. */ -#define JEMALLOC_OVERRIDE___LIBC_CALLOC -#define JEMALLOC_OVERRIDE___LIBC_FREE -#define JEMALLOC_OVERRIDE___LIBC_MALLOC -#define JEMALLOC_OVERRIDE___LIBC_MEMALIGN -#define JEMALLOC_OVERRIDE___LIBC_REALLOC -#define JEMALLOC_OVERRIDE___LIBC_VALLOC -#define JEMALLOC_OVERRIDE___POSIX_MEMALIGN +#if !defined(USE_MUSL) + #define JEMALLOC_OVERRIDE___LIBC_CALLOC + #define JEMALLOC_OVERRIDE___LIBC_FREE + #define JEMALLOC_OVERRIDE___LIBC_MALLOC + #define JEMALLOC_OVERRIDE___LIBC_MEMALIGN + #define JEMALLOC_OVERRIDE___LIBC_REALLOC + #define JEMALLOC_OVERRIDE___LIBC_VALLOC +#endif +/* #undef JEMALLOC_OVERRIDE___POSIX_MEMALIGN */ /* * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. @@ -50,29 +47,17 @@ #define LG_VADDR 48 /* Defined if C11 atomics are available. */ -#define JEMALLOC_C11_ATOMICS 1 +#define JEMALLOC_C11_ATOMICS /* Defined if GCC __atomic atomics are available. */ -#define JEMALLOC_GCC_ATOMIC_ATOMICS 1 +#define JEMALLOC_GCC_ATOMIC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS /* Defined if GCC __sync atomics are available. */ -#define JEMALLOC_GCC_SYNC_ATOMICS 1 - -/* - * Defined if __sync_add_and_fetch(uint32_t *, uint32_t) and - * __sync_sub_and_fetch(uint32_t *, uint32_t) are available, despite - * __GCC_HAVE_SYNC_COMPARE_AND_SWAP_4 not being defined (which means the - * functions are defined in libgcc instead of being inlines). - */ -/* #undef JE_FORCE_SYNC_COMPARE_AND_SWAP_4 */ - -/* - * Defined if __sync_add_and_fetch(uint64_t *, uint64_t) and - * __sync_sub_and_fetch(uint64_t *, uint64_t) are available, despite - * __GCC_HAVE_SYNC_COMPARE_AND_SWAP_8 not being defined (which means the - * functions are defined in libgcc instead of being inlines). - */ -/* #undef JE_FORCE_SYNC_COMPARE_AND_SWAP_8 */ +#define JEMALLOC_GCC_SYNC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_SYNC_ATOMICS /* * Defined if __builtin_clz() and __builtin_clzl() are available. @@ -84,20 +69,13 @@ */ /* #undef JEMALLOC_OS_UNFAIR_LOCK */ -/* - * Defined if OSSpin*() functions are available, as provided by Darwin, and - * documented in the spinlock(3) manual page. - */ -/* #undef JEMALLOC_OSSPIN */ - /* Defined if syscall(2) is usable. */ #define JEMALLOC_USE_SYSCALL /* * Defined if secure_getenv(3) is available. */ -// Don't want dependency on newer GLIBC -//#define JEMALLOC_HAVE_SECURE_GETENV +/* #undef JEMALLOC_HAVE_SECURE_GETENV */ /* * Defined if issetugid(2) is available. @@ -110,21 +88,32 @@ /* Defined if pthread_setname_np(3) is available. */ #define JEMALLOC_HAVE_PTHREAD_SETNAME_NP +/* Defined if pthread_getname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_GETNAME_NP + +/* Defined if pthread_get_name_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_GET_NAME_NP */ + /* * Defined if clock_gettime(CLOCK_MONOTONIC_COARSE, ...) is available. */ -#define JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE 1 +#define JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE /* * Defined if clock_gettime(CLOCK_MONOTONIC, ...) is available. */ -#define JEMALLOC_HAVE_CLOCK_MONOTONIC 1 +#define JEMALLOC_HAVE_CLOCK_MONOTONIC /* * Defined if mach_absolute_time() is available. */ /* #undef JEMALLOC_HAVE_MACH_ABSOLUTE_TIME */ +/* + * Defined if clock_gettime(CLOCK_REALTIME, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_REALTIME + /* * Defined if _malloc_thread_cleanup() exists. At least in the case of * FreeBSD, pthread_key_create() allocates, which if used during malloc @@ -160,6 +149,9 @@ /* JEMALLOC_STATS enables statistics calculation. */ #define JEMALLOC_STATS +/* JEMALLOC_EXPERIMENTAL_SMALLOCX_API enables experimental smallocx API. */ +/* #undef JEMALLOC_EXPERIMENTAL_SMALLOCX_API */ + /* JEMALLOC_PROF enables allocation profiling. */ /* #undef JEMALLOC_PROF */ @@ -184,6 +176,9 @@ /* Support utrace(2)-based tracing. */ /* #undef JEMALLOC_UTRACE */ +/* Support utrace(2)-based tracing (label based signature). */ +/* #undef JEMALLOC_UTRACE_LABEL */ + /* Support optional abort() on OOM. */ /* #undef JEMALLOC_XMALLOC */ @@ -199,6 +194,9 @@ /* One page is 2^LG_PAGE bytes. */ #define LG_PAGE 12 +/* Maximum number of regions in a slab. */ +/* #undef CONFIG_LG_SLAB_MAXREGS */ + /* * One huge page is 2^LG_HUGEPAGE bytes. Note that this is defined even if the * system does not explicitly support huge pages; system calls that require @@ -240,6 +238,12 @@ #define JEMALLOC_INTERNAL_FFSL __builtin_ffsl #define JEMALLOC_INTERNAL_FFS __builtin_ffs +/* + * popcount*() functions to use for bitmapping. + */ +#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl +#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount + /* * If defined, explicitly attempt to more uniformly distribute large allocation * pointer alignments across all cache indices. @@ -252,6 +256,12 @@ */ /* #undef JEMALLOC_LOG */ +/* + * If defined, use readlinkat() (instead of readlink()) to follow + * /etc/malloc_conf. + */ +/* #undef JEMALLOC_READLINKAT */ + /* * Darwin (OS X) uses zones to work around Mach-O symbol override shortcomings. */ @@ -288,7 +298,7 @@ * MADV_FREE, though typically with higher * system overhead. */ -//#define JEMALLOC_PURGE_MADVISE_FREE +#define JEMALLOC_PURGE_MADVISE_FREE #define JEMALLOC_PURGE_MADVISE_DONTNEED #define JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS @@ -300,17 +310,46 @@ */ #define JEMALLOC_MADVISE_DONTDUMP +/* + * Defined if MADV_[NO]CORE is supported as an argument to madvise. + */ +/* #undef JEMALLOC_MADVISE_NOCORE */ + +/* Defined if mprotect(2) is available. */ +#define JEMALLOC_HAVE_MPROTECT + /* * Defined if transparent huge pages (THPs) are supported via the * MADV_[NO]HUGEPAGE arguments to madvise(2), and THP support is enabled. */ /* #undef JEMALLOC_THP */ +/* Defined if posix_madvise is available. */ +/* #undef JEMALLOC_HAVE_POSIX_MADVISE */ + +/* + * Method for purging unused pages using posix_madvise. + * + * posix_madvise(..., POSIX_MADV_DONTNEED) + */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED_ZEROS */ + +/* + * Defined if memcntl page admin call is supported + */ +/* #undef JEMALLOC_HAVE_MEMCNTL */ + +/* + * Defined if malloc_size is supported + */ +/* #undef JEMALLOC_HAVE_MALLOC_SIZE */ + /* Define if operating system has alloca.h header. */ -#define JEMALLOC_HAS_ALLOCA_H 1 +#define JEMALLOC_HAS_ALLOCA_H /* C99 restrict keyword supported. */ -#define JEMALLOC_HAS_RESTRICT 1 +#define JEMALLOC_HAS_RESTRICT /* For use by hash code. */ /* #undef JEMALLOC_BIG_ENDIAN */ @@ -351,7 +390,7 @@ /* * If defined, all the features necessary for background threads are present. */ -#define JEMALLOC_BACKGROUND_THREAD 1 +#define JEMALLOC_BACKGROUND_THREAD /* * If defined, jemalloc symbols are not exported (doesn't work when @@ -360,20 +399,29 @@ /* #undef JEMALLOC_EXPORT */ /* config.malloc_conf options string. */ -#define JEMALLOC_CONFIG_MALLOC_CONF "" +#define JEMALLOC_CONFIG_MALLOC_CONF "@JEMALLOC_CONFIG_MALLOC_CONF@" /* If defined, jemalloc takes the malloc/free/etc. symbol names. */ -#define JEMALLOC_IS_MALLOC 1 +#define JEMALLOC_IS_MALLOC /* * Defined if strerror_r returns char * if _GNU_SOURCE is defined. */ #define JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE -/* - * popcount*() functions to use for bitmapping. - */ -#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl -#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount +/* Performs additional safety checks when defined. */ +/* #undef JEMALLOC_OPT_SAFETY_CHECKS */ + +/* Is C++ support being built? */ +/* #undef JEMALLOC_ENABLE_CXX */ + +/* Performs additional size checks when defined. */ +/* #undef JEMALLOC_OPT_SIZE_CHECKS */ + +/* Allows sampled junk and stash for checking use-after-free when defined. */ +/* #undef JEMALLOC_UAF_DETECTION */ + +/* Darwin VM_MAKE_TAG support */ +/* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ #endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_preamble.h b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_preamble.h deleted file mode 100644 index d79551e1f25..00000000000 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_preamble.h +++ /dev/null @@ -1,216 +0,0 @@ -#ifndef JEMALLOC_PREAMBLE_H -#define JEMALLOC_PREAMBLE_H - -#include "jemalloc_internal_defs.h" -#include "jemalloc/internal/jemalloc_internal_decls.h" - -#ifdef JEMALLOC_UTRACE -#include -#endif - -#define JEMALLOC_NO_DEMANGLE -#ifdef JEMALLOC_JET -# undef JEMALLOC_IS_MALLOC -# define JEMALLOC_N(n) jet_##n -# include "jemalloc/internal/public_namespace.h" -# define JEMALLOC_NO_RENAME -# include "jemalloc/jemalloc.h" -# undef JEMALLOC_NO_RENAME -#else -# define JEMALLOC_N(n) je_##n -# include "jemalloc/jemalloc.h" -#endif - -#if (defined(JEMALLOC_OSATOMIC) || defined(JEMALLOC_OSSPIN)) -#include -#endif - -#ifdef JEMALLOC_ZONE -#include -#include -#include -#endif - -#include "jemalloc/internal/jemalloc_internal_macros.h" - -/* - * Note that the ordering matters here; the hook itself is name-mangled. We - * want the inclusion of hooks to happen early, so that we hook as much as - * possible. - */ -#ifndef JEMALLOC_NO_PRIVATE_NAMESPACE -# ifndef JEMALLOC_JET -# include "jemalloc/internal/private_namespace.h" -# else -# include "jemalloc/internal/private_namespace_jet.h" -# endif -#endif -#include "jemalloc/internal/test_hooks.h" - -#ifdef JEMALLOC_DEFINE_MADVISE_FREE -# define JEMALLOC_MADV_FREE 8 -#endif - -static const bool config_debug = -#ifdef JEMALLOC_DEBUG - true -#else - false -#endif - ; -static const bool have_dss = -#ifdef JEMALLOC_DSS - true -#else - false -#endif - ; -static const bool have_madvise_huge = -#ifdef JEMALLOC_HAVE_MADVISE_HUGE - true -#else - false -#endif - ; -static const bool config_fill = -#ifdef JEMALLOC_FILL - true -#else - false -#endif - ; -static const bool config_lazy_lock = -#ifdef JEMALLOC_LAZY_LOCK - true -#else - false -#endif - ; -static const char * const config_malloc_conf = JEMALLOC_CONFIG_MALLOC_CONF; -static const bool config_prof = -#ifdef JEMALLOC_PROF - true -#else - false -#endif - ; -static const bool config_prof_libgcc = -#ifdef JEMALLOC_PROF_LIBGCC - true -#else - false -#endif - ; -static const bool config_prof_libunwind = -#ifdef JEMALLOC_PROF_LIBUNWIND - true -#else - false -#endif - ; -static const bool maps_coalesce = -#ifdef JEMALLOC_MAPS_COALESCE - true -#else - false -#endif - ; -static const bool config_stats = -#ifdef JEMALLOC_STATS - true -#else - false -#endif - ; -static const bool config_tls = -#ifdef JEMALLOC_TLS - true -#else - false -#endif - ; -static const bool config_utrace = -#ifdef JEMALLOC_UTRACE - true -#else - false -#endif - ; -static const bool config_xmalloc = -#ifdef JEMALLOC_XMALLOC - true -#else - false -#endif - ; -static const bool config_cache_oblivious = -#ifdef JEMALLOC_CACHE_OBLIVIOUS - true -#else - false -#endif - ; -/* - * Undocumented, for jemalloc development use only at the moment. See the note - * in jemalloc/internal/log.h. - */ -static const bool config_log = -#ifdef JEMALLOC_LOG - true -#else - false -#endif - ; -#ifdef JEMALLOC_HAVE_SCHED_GETCPU -/* Currently percpu_arena depends on sched_getcpu. */ -#define JEMALLOC_PERCPU_ARENA -#endif -static const bool have_percpu_arena = -#ifdef JEMALLOC_PERCPU_ARENA - true -#else - false -#endif - ; -/* - * Undocumented, and not recommended; the application should take full - * responsibility for tracking provenance. - */ -static const bool force_ivsalloc = -#ifdef JEMALLOC_FORCE_IVSALLOC - true -#else - false -#endif - ; -static const bool have_background_thread = -#ifdef JEMALLOC_BACKGROUND_THREAD - true -#else - false -#endif - ; - -#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS 1 -#define JEMALLOC_GCC_U8_SYNC_ATOMICS 1 - -/* - * Are extra safety checks enabled; things like checking the size of sized - * deallocations, double-frees, etc. - */ -static const bool config_opt_safety_checks = -#ifdef JEMALLOC_OPT_SAFETY_CHECKS - true -#elif defined(JEMALLOC_DEBUG) - /* - * This lets us only guard safety checks by one flag instead of two; fast - * checks can guard solely by config_opt_safety_checks and run in debug mode - * too. - */ - true -#else - false -#endif - ; - -#endif /* JEMALLOC_PREAMBLE_H */ diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_macros.h b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_macros.h deleted file mode 100644 index 34235894285..00000000000 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_macros.h +++ /dev/null @@ -1,129 +0,0 @@ -#include -#include -#include -#include -#include - -#define JEMALLOC_VERSION "5.2.1-0-gea6b3e973b477b8061e0076bb257dbd7f3faa756" -#define JEMALLOC_VERSION_MAJOR 5 -#define JEMALLOC_VERSION_MINOR 2 -#define JEMALLOC_VERSION_BUGFIX 1 -#define JEMALLOC_VERSION_NREV 0 -#define JEMALLOC_VERSION_GID "ea6b3e973b477b8061e0076bb257dbd7f3faa756" -#define JEMALLOC_VERSION_GID_IDENT ea6b3e973b477b8061e0076bb257dbd7f3faa756 - -#define MALLOCX_LG_ALIGN(la) ((int)(la)) -#if LG_SIZEOF_PTR == 2 -# define MALLOCX_ALIGN(a) ((int)(ffs((int)(a))-1)) -#else -# define MALLOCX_ALIGN(a) \ - ((int)(((size_t)(a) < (size_t)INT_MAX) ? ffs((int)(a))-1 : \ - ffs((int)(((size_t)(a))>>32))+31)) -#endif -#define MALLOCX_ZERO ((int)0x40) -/* - * Bias tcache index bits so that 0 encodes "automatic tcache management", and 1 - * encodes MALLOCX_TCACHE_NONE. - */ -#define MALLOCX_TCACHE(tc) ((int)(((tc)+2) << 8)) -#define MALLOCX_TCACHE_NONE MALLOCX_TCACHE(-1) -/* - * Bias arena index bits so that 0 encodes "use an automatically chosen arena". - */ -#define MALLOCX_ARENA(a) ((((int)(a))+1) << 20) - -/* - * Use as arena index in "arena..{purge,decay,dss}" and - * "stats.arenas..*" mallctl interfaces to select all arenas. This - * definition is intentionally specified in raw decimal format to support - * cpp-based string concatenation, e.g. - * - * #define STRINGIFY_HELPER(x) #x - * #define STRINGIFY(x) STRINGIFY_HELPER(x) - * - * mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", NULL, NULL, NULL, - * 0); - */ -#define MALLCTL_ARENAS_ALL 4096 -/* - * Use as arena index in "stats.arenas..*" mallctl interfaces to select - * destroyed arenas. - */ -#define MALLCTL_ARENAS_DESTROYED 4097 - -#if defined(__cplusplus) && defined(JEMALLOC_USE_CXX_THROW) -# define JEMALLOC_CXX_THROW throw() -#else -# define JEMALLOC_CXX_THROW -#endif - -#if defined(_MSC_VER) -# define JEMALLOC_ATTR(s) -# define JEMALLOC_ALIGNED(s) __declspec(align(s)) -# define JEMALLOC_ALLOC_SIZE(s) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) -# ifndef JEMALLOC_EXPORT -# ifdef DLLEXPORT -# define JEMALLOC_EXPORT __declspec(dllexport) -# else -# define JEMALLOC_EXPORT __declspec(dllimport) -# endif -# endif -# define JEMALLOC_FORMAT_ARG(i) -# define JEMALLOC_FORMAT_PRINTF(s, i) -# define JEMALLOC_NOINLINE __declspec(noinline) -# ifdef __cplusplus -# define JEMALLOC_NOTHROW __declspec(nothrow) -# else -# define JEMALLOC_NOTHROW -# endif -# define JEMALLOC_SECTION(s) __declspec(allocate(s)) -# define JEMALLOC_RESTRICT_RETURN __declspec(restrict) -# if _MSC_VER >= 1900 && !defined(__EDG__) -# define JEMALLOC_ALLOCATOR __declspec(allocator) -# else -# define JEMALLOC_ALLOCATOR -# endif -#elif defined(JEMALLOC_HAVE_ATTR) -# define JEMALLOC_ATTR(s) __attribute__((s)) -# define JEMALLOC_ALIGNED(s) JEMALLOC_ATTR(aligned(s)) -# ifdef JEMALLOC_HAVE_ATTR_ALLOC_SIZE -# define JEMALLOC_ALLOC_SIZE(s) JEMALLOC_ATTR(alloc_size(s)) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) JEMALLOC_ATTR(alloc_size(s1, s2)) -# else -# define JEMALLOC_ALLOC_SIZE(s) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) -# endif -# ifndef JEMALLOC_EXPORT -# define JEMALLOC_EXPORT JEMALLOC_ATTR(visibility("default")) -# endif -# ifdef JEMALLOC_HAVE_ATTR_FORMAT_ARG -# define JEMALLOC_FORMAT_ARG(i) JEMALLOC_ATTR(__format_arg__(3)) -# else -# define JEMALLOC_FORMAT_ARG(i) -# endif -# ifdef JEMALLOC_HAVE_ATTR_FORMAT_GNU_PRINTF -# define JEMALLOC_FORMAT_PRINTF(s, i) JEMALLOC_ATTR(format(gnu_printf, s, i)) -# elif defined(JEMALLOC_HAVE_ATTR_FORMAT_PRINTF) -# define JEMALLOC_FORMAT_PRINTF(s, i) JEMALLOC_ATTR(format(printf, s, i)) -# else -# define JEMALLOC_FORMAT_PRINTF(s, i) -# endif -# define JEMALLOC_NOINLINE JEMALLOC_ATTR(noinline) -# define JEMALLOC_NOTHROW JEMALLOC_ATTR(nothrow) -# define JEMALLOC_SECTION(s) JEMALLOC_ATTR(section(s)) -# define JEMALLOC_RESTRICT_RETURN -# define JEMALLOC_ALLOCATOR -#else -# define JEMALLOC_ATTR(s) -# define JEMALLOC_ALIGNED(s) -# define JEMALLOC_ALLOC_SIZE(s) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) -# define JEMALLOC_EXPORT -# define JEMALLOC_FORMAT_PRINTF(s, i) -# define JEMALLOC_NOINLINE -# define JEMALLOC_NOTHROW -# define JEMALLOC_SECTION(s) -# define JEMALLOC_RESTRICT_RETURN -# define JEMALLOC_ALLOCATOR -#endif diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_protos.h b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_protos.h deleted file mode 100644 index ff025e30fa7..00000000000 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_protos.h +++ /dev/null @@ -1,66 +0,0 @@ -/* - * The je_ prefix on the following public symbol declarations is an artifact - * of namespace management, and should be omitted in application code unless - * JEMALLOC_NO_DEMANGLE is defined (see jemalloc_mangle.h). - */ -extern JEMALLOC_EXPORT const char *je_malloc_conf; -extern JEMALLOC_EXPORT void (*je_malloc_message)(void *cbopaque, - const char *s); - -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_malloc(size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE(1); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_calloc(size_t num, size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE2(1, 2); -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_posix_memalign(void **memptr, - size_t alignment, size_t size) JEMALLOC_CXX_THROW JEMALLOC_ATTR(nonnull(1)); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_aligned_alloc(size_t alignment, - size_t size) JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) - JEMALLOC_ALLOC_SIZE(2); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_realloc(void *ptr, size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ALLOC_SIZE(2); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_free(void *ptr) - JEMALLOC_CXX_THROW; - -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_mallocx(size_t size, int flags) - JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE(1); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_rallocx(void *ptr, size_t size, - int flags) JEMALLOC_ALLOC_SIZE(2); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_xallocx(void *ptr, size_t size, - size_t extra, int flags); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_sallocx(const void *ptr, - int flags) JEMALLOC_ATTR(pure); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_dallocx(void *ptr, int flags); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_sdallocx(void *ptr, size_t size, - int flags); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_nallocx(size_t size, int flags) - JEMALLOC_ATTR(pure); - -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_mallctl(const char *name, - void *oldp, size_t *oldlenp, void *newp, size_t newlen); -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_mallctlnametomib(const char *name, - size_t *mibp, size_t *miblenp); -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_mallctlbymib(const size_t *mib, - size_t miblen, void *oldp, size_t *oldlenp, void *newp, size_t newlen); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_malloc_stats_print( - void (*write_cb)(void *, const char *), void *je_cbopaque, - const char *opts); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_malloc_usable_size( - JEMALLOC_USABLE_SIZE_CONST void *ptr) JEMALLOC_CXX_THROW; - -#ifdef JEMALLOC_OVERRIDE_MEMALIGN -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_memalign(size_t alignment, size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc); -#endif - -#ifdef JEMALLOC_OVERRIDE_VALLOC -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_valloc(size_t size) JEMALLOC_CXX_THROW - JEMALLOC_ATTR(malloc); -#endif diff --git a/contrib/jemalloc-cmake/include_linux_x86_64_musl/jemalloc/internal/jemalloc_internal_defs.h.in b/contrib/jemalloc-cmake/include_linux_x86_64_musl/jemalloc/internal/jemalloc_internal_defs.h.in new file mode 100644 index 00000000000..684d4debb14 --- /dev/null +++ b/contrib/jemalloc-cmake/include_linux_x86_64_musl/jemalloc/internal/jemalloc_internal_defs.h.in @@ -0,0 +1,428 @@ +/* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ +#ifndef JEMALLOC_INTERNAL_DEFS_H_ +#define JEMALLOC_INTERNAL_DEFS_H_ +/* + * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all + * public APIs to be prefixed. This makes it possible, with some care, to use + * multiple allocators simultaneously. + */ +/* #undef JEMALLOC_PREFIX */ +/* #undef JEMALLOC_CPREFIX */ + +/* + * Define overrides for non-standard allocator-related functions if they are + * present on the system. + */ +#if !defined(USE_MUSL) + #define JEMALLOC_OVERRIDE___LIBC_CALLOC + #define JEMALLOC_OVERRIDE___LIBC_FREE + #define JEMALLOC_OVERRIDE___LIBC_MALLOC + #define JEMALLOC_OVERRIDE___LIBC_MEMALIGN + #define JEMALLOC_OVERRIDE___LIBC_REALLOC + #define JEMALLOC_OVERRIDE___LIBC_VALLOC +#endif +/* #undef JEMALLOC_OVERRIDE___POSIX_MEMALIGN */ + +/* + * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. + * For shared libraries, symbol visibility mechanisms prevent these symbols + * from being exported, but for static libraries, naming collisions are a real + * possibility. + */ +#define JEMALLOC_PRIVATE_NAMESPACE je_ + +/* + * Hyper-threaded CPUs may need a special instruction inside spin loops in + * order to yield to another virtual CPU. + */ +#define CPU_SPINWAIT __asm__ volatile("pause") +/* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ +#define HAVE_CPU_SPINWAIT 1 + +/* + * Number of significant bits in virtual addresses. This may be less than the + * total number of bits in a pointer, e.g. on x64, for which the uppermost 16 + * bits are the same as bit 47. + */ +#define LG_VADDR 48 + +/* Defined if C11 atomics are available. */ +#define JEMALLOC_C11_ATOMICS + +/* Defined if GCC __atomic atomics are available. */ +#define JEMALLOC_GCC_ATOMIC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_ATOMIC_ATOMICS + +/* Defined if GCC __sync atomics are available. */ +#define JEMALLOC_GCC_SYNC_ATOMICS +/* and the 8-bit variant support. */ +#define JEMALLOC_GCC_U8_SYNC_ATOMICS + +/* + * Defined if __builtin_clz() and __builtin_clzl() are available. + */ +#define JEMALLOC_HAVE_BUILTIN_CLZ + +/* + * Defined if os_unfair_lock_*() functions are available, as provided by Darwin. + */ +/* #undef JEMALLOC_OS_UNFAIR_LOCK */ + +/* Defined if syscall(2) is usable. */ +#define JEMALLOC_USE_SYSCALL + +/* + * Defined if secure_getenv(3) is available. + */ +/* #undef JEMALLOC_HAVE_SECURE_GETENV */ + +/* + * Defined if issetugid(2) is available. + */ +/* #undef JEMALLOC_HAVE_ISSETUGID */ + +/* Defined if pthread_atfork(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_ATFORK + +/* Defined if pthread_setname_np(3) is available. */ +#define JEMALLOC_HAVE_PTHREAD_SETNAME_NP + +/// musl doesn't support it +/* Defined if pthread_getname_np(3) is available. */ +/* #define JEMALLOC_HAVE_PTHREAD_GETNAME_NP */ + +/* Defined if pthread_get_name_np(3) is available. */ +/* #undef JEMALLOC_HAVE_PTHREAD_GET_NAME_NP */ + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC_COARSE, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE + +/* + * Defined if clock_gettime(CLOCK_MONOTONIC, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_MONOTONIC + +/* + * Defined if mach_absolute_time() is available. + */ +/* #undef JEMALLOC_HAVE_MACH_ABSOLUTE_TIME */ + +/* + * Defined if clock_gettime(CLOCK_REALTIME, ...) is available. + */ +#define JEMALLOC_HAVE_CLOCK_REALTIME + +/* + * Defined if _malloc_thread_cleanup() exists. At least in the case of + * FreeBSD, pthread_key_create() allocates, which if used during malloc + * bootstrapping will cause recursion into the pthreads library. Therefore, if + * _malloc_thread_cleanup() exists, use it as the basis for thread cleanup in + * malloc_tsd. + */ +/* #undef JEMALLOC_MALLOC_THREAD_CLEANUP */ + +/* + * Defined if threaded initialization is known to be safe on this platform. + * Among other things, it must be possible to initialize a mutex without + * triggering allocation in order for threaded allocation to be safe. + */ +#define JEMALLOC_THREADED_INIT + +/* + * Defined if the pthreads implementation defines + * _pthread_mutex_init_calloc_cb(), in which case the function is used in order + * to avoid recursive allocation during mutex initialization. + */ +/* #undef JEMALLOC_MUTEX_INIT_CB */ + +/* Non-empty if the tls_model attribute is supported. */ +#define JEMALLOC_TLS_MODEL __attribute__((tls_model("initial-exec"))) + +/* + * JEMALLOC_DEBUG enables assertions and other sanity checks, and disables + * inline functions. + */ +/* #undef JEMALLOC_DEBUG */ + +/* JEMALLOC_STATS enables statistics calculation. */ +#define JEMALLOC_STATS + +/* JEMALLOC_EXPERIMENTAL_SMALLOCX_API enables experimental smallocx API. */ +/* #undef JEMALLOC_EXPERIMENTAL_SMALLOCX_API */ + +/* JEMALLOC_PROF enables allocation profiling. */ +/* #undef JEMALLOC_PROF */ + +/* Use libunwind for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBUNWIND */ + +/* Use libgcc for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_LIBGCC */ + +/* Use gcc intrinsics for profile backtracing if defined. */ +/* #undef JEMALLOC_PROF_GCC */ + +/* + * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage + * segment (DSS). + */ +#define JEMALLOC_DSS + +/* Support memory filling (junk/zero). */ +#define JEMALLOC_FILL + +/* Support utrace(2)-based tracing. */ +/* #undef JEMALLOC_UTRACE */ + +/* Support utrace(2)-based tracing (label based signature). */ +/* #undef JEMALLOC_UTRACE_LABEL */ + +/* Support optional abort() on OOM. */ +/* #undef JEMALLOC_XMALLOC */ + +/* Support lazy locking (avoid locking unless a second thread is launched). */ +/* #undef JEMALLOC_LAZY_LOCK */ + +/* + * Minimum allocation alignment is 2^LG_QUANTUM bytes (ignoring tiny size + * classes). + */ +/* #undef LG_QUANTUM */ + +/* One page is 2^LG_PAGE bytes. */ +#define LG_PAGE 12 + +/* Maximum number of regions in a slab. */ +/* #undef CONFIG_LG_SLAB_MAXREGS */ + +/* + * One huge page is 2^LG_HUGEPAGE bytes. Note that this is defined even if the + * system does not explicitly support huge pages; system calls that require + * explicit huge page support are separately configured. + */ +#define LG_HUGEPAGE 21 + +/* + * If defined, adjacent virtual memory mappings with identical attributes + * automatically coalesce, and they fragment when changes are made to subranges. + * This is the normal order of things for mmap()/munmap(), but on Windows + * VirtualAlloc()/VirtualFree() operations must be precisely matched, i.e. + * mappings do *not* coalesce/fragment. + */ +#define JEMALLOC_MAPS_COALESCE + +/* + * If defined, retain memory for later reuse by default rather than using e.g. + * munmap() to unmap freed extents. This is enabled on 64-bit Linux because + * common sequences of mmap()/munmap() calls will cause virtual memory map + * holes. + */ +#define JEMALLOC_RETAIN + +/* TLS is used to map arenas and magazine caches to threads. */ +#define JEMALLOC_TLS + +/* + * Used to mark unreachable code to quiet "end of non-void" compiler warnings. + * Don't use this directly; instead use unreachable() from util.h + */ +#define JEMALLOC_INTERNAL_UNREACHABLE __builtin_unreachable + +/* + * ffs*() functions to use for bitmapping. Don't use these directly; instead, + * use ffs_*() from util.h. + */ +#define JEMALLOC_INTERNAL_FFSLL __builtin_ffsll +#define JEMALLOC_INTERNAL_FFSL __builtin_ffsl +#define JEMALLOC_INTERNAL_FFS __builtin_ffs + +/* + * popcount*() functions to use for bitmapping. + */ +#define JEMALLOC_INTERNAL_POPCOUNTL __builtin_popcountl +#define JEMALLOC_INTERNAL_POPCOUNT __builtin_popcount + +/* + * If defined, explicitly attempt to more uniformly distribute large allocation + * pointer alignments across all cache indices. + */ +#define JEMALLOC_CACHE_OBLIVIOUS + +/* + * If defined, enable logging facilities. We make this a configure option to + * avoid taking extra branches everywhere. + */ +/* #undef JEMALLOC_LOG */ + +/* + * If defined, use readlinkat() (instead of readlink()) to follow + * /etc/malloc_conf. + */ +/* #undef JEMALLOC_READLINKAT */ + +/* + * Darwin (OS X) uses zones to work around Mach-O symbol override shortcomings. + */ +/* #undef JEMALLOC_ZONE */ + +/* + * Methods for determining whether the OS overcommits. + * JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY: Linux's + * /proc/sys/vm.overcommit_memory file. + * JEMALLOC_SYSCTL_VM_OVERCOMMIT: FreeBSD's vm.overcommit sysctl. + */ +/* #undef JEMALLOC_SYSCTL_VM_OVERCOMMIT */ +#define JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY + +/* Defined if madvise(2) is available. */ +#define JEMALLOC_HAVE_MADVISE + +/* + * Defined if transparent huge pages are supported via the MADV_[NO]HUGEPAGE + * arguments to madvise(2). + */ +#define JEMALLOC_HAVE_MADVISE_HUGE + +/* + * Methods for purging unused pages differ between operating systems. + * + * madvise(..., MADV_FREE) : This marks pages as being unused, such that they + * will be discarded rather than swapped out. + * madvise(..., MADV_DONTNEED) : If JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS is + * defined, this immediately discards pages, + * such that new pages will be demand-zeroed if + * the address region is later touched; + * otherwise this behaves similarly to + * MADV_FREE, though typically with higher + * system overhead. + */ +#define JEMALLOC_PURGE_MADVISE_FREE +#define JEMALLOC_PURGE_MADVISE_DONTNEED +#define JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS + +/* Defined if madvise(2) is available but MADV_FREE is not (x86 Linux only). */ +/* #undef JEMALLOC_DEFINE_MADVISE_FREE */ + +/* + * Defined if MADV_DO[NT]DUMP is supported as an argument to madvise. + */ +#define JEMALLOC_MADVISE_DONTDUMP + +/* + * Defined if MADV_[NO]CORE is supported as an argument to madvise. + */ +/* #undef JEMALLOC_MADVISE_NOCORE */ + +/* Defined if mprotect(2) is available. */ +#define JEMALLOC_HAVE_MPROTECT + +/* + * Defined if transparent huge pages (THPs) are supported via the + * MADV_[NO]HUGEPAGE arguments to madvise(2), and THP support is enabled. + */ +/* #undef JEMALLOC_THP */ + +/* Defined if posix_madvise is available. */ +/* #undef JEMALLOC_HAVE_POSIX_MADVISE */ + +/* + * Method for purging unused pages using posix_madvise. + * + * posix_madvise(..., POSIX_MADV_DONTNEED) + */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED */ +/* #undef JEMALLOC_PURGE_POSIX_MADVISE_DONTNEED_ZEROS */ + +/* + * Defined if memcntl page admin call is supported + */ +/* #undef JEMALLOC_HAVE_MEMCNTL */ + +/* + * Defined if malloc_size is supported + */ +/* #undef JEMALLOC_HAVE_MALLOC_SIZE */ + +/* Define if operating system has alloca.h header. */ +#define JEMALLOC_HAS_ALLOCA_H + +/* C99 restrict keyword supported. */ +#define JEMALLOC_HAS_RESTRICT + +/* For use by hash code. */ +/* #undef JEMALLOC_BIG_ENDIAN */ + +/* sizeof(int) == 2^LG_SIZEOF_INT. */ +#define LG_SIZEOF_INT 2 + +/* sizeof(long) == 2^LG_SIZEOF_LONG. */ +#define LG_SIZEOF_LONG 3 + +/* sizeof(long long) == 2^LG_SIZEOF_LONG_LONG. */ +#define LG_SIZEOF_LONG_LONG 3 + +/* sizeof(intmax_t) == 2^LG_SIZEOF_INTMAX_T. */ +#define LG_SIZEOF_INTMAX_T 3 + +/* glibc malloc hooks (__malloc_hook, __realloc_hook, __free_hook). */ +#define JEMALLOC_GLIBC_MALLOC_HOOK + +/* glibc memalign hook. */ +#define JEMALLOC_GLIBC_MEMALIGN_HOOK + +/* pthread support */ +#define JEMALLOC_HAVE_PTHREAD + +/* dlsym() support */ +#define JEMALLOC_HAVE_DLSYM + +/* Adaptive mutex support in pthreads. */ +#define JEMALLOC_HAVE_PTHREAD_MUTEX_ADAPTIVE_NP + +/* GNU specific sched_getcpu support */ +#define JEMALLOC_HAVE_SCHED_GETCPU + +/* GNU specific sched_setaffinity support */ +#define JEMALLOC_HAVE_SCHED_SETAFFINITY + +/* + * If defined, all the features necessary for background threads are present. + */ +#define JEMALLOC_BACKGROUND_THREAD + +/* + * If defined, jemalloc symbols are not exported (doesn't work when + * JEMALLOC_PREFIX is not defined). + */ +/* #undef JEMALLOC_EXPORT */ + +/* config.malloc_conf options string. */ +#define JEMALLOC_CONFIG_MALLOC_CONF "@JEMALLOC_CONFIG_MALLOC_CONF@" + +/* If defined, jemalloc takes the malloc/free/etc. symbol names. */ +#define JEMALLOC_IS_MALLOC + +/* + * Defined if strerror_r returns char * if _GNU_SOURCE is defined. + */ +#define JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE + +/* Performs additional safety checks when defined. */ +/* #undef JEMALLOC_OPT_SAFETY_CHECKS */ + +/* Is C++ support being built? */ +/* #undef JEMALLOC_ENABLE_CXX */ + +/* Performs additional size checks when defined. */ +/* #undef JEMALLOC_OPT_SIZE_CHECKS */ + +/* Allows sampled junk and stash for checking use-after-free when defined. */ +/* #undef JEMALLOC_UAF_DETECTION */ + +/* Darwin VM_MAKE_TAG support */ +/* #undef JEMALLOC_HAVE_VM_MAKE_TAG */ + +#endif /* JEMALLOC_INTERNAL_DEFS_H_ */ From 7574b4b4ed47cec4f7948fb0c2d6fa7abbcc2dc5 Mon Sep 17 00:00:00 2001 From: hehechen Date: Wed, 6 Jul 2022 15:59:02 +0800 Subject: [PATCH 078/104] *: TiFlash pagectl/dttool use only-decryption mode (#5271) close pingcap/tiflash#5122 --- dbms/src/Server/CLIService.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Server/CLIService.h b/dbms/src/Server/CLIService.h index 9078fa991f3..0acffebb577 100644 --- a/dbms/src/Server/CLIService.h +++ b/dbms/src/Server/CLIService.h @@ -126,6 +126,8 @@ CLIService::TiFlashProxyConfig::TiFlashProxyConfig(Poco::Util::Layer args.push_back(v.first.data()); args.push_back(v.second.data()); } + // Start the decryption service without starting the raftstore service + args.push_back("--only-decryption"); is_proxy_runnable = true; } template From b2d8d50542cf5667a5da4fa5b1435ff2c4e9f904 Mon Sep 17 00:00:00 2001 From: bestwoody <89765764+bestwoody@users.noreply.github.com> Date: Wed, 6 Jul 2022 19:51:03 +0800 Subject: [PATCH 079/104] suppresion false positive report from tsan (#5303) close pingcap/tiflash#5088 --- .github/licenserc.yml | 1 + tests/sanitize/tsan.suppression | 1 + 2 files changed, 2 insertions(+) create mode 100644 tests/sanitize/tsan.suppression diff --git a/.github/licenserc.yml b/.github/licenserc.yml index eaf7a49eee4..b122b2c9775 100644 --- a/.github/licenserc.yml +++ b/.github/licenserc.yml @@ -25,6 +25,7 @@ header: - '**/LICENSE.TXT' - '**/cipher-file-256' - '**/asan.suppression' + - '**/tsan.suppression' - '**/LICENSE.TXT' - '**/LICENSE' - '**/README' diff --git a/tests/sanitize/tsan.suppression b/tests/sanitize/tsan.suppression new file mode 100644 index 00000000000..73824caa2b9 --- /dev/null +++ b/tests/sanitize/tsan.suppression @@ -0,0 +1 @@ +race:dbms/src/Common/TiFlashMetrics.h From 57d001c1f91ef656d70eff13008e0fbff5588873 Mon Sep 17 00:00:00 2001 From: Jiarui Li <34512395+Willendless@users.noreply.github.com> Date: Wed, 6 Jul 2022 11:15:02 -0400 Subject: [PATCH 080/104] Refine test framework code and tests (#5261) close pingcap/tiflash#5262 --- dbms/src/Flash/tests/gtest_executor.cpp | 124 +++++++-------- dbms/src/Flash/tests/gtest_limit_executor.cpp | 2 +- .../Flash/tests/gtest_projection_executor.cpp | 41 +++-- dbms/src/Flash/tests/gtest_topn_executor.cpp | 16 +- dbms/src/TestUtils/ExecutorTestUtils.cpp | 18 +-- dbms/src/TestUtils/ExecutorTestUtils.h | 14 +- dbms/src/TestUtils/FunctionTestUtils.cpp | 114 +++++++++++++- dbms/src/TestUtils/FunctionTestUtils.h | 19 ++- .../tests/gtest_window_functions.cpp | 142 ++++++++---------- 9 files changed, 294 insertions(+), 196 deletions(-) diff --git a/dbms/src/Flash/tests/gtest_executor.cpp b/dbms/src/Flash/tests/gtest_executor.cpp index b4ba1a75563..d0e7b7e6c67 100644 --- a/dbms/src/Flash/tests/gtest_executor.cpp +++ b/dbms/src/Flash/tests/gtest_executor.cpp @@ -69,18 +69,18 @@ try .filter(eq(col("s1"), col("s2"))) .build(context); { - executeStreams(request, - {toNullableVec({"banana"}), - toNullableVec({"banana"})}); + ASSERT_COLUMNS_EQ_R(executeStreams(request), + createColumns({toNullableVec({"banana"}), + toNullableVec({"banana"})})); } request = context.receive("exchange1") .filter(eq(col("s1"), col("s2"))) .build(context); { - executeStreams(request, - {toNullableVec({"banana"}), - toNullableVec({"banana"})}); + ASSERT_COLUMNS_EQ_R(executeStreams(request), + createColumns({toNullableVec({"banana"}), + toNullableVec({"banana"})})); } } CATCH @@ -99,25 +99,23 @@ try " table_scan_0 | {<0, String>, <1, String>}\n" " table_scan_1 | {<0, String>, <1, String>}\n"; ASSERT_DAGREQUEST_EQAUL(expected, request); - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 2); - - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 5); - - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); + + ASSERT_COLUMNS_EQ_R(executeStreams(request, 5), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); + + ASSERT_COLUMNS_EQ_R(executeStreams(request), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); } request = context .scan("test_db", "l_table") @@ -132,10 +130,9 @@ try " table_scan_0 | {<0, String>, <1, String>}\n" " table_scan_1 | {<0, String>, <1, String>}\n"; ASSERT_DAGREQUEST_EQAUL(expected, request); - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 2); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); } request = context @@ -149,18 +146,16 @@ try " table_scan_0 | {<0, String>, <1, String>}\n" " table_scan_1 | {<0, String>, <1, String>}\n"; ASSERT_DAGREQUEST_EQAUL(expected, request); - executeStreams(request, - {toNullableVec({"banana", "banana", "banana", "banana"}), - toNullableVec({"apple", "apple", "apple", "banana"}), - toNullableVec({"banana", "banana", "banana", {}}), - toNullableVec({"apple", "apple", "apple", {}})}, - 2); - executeStreams(request, - {toNullableVec({"banana", "banana", "banana", "banana"}), - toNullableVec({"apple", "apple", "apple", "banana"}), - toNullableVec({"banana", "banana", "banana", {}}), - toNullableVec({"apple", "apple", "apple", {}})}, - 3); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), + createColumns({toNullableVec({"banana", "banana", "banana", "banana"}), + toNullableVec({"apple", "apple", "apple", "banana"}), + toNullableVec({"banana", "banana", "banana", {}}), + toNullableVec({"apple", "apple", "apple", {}})})); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 3), + createColumns({toNullableVec({"banana", "banana", "banana", "banana"}), + toNullableVec({"apple", "apple", "apple", "banana"}), + toNullableVec({"banana", "banana", "banana", {}}), + toNullableVec({"apple", "apple", "apple", {}})})); } } CATCH @@ -179,25 +174,23 @@ try " exchange_receiver_0 | type:PassThrough, {<0, String>, <1, String>}\n" " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; ASSERT_DAGREQUEST_EQAUL(expected, request); - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 2); - - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 5); - - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); + + ASSERT_COLUMNS_EQ_R(executeStreams(request, 5), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); + + ASSERT_COLUMNS_EQ_R(executeStreams(request), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); } } CATCH @@ -216,12 +209,11 @@ try " table_scan_0 | {<0, String>, <1, String>}\n" " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; ASSERT_DAGREQUEST_EQAUL(expected, request); - executeStreams(request, - {toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})}, - 2); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); } } CATCH diff --git a/dbms/src/Flash/tests/gtest_limit_executor.cpp b/dbms/src/Flash/tests/gtest_limit_executor.cpp index e4a3aa5db5e..47482540b39 100644 --- a/dbms/src/Flash/tests/gtest_limit_executor.cpp +++ b/dbms/src/Flash/tests/gtest_limit_executor.cpp @@ -68,7 +68,7 @@ try else expect_cols = {toNullableVec(col_name, ColumnWithData(col0.begin(), col0.begin() + limit_num))}; - executeStreams(request, expect_cols); + ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols); } } CATCH diff --git a/dbms/src/Flash/tests/gtest_projection_executor.cpp b/dbms/src/Flash/tests/gtest_projection_executor.cpp index 4f6401eb483..2ff0fdff780 100644 --- a/dbms/src/Flash/tests/gtest_projection_executor.cpp +++ b/dbms/src/Flash/tests/gtest_projection_executor.cpp @@ -44,17 +44,16 @@ class ExecutorProjectionTestRunner : public DB::tests::ExecutorTest } template - std::shared_ptr buildDAGRequest(T param, const String & sort_col) + std::shared_ptr buildDAGRequest(T param) { - /// topN is introduced, so that we can get stable results in concurrency environment. - return context.scan(db_name, table_name).project(param).topN(sort_col, false, 100).build(context); + return context.scan(db_name, table_name).project(param).build(context); }; void executeWithConcurrency(const std::shared_ptr & request, const ColumnsWithTypeAndName & expect_columns) { for (size_t i = 1; i < 10; i += 2) { - executeStreams(request, expect_columns, i); + ASSERT_COLUMNS_EQ_UR(executeStreams(request, i), expect_columns); } } @@ -87,11 +86,11 @@ TEST_F(ExecutorProjectionTestRunner, Projection) try { /// Check single column - auto request = buildDAGRequest({col_names[4]}, col_names[4]); + auto request = buildDAGRequest({col_names[4]}); executeWithConcurrency(request, {toNullableVec(col_names[4], col4_sorted_asc)}); /// Check multi columns - request = buildDAGRequest({col_names[0], col_names[4]}, col_names[4]); + request = buildDAGRequest({col_names[0], col_names[4]}); executeWithConcurrency(request, { toNullableVec(col_names[0], col0_sorted_asc), @@ -99,14 +98,14 @@ try }); /// Check multi columns - request = buildDAGRequest({col_names[0], col_names[1], col_names[4]}, col_names[4]); + request = buildDAGRequest({col_names[0], col_names[1], col_names[4]}); executeWithConcurrency(request, {toNullableVec(col_names[0], col0_sorted_asc), toNullableVec(col_names[1], col1_sorted_asc), toNullableVec(col_names[4], col4_sorted_asc)}); /// Check duplicate columns - request = buildDAGRequest({col_names[4], col_names[4], col_names[4]}, col_names[4]); + request = buildDAGRequest({col_names[4], col_names[4], col_names[4]}); executeWithConcurrency(request, {toNullableVec(col_names[4], col4_sorted_asc), toNullableVec(col_names[4], col4_sorted_asc), @@ -125,7 +124,7 @@ try columns.push_back(expect_column); } - request = buildDAGRequest(projection_input, col_names[4]); + request = buildDAGRequest(projection_input); executeWithConcurrency(request, columns); } } @@ -139,18 +138,18 @@ try /// Test "equal" function /// Data type: TypeString - request = buildDAGRequest({eq(col(col_names[0]), col(col_names[0])), col(col_names[4])}, col_names[4]); + request = buildDAGRequest({eq(col(col_names[0]), col(col_names[0])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 1, 1, 1, 1, 1, 1}), toNullableVec(col_names[4], col4_sorted_asc)}); - request = buildDAGRequest({eq(col(col_names[0]), col(col_names[1])), col(col_names[4])}, col_names[4]); + request = buildDAGRequest({eq(col(col_names[0]), col(col_names[1])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 1, 0, {}, 0, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); /// Data type: TypeLong - request = buildDAGRequest({eq(col(col_names[3]), col(col_names[4])), col(col_names[4])}, col_names[4]); + request = buildDAGRequest({eq(col(col_names[3]), col(col_names[4])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 0, 0, {}, 1, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); @@ -159,23 +158,23 @@ try /// Test "greater" function /// Data type: TypeString - request = buildDAGRequest({gt(col(col_names[0]), col(col_names[1])), col(col_names[4])}, col_names[4]); + request = buildDAGRequest({gt(col(col_names[0]), col(col_names[1])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 0, 0, {}, 0, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); - request = buildDAGRequest({gt(col(col_names[1]), col(col_names[0])), col(col_names[4])}, col_names[4]); + request = buildDAGRequest({gt(col(col_names[1]), col(col_names[0])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 1, 0, 1, {}, 1, 1}), toNullableVec(col_names[4], col4_sorted_asc)}); /// Data type: TypeLong - request = buildDAGRequest({gt(col(col_names[3]), col(col_names[4])), col(col_names[4])}, col_names[4]); + request = buildDAGRequest({gt(col(col_names[3]), col(col_names[4])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 1, 1, {}, 0, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); - request = buildDAGRequest({gt(col(col_names[4]), col(col_names[3])), col(col_names[4])}, col_names[4]); + request = buildDAGRequest({gt(col(col_names[4]), col(col_names[3])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 1, 0, 0, {}, 0, 1}), toNullableVec(col_names[4], col4_sorted_asc)}); @@ -184,18 +183,18 @@ try /// Test "and" function /// Data type: TypeString - request = buildDAGRequest({And(col(col_names[0]), col(col_names[0])), col(col_names[4])}, col_names[4]); + request = buildDAGRequest({And(col(col_names[0]), col(col_names[0])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 0, 0, 0, 0, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); - request = buildDAGRequest({And(col(col_names[0]), col(col_names[1])), col(col_names[4])}, col_names[4]); + request = buildDAGRequest({And(col(col_names[0]), col(col_names[1])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({0, 0, 0, 0, 0, 0, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); /// Data type: TypeLong - request = buildDAGRequest({And(col(col_names[3]), col(col_names[4])), col(col_names[4])}, col_names[4]); + request = buildDAGRequest({And(col(col_names[3]), col(col_names[4])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 1, 0, 0, {}, 1, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); @@ -203,7 +202,7 @@ try /// Test "not" function /// Data type: TypeString - request = buildDAGRequest({NOT(col(col_names[0])), NOT(col(col_names[1])), NOT(col(col_names[2])), col(col_names[4])}, col_names[4]); + request = buildDAGRequest({NOT(col(col_names[0])), NOT(col(col_names[1])), NOT(col(col_names[2])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 1, 1, 1, 1, 1, 1}), toNullableVec({1, 1, 1, 1, {}, 1, 1}), @@ -211,7 +210,7 @@ try toNullableVec(col_names[4], col4_sorted_asc)}); /// Data type: TypeLong - request = buildDAGRequest({NOT(col(col_names[3])), NOT(col(col_names[4])), col(col_names[4])}, col_names[4]); + request = buildDAGRequest({NOT(col(col_names[3])), NOT(col(col_names[4])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 1, 0, {}, 0, 1}), toNullableVec({{}, 0, 0, 1, 0, 0, 0}), diff --git a/dbms/src/Flash/tests/gtest_topn_executor.cpp b/dbms/src/Flash/tests/gtest_topn_executor.cpp index 0e55702795d..d5466b5c87d 100644 --- a/dbms/src/Flash/tests/gtest_topn_executor.cpp +++ b/dbms/src/Flash/tests/gtest_topn_executor.cpp @@ -103,10 +103,10 @@ try else expect_cols.push_back({toNullableVec(single_col_name, ColumnWithString(col0.begin(), col0.begin() + limit_num))}); - executeStreams(request, expect_cols[0]); - executeStreams(request, expect_cols[0], 2); - executeStreams(request, expect_cols[0], 4); - executeStreams(request, expect_cols[0], 8); + ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[0]); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), expect_cols[0]); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 4), expect_cols[0]); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 8), expect_cols[0]); } } } @@ -139,7 +139,7 @@ try for (size_t i = 0; i < test_num; ++i) { request = buildDAGRequest(table_name, order_by_items[i], 100); - executeStreams(request, expect_cols[i]); + ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[i]); } } } @@ -173,7 +173,7 @@ try func_projection = {col0_ast, col1_ast, col2_ast, col3_ast, func_ast}; request = buildDAGRequest(table_name, order_by_items, 100, func_projection, output_projection); - executeStreams(request, expect_cols[0]); + ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[0]); } } @@ -191,7 +191,7 @@ try func_projection = {col0_ast, col1_ast, col2_ast, col3_ast, func_ast}; request = buildDAGRequest(table_name, order_by_items, 100, func_projection, output_projection); - executeStreams(request, expect_cols[0]); + ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[0]); } } @@ -209,7 +209,7 @@ try func_projection = {col0_ast, col1_ast, col2_ast, col3_ast, func_ast}; request = buildDAGRequest(table_name, order_by_items, 100, func_projection, output_projection); - executeStreams(request, expect_cols[0]); + ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[0]); } } diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index 881ebaf88db..634e483abd2 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -104,41 +104,39 @@ Block mergeBlocks(Blocks blocks) return Block(actual_columns); } -void readBlock(BlockInputStreamPtr stream, const ColumnsWithTypeAndName & expect_columns) +DB::ColumnsWithTypeAndName readBlock(BlockInputStreamPtr stream) { Blocks actual_blocks; - Block except_block(expect_columns); stream->readPrefix(); while (auto block = stream->read()) { actual_blocks.push_back(block); } stream->readSuffix(); - Block actual_block = mergeBlocks(actual_blocks); - ASSERT_BLOCK_EQ(except_block, actual_block); + return mergeBlocks(actual_blocks).getColumnsWithTypeAndName(); } } // namespace -void ExecutorTest::executeStreams(const std::shared_ptr & request, std::unordered_map & source_columns_map, const ColumnsWithTypeAndName & expect_columns, size_t concurrency) +DB::ColumnsWithTypeAndName ExecutorTest::executeStreams(const std::shared_ptr & request, std::unordered_map & source_columns_map, size_t concurrency) { DAGContext dag_context(*request, "executor_test", concurrency); dag_context.setColumnsForTest(source_columns_map); context.context.setDAGContext(&dag_context); // Currently, don't care about regions information in tests. DAGQuerySource dag(context.context); - readBlock(executeQuery(dag, context.context, false, QueryProcessingStage::Complete).in, expect_columns); + return readBlock(executeQuery(dag, context.context, false, QueryProcessingStage::Complete).in); } -void ExecutorTest::executeStreams(const std::shared_ptr & request, const ColumnsWithTypeAndName & expect_columns, size_t concurrency) +DB::ColumnsWithTypeAndName ExecutorTest::executeStreams(const std::shared_ptr & request, size_t concurrency) { - executeStreams(request, context.executorIdColumnsMap(), expect_columns, concurrency); + return executeStreams(request, context.executorIdColumnsMap(), concurrency); } -void ExecutorTest::executeStreamsWithSingleSource(const std::shared_ptr & request, const ColumnsWithTypeAndName & source_columns, const ColumnsWithTypeAndName & expect_columns, SourceType type, size_t concurrency) +DB::ColumnsWithTypeAndName ExecutorTest::executeStreamsWithSingleSource(const std::shared_ptr & request, const ColumnsWithTypeAndName & source_columns, SourceType type, size_t concurrency) { std::unordered_map source_columns_map; source_columns_map[getSourceName(type)] = source_columns; - executeStreams(request, source_columns_map, expect_columns, concurrency); + return executeStreams(request, source_columns_map, concurrency); } void ExecutorTest::dagRequestEqual(const String & expected_string, const std::shared_ptr & actual) diff --git a/dbms/src/TestUtils/ExecutorTestUtils.h b/dbms/src/TestUtils/ExecutorTestUtils.h index 87bb7115bed..59b829e04b5 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.h +++ b/dbms/src/TestUtils/ExecutorTestUtils.h @@ -25,6 +25,9 @@ namespace DB::tests { void executeInterpreter(const std::shared_ptr & request, Context & context); + +::testing::AssertionResult check_columns_equality(const ColumnsWithTypeAndName & expected, const ColumnsWithTypeAndName & actual, bool _restrict); + class ExecutorTest : public ::testing::Test { protected: @@ -72,20 +75,17 @@ class ExecutorTest : public ::testing::Test } } - void executeStreams( + ColumnsWithTypeAndName executeStreams( const std::shared_ptr & request, std::unordered_map & source_columns_map, - const ColumnsWithTypeAndName & expect_columns, size_t concurrency = 1); - void executeStreams( + ColumnsWithTypeAndName executeStreams( const std::shared_ptr & request, - const ColumnsWithTypeAndName & expect_columns, size_t concurrency = 1); - void executeStreamsWithSingleSource( + ColumnsWithTypeAndName executeStreamsWithSingleSource( const std::shared_ptr & request, const ColumnsWithTypeAndName & source_columns, - const ColumnsWithTypeAndName & expect_columns, SourceType type = TableScan, size_t concurrency = 1); @@ -96,4 +96,4 @@ class ExecutorTest : public ::testing::Test #define ASSERT_DAGREQUEST_EQAUL(str, request) dagRequestEqual((str), (request)); #define ASSERT_BLOCKINPUTSTREAM_EQAUL(str, request, concurrency) executeInterpreter((str), (request), (concurrency)) -} // namespace DB::tests \ No newline at end of file +} // namespace DB::tests diff --git a/dbms/src/TestUtils/FunctionTestUtils.cpp b/dbms/src/TestUtils/FunctionTestUtils.cpp index 7fb526aeb01..9fbf3c9691f 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.cpp +++ b/dbms/src/TestUtils/FunctionTestUtils.cpp @@ -14,6 +14,7 @@ #include #include +#include #include #include #include @@ -23,7 +24,10 @@ #include #include #include -#include + +#include +#include + namespace DB { @@ -103,22 +107,117 @@ ::testing::AssertionResult columnEqual( return columnEqual(expected.column, actual.column); } -void blockEqual( +::testing::AssertionResult blockEqual( const Block & expected, const Block & actual) { size_t columns = actual.columns(); size_t expected_columns = expected.columns(); - ASSERT_EQ(expected_columns, columns); + ASSERT_EQUAL(expected_columns, columns, "Block size mismatch"); for (size_t i = 0; i < columns; ++i) { const auto & expected_col = expected.getByPosition(i); const auto & actual_col = actual.getByPosition(i); - ASSERT_EQ(actual_col.type->getName(), expected_col.type->getName()); - ASSERT_COLUMN_EQ(expected_col.column, actual_col.column); + auto cmp_res = columnEqual(expected_col, actual_col); + if (!cmp_res) + return cmp_res; + } + return ::testing::AssertionSuccess(); +} + +/// size of each column should be the same +std::multiset columnsToRowSet(const ColumnsWithTypeAndName & cols) +{ + if (cols.empty()) + return {}; + if (cols[0].column->empty()) + return {}; + + size_t cols_size = cols.size(); + std::vector rows{cols[0].column->size()}; + + for (auto & r : rows) + { + r.resize(cols_size, true); + } + + for (auto const & [col_id, col] : ext::enumerate(cols)) + { + for (size_t i = 0, size = col.column->size(); i < size; ++i) + { + new (rows[i].place(col_id)) Field((*col.column)[i]); + } + } + return {std::make_move_iterator(rows.begin()), std::make_move_iterator(rows.end())}; +} + +::testing::AssertionResult columnsEqual( + const ColumnsWithTypeAndName & expected, + const ColumnsWithTypeAndName & actual, + bool _restrict) +{ + if (_restrict) + return blockEqual(Block(expected), Block(actual)); + + auto expect_cols_size = expected.size(); + auto actual_cols_size = actual.size(); + + ASSERT_EQUAL(expect_cols_size, actual_cols_size, "Columns size mismatch"); + + for (size_t i = 0; i < expect_cols_size; ++i) + { + auto const & expect_col = expected[i]; + auto const & actual_col = actual[i]; + ASSERT_EQUAL(expect_col.column->getName(), actual_col.column->getName(), fmt::format("Column {} name mismatch", i)); + ASSERT_EQUAL(expect_col.column->size(), actual_col.column->size(), fmt::format("Column {} size mismatch", i)); + auto type_eq = dataTypeEqual(expected[i].type, actual[i].type); + if (!type_eq) + return type_eq; + } + + auto const expected_row_set = columnsToRowSet(expected); + auto const actual_row_set = columnsToRowSet(actual); + + if (expected_row_set != actual_row_set) + { + FmtBuffer buf; + + auto expect_it = expected_row_set.begin(); + auto actual_it = actual_row_set.begin(); + + buf.append("Columns row set mismatch\n").append("expected_row_set:\n"); + for (; expect_it != expected_row_set.end(); ++expect_it, ++actual_it) + { + buf.joinStr( + expect_it->begin(), + expect_it->end(), + [](const auto & v, FmtBuffer & fb) { fb.append(v.toString()); }, + " ") + .append("\n"); + if (*expect_it != *actual_it) + break; + } + + ++actual_it; + + buf.append("...\nactual_row_set:\n"); + for (auto it = actual_row_set.begin(); it != actual_it; ++it) + { + buf.joinStr( + it->begin(), + it->end(), + [](const auto & v, FmtBuffer & fb) { fb.append(v.toString()); }, + " ") + .append("\n"); + } + buf.append("...\n"); + + return testing::AssertionFailure() << buf.toString(); } + + return testing::AssertionSuccess(); } std::pair buildFunction( @@ -275,5 +374,10 @@ ColumnWithTypeAndName toNullableDatetimeVec(String name, const std::vector(fsp)); return {makeColumn>(data_type, vec), data_type, name, 0}; } + +ColumnsWithTypeAndName createColumns(const ColumnsWithTypeAndName & cols) +{ + return cols; +} } // namespace tests } // namespace DB diff --git a/dbms/src/TestUtils/FunctionTestUtils.h b/dbms/src/TestUtils/FunctionTestUtils.h index d6b7351df05..ad01e2e8441 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.h +++ b/dbms/src/TestUtils/FunctionTestUtils.h @@ -514,6 +514,13 @@ ColumnWithTypeAndName createConstColumn( return createConstColumn(data_type_args, size, InferredFieldType(std::nullopt), name); } +// This wrapper function only serves to construct columns input for function-like macros, +// since preprocessor recognizes `{col1, col2, col3}` as three arguments instead of one. +// E.g. preprocessor does not allow us to write `ASSERT_COLUMNS_EQ_R({col1, col2, col3}, actual_cols)`, +// but with this func we can write `ASSERT_COLUMNS_EQ_R(createColumns{col1, col2, col3}, actual_cols)` instead. +ColumnsWithTypeAndName createColumns(const ColumnsWithTypeAndName & cols); + + ::testing::AssertionResult dataTypeEqual( const DataTypePtr & expected, const DataTypePtr & actual); @@ -527,10 +534,15 @@ ::testing::AssertionResult columnEqual( const ColumnWithTypeAndName & expected, const ColumnWithTypeAndName & actual); -void blockEqual( +::testing::AssertionResult blockEqual( const Block & expected, const Block & actual); +::testing::AssertionResult columnsEqual( + const ColumnsWithTypeAndName & expected, + const ColumnsWithTypeAndName & actual, + bool _restrict); + ColumnWithTypeAndName executeFunction( Context & context, const String & func_name, @@ -756,5 +768,10 @@ class FunctionTest : public ::testing::Test #define ASSERT_COLUMN_EQ(expected, actual) ASSERT_TRUE(DB::tests::columnEqual((expected), (actual))) #define ASSERT_BLOCK_EQ(expected, actual) DB::tests::blockEqual((expected), (actual)) + +/// restrictly checking columns equality, both data set and each row's offset should be the same +#define ASSERT_COLUMNS_EQ_R(expected, actual) ASSERT_TRUE(DB::tests::columnsEqual((expected), (actual), true)) +/// unrestrictly checking columns equality, only checking data set equality +#define ASSERT_COLUMNS_EQ_UR(expected, actual) ASSERT_TRUE(DB::tests::columnsEqual((expected), (actual), false)) } // namespace tests } // namespace DB diff --git a/dbms/src/WindowFunctions/tests/gtest_window_functions.cpp b/dbms/src/WindowFunctions/tests/gtest_window_functions.cpp index 3addf73a642..06253cac66e 100644 --- a/dbms/src/WindowFunctions/tests/gtest_window_functions.cpp +++ b/dbms/src/WindowFunctions/tests/gtest_window_functions.cpp @@ -69,11 +69,10 @@ try .sort({{"partition", false}, {"order", false}, {"partition", false}, {"order", false}}, true) .window(RowNumber(), {"order", false}, {"partition", false}, buildDefaultRowsFrame()) .build(context); - executeStreams( - request, - {toNullableVec("partition", {1, 1, 1, 1, 2, 2, 2, 2}), - toNullableVec("order", {1, 1, 2, 2, 1, 1, 2, 2}), - toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})}); + ASSERT_COLUMNS_EQ_R(executeStreams(request), + createColumns({toNullableVec("partition", {1, 1, 1, 1, 2, 2, 2, 2}), + toNullableVec("order", {1, 1, 2, 2, 1, 1, 2, 2}), + toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})})); // null input executeStreamsWithSingleSource( @@ -82,10 +81,8 @@ try {}); // nullable - executeStreamsWithSingleSource( - request, - {toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), {toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2})}}, - {toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2}), toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})}); + ASSERT_COLUMNS_EQ_R(executeStreamsWithSingleSource(request, {toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), {toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2})}}), + createColumns({toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2}), toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})})); // string - sql : select *, row_number() over w1 from test2 window w1 as (partition by partition_string order by order_string) request = context @@ -94,20 +91,18 @@ try .window(RowNumber(), {"order", false}, {"partition", false}, buildDefaultRowsFrame()) .build(context); - executeStreams( - request, - {toNullableVec("partition", {"apple", "apple", "apple", "apple", "banana", "banana", "banana", "banana"}), - toNullableVec("order", {"apple", "apple", "banana", "banana", "apple", "apple", "banana", "banana"}), - toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})}); + ASSERT_COLUMNS_EQ_R(executeStreams(request), + createColumns({toNullableVec("partition", {"apple", "apple", "apple", "apple", "banana", "banana", "banana", "banana"}), + toNullableVec("order", {"apple", "apple", "banana", "banana", "apple", "apple", "banana", "banana"}), + toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})})); // nullable - executeStreamsWithSingleSource( - request, - {toNullableVec("partition", {"banana", "banana", "banana", "banana", {}, "apple", "apple", "apple", "apple"}), - toNullableVec("order", {"apple", "apple", "banana", "banana", {}, "apple", "apple", "banana", "banana"})}, - {toNullableVec("partition", {{}, "apple", "apple", "apple", "apple", "banana", "banana", "banana", "banana"}), - toNullableVec("order", {{}, "apple", "apple", "banana", "banana", "apple", "apple", "banana", "banana"}), - toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})}); + ASSERT_COLUMNS_EQ_R(executeStreamsWithSingleSource(request, + {toNullableVec("partition", {"banana", "banana", "banana", "banana", {}, "apple", "apple", "apple", "apple"}), + toNullableVec("order", {"apple", "apple", "banana", "banana", {}, "apple", "apple", "banana", "banana"})}), + createColumns({toNullableVec("partition", {{}, "apple", "apple", "apple", "apple", "banana", "banana", "banana", "banana"}), + toNullableVec("order", {{}, "apple", "apple", "banana", "banana", "apple", "apple", "banana", "banana"}), + toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})})); // float64 - sql : select *, row_number() over w1 from test3 window w1 as (partition by partition_float order by order_float64) request = context @@ -116,20 +111,18 @@ try .window(RowNumber(), {"order", false}, {"partition", false}, buildDefaultRowsFrame()) .build(context); - executeStreams( - request, - {toNullableVec("partition", {1.00, 1.00, 1.00, 1.00, 2.00, 2.00, 2.00, 2.00}), - toNullableVec("order", {1.00, 1.00, 2.00, 2.00, 1.00, 1.00, 2.00, 2.00}), - toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})}); + ASSERT_COLUMNS_EQ_R(executeStreams(request), + createColumns({toNullableVec("partition", {1.00, 1.00, 1.00, 1.00, 2.00, 2.00, 2.00, 2.00}), + toNullableVec("order", {1.00, 1.00, 2.00, 2.00, 1.00, 1.00, 2.00, 2.00}), + toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})})); // nullable - executeStreamsWithSingleSource( - request, - {toNullableVec("partition", {{}, 1.00, 1.00, 1.00, 1.00, 2.00, 2.00, 2.00, 2.00}), - toNullableVec("order", {{}, 1.00, 1.00, 2.00, 2.00, 1.00, 1.00, 2.00, 2.00})}, - {toNullableVec("partition", {{}, 1.00, 1.00, 1.00, 1.00, 2.00, 2.00, 2.00, 2.00}), - toNullableVec("order", {{}, 1.00, 1.00, 2.00, 2.00, 1.00, 1.00, 2.00, 2.00}), - toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})}); + ASSERT_COLUMNS_EQ_R(executeStreamsWithSingleSource(request, + {toNullableVec("partition", {{}, 1.00, 1.00, 1.00, 1.00, 2.00, 2.00, 2.00, 2.00}), + toNullableVec("order", {{}, 1.00, 1.00, 2.00, 2.00, 1.00, 1.00, 2.00, 2.00})}), + createColumns({toNullableVec("partition", {{}, 1.00, 1.00, 1.00, 1.00, 2.00, 2.00, 2.00, 2.00}), + toNullableVec("order", {{}, 1.00, 1.00, 2.00, 2.00, 1.00, 1.00, 2.00, 2.00}), + toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})})); // datetime - select *, row_number() over w1 from test4 window w1 as (partition by partition_datetime order by order_datetime); request = context @@ -137,22 +130,20 @@ try .sort({{"partition", false}, {"order", false}, {"partition", false}, {"order", false}}, true) .window(RowNumber(), {"order", false}, {"partition", false}, buildDefaultRowsFrame()) .build(context); - executeStreamsWithSingleSource( - request, - {toNullableDatetimeVec("partition", {"20220101010102", "20220101010102", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010101", "20220101010101"}, 0), - toDatetimeVec("order", {"20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010102", "20220101010102"}, 0)}, - {toNullableDatetimeVec("partition", {"20220101010101", "20220101010101", "20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010102", "20220101010102"}, 0), - toNullableDatetimeVec("order", {"20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010102", "20220101010102"}, 0), - toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})}); + ASSERT_COLUMNS_EQ_R(executeStreamsWithSingleSource(request, + {toNullableDatetimeVec("partition", {"20220101010102", "20220101010102", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010101", "20220101010101"}, 0), + toDatetimeVec("order", {"20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010102", "20220101010102"}, 0)}), + createColumns({toNullableDatetimeVec("partition", {"20220101010101", "20220101010101", "20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010102", "20220101010102"}, 0), + toNullableDatetimeVec("order", {"20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010102", "20220101010102"}, 0), + toNullableVec("row_number", {1, 2, 3, 4, 1, 2, 3, 4})})); // nullable - executeStreamsWithSingleSource( - request, - {toNullableDatetimeVec("partition", {"20220101010102", {}, "20220101010102", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010101", "20220101010101"}, 0), - toNullableDatetimeVec("order", {"20220101010101", {}, "20220101010101", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010102", "20220101010102"}, 0)}, - {toNullableDatetimeVec("partition", {{}, "20220101010101", "20220101010101", "20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010102", "20220101010102"}, 0), - toNullableDatetimeVec("order", {{}, "20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010102", "20220101010102"}, 0), - toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})}); + ASSERT_COLUMNS_EQ_R(executeStreamsWithSingleSource(request, + {toNullableDatetimeVec("partition", {"20220101010102", {}, "20220101010102", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010101", "20220101010101"}, 0), + toNullableDatetimeVec("order", {"20220101010101", {}, "20220101010101", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010102", "20220101010102"}, 0)}), + createColumns({toNullableDatetimeVec("partition", {{}, "20220101010101", "20220101010101", "20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010102", "20220101010102"}, 0), + toNullableDatetimeVec("order", {{}, "20220101010101", "20220101010101", "20220101010102", "20220101010102", "20220101010101", "20220101010101", "20220101010102", "20220101010102"}, 0), + toNullableVec("row_number", {1, 1, 2, 3, 4, 1, 2, 3, 4})})); // 2 partiton key and 2 order key // sql : select *, row_number() over w1 from test6 window w1 as (partition by partition_int1, partition_int2 order by order_int1,order_int2) @@ -162,41 +153,38 @@ try .window(RowNumber(), {{"order1", false}, {"order2", false}}, {{"partition1", false}, {"partition2", false}}, buildDefaultRowsFrame()) .build(context); - executeStreams( - request, - {toNullableVec("partition1", {1, 1, 1, 1, 1, 1, 2, 2, 2, 2, 2, 2}), - toNullableVec("partition2", {1, 1, 1, 2, 2, 2, 1, 1, 1, 2, 2, 2}), - toNullableVec("order1", {1, 1, 2, 1, 1, 2, 1, 1, 2, 1, 1, 2}), - toNullableVec("order2", {1, 2, 2, 1, 2, 2, 1, 2, 2, 1, 2, 2}), - toNullableVec("row_number", {1, 2, 3, 1, 2, 3, 1, 2, 3, 1, 2, 3})}); + ASSERT_COLUMNS_EQ_R(executeStreams(request), + createColumns({toNullableVec("partition1", {1, 1, 1, 1, 1, 1, 2, 2, 2, 2, 2, 2}), + toNullableVec("partition2", {1, 1, 1, 2, 2, 2, 1, 1, 1, 2, 2, 2}), + toNullableVec("order1", {1, 1, 2, 1, 1, 2, 1, 1, 2, 1, 1, 2}), + toNullableVec("order2", {1, 2, 2, 1, 2, 2, 1, 2, 2, 1, 2, 2}), + toNullableVec("row_number", {1, 2, 3, 1, 2, 3, 1, 2, 3, 1, 2, 3})})); /***** rank, dense_rank *****/ request = context.scan("test_db", "test_table_for_rank").sort({{"partition", false}, {"order", false}}, true).window({Rank(), DenseRank()}, {{"order", false}}, {{"partition", false}}, MockWindowFrame{}).build(context); - executeStreams( - request, - {toNullableVec("partition", {1, 1, 1, 1, 2, 2, 2, 2}), - toNullableVec("order", {1, 1, 2, 2, 1, 1, 2, 2}), - toNullableVec("rank", {1, 1, 3, 3, 1, 1, 3, 3}), - toNullableVec("dense_rank", {1, 1, 2, 2, 1, 1, 2, 2})}); + ASSERT_COLUMNS_EQ_R(executeStreams(request), + createColumns({toNullableVec("partition", {1, 1, 1, 1, 2, 2, 2, 2}), + toNullableVec("order", {1, 1, 2, 2, 1, 1, 2, 2}), + toNullableVec("rank", {1, 1, 3, 3, 1, 1, 3, 3}), + toNullableVec("dense_rank", {1, 1, 2, 2, 1, 1, 2, 2})})); // nullable - executeStreamsWithSingleSource( - request, - {toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), - toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2})}, - {toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), - toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2}), - toNullableVec("rank", {1, 1, 1, 3, 3, 1, 1, 3, 3}), - toNullableVec("dense_rank", {1, 1, 1, 2, 2, 1, 1, 2, 2})}); - - executeStreamsWithSingleSource( - request, - {toNullableVec("partition", {{}, {}, 1, 1, 1, 1, 2, 2, 2, 2}), - toNullableVec("order", {{}, 1, 1, 1, 2, 2, 1, 1, 2, 2})}, - {toNullableVec("partition", {{}, {}, 1, 1, 1, 1, 2, 2, 2, 2}), - toNullableVec("order", {{}, 1, 1, 1, 2, 2, 1, 1, 2, 2}), - toNullableVec("rank", {1, 2, 1, 1, 3, 3, 1, 1, 3, 3}), - toNullableVec("dense_rank", {1, 2, 1, 1, 2, 2, 1, 1, 2, 2})}); + ASSERT_COLUMNS_EQ_R(executeStreamsWithSingleSource(request, + {toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), + toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2})}), + createColumns({toNullableVec("partition", {{}, 1, 1, 1, 1, 2, 2, 2, 2}), + toNullableVec("order", {{}, 1, 1, 2, 2, 1, 1, 2, 2}), + toNullableVec("rank", {1, 1, 1, 3, 3, 1, 1, 3, 3}), + toNullableVec("dense_rank", {1, 1, 1, 2, 2, 1, 1, 2, 2})})); + + ASSERT_COLUMNS_EQ_R(executeStreamsWithSingleSource( + request, + {toNullableVec("partition", {{}, {}, 1, 1, 1, 1, 2, 2, 2, 2}), + toNullableVec("order", {{}, 1, 1, 1, 2, 2, 1, 1, 2, 2})}), + createColumns({toNullableVec("partition", {{}, {}, 1, 1, 1, 1, 2, 2, 2, 2}), + toNullableVec("order", {{}, 1, 1, 1, 2, 2, 1, 1, 2, 2}), + toNullableVec("rank", {1, 2, 1, 1, 3, 3, 1, 1, 3, 3}), + toNullableVec("dense_rank", {1, 2, 1, 1, 2, 2, 1, 1, 2, 2})})); } CATCH From fe2b539863e564acd0c0c74a556418a2cb85858f Mon Sep 17 00:00:00 2001 From: Lloyd-Pottiger <60744015+Lloyd-Pottiger@users.noreply.github.com> Date: Thu, 7 Jul 2022 11:29:02 +0800 Subject: [PATCH 081/104] feat: add logical cpu cores and memory into grafana (#5124) close pingcap/tiflash#3821 --- dbms/src/Common/CurrentMetrics.cpp | 2 ++ dbms/src/Server/Server.cpp | 8 ++++++ metrics/grafana/tiflash_summary.json | 37 ++++++++++++++++++++++++---- 3 files changed, 42 insertions(+), 5 deletions(-) diff --git a/dbms/src/Common/CurrentMetrics.cpp b/dbms/src/Common/CurrentMetrics.cpp index b7ce9fd1e89..8673784c590 100644 --- a/dbms/src/Common/CurrentMetrics.cpp +++ b/dbms/src/Common/CurrentMetrics.cpp @@ -22,6 +22,8 @@ M(OpenFileForReadWrite) \ M(MemoryTracking) \ M(MemoryTrackingInBackgroundProcessingPool) \ + M(LogicalCPUCores) \ + M(MemoryCapacity) \ M(PSMVCCNumSnapshots) \ M(PSMVCCSnapshotsList) \ M(RWLockWaitingReaders) \ diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 3358ae2a60e..a398aa9c74d 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -178,6 +178,12 @@ namespace } } // namespace +namespace CurrentMetrics +{ +extern const Metric LogicalCPUCores; +extern const Metric MemoryCapacity; +} // namespace CurrentMetrics + namespace DB { namespace ErrorCodes @@ -1427,6 +1433,8 @@ int Server::main(const std::vector & /*args*/) { // on ARM processors it can show only enabled at current moment cores + CurrentMetrics::set(CurrentMetrics::LogicalCPUCores, server_info.cpu_info.logical_cores); + CurrentMetrics::set(CurrentMetrics::MemoryCapacity, server_info.memory_info.capacity); LOG_FMT_INFO( log, "Available RAM = {}; physical cores = {}; logical cores = {}.", diff --git a/metrics/grafana/tiflash_summary.json b/metrics/grafana/tiflash_summary.json index 8e18357031d..0d72f950add 100644 --- a/metrics/grafana/tiflash_summary.json +++ b/metrics/grafana/tiflash_summary.json @@ -542,7 +542,14 @@ "pointradius": 5, "points": false, "renderer": "flot", - "seriesOverrides": [], + "seriesOverrides": [ + { + "alias": "/limit/", + "fill": 0, + "nullPointMode": "null", + "color": "#C4162A" + } + ], "spaceLength": 10, "stack": false, "steppedLine": false, @@ -633,6 +640,13 @@ "intervalFactor": 1, "legendFormat": "{{instance}}", "refId": "K" + }, + { + "expr": "sum(tiflash_system_current_metric_MemoryCapacity{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "legendFormat": "limit-{{instance}}", + "exemplar": true, + "refId": "L", + "hide": false } ], "thresholds": [], @@ -701,15 +715,15 @@ "hiddenSeries": false, "id": 51, "legend": { - "alignAsTable": false, + "alignAsTable": true, "avg": false, - "current": false, + "current": true, "max": false, "min": false, - "rightSide": false, + "rightSide": true, "show": true, "total": false, - "values": false + "values": true }, "lines": true, "linewidth": 1, @@ -728,6 +742,12 @@ "alias": "total", "fill": 0, "lines": false + }, + { + "alias": "/limit/", + "fill": 0, + "nullPointMode": "null", + "color": "#C4162A" } ], "spaceLength": 10, @@ -742,6 +762,13 @@ "legendFormat": "{{instance}}", "refId": "A", "step": 40 + }, + { + "expr": "sum(tiflash_system_current_metric_LogicalCPUCores{k8s_cluster=\"$k8s_cluster\", tidb_cluster=\"$tidb_cluster\", instance=~\"$instance\"}) by (instance)", + "legendFormat": "limit-{{instance}}", + "exemplar": true, + "refId": "B", + "intervalFactor": 1 } ], "thresholds": [], From d9b7086797036f3050d072a45057312edf492526 Mon Sep 17 00:00:00 2001 From: Iggie Wang Date: Thu, 7 Jul 2022 12:01:02 +0800 Subject: [PATCH 082/104] Implement TimeToSec function push down (#5235) close pingcap/tiflash#5116 --- dbms/src/Common/MyDuration.cpp | 2 +- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 2 +- dbms/src/Functions/FunctionsDuration.cpp | 69 ++++++++++++++++ dbms/src/Functions/FunctionsDuration.h | 19 +++++ .../tests/gtest_duration_pushdown.cpp | 80 +++++++++++++++++++ .../expr/duration_pushdown.test | 8 ++ 6 files changed, 178 insertions(+), 2 deletions(-) diff --git a/dbms/src/Common/MyDuration.cpp b/dbms/src/Common/MyDuration.cpp index 8801ae0de44..513c40b6dbc 100644 --- a/dbms/src/Common/MyDuration.cpp +++ b/dbms/src/Common/MyDuration.cpp @@ -67,4 +67,4 @@ String MyDuration::toString() const auto frac_str = fmt::format("{:06}", microsecond); return fmt::format(fmt_str, sign > 0 ? "" : "-", hour, minute, second, frac_str); } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 9ffa29cd14d..22758ad55cb 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -561,7 +561,7 @@ const std::unordered_map scalar_func_map({ {tipb::ScalarFuncSig::Quarter, "toQuarter"}, //{tipb::ScalarFuncSig::SecToTime, "cast"}, - //{tipb::ScalarFuncSig::TimeToSec, "cast"}, + {tipb::ScalarFuncSig::TimeToSec, "tidbTimeToSec"}, //{tipb::ScalarFuncSig::TimestampAdd, "cast"}, {tipb::ScalarFuncSig::ToDays, "tidbToDays"}, {tipb::ScalarFuncSig::ToSeconds, "tidbToSeconds"}, diff --git a/dbms/src/Functions/FunctionsDuration.cpp b/dbms/src/Functions/FunctionsDuration.cpp index ea7b86ac670..9ccafd2794d 100644 --- a/dbms/src/Functions/FunctionsDuration.cpp +++ b/dbms/src/Functions/FunctionsDuration.cpp @@ -97,6 +97,57 @@ void FunctionDurationSplit::executeImpl(Block & block, const ColumnNumbers ErrorCodes::ILLEGAL_COLUMN); }; +template +DataTypePtr FunctionMyDurationToSec::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const +{ + if (!arguments[0].type->isMyTime()) + { + throw Exception( + fmt::format("Illegal type {} of the first argument of function {}", arguments[0].type->getName(), getName()), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + return std::make_shared(); +} + +template +void FunctionMyDurationToSec::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const +{ + const auto * from_type = checkAndGetDataType(block.getByPosition(arguments[0]).type.get()); + if (from_type == nullptr) + { + throw Exception( + fmt::format( + "Illegal column {} of the first argument of function {}", + block.getByPosition(arguments[0]).column->getName(), + name), + ErrorCodes::ILLEGAL_COLUMN); + } + + using FromFieldType = typename DataTypeMyDuration::FieldType; + const auto * col_from = checkAndGetColumn>(block.getByPosition(arguments[0]).column.get()); + if (col_from != nullptr) + { + const typename ColumnVector::Container & vec_from = col_from->getData(); + const size_t size = vec_from.size(); + auto col_to = ColumnVector::create(size); + typename ColumnVector::Container & vec_to = col_to->getData(); + + for (size_t i = 0; i < size; ++i) + { + MyDuration val(vec_from[i], from_type->getFsp()); + vec_to[i] = Impl::apply(val); + } + block.getByPosition(result).column = std::move(col_to); + } + else + throw Exception( + fmt::format( + "Illegal column {} of the first argument of function {}", + block.getByPosition(arguments[0]).column->getName(), + name), + ErrorCodes::ILLEGAL_COLUMN); +} + struct DurationSplitHourImpl { static constexpr auto name = "hour"; @@ -133,11 +184,27 @@ struct DurationSplitMicroSecondImpl } }; +struct TiDBTimeToSecTransformerImpl +{ + static constexpr auto name = "tidbTimeToSec"; + static Int64 apply(const MyDuration & val) + { + Int64 sign = 1; + if (val.isNeg()) + { + sign = -1; + } + return sign * (val.hours() * 3600 + val.minutes() * 60 + val.seconds()); + } +}; + using FunctionDurationHour = FunctionDurationSplit; using FunctionDurationMinute = FunctionDurationSplit; using FunctionDurationSecond = FunctionDurationSplit; using FunctionDurationMicroSecond = FunctionDurationSplit; +using FunctionToTiDBTimeToSec = FunctionMyDurationToSec; + void registerFunctionsDuration(FunctionFactory & factory) { factory.registerFunction(); @@ -146,5 +213,7 @@ void registerFunctionsDuration(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + + factory.registerFunction(); } } // namespace DB diff --git a/dbms/src/Functions/FunctionsDuration.h b/dbms/src/Functions/FunctionsDuration.h index 4247cde03ff..5bc54d425f4 100644 --- a/dbms/src/Functions/FunctionsDuration.h +++ b/dbms/src/Functions/FunctionsDuration.h @@ -69,4 +69,23 @@ class FunctionDurationSplit : public IFunction void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override; }; +template +class FunctionMyDurationToSec : public IFunction +{ +public: + static constexpr auto name = Impl::name; + + static FunctionPtr create(const Context &) { return std::make_shared(); }; + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 1; } + + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override; +}; + } // namespace DB \ No newline at end of file diff --git a/dbms/src/Functions/tests/gtest_duration_pushdown.cpp b/dbms/src/Functions/tests/gtest_duration_pushdown.cpp index 4501a4c9fae..106f3d84642 100644 --- a/dbms/src/Functions/tests/gtest_duration_pushdown.cpp +++ b/dbms/src/Functions/tests/gtest_duration_pushdown.cpp @@ -166,5 +166,85 @@ try ASSERT_COLUMN_EQ(microSecond_out, executeFunction("microSecond", input4)); } CATCH + +TEST_F(DurationPushDown, timeToSecPushDownTest) +try +{ + ColumnWithTypeAndName input( + createColumn>({(838 * 3600 + 59 * 60 + 59) * 1000000000L + 999999000L, + -(838 * 3600 + 59 * 60 + 59) * 1000000000L - 123456000L, + 0, + (1 * 3600 + 2 * 60 + 3) * 1000000000L + 4000L}) + .column, + makeNullable(std::make_shared(6)), + "input"); + auto second_output = createColumn>({3020399, -3020399, 0, 3723}); + ASSERT_COLUMN_EQ(second_output, executeFunction("tidbTimeToSec", input)); + + // Test Overflow + ColumnWithTypeAndName input2( + createColumn>({(838 * 3600 + 59 * 60 + 59) * 1000000000L + 999999000L + 1000L}).column, + makeNullable(std::make_shared(6)), + "result"); + try + { + auto result = executeFunction("tidbTimeToSec", input2); + FAIL() << "Expected overflow"; + } + catch (DB::Exception & e) + { + ASSERT_EQ(e.message(), std::string("nanos must >= -3020399999999000 and <= 3020399999999000")); + } + catch (...) + { + FAIL() << "Expected overflow"; + }; + + ColumnWithTypeAndName input3( + createColumn>({-(838 * 3600 + 59 * 60 + 59) * 1000000000L - 999999000L - 1000L}).column, + makeNullable(std::make_shared(6)), + "result"); + try + { + auto result = executeFunction("tidbTimeToSec", input3); + FAIL() << "Expected overflow"; + } + catch (DB::Exception & e) + { + ASSERT_EQ(e.message(), std::string("nanos must >= -3020399999999000 and <= 3020399999999000")); + } + catch (...) + { + FAIL() << "Expected overflow"; + }; + + // Random Test + constexpr int rowNum = 1000; + auto dur_column = ColumnVector::create(); + auto & dur_data = dur_column->getData(); + auto second_column = ColumnVector::create(); + auto & second_data = second_column->getData(); + dur_data.resize(rowNum); + second_data.resize(rowNum); + + std::random_device rd; + std::default_random_engine gen = std::default_random_engine(rd()); + std::uniform_int_distribution sign_dis(0, 1), hour_dis(0, 838), minute_dis(0, 59), second_dis(0, 59), microSecond_dis(0, 999999); + for (int i = 0; i < rowNum; ++i) + { + auto sign = (sign_dis(gen) == 0) ? 1 : -1; + auto hour = hour_dis(gen); + auto minute = minute_dis(gen); + auto second = second_dis(gen); + auto microSecond = microSecond_dis(gen); + dur_data[i] = sign * ((hour * 3600 + minute * 60 + second) * 1000000000L + microSecond * 1000L); + second_data[i] = sign * (hour * 3600 + minute * 60 + second); + } + + ColumnWithTypeAndName input4(std::move(dur_column), std::make_shared(6), "duration"); + ColumnWithTypeAndName second_out(std::move(second_column), std::make_shared(), "time_to_sec"); + ASSERT_COLUMN_EQ(second_out, executeFunction("tidbTimeToSec", input4)); +} +CATCH } // namespace tests } // namespace DB \ No newline at end of file diff --git a/tests/fullstack-test/expr/duration_pushdown.test b/tests/fullstack-test/expr/duration_pushdown.test index 63106fa1788..442a708a802 100644 --- a/tests/fullstack-test/expr/duration_pushdown.test +++ b/tests/fullstack-test/expr/duration_pushdown.test @@ -106,6 +106,14 @@ mysql> use test; set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflas # | 123500 | # +----------------+ +mysql> use test; set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select time_to_sec(a) from t; ++----------------+ +| time_to_sec(a) | ++----------------+ +| 2520610 | +| -2520610 | ++----------------+ + mysql> drop table if exists test.time_test; mysql> create table test.time_test(id int(11),v1 time(3) not null, v2 time(3)); From cb69d5c1877ff346923f744e0c25112b94c6473b Mon Sep 17 00:00:00 2001 From: Jiarui Li <34512395+Willendless@users.noreply.github.com> Date: Thu, 7 Jul 2022 00:33:02 -0400 Subject: [PATCH 083/104] feat: implement shiftRight function push down (#5156) close pingcap/tiflash#5100 --- dbms/src/DataTypes/NumberTraits.h | 1 + .../DAGExpressionAnalyzerHelper.cpp | 1 + dbms/src/Flash/Coprocessor/DAGUtils.cpp | 2 +- dbms/src/Functions/bitShiftRight.cpp | 18 +- .../Functions/tests/gtest_bitshiftright.cpp | 273 ++++++++++++++++++ .../expr/bitshift_operator.test | 43 +++ 6 files changed, 335 insertions(+), 3 deletions(-) create mode 100644 dbms/src/Functions/tests/gtest_bitshiftright.cpp create mode 100644 tests/fullstack-test/expr/bitshift_operator.test diff --git a/dbms/src/DataTypes/NumberTraits.h b/dbms/src/DataTypes/NumberTraits.h index 925628a8894..a8b91b88075 100644 --- a/dbms/src/DataTypes/NumberTraits.h +++ b/dbms/src/DataTypes/NumberTraits.h @@ -277,6 +277,7 @@ struct ResultOfAbs> }; /** For bitwise operations, an integer is obtained with number of bits is equal to the maximum of the arguments. + * todo: note that MySQL handles only unsigned 64-bit integer argument and result values. We should refine the code. */ template struct ResultOfBit diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.cpp index ee529680d28..23bbb4586b3 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzerHelper.cpp @@ -450,6 +450,7 @@ DAGExpressionAnalyzerHelper::FunctionBuilderMap DAGExpressionAnalyzerHelper::fun {"bitOr", DAGExpressionAnalyzerHelper::buildBitwiseFunction}, {"bitXor", DAGExpressionAnalyzerHelper::buildBitwiseFunction}, {"bitNot", DAGExpressionAnalyzerHelper::buildBitwiseFunction}, + {"bitShiftRight", DAGExpressionAnalyzerHelper::buildBitwiseFunction}, {"leftUTF8", DAGExpressionAnalyzerHelper::buildLeftUTF8Function}, {"date_add", DAGExpressionAnalyzerHelper::buildDateAddOrSubFunction}, {"date_sub", DAGExpressionAnalyzerHelper::buildDateAddOrSubFunction}, diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 22758ad55cb..f0800eda4df 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -332,7 +332,7 @@ const std::unordered_map scalar_func_map({ {tipb::ScalarFuncSig::DecimalIsFalseWithNull, "isFalseWithNull"}, //{tipb::ScalarFuncSig::LeftShift, "cast"}, - //{tipb::ScalarFuncSig::RightShift, "cast"}, + {tipb::ScalarFuncSig::RightShift, "bitShiftRight"}, //{tipb::ScalarFuncSig::BitCount, "cast"}, //{tipb::ScalarFuncSig::GetParamString, "cast"}, diff --git a/dbms/src/Functions/bitShiftRight.cpp b/dbms/src/Functions/bitShiftRight.cpp index 961f7459f68..90b365771de 100644 --- a/dbms/src/Functions/bitShiftRight.cpp +++ b/dbms/src/Functions/bitShiftRight.cpp @@ -13,6 +13,9 @@ // limitations under the License. #include +#include + +#include namespace DB { @@ -29,7 +32,18 @@ struct BitShiftRightImpl template static Result apply(A a, B b) { - return static_cast(a) >> static_cast(b); + // It is an undefined behavior for shift operation in c++ that the right operand is negative or greater than + // or equal to the number of digits of the bits in the (promoted) left operand. + // See https://en.cppreference.com/w/cpp/language/operator_arithmetic for details. + if (static_cast(b) >= std::numeric_limits(a))>::digits) + { + return static_cast(0); + } + // Note that we do not consider the case that the right operand is negative, + // since other types will all be cast to uint64 before shift operation + // according to DAGExpressionAnalyzerHelper::buildBitwiseFunction. + // Therefore, we simply suppress clang-tidy checking here. + return static_cast(a) >> static_cast(b); // NOLINT(clang-analyzer-core.UndefinedBinaryOperatorResult) } template static Result apply(A, B, UInt8 &) @@ -87,4 +101,4 @@ void registerFunctionBitShiftRight(FunctionFactory & factory) factory.registerFunction(); } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Functions/tests/gtest_bitshiftright.cpp b/dbms/src/Functions/tests/gtest_bitshiftright.cpp new file mode 100644 index 00000000000..a4af6336099 --- /dev/null +++ b/dbms/src/Functions/tests/gtest_bitshiftright.cpp @@ -0,0 +1,273 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +namespace DB +{ +namespace tests +{ +class TestFunctionBitShiftRight : public DB::tests::FunctionTest +{ +}; + +#define ASSERT_BITSHIFTRIGHT(t1, t2, result) \ + ASSERT_COLUMN_EQ(result, executeFunction("bitShiftRight", {t1, t2})) + +TEST_F(TestFunctionBitShiftRight, Simple) +try +{ + ASSERT_BITSHIFTRIGHT(createColumn>({8}), + createColumn>({2}), + createColumn>({2})); +} +CATCH + +/// Note: Only IntX and UIntX will be received by BitShiftRight, others will be casted by TiDB planner. +/// Note: BitShiftRight will further cast other types to UInt64 before doing shift. +TEST_F(TestFunctionBitShiftRight, TypePromotion) +try +{ + // Type Promotion + ASSERT_BITSHIFTRIGHT(createColumn>({-1}), createColumn>({1}), createColumn>({9223372036854775807ull})); + ASSERT_BITSHIFTRIGHT(createColumn>({-1}), createColumn>({1}), createColumn>({9223372036854775807ull})); + ASSERT_BITSHIFTRIGHT(createColumn>({-1}), createColumn>({1}), createColumn>({9223372036854775807ull})); + ASSERT_BITSHIFTRIGHT(createColumn>({-1}), createColumn>({1}), createColumn>({9223372036854775807ull})); + + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn>({0}), createColumn>({1})); + + // Type Promotion across signed/unsigned + ASSERT_BITSHIFTRIGHT(createColumn>({-1}), createColumn>({0}), createColumn>({18446744073709551615ull})); + ASSERT_BITSHIFTRIGHT(createColumn>({-1}), createColumn>({0}), createColumn>({18446744073709551615ull})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn>({0}), createColumn>({1})); +} +CATCH + +TEST_F(TestFunctionBitShiftRight, Nullable) +try +{ + // Non Nullable + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn({0}), createColumn({1})); + + // Across Nullable and non-Nullable + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn({1}), createColumn>({0}), createColumn>({1})); + + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); + ASSERT_BITSHIFTRIGHT(createColumn>({1}), createColumn({0}), createColumn>({1})); +} +CATCH + +TEST_F(TestFunctionBitShiftRight, TypeCastWithConst) +try +{ + /// need test these kinds of columns: + /// 1. ColumnVector + /// 2. ColumnVector + /// 3. ColumnConst + /// 4. ColumnConst, value != null + /// 5. ColumnConst, value = null + + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, 1, 1}), createColumn({0, 1, 0, 1}), createColumn({0, 0, 1, 0})); + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, 1, 1}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 0, std::nullopt, std::nullopt})); + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, 1, 1}), createConstColumn(4, 0), createColumn({0, 0, 1, 1})); + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, 1, 1}), createConstColumn>(4, 0), createColumn({0, 0, 1, 1})); + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, 1, 1}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); // become const in wrapInNullable + + ASSERT_BITSHIFTRIGHT(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn({0, 1, 0, 1}), createColumn>({0, 0, std::nullopt, std::nullopt})); + ASSERT_BITSHIFTRIGHT(createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({0, 0, std::nullopt, std::nullopt})); + ASSERT_BITSHIFTRIGHT(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITSHIFTRIGHT(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn(4, 0), createColumn>({0, 1, std::nullopt, std::nullopt})); + ASSERT_BITSHIFTRIGHT(createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + + ASSERT_BITSHIFTRIGHT(createConstColumn(4, 1), createColumn({0, 1, 0, 1}), createColumn({1, 0, 1, 0})); + ASSERT_BITSHIFTRIGHT(createConstColumn(4, 1), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({1, 0, std::nullopt, std::nullopt})); + ASSERT_BITSHIFTRIGHT(createConstColumn(4, 1), createConstColumn(4, 0), createConstColumn(4, 1)); + ASSERT_BITSHIFTRIGHT(createConstColumn(4, 1), createConstColumn>(4, 0), createConstColumn(4, 1)); + ASSERT_BITSHIFTRIGHT(createConstColumn(4, 1), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, 1), createColumn({0, 1, 0, 1}), createColumn({1, 0, 1, 0})); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, 1), createColumn>({0, 1, std::nullopt, std::nullopt}), createColumn>({1, 0, std::nullopt, std::nullopt})); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, 1), createConstColumn(4, 0), createConstColumn(4, 1)); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, 1), createConstColumn>(4, 0), createConstColumn(4, 1)); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, 1), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); + + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, std::nullopt), createColumn({0, 1, 0, 1}), createConstColumn>(4, std::nullopt)); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, std::nullopt), createColumn>({0, 1, std::nullopt, std::nullopt}), createConstColumn>(4, std::nullopt)); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, std::nullopt), createConstColumn(4, 0), createConstColumn>(4, std::nullopt)); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, std::nullopt), createConstColumn(4, 0), createConstColumn>(4, std::nullopt)); + ASSERT_BITSHIFTRIGHT(createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt), createConstColumn>(4, std::nullopt)); +} +CATCH + +TEST_F(TestFunctionBitShiftRight, Boundary) +try +{ + ASSERT_BITSHIFTRIGHT(createColumn({127, 127, -128, -128}), createColumn({0, 7, 0, 7}), createColumn({127, 0, 18446744073709551488ull, 144115188075855871ull})); + ASSERT_BITSHIFTRIGHT(createColumn({127, 127, -128, -128}), createColumn({0, 7, 0, 7}), createColumn({127, 0, 18446744073709551488ull, 144115188075855871ull})); + ASSERT_BITSHIFTRIGHT(createColumn({32767, 32767, -32768, -32768}), createColumn({0, 15, 0, 15}), createColumn({32767, 0, 18446744073709518848ull, 562949953421311ull})); + + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, 1, 1, -1, -1, INT64_MAX, INT64_MAX, INT64_MIN, INT64_MIN}), + createColumn({0, 63, 0, 63, 0, 63, 0, 63, 0, 63}), + createColumn({0, 0, 1, 0, 18446744073709551615ull, 1, INT64_MAX, 0, 9223372036854775808ull, 1})); +} +CATCH + +TEST_F(TestFunctionBitShiftRight, UINT64) +try +{ + ASSERT_BITSHIFTRIGHT(createColumn({0, UINT64_MAX}), + createColumn({63, 63}), + createColumn({0, 1})); + + ASSERT_BITSHIFTRIGHT(createColumn>({0, UINT64_MAX, std::nullopt}), + createColumn>({63, 63, 63}), + createColumn>({0, 1, std::nullopt})); + + ASSERT_BITSHIFTRIGHT(createColumn>({0, UINT64_MAX, std::nullopt}), + createColumn({63, 63, 63}), + createColumn>({0, 1, std::nullopt})); + + ASSERT_BITSHIFTRIGHT(createColumn({0, UINT64_MAX}), + createColumn>({63, 63}), + createColumn>({0, 1})); + + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, 1, 1, -1, -1, INT64_MAX, INT64_MAX, INT64_MIN, INT64_MIN}), + createColumn({0, UINT64_MAX, 0, UINT64_MAX, 0, UINT64_MAX, 0, UINT64_MAX, 0, UINT64_MAX}), + createColumn({0, 0, 1, 0, 18446744073709551615ull, 0, INT64_MAX, 0, 9223372036854775808ull, 0})); + + + ASSERT_BITSHIFTRIGHT(createColumn({0, 0, UINT64_MAX, UINT64_MAX}), + createColumn({0, UINT64_MAX, 0, UINT64_MAX}), + createColumn({0, 0, UINT64_MAX, 0})); + + ASSERT_BITSHIFTRIGHT(createColumn>({0, 0, UINT64_MAX, UINT64_MAX, 0, std::nullopt}), + createColumn>({0, UINT64_MAX, 0, UINT64_MAX, std::nullopt, 0}), + createColumn>({0, 0, UINT64_MAX, 0, std::nullopt, std::nullopt})); + + ASSERT_BITSHIFTRIGHT(createColumn>({0, 0, UINT64_MAX, UINT64_MAX, std::nullopt}), + createColumn({0, UINT64_MAX, 0, UINT64_MAX, 0}), + createColumn>({0, 0, UINT64_MAX, 0, std::nullopt})); + + ASSERT_BITSHIFTRIGHT(createColumn({0, UINT64_MAX, 0, UINT64_MAX, 0}), + createColumn>({0, 0, UINT64_MAX, UINT64_MAX, std::nullopt}), + createColumn>({0, UINT64_MAX, 0, 0, std::nullopt})); + + /* + std::mt19937 gen(std::random_device{}()); + std::uniform_int_distribution dis( + std::numeric_limits::min(), + std::numeric_limits::max() + ); + size_t count = 100; + std::vector v1(count), v2(count), res(count); + for (size_t i=0; i> v2[i]; + } + */ + // clang-format off + ASSERT_BITSHIFTRIGHT(createColumn({4286230172992429668ull,11550684080080434735ull,775195682263841867ull,18390588538388462661ull,15578761645824658314ull,20662948907547635ull,8403266546632871011ull,10316916867086714284ull,14494183568060929367ull,11741337603037632348ull,10803264694948981380ull,2181969932373516503ull,9673801579564730047ull,12998855911221966916ull,13852157931865274857ull,9203926828777338586ull,8903261359104369984ull,3296258311466476456ull,14658801806079697908ull,7542518003247963618ull,7751150277360944372ull,12225694156629117269ull,3173837214287201256ull,10555082060194839563ull,14202570947308501213ull,13841194359225980123ull,9085267378073816945ull,15975493157631073381ull,1890233386459299033ull,2368634323417847398ull,691423931511513606ull,986000479038857169ull,6676906740954304741ull,2841686799872009560ull,6483676442160212821ull,12550114481083571140ull,1973026146580965947ull,15006687639313690830ull,6443617813685195609ull,13648732879238232658ull,173820604016606515ull,2669428687588070677ull,15361476519767969236ull,8957522718906827285ull,10484385204137290737ull,12390466571993898199ull,13655746682011856065ull,4183302523705398003ull,9898692767945122925ull,16701902679050716746ull,15003324714492513897ull,15554724240808081962ull,7754458312088240871ull,16060968032680196798ull,12619581440986221928ull,15462661961676206824ull,2991773628650321635ull,16341599119345297909ull,14943939970889580769ull,17589764776976679210ull,15274914527536421890ull,16268454608136611433ull,14617646699124891378ull,466927094873143934ull,10558583305251737283ull,255559140356160501ull,5962789691899784330ull,8004603198837555992ull,1881892337023478820ull,6549167700870881840ull,17551996157828573642ull,3349744237253314638ull,2876698686583880568ull,16792783373922568330ull,16231348759981899800ull,17731631990557975899ull,1305376485657663531ull,3568754485566225727ull,10076204423028931225ull,1206238310176455071ull,4297062324543635867ull,5116785256928623516ull,4216305034157620433ull,412817651268481791ull,11256299741838589766ull,10786197076871163667ull,8588357635228913652ull,6361409982074778071ull,4750871994764527580ull,12851835128796581697ull,13871712051825681122ull,12445309465661589227ull,1668617678034382020ull,10152918068481134781ull,16242941973571224246ull,12988338226657152812ull,2352083670492692674ull,10735026236980245779ull,14986388012066843516ull,17651064432466444102ull}), + createColumn({0,58,55,24,5,35,34,54,43,45,17,36,51,54,19,55,55,8,37,49,15,11,36,0,5,41,46,54,2,59,11,25,43,29,31,8,59,2,11,19,56,35,57,13,2,35,6,54,17,0,49,5,15,3,60,44,16,6,57,44,58,54,26,23,58,23,26,29,56,40,45,2,21,9,57,40,4,46,17,15,62,21,5,54,22,47,10,24,53,61,43,52,23,10,61,43,26,31,38,2}), + createColumn({4286230172992429668ull,40,21,1096164497041ull,486836301432020572ull,601370,489134489,572,1647797,333708,82422368583289ull,31751841,4296,721,26420894492846ull,255,247,12876009029165923ull,106656820,13398,236546334147978ull,5969577224916561ull,46185410,10555082060194839563ull,443830342103390662ull,6294246,129109,886,472558346614824758ull,4,337609341558356ull,29385104150ull,759076,5293054133ull,3019197118ull,49023884691732699ull,3,3751671909828422707ull,3146297760588474ull,26032891996838ull,2,77690599,106,1093447597522806ull,2621096301034322684ull,360610038,213371041906435251ull,232,75521032470284ull,16701902679050716746ull,26651,486085132525252561ull,236647287356208ull,2007621004085024599ull,10,878950,45650842722325ull,255337486239770279ull,103,999862,52,903,217819909738ull,55662047251ull,36,30465023560ull,88852490364ull,14909735319ull,26,5956433,498857,837436059313328659ull,1371716826717ull,32798405027192516ull,112,16126825,81586030353603970ull,50715,76875338920813ull,36811471868177ull,0,2439873341049ull,131759532317425638ull,22,2683710990390ull,76640,8387068003153235ull,379169582252ull,527,5,1577031,2763,198914727930ull,9914959051251108ull,7,1476603,35048777915ull,4998886136ull,54520161,4412766108116611025ull})); + // clang-format on +} +CATCH + +TEST_F(TestFunctionBitShiftRight, UB) +try +{ + ASSERT_BITSHIFTRIGHT(createColumn({127, -128}), createColumn({64, 64}), createColumn({0, 0})); + ASSERT_BITSHIFTRIGHT(createColumn({127, -128}), createColumn({64, 64}), createColumn({0, 0})); + ASSERT_BITSHIFTRIGHT(createColumn({32767, -32768}), createColumn({64, 64}), createColumn({0, 0})); + ASSERT_BITSHIFTRIGHT(createColumn({INT32_MAX, INT32_MIN}), createColumn({64, 64}), createColumn({0, 0})); + ASSERT_BITSHIFTRIGHT(createColumn({INT64_MAX, INT64_MIN}), createColumn({64, 64}), createColumn({0, 0})); + + ASSERT_BITSHIFTRIGHT(createColumn({255}), createColumn({64}), createColumn({0})); + ASSERT_BITSHIFTRIGHT(createColumn({255}), createColumn({64}), createColumn({0})); + ASSERT_BITSHIFTRIGHT(createColumn({65535}), createColumn({64}), createColumn({0})); + ASSERT_BITSHIFTRIGHT(createColumn({UINT32_MAX}), createColumn({64}), createColumn({0})); + ASSERT_BITSHIFTRIGHT(createColumn({UINT64_MAX}), createColumn({64}), createColumn({0})); + + /* + std::mt19937 gen(std::random_device{}()); + std::uniform_int_distribution dis1( + std::numeric_limits::min(), + std::numeric_limits::max() + ); + std::uniform_int_distribution dis2( + 64, + std::numeric_limits::max() + ); + size_t count = 100; + std::vector v1(count), v2(count), res(count); + for (size_t i=0; i({17563387625296433369ull,5842891814427459261ull,15074502074821508463ull,386435802999553003ull,5487893274931198395ull,8125923807366590570ull,13340330062727071249ull,14908193031091561411ull,296805448857369387ull,8684453485792353774ull,13117933444495098288ull,3225762988982100714ull,11290506757949810556ull,14617912756126856962ull,9479575714707174581ull,11720728318194739598ull,14410575429605211363ull,12068356718035872518ull,80682389916710599ull,11003236134534292734ull,4412447398096224810ull,5331184707993902906ull,13827083432789678788ull,958142831027309576ull,16716461997317184701ull,17128750834581527743ull,11590434571174666313ull,10204342520615148287ull,11067791415848657283ull,17583875436196878829ull,186304014359496415ull,9381729025189804702ull,11502205568225715300ull,16472133582690439104ull,3743303387826342067ull,12860029445868505658ull,2244056593742923769ull,3275687468466891223ull,1545828456957460699ull,14187252460708728077ull,7551907967738536187ull,9754400233340010491ull,16293183350230169116ull,6298812696728711031ull,5915538565572009956ull,2284684518775825662ull,1130711226902262476ull,17158957721471765323ull,4220824385439711070ull,16559772875254313109ull,15397179690017513678ull,6300413832999049491ull,13787530251307637715ull,10132349060092695582ull,10446586881482901699ull,15759779838283537085ull,14402587207027333363ull,5546051719872960161ull,6545031029710296628ull,17407295406267098658ull,4259019625544816073ull,791895457880289787ull,8549227257401578066ull,15246278171168501125ull,1674668228908076954ull,849762797502000057ull,13302651500925764574ull,12438174880334092333ull,17701249772557033303ull,10742459186038873636ull,15671491258945407856ull,9352557101631889001ull,8914093883925002585ull,17935292744735591949ull,606989231583658922ull,6528503454270721815ull,14980539549624989095ull,13765196438235456668ull,3058323869228644592ull,14346577759191739044ull,1543206286382906519ull,1025562312317433790ull,17052896445025268012ull,18349597294988935754ull,17174604730104962524ull,11924965352621110201ull,502032511104181724ull,13845633389643139332ull,15436039204445155412ull,17809579006694175565ull,15166364145138562881ull,14062748599121933798ull,1777457178576774356ull,4985224560472716170ull,3881603168175384251ull,11555031280550342082ull,1252677486917153396ull,8744807353133366467ull,2048964426549800495ull,11945831330508218140ull}), + createColumn({7570379165150948640ull,2086259313016069849ull,3606689596671293211ull,14039117280692395662ull,13678665403528829741ull,16069000531561010558ull,18229345530821449414ull,433464578739092378ull,6298872104011095934ull,4518228872693063137ull,14988726875963869472ull,9568218424260764817ull,5383191468426384555ull,8698762658876708752ull,9487599666567205013ull,14370091126330876161ull,10702068376663045773ull,8045701071228357739ull,10878469353312437370ull,3183167829827610494ull,5928881618833110378ull,10410530709181481816ull,249988564503361262ull,13482614555530280987ull,5522946068620734806ull,12797173590813112894ull,14133419908717831141ull,10825732602137508628ull,13271177233899692778ull,1157753039017783757ull,3370600557036147696ull,2957689395775524062ull,11963898745206689513ull,4828931188614542720ull,15157289330857160797ull,369467010700905309ull,6278071805692607460ull,17817858137511910604ull,17789013631125929528ull,2861684947245777353ull,2583152408663154190ull,7935135702156687355ull,3033127046167579202ull,14224256960933395097ull,10838403249753694181ull,2154089102842257532ull,7860358918492191001ull,2982010253383852617ull,16385171982396620123ull,12241857497176342828ull,2080931105225959532ull,1046322072991155713ull,6146917059052005252ull,17411786298437646544ull,5497869583209795613ull,11701448129764809247ull,12642962700918363620ull,15936842187305218463ull,7811510447588439153ull,3558405966224377785ull,977960926168429540ull,9505800334935014018ull,12114068456102275321ull,5141880021314950000ull,6719615890604904521ull,1341445859098821585ull,3883912906202435997ull,2107770591867486616ull,2657186337437393032ull,2640917573672927653ull,3746140861437224253ull,15057648507099656234ull,12051189681068107042ull,2259769676757597701ull,2935229535510718769ull,6368233316971463582ull,14384644474340782197ull,2553547617837260603ull,14238122466576902747ull,9555765226032904481ull,15522640015319979866ull,10274396157562093026ull,5996101113505388770ull,16915812546351047056ull,4956089714130804219ull,17126605744801075545ull,12036643325202409080ull,11257234688654558199ull,375338337104024778ull,11152980243617851986ull,12325805905403174063ull,8653948654121626815ull,15348912598299408338ull,6883296938248095081ull,6484642948886870833ull,16936141613107270500ull,17012171815528507292ull,2574129622316042070ull,17178726110735453748ull,16578303277501346489ull}), + createColumn({0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0})); + // clang-format on +} +CATCH + +} // namespace tests +} // namespace DB diff --git a/tests/fullstack-test/expr/bitshift_operator.test b/tests/fullstack-test/expr/bitshift_operator.test new file mode 100644 index 00000000000..0d55a1b56a9 --- /dev/null +++ b/tests/fullstack-test/expr/bitshift_operator.test @@ -0,0 +1,43 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +mysql> drop table if exists test.t; +mysql> create table test.t (a int); +mysql> alter table test.t set tiflash replica 1; +mysql> insert into test.t values(-1); + +func> wait_table test t + +mysql> set tidb_enforce_mpp=1; set @@session.tidb_isolation_read_engines = "tiflash"; select a>>0 as v1, a>>64 as v2, a>>10 as v3 from test.t; ++----------------------+------+-------------------+ +| v1 | v2 | v3 | ++----------------------+------+-------------------+ +| 18446744073709551615 | 0 | 18014398509481983 | ++----------------------+------+-------------------+ + +mysql> set tidb_enforce_mpp=1; set @@session.tidb_isolation_read_engines = "tiflash"; select a from test.t where a>>100000=0; ++------+ +| a | ++------+ +| -1 | ++------+ + +mysql> set tidb_enforce_mpp=1; set @@session.tidb_isolation_read_engines = "tiflash"; select a from test.t where a>>63=1; ++------+ +| a | ++------+ +| -1 | ++------+ + +mysql> drop table if exists test.t From 597f8b8a779f169c62f7c8031ce30615f178dc66 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Thu, 7 Jul 2022 15:23:02 +0800 Subject: [PATCH 084/104] schema : make update to partition tables when 'set tiflash replica' (#5267) close pingcap/tiflash#5266 --- dbms/src/Debug/DBGInvoker.cpp | 2 + dbms/src/Debug/dbgFuncSchema.cpp | 2 + dbms/src/Debug/dbgFuncSchema.h | 1 - dbms/src/Debug/dbgFuncSchemaName.cpp | 53 +++++++++++++ dbms/src/Debug/dbgFuncSchemaName.h | 10 +++ dbms/src/TiDB/Schema/SchemaBuilder.cpp | 74 ++++++++++++------- dbms/src/TiDB/Schema/SchemaBuilder.h | 31 ++++---- .../ddl/alter_table_tiflash_replica.test | 55 ++++++++++++++ 8 files changed, 185 insertions(+), 43 deletions(-) create mode 100644 tests/fullstack-test2/ddl/alter_table_tiflash_replica.test diff --git a/dbms/src/Debug/DBGInvoker.cpp b/dbms/src/Debug/DBGInvoker.cpp index 3f633c08e67..3654a437bf7 100644 --- a/dbms/src/Debug/DBGInvoker.cpp +++ b/dbms/src/Debug/DBGInvoker.cpp @@ -118,6 +118,8 @@ DBGInvoker::DBGInvoker() regSchemalessFunc("mapped_database", dbgFuncMappedDatabase); regSchemalessFunc("mapped_table", dbgFuncMappedTable); regSchemafulFunc("query_mapped", dbgFuncQueryMapped); + regSchemalessFunc("get_tiflash_replica_count", dbgFuncGetTiflashReplicaCount); + regSchemalessFunc("get_partition_tables_tiflash_replica_count", dbgFuncGetPartitionTablesTiflashReplicaCount); regSchemalessFunc("search_log_for_key", dbgFuncSearchLogForKey); regSchemalessFunc("tidb_dag", dbgFuncTiDBQueryFromNaturalDag); diff --git a/dbms/src/Debug/dbgFuncSchema.cpp b/dbms/src/Debug/dbgFuncSchema.cpp index c388015dc10..9ef07f16e8b 100644 --- a/dbms/src/Debug/dbgFuncSchema.cpp +++ b/dbms/src/Debug/dbgFuncSchema.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -137,4 +138,5 @@ void dbgFuncIsTombstone(Context & context, const ASTs & args, DBGInvoker::Printe output(fmt_buf.toString()); } + } // namespace DB \ No newline at end of file diff --git a/dbms/src/Debug/dbgFuncSchema.h b/dbms/src/Debug/dbgFuncSchema.h index 162bc0af46b..51ab3ad41cf 100644 --- a/dbms/src/Debug/dbgFuncSchema.h +++ b/dbms/src/Debug/dbgFuncSchema.h @@ -46,5 +46,4 @@ void dbgFuncResetSchemas(Context & context, const ASTs & args, DBGInvoker::Print // Usage: // ./storage-client.sh "DBGInvoke is_tombstone(db_name, table_name)" void dbgFuncIsTombstone(Context & context, const ASTs & args, DBGInvoker::Printer output); - } // namespace DB diff --git a/dbms/src/Debug/dbgFuncSchemaName.cpp b/dbms/src/Debug/dbgFuncSchemaName.cpp index 4c2ad86bd62..5f10da6685d 100644 --- a/dbms/src/Debug/dbgFuncSchemaName.cpp +++ b/dbms/src/Debug/dbgFuncSchemaName.cpp @@ -128,4 +128,57 @@ BlockInputStreamPtr dbgFuncQueryMapped(Context & context, const ASTs & args) return executeQuery(query, context, true).in; } + +void dbgFuncGetTiflashReplicaCount(Context & context, const ASTs & args, DBGInvoker::Printer output) +{ + if (args.empty() || args.size() != 2) + throw Exception("Args not matched, should be: database-name[, table-name]", ErrorCodes::BAD_ARGUMENTS); + + const String & database_name = typeid_cast(*args[0]).name; + FmtBuffer fmt_buf; + + const String & table_name = typeid_cast(*args[1]).name; + auto mapped = mappedTable(context, database_name, table_name); + auto storage = context.getTable(mapped->first, mapped->second); + auto managed_storage = std::dynamic_pointer_cast(storage); + if (!managed_storage) + throw Exception(database_name + "." + table_name + " is not ManageableStorage", ErrorCodes::BAD_ARGUMENTS); + + fmt_buf.append((std::to_string(managed_storage->getTableInfo().replica_info.count))); + + output(fmt_buf.toString()); +} + +void dbgFuncGetPartitionTablesTiflashReplicaCount(Context & context, const ASTs & args, DBGInvoker::Printer output) +{ + if (args.empty() || args.size() != 2) + throw Exception("Args not matched, should be: database-name[, table-name]", ErrorCodes::BAD_ARGUMENTS); + + const String & database_name = typeid_cast(*args[0]).name; + FmtBuffer fmt_buf; + + const String & table_name = typeid_cast(*args[1]).name; + auto mapped = mappedTable(context, database_name, table_name); + auto storage = context.getTable(mapped->first, mapped->second); + auto managed_storage = std::dynamic_pointer_cast(storage); + if (!managed_storage) + throw Exception(database_name + "." + table_name + " is not ManageableStorage", ErrorCodes::BAD_ARGUMENTS); + + auto table_info = managed_storage->getTableInfo(); + + if (!table_info.isLogicalPartitionTable()) + throw Exception(database_name + "." + table_name + " is not logical partition table", ErrorCodes::BAD_ARGUMENTS); + + SchemaNameMapper name_mapper; + for (const auto & part_def : table_info.partition.definitions) + { + auto paritition_table_info = table_info.producePartitionTableInfo(part_def.id, name_mapper); + auto partition_storage = context.getTMTContext().getStorages().get(paritition_table_info->id); + fmt_buf.append((std::to_string(partition_storage->getTableInfo().replica_info.count))); + fmt_buf.append("/"); + } + + output(fmt_buf.toString()); +} + } // namespace DB diff --git a/dbms/src/Debug/dbgFuncSchemaName.h b/dbms/src/Debug/dbgFuncSchemaName.h index 8e95aaab908..2a31b3a7f6d 100644 --- a/dbms/src/Debug/dbgFuncSchemaName.h +++ b/dbms/src/Debug/dbgFuncSchemaName.h @@ -40,4 +40,14 @@ void dbgFuncMappedTable(Context & context, const ASTs & args, DBGInvoker::Printe // ./storage-client.sh "DBGInvoke query_mapped('select * from $d.$t', database_name[, table_name])" BlockInputStreamPtr dbgFuncQueryMapped(Context & context, const ASTs & args); +// Get table's tiflash replica counts with mapped table name +// Usage: +// ./storage-client.sh "DBGInvoke get_tiflash_replica_count(db_name, table_name)" +void dbgFuncGetTiflashReplicaCount(Context & context, const ASTs & args, DBGInvoker::Printer output); + +// Get the logical table's partition tables' tiflash replica counts with mapped table name +// Usage: +// ./storage-client.sh "DBGInvoke get_partition_tables_tiflash_replica_count(db_name, table_name)" +void dbgFuncGetPartitionTablesTiflashReplicaCount(Context & context, const ASTs & args, DBGInvoker::Printer output); + } // namespace DB diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.cpp b/dbms/src/TiDB/Schema/SchemaBuilder.cpp index ae78923fc61..abb575c75bc 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.cpp +++ b/dbms/src/TiDB/Schema/SchemaBuilder.cpp @@ -320,7 +320,7 @@ inline SchemaChanges detectSchemaChanges( } template -void SchemaBuilder::applyAlterPhysicalTable(DBInfoPtr db_info, TableInfoPtr table_info, ManageableStoragePtr storage) +void SchemaBuilder::applyAlterPhysicalTable(const DBInfoPtr & db_info, const TableInfoPtr & table_info, const ManageableStoragePtr & storage) { LOG_FMT_INFO(log, "Altering table {}", name_mapper.debugCanonicalName(*db_info, *table_info)); @@ -394,7 +394,7 @@ void SchemaBuilder::applyAlterPhysicalTable(DBInfoPtr db_inf } template -void SchemaBuilder::applyAlterTable(DBInfoPtr db_info, TableID table_id) +void SchemaBuilder::applyAlterTable(const DBInfoPtr & db_info, TableID table_id) { auto table_info = getter.getTableInfo(db_info->id, table_id); if (table_info == nullptr) @@ -413,7 +413,7 @@ void SchemaBuilder::applyAlterTable(DBInfoPtr db_info, Table } template -void SchemaBuilder::applyAlterLogicalTable(DBInfoPtr db_info, TableInfoPtr table_info, ManageableStoragePtr storage) +void SchemaBuilder::applyAlterLogicalTable(const DBInfoPtr & db_info, const TableInfoPtr & table_info, const ManageableStoragePtr & storage) { // Alter logical table first. applyAlterPhysicalTable(db_info, table_info, storage); @@ -569,7 +569,7 @@ void SchemaBuilder::applyDiff(const SchemaDiff & diff) } template -void SchemaBuilder::applyPartitionDiff(TiDB::DBInfoPtr db_info, TableID table_id) +void SchemaBuilder::applyPartitionDiff(const TiDB::DBInfoPtr & db_info, TableID table_id) { auto table_info = getter.getTableInfo(db_info->id, table_id); if (table_info == nullptr) @@ -593,7 +593,7 @@ void SchemaBuilder::applyPartitionDiff(TiDB::DBInfoPtr db_in } template -void SchemaBuilder::applyPartitionDiff(TiDB::DBInfoPtr db_info, TableInfoPtr table_info, ManageableStoragePtr storage) +void SchemaBuilder::applyPartitionDiff(const TiDB::DBInfoPtr & db_info, const TableInfoPtr & table_info, const ManageableStoragePtr & storage) { const auto & orig_table_info = storage->getTableInfo(); if (!orig_table_info.isLogicalPartitionTable()) @@ -659,7 +659,7 @@ void SchemaBuilder::applyPartitionDiff(TiDB::DBInfoPtr db_in } template -void SchemaBuilder::applyRenameTable(DBInfoPtr new_db_info, TableID table_id) +void SchemaBuilder::applyRenameTable(const DBInfoPtr & new_db_info, TableID table_id) { auto new_table_info = getter.getTableInfo(new_db_info->id, table_id); if (new_table_info == nullptr) @@ -679,9 +679,9 @@ void SchemaBuilder::applyRenameTable(DBInfoPtr new_db_info, template void SchemaBuilder::applyRenameLogicalTable( - DBInfoPtr new_db_info, - TableInfoPtr new_table_info, - ManageableStoragePtr storage) + const DBInfoPtr & new_db_info, + const TableInfoPtr & new_table_info, + const ManageableStoragePtr & storage) { applyRenamePhysicalTable(new_db_info, *new_table_info, storage); @@ -703,9 +703,9 @@ void SchemaBuilder::applyRenameLogicalTable( template void SchemaBuilder::applyRenamePhysicalTable( - DBInfoPtr new_db_info, - TableInfo & new_table_info, - ManageableStoragePtr storage) + const DBInfoPtr & new_db_info, + const TableInfo & new_table_info, + const ManageableStoragePtr & storage) { const auto old_mapped_db_name = storage->getDatabaseName(); const auto new_mapped_db_name = name_mapper.mapDatabaseName(*new_db_info); @@ -908,7 +908,7 @@ String createDatabaseStmt(Context & context, const DBInfo & db_info, const Schem } template -void SchemaBuilder::applyCreateSchema(TiDB::DBInfoPtr db_info) +void SchemaBuilder::applyCreateSchema(const TiDB::DBInfoPtr & db_info) { GET_METRIC(tiflash_schema_internal_ddl_count, type_create_db).Increment(); LOG_FMT_INFO(log, "Creating database {}", name_mapper.debugDatabaseName(*db_info)); @@ -1047,7 +1047,7 @@ String createTableStmt( } template -void SchemaBuilder::applyCreatePhysicalTable(DBInfoPtr db_info, TableInfoPtr table_info) +void SchemaBuilder::applyCreatePhysicalTable(const DBInfoPtr & db_info, const TableInfoPtr & table_info) { GET_METRIC(tiflash_schema_internal_ddl_count, type_create_table).Increment(); LOG_FMT_INFO(log, "Creating table {}", name_mapper.debugCanonicalName(*db_info, *table_info)); @@ -1109,7 +1109,7 @@ void SchemaBuilder::applyCreatePhysicalTable(DBInfoPtr db_in } template -void SchemaBuilder::applyCreateTable(TiDB::DBInfoPtr db_info, TableID table_id) +void SchemaBuilder::applyCreateTable(const TiDB::DBInfoPtr & db_info, TableID table_id) { auto table_info = getter.getTableInfo(db_info->id, table_id); if (table_info == nullptr) @@ -1123,7 +1123,7 @@ void SchemaBuilder::applyCreateTable(TiDB::DBInfoPtr db_info } template -void SchemaBuilder::applyCreateLogicalTable(TiDB::DBInfoPtr db_info, TableInfoPtr table_info) +void SchemaBuilder::applyCreateLogicalTable(const TiDB::DBInfoPtr & db_info, const TableInfoPtr & table_info) { if (table_info->isLogicalPartitionTable()) { @@ -1169,7 +1169,7 @@ void SchemaBuilder::applyDropPhysicalTable(const String & db } template -void SchemaBuilder::applyDropTable(DBInfoPtr db_info, TableID table_id) +void SchemaBuilder::applyDropTable(const DBInfoPtr & db_info, TableID table_id) { auto & tmt_context = context.getTMTContext(); auto * storage = tmt_context.getStorages().get(table_id).get(); @@ -1193,13 +1193,14 @@ void SchemaBuilder::applyDropTable(DBInfoPtr db_info, TableI } template -void SchemaBuilder::applySetTiFlashReplica(TiDB::DBInfoPtr db_info, TableID table_id) +void SchemaBuilder::applySetTiFlashReplica(const TiDB::DBInfoPtr & db_info, TableID table_id) { auto latest_table_info = getter.getTableInfo(db_info->id, table_id); if (unlikely(latest_table_info == nullptr)) { throw TiFlashException(fmt::format("miss table in TiKV : {}", table_id), Errors::DDL::StaleSchema); } + auto & tmt_context = context.getTMTContext(); auto storage = tmt_context.getStorages().get(latest_table_info->id); if (unlikely(storage == nullptr)) @@ -1208,18 +1209,37 @@ void SchemaBuilder::applySetTiFlashReplica(TiDB::DBInfoPtr d Errors::DDL::MissingTable); } - auto managed_storage = std::dynamic_pointer_cast(storage); - if (unlikely(!managed_storage)) - throw Exception(fmt::format("{} is not a ManageableStorage", name_mapper.debugCanonicalName(*db_info, *latest_table_info))); + applySetTiFlashReplicaOnLogicalTable(db_info, latest_table_info, storage); +} + +template +void SchemaBuilder::applySetTiFlashReplicaOnLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage) +{ + applySetTiFlashReplicaOnPhysicalTable(db_info, table_info, storage); + + if (table_info->isLogicalPartitionTable()) + { + auto & tmt_context = context.getTMTContext(); - applySetTiFlashReplica(db_info, latest_table_info, managed_storage); + for (const auto & part_def : table_info->partition.definitions) + { + auto new_part_table_info = table_info->producePartitionTableInfo(part_def.id, name_mapper); + auto part_storage = tmt_context.getStorages().get(new_part_table_info->id); + if (unlikely(part_storage == nullptr)) + { + throw TiFlashException(fmt::format("miss table in TiFlash : {}", name_mapper.debugCanonicalName(*db_info, *new_part_table_info)), + Errors::DDL::MissingTable); + } + applySetTiFlashReplicaOnPhysicalTable(db_info, new_part_table_info, part_storage); + } + } } template -void SchemaBuilder::applySetTiFlashReplica( - TiDB::DBInfoPtr db_info, - TiDB::TableInfoPtr latest_table_info, - ManageableStoragePtr storage) +void SchemaBuilder::applySetTiFlashReplicaOnPhysicalTable( + const TiDB::DBInfoPtr & db_info, + const TiDB::TableInfoPtr & latest_table_info, + const ManageableStoragePtr & storage) { if (storage->getTableInfo().replica_info.count == latest_table_info->replica_info.count) return; @@ -1306,7 +1326,7 @@ void SchemaBuilder::syncAllSchema() /// Rename if needed. applyRenameLogicalTable(db, table, storage); /// Update replica info if needed. - applySetTiFlashReplica(db, table, storage); + applySetTiFlashReplicaOnLogicalTable(db, table, storage); /// Alter if needed. applyAlterLogicalTable(db, table, storage); LOG_FMT_DEBUG(log, "Table {} synced during sync all schemas", name_mapper.debugCanonicalName(*db, *table)); diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.h b/dbms/src/TiDB/Schema/SchemaBuilder.h index 8446765f74a..461d7ff9c12 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.h +++ b/dbms/src/TiDB/Schema/SchemaBuilder.h @@ -55,39 +55,40 @@ struct SchemaBuilder bool applyCreateSchema(DatabaseID schema_id); - void applyCreateSchema(TiDB::DBInfoPtr db_info); + void applyCreateSchema(const TiDB::DBInfoPtr & db_info); - void applyCreateTable(TiDB::DBInfoPtr db_info, TableID table_id); + void applyCreateTable(const TiDB::DBInfoPtr & db_info, TableID table_id); - void applyCreateLogicalTable(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info); + void applyCreateLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info); - void applyCreatePhysicalTable(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info); + void applyCreatePhysicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info); - void applyDropTable(TiDB::DBInfoPtr db_info, TableID table_id); + void applyDropTable(const TiDB::DBInfoPtr & db_info, TableID table_id); /// Parameter schema_name should be mapped. void applyDropPhysicalTable(const String & db_name, TableID table_id); - void applyPartitionDiff(TiDB::DBInfoPtr db_info, TableID table_id); + void applyPartitionDiff(const TiDB::DBInfoPtr & db_info, TableID table_id); - void applyPartitionDiff(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info, ManageableStoragePtr storage); + void applyPartitionDiff(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); - void applyAlterTable(TiDB::DBInfoPtr db_info, TableID table_id); + void applyAlterTable(const TiDB::DBInfoPtr & db_info, TableID table_id); - void applyAlterLogicalTable(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info, ManageableStoragePtr storage); + void applyAlterLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); - void applyAlterPhysicalTable(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info, ManageableStoragePtr storage); + void applyAlterPhysicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); - void applyRenameTable(TiDB::DBInfoPtr new_db_info, TiDB::TableID table_id); + void applyRenameTable(const TiDB::DBInfoPtr & new_db_info, TiDB::TableID table_id); - void applyRenameLogicalTable(TiDB::DBInfoPtr new_db_info, TiDB::TableInfoPtr new_table_info, ManageableStoragePtr storage); + void applyRenameLogicalTable(const TiDB::DBInfoPtr & new_db_info, const TiDB::TableInfoPtr & new_table_info, const ManageableStoragePtr & storage); - void applyRenamePhysicalTable(TiDB::DBInfoPtr new_db_info, TiDB::TableInfo & new_table_info, ManageableStoragePtr storage); + void applyRenamePhysicalTable(const TiDB::DBInfoPtr & new_db_info, const TiDB::TableInfo & new_table_info, const ManageableStoragePtr & storage); void applyExchangeTablePartition(const SchemaDiff & diff); - void applySetTiFlashReplica(TiDB::DBInfoPtr db_info, TableID table_id); - void applySetTiFlashReplica(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info, ManageableStoragePtr storage); + void applySetTiFlashReplica(const TiDB::DBInfoPtr & db_info, TableID table_id); + void applySetTiFlashReplicaOnLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); + void applySetTiFlashReplicaOnPhysicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); }; } // namespace DB diff --git a/tests/fullstack-test2/ddl/alter_table_tiflash_replica.test b/tests/fullstack-test2/ddl/alter_table_tiflash_replica.test new file mode 100644 index 00000000000..1262cd60882 --- /dev/null +++ b/tests/fullstack-test2/ddl/alter_table_tiflash_replica.test @@ -0,0 +1,55 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# test tiflash replica for normal case +mysql> drop table if exists test.t +mysql> create table test.t(a int) +mysql> alter table test.t set tiflash replica 1 + +func> wait_table test t + +>> DBGInvoke get_tiflash_replica_count("test", "t") +┌─get_tiflash_replica_count(test, t)─┐ +│ 1 │ +└────────────────────────────────────┘ + +# test replica for partition tables +mysql> drop table if exists test.t +mysql> create table test.t (x int) partition by range (x) (partition p0 values less than (5), partition p1 values less than (10)); +mysql> alter table test.t set tiflash replica 1 + +func> wait_table test t + +>> DBGInvoke get_tiflash_replica_count("test", "t") +┌─get_tiflash_replica_count(test, t)─┐ +│ 1 │ +└────────────────────────────────────┘ + +>> DBGInvoke get_partition_tables_tiflash_replica_count("test", "t") +┌─get_partition_tables_tiflash_replica_count(test, t)─┐ +│ 1/1/ │ +└─────────────────────────────────────────────────────┘ + +# test replica for add partition tables after set replica +mysql> alter table test.t add partition (partition p2 values less than (2010)); + +>> DBGInvoke __refresh_schemas() + +>> DBGInvoke get_partition_tables_tiflash_replica_count("test", "t") +┌─get_partition_tables_tiflash_replica_count(test, t)─┐ +│ 1/1/1/ │ +└─────────────────────────────────────────────────────┘ + + + From bf4764bc5206e2233cb6e02bc626c0b266368518 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Thu, 7 Jul 2022 16:19:03 +0800 Subject: [PATCH 085/104] Replace initializer_list with vector for planner test framework (#5307) close pingcap/tiflash#5295 --- dbms/src/Flash/tests/bench_window.cpp | 2 +- dbms/src/Flash/tests/gtest_interpreter.cpp | 6 +- .../Flash/tests/gtest_projection_executor.cpp | 36 +++--- dbms/src/Flash/tests/gtest_topn_executor.cpp | 10 +- dbms/src/TestUtils/mockExecutor.cpp | 121 ++++-------------- dbms/src/TestUtils/mockExecutor.h | 69 +++++----- .../TestUtils/tests/gtest_mock_executors.cpp | 2 +- 7 files changed, 85 insertions(+), 161 deletions(-) diff --git a/dbms/src/Flash/tests/bench_window.cpp b/dbms/src/Flash/tests/bench_window.cpp index da9df20fdf3..dfdb358c46c 100644 --- a/dbms/src/Flash/tests/bench_window.cpp +++ b/dbms/src/Flash/tests/bench_window.cpp @@ -27,7 +27,7 @@ class WindowFunctionBench : public ExchangeBench // build tipb::Window and tipb::Sort. // select row_number() over w1 from t1 window w1 as (partition by c1, c2, c3 order by c1, c2, c3); ExchangeBench::SetUp(state); - MockColumnInfos columns{ + MockColumnInfoVec columns{ {"c1", TiDB::TP::TypeLongLong}, {"c2", TiDB::TP::TypeString}, {"c3", TiDB::TP::TypeLongLong}, diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp index ba7d8fd15ee..2d4d4014780 100644 --- a/dbms/src/Flash/tests/gtest_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_interpreter.cpp @@ -96,7 +96,7 @@ try auto request = context.scan("test_db", "test_table_1") .project({"s1", "s2", "s3"}) .project({"s1", "s2"}) - .project("s1") + .project({"s1"}) .build(context); { String expected = R"( @@ -256,7 +256,7 @@ CreatingSets request = context.receive("sender_1") .project({"s1", "s2", "s3"}) .project({"s1", "s2"}) - .project("s1") + .project({"s1"}) .build(context); { String expected = R"( @@ -278,7 +278,7 @@ Union: request = context.receive("sender_1") .project({"s1", "s2", "s3"}) .project({"s1", "s2"}) - .project("s1") + .project({"s1"}) .exchangeSender(tipb::Broadcast) .build(context); { diff --git a/dbms/src/Flash/tests/gtest_projection_executor.cpp b/dbms/src/Flash/tests/gtest_projection_executor.cpp index 2ff0fdff780..8443dedeb49 100644 --- a/dbms/src/Flash/tests/gtest_projection_executor.cpp +++ b/dbms/src/Flash/tests/gtest_projection_executor.cpp @@ -86,11 +86,11 @@ TEST_F(ExecutorProjectionTestRunner, Projection) try { /// Check single column - auto request = buildDAGRequest({col_names[4]}); + auto request = buildDAGRequest({col_names[4]}); executeWithConcurrency(request, {toNullableVec(col_names[4], col4_sorted_asc)}); /// Check multi columns - request = buildDAGRequest({col_names[0], col_names[4]}); + request = buildDAGRequest({col_names[0], col_names[4]}); executeWithConcurrency(request, { toNullableVec(col_names[0], col0_sorted_asc), @@ -98,14 +98,14 @@ try }); /// Check multi columns - request = buildDAGRequest({col_names[0], col_names[1], col_names[4]}); + request = buildDAGRequest({col_names[0], col_names[1], col_names[4]}); executeWithConcurrency(request, {toNullableVec(col_names[0], col0_sorted_asc), toNullableVec(col_names[1], col1_sorted_asc), toNullableVec(col_names[4], col4_sorted_asc)}); /// Check duplicate columns - request = buildDAGRequest({col_names[4], col_names[4], col_names[4]}); + request = buildDAGRequest({col_names[4], col_names[4], col_names[4]}); executeWithConcurrency(request, {toNullableVec(col_names[4], col4_sorted_asc), toNullableVec(col_names[4], col4_sorted_asc), @@ -114,7 +114,7 @@ try { /// Check large number of columns const size_t col_num = 100; - MockColumnNamesVec projection_input; + MockColumnNameVec projection_input; ColumnsWithTypeAndName columns; auto expect_column = toNullableVec(col_names[4], col4_sorted_asc); @@ -124,7 +124,7 @@ try columns.push_back(expect_column); } - request = buildDAGRequest(projection_input); + request = buildDAGRequest(projection_input); executeWithConcurrency(request, columns); } } @@ -138,18 +138,18 @@ try /// Test "equal" function /// Data type: TypeString - request = buildDAGRequest({eq(col(col_names[0]), col(col_names[0])), col(col_names[4])}); + request = buildDAGRequest({eq(col(col_names[0]), col(col_names[0])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 1, 1, 1, 1, 1, 1}), toNullableVec(col_names[4], col4_sorted_asc)}); - request = buildDAGRequest({eq(col(col_names[0]), col(col_names[1])), col(col_names[4])}); + request = buildDAGRequest({eq(col(col_names[0]), col(col_names[1])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 1, 0, {}, 0, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); /// Data type: TypeLong - request = buildDAGRequest({eq(col(col_names[3]), col(col_names[4])), col(col_names[4])}); + request = buildDAGRequest({eq(col(col_names[3]), col(col_names[4])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 0, 0, {}, 1, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); @@ -158,23 +158,23 @@ try /// Test "greater" function /// Data type: TypeString - request = buildDAGRequest({gt(col(col_names[0]), col(col_names[1])), col(col_names[4])}); + request = buildDAGRequest({gt(col(col_names[0]), col(col_names[1])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 0, 0, {}, 0, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); - request = buildDAGRequest({gt(col(col_names[1]), col(col_names[0])), col(col_names[4])}); + request = buildDAGRequest({gt(col(col_names[1]), col(col_names[0])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 1, 0, 1, {}, 1, 1}), toNullableVec(col_names[4], col4_sorted_asc)}); /// Data type: TypeLong - request = buildDAGRequest({gt(col(col_names[3]), col(col_names[4])), col(col_names[4])}); + request = buildDAGRequest({gt(col(col_names[3]), col(col_names[4])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 1, 1, {}, 0, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); - request = buildDAGRequest({gt(col(col_names[4]), col(col_names[3])), col(col_names[4])}); + request = buildDAGRequest({gt(col(col_names[4]), col(col_names[3])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 1, 0, 0, {}, 0, 1}), toNullableVec(col_names[4], col4_sorted_asc)}); @@ -183,18 +183,18 @@ try /// Test "and" function /// Data type: TypeString - request = buildDAGRequest({And(col(col_names[0]), col(col_names[0])), col(col_names[4])}); + request = buildDAGRequest({And(col(col_names[0]), col(col_names[0])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 0, 0, 0, 0, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); - request = buildDAGRequest({And(col(col_names[0]), col(col_names[1])), col(col_names[4])}); + request = buildDAGRequest({And(col(col_names[0]), col(col_names[1])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({0, 0, 0, 0, 0, 0, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); /// Data type: TypeLong - request = buildDAGRequest({And(col(col_names[3]), col(col_names[4])), col(col_names[4])}); + request = buildDAGRequest({And(col(col_names[3]), col(col_names[4])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 1, 0, 0, {}, 1, 0}), toNullableVec(col_names[4], col4_sorted_asc)}); @@ -202,7 +202,7 @@ try /// Test "not" function /// Data type: TypeString - request = buildDAGRequest({NOT(col(col_names[0])), NOT(col(col_names[1])), NOT(col(col_names[2])), col(col_names[4])}); + request = buildDAGRequest({NOT(col(col_names[0])), NOT(col(col_names[1])), NOT(col(col_names[2])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 1, 1, 1, 1, 1, 1}), toNullableVec({1, 1, 1, 1, {}, 1, 1}), @@ -210,7 +210,7 @@ try toNullableVec(col_names[4], col4_sorted_asc)}); /// Data type: TypeLong - request = buildDAGRequest({NOT(col(col_names[3])), NOT(col(col_names[4])), col(col_names[4])}); + request = buildDAGRequest({NOT(col(col_names[3])), NOT(col(col_names[4])), col(col_names[4])}); executeWithConcurrency(request, {toNullableVec({{}, 0, 1, 0, {}, 0, 1}), toNullableVec({{}, 0, 0, 1, 0, 0, 0}), diff --git a/dbms/src/Flash/tests/gtest_topn_executor.cpp b/dbms/src/Flash/tests/gtest_topn_executor.cpp index d5466b5c87d..597ac9f279a 100644 --- a/dbms/src/Flash/tests/gtest_topn_executor.cpp +++ b/dbms/src/Flash/tests/gtest_topn_executor.cpp @@ -52,7 +52,7 @@ class ExecutorTopNTestRunner : public DB::tests::ExecutorTest return context.scan(db_name, table_name).topN(col_name, is_desc, limit_num).build(context); } - std::shared_ptr buildDAGRequest(const String & table_name, MockOrderByItems order_by_items, int limit, MockAsts func_proj_ast = {}, MockColumnNames out_proj_ast = {}) + std::shared_ptr buildDAGRequest(const String & table_name, MockOrderByItemVec order_by_items, int limit, MockAstVec func_proj_ast = {}, MockColumnNameVec out_proj_ast = {}) { if (func_proj_ast.size() == 0) return context.scan(db_name, table_name).topN(order_by_items, limit).build(context); @@ -126,7 +126,7 @@ try toNullableVec(col_name[2], ColumnWithString{"china", "china", "usa", "china", "korea", "usa"}), toNullableVec(col_name[3], ColumnWithInt32{-300, {}, {}, 900, 1300, 0})}}; - std::vector order_by_items{ + std::vector order_by_items{ /// select * from clerk order by age DESC, gender DESC; {MockOrderByItem(col_name[0], true), MockOrderByItem(col_name[1], true)}, /// select * from clerk order by gender DESC, salary ASC; @@ -150,9 +150,9 @@ try { std::shared_ptr request; std::vector expect_cols; - MockColumnNames output_projection{col_name[0], col_name[1], col_name[2], col_name[3]}; - MockAsts func_projection; // Do function operation for topn - MockOrderByItems order_by_items; + MockColumnNameVec output_projection{col_name[0], col_name[1], col_name[2], col_name[3]}; + MockAstVec func_projection; // Do function operation for topn + MockOrderByItemVec order_by_items; ASTPtr col0_ast = col(col_name[0]); ASTPtr col1_ast = col(col_name[1]); ASTPtr col2_ast = col(col_name[2]); diff --git a/dbms/src/TestUtils/mockExecutor.cpp b/dbms/src/TestUtils/mockExecutor.cpp index 9a6e92dd9c1..49ae9787ea4 100644 --- a/dbms/src/TestUtils/mockExecutor.cpp +++ b/dbms/src/TestUtils/mockExecutor.cpp @@ -35,7 +35,7 @@ ASTPtr buildLiteral(const Field & field) return std::make_shared(field); } -ASTPtr buildOrderByItemList(MockOrderByItems order_by_items) +ASTPtr buildOrderByItemVec(MockOrderByItemVec order_by_items) { std::vector vec(order_by_items.size()); size_t i = 0; @@ -92,7 +92,7 @@ std::shared_ptr DAGRequestBuilder::build(MockDAGRequestContext return dag_request_ptr; } -DAGRequestBuilder & DAGRequestBuilder::mockTable(const String & db, const String & table, const MockColumnInfos & columns) +DAGRequestBuilder & DAGRequestBuilder::mockTable(const String & db, const String & table, const MockColumnInfoVec & columns) { assert(!columns.empty()); TableInfo table_info; @@ -114,27 +114,17 @@ DAGRequestBuilder & DAGRequestBuilder::mockTable(const String & db, const String return *this; } -DAGRequestBuilder & DAGRequestBuilder::mockTable(const MockTableName & name, const MockColumnInfos & columns) +DAGRequestBuilder & DAGRequestBuilder::mockTable(const MockTableName & name, const MockColumnInfoVec & columns) { return mockTable(name.first, name.second, columns); } -DAGRequestBuilder & DAGRequestBuilder::mockTable(const MockTableName & name, const MockColumnInfoList & columns) -{ - return mockTable(name.first, name.second, columns); -} - -DAGRequestBuilder & DAGRequestBuilder::exchangeReceiver(const MockColumnInfos & columns) -{ - return buildExchangeReceiver(columns); -} - -DAGRequestBuilder & DAGRequestBuilder::exchangeReceiver(const MockColumnInfoList & columns) +DAGRequestBuilder & DAGRequestBuilder::exchangeReceiver(const MockColumnInfoVec & columns) { return buildExchangeReceiver(columns); } -DAGRequestBuilder & DAGRequestBuilder::buildExchangeReceiver(const MockColumnInfos & columns) +DAGRequestBuilder & DAGRequestBuilder::buildExchangeReceiver(const MockColumnInfoVec & columns) { DAGSchema schema; for (const auto & column : columns) @@ -180,33 +170,23 @@ DAGRequestBuilder & DAGRequestBuilder::topN(ASTPtr order_exprs, ASTPtr limit_exp DAGRequestBuilder & DAGRequestBuilder::topN(const String & col_name, bool desc, int limit) { assert(root); - root = compileTopN(root, getExecutorIndex(), buildOrderByItemList({{col_name, desc}}), buildLiteral(Field(static_cast(limit)))); + root = compileTopN(root, getExecutorIndex(), buildOrderByItemVec({{col_name, desc}}), buildLiteral(Field(static_cast(limit)))); return *this; } -DAGRequestBuilder & DAGRequestBuilder::topN(MockOrderByItems order_by_items, int limit) +DAGRequestBuilder & DAGRequestBuilder::topN(MockOrderByItemVec order_by_items, int limit) { return topN(order_by_items, buildLiteral(Field(static_cast(limit)))); } -DAGRequestBuilder & DAGRequestBuilder::topN(MockOrderByItems order_by_items, ASTPtr limit_expr) +DAGRequestBuilder & DAGRequestBuilder::topN(MockOrderByItemVec order_by_items, ASTPtr limit_expr) { assert(root); - root = compileTopN(root, getExecutorIndex(), buildOrderByItemList(order_by_items), limit_expr); + root = compileTopN(root, getExecutorIndex(), buildOrderByItemVec(order_by_items), limit_expr); return *this; } -DAGRequestBuilder & DAGRequestBuilder::project(const String & col_name) -{ - assert(root); - auto exp_list = std::make_shared(); - exp_list->children.push_back(buildColumn(col_name)); - - root = compileProject(root, getExecutorIndex(), exp_list); - return *this; -} - -DAGRequestBuilder & DAGRequestBuilder::project(MockAsts exprs) +DAGRequestBuilder & DAGRequestBuilder::project(MockAstVec exprs) { assert(root); auto exp_list = std::make_shared(); @@ -218,12 +198,7 @@ DAGRequestBuilder & DAGRequestBuilder::project(MockAsts exprs) return *this; } -DAGRequestBuilder & DAGRequestBuilder::project(MockColumnNames col_names) -{ - return project(MockColumnNamesVec(col_names)); -} - -DAGRequestBuilder & DAGRequestBuilder::project(MockColumnNamesVec col_names) +DAGRequestBuilder & DAGRequestBuilder::project(MockColumnNameVec col_names) { assert(root); auto exp_list = std::make_shared(); @@ -242,12 +217,12 @@ DAGRequestBuilder & DAGRequestBuilder::exchangeSender(tipb::ExchangeType exchang return *this; } -DAGRequestBuilder & DAGRequestBuilder::join(const DAGRequestBuilder & right, MockAsts exprs) +DAGRequestBuilder & DAGRequestBuilder::join(const DAGRequestBuilder & right, MockAstVec exprs) { return join(right, exprs, ASTTableJoin::Kind::Inner); } -DAGRequestBuilder & DAGRequestBuilder::join(const DAGRequestBuilder & right, MockAsts exprs, ASTTableJoin::Kind kind) +DAGRequestBuilder & DAGRequestBuilder::join(const DAGRequestBuilder & right, MockAstVec exprs, ASTTableJoin::Kind kind) { assert(root); assert(right.root); @@ -273,7 +248,7 @@ DAGRequestBuilder & DAGRequestBuilder::aggregation(ASTPtr agg_func, ASTPtr group return buildAggregation(agg_funcs, group_by_exprs); } -DAGRequestBuilder & DAGRequestBuilder::aggregation(MockAsts agg_funcs, MockAsts group_by_exprs) +DAGRequestBuilder & DAGRequestBuilder::aggregation(MockAstVec agg_funcs, MockAstVec group_by_exprs) { auto agg_func_list = std::make_shared(); auto group_by_expr_list = std::make_shared(); @@ -296,80 +271,58 @@ DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByIte assert(root); auto window_func_list = std::make_shared(); window_func_list->children.push_back(window_func); - root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemList({partition_by}), buildOrderByItemList({order_by}), frame); + root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec({partition_by}), buildOrderByItemVec({order_by}), frame); return *this; } -DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByItems order_by_list, MockPartitionByItems partition_by_list, MockWindowFrame frame) +DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame) { assert(root); auto window_func_list = std::make_shared(); window_func_list->children.push_back(window_func); - root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemList(partition_by_list), buildOrderByItemList(order_by_list), frame); + root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec(partition_by_vec), buildOrderByItemVec(order_by_vec), frame); return *this; } -DAGRequestBuilder & DAGRequestBuilder::window(MockAsts window_funcs, MockOrderByItems order_by_list, MockPartitionByItems partition_by_list, MockWindowFrame frame) +DAGRequestBuilder & DAGRequestBuilder::window(MockAstVec window_funcs, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame) { assert(root); auto window_func_list = std::make_shared(); for (const auto & func : window_funcs) window_func_list->children.push_back(func); - root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemList(partition_by_list), buildOrderByItemList(order_by_list), frame); + root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec(partition_by_vec), buildOrderByItemVec(order_by_vec), frame); return *this; } DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItem order_by, bool is_partial_sort) { assert(root); - root = compileSort(root, getExecutorIndex(), buildOrderByItemList({order_by}), is_partial_sort); + root = compileSort(root, getExecutorIndex(), buildOrderByItemVec({order_by}), is_partial_sort); return *this; } -DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItems order_by_list, bool is_partial_sort) +DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItemVec order_by_vec, bool is_partial_sort) { assert(root); - root = compileSort(root, getExecutorIndex(), buildOrderByItemList(order_by_list), is_partial_sort); + root = compileSort(root, getExecutorIndex(), buildOrderByItemVec(order_by_vec), is_partial_sort); return *this; } -void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfoList & columnInfos) -{ - std::vector v_column_info(columnInfos.size()); - size_t i = 0; - for (const auto & info : columnInfos) - { - v_column_info[i++] = std::move(info); - } - mock_tables[name.first + "." + name.second] = v_column_info; -} - -void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfos & columnInfos) +void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfoVec & columnInfos) { mock_tables[db + "." + table] = columnInfos; } -void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfos & columnInfos) +void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfoVec & columnInfos) { mock_tables[name.first + "." + name.second] = columnInfos; } -void MockDAGRequestContext::addExchangeRelationSchema(String name, const MockColumnInfos & columnInfos) +void MockDAGRequestContext::addExchangeRelationSchema(String name, const MockColumnInfoVec & columnInfos) { exchange_schemas[name] = columnInfos; } -void MockDAGRequestContext::addExchangeRelationSchema(String name, const MockColumnInfoList & columnInfos) -{ - std::vector v_column_info(columnInfos.size()); - size_t i = 0; - for (const auto & info : columnInfos) - { - v_column_info[i++] = std::move(info); - } - exchange_schemas[name] = v_column_info; -} - void MockDAGRequestContext::addMockTableColumnData(const String & db, const String & table, ColumnsWithTypeAndName columns) { mock_table_columns[db + "." + table] = columns; @@ -385,37 +338,19 @@ void MockDAGRequestContext::addExchangeReceiverColumnData(const String & name, C mock_exchange_columns[name] = columns; } -void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfoList & columnInfos, ColumnsWithTypeAndName columns) +void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfoVec & columnInfos, ColumnsWithTypeAndName columns) { addMockTable(db, table, columnInfos); addMockTableColumnData(db, table, columns); } -void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfos & columnInfos, ColumnsWithTypeAndName columns) -{ - addMockTable(db, table, columnInfos); - addMockTableColumnData(db, table, columns); -} - -void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfoList & columnInfos, ColumnsWithTypeAndName columns) -{ - addMockTable(name, columnInfos); - addMockTableColumnData(name, columns); -} - -void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfos & columnInfos, ColumnsWithTypeAndName columns) +void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfoVec & columnInfos, ColumnsWithTypeAndName columns) { addMockTable(name, columnInfos); addMockTableColumnData(name, columns); } -void MockDAGRequestContext::addExchangeReceiver(const String & name, MockColumnInfos columnInfos, ColumnsWithTypeAndName columns) -{ - addExchangeRelationSchema(name, columnInfos); - addExchangeReceiverColumnData(name, columns); -} - -void MockDAGRequestContext::addExchangeReceiver(const String & name, MockColumnInfoList columnInfos, ColumnsWithTypeAndName columns) +void MockDAGRequestContext::addExchangeReceiver(const String & name, MockColumnInfoVec columnInfos, ColumnsWithTypeAndName columns) { addExchangeRelationSchema(name, columnInfos); addExchangeReceiverColumnData(name, columns); diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index bad92c4226d..971061e93e7 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -23,16 +23,14 @@ namespace DB::tests { using MockColumnInfo = std::pair; -using MockColumnInfos = std::vector; -using MockColumnInfoList = std::initializer_list; +using MockColumnInfoVec = std::vector; using MockTableName = std::pair; using MockOrderByItem = std::pair; -using MockOrderByItems = std::initializer_list; +using MockOrderByItemVec = std::vector; using MockPartitionByItem = std::pair; -using MockPartitionByItems = std::initializer_list; -using MockColumnNames = std::initializer_list; -using MockColumnNamesVec = std::vector; -using MockAsts = std::initializer_list; +using MockPartitionByItemVec = std::vector; +using MockColumnNameVec = std::vector; +using MockAstVec = std::vector; using MockWindowFrame = mock::MockWindowFrame; class MockDAGRequestContext; @@ -65,12 +63,10 @@ class DAGRequestBuilder std::shared_ptr build(MockDAGRequestContext & mock_context); - DAGRequestBuilder & mockTable(const String & db, const String & table, const MockColumnInfos & columns); - DAGRequestBuilder & mockTable(const MockTableName & name, const MockColumnInfos & columns); - DAGRequestBuilder & mockTable(const MockTableName & name, const MockColumnInfoList & columns); + DAGRequestBuilder & mockTable(const String & db, const String & table, const MockColumnInfoVec & columns); + DAGRequestBuilder & mockTable(const MockTableName & name, const MockColumnInfoVec & columns); - DAGRequestBuilder & exchangeReceiver(const MockColumnInfos & columns); - DAGRequestBuilder & exchangeReceiver(const MockColumnInfoList & columns); + DAGRequestBuilder & exchangeReceiver(const MockColumnInfoVec & columns); DAGRequestBuilder & filter(ASTPtr filter_expr); @@ -79,36 +75,34 @@ class DAGRequestBuilder DAGRequestBuilder & topN(ASTPtr order_exprs, ASTPtr limit_expr); DAGRequestBuilder & topN(const String & col_name, bool desc, int limit); - DAGRequestBuilder & topN(MockOrderByItems order_by_items, int limit); - DAGRequestBuilder & topN(MockOrderByItems order_by_items, ASTPtr limit_expr); + DAGRequestBuilder & topN(MockOrderByItemVec order_by_items, int limit); + DAGRequestBuilder & topN(MockOrderByItemVec order_by_items, ASTPtr limit_expr); - DAGRequestBuilder & project(const String & col_name); - DAGRequestBuilder & project(MockAsts expr); - DAGRequestBuilder & project(MockColumnNames col_names); - DAGRequestBuilder & project(MockColumnNamesVec col_names); + DAGRequestBuilder & project(MockAstVec exprs); + DAGRequestBuilder & project(MockColumnNameVec col_names); DAGRequestBuilder & exchangeSender(tipb::ExchangeType exchange_type); // Currentlt only support inner join, left join and right join. // TODO support more types of join. - DAGRequestBuilder & join(const DAGRequestBuilder & right, MockAsts exprs); - DAGRequestBuilder & join(const DAGRequestBuilder & right, MockAsts exprs, ASTTableJoin::Kind kind); + DAGRequestBuilder & join(const DAGRequestBuilder & right, MockAstVec exprs); + DAGRequestBuilder & join(const DAGRequestBuilder & right, MockAstVec exprs, ASTTableJoin::Kind kind); // aggregation DAGRequestBuilder & aggregation(ASTPtr agg_func, ASTPtr group_by_expr); - DAGRequestBuilder & aggregation(MockAsts agg_funcs, MockAsts group_by_exprs); + DAGRequestBuilder & aggregation(MockAstVec agg_funcs, MockAstVec group_by_exprs); // window DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItem order_by, MockPartitionByItem partition_by, MockWindowFrame frame); - DAGRequestBuilder & window(MockAsts window_funcs, MockOrderByItems order_by_list, MockPartitionByItems partition_by_list, MockWindowFrame frame); - DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItems order_by_list, MockPartitionByItems partition_by_list, MockWindowFrame frame); + DAGRequestBuilder & window(MockAstVec window_funcs, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame); + DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame); DAGRequestBuilder & sort(MockOrderByItem order_by, bool is_partial_sort); - DAGRequestBuilder & sort(MockOrderByItems order_by_list, bool is_partial_sort); + DAGRequestBuilder & sort(MockOrderByItemVec order_by_vec, bool is_partial_sort); private: void initDAGRequest(tipb::DAGRequest & dag_request); DAGRequestBuilder & buildAggregation(ASTPtr agg_funcs, ASTPtr group_by_exprs); - DAGRequestBuilder & buildExchangeReceiver(const MockColumnInfos & columns); + DAGRequestBuilder & buildExchangeReceiver(const MockColumnInfoVec & columns); ExecutorPtr root; DAGProperties properties; @@ -132,20 +126,15 @@ class MockDAGRequestContext return DAGRequestBuilder(index); } - void addMockTable(const MockTableName & name, const MockColumnInfoList & columnInfos); - void addMockTable(const String & db, const String & table, const MockColumnInfos & columnInfos); - void addMockTable(const MockTableName & name, const MockColumnInfos & columnInfos); - void addExchangeRelationSchema(String name, const MockColumnInfos & columnInfos); - void addExchangeRelationSchema(String name, const MockColumnInfoList & columnInfos); + void addMockTable(const String & db, const String & table, const MockColumnInfoVec & columnInfos); + void addMockTable(const MockTableName & name, const MockColumnInfoVec & columnInfos); + void addExchangeRelationSchema(String name, const MockColumnInfoVec & columnInfos); void addMockTableColumnData(const String & db, const String & table, ColumnsWithTypeAndName columns); - void addMockTable(const String & db, const String & table, const MockColumnInfoList & columnInfos, ColumnsWithTypeAndName columns); - void addMockTable(const String & db, const String & table, const MockColumnInfos & columnInfos, ColumnsWithTypeAndName columns); - void addMockTable(const MockTableName & name, const MockColumnInfoList & columnInfos, ColumnsWithTypeAndName columns); - void addMockTable(const MockTableName & name, const MockColumnInfos & columnInfos, ColumnsWithTypeAndName columns); + void addMockTable(const String & db, const String & table, const MockColumnInfoVec & columnInfos, ColumnsWithTypeAndName columns); + void addMockTable(const MockTableName & name, const MockColumnInfoVec & columnInfos, ColumnsWithTypeAndName columns); void addMockTableColumnData(const MockTableName & name, ColumnsWithTypeAndName columns); void addExchangeReceiverColumnData(const String & name, ColumnsWithTypeAndName columns); - void addExchangeReceiver(const String & name, MockColumnInfos columnInfos, ColumnsWithTypeAndName columns); - void addExchangeReceiver(const String & name, MockColumnInfoList columnInfos, ColumnsWithTypeAndName columns); + void addExchangeReceiver(const String & name, MockColumnInfoVec columnInfos, ColumnsWithTypeAndName columns); std::unordered_map & executorIdColumnsMap() { return executor_id_columns_map; } @@ -154,8 +143,8 @@ class MockDAGRequestContext private: size_t index; - std::unordered_map mock_tables; - std::unordered_map exchange_schemas; + std::unordered_map mock_tables; + std::unordered_map exchange_schemas; std::unordered_map mock_table_columns; std::unordered_map mock_exchange_columns; std::unordered_map executor_id_columns_map; /// @@ -170,8 +159,8 @@ class MockDAGRequestContext ASTPtr buildColumn(const String & column_name); ASTPtr buildLiteral(const Field & field); -ASTPtr buildFunction(MockAsts exprs, const String & name); -ASTPtr buildOrderByItemList(MockOrderByItems order_by_items); +ASTPtr buildFunction(MockAstVec exprs, const String & name); +ASTPtr buildOrderByItemVec(MockOrderByItemVec order_by_items); MockWindowFrame buildDefaultRowsFrame(); diff --git a/dbms/src/TestUtils/tests/gtest_mock_executors.cpp b/dbms/src/TestUtils/tests/gtest_mock_executors.cpp index 8bed0f2fc6c..72f0bb505d1 100644 --- a/dbms/src/TestUtils/tests/gtest_mock_executors.cpp +++ b/dbms/src/TestUtils/tests/gtest_mock_executors.cpp @@ -76,7 +76,7 @@ TEST_F(MockDAGRequestTest, Projection) try { auto request = context.scan("test_db", "test_table") - .project("s1") + .project({"s1"}) .build(context); { String expected = "project_1 | {<0, String>}\n" From 0aec04afc4c51c1965277af7cac58ddf0c918a1d Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Thu, 7 Jul 2022 17:37:03 +0800 Subject: [PATCH 086/104] KVStore: decouple flush region and CompactLog with a new FFI fn_try_flush_data (#5283) ref pingcap/tiflash#5170 --- contrib/tiflash-proxy | 2 +- dbms/src/Storages/Transaction/KVStore.cpp | 88 +++++++++++++------ dbms/src/Storages/Transaction/KVStore.h | 8 ++ dbms/src/Storages/Transaction/ProxyFFI.cpp | 28 ++++++ dbms/src/Storages/Transaction/ProxyFFI.h | 4 + .../Transaction/tests/gtest_kvstore.cpp | 6 ++ 6 files changed, 107 insertions(+), 29 deletions(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 6ea4d608b1c..0080c6866d8 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 6ea4d608b1c03fab89d17f54a2e399602231e27c +Subproject commit 0080c6866d8bbba32bdf198437dbc98cc078ea7e diff --git a/dbms/src/Storages/Transaction/KVStore.cpp b/dbms/src/Storages/Transaction/KVStore.cpp index f9d6d01955e..fb31e4476bb 100644 --- a/dbms/src/Storages/Transaction/KVStore.cpp +++ b/dbms/src/Storages/Transaction/KVStore.cpp @@ -327,6 +327,64 @@ void KVStore::persistRegion(const Region & region, const RegionTaskLock & region LOG_FMT_DEBUG(log, "Persist {} done", region.toString(false)); } +bool KVStore::needFlushRegionData(UInt64 region_id, TMTContext & tmt) +{ + auto region_task_lock = region_manager.genRegionTaskLock(region_id); + const RegionPtr curr_region_ptr = getRegion(region_id); + return canFlushRegionDataImpl(curr_region_ptr, false, false, tmt, region_task_lock); +} + +bool KVStore::tryFlushRegionData(UInt64 region_id, bool try_until_succeed, TMTContext & tmt) +{ + auto region_task_lock = region_manager.genRegionTaskLock(region_id); + const RegionPtr curr_region_ptr = getRegion(region_id); + return canFlushRegionDataImpl(curr_region_ptr, true, try_until_succeed, tmt, region_task_lock); +} + +bool KVStore::canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 flush_if_possible, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock) +{ + if (curr_region_ptr == nullptr) + { + throw Exception(fmt::format("region not found when trying flush", ErrorCodes::LOGICAL_ERROR)); + } + auto & curr_region = *curr_region_ptr; + + auto [rows, size_bytes] = curr_region.getApproxMemCacheInfo(); + + LOG_FMT_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}", curr_region.toString(false), rows, size_bytes); + + bool can_flush = false; + if (rows >= region_compact_log_min_rows.load(std::memory_order_relaxed) + || size_bytes >= region_compact_log_min_bytes.load(std::memory_order_relaxed)) + { + // if rows or bytes more than threshold, flush cache and persist mem data. + can_flush = true; + } + else + { + // if there is little data in mem, wait until time interval reached threshold. + // use random period so that lots of regions will not be persisted at same time. + auto compact_log_period = std::rand() % region_compact_log_period.load(std::memory_order_relaxed); // NOLINT + can_flush = !(curr_region.lastCompactLogTime() + Seconds{compact_log_period} > Clock::now()); + } + if (can_flush && flush_if_possible) + { + LOG_FMT_DEBUG(log, "{} flush region due to can_flush_data", curr_region.toString(false)); + if (tryFlushRegionCacheInStorage(tmt, curr_region, log, try_until_succeed)) + { + persistRegion(curr_region, region_task_lock, "compact raft log"); + curr_region.markCompactLog(); + curr_region.cleanApproxMemCacheInfo(); + return true; + } + else + { + return false; + } + } + return can_flush; +} + EngineStoreApplyRes KVStore::handleUselessAdminRaftCmd( raft_cmdpb::AdminCmdType cmd_type, UInt64 curr_region_id, @@ -360,39 +418,13 @@ EngineStoreApplyRes KVStore::handleUselessAdminRaftCmd( } else { - auto [rows, size_bytes] = curr_region.getApproxMemCacheInfo(); - - LOG_FMT_DEBUG(log, "{} approx mem cache info: rows {}, bytes {}", curr_region.toString(false), rows, size_bytes); - - if (rows >= region_compact_log_min_rows.load(std::memory_order_relaxed) - || size_bytes >= region_compact_log_min_bytes.load(std::memory_order_relaxed)) - { - // if rows or bytes more than threshold, try to flush cache and persist mem data. - return true; - } - else - { - // if there is little data in mem, wait until time interval reached threshold. - // use random period so that lots of regions will not be persisted at same time. - auto compact_log_period = std::rand() % region_compact_log_period.load(std::memory_order_relaxed); // NOLINT - return !(curr_region.lastCompactLogTime() + Seconds{compact_log_period} > Clock::now()); - } + return canFlushRegionDataImpl(curr_region_ptr, true, /* try_until_succeed */ false, tmt, region_task_lock); } }; if (check_sync_log()) { - if (tryFlushRegionCacheInStorage(tmt, curr_region, log, /* try_until_succeed */ false)) - { - persistRegion(curr_region, region_task_lock, "compact raft log"); - curr_region.markCompactLog(); - curr_region.cleanApproxMemCacheInfo(); - return EngineStoreApplyRes::Persist; - } - else - { - return EngineStoreApplyRes::None; - } + return EngineStoreApplyRes::Persist; } return EngineStoreApplyRes::None; } diff --git a/dbms/src/Storages/Transaction/KVStore.h b/dbms/src/Storages/Transaction/KVStore.h index 66e2fe32b75..b58083557a1 100644 --- a/dbms/src/Storages/Transaction/KVStore.h +++ b/dbms/src/Storages/Transaction/KVStore.h @@ -108,6 +108,9 @@ class KVStore final : private boost::noncopyable TMTContext & tmt); EngineStoreApplyRes handleWriteRaftCmd(const WriteCmdsView & cmds, UInt64 region_id, UInt64 index, UInt64 term, TMTContext & tmt); + bool needFlushRegionData(UInt64 region_id, TMTContext & tmt); + bool tryFlushRegionData(UInt64 region_id, bool try_until_succeed, TMTContext & tmt); + void handleApplySnapshot(metapb::Region && region, uint64_t peer_id, const SSTViewVec, uint64_t index, uint64_t term, TMTContext & tmt); std::vector /* */ preHandleSnapshotToFiles( @@ -219,6 +222,11 @@ class KVStore final : private boost::noncopyable UInt64 term, TMTContext & tmt); + /// Notice that if flush_if_possible is set to false, we only check if a flush is allowed by rowsize/size/interval. + /// It will not check if a flush will eventually succeed. + /// In other words, `canFlushRegionDataImpl(flush_if_possible=true)` can return false. + bool canFlushRegionDataImpl(const RegionPtr & curr_region_ptr, UInt8 flush_if_possible, bool try_until_succeed, TMTContext & tmt, const RegionTaskLock & region_task_lock); + void persistRegion(const Region & region, const RegionTaskLock & region_task_lock, const char * caller); void releaseReadIndexWorkers(); void handleDestroy(UInt64 region_id, TMTContext & tmt, const KVStoreTaskLock &); diff --git a/dbms/src/Storages/Transaction/ProxyFFI.cpp b/dbms/src/Storages/Transaction/ProxyFFI.cpp index 8a40ca9b15e..d4ba50d5714 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.cpp +++ b/dbms/src/Storages/Transaction/ProxyFFI.cpp @@ -128,6 +128,34 @@ EngineStoreApplyRes HandleAdminRaftCmd( } } +uint8_t NeedFlushData(EngineStoreServerWrap * server, uint64_t region_id) +{ + try + { + auto & kvstore = server->tmt->getKVStore(); + return kvstore->needFlushRegionData(region_id, *server->tmt); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + exit(-1); + } +} + +uint8_t TryFlushData(EngineStoreServerWrap * server, uint64_t region_id, uint8_t until_succeed) +{ + try + { + auto & kvstore = server->tmt->getKVStore(); + return kvstore->tryFlushRegionData(region_id, until_succeed, *server->tmt); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + exit(-1); + } +} + static_assert(sizeof(RaftStoreProxyFFIHelper) == sizeof(TiFlashRaftProxyHelper)); static_assert(alignof(RaftStoreProxyFFIHelper) == alignof(TiFlashRaftProxyHelper)); diff --git a/dbms/src/Storages/Transaction/ProxyFFI.h b/dbms/src/Storages/Transaction/ProxyFFI.h index e1c01599275..aafe4b375eb 100644 --- a/dbms/src/Storages/Transaction/ProxyFFI.h +++ b/dbms/src/Storages/Transaction/ProxyFFI.h @@ -125,6 +125,8 @@ EngineStoreApplyRes HandleAdminRaftCmd( EngineStoreApplyRes HandleWriteRaftCmd(const EngineStoreServerWrap * server, WriteCmdsView cmds, RaftCmdHeader header); +uint8_t NeedFlushData(EngineStoreServerWrap * server, uint64_t region_id); +uint8_t TryFlushData(EngineStoreServerWrap * server, uint64_t region_id, uint8_t until_succeed); void AtomicUpdateProxy(EngineStoreServerWrap * server, RaftStoreProxyFFIHelper * proxy); void HandleDestroy(EngineStoreServerWrap * server, uint64_t region_id); EngineStoreApplyRes HandleIngestSST(EngineStoreServerWrap * server, SSTViewVec snaps, RaftCmdHeader header); @@ -158,6 +160,8 @@ inline EngineStoreServerHelper GetEngineStoreServerHelper( .fn_gen_cpp_string = GenCppRawString, .fn_handle_write_raft_cmd = HandleWriteRaftCmd, .fn_handle_admin_raft_cmd = HandleAdminRaftCmd, + .fn_need_flush_data = NeedFlushData, + .fn_try_flush_data = TryFlushData, .fn_atomic_update_proxy = AtomicUpdateProxy, .fn_handle_destroy = HandleDestroy, .fn_handle_ingest_sst = HandleIngestSST, diff --git a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp index 36a91522bb6..77aab06f6cf 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_kvstore.cpp @@ -1179,6 +1179,12 @@ void RegionKVStoreTest::testKVStore() ASSERT_EQ(e.message(), "unsupported admin command type InvalidAdmin"); } } + { + // There shall be data to flush. + ASSERT_EQ(kvs.needFlushRegionData(19, ctx.getTMTContext()), true); + // Force flush until succeed only for testing. + ASSERT_EQ(kvs.tryFlushRegionData(19, true, ctx.getTMTContext()), true); + } } void test_mergeresult() From 3e8df4bd1e569a0592bfdabd7e8fb0c8dd827a63 Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 7 Jul 2022 18:31:02 +0800 Subject: [PATCH 087/104] refine error message in mpptask (#5304) ref pingcap/tiflash#5095 --- dbms/src/Flash/EstablishCall.cpp | 2 +- dbms/src/Flash/Mpp/MPPTask.cpp | 4 +- dbms/src/Flash/Mpp/MPPTunnel.cpp | 4 ++ dbms/src/Flash/Mpp/Utils.cpp | 11 +++++ dbms/src/Flash/Mpp/Utils.h | 1 + dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp | 4 +- tests/fullstack-test/mpp/issue_2471.test | 10 +--- tests/fullstack-test/mpp/mpp_fail.test | 50 ++------------------ 8 files changed, 28 insertions(+), 58 deletions(-) diff --git a/dbms/src/Flash/EstablishCall.cpp b/dbms/src/Flash/EstablishCall.cpp index 89857a2407e..2f8c7c15f56 100644 --- a/dbms/src/Flash/EstablishCall.cpp +++ b/dbms/src/Flash/EstablishCall.cpp @@ -143,7 +143,7 @@ void EstablishCallData::finishTunnelAndResponder() state = FINISH; if (mpp_tunnel) { - mpp_tunnel->consumerFinish("grpc writes failed.", true); //trigger mpp tunnel finish work + mpp_tunnel->consumerFinish(fmt::format("{}: finishTunnelAndResponder called.", mpp_tunnel->id()), true); //trigger mpp tunnel finish work } grpc::Status status(static_cast(GRPC_STATUS_UNKNOWN), "Consumer exits unexpected, grpc writes failed."); responder.Finish(status, this); diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index c2d5e6f49f8..da8f3034abc 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -379,7 +379,7 @@ void MPPTask::runImpl() } catch (...) { - err_msg = getCurrentExceptionMessage(true); + err_msg = getCurrentExceptionMessage(true, true); } if (err_msg.empty()) @@ -405,6 +405,8 @@ void MPPTask::runImpl() if (status == RUNNING) { LOG_FMT_ERROR(log, "task running meets error: {}", err_msg); + /// trim the stack trace to avoid too many useless information in log + trimStackTrace(err_msg); try { handleError(err_msg); diff --git a/dbms/src/Flash/Mpp/MPPTunnel.cpp b/dbms/src/Flash/Mpp/MPPTunnel.cpp index 13a7eaad95e..16fe4ae42cc 100644 --- a/dbms/src/Flash/Mpp/MPPTunnel.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnel.cpp @@ -220,7 +220,11 @@ void MPPTunnelBase::sendJob(bool need_lock) err_msg = "fatal error in sendJob()"; } if (!err_msg.empty()) + { + /// append tunnel id to error message + err_msg = fmt::format("{} meet error: {}", tunnel_id, err_msg); LOG_ERROR(log, err_msg); + } consumerFinish(err_msg, need_lock); if (is_async) writer->writeDone(grpc::Status::OK); diff --git a/dbms/src/Flash/Mpp/Utils.cpp b/dbms/src/Flash/Mpp/Utils.cpp index 477c478eef7..21d89b3cd52 100644 --- a/dbms/src/Flash/Mpp/Utils.cpp +++ b/dbms/src/Flash/Mpp/Utils.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include @@ -27,4 +28,14 @@ mpp::MPPDataPacket getPacketWithError(String reason) return data; } +void trimStackTrace(String & message) +{ + auto stack_trace_pos = message.find("Stack trace"); + if (stack_trace_pos != String::npos) + { + message.resize(stack_trace_pos); + Poco::trimRightInPlace(message); + } +} + } // namespace DB diff --git a/dbms/src/Flash/Mpp/Utils.h b/dbms/src/Flash/Mpp/Utils.h index 67e2dc3f641..021dc4407d5 100644 --- a/dbms/src/Flash/Mpp/Utils.h +++ b/dbms/src/Flash/Mpp/Utils.h @@ -23,5 +23,6 @@ namespace DB { mpp::MPPDataPacket getPacketWithError(String reason); +void trimStackTrace(String & message); } // namespace DB diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp index 47ce2ee6ee6..706c17ed036 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp @@ -382,7 +382,7 @@ TEST_F(TestMPPTunnelBase, WriteError) } catch (Exception & e) { - GTEST_ASSERT_EQ(e.message(), "Consumer exits unexpected, grpc writes failed."); + GTEST_ASSERT_EQ(e.message(), "Consumer exits unexpected, 0000_0001 meet error: grpc writes failed."); } } @@ -631,7 +631,7 @@ TEST_F(TestMPPTunnelBase, AsyncWriteError) } catch (Exception & e) { - GTEST_ASSERT_EQ(e.message(), "Consumer exits unexpected, grpc writes failed."); + GTEST_ASSERT_EQ(e.message(), "Consumer exits unexpected, 0000_0001 meet error: grpc writes failed."); } } diff --git a/tests/fullstack-test/mpp/issue_2471.test b/tests/fullstack-test/mpp/issue_2471.test index 497ce605893..9966eaadec3 100644 --- a/tests/fullstack-test/mpp/issue_2471.test +++ b/tests/fullstack-test/mpp/issue_2471.test @@ -35,15 +35,7 @@ mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_opt_bro => DBGInvoke __enable_fail_point(exception_in_creating_set_input_stream) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_opt_broadcast_cartesian_join=2; select * from a as t1 left join a as t2 on t1.id = t2.id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_in_creating_set_input_stream is triggered., e.what() = DB::Exception, Stack trace: -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_in_creating_set_input_stream is triggered., e.what() = DB::Exception, => DBGInvoke __disable_fail_point(exception_in_creating_set_input_stream) diff --git a/tests/fullstack-test/mpp/mpp_fail.test b/tests/fullstack-test/mpp/mpp_fail.test index e03c6150be6..0e272c0b621 100644 --- a/tests/fullstack-test/mpp/mpp_fail.test +++ b/tests/fullstack-test/mpp/mpp_fail.test @@ -71,44 +71,20 @@ ERROR 1105 (HY000) at line 1: DB::Exception: Fail point FailPoints::exception_be ## exception during mpp run non root task => DBGInvoke __enable_fail_point(exception_during_mpp_non_root_task_run) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered., e.what() = DB::Exception, Stack trace: -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered., e.what() = DB::Exception,, e.what() = DB::Exception, => DBGInvoke __disable_fail_point(exception_during_mpp_non_root_task_run) ## exception during mpp run root task => DBGInvoke __enable_fail_point(exception_during_mpp_root_task_run) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_root_task_run is triggered., e.what() = DB::Exception, Stack trace: -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_root_task_run is triggered., e.what() = DB::Exception, => DBGInvoke __disable_fail_point(exception_during_mpp_root_task_run) ## exception during mpp write err to tunnel => DBGInvoke __enable_fail_point(exception_during_mpp_non_root_task_run) => DBGInvoke __enable_fail_point(exception_during_mpp_write_err_to_tunnel) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Failed to write error msg to tunnel, e.what() = DB::Exception, Stack trace: -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Failed to write error msg to tunnel, e.what() = DB::Exception, => DBGInvoke __disable_fail_point(exception_during_mpp_non_root_task_run) => DBGInvoke __disable_fail_point(exception_during_mpp_write_err_to_tunnel) @@ -116,14 +92,7 @@ ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText => DBGInvoke __enable_fail_point(exception_during_mpp_non_root_task_run) => DBGInvoke __enable_fail_point(exception_during_mpp_close_tunnel) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; select count(value), id from t group by id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered., e.what() = DB::Exception, Stack trace: -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText() = DB::Exception: Exchange receiver meet error : Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_during_mpp_non_root_task_run is triggered., e.what() = DB::Exception,, e.what() = DB::Exception, => DBGInvoke __disable_fail_point(exception_during_mpp_non_root_task_run) => DBGInvoke __disable_fail_point(exception_during_mpp_close_tunnel) @@ -156,16 +125,7 @@ ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 0, e.displayText ## ensure build1, build2-probe1, probe2 in the CreatingSets, test the bug where build1 throw exception but not change the build state, thus block the build2-probe1, at last this query hangs. => DBGInvoke __enable_fail_point(exception_mpp_hash_build) mysql> use test; set @@tidb_isolation_read_engines='tiflash'; set @@tidb_allow_mpp=1; set @@tidb_broadcast_join_threshold_count=0; set @@tidb_broadcast_join_threshold_size=0; select t1.id from test.t t1 join test.t t2 on t1.id = t2.id and t1.id <2 join (select id from test.t group by id) t3 on t2.id=t3.id; -ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_mpp_hash_build is triggered., e.what() = DB::Exception, Stack trace: -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} -{#LINE} +ERROR 1105 (HY000) at line 1: other error for mpp stream: Code: 10007, e.displayText() = DB::Exception: Fail point FailPoints::exception_mpp_hash_build is triggered., e.what() = DB::Exception, => DBGInvoke __disable_fail_point(exception_mpp_hash_build) # Clean up. From 5295223a3a65b3e5793d9744520ad1c46e4ce405 Mon Sep 17 00:00:00 2001 From: lizhenhuan <1916038084@qq.com> Date: Thu, 7 Jul 2022 19:05:02 +0800 Subject: [PATCH 088/104] Implement ReverseUTF8/Reverse function push down (#5233) close pingcap/tiflash#5111 --- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 4 +- .../Functions/tests/gtest_strings_reverse.cpp | 120 ++++++++++++++++++ tests/fullstack-test/expr/reverse.test | 44 +++++++ 3 files changed, 166 insertions(+), 2 deletions(-) create mode 100644 dbms/src/Functions/tests/gtest_strings_reverse.cpp create mode 100644 tests/fullstack-test/expr/reverse.test diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index f0800eda4df..a4d491e5637 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -648,8 +648,8 @@ const std::unordered_map scalar_func_map({ //{tipb::ScalarFuncSig::Quote, "cast"}, //{tipb::ScalarFuncSig::Repeat, "cast"}, {tipb::ScalarFuncSig::Replace, "replaceAll"}, - //{tipb::ScalarFuncSig::ReverseUTF8, "cast"}, - //{tipb::ScalarFuncSig::Reverse, "cast"}, + {tipb::ScalarFuncSig::ReverseUTF8, "reverseUTF8"}, + {tipb::ScalarFuncSig::Reverse, "reverse"}, {tipb::ScalarFuncSig::RightUTF8, "rightUTF8"}, //{tipb::ScalarFuncSig::Right, "cast"}, {tipb::ScalarFuncSig::RpadUTF8, "rpadUTF8"}, diff --git a/dbms/src/Functions/tests/gtest_strings_reverse.cpp b/dbms/src/Functions/tests/gtest_strings_reverse.cpp new file mode 100644 index 00000000000..304a403db83 --- /dev/null +++ b/dbms/src/Functions/tests/gtest_strings_reverse.cpp @@ -0,0 +1,120 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include + +#include +#include + +#pragma GCC diagnostic pop + +namespace DB::tests +{ +class StringReverse : public DB::tests::FunctionTest +{ +protected: + static ColumnWithTypeAndName toVec(const std::vector & v) + { + return createColumn(v); + } + + static ColumnWithTypeAndName toNullableVec(const std::vector> & v) + { + return createColumn>(v); + } + + static ColumnWithTypeAndName toConst(const String & s) + { + return createConstColumn(1, s); + } +}; +// test reverse +TEST_F(StringReverse, stringReverseTest) +try +{ + std::vector candidate_strings = {"one week's time test", "abcdef", "abcabc", "moc.pacgnip"}; + std::vector reversed_strings = {"tset emit s'keew eno", "fedcba", "cbacba", "pingcap.com"}; + + // test vector + ASSERT_COLUMN_EQ( + toVec(reversed_strings), + executeFunction( + "reverse", + toVec(candidate_strings))); + + // test nullable + ASSERT_COLUMN_EQ( + toNullableVec({"", " ", {}, "pacgnip"}), + executeFunction( + "reverse", + toNullableVec({"", " ", {}, "pingcap"}))); + + // test const + ASSERT_COLUMN_EQ( + toConst("pacgnip"), + executeFunction( + "reverse", + toConst("pingcap"))); + + // test null + ASSERT_COLUMN_EQ( + toConst({}), + executeFunction( + "reverse", + toConst({}))); +} +CATCH + +// test reverseUTF8 +TEST_F(StringReverse, stringReverseUTF8Test) +try +{ + std::vector candidate_strings = {"one week's time test", "abc测试def", "abcテストabc", "ѐёђѓєѕіїјљњћќѝўџ", "+ѐ-ё*ђ/ѓ!є@ѕ#і$@ї%ј……љ&њ(ћ)ќ¥ѝ#ў@џ!^", "αβγδεζηθικλμνξοπρστυφχψωσ", "▲α▼βγ➨δε☎ζη✂θι€κλ♫μν✓ξο✚πρ℉στ♥υφ♖χψ♘ω★σ✕", "թփձջրչճժծքոեռտըւիօպասդֆգհյկլխզղցվբնմշ"}; + std::vector reversed_strings = {"tset emit s'keew eno", "fed试测cba", "cbaトステcba", "џўѝќћњљјїіѕєѓђёѐ", "^!џ@ў#ѝ¥ќ)ћ(њ&љ……ј%ї@$і#ѕ@є!ѓ/ђ*ё-ѐ+", "σωψχφυτσρποξνμλκιθηζεδγβα", "✕σ★ω♘ψχ♖φυ♥τσ℉ρπ✚οξ✓νμ♫λκ€ιθ✂ηζ☎εδ➨γβ▼α▲", "շմնբվցղզխլկյհգֆդսապօիւըտռեոքծժճչրջձփթ"}; + + // test vector + ASSERT_COLUMN_EQ( + toVec(reversed_strings), + executeFunction( + "reverseUTF8", + toVec(candidate_strings))); + + // test nullable + ASSERT_COLUMN_EQ( + toNullableVec({"", " ", {}, "pacgnip"}), + executeFunction( + "reverseUTF8", + toNullableVec({"", " ", {}, "pingcap"}))); + + // test const + ASSERT_COLUMN_EQ( + toConst("pacgnip"), + executeFunction( + "reverseUTF8", + toConst("pingcap"))); + + // test null + ASSERT_COLUMN_EQ( + toConst({}), + executeFunction( + "reverseUTF8", + toConst({}))); +} +CATCH + +} // namespace DB::tests \ No newline at end of file diff --git a/tests/fullstack-test/expr/reverse.test b/tests/fullstack-test/expr/reverse.test new file mode 100644 index 00000000000..9195adf2b7d --- /dev/null +++ b/tests/fullstack-test/expr/reverse.test @@ -0,0 +1,44 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +mysql> drop table if exists test.t; +mysql> create table if not exists test.t(a varchar(256)); + + +mysql> insert into test.t values('one week’s time test'); +mysql> insert into test.t values('abc测试def'); +mysql> insert into test.t values('abcテストabc'); +mysql> insert into test.t values('ѐёђѓєѕіїјљњћќѝўџ'); +mysql> insert into test.t values('+ѐ-ё*ђ/ѓ!є@ѕ#і@ї%ј……љ&њ(ћ)ќ¥ѝ#ў@џ!^'); +mysql> insert into test.t values('αβγδεζηθικλμνξοπρστυφχψωσ'); +mysql> insert into test.t values('▲α▼βγ➨δε☎ζη✂θι€κλ♫μν✓ξο✚πρ℉στ♥υφ♖χψ♘ω★σ✕'); +mysql> insert into test.t values('թփձջրչճժծքոեռտըւիօպասդֆգհյկլխզղցվբնմշ'); +mysql> insert into test.t values(NULL); +mysql> alter table test.t set tiflash replica 1; +func> wait_table test t + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select reverse(a) from test.t; ++-------------------------------------------------------------------------------------------------+ +| reverse(a) | ++-------------------------------------------------------------------------------------------------+ +| tset emit s’keew eno | +| fed试测cba | +| cbaトステcba | +| џўѝќћњљјїіѕєѓђёѐ | +| ^!џ@ў#ѝ¥ќ)ћ(њ&љ……ј%ї@і#ѕ@є!ѓ/ђ*ё-ѐ+ | +| σωψχφυτσρποξνμλκιθηζεδγβα | +| ✕σ★ω♘ψχ♖φυ♥τσ℉ρπ✚οξ✓νμ♫λκ€ιθ✂ηζ☎εδ➨γβ▼α▲ | +| շմնբվցղզխլկյհգֆդսապօիւըտռեոքծժճչրջձփթ | +| NULL | ++-------------------------------------------------------------------------------------------------+ From 97342db2a1dd705b1bd98f3481d51f48a4f0300d Mon Sep 17 00:00:00 2001 From: Zhigao Tong Date: Thu, 7 Jul 2022 19:57:02 +0800 Subject: [PATCH 089/104] Optimize comparision for collation `UTF8_BIN` and `UTF8MB4_BIN` (#5299) ref pingcap/tiflash#5294 --- dbms/src/Columns/ColumnConst.h | 3 +- .../Functions/CollationOperatorOptimized.h | 210 ++++++++++++++++++ dbms/src/Functions/FunctionsComparison.h | 54 ++++- dbms/src/Storages/Transaction/Collator.cpp | 36 ++- .../tidb-ci/new_collation_fullstack/expr.test | 14 ++ 5 files changed, 287 insertions(+), 30 deletions(-) create mode 100644 dbms/src/Functions/CollationOperatorOptimized.h diff --git a/dbms/src/Columns/ColumnConst.h b/dbms/src/Columns/ColumnConst.h index 27283c0f24a..da071507a72 100644 --- a/dbms/src/Columns/ColumnConst.h +++ b/dbms/src/Columns/ColumnConst.h @@ -233,7 +233,8 @@ class ColumnConst final : public COWPtrHelper template T getValue() const { - return getField().safeGet::Type>(); + auto && tmp = getField(); + return std::move(tmp.safeGet::Type>()); } }; diff --git a/dbms/src/Functions/CollationOperatorOptimized.h b/dbms/src/Functions/CollationOperatorOptimized.h new file mode 100644 index 00000000000..395ecc5b9eb --- /dev/null +++ b/dbms/src/Functions/CollationOperatorOptimized.h @@ -0,0 +1,210 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#pragma once + +#include +#include +#include +#include +#include + +#include +#include + + +namespace DB +{ + +template +ALWAYS_INLINE inline int signum(T val) +{ + return (0 < val) - (val < 0); +} + +// Check equality is much faster than other comparison. +// - check size first +// - return 0 if equal else 1 +__attribute__((flatten, always_inline, pure)) inline uint8_t RawStrEqualCompare(const std::string_view & lhs, const std::string_view & rhs) +{ + return StringRef(lhs) == StringRef(rhs) ? 0 : 1; +} + +// Compare str view by memcmp +__attribute__((flatten, always_inline, pure)) inline int RawStrCompare(const std::string_view & v1, const std::string_view & v2) +{ + return signum(v1.compare(v2)); +} + +constexpr char SPACE = ' '; + +// Remove tail space +__attribute__((flatten, always_inline, pure)) inline std::string_view RightTrim(const std::string_view & v) +{ + if (likely(v.empty() || v.back() != SPACE)) + return v; + size_t end = v.find_last_not_of(SPACE); + return end == std::string_view::npos ? std::string_view{} : std::string_view(v.data(), end + 1); +} + +__attribute__((flatten, always_inline, pure)) inline int RtrimStrCompare(const std::string_view & va, const std::string_view & vb) +{ + return RawStrCompare(RightTrim(va), RightTrim(vb)); +} + +// If true, only need to check equal or not. +template +struct IsEqualRelated +{ + static constexpr const bool value = false; +}; + +// For `EqualsOp` and `NotEqualsOp`, value is true. +template +struct IsEqualRelated> +{ + static constexpr const bool value = true; +}; +template +struct IsEqualRelated> +{ + static constexpr const bool value = true; +}; + +// Loop columns and invoke callback for each pair. +template +__attribute__((flatten, always_inline)) inline void LoopTwoColumns( + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const ColumnString::Chars_t & b_data, + const ColumnString::Offsets & b_offsets, + size_t size, + F && func) +{ + for (size_t i = 0; i < size; ++i) + { + size_t a_size = StringUtil::sizeAt(a_offsets, i) - 1; + size_t b_size = StringUtil::sizeAt(b_offsets, i) - 1; + const auto * a_ptr = reinterpret_cast(&a_data[StringUtil::offsetAt(a_offsets, i)]); + const auto * b_ptr = reinterpret_cast(&b_data[StringUtil::offsetAt(b_offsets, i)]); + + func({a_ptr, a_size}, {b_ptr, b_size}, i); + } +} + +// Loop one column and invoke callback for each pair. +template +__attribute__((flatten, always_inline)) inline void LoopOneColumn( + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + size_t size, + F && func) +{ + for (size_t i = 0; i < size; ++i) + { + size_t a_size = StringUtil::sizeAt(a_offsets, i) - 1; + const auto * a_ptr = reinterpret_cast(&a_data[StringUtil::offsetAt(a_offsets, i)]); + + func({a_ptr, a_size}, i); + } +} + +// Handle str-column compare str-column. +// - Optimize UTF8_BIN and UTF8MB4_BIN +// - Check if columns do NOT contain tail space +// - If Op is `EqualsOp` or `NotEqualsOp`, optimize comparison by faster way +template +ALWAYS_INLINE inline bool StringVectorStringVector( + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const ColumnString::Chars_t & b_data, + const ColumnString::Offsets & b_offsets, + const TiDB::TiDBCollatorPtr & collator, + Result & c) +{ + bool use_optimized_path = false; + + switch (collator->getCollatorId()) + { + case TiDB::ITiDBCollator::UTF8MB4_BIN: + case TiDB::ITiDBCollator::UTF8_BIN: + { + size_t size = a_offsets.size(); + + LoopTwoColumns(a_data, a_offsets, b_data, b_offsets, size, [&c](const std::string_view & va, const std::string_view & vb, size_t i) { + if constexpr (IsEqualRelated::value) + { + c[i] = Op::apply(RawStrEqualCompare(RightTrim(va), RightTrim(vb)), 0); + } + else + { + c[i] = Op::apply(RtrimStrCompare(va, vb), 0); + } + }); + + use_optimized_path = true; + + break; + } + default: + break; + } + return use_optimized_path; +} + +// Handle str-column compare const-str. +// - Optimize UTF8_BIN and UTF8MB4_BIN +// - Right trim const-str first +// - Check if column does NOT contain tail space +// - If Op is `EqualsOp` or `NotEqualsOp`, optimize comparison by faster way +template +ALWAYS_INLINE inline bool StringVectorConstant( + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const std::string_view & b, + const TiDB::TiDBCollatorPtr & collator, + Result & c) +{ + bool use_optimized_path = false; + + switch (collator->getCollatorId()) + { + case TiDB::ITiDBCollator::UTF8MB4_BIN: + case TiDB::ITiDBCollator::UTF8_BIN: + { + size_t size = a_offsets.size(); + + std::string_view tar_str_view = RightTrim(b); // right trim const-str first + + LoopOneColumn(a_data, a_offsets, size, [&c, &tar_str_view](const std::string_view & view, size_t i) { + if constexpr (IsEqualRelated::value) + { + c[i] = Op::apply(RawStrEqualCompare(RightTrim(view), tar_str_view), 0); + } + else + { + c[i] = Op::apply(RawStrCompare(RightTrim(view), tar_str_view), 0); + } + }); + + use_optimized_path = true; + break; + } + default: + break; + } + return use_optimized_path; +} + +} // namespace DB diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index 1c63a286452..8f7502fba85 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -33,6 +33,7 @@ #include #include #include +#include #include #include #include @@ -301,6 +302,12 @@ struct StringComparisonWithCollatorImpl const TiDB::TiDBCollatorPtr & collator, PaddedPODArray & c) { + bool optimized_path = StringVectorStringVector(a_data, a_offsets, b_data, b_offsets, collator, c); + if (optimized_path) + { + return; + } + size_t size = a_offsets.size(); for (size_t i = 0; i < size; ++i) @@ -317,10 +324,17 @@ struct StringComparisonWithCollatorImpl static void NO_INLINE stringVectorConstant( const ColumnString::Chars_t & a_data, const ColumnString::Offsets & a_offsets, - const std::string & b, + const std::string_view & b, const TiDB::TiDBCollatorPtr & collator, PaddedPODArray & c) { + bool optimized_path = StringVectorConstant(a_data, a_offsets, b, collator, c); + + if (optimized_path) + { + return; + } + size_t size = a_offsets.size(); ColumnString::Offset b_size = b.size(); const char * b_data = reinterpret_cast(b.data()); @@ -332,7 +346,7 @@ struct StringComparisonWithCollatorImpl } static void constantStringVector( - const std::string & a, + const std::string_view & a, const ColumnString::Chars_t & b_data, const ColumnString::Offsets & b_offsets, const TiDB::TiDBCollatorPtr & collator, @@ -342,8 +356,8 @@ struct StringComparisonWithCollatorImpl } static void constantConstant( - const std::string & a, - const std::string & b, + const std::string_view & a, + const std::string_view & b, const TiDB::TiDBCollatorPtr & collator, ResultType & c) { @@ -706,6 +720,25 @@ class FunctionComparison : public IFunction } } + static inline std::string_view genConstStrRef(const ColumnConst * c0_const) + { + std::string_view c0_const_str_ref{}; + if (c0_const) + { + if (const auto * c0_const_string = checkAndGetColumn(&c0_const->getDataColumn()); c0_const_string) + { + c0_const_str_ref = std::string_view(c0_const_string->getDataAt(0)); + } + else if (const auto * c0_const_fixed_string = checkAndGetColumn(&c0_const->getDataColumn()); c0_const_fixed_string) + { + c0_const_str_ref = std::string_view(c0_const_fixed_string->getDataAt(0)); + } + else + throw Exception("Logical error: ColumnConst contains not String nor FixedString column", ErrorCodes::ILLEGAL_COLUMN); + } + return c0_const_str_ref; + } + template bool executeStringWithCollator( Block & block, @@ -720,10 +753,13 @@ class FunctionComparison : public IFunction using ResultType = typename ResultColumnType::value_type; using StringImpl = StringComparisonWithCollatorImpl, ResultType>; + std::string_view c0_const_str_ref = genConstStrRef(c0_const); + std::string_view c1_const_str_ref = genConstStrRef(c1_const); + if (c0_const && c1_const) { ResultType res = 0; - StringImpl::constantConstant(c0_const->getValue(), c1_const->getValue(), collator, res); + StringImpl::constantConstant(c0_const_str_ref, c1_const_str_ref, collator, res); block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(c0_const->size(), toField(res)); return true; } @@ -745,12 +781,12 @@ class FunctionComparison : public IFunction StringImpl::stringVectorConstant( c0_string->getChars(), c0_string->getOffsets(), - c1_const->getValue(), + c1_const_str_ref, collator, c_res->getData()); else if (c0_const && c1_string) StringImpl::constantStringVector( - c0_const->getValue(), + c0_const_str_ref, c1_string->getChars(), c1_string->getOffsets(), collator, @@ -770,8 +806,8 @@ class FunctionComparison : public IFunction template bool executeString(Block & block, size_t result, const IColumn * c0, const IColumn * c1) const { - const ColumnString * c0_string = checkAndGetColumn(c0); - const ColumnString * c1_string = checkAndGetColumn(c1); + const auto * c0_string = checkAndGetColumn(c0); + const auto * c1_string = checkAndGetColumn(c1); const ColumnConst * c0_const = checkAndGetColumnConstStringOrFixedString(c0); const ColumnConst * c1_const = checkAndGetColumnConstStringOrFixedString(c1); diff --git a/dbms/src/Storages/Transaction/Collator.cpp b/dbms/src/Storages/Transaction/Collator.cpp index a9b4d0784be..1b0221a6829 100644 --- a/dbms/src/Storages/Transaction/Collator.cpp +++ b/dbms/src/Storages/Transaction/Collator.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include @@ -29,17 +30,10 @@ TiDBCollators dummy_collators; std::vector dummy_sort_key_contaners; std::string dummy_sort_key_contaner; -std::string_view rtrim(const char * s, size_t length) +ALWAYS_INLINE std::string_view rtrim(const char * s, size_t length) { auto v = std::string_view(s, length); - size_t end = v.find_last_not_of(' '); - return end == std::string_view::npos ? "" : v.substr(0, end + 1); -} - -template -int signum(T val) -{ - return (0 < val) - (val < 0); + return DB::RightTrim(v); } using Rune = int32_t; @@ -183,26 +177,26 @@ class Pattern : public ITiDBCollator::IPattern }; template -class BinCollator : public ITiDBCollator +class BinCollator final : public ITiDBCollator { public: explicit BinCollator(int32_t id) : ITiDBCollator(id) {} + int compare(const char * s1, size_t length1, const char * s2, size_t length2) const override { if constexpr (padding) - return signum(rtrim(s1, length1).compare(rtrim(s2, length2))); + return DB::RtrimStrCompare({s1, length1}, {s2, length2}); else - return signum(std::string_view(s1, length1).compare(std::string_view(s2, length2))); + return DB::RawStrCompare({s1, length1}, {s2, length2}); } StringRef sortKey(const char * s, size_t length, std::string &) const override { if constexpr (padding) { - auto v = rtrim(s, length); - return StringRef(v.data(), v.length()); + return StringRef(rtrim(s, length)); } else { @@ -249,7 +243,7 @@ using WeightType = uint16_t; extern const std::array weight_lut; } // namespace GeneralCI -class GeneralCICollator : public ITiDBCollator +class GeneralCICollator final : public ITiDBCollator { public: explicit GeneralCICollator(int32_t id) @@ -270,7 +264,7 @@ class GeneralCICollator : public ITiDBCollator auto sk2 = weight(c2); auto cmp = sk1 - sk2; if (cmp != 0) - return signum(cmp); + return DB::signum(cmp); } return (offset1 < v1.length()) - (offset2 < v2.length()); @@ -365,7 +359,7 @@ const std::array weight_lut_long = { } // namespace UnicodeCI -class UnicodeCICollator : public ITiDBCollator +class UnicodeCICollator final : public ITiDBCollator { public: explicit UnicodeCICollator(int32_t id) @@ -420,7 +414,7 @@ class UnicodeCICollator : public ITiDBCollator } else { - return signum(static_cast(s1_first & 0xFFFF) - static_cast(s2_first & 0xFFFF)); + return DB::signum(static_cast(s1_first & 0xFFFF) - static_cast(s2_first & 0xFFFF)); } } } @@ -593,6 +587,8 @@ class UnicodeCICollator : public ITiDBCollator friend class Pattern; }; +using UTF8MB4_BIN_TYPE = BinCollator; + TiDBCollatorPtr ITiDBCollator::getCollator(int32_t id) { switch (id) @@ -607,10 +603,10 @@ TiDBCollatorPtr ITiDBCollator::getCollator(int32_t id) static const auto latin1_collator = BinCollator(LATIN1_BIN); return &latin1_collator; case ITiDBCollator::UTF8MB4_BIN: - static const auto utf8mb4_collator = BinCollator(UTF8MB4_BIN); + static const auto utf8mb4_collator = UTF8MB4_BIN_TYPE(UTF8MB4_BIN); return &utf8mb4_collator; case ITiDBCollator::UTF8_BIN: - static const auto utf8_collator = BinCollator(UTF8_BIN); + static const auto utf8_collator = UTF8MB4_BIN_TYPE(UTF8_BIN); return &utf8_collator; case ITiDBCollator::UTF8_GENERAL_CI: static const auto utf8_general_ci_collator = GeneralCICollator(UTF8_GENERAL_CI); diff --git a/tests/tidb-ci/new_collation_fullstack/expr.test b/tests/tidb-ci/new_collation_fullstack/expr.test index 15ada0f335c..1e2135c4f2d 100644 --- a/tests/tidb-ci/new_collation_fullstack/expr.test +++ b/tests/tidb-ci/new_collation_fullstack/expr.test @@ -35,6 +35,13 @@ mysql> set session tidb_isolation_read_engines='tiflash'; select /*+ read_from_s | 2 | abc | +------+-------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select /*+ read_from_storage(tiflash[t]) */ id, value1 from test.t where value1 = 'abc '; ++------+-------+ +| id | value1| ++------+-------+ +| 1 | abc | +| 2 | abc | ++------+-------+ mysql> set session tidb_isolation_read_engines='tiflash'; select /*+ read_from_storage(tiflash[t]) */ id, value from test.t where value like 'aB%'; +------+-------+ @@ -62,6 +69,13 @@ mysql> set session tidb_isolation_read_engines='tiflash'; select /*+ read_from_s | 3 | def | +------+-------+ +mysql> set session tidb_isolation_read_engines='tiflash'; select /*+ read_from_storage(tiflash[t]) */ id, value1 from test.t where value1 = 'def '; ++------+-------+ +| id | value1| ++------+-------+ +| 3 | def | ++------+-------+ + mysql> set session tidb_isolation_read_engines='tiflash'; select /*+ read_from_storage(tiflash[t]) */ id, value1 from test.t where value1 in ('Abc','def'); +------+-------+ | id | value1| From 47657d34c75d8993c598217f7d96b190666e5f55 Mon Sep 17 00:00:00 2001 From: hongyunyan <649330952@qq.com> Date: Thu, 7 Jul 2022 20:37:02 +0800 Subject: [PATCH 090/104] feat : support set tiflash mode ddl action (#5256) ref pingcap/tiflash#5252 --- dbms/src/Debug/DBGInvoker.cpp | 2 + dbms/src/Debug/dbgFuncSchemaName.cpp | 52 +++++++++++++ dbms/src/Debug/dbgFuncSchemaName.h | 10 +++ dbms/src/Storages/Transaction/TiDB.cpp | 42 ++++++++++ dbms/src/Storages/Transaction/TiDB.h | 11 +++ .../Transaction/tests/gtest_table_info.cpp | 34 ++++---- dbms/src/TiDB/Schema/SchemaBuilder.cpp | 77 +++++++++++++++++++ dbms/src/TiDB/Schema/SchemaBuilder.h | 4 + dbms/src/TiDB/Schema/SchemaGetter.h | 5 +- ...alter_table_tiflash_replica_and_mode.test} | 34 ++++++++ .../ddl/alter_tiflash_mode.test | 48 ++++++++++++ 11 files changed, 301 insertions(+), 18 deletions(-) rename tests/fullstack-test2/ddl/{alter_table_tiflash_replica.test => alter_table_tiflash_replica_and_mode.test} (60%) create mode 100644 tests/fullstack-test2/ddl/alter_tiflash_mode.test diff --git a/dbms/src/Debug/DBGInvoker.cpp b/dbms/src/Debug/DBGInvoker.cpp index 3654a437bf7..df993d8e6e9 100644 --- a/dbms/src/Debug/DBGInvoker.cpp +++ b/dbms/src/Debug/DBGInvoker.cpp @@ -120,6 +120,8 @@ DBGInvoker::DBGInvoker() regSchemafulFunc("query_mapped", dbgFuncQueryMapped); regSchemalessFunc("get_tiflash_replica_count", dbgFuncGetTiflashReplicaCount); regSchemalessFunc("get_partition_tables_tiflash_replica_count", dbgFuncGetPartitionTablesTiflashReplicaCount); + regSchemalessFunc("get_tiflash_mode", dbgFuncGetTiflashMode); + regSchemalessFunc("get_partition_tables_tiflash_mode", dbgFuncGetPartitionTablesTiflashMode); regSchemalessFunc("search_log_for_key", dbgFuncSearchLogForKey); regSchemalessFunc("tidb_dag", dbgFuncTiDBQueryFromNaturalDag); diff --git a/dbms/src/Debug/dbgFuncSchemaName.cpp b/dbms/src/Debug/dbgFuncSchemaName.cpp index 5f10da6685d..3aa7b6e3af4 100644 --- a/dbms/src/Debug/dbgFuncSchemaName.cpp +++ b/dbms/src/Debug/dbgFuncSchemaName.cpp @@ -181,4 +181,56 @@ void dbgFuncGetPartitionTablesTiflashReplicaCount(Context & context, const ASTs output(fmt_buf.toString()); } +void dbgFuncGetTiflashMode(Context & context, const ASTs & args, DBGInvoker::Printer output) +{ + if (args.empty() || args.size() != 2) + throw Exception("Args not matched, should be: database-name[, table-name]", ErrorCodes::BAD_ARGUMENTS); + + const String & database_name = typeid_cast(*args[0]).name; + FmtBuffer fmt_buf; + + const String & table_name = typeid_cast(*args[1]).name; + auto mapped = mappedTable(context, database_name, table_name); + auto storage = context.getTable(mapped->first, mapped->second); + auto managed_storage = std::dynamic_pointer_cast(storage); + if (!managed_storage) + throw Exception(database_name + "." + table_name + " is not ManageableStorage", ErrorCodes::BAD_ARGUMENTS); + + fmt_buf.append((TiFlashModeToString(managed_storage->getTableInfo().tiflash_mode))); + + output(fmt_buf.toString()); +} + +void dbgFuncGetPartitionTablesTiflashMode(Context & context, const ASTs & args, DBGInvoker::Printer output) +{ + if (args.empty() || args.size() != 2) + throw Exception("Args not matched, should be: database-name[, table-name]", ErrorCodes::BAD_ARGUMENTS); + + const String & database_name = typeid_cast(*args[0]).name; + FmtBuffer fmt_buf; + + const String & table_name = typeid_cast(*args[1]).name; + auto mapped = mappedTable(context, database_name, table_name); + auto storage = context.getTable(mapped->first, mapped->second); + auto managed_storage = std::dynamic_pointer_cast(storage); + if (!managed_storage) + throw Exception(database_name + "." + table_name + " is not ManageableStorage", ErrorCodes::BAD_ARGUMENTS); + + auto table_info = managed_storage->getTableInfo(); + + if (!table_info.isLogicalPartitionTable()) + throw Exception(database_name + "." + table_name + " is not logical partition table", ErrorCodes::BAD_ARGUMENTS); + + SchemaNameMapper name_mapper; + for (const auto & part_def : table_info.partition.definitions) + { + auto paritition_table_info = table_info.producePartitionTableInfo(part_def.id, name_mapper); + auto partition_storage = context.getTMTContext().getStorages().get(paritition_table_info->id); + fmt_buf.append((TiFlashModeToString(partition_storage->getTableInfo().tiflash_mode))); + fmt_buf.append("/"); + } + + output(fmt_buf.toString()); +} + } // namespace DB diff --git a/dbms/src/Debug/dbgFuncSchemaName.h b/dbms/src/Debug/dbgFuncSchemaName.h index 2a31b3a7f6d..ec18f89e911 100644 --- a/dbms/src/Debug/dbgFuncSchemaName.h +++ b/dbms/src/Debug/dbgFuncSchemaName.h @@ -50,4 +50,14 @@ void dbgFuncGetTiflashReplicaCount(Context & context, const ASTs & args, DBGInvo // ./storage-client.sh "DBGInvoke get_partition_tables_tiflash_replica_count(db_name, table_name)" void dbgFuncGetPartitionTablesTiflashReplicaCount(Context & context, const ASTs & args, DBGInvoker::Printer output); +// Get table's tiflash mode with mapped table name +// Usage: +// ./storage-client.sh "DBGInvoke get_tiflash_mode(db_name, table_name)" +void dbgFuncGetTiflashMode(Context & context, const ASTs & args, DBGInvoker::Printer output); + +// Get the logical table's partition tables' tiflash replica counts with mapped table name +// Usage: +// ./storage-client.sh "DBGInvoke get_partition_tables_tiflash_mode(db_name, table_name)" +void dbgFuncGetPartitionTablesTiflashMode(Context & context, const ASTs & args, DBGInvoker::Printer output); + } // namespace DB diff --git a/dbms/src/Storages/Transaction/TiDB.cpp b/dbms/src/Storages/Transaction/TiDB.cpp index dc7f1f3e348..6d07c47f235 100644 --- a/dbms/src/Storages/Transaction/TiDB.cpp +++ b/dbms/src/Storages/Transaction/TiDB.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include @@ -772,6 +773,37 @@ catch (const Poco::Exception & e) DB::Exception(e)); } +String TiFlashModeToString(TiFlashMode tiflash_mode) +{ + switch (tiflash_mode) + { + case TiFlashMode::Normal: + return ""; + case TiFlashMode::Fast: + return "fast"; + default: + LOG_FMT_WARNING(&Poco::Logger::get("TiDB"), "TiFlashModeToString with invalid tiflash mode {}", tiflash_mode); + return ""; + } +} + +TiFlashMode parseTiFlashMode(String mode_str) +{ + if (mode_str.empty()) + { + return TiFlashMode::Normal; + } + else if (mode_str == "fast") + { + return TiFlashMode::Fast; + } + else + { + throw DB::Exception( + std::string(__PRETTY_FUNCTION__) + + " ParseTiFlashMode Failed. mode " + mode_str + " is unvalid, please set mode as fast/normal"); + } +} /////////////////////// ////// TableInfo ////// /////////////////////// @@ -840,6 +872,8 @@ try json->set("tiflash_replica", replica_info.getJSONObject()); + json->set("tiflash_mode", std::string(TiFlashModeToString(tiflash_mode))); + json->stringify(buf); return buf.str(); @@ -926,6 +960,14 @@ try replica_info.deserialize(replica_obj); } } + if (obj->has("tiflash_mode")) + { + auto mode = obj->getValue("tiflash_mode"); + if (!mode.empty()) + { + tiflash_mode = parseTiFlashMode(mode); + } + } if (is_common_handle && index_infos.size() != 1) { throw DB::Exception( diff --git a/dbms/src/Storages/Transaction/TiDB.h b/dbms/src/Storages/Transaction/TiDB.h index 4c28a614857..a9d46b60c13 100644 --- a/dbms/src/Storages/Transaction/TiDB.h +++ b/dbms/src/Storages/Transaction/TiDB.h @@ -333,6 +333,12 @@ struct IndexInfo bool is_global; }; +enum class TiFlashMode +{ + Normal, + Fast, +}; + struct TableInfo { TableInfo() = default; @@ -382,6 +388,8 @@ struct TableInfo // The TiFlash replica info persisted by TiDB TiFlashReplicaInfo replica_info; + TiFlashMode tiflash_mode = TiFlashMode::Normal; + ::TiDB::StorageEngine engine_type = ::TiDB::StorageEngine::UNSPECIFIED; ColumnID getColumnID(const String & name) const; @@ -413,4 +421,7 @@ String genJsonNull(); tipb::FieldType columnInfoToFieldType(const ColumnInfo & ci); ColumnInfo fieldTypeToColumnInfo(const tipb::FieldType & field_type); +String TiFlashModeToString(TiFlashMode tiflash_mode); +TiFlashMode parseTiFlashMode(String mode_str); + } // namespace TiDB diff --git a/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp b/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp index 516a173b151..871153cb0e9 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_table_info.cpp @@ -42,7 +42,7 @@ struct ParseCase std::function check; }; -TEST(TiDBTableInfo_test, ParseFromJSON) +TEST(TiDBTableInfoTest, ParseFromJSON) try { auto cases = { @@ -136,54 +136,54 @@ struct StmtCase } }; -TEST(TiDBTableInfo_test, GenCreateTableStatement) +TEST(TiDBTableInfoTest, GenCreateTableStatement) try { auto cases = // {StmtCase{ 1145, // R"json({"id":1939,"db_name":{"O":"customer","L":"customer"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":1145,"name":{"O":"customerdebt","L":"customerdebt"},"cols":[{"id":1,"name":{"O":"id","L":"id"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"type":{"Tp":8,"Flag":515,"Flen":20,"Decimal":0},"state":5,"comment":"i\"d"}],"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"负债信息","partition":null})json", // - R"stmt(CREATE TABLE `customer`.`customerdebt`(`id` Int64) Engine = DeltaMerge((`id`), '{"cols":[{"comment":"i\\"d","default":null,"default_bit":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":515,"Flen":20,"Tp":8}}],"comment":"\\u8D1F\\u503A\\u4FE1\\u606F","id":1145,"index_info":[],"is_common_handle":false,"name":{"L":"customerdebt","O":"customerdebt"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":0}'))stmt", // + R"json({"id":1145,"name":{"O":"customerdebt","L":"customerdebt"},"cols":[{"id":1,"name":{"O":"id","L":"id"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"type":{"Tp":8,"Flag":515,"Flen":20,"Decimal":0},"state":5,"comment":"i\"d"}],"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"负债信息","partition":null,"tiflash_mode":"fast"})json", // + R"stmt(CREATE TABLE `customer`.`customerdebt`(`id` Int64) Engine = DeltaMerge((`id`), '{"cols":[{"comment":"i\\"d","default":null,"default_bit":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":515,"Flen":20,"Tp":8}}],"comment":"\\u8D1F\\u503A\\u4FE1\\u606F","id":1145,"index_info":[],"is_common_handle":false,"name":{"L":"customerdebt","O":"customerdebt"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_mode":"fast","tiflash_replica":{"Count":0},"update_timestamp":0}'))stmt", // }, StmtCase{ 2049, // R"json({"id":1939,"db_name":{"O":"customer","L":"customer"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":2049,"name":{"O":"customerdebt","L":"customerdebt"},"cols":[{"id":1,"name":{"O":"id","L":"id"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"type":{"Tp":8,"Flag":515,"Flen":20,"Decimal":0},"state":5,"comment":"i\"d"}],"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"负债信息","update_timestamp":404545295996944390,"partition":null})json", // - R"stmt(CREATE TABLE `customer`.`customerdebt`(`id` Int64) Engine = DeltaMerge((`id`), '{"cols":[{"comment":"i\\"d","default":null,"default_bit":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":515,"Flen":20,"Tp":8}}],"comment":"\\u8D1F\\u503A\\u4FE1\\u606F","id":2049,"index_info":[],"is_common_handle":false,"name":{"L":"customerdebt","O":"customerdebt"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}'))stmt", // + R"json({"id":2049,"name":{"O":"customerdebt","L":"customerdebt"},"cols":[{"id":1,"name":{"O":"id","L":"id"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"type":{"Tp":8,"Flag":515,"Flen":20,"Decimal":0},"state":5,"comment":"i\"d"}],"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"负债信息","update_timestamp":404545295996944390,"partition":null,"tiflash_mode":""})json", // + R"stmt(CREATE TABLE `customer`.`customerdebt`(`id` Int64) Engine = DeltaMerge((`id`), '{"cols":[{"comment":"i\\"d","default":null,"default_bit":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":null,"Collate":null,"Decimal":0,"Elems":null,"Flag":515,"Flen":20,"Tp":8}}],"comment":"\\u8D1F\\u503A\\u4FE1\\u606F","id":2049,"index_info":[],"is_common_handle":false,"name":{"L":"customerdebt","O":"customerdebt"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_mode":"","tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}'))stmt", // }, StmtCase{ 31, // R"json({"id":1,"db_name":{"O":"db1","L":"db1"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":31,"name":{"O":"simple_t","L":"simple_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545295996944390,"ShardRowIDBits":0,"partition":null})json", // - R"stmt(CREATE TABLE `db1`.`simple_t`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":31,"index_info":[],"is_common_handle":false,"name":{"L":"simple_t","O":"simple_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}'))stmt", // + R"json({"id":31,"name":{"O":"simple_t","L":"simple_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545295996944390,"ShardRowIDBits":0,"partition":null,"tiflash_mode":""})json", // + R"stmt(CREATE TABLE `db1`.`simple_t`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":31,"index_info":[],"is_common_handle":false,"name":{"L":"simple_t","O":"simple_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_mode":"","tiflash_replica":{"Count":0},"update_timestamp":404545295996944390}'))stmt", // }, StmtCase{ 33, // R"json({"id":2,"db_name":{"O":"db2","L":"db2"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":33,"name":{"O":"pk_t","L":"pk_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":3,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545312978108418,"ShardRowIDBits":0,"partition":null})json", // - R"stmt(CREATE TABLE `db2`.`pk_t`(`i` Int32) Engine = DeltaMerge((`i`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":3,"Flen":11,"Tp":3}}],"comment":"","id":33,"index_info":[],"is_common_handle":false,"name":{"L":"pk_t","O":"pk_t"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545312978108418}'))stmt", // + R"json({"id":33,"name":{"O":"pk_t","L":"pk_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":3,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545312978108418,"ShardRowIDBits":0,"partition":null,"tiflash_mode":""})json", // + R"stmt(CREATE TABLE `db2`.`pk_t`(`i` Int32) Engine = DeltaMerge((`i`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":3,"Flen":11,"Tp":3}}],"comment":"","id":33,"index_info":[],"is_common_handle":false,"name":{"L":"pk_t","O":"pk_t"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_mode":"","tiflash_replica":{"Count":0},"update_timestamp":404545312978108418}'))stmt", // }, StmtCase{ 35, // R"json({"id":1,"db_name":{"O":"db1","L":"db1"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":35,"name":{"O":"not_null_t","L":"not_null_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":4097,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545324922961926,"ShardRowIDBits":0,"partition":null})json", // - R"stmt(CREATE TABLE `db1`.`not_null_t`(`i` Int32, `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":35,"index_info":[],"is_common_handle":false,"name":{"L":"not_null_t","O":"not_null_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404545324922961926}'))stmt", // + R"json({"id":35,"name":{"O":"not_null_t","L":"not_null_t"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":4097,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":""}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":404545324922961926,"ShardRowIDBits":0,"partition":null,"tiflash_mode":""})json", // + R"stmt(CREATE TABLE `db1`.`not_null_t`(`i` Int32, `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":35,"index_info":[],"is_common_handle":false,"name":{"L":"not_null_t","O":"not_null_t"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_mode":"","tiflash_replica":{"Count":0},"update_timestamp":404545324922961926}'))stmt", // }, StmtCase{ 37, // R"json({"id":2,"db_name":{"O":"db2","L":"db2"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", - R"json({"id":37,"name":{"O":"mytable","L":"mytable"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"mycol","L":"mycol"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":15,"Flag":4099,"Flen":256,"Decimal":0,"Charset":"utf8","Collate":"utf8_bin","Elems":null},"state":5,"comment":""}],"index_info":[{"id":1,"idx_name":{"O":"PRIMARY","L":"primary"},"tbl_name":{"O":"","L":""},"idx_cols":[{"name":{"O":"mycol","L":"mycol"},"offset":0,"length":-1}],"is_unique":true,"is_primary":true,"state":5,"comment":"","index_type":1}],"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":1,"update_timestamp":404566455285710853,"ShardRowIDBits":0,"partition":null})json", // - R"stmt(CREATE TABLE `db2`.`mytable`(`mycol` String) Engine = DeltaMerge((`mycol`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"mycol","O":"mycol"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"utf8","Collate":"utf8_bin","Decimal":0,"Elems":null,"Flag":4099,"Flen":256,"Tp":15}}],"comment":"","id":37,"index_info":[],"is_common_handle":false,"name":{"L":"mytable","O":"mytable"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":404566455285710853}'))stmt", // + R"json({"id":37,"name":{"O":"mytable","L":"mytable"},"charset":"","collate":"","cols":[{"id":1,"name":{"O":"mycol","L":"mycol"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":15,"Flag":4099,"Flen":256,"Decimal":0,"Charset":"utf8","Collate":"utf8_bin","Elems":null},"state":5,"comment":""}],"index_info":[{"id":1,"idx_name":{"O":"PRIMARY","L":"primary"},"tbl_name":{"O":"","L":""},"idx_cols":[{"name":{"O":"mycol","L":"mycol"},"offset":0,"length":-1}],"is_unique":true,"is_primary":true,"state":5,"comment":"","index_type":1}],"fk_info":null,"state":5,"pk_is_handle":true,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":1,"update_timestamp":404566455285710853,"ShardRowIDBits":0,"partition":null,"tiflash_mode":""})json", // + R"stmt(CREATE TABLE `db2`.`mytable`(`mycol` String) Engine = DeltaMerge((`mycol`), '{"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"mycol","O":"mycol"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"utf8","Collate":"utf8_bin","Decimal":0,"Elems":null,"Flag":4099,"Flen":256,"Tp":15}}],"comment":"","id":37,"index_info":[],"is_common_handle":false,"name":{"L":"mytable","O":"mytable"},"partition":null,"pk_is_handle":true,"schema_version":-1,"state":5,"tiflash_mode":"","tiflash_replica":{"Count":0},"update_timestamp":404566455285710853}'))stmt", // }, StmtCase{ 32, // R"json({"id":1,"db_name":{"O":"test","L":"test"},"charset":"utf8mb4","collate":"utf8mb4_bin","state":5})json", // - R"json({"id":31,"name":{"O":"range_part_t","L":"range_part_t"},"charset":"utf8mb4","collate":"utf8mb4_bin","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","version":0}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":407445773801488390,"ShardRowIDBits":0,"partition":{"type":1,"expr":"`i`","columns":null,"enable":true,"definitions":[{"id":32,"name":{"O":"p0","L":"p0"},"less_than":["0"]},{"id":33,"name":{"O":"p1","L":"p1"},"less_than":["100"]}],"num":0},"compression":"","version":1})json", // - R"stmt(CREATE TABLE `test`.`range_part_t_32`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"belonging_table_id":31,"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":32,"index_info":[],"is_common_handle":false,"is_partition_sub_table":true,"name":{"L":"range_part_t_32","O":"range_part_t_32"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_replica":{"Count":0},"update_timestamp":407445773801488390}'))stmt", // + R"json({"id":31,"name":{"O":"range_part_t","L":"range_part_t"},"charset":"utf8mb4","collate":"utf8mb4_bin","cols":[{"id":1,"name":{"O":"i","L":"i"},"offset":0,"origin_default":null,"default":null,"default_bit":null,"generated_expr_string":"","generated_stored":false,"dependences":null,"type":{"Tp":3,"Flag":0,"Flen":11,"Decimal":0,"Charset":"binary","Collate":"binary","Elems":null},"state":5,"comment":"","version":0}],"index_info":null,"fk_info":null,"state":5,"pk_is_handle":false,"schema_version":-1,"comment":"","auto_inc_id":0,"max_col_id":1,"max_idx_id":0,"update_timestamp":407445773801488390,"ShardRowIDBits":0,"partition":{"type":1,"expr":"`i`","columns":null,"enable":true,"definitions":[{"id":32,"name":{"O":"p0","L":"p0"},"less_than":["0"]},{"id":33,"name":{"O":"p1","L":"p1"},"less_than":["100"]}],"num":0},"compression":"","version":1,"tiflash_mode":""})json", // + R"stmt(CREATE TABLE `test`.`range_part_t_32`(`i` Nullable(Int32), `_tidb_rowid` Int64) Engine = DeltaMerge((`_tidb_rowid`), '{"belonging_table_id":31,"cols":[{"comment":"","default":null,"default_bit":null,"id":1,"name":{"L":"i","O":"i"},"offset":0,"origin_default":null,"state":5,"type":{"Charset":"binary","Collate":"binary","Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":32,"index_info":[],"is_common_handle":false,"is_partition_sub_table":true,"name":{"L":"range_part_t_32","O":"range_part_t_32"},"partition":null,"pk_is_handle":false,"schema_version":-1,"state":5,"tiflash_mode":"","tiflash_replica":{"Count":0},"update_timestamp":407445773801488390}'))stmt", // }}; - for (auto & c : cases) + for (const auto & c : cases) { c.verifyTableInfo(); } diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.cpp b/dbms/src/TiDB/Schema/SchemaBuilder.cpp index abb575c75bc..6e4ad10e344 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.cpp +++ b/dbms/src/TiDB/Schema/SchemaBuilder.cpp @@ -36,6 +36,7 @@ #include #include #include +#include #include #include #include @@ -542,6 +543,11 @@ void SchemaBuilder::applyDiff(const SchemaDiff & diff) applySetTiFlashReplica(db_info, diff.table_id); break; } + case SchemaActionType::SetTiFlashMode: + { + applySetTiFlashMode(db_info, diff.table_id); + break; + } default: { if (diff.type < SchemaActionType::MaxRecognizedType) @@ -1258,6 +1264,75 @@ void SchemaBuilder::applySetTiFlashReplicaOnPhysicalTable( LOG_FMT_INFO(log, "Updated replica info for {}", name_mapper.debugCanonicalName(*db_info, table_info)); } + +template +void SchemaBuilder::applySetTiFlashMode(const TiDB::DBInfoPtr & db_info, TableID table_id) +{ + auto latest_table_info = getter.getTableInfo(db_info->id, table_id); + + if (unlikely(latest_table_info == nullptr)) + { + throw TiFlashException(fmt::format("miss table in TiKV : {}", table_id), Errors::DDL::StaleSchema); + } + + auto & tmt_context = context.getTMTContext(); + auto storage = tmt_context.getStorages().get(latest_table_info->id); + if (unlikely(storage == nullptr)) + { + throw TiFlashException(fmt::format("miss table in TiFlash : {}", name_mapper.debugCanonicalName(*db_info, *latest_table_info)), + Errors::DDL::MissingTable); + } + + applySetTiFlashModeOnLogicalTable(db_info, latest_table_info, storage); +} + +template +void SchemaBuilder::applySetTiFlashModeOnLogicalTable( + const TiDB::DBInfoPtr & db_info, + const TiDB::TableInfoPtr & table_info, + const ManageableStoragePtr & storage) +{ + applySetTiFlashModeOnPhysicalTable(db_info, table_info, storage); + + if (table_info->isLogicalPartitionTable()) + { + auto & tmt_context = context.getTMTContext(); + for (const auto & part_def : table_info->partition.definitions) + { + auto new_part_table_info = table_info->producePartitionTableInfo(part_def.id, name_mapper); + auto part_storage = tmt_context.getStorages().get(table_info->id); + if (unlikely(part_storage == nullptr)) + { + throw TiFlashException(fmt::format("miss table in TiFlash : {}", name_mapper.debugCanonicalName(*db_info, *new_part_table_info)), + Errors::DDL::MissingTable); + } + applySetTiFlashModeOnPhysicalTable(db_info, new_part_table_info, part_storage); + } + } +} + + +template +void SchemaBuilder::applySetTiFlashModeOnPhysicalTable( + const TiDB::DBInfoPtr & db_info, + const TiDB::TableInfoPtr & latest_table_info, + const ManageableStoragePtr & storage) +{ + if (storage->getTableInfo().tiflash_mode == latest_table_info->tiflash_mode) + return; + + TiDB::TableInfo table_info = storage->getTableInfo(); + table_info.tiflash_mode = latest_table_info->tiflash_mode; + AlterCommands commands; + + LOG_FMT_INFO(log, "Updating tiflash mode for {} to {}", name_mapper.debugCanonicalName(*db_info, table_info), TiFlashModeToString(table_info.tiflash_mode)); + + auto alter_lock = storage->lockForAlter(getThreadName()); + storage->alterFromTiDB(alter_lock, commands, name_mapper.mapDatabaseName(*db_info), table_info, name_mapper, context); + LOG_FMT_INFO(log, "Updated tiflash mode for {} to {}", name_mapper.debugCanonicalName(*db_info, table_info), TiFlashModeToString(table_info.tiflash_mode)); +} + + template void SchemaBuilder::syncAllSchema() { @@ -1327,6 +1402,8 @@ void SchemaBuilder::syncAllSchema() applyRenameLogicalTable(db, table, storage); /// Update replica info if needed. applySetTiFlashReplicaOnLogicalTable(db, table, storage); + /// Update tiflash mode if needed. + applySetTiFlashModeOnLogicalTable(db, table, storage); /// Alter if needed. applyAlterLogicalTable(db, table, storage); LOG_FMT_DEBUG(log, "Table {} synced during sync all schemas", name_mapper.debugCanonicalName(*db, *table)); diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.h b/dbms/src/TiDB/Schema/SchemaBuilder.h index 461d7ff9c12..827203a682f 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.h +++ b/dbms/src/TiDB/Schema/SchemaBuilder.h @@ -89,6 +89,10 @@ struct SchemaBuilder void applySetTiFlashReplica(const TiDB::DBInfoPtr & db_info, TableID table_id); void applySetTiFlashReplicaOnLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); void applySetTiFlashReplicaOnPhysicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); + + void applySetTiFlashMode(const TiDB::DBInfoPtr & db_info, TableID table_id); + void applySetTiFlashModeOnLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); + void applySetTiFlashModeOnPhysicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); }; } // namespace DB diff --git a/dbms/src/TiDB/Schema/SchemaGetter.h b/dbms/src/TiDB/Schema/SchemaGetter.h index fe0ecd59af0..72fd00678f7 100644 --- a/dbms/src/TiDB/Schema/SchemaGetter.h +++ b/dbms/src/TiDB/Schema/SchemaGetter.h @@ -94,11 +94,14 @@ enum class SchemaActionType : Int8 AlterTableStatsOptions = 58, AlterNoCacheTable = 59, CreateTables = 60, + ActionMultiSchemaChange = 61, + SetTiFlashMode = 62, + // If we supporte new type from TiDB. // MaxRecognizedType also needs to be changed. // It should always be equal to the maximum supported type + 1 - MaxRecognizedType = 61, + MaxRecognizedType = 63, }; struct AffectedOption diff --git a/tests/fullstack-test2/ddl/alter_table_tiflash_replica.test b/tests/fullstack-test2/ddl/alter_table_tiflash_replica_and_mode.test similarity index 60% rename from tests/fullstack-test2/ddl/alter_table_tiflash_replica.test rename to tests/fullstack-test2/ddl/alter_table_tiflash_replica_and_mode.test index 1262cd60882..5e43936379b 100644 --- a/tests/fullstack-test2/ddl/alter_table_tiflash_replica.test +++ b/tests/fullstack-test2/ddl/alter_table_tiflash_replica_and_mode.test @@ -24,9 +24,26 @@ func> wait_table test t │ 1 │ └────────────────────────────────────┘ +# test tiflash mode in normal mode +>> DBGInvoke get_tiflash_mode("test", "t") +┌─get_tiflash_mode(test, t)─┐ +│ │ +└───────────────────────────┘ + +mysql> alter table test.t set tiflash mode fast + +>> DBGInvoke __refresh_schemas() + +# test tiflash mode in fast mode +>> DBGInvoke get_tiflash_mode("test", "t") +┌─get_tiflash_mode(test, t)───┐ +│ fast │ +└─────────────────────────────┘ + # test replica for partition tables mysql> drop table if exists test.t mysql> create table test.t (x int) partition by range (x) (partition p0 values less than (5), partition p1 values less than (10)); +mysql> alter table test.t set tiflash mode fast mysql> alter table test.t set tiflash replica 1 func> wait_table test t @@ -36,11 +53,22 @@ func> wait_table test t │ 1 │ └────────────────────────────────────┘ +>> DBGInvoke get_tiflash_mode("test", "t") +┌─get_tiflash_mode(test, t)──────────┐ +│ fast │ +└────────────────────────────────────┘ + >> DBGInvoke get_partition_tables_tiflash_replica_count("test", "t") ┌─get_partition_tables_tiflash_replica_count(test, t)─┐ │ 1/1/ │ └─────────────────────────────────────────────────────┘ +# test tiflash mode for partition tables +>> DBGInvoke get_partition_tables_tiflash_mode("test", "t") +┌─get_partition_tables_tiflash_mode(test, t)─┐ +│ fast/fast/ │ +└────────────────────────────────────────────┘ + # test replica for add partition tables after set replica mysql> alter table test.t add partition (partition p2 values less than (2010)); @@ -51,5 +79,11 @@ mysql> alter table test.t add partition (partition p2 values less than (2010)); │ 1/1/1/ │ └─────────────────────────────────────────────────────┘ +# test tiflash mode for add partition tables after set replica +>> DBGInvoke get_partition_tables_tiflash_mode("test", "t") +┌─get_partition_tables_tiflash_mode(test, t)─┐ +│ fast/fast/fast/ │ +└────────────────────────────────────────────┘ + diff --git a/tests/fullstack-test2/ddl/alter_tiflash_mode.test b/tests/fullstack-test2/ddl/alter_tiflash_mode.test new file mode 100644 index 00000000000..c9f3ef488c4 --- /dev/null +++ b/tests/fullstack-test2/ddl/alter_tiflash_mode.test @@ -0,0 +1,48 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +mysql> drop table if exists test.t +mysql> create table test.t(a int, b int) +mysql> alter table test.t set tiflash replica 1 + +func> wait_table test t + +# check default mode of tiflash table +mysql> select table_schema,table_name,replica_count,available,table_mode from information_schema.tiflash_replica where table_schema='test' and table_name='t'; ++--------------+------------+---------------+-----------+-----------+ +| table_schema | table_name | replica_count | available | table_mode| ++--------------+------------+---------------+-----------+-----------+ +| test | t | 1 | 1 | NORMAL | ++--------------+------------+---------------+-----------+-----------+ + +# check change mode + +mysql> alter table test.t set tiflash mode fast +mysql> select table_schema,table_name,replica_count,available,table_mode from information_schema.tiflash_replica where table_schema='test' and table_name='t'; ++--------------+------------+---------------+-----------+-----------+ +| table_schema | table_name | replica_count | available | table_mode| ++--------------+------------+---------------+-----------+-----------+ +| test | t | 1 | 1 | FAST | ++--------------+------------+---------------+-----------+-----------+ + +# check change mode +mysql> alter table test.t set tiflash mode normal +mysql> select table_schema,table_name,replica_count,available,table_mode from information_schema.tiflash_replica where table_schema='test' and table_name='t'; ++--------------+------------+---------------+-----------+-----------+ +| table_schema | table_name | replica_count | available | table_mode| ++--------------+------------+---------------+-----------+-----------+ +| test | t | 1 | 1 | NORMAL | ++--------------+------------+---------------+-----------+-----------+ + +mysql> drop table if exists test.t \ No newline at end of file From cbc6a95312903ca5b931221c129425c7cf4c76fc Mon Sep 17 00:00:00 2001 From: Liqi Geng Date: Thu, 7 Jul 2022 23:25:02 +0800 Subject: [PATCH 091/104] Add non-blocking functions for MPMCQueue (#5311) close pingcap/tiflash#5310 --- dbms/src/Common/MPMCQueue.h | 95 ++++++++++++------- dbms/src/Common/tests/gtest_mpmc_queue.cpp | 25 ++--- dbms/src/Common/tests/mpmc_queue_perftest.cpp | 2 +- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 2 +- 4 files changed, 78 insertions(+), 46 deletions(-) diff --git a/dbms/src/Common/MPMCQueue.h b/dbms/src/Common/MPMCQueue.h index f550ecc7ca2..31dfc65a174 100644 --- a/dbms/src/Common/MPMCQueue.h +++ b/dbms/src/Common/MPMCQueue.h @@ -74,56 +74,80 @@ class MPMCQueue destruct(getObj(read_pos)); } - /// Block util: + /// Block until: /// 1. Pop succeeds with a valid T: return true. /// 2. The queue is cancelled or finished: return false. - bool pop(T & obj) + ALWAYS_INLINE bool pop(T & obj) { - return popObj(obj); + return popObj(obj); } - /// Besides all conditions mentioned at `pop`, `tryPop` will return false if `timeout` is exceeded. + /// Besides all conditions mentioned at `pop`, `popTimeout` will return false if `timeout` is exceeded. template - bool tryPop(T & obj, const Duration & timeout) + ALWAYS_INLINE bool popTimeout(T & obj, const Duration & timeout) { /// std::condition_variable::wait_until will always use system_clock. auto deadline = std::chrono::system_clock::now() + timeout; - return popObj(obj, &deadline); + return popObj(obj, &deadline); } - /// Block util: + /// Non-blocking function. + /// Return true if pop succeed. + /// else return false. + ALWAYS_INLINE bool tryPop(T & obj) + { + return popObj(obj); + } + + /// Block until: /// 1. Push succeeds and return true. /// 2. The queue is cancelled and return false. /// 3. The queue has finished and return false. template ALWAYS_INLINE bool push(U && u) { - return pushObj(std::forward(u)); + return pushObj(std::forward(u)); } - /// Besides all conditions mentioned at `push`, `tryPush` will return false if `timeout` is exceeded. + /// Besides all conditions mentioned at `push`, `pushTimeout` will return false if `timeout` is exceeded. template - ALWAYS_INLINE bool tryPush(U && u, const Duration & timeout) + ALWAYS_INLINE bool pushTimeout(U && u, const Duration & timeout) { /// std::condition_variable::wait_until will always use system_clock. auto deadline = std::chrono::system_clock::now() + timeout; - return pushObj(std::forward(u), &deadline); + return pushObj(std::forward(u), &deadline); + } + + /// Non-blocking function. + /// Return true if push succeed. + /// else return false. + template + ALWAYS_INLINE bool tryPush(U && u) + { + return pushObj(std::forward(u)); } /// The same as `push` except it will construct the object in place. template ALWAYS_INLINE bool emplace(Args &&... args) { - return emplaceObj(nullptr, std::forward(args)...); + return emplaceObj(nullptr, std::forward(args)...); } - /// The same as `tryPush` except it will construct the object in place. + /// The same as `pushTimeout` except it will construct the object in place. template - ALWAYS_INLINE bool tryEmplace(Args &&... args, const Duration & timeout) + ALWAYS_INLINE bool emplaceTimeout(Args &&... args, const Duration & timeout) { /// std::condition_variable::wait_until will always use system_clock. auto deadline = std::chrono::system_clock::now() + timeout; - return emplaceObj(&deadline, std::forward(args)...); + return emplaceObj(&deadline, std::forward(args)...); + } + + /// The same as `tryPush` except it will construct the object in place. + template + ALWAYS_INLINE bool tryEmplace(Args &&... args) + { + return emplaceObj(nullptr, std::forward(args)...); } /// Cancel a NORMAL queue will wake up all blocking readers and writers. @@ -233,7 +257,8 @@ class MPMCQueue } } - bool popObj(T & res, const TimePoint * deadline = nullptr) + template + bool popObj(T & res, [[maybe_unused]] const TimePoint * deadline = nullptr) { #ifdef __APPLE__ WaitingNode node; @@ -241,14 +266,16 @@ class MPMCQueue thread_local WaitingNode node; #endif { - /// read_pos < write_pos means the queue isn't empty - auto pred = [&] { - return read_pos < write_pos || !isNormal(); - }; - std::unique_lock lock(mu); - wait(lock, reader_head, node, pred, deadline); + if constexpr (need_wait) + { + /// read_pos < write_pos means the queue isn't empty + auto pred = [&] { + return read_pos < write_pos || !isNormal(); + }; + wait(lock, reader_head, node, pred, deadline); + } if (!isCancelled() && read_pos < write_pos) { @@ -272,21 +299,23 @@ class MPMCQueue return false; } - template - bool assignObj(const TimePoint * deadline, F && assigner) + template + bool assignObj([[maybe_unused]] const TimePoint * deadline, F && assigner) { #ifdef __APPLE__ WaitingNode node; #else thread_local WaitingNode node; #endif - auto pred = [&] { - return write_pos - read_pos < capacity || !isNormal(); - }; - std::unique_lock lock(mu); - wait(lock, writer_head, node, pred, deadline); + if constexpr (need_wait) + { + auto pred = [&] { + return write_pos - read_pos < capacity || !isNormal(); + }; + wait(lock, writer_head, node, pred, deadline); + } /// double check status after potential wait /// check write_pos because timeouted will also reach here. @@ -305,16 +334,16 @@ class MPMCQueue return false; } - template + template ALWAYS_INLINE bool pushObj(U && u, const TimePoint * deadline = nullptr) { - return assignObj(deadline, [&](void * addr) { new (addr) T(std::forward(u)); }); + return assignObj(deadline, [&](void * addr) { new (addr) T(std::forward(u)); }); } - template + template ALWAYS_INLINE bool emplaceObj(const TimePoint * deadline, Args &&... args) { - return assignObj(deadline, [&](void * addr) { new (addr) T(std::forward(args)...); }); + return assignObj(deadline, [&](void * addr) { new (addr) T(std::forward(args)...); }); } ALWAYS_INLINE bool isNormal() const diff --git a/dbms/src/Common/tests/gtest_mpmc_queue.cpp b/dbms/src/Common/tests/gtest_mpmc_queue.cpp index 85ad1892067..3f2748b452b 100644 --- a/dbms/src/Common/tests/gtest_mpmc_queue.cpp +++ b/dbms/src/Common/tests/gtest_mpmc_queue.cpp @@ -98,12 +98,14 @@ class MPMCQueueTest : public ::testing::Test void testCannotTryPush(MPMCQueue & queue) { auto old_size = queue.size(); - auto res = queue.tryPush(ValueHelper::make(-1), std::chrono::microseconds(1)); - auto new_size = queue.size(); - if (res) + bool ok1 = queue.tryPush(ValueHelper::make(-1)); + auto new_size1 = queue.size(); + bool ok2 = queue.pushTimeout(ValueHelper::make(-1), std::chrono::microseconds(1)); + auto new_size2 = queue.size(); + if (ok1 || ok2) throw TiFlashTestException("Should push fail"); - if (old_size != new_size) - throw TiFlashTestException(fmt::format("Size changed from {} to {} without push", old_size, new_size)); + if (old_size != new_size1 || old_size != new_size2) + throw TiFlashTestException(fmt::format("Size changed from {} to {} and {} without push", old_size, new_size1, new_size2)); } template @@ -124,12 +126,14 @@ class MPMCQueueTest : public ::testing::Test { auto old_size = queue.size(); T res; - bool ok = queue.tryPop(res, std::chrono::microseconds(1)); - auto new_size = queue.size(); - if (ok) + bool ok1 = queue.tryPop(res); + auto new_size1 = queue.size(); + bool ok2 = queue.popTimeout(res, std::chrono::microseconds(1)); + auto new_size2 = queue.size(); + if (ok1 || ok2) throw TiFlashTestException("Should pop fail"); - if (old_size != new_size) - throw TiFlashTestException(fmt::format("Size changed from {} to {} without pop", old_size, new_size)); + if (old_size != new_size1 || old_size != new_size2) + throw TiFlashTestException(fmt::format("Size changed from {} to {} and {} without pop", old_size, new_size1, new_size2)); } template @@ -474,7 +478,6 @@ class MPMCQueueTest : public ::testing::Test throwOrMove(std::move(rhs)); } - ThrowInjectable & operator=(ThrowInjectable && rhs) { if (this != &rhs) diff --git a/dbms/src/Common/tests/mpmc_queue_perftest.cpp b/dbms/src/Common/tests/mpmc_queue_perftest.cpp index d047b5d498f..ba0d00001a3 100644 --- a/dbms/src/Common/tests/mpmc_queue_perftest.cpp +++ b/dbms/src/Common/tests/mpmc_queue_perftest.cpp @@ -87,7 +87,7 @@ struct Helper> template static void pushOneTo(MPMCQueue & queue, U && data) { - queue.tryPush(std::forward(data), std::chrono::milliseconds(1)); + queue.pushTimeout(std::forward(data), std::chrono::milliseconds(1)); } }; diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 3b36adf2c40..966babb832f 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -424,7 +424,7 @@ void ExchangeReceiverBase::reactor(const std::vector & asyn for (Int32 i = 0; i < check_waiting_requests_freq; ++i) { AsyncHandler * handler = nullptr; - if (unlikely(!ready_requests.tryPop(handler, timeout))) + if (unlikely(!ready_requests.popTimeout(handler, timeout))) break; handler->handle(); From e58a007b48308369caa5fd458469c86fbd069602 Mon Sep 17 00:00:00 2001 From: hehechen Date: Fri, 8 Jul 2022 10:57:02 +0800 Subject: [PATCH 092/104] add random segment test for CI weekly (#5300) close pingcap/tiflash#5301 --- .../Storages/DeltaMerge/tests/gtest_segment.cpp | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp b/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp index 1c68ba3bb2a..dc43ef3713b 100644 --- a/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp +++ b/dbms/src/Storages/DeltaMerge/tests/gtest_segment.cpp @@ -75,12 +75,26 @@ CATCH TEST_F(SegmentOperationTest, TestSegmentRandom) try { + srand(time(nullptr)); SegmentTestOptions options; options.is_common_handle = true; reloadWithOptions(options); randomSegmentTest(100); } CATCH + +// run in CI weekly +TEST_F(SegmentOperationTest, DISABLED_TestSegmentRandomForCI) +try +{ + srand(time(nullptr)); + SegmentTestOptions options; + options.is_common_handle = true; + reloadWithOptions(options); + randomSegmentTest(10000); +} +CATCH + } // namespace tests } // namespace DM } // namespace DB From b822e9913a908a97b07ca86cb056aafc1c105d2b Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Thu, 7 Jul 2022 20:59:03 -0700 Subject: [PATCH 093/104] *: tidy FunctionString.cpp (#5312) close pingcap/tiflash#5313 --- dbms/src/Functions/FunctionsString.cpp | 102 ++++++++++++------------- 1 file changed, 51 insertions(+), 51 deletions(-) diff --git a/dbms/src/Functions/FunctionsString.cpp b/dbms/src/Functions/FunctionsString.cpp index b9f20e45134..ad75c69a090 100644 --- a/dbms/src/Functions/FunctionsString.cpp +++ b/dbms/src/Functions/FunctionsString.cpp @@ -992,7 +992,7 @@ class FunctionStringOrArrayToT : public IFunction void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override { const ColumnPtr column = block.getByPosition(arguments[0]).column; - if (const ColumnString * col = checkAndGetColumn(column.get())) + if (const auto * col = checkAndGetColumn(column.get())) { auto col_res = ColumnVector::create(); @@ -1002,7 +1002,7 @@ class FunctionStringOrArrayToT : public IFunction block.getByPosition(result).column = std::move(col_res); } - else if (const ColumnFixedString * col = checkAndGetColumn(column.get())) + else if (const auto * col = checkAndGetColumn(column.get())) { if (Impl::is_fixed_to_constant) { @@ -1022,7 +1022,7 @@ class FunctionStringOrArrayToT : public IFunction block.getByPosition(result).column = std::move(col_res); } } - else if (const ColumnArray * col = checkAndGetColumn(column.get())) + else if (const auto * col = checkAndGetColumn(column.get())) { auto col_res = ColumnVector::create(); @@ -1081,13 +1081,13 @@ class FunctionReverse : public IFunction void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override { const ColumnPtr column = block.getByPosition(arguments[0]).column; - if (const ColumnString * col = checkAndGetColumn(column.get())) + if (const auto * col = checkAndGetColumn(column.get())) { auto col_res = ColumnString::create(); ReverseImpl::vector(col->getChars(), col->getOffsets(), col_res->getChars(), col_res->getOffsets()); block.getByPosition(result).column = std::move(col_res); } - else if (const ColumnFixedString * col = checkAndGetColumn(column.get())) + else if (const auto * col = checkAndGetColumn(column.get())) { auto col_res = ColumnFixedString::create(col->getN()); ReverseImpl::vectorFixed(col->getChars(), col->getN(), col_res->getChars()); @@ -1131,7 +1131,7 @@ class FunctionJsonLength : public IFunction void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override { const ColumnPtr column = block.getByPosition(arguments[0]).column; - if (const ColumnString * col = checkAndGetColumn(column.get())) + if (const auto * col = checkAndGetColumn(column.get())) { auto col_res = ColumnUInt64::create(); typename ColumnUInt64::Container & vec_col_res = col_res->getData(); @@ -1232,8 +1232,8 @@ class ConcatImpl : public IFunction const IColumn * c0 = block.getByPosition(arguments[0]).column.get(); const IColumn * c1 = block.getByPosition(arguments[1]).column.get(); - const ColumnString * c0_string = checkAndGetColumn(c0); - const ColumnString * c1_string = checkAndGetColumn(c1); + const auto * c0_string = checkAndGetColumn(c0); + const auto * c1_string = checkAndGetColumn(c1); const ColumnConst * c0_const_string = checkAndGetColumnConst(c0); const ColumnConst * c1_const_string = checkAndGetColumnConst(c1); @@ -1552,7 +1552,7 @@ class FunctionSubstring : public IFunction if (number_of_arguments == 3) column_length = block.getByPosition(arguments[2]).column; - const ColumnConst * column_start_const = checkAndGetColumn(column_start.get()); + const auto * column_start_const = checkAndGetColumn(column_start.get()); const ColumnConst * column_length_const = nullptr; if (number_of_arguments == 3) @@ -1572,9 +1572,9 @@ class FunctionSubstring : public IFunction throw Exception("Third argument provided for function substring could not be negative.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); } - if (const ColumnString * col = checkAndGetColumn(column_string.get())) + if (const auto * col = checkAndGetColumn(column_string.get())) executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, length_value, block, result, StringSource(*col)); - else if (const ColumnFixedString * col = checkAndGetColumn(column_string.get())) + else if (const auto * col = checkAndGetColumn(column_string.get())) executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, length_value, block, result, FixedStringSource(*col)); else if (const ColumnConst * col = checkAndGetColumnConst(column_string.get())) executeForSource(column_start, column_length, column_start_const, column_length_const, start_value, length_value, block, result, ConstSource(*col)); @@ -1676,7 +1676,7 @@ class FunctionSubstringUTF8 : public IFunction return true; } - const ColumnString * col = checkAndGetColumn(column_string.get()); + const auto * col = checkAndGetColumn(column_string.get()); assert(col); auto col_res = ColumnString::create(); getVectorConstConstFunc(implicit_length, is_positive)(col->getChars(), col->getOffsets(), start_abs, length, col_res->getChars(), col_res->getOffsets()); @@ -1732,7 +1732,7 @@ class FunctionSubstringUTF8 : public IFunction // convert to vector if string is const. ColumnPtr full_column_string = column_string->isColumnConst() ? column_string->convertToFullColumnIfConst() : column_string; - const ColumnString * col = checkAndGetColumn(full_column_string.get()); + const auto * col = checkAndGetColumn(full_column_string.get()); assert(col); auto col_res = ColumnString::create(); if (implicit_length) @@ -1869,7 +1869,7 @@ class FunctionRightUTF8 : public IFunction using LengthFieldType = typename LengthType::FieldType; auto col_res = ColumnString::create(); - if (const ColumnString * col_string = checkAndGetColumn(column_string.get())) + if (const auto * col_string = checkAndGetColumn(column_string.get())) { if (column_length->isColumnConst()) { @@ -1897,7 +1897,7 @@ class FunctionRightUTF8 : public IFunction else if (const ColumnConst * col_const_string = checkAndGetColumnConst(column_string.get())) { // const vector - const ColumnString * col_string_from_const = checkAndGetColumn(col_const_string->getDataColumnPtr().get()); + const auto * col_string_from_const = checkAndGetColumn(col_const_string->getDataColumnPtr().get()); assert(col_string_from_const); // When useDefaultImplementationForConstants is true, string and length are not both constants assert(!column_length->isColumnConst()); @@ -1993,7 +1993,7 @@ class FunctionAppendTrailingCharIfAbsent : public IFunction if (!checkColumnConst(column_char.get())) throw Exception(fmt::format("Second argument of function {} must be a constant string", getName()), ErrorCodes::ILLEGAL_COLUMN); - String trailing_char_str = static_cast(*column_char).getValue(); + auto trailing_char_str = static_cast(*column_char).getValue(); if (trailing_char_str.size() != 1) throw Exception(fmt::format("Second argument of function {} must be a one-character string", getName()), ErrorCodes::BAD_ARGUMENTS); @@ -2101,7 +2101,7 @@ class TrimImpl : public IFunction void executeTrim(Block & block, const ColumnNumbers & arguments, const size_t result) const { const IColumn * c0 = block.getByPosition(arguments[0]).column.get(); - const ColumnString * c0_string = checkAndGetColumn(c0); + const auto * c0_string = checkAndGetColumn(c0); const ColumnConst * c0_const_string = checkAndGetColumnConst(c0); auto c_res = ColumnString::create(); @@ -2121,8 +2121,8 @@ class TrimImpl : public IFunction const IColumn * c0 = block.getByPosition(arguments[0]).column.get(); const IColumn * c1 = block.getByPosition(arguments[1]).column.get(); - const ColumnString * c0_string = checkAndGetColumn(c0); - const ColumnString * c1_string = checkAndGetColumn(c1); + const auto * c0_string = checkAndGetColumn(c0); + const auto * c1_string = checkAndGetColumn(c1); const ColumnConst * c0_const_string = checkAndGetColumnConst(c0); const ColumnConst * c1_const_string = checkAndGetColumnConst(c1); @@ -2202,7 +2202,7 @@ class TrimUTF8Impl : public IFunction void executeTrim(Block & block, const ColumnNumbers & arguments, const size_t result) const { const IColumn * c0 = block.getByPosition(arguments[0]).column.get(); - const ColumnString * c0_string = checkAndGetColumn(c0); + const auto * c0_string = checkAndGetColumn(c0); const ColumnConst * c0_const_string = checkAndGetColumnConst(c0); auto c_res = ColumnString::create(); @@ -2225,7 +2225,7 @@ class TrimUTF8Impl : public IFunction const IColumn * c0 = block.getByPosition(arguments[0]).column.get(); const IColumn * c1 = block.getByPosition(arguments[1]).column.get(); - const ColumnString * c0_string = checkAndGetColumn(c0); + const auto * c0_string = checkAndGetColumn(c0); const ColumnConst * c0_const_string = checkAndGetColumnConst(c0); const ColumnConst * c1_const_string = checkAndGetColumnConst(c1); const auto * column_trim_string = checkAndGetColumn(c1_const_string->getDataColumnPtr().get()); @@ -2716,7 +2716,7 @@ class FunctionTiDBTrim : public IFunction ColumnPtr & column_data = block.getByPosition(arguments[0]).column; auto res_col = ColumnString::create(); - const ColumnString * data_col = checkAndGetColumn(column_data.get()); + const auto * data_col = checkAndGetColumn(column_data.get()); static constexpr std::string_view default_rem = " "; static const auto * remstr_ptr = reinterpret_cast(default_rem.data()); @@ -2738,25 +2738,25 @@ class FunctionTiDBTrim : public IFunction if (data_const && !remstr_const) { const ColumnConst * data_col = checkAndGetColumnConst(column_data.get()); - const ColumnString * remstr_col = checkAndGetColumn(column_remstr.get()); + const auto * remstr_col = checkAndGetColumn(column_remstr.get()); - const std::string data = data_col->getValue(); + const auto data = data_col->getValue(); const auto * data_ptr = reinterpret_cast(data.c_str()); constVector(is_ltrim, is_rtrim, data_ptr, data.size() + 1, remstr_col->getChars(), remstr_col->getOffsets(), res_col->getChars(), res_col->getOffsets()); } else if (remstr_const && !data_const) { const ColumnConst * remstr_col = checkAndGetColumnConst(column_remstr.get()); - const ColumnString * data_col = checkAndGetColumn(column_data.get()); + const auto * data_col = checkAndGetColumn(column_data.get()); - const std::string remstr = remstr_col->getValue(); + const auto remstr = remstr_col->getValue(); const auto * remstr_ptr = reinterpret_cast(remstr.c_str()); vectorConst(is_ltrim, is_rtrim, data_col->getChars(), data_col->getOffsets(), remstr_ptr, remstr.size() + 1, res_col->getChars(), res_col->getOffsets()); } else { - const ColumnString * data_col = checkAndGetColumn(column_data.get()); - const ColumnString * remstr_col = checkAndGetColumn(column_remstr.get()); + const auto * data_col = checkAndGetColumn(column_data.get()); + const auto * remstr_col = checkAndGetColumn(column_remstr.get()); vectorVector(is_ltrim, is_rtrim, data_col->getChars(), data_col->getOffsets(), remstr_col->getChars(), remstr_col->getOffsets(), res_col->getChars(), res_col->getOffsets()); } @@ -2769,7 +2769,7 @@ class FunctionTiDBTrim : public IFunction ColumnPtr & column_direction = block.getByPosition(arguments[2]).column; if (!column_direction->isColumnConst()) throw Exception(fmt::format("3nd argument of function {} must be constant.", getName())); - const ColumnConst * direction_col = checkAndGetColumn(column_direction.get()); + const auto * direction_col = checkAndGetColumn(column_direction.get()); static constexpr Int64 trim_both_default = 0; // trims from both direction by default static constexpr Int64 trim_both = 1; // trims from both direction with explicit notation @@ -2989,7 +2989,7 @@ class TidbPadImpl { continue; } - int32_t len = static_cast(column_length->getInt(i)); + auto len = static_cast(column_length->getInt(i)); if (len <= 0) { len = 0; @@ -3051,7 +3051,7 @@ class TidbPadImpl } else { - const ColumnString * column_string = checkAndGetColumn(column_string_ptr.get()); + const auto * column_string = checkAndGetColumn(column_string_ptr.get()); const ColumnString::Offsets & string_offsets = column_string->getOffsets(); const ColumnString::Chars_t & string_data = column_string->getChars(); @@ -3233,7 +3233,7 @@ class TidbPadImpl return true; } - ColumnString::Offset tmp_target_len = static_cast(target_len); + auto tmp_target_len = static_cast(target_len); ColumnString::Offset per_pad_offset = 0; ColumnString::Offset pad_bytes = 0; ColumnString::Offset left = 0; @@ -3300,7 +3300,7 @@ class TidbPadImpl return true; } - ColumnString::Offset tmp_target_len = static_cast(target_len); + auto tmp_target_len = static_cast(target_len); if (data_len < tmp_target_len) { ColumnString::Offset left = tmp_target_len - data_len; @@ -3421,7 +3421,7 @@ class PadImpl : public IFunction ColumnPtr column_length = block.getByPosition(arguments[1]).column; ColumnPtr column_padding = block.getByPosition(arguments[2]).column; - const ColumnConst * column_length_const = checkAndGetColumn(column_length.get()); + const auto * column_length_const = checkAndGetColumn(column_length.get()); const ColumnConst * column_padding_const = checkAndGetColumnConst(column_padding.get()); Int64 length_value = 0; @@ -3441,7 +3441,7 @@ class PadImpl : public IFunction auto c_res = ColumnString::create(); - if (const ColumnString * col = checkAndGetColumn(column_string.get())) + if (const auto * col = checkAndGetColumn(column_string.get())) pad, StringSink>( StringSource(*col), ConstSource(*column_padding_const), @@ -3548,7 +3548,7 @@ class PadUTF8Impl : public IFunction ColumnPtr column_length = block.getByPosition(arguments[1]).column; ColumnPtr column_padding = block.getByPosition(arguments[2]).column; - const ColumnConst * column_length_const = checkAndGetColumn(column_length.get()); + const auto * column_length_const = checkAndGetColumn(column_length.get()); const ColumnConst * column_padding_const = checkAndGetColumnConst(column_padding.get()); Int64 length_value = 0; @@ -3568,7 +3568,7 @@ class PadUTF8Impl : public IFunction auto c_res = ColumnString::create(); const auto * column_padding_string = checkAndGetColumn(column_padding_const->getDataColumnPtr().get()); - if (const ColumnString * col = checkAndGetColumn(column_string.get())) + if (const auto * col = checkAndGetColumn(column_string.get())) vector(col->getChars(), col->getOffsets(), length_value, column_padding_string->getChars(), column_padding_string->getOffsets(), c_res->getChars(), c_res->getOffsets()); else if (const ColumnConst * col = checkAndGetColumnConst(column_string.get())) { @@ -4114,8 +4114,8 @@ class FunctionASCII : public IFunction void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override { const IColumn * c0_col = block.getByPosition(arguments[0]).column.get(); - const ColumnConst * c0_const = checkAndGetColumn(c0_col); - const ColumnString * c0_string = checkAndGetColumn(c0_col); + const auto * c0_const = checkAndGetColumn(c0_col); + const auto * c0_string = checkAndGetColumn(c0_col); Field res_field; int val_num = c0_col->size(); @@ -4165,8 +4165,8 @@ class FunctionLength : public IFunction void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override { const IColumn * c0_col = block.getByPosition(arguments[0]).column.get(); - const ColumnConst * c0_const = checkAndGetColumn(c0_col); - const ColumnString * c0_string = checkAndGetColumn(c0_col); + const auto * c0_const = checkAndGetColumn(c0_col); + const auto * c0_string = checkAndGetColumn(c0_col); Field res_field; int val_num = c0_col->size(); @@ -4215,13 +4215,13 @@ class FunctionPosition : public IFunction void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override { const IColumn * c0_col = block.getByPosition(arguments[0]).column.get(); - const ColumnConst * c0_const = checkAndGetColumn(c0_col); - const ColumnString * c0_string = checkAndGetColumn(c0_col); + const auto * c0_const = checkAndGetColumn(c0_col); + const auto * c0_string = checkAndGetColumn(c0_col); Field c0_field; const IColumn * c1_col = block.getByPosition(arguments[1]).column.get(); - const ColumnConst * c1_const = checkAndGetColumn(c1_col); - const ColumnString * c1_string = checkAndGetColumn(c1_col); + const auto * c1_const = checkAndGetColumn(c1_col); + const auto * c1_string = checkAndGetColumn(c1_col); Field c1_field; if ((c0_const == nullptr && c0_string == nullptr) || (c1_const == nullptr && c1_string == nullptr)) @@ -4331,7 +4331,7 @@ class FunctionSubStringIndex : public IFunction column_str = column_str->isColumnConst() ? column_str->convertToFullColumnIfConst() : column_str; if (delim_const && count_const) { - const ColumnString * str_col = checkAndGetColumn(column_str.get()); + const auto * str_col = checkAndGetColumn(column_str.get()); const ColumnConst * delim_col = checkAndGetColumnConst(column_delim.get()); const ColumnConst * count_col = checkAndGetColumnConst>(column_count.get()); if (str_col == nullptr || delim_col == nullptr || count_col == nullptr) @@ -4339,7 +4339,7 @@ class FunctionSubStringIndex : public IFunction return false; } auto col_res = ColumnString::create(); - IntType count = count_col->getValue(); + auto count = count_col->getValue(); vectorConstConst( str_col->getChars(), str_col->getOffsets(), @@ -4353,9 +4353,9 @@ class FunctionSubStringIndex : public IFunction { column_delim = column_delim->isColumnConst() ? column_delim->convertToFullColumnIfConst() : column_delim; column_count = column_count->isColumnConst() ? column_count->convertToFullColumnIfConst() : column_count; - const ColumnString * str_col = checkAndGetColumn(column_str.get()); - const ColumnString * delim_col = checkAndGetColumn(column_delim.get()); - const ColumnVector * count_col = checkAndGetColumn>(column_count.get()); + const auto * str_col = checkAndGetColumn(column_str.get()); + const auto * delim_col = checkAndGetColumn(column_delim.get()); + const auto * count_col = checkAndGetColumn>(column_count.get()); if (str_col == nullptr || delim_col == nullptr || count_col == nullptr) { return false; @@ -4870,7 +4870,7 @@ class FunctionFormatWithLocale : public IFunction } else { - const String value = locale_const->getValue(); + const auto value = locale_const->getValue(); if (!boost::iequals(value, supported_locale)) { const auto & msg = genWarningMsg(value); From be8ea6ab6496adf194ea81b6d82a633f2f501ebc Mon Sep 17 00:00:00 2001 From: Liqi Geng Date: Fri, 8 Jul 2022 17:21:03 +0800 Subject: [PATCH 094/104] ci: fix check-license github action (#5318) close pingcap/tiflash#5317 --- .github/workflows/license-checker.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/license-checker.yml b/.github/workflows/license-checker.yml index e156c1b2b4c..2b23cca7eaa 100644 --- a/.github/workflows/license-checker.yml +++ b/.github/workflows/license-checker.yml @@ -15,7 +15,7 @@ jobs: steps: - uses: actions/checkout@v2 - name: Check License Header - uses: apache/skywalking-eyes@main + uses: apache/skywalking-eyes@v0.3.0 env: GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} with: From d7fdbd46e9d19f5dd54260f8a5652d5efc79df14 Mon Sep 17 00:00:00 2001 From: Calvin Neo Date: Fri, 8 Jul 2022 22:11:03 +0800 Subject: [PATCH 095/104] update proxy to raftstore-proxy-6.2 (#5316) ref pingcap/tiflash#4982 --- contrib/tiflash-proxy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/tiflash-proxy b/contrib/tiflash-proxy index 0080c6866d8..42ede65b66a 160000 --- a/contrib/tiflash-proxy +++ b/contrib/tiflash-proxy @@ -1 +1 @@ -Subproject commit 0080c6866d8bbba32bdf198437dbc98cc078ea7e +Subproject commit 42ede65b66aed69debc80b60a31c63e41010d450 From b62dc6aacf297cf1b593efb257ac851a81cc0a08 Mon Sep 17 00:00:00 2001 From: Liqi Geng Date: Fri, 8 Jul 2022 23:57:03 +0800 Subject: [PATCH 096/104] Change one `additional_input_at_end` to many streams in `ParallelInputsProcessor` (#5274) close pingcap/tiflash#4856, close pingcap/tiflash#5263 --- .../ParallelAggregatingBlockInputStream.cpp | 7 +- .../ParallelAggregatingBlockInputStream.h | 2 +- .../src/DataStreams/ParallelInputsProcessor.h | 282 ++++++------- dbms/src/DataStreams/UnionBlockInputStream.h | 9 +- dbms/src/DataStreams/tests/union_stream2.cpp | 2 +- dbms/src/Debug/astToExecutor.cpp | 2 +- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 23 +- .../Flash/Coprocessor/InterpreterUtils.cpp | 58 ++- dbms/src/Flash/tests/bench_exchange.cpp | 6 +- dbms/src/Flash/tests/bench_window.cpp | 2 +- dbms/src/Flash/tests/gtest_interpreter.cpp | 374 ++++++++++++------ .../Interpreters/InterpreterSelectQuery.cpp | 58 +-- .../src/Interpreters/InterpreterSelectQuery.h | 8 +- .../InterpreterSelectWithUnionQuery.cpp | 2 +- dbms/src/TestUtils/mockExecutor.h | 3 +- 15 files changed, 465 insertions(+), 373 deletions(-) diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp index f983de91b37..cd9d6235f52 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp @@ -24,7 +24,7 @@ namespace DB { ParallelAggregatingBlockInputStream::ParallelAggregatingBlockInputStream( const BlockInputStreams & inputs, - const BlockInputStreamPtr & additional_input_at_end, + const BlockInputStreams & additional_inputs_at_end, const Aggregator::Params & params_, const FileProviderPtr & file_provider_, bool final_, @@ -41,11 +41,10 @@ ParallelAggregatingBlockInputStream::ParallelAggregatingBlockInputStream( , keys_size(params.keys_size) , aggregates_size(params.aggregates_size) , handler(*this) - , processor(inputs, additional_input_at_end, max_threads, handler, log) + , processor(inputs, additional_inputs_at_end, max_threads, handler, log) { children = inputs; - if (additional_input_at_end) - children.push_back(additional_input_at_end); + children.insert(children.end(), additional_inputs_at_end.begin(), additional_inputs_at_end.end()); } diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h index 41e61786370..907622c8364 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h @@ -36,7 +36,7 @@ class ParallelAggregatingBlockInputStream : public IProfilingBlockInputStream */ ParallelAggregatingBlockInputStream( const BlockInputStreams & inputs, - const BlockInputStreamPtr & additional_input_at_end, + const BlockInputStreams & additional_inputs_at_end, const Aggregator::Params & params_, const FileProviderPtr & file_provider_, bool final_, diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index 34c70a7085e..57ab37e1756 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -16,6 +16,7 @@ #include #include +#include #include #include #include @@ -83,9 +84,8 @@ template class ParallelInputsProcessor { public: - /** additional_input_at_end - if not nullptr, - * then the blocks from this source will start to be processed only after all other sources are processed. - * This is done in the main thread. + /** additional_inputs_at_end - if not empty, + * then the blocks from the sources will start to be processed only after all other sources are processed. * * Intended for implementation of FULL and RIGHT JOIN * - where you must first make JOIN in parallel, while noting which keys are not found, @@ -93,19 +93,18 @@ class ParallelInputsProcessor */ ParallelInputsProcessor( const BlockInputStreams & inputs_, - const BlockInputStreamPtr & additional_input_at_end_, + const BlockInputStreams & additional_inputs_at_end_, size_t max_threads_, Handler & handler_, const LoggerPtr & log_) : inputs(inputs_) - , additional_input_at_end(additional_input_at_end_) - , max_threads(std::min(inputs_.size(), max_threads_)) + , additional_inputs_at_end(additional_inputs_at_end_) + , max_threads(std::min(std::max(inputs_.size(), additional_inputs_at_end_.size()), max_threads_)) , handler(handler_) + , working_inputs(inputs_) + , working_additional_inputs(additional_inputs_at_end_) , log(log_) - { - for (size_t i = 0; i < inputs_.size(); ++i) - unprepared_inputs.emplace(inputs_[i], i); - } + {} ~ParallelInputsProcessor() { @@ -132,36 +131,21 @@ class ParallelInputsProcessor /// Ask all sources to stop earlier than they run out. void cancel(bool kill) { - finish = true; + working_inputs.available_inputs.cancel(); + working_additional_inputs.available_inputs.cancel(); - for (auto & input : inputs) - { - if (IProfilingBlockInputStream * child = dynamic_cast(&*input)) - { - try - { - child->cancel(kill); - } - catch (...) - { - /** If you can not ask one or more sources to stop. - * (for example, the connection is broken for distributed query processing) - * - then do not care. - */ - LOG_FMT_ERROR(log, "Exception while cancelling {}", child->getName()); - } - } - } + cancelStreams(inputs, kill); + cancelStreams(additional_inputs_at_end, kill); } /// Wait until all threads are finished, before the destructor. void wait() { - if (joined_threads) - return; if (thread_manager) + { thread_manager->wait(); - joined_threads = true; + thread_manager.reset(); + } } size_t getNumActiveThreads() const @@ -181,13 +165,78 @@ class ParallelInputsProcessor BlockInputStreamPtr in; size_t i; /// The source number (for debugging). - InputData() {} + InputData() + : i(0) + {} InputData(const BlockInputStreamPtr & in_, size_t i_) : in(in_) , i(i_) {} }; + struct WorkingInputs + { + explicit WorkingInputs(const BlockInputStreams & inputs_) + : available_inputs(inputs_.size()) + , active_inputs(inputs_.size()) + , unprepared_inputs(inputs_.size()) + { + for (size_t i = 0; i < inputs_.size(); ++i) + unprepared_inputs.emplace(inputs_[i], i); + } + /** A set of available sources that are not currently processed by any thread. + * Each thread takes one source from this set, takes a block out of the source (at this moment the source does the calculations) + * and (if the source is not run out), puts it back into the set of available sources. + * + * The question arises what is better to use: + * - the queue (just processed source will be processed the next time later than the rest) + * - stack (just processed source will be processed as soon as possible). + * + * The stack is better than the queue when you need to do work on reading one source more consequentially, + * and theoretically, this allows you to achieve more consequent/consistent reads from the disk. + * + * But when using the stack, there is a problem with distributed query processing: + * data is read only from a part of the servers, and on the other servers + * a timeout occurs during send, and the request processing ends with an exception. + * + * Therefore, a queue is used. This can be improved in the future. + */ + using AvailableInputs = MPMCQueue; + AvailableInputs available_inputs; + + /// How many active input streams. + std::atomic active_inputs; + + /** For parallel preparing (readPrefix) child streams. + * First, streams are located here. + * After a stream was prepared, it is moved to "available_inputs" for reading. + */ + using UnpreparedInputs = MPMCQueue; + UnpreparedInputs unprepared_inputs; + }; + + void cancelStreams(const BlockInputStreams & streams, bool kill) + { + for (const auto & input : streams) + { + if (auto * p_child = dynamic_cast(&*input)) + { + try + { + p_child->cancel(kill); + } + catch (...) + { + /** If you can not ask one or more sources to stop. + * (for example, the connection is broken for distributed query processing) + * - then do not care. + */ + LOG_FMT_ERROR(log, "Exception while cancelling {}", p_child->getName()); + } + } + } + } + void publishPayload(BlockInputStreamPtr & stream, Block & block, size_t thread_num) { if constexpr (mode == StreamUnionMode::Basic) @@ -201,32 +250,24 @@ class ParallelInputsProcessor void thread(size_t thread_num) { - std::exception_ptr exception; + work(thread_num, working_inputs); + work(thread_num, working_additional_inputs); - try - { - while (!finish) - { - InputData unprepared_input; - { - std::lock_guard lock(unprepared_inputs_mutex); - - if (unprepared_inputs.empty()) - break; - - unprepared_input = unprepared_inputs.front(); - unprepared_inputs.pop(); - } + handler.onFinishThread(thread_num); - unprepared_input.in->readPrefix(); + if (0 == --active_threads) + { + handler.onFinish(); + } + } - { - std::lock_guard lock(available_inputs_mutex); - available_inputs.push(unprepared_input); - } - } + void work(size_t thread_num, WorkingInputs & work) + { + std::exception_ptr exception; - loop(thread_num); + try + { + loop(thread_num, work); } catch (...) { @@ -237,134 +278,63 @@ class ParallelInputsProcessor { handler.onException(exception, thread_num); } - - handler.onFinishThread(thread_num); - - /// The last thread on the output indicates that there is no more data. - if (0 == --active_threads) - { - /// And then it processes an additional source, if there is one. - if (additional_input_at_end) - { - try - { - additional_input_at_end->readPrefix(); - while (Block block = additional_input_at_end->read()) - publishPayload(additional_input_at_end, block, thread_num); - } - catch (...) - { - exception = std::current_exception(); - } - - if (exception) - { - handler.onException(exception, thread_num); - } - } - - handler.onFinish(); /// TODO If in `onFinish` or `onFinishThread` there is an exception, then std::terminate is called. - } } - void loop(size_t thread_num) + /// This function may be called in different threads. + /// If no exception occurs, we can ensure that the work is all done when the function + /// returns in any thread. + void loop(size_t thread_num, WorkingInputs & work) { - while (!finish) /// You may need to stop work earlier than all sources run out. + if (work.active_inputs == 0) { - InputData input; + return; + } - /// Select the next source. - { - std::lock_guard lock(available_inputs_mutex); + InputData input; - /// If there are no free sources, then this thread is no longer needed. (But other threads can work with their sources.) - if (available_inputs.empty()) - break; - - input = available_inputs.front(); + while (work.unprepared_inputs.tryPop(input)) + { + input.in->readPrefix(); - /// We remove the source from the queue of available sources. - available_inputs.pop(); - } + work.available_inputs.push(input); + } + // The condition is false when all input streams are exhausted or + // an exception occurred then the queue was cancelled. + while (work.available_inputs.pop(input)) + { /// The main work. Block block = input.in->read(); + if (block) { - if (finish) - break; - - /// If this source is not run out yet, then put the resulting block in the ready queue. + work.available_inputs.push(input); + publishPayload(input.in, block, thread_num); + } + else + { + if (0 == --work.active_inputs) { - std::lock_guard lock(available_inputs_mutex); - - if (block) - { - available_inputs.push(input); - } - else - { - if (available_inputs.empty()) - break; - } - } - - if (finish) + work.available_inputs.finish(); break; - - if (block) - publishPayload(input.in, block, thread_num); + } } } } - BlockInputStreams inputs; - BlockInputStreamPtr additional_input_at_end; + const BlockInputStreams inputs; + const BlockInputStreams additional_inputs_at_end; unsigned max_threads; Handler & handler; std::shared_ptr thread_manager; - /** A set of available sources that are not currently processed by any thread. - * Each thread takes one source from this set, takes a block out of the source (at this moment the source does the calculations) - * and (if the source is not run out), puts it back into the set of available sources. - * - * The question arises what is better to use: - * - the queue (just processed source will be processed the next time later than the rest) - * - stack (just processed source will be processed as soon as possible). - * - * The stack is better than the queue when you need to do work on reading one source more consequentially, - * and theoretically, this allows you to achieve more consequent/consistent reads from the disk. - * - * But when using the stack, there is a problem with distributed query processing: - * data is read only from a part of the servers, and on the other servers - * a timeout occurs during send, and the request processing ends with an exception. - * - * Therefore, a queue is used. This can be improved in the future. - */ - using AvailableInputs = std::queue; - AvailableInputs available_inputs; - - /** For parallel preparing (readPrefix) child streams. - * First, streams are located here. - * After a stream was prepared, it is moved to "available_inputs" for reading. - */ - using UnpreparedInputs = std::queue; - UnpreparedInputs unprepared_inputs; - - /// For operations with available_inputs. - std::mutex available_inputs_mutex; - - /// For operations with unprepared_inputs. - std::mutex unprepared_inputs_mutex; + WorkingInputs working_inputs; + WorkingInputs working_additional_inputs; /// How many sources ran out. std::atomic active_threads{0}; - /// Finish the threads work (before the sources run out). - std::atomic finish{false}; - /// Wait for the completion of all threads. - std::atomic joined_threads{false}; const LoggerPtr log; }; diff --git a/dbms/src/DataStreams/UnionBlockInputStream.h b/dbms/src/DataStreams/UnionBlockInputStream.h index a782c3dd087..ffcc8d77c10 100644 --- a/dbms/src/DataStreams/UnionBlockInputStream.h +++ b/dbms/src/DataStreams/UnionBlockInputStream.h @@ -94,20 +94,19 @@ class UnionBlockInputStream final : public IProfilingBlockInputStream public: UnionBlockInputStream( BlockInputStreams inputs, - BlockInputStreamPtr additional_input_at_end, + BlockInputStreams additional_inputs_at_end, size_t max_threads, const String & req_id, ExceptionCallback exception_callback_ = ExceptionCallback()) - : output_queue(std::min(inputs.size(), max_threads) * 5) // reduce contention + : output_queue(std::min(std::max(inputs.size(), additional_inputs_at_end.size()), max_threads) * 5) // reduce contention , log(Logger::get(NAME, req_id)) , handler(*this) - , processor(inputs, additional_input_at_end, max_threads, handler, log) + , processor(inputs, additional_inputs_at_end, max_threads, handler, log) , exception_callback(exception_callback_) { // TODO: assert capacity of output_queue is not less than processor.getMaxThreads() children = inputs; - if (additional_input_at_end) - children.push_back(additional_input_at_end); + children.insert(children.end(), additional_inputs_at_end.begin(), additional_inputs_at_end.end()); size_t num_children = children.size(); if (num_children > 1) diff --git a/dbms/src/DataStreams/tests/union_stream2.cpp b/dbms/src/DataStreams/tests/union_stream2.cpp index f939cda4e14..fb3f7238414 100644 --- a/dbms/src/DataStreams/tests/union_stream2.cpp +++ b/dbms/src/DataStreams/tests/union_stream2.cpp @@ -51,7 +51,7 @@ try for (size_t i = 0, size = streams.size(); i < size; ++i) streams[i] = std::make_shared(streams[i]); - BlockInputStreamPtr stream = std::make_shared>(streams, nullptr, settings.max_threads, /*req_id=*/""); + BlockInputStreamPtr stream = std::make_shared>(streams, BlockInputStreams{}, settings.max_threads, /*req_id=*/""); stream = std::make_shared(stream, 10, 0, ""); WriteBufferFromFileDescriptor wb(STDERR_FILENO); diff --git a/dbms/src/Debug/astToExecutor.cpp b/dbms/src/Debug/astToExecutor.cpp index 7d1f3bc7209..481eac65fe2 100644 --- a/dbms/src/Debug/astToExecutor.cpp +++ b/dbms/src/Debug/astToExecutor.cpp @@ -1545,7 +1545,7 @@ ExecutorPtr compileAggregation(ExecutorPtr input, size_t & executor_index, ASTPt ci.tp = TiDB::TypeLongLong; ci.flag = TiDB::ColumnFlagUnsigned | TiDB::ColumnFlagNotNull; } - else if (func->name == "max" || func->name == "min" || func->name == "first_row") + else if (func->name == "max" || func->name == "min" || func->name == "first_row" || func->name == "sum") { ci = children_ci[0]; ci.flag &= ~TiDB::ColumnFlagNotNull; diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index e322a830744..bf695da34c1 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -383,34 +383,39 @@ void DAGQueryBlockInterpreter::executeAggregation( is_final_agg); /// If there are several sources, then we perform parallel aggregation - if (pipeline.streams.size() > 1) + if (pipeline.streams.size() > 1 || pipeline.streams_with_non_joined_data.size() > 1) { const Settings & settings = context.getSettingsRef(); - BlockInputStreamPtr stream_with_non_joined_data = combinedNonJoinedDataStream(pipeline, max_streams, log); - pipeline.firstStream() = std::make_shared( + BlockInputStreamPtr stream = std::make_shared( pipeline.streams, - stream_with_non_joined_data, + pipeline.streams_with_non_joined_data, params, context.getFileProvider(), true, max_streams, settings.aggregation_memory_efficient_merge_threads ? static_cast(settings.aggregation_memory_efficient_merge_threads) : static_cast(settings.max_threads), log->identifier()); + pipeline.streams.resize(1); + pipeline.streams_with_non_joined_data.clear(); + pipeline.firstStream() = std::move(stream); + // should record for agg before restore concurrency. See #3804. recordProfileStreams(pipeline, query_block.aggregation_name); restorePipelineConcurrency(pipeline); } else { - BlockInputStreamPtr stream_with_non_joined_data = combinedNonJoinedDataStream(pipeline, max_streams, log); BlockInputStreams inputs; if (!pipeline.streams.empty()) inputs.push_back(pipeline.firstStream()); - else - pipeline.streams.resize(1); - if (stream_with_non_joined_data) - inputs.push_back(stream_with_non_joined_data); + + if (!pipeline.streams_with_non_joined_data.empty()) + inputs.push_back(pipeline.streams_with_non_joined_data.at(0)); + + pipeline.streams.resize(1); + pipeline.streams_with_non_joined_data.clear(); + pipeline.firstStream() = std::make_shared( std::make_shared(inputs, log->identifier()), params, diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index c9810454218..6415d36389b 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -39,32 +39,6 @@ void restoreConcurrency( } } -BlockInputStreamPtr combinedNonJoinedDataStream( - DAGPipeline & pipeline, - size_t max_threads, - const LoggerPtr & log, - bool ignore_block) -{ - BlockInputStreamPtr ret = nullptr; - if (pipeline.streams_with_non_joined_data.size() == 1) - ret = pipeline.streams_with_non_joined_data.at(0); - else if (pipeline.streams_with_non_joined_data.size() > 1) - { - if (ignore_block) - { - ret = std::make_shared(pipeline.streams_with_non_joined_data, nullptr, max_threads, log->identifier()); - ret->setExtraInfo("combine non joined(ignore block)"); - } - else - { - ret = std::make_shared(pipeline.streams_with_non_joined_data, nullptr, max_threads, log->identifier()); - ret->setExtraInfo("combine non joined"); - } - } - pipeline.streams_with_non_joined_data.clear(); - return ret; -} - void executeUnion( DAGPipeline & pipeline, size_t max_streams, @@ -72,21 +46,33 @@ void executeUnion( bool ignore_block, const String & extra_info) { - if (pipeline.streams.size() == 1 && pipeline.streams_with_non_joined_data.empty()) - return; - auto non_joined_data_stream = combinedNonJoinedDataStream(pipeline, max_streams, log, ignore_block); - if (!pipeline.streams.empty()) + switch (pipeline.streams.size() + pipeline.streams_with_non_joined_data.size()) { + case 0: + break; + case 1: + { + if (pipeline.streams.size() == 1) + break; + // streams_with_non_joined_data's size is 1. + pipeline.streams.push_back(pipeline.streams_with_non_joined_data.at(0)); + pipeline.streams_with_non_joined_data.clear(); + break; + } + default: + { + BlockInputStreamPtr stream; if (ignore_block) - pipeline.firstStream() = std::make_shared(pipeline.streams, non_joined_data_stream, max_streams, log->identifier()); + stream = std::make_shared(pipeline.streams, pipeline.streams_with_non_joined_data, max_streams, log->identifier()); else - pipeline.firstStream() = std::make_shared(pipeline.streams, non_joined_data_stream, max_streams, log->identifier()); - pipeline.firstStream()->setExtraInfo(extra_info); + stream = std::make_shared(pipeline.streams, pipeline.streams_with_non_joined_data, max_streams, log->identifier()); + stream->setExtraInfo(extra_info); + pipeline.streams.resize(1); + pipeline.streams_with_non_joined_data.clear(); + pipeline.firstStream() = std::move(stream); + break; } - else if (non_joined_data_stream != nullptr) - { - pipeline.streams.push_back(non_joined_data_stream); } } diff --git a/dbms/src/Flash/tests/bench_exchange.cpp b/dbms/src/Flash/tests/bench_exchange.cpp index fbb53bfd4a4..cbbdf060580 100644 --- a/dbms/src/Flash/tests/bench_exchange.cpp +++ b/dbms/src/Flash/tests/bench_exchange.cpp @@ -215,7 +215,7 @@ std::vector ReceiverHelper::buildExchangeReceiverStream() BlockInputStreamPtr ReceiverHelper::buildUnionStream() { auto streams = buildExchangeReceiverStream(); - return std::make_shared>(streams, nullptr, concurrency, /*req_id=*/""); + return std::make_shared>(streams, BlockInputStreams{}, concurrency, /*req_id=*/""); } void ReceiverHelper::finish() @@ -290,7 +290,7 @@ BlockInputStreamPtr SenderHelper::buildUnionStream( send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); } - return std::make_shared>(send_streams, nullptr, concurrency, /*req_id=*/""); + return std::make_shared>(send_streams, BlockInputStreams{}, concurrency, /*req_id=*/""); } BlockInputStreamPtr SenderHelper::buildUnionStream(size_t total_rows, const std::vector & blocks) @@ -312,7 +312,7 @@ BlockInputStreamPtr SenderHelper::buildUnionStream(size_t total_rows, const std: send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); } - return std::make_shared>(send_streams, nullptr, concurrency, /*req_id=*/""); + return std::make_shared>(send_streams, BlockInputStreams{}, concurrency, /*req_id=*/""); } void SenderHelper::finish() diff --git a/dbms/src/Flash/tests/bench_window.cpp b/dbms/src/Flash/tests/bench_window.cpp index dfdb358c46c..356f544a836 100644 --- a/dbms/src/Flash/tests/bench_window.cpp +++ b/dbms/src/Flash/tests/bench_window.cpp @@ -71,7 +71,7 @@ class WindowFunctionBench : public ExchangeBench pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, 8192, 0, "mock_executor_id_squashing"); }); - receiver_stream = std::make_shared>(pipeline.streams, nullptr, concurrency, /*req_id=*/""); + receiver_stream = std::make_shared>(pipeline.streams, BlockInputStreams{}, concurrency, /*req_id=*/""); } tipb::Window window; diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp index 2d4d4014780..c583fbf35c6 100644 --- a/dbms/src/Flash/tests/gtest_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_interpreter.cpp @@ -31,8 +31,8 @@ class InterpreterExecuteTest : public DB::tests::ExecutorTest context.addMockTable({"test_db", "r_table"}, {{"r_a", TiDB::TP::TypeLong}, {"r_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "l_table"}, {{"l_a", TiDB::TP::TypeLong}, {"l_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); context.addExchangeRelationSchema("sender_1", {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); - context.addExchangeRelationSchema("sender_l", {{"l_a", TiDB::TP::TypeString}, {"l_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); - context.addExchangeRelationSchema("sender_r", {{"r_a", TiDB::TP::TypeString}, {"r_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + context.addExchangeRelationSchema("sender_l", {{"l_a", TiDB::TP::TypeLong}, {"l_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + context.addExchangeRelationSchema("sender_r", {{"r_a", TiDB::TP::TypeLong}, {"r_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); } }; @@ -212,47 +212,6 @@ Union: ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } - // Join Source. - DAGRequestBuilder table1 = context.scan("test_db", "r_table"); - DAGRequestBuilder table2 = context.scan("test_db", "l_table"); - DAGRequestBuilder table3 = context.scan("test_db", "r_table"); - DAGRequestBuilder table4 = context.scan("test_db", "l_table"); - - request = table1.join( - table2.join( - table3.join(table4, - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left) - .build(context); - { - String expected = R"( -CreatingSets - Union: - HashJoinBuildBlockInputStream x 10: , join_kind = Left - Expression: - Expression: - MockTableScan - Union x 2: - HashJoinBuildBlockInputStream x 10: , join_kind = Left - Expression: - Expression: - Expression: - HashJoinProbe: - Expression: - MockTableScan - Union: - Expression x 10: - Expression: - HashJoinProbe: - Expression: - MockTableScan)"; - ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - } - request = context.receive("sender_1") .project({"s1", "s2", "s3"}) .project({"s1", "s2"}) @@ -298,90 +257,6 @@ Union: MockExchangeReceiver)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } - - // only join + ExchangeReceiver - DAGRequestBuilder receiver1 = context.receive("sender_l"); - DAGRequestBuilder receiver2 = context.receive("sender_r"); - DAGRequestBuilder receiver3 = context.receive("sender_l"); - DAGRequestBuilder receiver4 = context.receive("sender_r"); - - request = receiver1.join( - receiver2.join( - receiver3.join(receiver4, - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left) - .build(context); - { - String expected = R"( -CreatingSets - Union: - HashJoinBuildBlockInputStream x 10: , join_kind = Left - Expression: - Expression: - MockExchangeReceiver - Union x 2: - HashJoinBuildBlockInputStream x 10: , join_kind = Left - Expression: - Expression: - Expression: - HashJoinProbe: - Expression: - MockExchangeReceiver - Union: - Expression x 10: - Expression: - HashJoinProbe: - Expression: - MockExchangeReceiver)"; - ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - } - - // join + receiver + sender - // TODO: Find a way to write the request easier. - DAGRequestBuilder receiver5 = context.receive("sender_l"); - DAGRequestBuilder receiver6 = context.receive("sender_r"); - DAGRequestBuilder receiver7 = context.receive("sender_l"); - DAGRequestBuilder receiver8 = context.receive("sender_r"); - request = receiver5.join( - receiver6.join( - receiver7.join(receiver8, - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left) - .exchangeSender(tipb::PassThrough) - .build(context); - { - String expected = R"( -CreatingSets - Union: - HashJoinBuildBlockInputStream x 10: , join_kind = Left - Expression: - Expression: - MockExchangeReceiver - Union x 2: - HashJoinBuildBlockInputStream x 10: , join_kind = Left - Expression: - Expression: - Expression: - HashJoinProbe: - Expression: - MockExchangeReceiver - Union: - MockExchangeSender x 10 - Expression: - Expression: - HashJoinProbe: - Expression: - MockExchangeReceiver)"; - ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - } } CATCH @@ -465,5 +340,250 @@ Union: } CATCH +TEST_F(InterpreterExecuteTest, Join) +try +{ + // TODO: Find a way to write the request easier. + { + // Join Source. + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + DAGRequestBuilder table3 = context.scan("test_db", "r_table"); + DAGRequestBuilder table4 = context.scan("test_db", "l_table"); + + auto request = table1.join( + table2.join( + table3.join(table4, + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + MockTableScan + Union x 2: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + Expression: + HashJoinProbe: + Expression: + MockTableScan + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + // only join + ExchangeReceiver + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r"); + DAGRequestBuilder receiver3 = context.receive("sender_l"); + DAGRequestBuilder receiver4 = context.receive("sender_r"); + + auto request = receiver1.join( + receiver2.join( + receiver3.join(receiver4, + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + MockExchangeReceiver + Union x 2: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + // join + receiver + sender + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r"); + DAGRequestBuilder receiver3 = context.receive("sender_l"); + DAGRequestBuilder receiver4 = context.receive("sender_r"); + + auto request = receiver1.join( + receiver2.join( + receiver3.join(receiver4, + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left) + .exchangeSender(tipb::PassThrough) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + MockExchangeReceiver + Union x 2: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver + Union: + MockExchangeSender x 10 + Expression: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(InterpreterExecuteTest, JoinThenAgg) +try +{ + { + // Left Join. + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + + auto request = table1.join( + table2, + {col("join_c")}, + ASTTableJoin::Kind::Left) + .aggregation({Max(col("r_a"))}, {col("join_c")}) + .build(context); + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + MockTableScan + Union: + Expression x 10: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + // Right Join + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + + auto request = table1.join( + table2, + {col("join_c")}, + ASTTableJoin::Kind::Right) + .aggregation({Max(col("r_a"))}, {col("join_c")}) + .build(context); + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Right + Expression: + Expression: + MockTableScan + Union: + Expression x 10: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + HashJoinProbe: + Expression: + Expression: + MockTableScan + Expression x 10: + Expression: + NonJoined: )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + // Right join + receiver + sender + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r"); + + auto request = receiver1.join( + receiver2, + {col("join_c")}, + ASTTableJoin::Kind::Right) + .aggregation({Sum(col("r_a"))}, {col("join_c")}) + .exchangeSender(tipb::PassThrough) + .limit(10) + .build(context); + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 20: , join_kind = Right + Expression: + Expression: + MockExchangeReceiver + Union: + MockExchangeSender x 20 + SharedQuery: + Limit, limit = 10 + Union: + Limit x 20, limit = 10 + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 20, final: true + Expression x 20: + Expression: + HashJoinProbe: + Expression: + Expression: + MockExchangeReceiver + Expression x 20: + Expression: + NonJoined: )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 20); + } +} +CATCH + } // namespace tests } // namespace DB \ No newline at end of file diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index fe8f04427a0..3514f915626 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -512,13 +512,13 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt { const auto & join = static_cast(*query.join()->table_join); if (join.kind == ASTTableJoin::Kind::Full || join.kind == ASTTableJoin::Kind::Right) - pipeline.stream_with_non_joined_data = expressions.before_join->createStreamWithNonJoinedDataIfFullOrRightJoin( + pipeline.streams_with_non_joined_data.push_back(expressions.before_join->createStreamWithNonJoinedDataIfFullOrRightJoin( pipeline.firstStream()->getHeader(), 0, 1, - settings.max_block_size); + settings.max_block_size)); - for (auto & stream : pipeline.streams) /// Applies to all sources except stream_with_non_joined_data. + for (auto & stream : pipeline.streams) /// Applies to all sources except streams_with_non_joined_data. stream = std::make_shared(stream, expressions.before_join, /*req_id=*/""); } @@ -603,7 +603,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt if (need_second_distinct_pass || query.limit_length || query.limit_by_expression_list - || pipeline.stream_with_non_joined_data) + || !pipeline.streams_with_non_joined_data.empty()) { need_merge_streams = true; } @@ -987,11 +987,11 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre Aggregator::Params params(header, keys, aggregates, overflow_row, settings.max_rows_to_group_by, settings.group_by_overflow_mode, allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, context.getTemporaryPath()); /// If there are several sources, then we perform parallel aggregation - if (pipeline.streams.size() > 1) + if (pipeline.streams.size() > 1 || pipeline.streams_with_non_joined_data.size() > 1) { - pipeline.firstStream() = std::make_shared( + auto stream = std::make_shared( pipeline.streams, - pipeline.stream_with_non_joined_data, + pipeline.streams_with_non_joined_data, params, file_provider, final, @@ -1001,19 +1001,21 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre : static_cast(settings.max_threads), /*req_id=*/""); - pipeline.stream_with_non_joined_data = nullptr; pipeline.streams.resize(1); + pipeline.streams_with_non_joined_data.clear(); + pipeline.firstStream() = std::move(stream); } else { BlockInputStreams inputs; if (!pipeline.streams.empty()) inputs.push_back(pipeline.firstStream()); - else - pipeline.streams.resize(1); - if (pipeline.stream_with_non_joined_data) - inputs.push_back(pipeline.stream_with_non_joined_data); + if (!pipeline.streams_with_non_joined_data.empty()) + inputs.push_back(pipeline.streams_with_non_joined_data.at(0)); + + pipeline.streams.resize(1); + pipeline.streams_with_non_joined_data.clear(); pipeline.firstStream() = std::make_shared( std::make_shared(inputs, /*req_id=*/""), @@ -1021,8 +1023,6 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre file_provider, final, /*req_id=*/""); - - pipeline.stream_with_non_joined_data = nullptr; } } @@ -1244,21 +1244,33 @@ void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_or void InterpreterSelectQuery::executeUnion(Pipeline & pipeline) { - /// If there are still several streams, then we combine them into one - if (pipeline.hasMoreThanOneStream()) + switch (pipeline.streams.size() + pipeline.streams_with_non_joined_data.size()) { - pipeline.firstStream() = std::make_shared>( + case 0: + break; + case 1: + { + if (pipeline.streams.size() == 1) + break; + // streams_with_non_joined_data's size is 1. + pipeline.streams.push_back(pipeline.streams_with_non_joined_data.at(0)); + pipeline.streams_with_non_joined_data.clear(); + break; + } + default: + { + BlockInputStreamPtr stream = std::make_shared>( pipeline.streams, - pipeline.stream_with_non_joined_data, + pipeline.streams_with_non_joined_data, max_streams, /*req_id=*/""); - pipeline.stream_with_non_joined_data = nullptr; + ; + pipeline.streams.resize(1); + pipeline.streams_with_non_joined_data.clear(); + pipeline.firstStream() = std::move(stream); + break; } - else if (pipeline.stream_with_non_joined_data) - { - pipeline.streams.push_back(pipeline.stream_with_non_joined_data); - pipeline.stream_with_non_joined_data = nullptr; } } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 474ace7ee84..d1bcec2a3dd 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -95,7 +95,7 @@ class InterpreterSelectQuery : public IInterpreter * It has a special meaning, since reading from it should be done after reading from the main streams. * It is appended to the main streams in UnionBlockInputStream or ParallelAggregatingBlockInputStream. */ - BlockInputStreamPtr stream_with_non_joined_data; + BlockInputStreams streams_with_non_joined_data; BlockInputStreamPtr & firstStream() { return streams.at(0); } @@ -105,13 +105,13 @@ class InterpreterSelectQuery : public IInterpreter for (auto & stream : streams) transform(stream); - if (stream_with_non_joined_data) - transform(stream_with_non_joined_data); + for (auto & stream : streams_with_non_joined_data) + transform(stream); } bool hasMoreThanOneStream() const { - return streams.size() + (stream_with_non_joined_data ? 1 : 0) > 1; + return streams.size() + streams_with_non_joined_data.size() > 1; } }; diff --git a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 5e73b1e5f3e..076c290cc9d 100644 --- a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -224,7 +224,7 @@ BlockIO InterpreterSelectWithUnionQuery::execute() } else { - result_stream = std::make_shared>(nested_streams, nullptr, settings.max_threads, /*req_id=*/""); + result_stream = std::make_shared>(nested_streams, BlockInputStreams{}, settings.max_threads, /*req_id=*/""); nested_streams.clear(); } diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index 971061e93e7..5f752e58da6 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -83,7 +83,7 @@ class DAGRequestBuilder DAGRequestBuilder & exchangeSender(tipb::ExchangeType exchange_type); - // Currentlt only support inner join, left join and right join. + // Currently only support inner join, left join and right join. // TODO support more types of join. DAGRequestBuilder & join(const DAGRequestBuilder & right, MockAstVec exprs); DAGRequestBuilder & join(const DAGRequestBuilder & right, MockAstVec exprs, ASTTableJoin::Kind kind); @@ -174,6 +174,7 @@ MockWindowFrame buildDefaultRowsFrame(); #define Or(expr1, expr2) makeASTFunction("or", (expr1), (expr2)) #define NOT(expr) makeASTFunction("not", (expr)) #define Max(expr) makeASTFunction("max", (expr)) +#define Sum(expr) makeASTFunction("sum", (expr)) /// Window functions #define RowNumber() makeASTFunction("RowNumber") #define Rank() makeASTFunction("Rank") From 649919d86632f8dbf95b4f294e19e52de84bc1bf Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Mon, 11 Jul 2022 11:21:05 +0800 Subject: [PATCH 097/104] support fine grained shuffle for window function (#5048) close pingcap/tiflash#5142 --- contrib/kvproto | 2 +- contrib/tipb | 2 +- .../DataStreams/TiRemoteBlockInputStream.h | 10 +- dbms/src/Debug/astToExecutor.cpp | 18 +- dbms/src/Debug/astToExecutor.h | 19 +- dbms/src/Debug/dbgFuncCoprocessor.cpp | 5 +- .../src/Flash/Coprocessor/CoprocessorReader.h | 3 +- dbms/src/Flash/Coprocessor/DAGContext.h | 5 + dbms/src/Flash/Coprocessor/DAGDriver.cpp | 7 +- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 166 ++++++--- .../Coprocessor/DAGQueryBlockInterpreter.h | 11 +- .../Coprocessor/DAGStorageInterpreter.cpp | 2 +- dbms/src/Flash/Coprocessor/DecodeDetail.h | 8 +- .../StreamingDAGResponseWriter.cpp | 327 +++++++++++++----- .../Coprocessor/StreamingDAGResponseWriter.h | 18 +- .../tests/gtest_streaming_dag_writer.cpp | 184 ++++++++++ dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 236 ++++++++++--- dbms/src/Flash/Mpp/ExchangeReceiver.h | 41 ++- dbms/src/Flash/Mpp/MPPTask.cpp | 3 +- dbms/src/Flash/tests/WindowTestUtil.h | 22 +- dbms/src/Flash/tests/bench_exchange.cpp | 207 +++++++---- dbms/src/Flash/tests/bench_exchange.h | 22 +- dbms/src/Flash/tests/bench_window.cpp | 86 ++++- dbms/src/Flash/tests/gtest_interpreter.cpp | 74 +++- dbms/src/TestUtils/bench_dbms_main.cpp | 2 + dbms/src/TestUtils/mockExecutor.cpp | 33 +- dbms/src/TestUtils/mockExecutor.h | 16 +- tests/fullstack-test/mpp/window.test | 32 ++ 28 files changed, 1196 insertions(+), 365 deletions(-) create mode 100644 dbms/src/Flash/Coprocessor/tests/gtest_streaming_dag_writer.cpp create mode 100644 tests/fullstack-test/mpp/window.test diff --git a/contrib/kvproto b/contrib/kvproto index 12e2f5a9d16..a5d4ffd2ba3 160000 --- a/contrib/kvproto +++ b/contrib/kvproto @@ -1 +1 @@ -Subproject commit 12e2f5a9d167f46602804840857ddc8ff06dc695 +Subproject commit a5d4ffd2ba337dad0bc99e9fb53bf665864a3f3b diff --git a/contrib/tipb b/contrib/tipb index bfb5c2c5518..0f4f873beca 160000 --- a/contrib/tipb +++ b/contrib/tipb @@ -1 +1 @@ -Subproject commit bfb5c2c55188c254018d3cf77bfad73b4d4b77ec +Subproject commit 0f4f873beca8d5078dde0a23d15ad5ce3188ed0d diff --git a/dbms/src/DataStreams/TiRemoteBlockInputStream.h b/dbms/src/DataStreams/TiRemoteBlockInputStream.h index f249bf1a0dc..cfa3e95c440 100644 --- a/dbms/src/DataStreams/TiRemoteBlockInputStream.h +++ b/dbms/src/DataStreams/TiRemoteBlockInputStream.h @@ -58,6 +58,11 @@ class TiRemoteBlockInputStream : public IProfilingBlockInputStream uint64_t total_rows; + // For fine grained shuffle, sender will partition data into muiltiple streams by hashing. + // ExchangeReceiverBlockInputStream only need to read its own stream, i.e., streams[stream_id]. + // CoprocessorBlockInputStream doesn't take care of this. + size_t stream_id; + void initRemoteExecutionSummaries(tipb::SelectResponse & resp, size_t index) { for (const auto & execution_summary : resp.execution_summaries()) @@ -120,7 +125,7 @@ class TiRemoteBlockInputStream : public IProfilingBlockInputStream bool fetchRemoteResult() { - auto result = remote_reader->nextResult(block_queue, sample_block); + auto result = remote_reader->nextResult(block_queue, sample_block, stream_id); if (result.meet_error) { LOG_FMT_WARNING(log, "remote reader meets error: {}", result.error_msg); @@ -168,13 +173,14 @@ class TiRemoteBlockInputStream : public IProfilingBlockInputStream } public: - TiRemoteBlockInputStream(std::shared_ptr remote_reader_, const String & req_id, const String & executor_id) + TiRemoteBlockInputStream(std::shared_ptr remote_reader_, const String & req_id, const String & executor_id, size_t stream_id_) : remote_reader(remote_reader_) , source_num(remote_reader->getSourceNum()) , name(fmt::format("TiRemoteBlockInputStream({})", RemoteReader::name)) , execution_summaries_inited(source_num) , log(Logger::get(name, req_id, executor_id)) , total_rows(0) + , stream_id(stream_id_) { // generate sample block ColumnsWithTypeAndName columns; diff --git a/dbms/src/Debug/astToExecutor.cpp b/dbms/src/Debug/astToExecutor.cpp index 481eac65fe2..e02dd3aa740 100644 --- a/dbms/src/Debug/astToExecutor.cpp +++ b/dbms/src/Debug/astToExecutor.cpp @@ -851,6 +851,7 @@ bool ExchangeReceiver::toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t c { tipb_executor->set_tp(tipb::ExecType::TypeExchangeReceiver); tipb_executor->set_executor_id(name); + tipb_executor->set_fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count); tipb::ExchangeReceiver * exchange_receiver = tipb_executor->mutable_exchange_receiver(); for (auto & field : output_schema) { @@ -1354,6 +1355,7 @@ bool Window::toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t collator_id { tipb_executor->set_tp(tipb::ExecType::TypeWindow); tipb_executor->set_executor_id(name); + tipb_executor->set_fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count); tipb::Window * window = tipb_executor->mutable_window(); auto & input_schema = children[0]->output_schema; for (const auto & expr : func_descs) @@ -1430,6 +1432,7 @@ bool Sort::toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t collator_id, { tipb_executor->set_tp(tipb::ExecType::TypeSort); tipb_executor->set_executor_id(name); + tipb_executor->set_fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count); tipb::Sort * sort = tipb_executor->mutable_sort(); sort->set_ispartialsort(is_partial_sort); @@ -1665,13 +1668,13 @@ ExecutorPtr compileExchangeSender(ExecutorPtr input, size_t & executor_index, ti return exchange_sender; } -ExecutorPtr compileExchangeReceiver(size_t & executor_index, DAGSchema schema) +ExecutorPtr compileExchangeReceiver(size_t & executor_index, DAGSchema schema, uint64_t fine_grained_shuffle_stream_count) { - ExecutorPtr exchange_receiver = std::make_shared(executor_index, schema); + ExecutorPtr exchange_receiver = std::make_shared(executor_index, schema, fine_grained_shuffle_stream_count); return exchange_receiver; } -ExecutorPtr compileWindow(ExecutorPtr input, size_t & executor_index, ASTPtr func_desc_list, ASTPtr partition_by_expr_list, ASTPtr order_by_expr_list, mock::MockWindowFrame frame) +ExecutorPtr compileWindow(ExecutorPtr input, size_t & executor_index, ASTPtr func_desc_list, ASTPtr partition_by_expr_list, ASTPtr order_by_expr_list, mock::MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count) { std::vector partition_columns; if (partition_by_expr_list != nullptr) @@ -1739,12 +1742,13 @@ ExecutorPtr compileWindow(ExecutorPtr input, size_t & executor_index, ASTPtr fun window_exprs, std::move(partition_columns), std::move(order_columns), - frame); + frame, + fine_grained_shuffle_stream_count); window->children.push_back(input); return window; } -ExecutorPtr compileSort(ExecutorPtr input, size_t & executor_index, ASTPtr order_by_expr_list, bool is_partial_sort) +ExecutorPtr compileSort(ExecutorPtr input, size_t & executor_index, ASTPtr order_by_expr_list, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count) { std::vector order_columns; if (order_by_expr_list != nullptr) @@ -1758,8 +1762,8 @@ ExecutorPtr compileSort(ExecutorPtr input, size_t & executor_index, ASTPtr order compileExpr(input->output_schema, elem->children[0]); } } - ExecutorPtr sort = std::make_shared(executor_index, input->output_schema, std::move(order_columns), is_partial_sort); + ExecutorPtr sort = std::make_shared(executor_index, input->output_schema, std::move(order_columns), is_partial_sort, fine_grained_shuffle_stream_count); sort->children.push_back(input); return sort; } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Debug/astToExecutor.h b/dbms/src/Debug/astToExecutor.h index 4d87c0db77e..f39f4059d26 100644 --- a/dbms/src/Debug/astToExecutor.h +++ b/dbms/src/Debug/astToExecutor.h @@ -139,8 +139,11 @@ struct ExchangeSender : Executor struct ExchangeReceiver : Executor { TaskMetas task_metas; - ExchangeReceiver(size_t & index, const DAGSchema & output) + uint64_t fine_grained_shuffle_stream_count; + + ExchangeReceiver(size_t & index, const DAGSchema & output, uint64_t fine_grained_shuffle_stream_count_ = 0) : Executor(index, "exchange_receiver_" + std::to_string(index), output) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) {} void columnPrune(std::unordered_set &) override { throw Exception("Should not reach here"); } bool toTiPBExecutor(tipb::Executor * tipb_executor, uint32_t collator_id, const MPPInfo & mpp_info, const Context &) override; @@ -292,13 +295,15 @@ struct Window : Executor std::vector partition_by_exprs; std::vector order_by_exprs; MockWindowFrame frame; + uint64_t fine_grained_shuffle_stream_count; - Window(size_t & index_, const DAGSchema & output_schema_, std::vector func_descs_, std::vector partition_by_exprs_, std::vector order_by_exprs_, MockWindowFrame frame_) + Window(size_t & index_, const DAGSchema & output_schema_, std::vector func_descs_, std::vector partition_by_exprs_, std::vector order_by_exprs_, MockWindowFrame frame_, uint64_t fine_grained_shuffle_stream_count_ = 0) : Executor(index_, "window_" + std::to_string(index_), output_schema_) , func_descs(std::move(func_descs_)) , partition_by_exprs(std::move(partition_by_exprs_)) , order_by_exprs(order_by_exprs_) , frame(frame_) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) { } // Currently only use Window Executor in Unit Test which don't call columnPrume. @@ -311,11 +316,13 @@ struct Sort : Executor { std::vector by_exprs; bool is_partial_sort; + uint64_t fine_grained_shuffle_stream_count; - Sort(size_t & index_, const DAGSchema & output_schema_, std::vector by_exprs_, bool is_partial_sort_) + Sort(size_t & index_, const DAGSchema & output_schema_, std::vector by_exprs_, bool is_partial_sort_, uint64_t fine_grained_shuffle_stream_count_ = 0) : Executor(index_, "sort_" + std::to_string(index_), output_schema_) , by_exprs(by_exprs_) , is_partial_sort(is_partial_sort_) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) { } // Currently only use Sort Executor in Unit Test which don't call columnPrume. @@ -343,11 +350,11 @@ ExecutorPtr compileJoin(size_t & executor_index, ExecutorPtr left, ExecutorPtr r ExecutorPtr compileExchangeSender(ExecutorPtr input, size_t & executor_index, tipb::ExchangeType exchange_type); -ExecutorPtr compileExchangeReceiver(size_t & executor_index, DAGSchema schema); +ExecutorPtr compileExchangeReceiver(size_t & executor_index, DAGSchema schema, uint64_t fine_grained_shuffle_stream_count = 0); -ExecutorPtr compileWindow(ExecutorPtr input, size_t & executor_index, ASTPtr func_desc_list, ASTPtr partition_by_expr_list, ASTPtr order_by_expr_list, mock::MockWindowFrame frame); +ExecutorPtr compileWindow(ExecutorPtr input, size_t & executor_index, ASTPtr func_desc_list, ASTPtr partition_by_expr_list, ASTPtr order_by_expr_list, mock::MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count = 0); -ExecutorPtr compileSort(ExecutorPtr input, size_t & executor_index, ASTPtr order_by_expr_list, bool is_partial_sort); +ExecutorPtr compileSort(ExecutorPtr input, size_t & executor_index, ASTPtr order_by_expr_list, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count = 0); void literalFieldToTiPBExpr(const ColumnInfo & ci, const Field & field, tipb::Expr * expr, Int32 collator_id); } // namespace DB diff --git a/dbms/src/Debug/dbgFuncCoprocessor.cpp b/dbms/src/Debug/dbgFuncCoprocessor.cpp index e9335d1e2bd..62a8b7537f1 100644 --- a/dbms/src/Debug/dbgFuncCoprocessor.cpp +++ b/dbms/src/Debug/dbgFuncCoprocessor.cpp @@ -290,8 +290,9 @@ BlockInputStreamPtr executeQuery(Context & context, RegionID region_id, const DA tipb_exchange_receiver.encoded_task_meta_size(), 10, /*req_id=*/"", - /*executor_id=*/""); - BlockInputStreamPtr ret = std::make_shared(exchange_receiver, /*req_id=*/"", /*executor_id=*/""); + /*executor_id=*/"", + /*fine_grained_shuffle_stream_count=*/0); + BlockInputStreamPtr ret = std::make_shared(exchange_receiver, /*req_id=*/"", /*executor_id=*/"", /*stream_id*/ 0); return ret; } else diff --git a/dbms/src/Flash/Coprocessor/CoprocessorReader.h b/dbms/src/Flash/Coprocessor/CoprocessorReader.h index 25c07cff49c..b48fdbcd6dc 100644 --- a/dbms/src/Flash/Coprocessor/CoprocessorReader.h +++ b/dbms/src/Flash/Coprocessor/CoprocessorReader.h @@ -139,7 +139,8 @@ class CoprocessorReader return detail; } - CoprocessorReaderResult nextResult(std::queue & block_queue, const Block & header) + // stream_id is only meaningful for ExchagneReceiver. + CoprocessorReaderResult nextResult(std::queue & block_queue, const Block & header, size_t /*stream_id*/) { auto && [result, has_next] = resp_iter.next(); if (!result.error.empty()) diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index a50a4d4007b..10190074a0f 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -116,6 +116,11 @@ constexpr UInt64 NO_ENGINE_SUBSTITUTION = 1ul << 30ul; constexpr UInt64 ALLOW_INVALID_DATES = 1ul << 32ul; } // namespace TiDBSQLMode +inline bool enableFineGrainedShuffle(uint64_t stream_count) +{ + return stream_count > 0; +} + /// A context used to track the information that needs to be passed around during DAG planning. class DAGContext { diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index 55a2024a8bc..9fe388f8fe4 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -72,6 +72,7 @@ DAGDriver::DAGDriver( ::grpc::ServerWriter<::coprocessor::BatchResponse> * writer_, bool internal_) : context(context_) + , dag_response(nullptr) , writer(writer_) , internal(internal_) , log(&Poco::Logger::get("DAGDriver")) @@ -129,7 +130,7 @@ try auto streaming_writer = std::make_shared(writer); TiDB::TiDBCollators collators; - std::unique_ptr response_writer = std::make_unique>( + std::unique_ptr response_writer = std::make_unique>( streaming_writer, std::vector(), collators, @@ -137,7 +138,9 @@ try context.getSettingsRef().dag_records_per_chunk, context.getSettingsRef().batch_send_min_limit, true, - dag_context); + dag_context, + /*fine_grained_shuffle_stream_count=*/0, + /*fine_grained_shuffle_batch_size=*/0); dag_output_stream = std::make_shared(streams.in->getHeader(), std::move(response_writer)); copyData(*streams.in, *dag_output_stream); } diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index bf695da34c1..4714580fda0 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -57,6 +57,10 @@ namespace FailPoints { extern const char minimum_block_size_for_cross_join[]; } // namespace FailPoints +namespace +{ +const String enableFineGrainedShuffleExtraInfo = "enable fine grained shuffle"; +} DAGQueryBlockInterpreter::DAGQueryBlockInterpreter( Context & context_, @@ -347,14 +351,26 @@ void DAGQueryBlockInterpreter::executeWhere(DAGPipeline & pipeline, const Expres void DAGQueryBlockInterpreter::executeWindow( DAGPipeline & pipeline, - WindowDescription & window_description) + WindowDescription & window_description, + bool enable_fine_grained_shuffle) { executeExpression(pipeline, window_description.before_window, "before window"); - /// If there are several streams, we merge them into one - executeUnion(pipeline, max_streams, log, false, "merge into one for window input"); - assert(pipeline.streams.size() == 1); - pipeline.firstStream() = std::make_shared(pipeline.firstStream(), window_description, log->identifier()); + if (enable_fine_grained_shuffle) + { + /// Window function can be multiple threaded when fine grained shuffle is enabled. + pipeline.transform([&](auto & stream) { + stream = std::make_shared(stream, window_description, log->identifier()); + stream->setExtraInfo(enableFineGrainedShuffleExtraInfo); + }); + } + else + { + /// If there are several streams, we merge them into one. + executeUnion(pipeline, max_streams, log, false, "merge into one for window input"); + assert(pipeline.streams.size() == 1); + pipeline.firstStream() = std::make_shared(pipeline.firstStream(), window_description, log->identifier()); + } } void DAGQueryBlockInterpreter::executeAggregation( @@ -437,20 +453,23 @@ void DAGQueryBlockInterpreter::executeExpression(DAGPipeline & pipeline, const E } } -void DAGQueryBlockInterpreter::executeWindowOrder(DAGPipeline & pipeline, SortDescription sort_desc) +void DAGQueryBlockInterpreter::executeWindowOrder(DAGPipeline & pipeline, SortDescription sort_desc, bool enable_fine_grained_shuffle) { - orderStreams(pipeline, sort_desc, 0); + orderStreams(pipeline, sort_desc, 0, enable_fine_grained_shuffle); } void DAGQueryBlockInterpreter::executeOrder(DAGPipeline & pipeline, const NamesAndTypes & order_columns) { Int64 limit = query_block.limit_or_topn->topn().limit(); - orderStreams(pipeline, getSortDescription(order_columns, query_block.limit_or_topn->topn().order_by()), limit); + orderStreams(pipeline, getSortDescription(order_columns, query_block.limit_or_topn->topn().order_by()), limit, false); } -void DAGQueryBlockInterpreter::orderStreams(DAGPipeline & pipeline, SortDescription order_descr, Int64 limit) +void DAGQueryBlockInterpreter::orderStreams(DAGPipeline & pipeline, SortDescription order_descr, Int64 limit, bool enable_fine_grained_shuffle) { const Settings & settings = context.getSettingsRef(); + String extra_info; + if (enable_fine_grained_shuffle) + extra_info = enableFineGrainedShuffleExtraInfo; pipeline.transform([&](auto & stream) { auto sorting_stream = std::make_shared(stream, order_descr, log->identifier(), limit); @@ -462,20 +481,38 @@ void DAGQueryBlockInterpreter::orderStreams(DAGPipeline & pipeline, SortDescript sorting_stream->setLimits(limits); stream = sorting_stream; + stream->setExtraInfo(extra_info); }); - /// If there are several streams, we merge them into one - executeUnion(pipeline, max_streams, log, false, "for partial order"); - - /// Merge the sorted blocks. - pipeline.firstStream() = std::make_shared( - pipeline.firstStream(), - order_descr, - settings.max_block_size, - limit, - settings.max_bytes_before_external_sort, - context.getTemporaryPath(), - log->identifier()); + if (enable_fine_grained_shuffle) + { + pipeline.transform([&](auto & stream) { + stream = std::make_shared( + stream, + order_descr, + settings.max_block_size, + limit, + settings.max_bytes_before_external_sort, + context.getTemporaryPath(), + log->identifier()); + stream->setExtraInfo(enableFineGrainedShuffleExtraInfo); + }); + } + else + { + /// If there are several streams, we merge them into one + executeUnion(pipeline, max_streams, log, false, "for partial order"); + + /// Merge the sorted blocks. + pipeline.firstStream() = std::make_shared( + pipeline.firstStream(), + order_descr, + settings.max_block_size, + limit, + settings.max_bytes_before_external_sort, + context.getTemporaryPath(), + log->identifier()); + } } void DAGQueryBlockInterpreter::recordProfileStreams(DAGPipeline & pipeline, const String & key) @@ -491,12 +528,25 @@ void DAGQueryBlockInterpreter::handleExchangeReceiver(DAGPipeline & pipeline) throw Exception("Can not find exchange receiver for " + query_block.source_name, ErrorCodes::LOGICAL_ERROR); // todo choose a more reasonable stream number auto & exchange_receiver_io_input_streams = dagContext().getInBoundIOInputStreamsMap()[query_block.source_name]; - for (size_t i = 0; i < max_streams; ++i) + + const bool enable_fine_grained_shuffle = enableFineGrainedShuffle(exchange_receiver->getFineGrainedShuffleStreamCount()); + String extra_info = "squashing after exchange receiver"; + size_t stream_count = max_streams; + if (enable_fine_grained_shuffle) + { + extra_info += ", " + enableFineGrainedShuffleExtraInfo; + stream_count = std::min(max_streams, exchange_receiver->getFineGrainedShuffleStreamCount()); + } + + for (size_t i = 0; i < stream_count; ++i) { - BlockInputStreamPtr stream = std::make_shared(exchange_receiver, log->identifier(), query_block.source_name); + BlockInputStreamPtr stream = std::make_shared(exchange_receiver, + log->identifier(), + query_block.source_name, + /*stream_id=*/enable_fine_grained_shuffle ? i : 0); exchange_receiver_io_input_streams.push_back(stream); stream = std::make_shared(stream, 8192, 0, log->identifier()); - stream->setExtraInfo("squashing after exchange receiver"); + stream->setExtraInfo(extra_info); pipeline.streams.push_back(stream); } NamesAndTypes source_columns; @@ -561,7 +611,7 @@ void DAGQueryBlockInterpreter::handleProjection(DAGPipeline & pipeline, const ti analyzer = std::make_unique(std::move(output_columns), context); } -void DAGQueryBlockInterpreter::handleWindow(DAGPipeline & pipeline, const tipb::Window & window) +void DAGQueryBlockInterpreter::handleWindow(DAGPipeline & pipeline, const tipb::Window & window, bool enable_fine_grained_shuffle) { NamesAndTypes input_columns; assert(input_streams_vec.size() == 1); @@ -570,13 +620,13 @@ void DAGQueryBlockInterpreter::handleWindow(DAGPipeline & pipeline, const tipb:: input_columns.emplace_back(p.name, p.type); DAGExpressionAnalyzer dag_analyzer(input_columns, context); WindowDescription window_description = dag_analyzer.buildWindowDescription(window); - executeWindow(pipeline, window_description); + executeWindow(pipeline, window_description, enable_fine_grained_shuffle); executeExpression(pipeline, window_description.after_window, "cast after window"); analyzer = std::make_unique(window_description.after_window_columns, context); } -void DAGQueryBlockInterpreter::handleWindowOrder(DAGPipeline & pipeline, const tipb::Sort & window_sort) +void DAGQueryBlockInterpreter::handleWindowOrder(DAGPipeline & pipeline, const tipb::Sort & window_sort, bool enable_fine_grained_shuffle) { NamesAndTypes input_columns; assert(input_streams_vec.size() == 1); @@ -585,7 +635,7 @@ void DAGQueryBlockInterpreter::handleWindowOrder(DAGPipeline & pipeline, const t input_columns.emplace_back(p.name, p.type); DAGExpressionAnalyzer dag_analyzer(input_columns, context); auto order_columns = dag_analyzer.buildWindowOrderColumns(window_sort); - executeWindowOrder(pipeline, getSortDescription(order_columns, window_sort.byitems())); + executeWindowOrder(pipeline, getSortDescription(order_columns, window_sort.byitems()), enable_fine_grained_shuffle); analyzer = std::make_unique(std::move(input_columns), context); } @@ -633,13 +683,13 @@ void DAGQueryBlockInterpreter::executeImpl(DAGPipeline & pipeline) } else if (query_block.source->tp() == tipb::ExecType::TypeWindow) { - handleWindow(pipeline, query_block.source->window()); + handleWindow(pipeline, query_block.source->window(), enableFineGrainedShuffle(query_block.source->fine_grained_shuffle_stream_count())); recordProfileStreams(pipeline, query_block.source_name); restorePipelineConcurrency(pipeline); } else if (query_block.source->tp() == tipb::ExecType::TypeSort) { - handleWindowOrder(pipeline, query_block.source->sort()); + handleWindowOrder(pipeline, query_block.source->sort(), enableFineGrainedShuffle(query_block.source->fine_grained_shuffle_stream_count())); recordProfileStreams(pipeline, query_block.source_name); } else @@ -748,19 +798,47 @@ void DAGQueryBlockInterpreter::handleExchangeSender(DAGPipeline & pipeline) std::vector partition_col_ids = ExchangeSenderInterpreterHelper::genPartitionColIds(exchange_sender); TiDB::TiDBCollators partition_col_collators = ExchangeSenderInterpreterHelper::genPartitionColCollators(exchange_sender); int stream_id = 0; - pipeline.transform([&](auto & stream) { - // construct writer - std::unique_ptr response_writer = std::make_unique>( - context.getDAGContext()->tunnel_set, - partition_col_ids, - partition_col_collators, - exchange_sender.tp(), - context.getSettingsRef().dag_records_per_chunk, - context.getSettingsRef().batch_send_min_limit, - stream_id++ == 0, /// only one stream needs to sending execution summaries for the last response - dagContext()); - stream = std::make_shared(stream, std::move(response_writer), log->identifier()); - }); + const uint64_t stream_count = query_block.exchange_sender->fine_grained_shuffle_stream_count(); + const uint64_t batch_size = query_block.exchange_sender->fine_grained_shuffle_batch_size(); + + if (enableFineGrainedShuffle(stream_count)) + { + pipeline.transform([&](auto & stream) { + // construct writer + std::unique_ptr response_writer = std::make_unique>( + context.getDAGContext()->tunnel_set, + partition_col_ids, + partition_col_collators, + exchange_sender.tp(), + context.getSettingsRef().dag_records_per_chunk, + context.getSettingsRef().batch_send_min_limit, + stream_id++ == 0, /// only one stream needs to sending execution summaries for the last response + dagContext(), + stream_count, + batch_size); + stream = std::make_shared(stream, std::move(response_writer), log->identifier()); + stream->setExtraInfo(enableFineGrainedShuffleExtraInfo); + }); + RUNTIME_CHECK(exchange_sender.tp() == tipb::ExchangeType::Hash, Exception, "exchange_sender has to be hash partition when fine grained shuffle is enabled"); + RUNTIME_CHECK(stream_count <= 1024, Exception, "fine_grained_shuffle_stream_count should not be greater than 1024"); + } + else + { + pipeline.transform([&](auto & stream) { + std::unique_ptr response_writer = std::make_unique>( + context.getDAGContext()->tunnel_set, + partition_col_ids, + partition_col_collators, + exchange_sender.tp(), + context.getSettingsRef().dag_records_per_chunk, + context.getSettingsRef().batch_send_min_limit, + stream_id++ == 0, /// only one stream needs to sending execution summaries for the last response + dagContext(), + stream_count, + batch_size); + stream = std::make_shared(stream, std::move(response_writer), log->identifier()); + }); + } } void DAGQueryBlockInterpreter::handleMockExchangeSender(DAGPipeline & pipeline) @@ -788,4 +866,4 @@ BlockInputStreams DAGQueryBlockInterpreter::execute() return pipeline.streams; } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index e68c4f91cee..532dceabce9 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -64,17 +64,18 @@ class DAGQueryBlockInterpreter void handleExchangeReceiver(DAGPipeline & pipeline); void handleMockExchangeReceiver(DAGPipeline & pipeline); void handleProjection(DAGPipeline & pipeline, const tipb::Projection & projection); - void handleWindow(DAGPipeline & pipeline, const tipb::Window & window); - void handleWindowOrder(DAGPipeline & pipeline, const tipb::Sort & window_sort); + void handleWindow(DAGPipeline & pipeline, const tipb::Window & window, bool enable_fine_grained_shuffle); + void handleWindowOrder(DAGPipeline & pipeline, const tipb::Sort & window_sort, bool enable_fine_grained_shuffle); void executeWhere(DAGPipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, String & filter_column, const String & extra_info = ""); void executeExpression(DAGPipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, const String & extra_info = ""); - void executeWindowOrder(DAGPipeline & pipeline, SortDescription sort_desc); - void orderStreams(DAGPipeline & pipeline, SortDescription order_descr, Int64 limit); + void executeWindowOrder(DAGPipeline & pipeline, SortDescription sort_desc, bool enable_fine_grained_shuffle); + void orderStreams(DAGPipeline & pipeline, SortDescription order_descr, Int64 limit, bool enable_fine_grained_shuffle); void executeOrder(DAGPipeline & pipeline, const NamesAndTypes & order_columns); void executeLimit(DAGPipeline & pipeline); void executeWindow( DAGPipeline & pipeline, - WindowDescription & window_description); + WindowDescription & window_description, + bool enable_fine_grained_shuffle); void executeAggregation( DAGPipeline & pipeline, const ExpressionActionsPtr & expression_actions_ptr, diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index ad2de7217e0..390ce7b9948 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -487,7 +487,7 @@ void DAGStorageInterpreter::buildRemoteStreams(std::vector && rem auto coprocessor_reader = std::make_shared(schema, cluster, tasks, has_enforce_encode_type, 1); context.getDAGContext()->addCoprocessorReader(coprocessor_reader); - BlockInputStreamPtr input = std::make_shared(coprocessor_reader, log->identifier(), table_scan.getTableScanExecutorID()); + BlockInputStreamPtr input = std::make_shared(coprocessor_reader, log->identifier(), table_scan.getTableScanExecutorID(), /*stream_id=*/0); pipeline.streams.push_back(input); task_start = task_end; } diff --git a/dbms/src/Flash/Coprocessor/DecodeDetail.h b/dbms/src/Flash/Coprocessor/DecodeDetail.h index 9bad0ca2b72..91851650d9e 100644 --- a/dbms/src/Flash/Coprocessor/DecodeDetail.h +++ b/dbms/src/Flash/Coprocessor/DecodeDetail.h @@ -21,8 +21,12 @@ namespace DB /// Detail of the packet that decoding in TiRemoteInputStream.RemoteReader.decodeChunks() struct DecodeDetail { + // For fine grained shuffle, each ExchangeReceiver/thread will decode its own blocks. + // So this is the row number of partial blocks of the original packet. + // This will be the row number of all blocks of the original packet if it's not fine grained shuffle. Int64 rows = 0; - // byte size of origin packet. + + // Total byte size of the origin packet, even for fine grained shuffle. Int64 packet_bytes = 0; }; -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp index f915653fe96..a72dfcc16ef 100644 --- a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp +++ b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.cpp @@ -23,6 +23,8 @@ #include #include +#include + namespace DB { namespace ErrorCodes @@ -37,8 +39,8 @@ inline void serializeToPacket(mpp::MPPDataPacket & packet, const tipb::SelectRes throw Exception(fmt::format("Fail to serialize response, response size: {}", response.ByteSizeLong())); } -template -StreamingDAGResponseWriter::StreamingDAGResponseWriter( +template +StreamingDAGResponseWriter::StreamingDAGResponseWriter( StreamWriterPtr writer_, std::vector partition_col_ids_, TiDB::TiDBCollators collators_, @@ -46,7 +48,9 @@ StreamingDAGResponseWriter::StreamingDAGResponseWriter( Int64 records_per_chunk_, Int64 batch_send_min_limit_, bool should_send_exec_summary_at_last_, - DAGContext & dag_context_) + DAGContext & dag_context_, + uint64_t fine_grained_shuffle_stream_count_, + UInt64 fine_grained_shuffle_batch_size_) : DAGResponseWriter(records_per_chunk_, dag_context_) , batch_send_min_limit(batch_send_min_limit_) , should_send_exec_summary_at_last(should_send_exec_summary_at_last_) @@ -54,6 +58,8 @@ StreamingDAGResponseWriter::StreamingDAGResponseWriter( , writer(writer_) , partition_col_ids(std::move(partition_col_ids_)) , collators(std::move(collators_)) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) + , fine_grained_shuffle_batch_size(fine_grained_shuffle_batch_size_) { rows_in_blocks = 0; partition_num = writer_->getPartitionNum(); @@ -71,17 +77,37 @@ StreamingDAGResponseWriter::StreamingDAGResponseWriter( } } -template -void StreamingDAGResponseWriter::finishWrite() +template +void StreamingDAGResponseWriter::finishWrite() { if (should_send_exec_summary_at_last) - batchWrite(); + { + if constexpr (enable_fine_grained_shuffle) + { + assert(exchange_type == tipb::ExchangeType::Hash); + batchWriteFineGrainedShuffle(); + } + else + { + batchWrite(); + } + } else - batchWrite(); + { + if constexpr (enable_fine_grained_shuffle) + { + assert(exchange_type == tipb::ExchangeType::Hash); + batchWriteFineGrainedShuffle(); + } + else + { + batchWrite(); + } + } } -template -void StreamingDAGResponseWriter::write(const Block & block) +template +void StreamingDAGResponseWriter::write(const Block & block) { if (block.columns() != dag_context.result_field_types.size()) throw TiFlashException("Output column size mismatch with field type size", Errors::Coprocessor::Internal); @@ -91,15 +117,23 @@ void StreamingDAGResponseWriter::write(const Block & block) { blocks.push_back(block); } - if (static_cast(rows_in_blocks) > (dag_context.encode_type == tipb::EncodeType::TypeCHBlock ? batch_send_min_limit : records_per_chunk - 1)) + + if constexpr (enable_fine_grained_shuffle) { - batchWrite(); + assert(exchange_type == tipb::ExchangeType::Hash); + if (static_cast(rows_in_blocks) >= fine_grained_shuffle_batch_size) + batchWriteFineGrainedShuffle(); + } + else + { + if (static_cast(rows_in_blocks) > (dag_context.encode_type == tipb::EncodeType::TypeCHBlock ? batch_send_min_limit : records_per_chunk - 1)) + batchWrite(); } } -template +template template -void StreamingDAGResponseWriter::encodeThenWriteBlocks( +void StreamingDAGResponseWriter::encodeThenWriteBlocks( const std::vector & input_blocks, tipb::SelectResponse & response) const { @@ -191,133 +225,238 @@ void StreamingDAGResponseWriter::encodeThenWriteBlocks( } } -/// hash exchanging data among only TiFlash nodes. -template + +template template -void StreamingDAGResponseWriter::partitionAndEncodeThenWriteBlocks( - std::vector & input_blocks, - tipb::SelectResponse & response) const +void StreamingDAGResponseWriter::batchWrite() { - std::vector packet(partition_num); - - std::vector responses_row_count(partition_num); + tipb::SelectResponse response; + if constexpr (send_exec_summary_at_last) + addExecuteSummaries(response, !dag_context.isMPPTask() || dag_context.isRootMPPTask()); + if (exchange_type == tipb::ExchangeType::Hash) + { + partitionAndEncodeThenWriteBlocks(blocks, response); + } + else + { + encodeThenWriteBlocks(blocks, response); + } + blocks.clear(); + rows_in_blocks = 0; +} +template +template +void StreamingDAGResponseWriter::handleExecSummary( + const std::vector & input_blocks, + std::vector & packet, + tipb::SelectResponse & response) const +{ if constexpr (send_exec_summary_at_last) { /// Sending the response to only one node, default the first one. serializeToPacket(packet[0], response); - } - if (input_blocks.empty()) - { - if constexpr (send_exec_summary_at_last) + // No need to send data when blocks are not empty, + // because exec_summary will be sent together with blocks. + if (input_blocks.empty()) { for (auto part_id = 0; part_id < partition_num; ++part_id) { writer->write(packet[part_id], part_id); } } - return; } +} - // partition tuples in blocks - // 1) compute partition id - // 2) partition each row - // 3) encode each chunk and send it - std::vector partition_key_containers(collators.size()); - for (auto & block : input_blocks) +template +template +void StreamingDAGResponseWriter::writePackets(const std::vector & responses_row_count, + std::vector & packets) const +{ + for (size_t part_id = 0; part_id < packets.size(); ++part_id) { - std::vector dest_blocks(partition_num); - std::vector dest_tbl_cols(partition_num); - - for (size_t i = 0; i < block.columns(); ++i) + if constexpr (send_exec_summary_at_last) { - if (ColumnPtr converted = block.getByPosition(i).column->convertToFullColumnIfConst()) - { - block.getByPosition(i).column = converted; - } + writer->write(packets[part_id], part_id); } - - for (auto i = 0; i < partition_num; ++i) + else { - dest_tbl_cols[i] = block.cloneEmptyColumns(); - dest_blocks[i] = block.cloneEmpty(); + if (responses_row_count[part_id] > 0) + writer->write(packets[part_id], part_id); } + } +} - size_t rows = block.rows(); - WeakHash32 hash(rows); - - // get hash values by all partition key columns - for (size_t i = 0; i < partition_col_ids.size(); i++) +inline void initInputBlocks(std::vector & input_blocks) +{ + for (auto & input_block : input_blocks) + { + for (size_t i = 0; i < input_block.columns(); ++i) { - block.getByPosition(partition_col_ids[i]).column->updateWeakHash32(hash, collators[i], partition_key_containers[i]); + if (ColumnPtr converted = input_block.getByPosition(i).column->convertToFullColumnIfConst()) + input_block.getByPosition(i).column = converted; } - const auto & hash_data = hash.getData(); + } +} - // partition each row - IColumn::Selector selector(rows); - for (size_t row = 0; row < rows; ++row) - { - /// Row from interval [(2^32 / partition_num) * i, (2^32 / partition_num) * (i + 1)) goes to bucket with number i. - selector[row] = hash_data[row]; /// [0, 2^32) - selector[row] *= partition_num; /// [0, partition_num * 2^32), selector stores 64 bit values. - selector[row] >>= 32u; /// [0, partition_num) - } +inline void initDestColumns(const Block & input_block, std::vector & dest_tbl_cols) +{ + for (auto & cols : dest_tbl_cols) + { + cols = input_block.cloneEmptyColumns(); + } +} - for (size_t col_id = 0; col_id < block.columns(); ++col_id) - { - // Scatter columns to different partitions - auto scattered_columns = block.getByPosition(col_id).column->scatter(partition_num, selector); - for (size_t part_id = 0; part_id < partition_num; ++part_id) - { - dest_tbl_cols[part_id][col_id] = std::move(scattered_columns[part_id]); - } - } - // serialize each partitioned block and write it to its destination - for (auto part_id = 0; part_id < partition_num; ++part_id) - { - dest_blocks[part_id].setColumns(std::move(dest_tbl_cols[part_id])); - responses_row_count[part_id] += dest_blocks[part_id].rows(); - chunk_codec_stream->encode(dest_blocks[part_id], 0, dest_blocks[part_id].rows()); - packet[part_id].add_chunks(chunk_codec_stream->getString()); - chunk_codec_stream->clear(); - } +void computeHash(const Block & input_block, + uint32_t bucket_num, + const TiDB::TiDBCollators & collators, + std::vector & partition_key_containers, + const std::vector & partition_col_ids, + std::vector> & result_columns) +{ + size_t rows = input_block.rows(); + WeakHash32 hash(rows); + + // get hash values by all partition key columns + for (size_t i = 0; i < partition_col_ids.size(); ++i) + { + input_block.getByPosition(partition_col_ids[i]).column->updateWeakHash32(hash, collators[i], partition_key_containers[i]); } - for (auto part_id = 0; part_id < partition_num; ++part_id) + const auto & hash_data = hash.getData(); + + // partition each row + IColumn::Selector selector(rows); + for (size_t row = 0; row < rows; ++row) { - if constexpr (send_exec_summary_at_last) + /// Row from interval [(2^32 / bucket_num) * i, (2^32 / bucket_num) * (i + 1)) goes to bucket with number i. + selector[row] = hash_data[row]; /// [0, 2^32) + selector[row] *= bucket_num; /// [0, bucket_num * 2^32), selector stores 64 bit values. + selector[row] >>= 32u; /// [0, bucket_num) + } + + for (size_t col_id = 0; col_id < input_block.columns(); ++col_id) + { + // Scatter columns to different partitions + std::vector part_columns = input_block.getByPosition(col_id).column->scatter(bucket_num, selector); + assert(part_columns.size() == bucket_num); + for (size_t bucket_idx = 0; bucket_idx < bucket_num; ++bucket_idx) { - writer->write(packet[part_id], part_id); + result_columns[bucket_idx][col_id] = std::move(part_columns[bucket_idx]); } - else + } +} + +/// Hash exchanging data among only TiFlash nodes. Only be called when enable_fine_grained_shuffle is false. +template +template +void StreamingDAGResponseWriter::partitionAndEncodeThenWriteBlocks( + std::vector & input_blocks, + tipb::SelectResponse & response) const +{ + static_assert(!enable_fine_grained_shuffle); + std::vector packet(partition_num); + std::vector responses_row_count(partition_num); + handleExecSummary(input_blocks, packet, response); + if (input_blocks.empty()) + return; + + initInputBlocks(input_blocks); + Block dest_block = input_blocks[0].cloneEmpty(); + std::vector partition_key_containers(collators.size()); + for (const auto & block : input_blocks) + { + std::vector dest_tbl_cols(partition_num); + initDestColumns(block, dest_tbl_cols); + + computeHash(block, partition_num, collators, partition_key_containers, partition_col_ids, dest_tbl_cols); + + for (size_t part_id = 0; part_id < partition_num; ++part_id) { - if (responses_row_count[part_id] > 0) - writer->write(packet[part_id], part_id); + dest_block.setColumns(std::move(dest_tbl_cols[part_id])); + responses_row_count[part_id] += dest_block.rows(); + chunk_codec_stream->encode(dest_block, 0, dest_block.rows()); + packet[part_id].add_chunks(chunk_codec_stream->getString()); + chunk_codec_stream->clear(); } } + + writePackets(responses_row_count, packet); } -template +/// Hash exchanging data among only TiFlash nodes. Only be called when enable_fine_grained_shuffle is true. +template template -void StreamingDAGResponseWriter::batchWrite() +void StreamingDAGResponseWriter::batchWriteFineGrainedShuffle() { + static_assert(enable_fine_grained_shuffle); + assert(exchange_type == tipb::ExchangeType::Hash); + assert(fine_grained_shuffle_stream_count <= 1024); + tipb::SelectResponse response; if constexpr (send_exec_summary_at_last) addExecuteSummaries(response, !dag_context.isMPPTask() || dag_context.isRootMPPTask()); - if (exchange_type == tipb::ExchangeType::Hash) - { - partitionAndEncodeThenWriteBlocks(blocks, response); - } - else + + std::vector packet(partition_num); + std::vector responses_row_count(partition_num, 0); + + // fine_grained_shuffle_stream_count is in [0, 1024], and partition_num is uint16_t, so will not overflow. + uint32_t bucket_num = partition_num * fine_grained_shuffle_stream_count; + handleExecSummary(blocks, packet, response); + if (!blocks.empty()) { - encodeThenWriteBlocks(blocks, response); + std::vector final_dest_tbl_columns(bucket_num); + initInputBlocks(blocks); + initDestColumns(blocks[0], final_dest_tbl_columns); + + // Hash partition input_blocks into bucket_num. + for (const auto & block : blocks) + { + std::vector partition_key_containers(collators.size()); + std::vector dest_tbl_columns(bucket_num); + initDestColumns(block, dest_tbl_columns); + computeHash(block, bucket_num, collators, partition_key_containers, partition_col_ids, dest_tbl_columns); + for (size_t bucket_idx = 0; bucket_idx < bucket_num; ++bucket_idx) + { + for (size_t col_id = 0; col_id < block.columns(); ++col_id) + { + const MutableColumnPtr & src_col = dest_tbl_columns[bucket_idx][col_id]; + final_dest_tbl_columns[bucket_idx][col_id]->insertRangeFrom(*src_col, 0, src_col->size()); + } + } + } + + // For i-th stream_count buckets, send to i-th tiflash node. + for (size_t bucket_idx = 0; bucket_idx < bucket_num; bucket_idx += fine_grained_shuffle_stream_count) + { + size_t part_id = bucket_idx / fine_grained_shuffle_stream_count; // NOLINT(clang-analyzer-core.DivideZero) + size_t row_count_per_part = 0; + for (uint64_t stream_idx = 0; stream_idx < fine_grained_shuffle_stream_count; ++stream_idx) + { + Block dest_block = blocks[0].cloneEmpty(); + // For now we put all rows into one Block, may cause this Block too large. + dest_block.setColumns(std::move(final_dest_tbl_columns[bucket_idx + stream_idx])); + row_count_per_part += dest_block.rows(); + + chunk_codec_stream->encode(dest_block, 0, dest_block.rows()); + packet[part_id].add_chunks(chunk_codec_stream->getString()); + packet[part_id].add_stream_ids(stream_idx); + chunk_codec_stream->clear(); + } + responses_row_count[part_id] = row_count_per_part; + } } + + writePackets(responses_row_count, packet); + blocks.clear(); rows_in_blocks = 0; } -template class StreamingDAGResponseWriter; -template class StreamingDAGResponseWriter; +template class StreamingDAGResponseWriter; +template class StreamingDAGResponseWriter; +template class StreamingDAGResponseWriter; +template class StreamingDAGResponseWriter; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h index 9b5e3864c64..cd7559d1e79 100644 --- a/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h +++ b/dbms/src/Flash/Coprocessor/StreamingDAGResponseWriter.h @@ -33,7 +33,7 @@ namespace DB /// Serializes the stream of blocks and sends them to TiDB or TiFlash with different serialization paths. /// When sending data to TiDB, blocks with extra info are written into tipb::SelectResponse, then the whole tipb::SelectResponse is further serialized into mpp::MPPDataPacket.data. /// Differently when sending data to TiFlash, blocks with only tuples are directly serialized into mpp::MPPDataPacket.chunks, but for the last block, its extra info (like execution summaries) is written into tipb::SelectResponse, then further serialized into mpp::MPPDataPacket.data. -template +template class StreamingDAGResponseWriter : public DAGResponseWriter { public: @@ -45,18 +45,30 @@ class StreamingDAGResponseWriter : public DAGResponseWriter Int64 records_per_chunk_, Int64 batch_send_min_limit_, bool should_send_exec_summary_at_last, - DAGContext & dag_context_); + DAGContext & dag_context_, + UInt64 fine_grained_shuffle_stream_count_, + UInt64 fine_grained_shuffle_batch_size); void write(const Block & block) override; void finishWrite() override; private: template void batchWrite(); + template + void batchWriteFineGrainedShuffle(); + template void encodeThenWriteBlocks(const std::vector & input_blocks, tipb::SelectResponse & response) const; template void partitionAndEncodeThenWriteBlocks(std::vector & input_blocks, tipb::SelectResponse & response) const; + template + void handleExecSummary(const std::vector & input_blocks, + std::vector & packet, + tipb::SelectResponse & response) const; + template + void writePackets(const std::vector & responses_row_count, std::vector & packets) const; + Int64 batch_send_min_limit; bool should_send_exec_summary_at_last; /// only one stream needs to sending execution summaries at last. tipb::ExchangeType exchange_type; @@ -67,6 +79,8 @@ class StreamingDAGResponseWriter : public DAGResponseWriter size_t rows_in_blocks; uint16_t partition_num; std::unique_ptr chunk_codec_stream; + UInt64 fine_grained_shuffle_stream_count; + UInt64 fine_grained_shuffle_batch_size; }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/tests/gtest_streaming_dag_writer.cpp b/dbms/src/Flash/Coprocessor/tests/gtest_streaming_dag_writer.cpp new file mode 100644 index 00000000000..5d4186123b7 --- /dev/null +++ b/dbms/src/Flash/Coprocessor/tests/gtest_streaming_dag_writer.cpp @@ -0,0 +1,184 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include + +#include +#include + +namespace DB +{ +namespace tests +{ + +using BlockPtr = std::shared_ptr; +class TestStreamingDAGResponseWriter : public testing::Test +{ +protected: + void SetUp() override + { + dag_context_ptr = std::make_unique(1024); + dag_context_ptr->encode_type = tipb::EncodeType::TypeCHBlock; + dag_context_ptr->is_mpp_task = true; + dag_context_ptr->is_root_mpp_task = false; + dag_context_ptr->result_field_types = makeFields(); + context.setDAGContext(dag_context_ptr.get()); + } + +public: + TestStreamingDAGResponseWriter() + : context(TiFlashTestEnv::getContext()) + , part_col_ids{0} + , part_col_collators{ + TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::BINARY)} + {} + + // Return 10 Int64 column. + static std::vector makeFields() + { + std::vector fields(10); + for (int i = 0; i < 10; ++i) + { + fields[i].set_tp(TiDB::TypeLongLong); + } + return fields; + } + + // Return a block with **rows** and 10 Int64 column. + static BlockPtr prepareBlock(const std::vector & rows) + { + BlockPtr block = std::make_shared(); + for (int i = 0; i < 10; ++i) + { + DataTypePtr int64_data_type = std::make_shared(); + DataTypePtr nullable_int64_data_type = std::make_shared(int64_data_type); + MutableColumnPtr int64_col = nullable_int64_data_type->createColumn(); + for (Int64 r : rows) + { + int64_col->insert(Field(r)); + } + block->insert(ColumnWithTypeAndName{std::move(int64_col), + nullable_int64_data_type, + String("col") + std::to_string(i)}); + } + return block; + } + + Context context; + std::vector part_col_ids; + TiDB::TiDBCollators part_col_collators; + + std::unique_ptr dag_context_ptr; +}; + +using MockStreamWriterChecker = std::function; + +struct MockStreamWriter +{ + MockStreamWriter(MockStreamWriterChecker checker_, + uint16_t part_num_) + : checker(checker_) + , part_num(part_num_) + {} + + void write(mpp::MPPDataPacket &) { FAIL() << "cannot reach here, because we only expect hash partition"; } + void write(mpp::MPPDataPacket & packet, uint16_t part_id) { checker(packet, part_id); } + void write(tipb::SelectResponse &, uint16_t) { FAIL() << "cannot reach here, only consider CH Block format"; } + void write(tipb::SelectResponse &) { FAIL() << "cannot reach here, only consider CH Block format"; } + uint16_t getPartitionNum() const { return part_num; } + +private: + MockStreamWriterChecker checker; + uint16_t part_num; +}; + +// Input block data is distributed uniform. +// partition_num: 4 +// fine_grained_shuffle_stream_count: 8 +TEST_F(TestStreamingDAGResponseWriter, testBatchWriteFineGrainedShuffle) +try +{ + const size_t block_rows = 1024; + const uint16_t part_num = 4; + const uint32_t fine_grained_shuffle_stream_count = 8; + const Int64 fine_grained_shuffle_batch_size = 4096; + + // Set these to 1, because when fine grained shuffle is enabled, + // batchWriteFineGrainedShuffle() only check fine_grained_shuffle_batch_size. + // records_per_chunk and batch_send_min_limit are useless. + const Int64 records_per_chunk = 1; + const Int64 batch_send_min_limit = 1; + const bool should_send_exec_summary_at_last = true; + + // 1. Build Block. + std::vector uniform_data_set; + for (size_t i = 0; i < block_rows; ++i) + { + uniform_data_set.push_back(i); + } + BlockPtr block = prepareBlock(uniform_data_set); + + // 2. Build MockStreamWriter. + std::unordered_map write_report; + auto checker = [&write_report](mpp::MPPDataPacket & packet, uint16_t part_id) { + auto res = write_report.insert({part_id, packet}); + // Should always insert succeed. + // Because block.rows(1024) < fine_grained_shuffle_batch_size(4096), + // batchWriteFineGrainedShuffle() only called once, so will only be one packet for each partition. + ASSERT_TRUE(res.second); + }; + auto mock_writer = std::make_shared(checker, part_num); + + // 3. Start to write. + auto dag_writer = std::make_shared, /*enable_fine_grained_shuffle=*/true>>( + mock_writer, + part_col_ids, + part_col_collators, + tipb::ExchangeType::Hash, + records_per_chunk, + batch_send_min_limit, + should_send_exec_summary_at_last, + *dag_context_ptr, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size); + dag_writer->write(*block); + dag_writer->finishWrite(); + + // 4. Start to check write_report. + std::vector decoded_blocks; + ASSERT_EQ(write_report.size(), part_num); + for (const auto & ele : write_report) + { + const mpp::MPPDataPacket & packet = ele.second; + ASSERT_EQ(packet.chunks_size(), packet.stream_ids_size()); + for (int i = 0; i < packet.chunks_size(); ++i) + { + decoded_blocks.push_back(CHBlockChunkCodec::decode(packet.chunks(i), *block)); + } + } + ASSERT_EQ(decoded_blocks.size(), fine_grained_shuffle_stream_count * part_num); + for (const auto & block : decoded_blocks) + { + ASSERT_EQ(block.rows(), block_rows / (fine_grained_shuffle_stream_count * part_num)); + } +} +CATCH + +} // namespace tests +} // namespace DB diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 966babb832f..ab8d83a1481 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include @@ -48,6 +49,106 @@ String getReceiverStateStr(const ExchangeReceiverState & s) } } +// If enable_fine_grained_shuffle: +// Seperate chunks according to packet.stream_ids[i], then push to msg_channels[stream_id]. +// If fine grained_shuffle is disabled: +// Push all chunks to msg_channels[0]. +// Return true if all push succeed, otherwise return false. +// NOTE: shared_ptr will be hold by all ExchangeReceiverBlockInputStream to make chunk pointer valid. +template +bool pushPacket(size_t source_index, + const String & req_info, + MPPDataPacketPtr & packet, + const std::vector & msg_channels, + LoggerPtr & log) +{ + bool push_succeed = true; + + const mpp::Error * error_ptr = nullptr; + if (packet->has_error()) + error_ptr = &packet->error(); + const String * resp_ptr = nullptr; + if (!packet->data().empty()) + resp_ptr = &packet->data(); + + if constexpr (enable_fine_grained_shuffle) + { + std::vector> chunks(msg_channels.size()); + if (!packet->chunks().empty()) + { + // Packet not empty. + if (unlikely(packet->stream_ids().empty())) + { + // Fine grained shuffle is enabled in receiver, but sender didn't. We cannot handle this, so return error. + // This can happen when there are old version nodes when upgrading. + LOG_FMT_ERROR(log, "MPPDataPacket.stream_ids empty, it means ExchangeSender is old version of binary " + "(source_index: {}) while fine grained shuffle of ExchangeReceiver is enabled. " + "Cannot handle this.", + source_index); + return false; + } + // packet.stream_ids[i] is corresponding to packet.chunks[i], + // indicating which stream_id this chunk belongs to. + assert(packet->chunks_size() == packet->stream_ids_size()); + + for (int i = 0; i < packet->stream_ids_size(); ++i) + { + UInt64 stream_id = packet->stream_ids(i) % msg_channels.size(); + chunks[stream_id].push_back(&packet->chunks(i)); + } + } + // Still need to send error_ptr or resp_ptr even if packet.chunks_size() is zero. + for (size_t i = 0; i < msg_channels.size() && push_succeed; ++i) + { + if (resp_ptr == nullptr && error_ptr == nullptr && chunks[i].empty()) + continue; + + std::shared_ptr recv_msg = std::make_shared( + source_index, + req_info, + packet, + error_ptr, + resp_ptr, + std::move(chunks[i])); + push_succeed = msg_channels[i]->push(std::move(recv_msg)); + if constexpr (is_sync) + fiu_do_on(FailPoints::random_receiver_sync_msg_push_failure_failpoint, push_succeed = false;); + else + fiu_do_on(FailPoints::random_receiver_async_msg_push_failure_failpoint, push_succeed = false;); + + // Only the first ExchangeReceiverInputStream need to handle resp. + resp_ptr = nullptr; + } + } + else + { + std::vector chunks(packet->chunks_size()); + for (int i = 0; i < packet->chunks_size(); ++i) + { + chunks[i] = &packet->chunks(i); + } + + if (!(resp_ptr == nullptr && error_ptr == nullptr && chunks.empty())) + { + std::shared_ptr recv_msg = std::make_shared( + source_index, + req_info, + packet, + error_ptr, + resp_ptr, + std::move(chunks)); + + push_succeed = msg_channels[0]->push(std::move(recv_msg)); + if constexpr (is_sync) + fiu_do_on(FailPoints::random_receiver_sync_msg_push_failure_failpoint, push_succeed = false;); + else + fiu_do_on(FailPoints::random_receiver_async_msg_push_failure_failpoint, push_succeed = false;); + } + } + LOG_FMT_DEBUG(log, "push recv_msg to msg_channels(size: {}) succeed:{}, enable_fine_grained_shuffle: {}", msg_channels.size(), push_succeed, enable_fine_grained_shuffle); + return push_succeed; +} + enum class AsyncRequestStage { NEED_INIT, @@ -64,25 +165,25 @@ using TimePoint = Clock::time_point; constexpr Int32 max_retry_times = 10; constexpr Int32 batch_packet_count = 16; -template +template class AsyncRequestHandler : public UnaryCallback { public: using Status = typename RPCContext::Status; using Request = typename RPCContext::Request; using AsyncReader = typename RPCContext::AsyncReader; - using Self = AsyncRequestHandler; + using Self = AsyncRequestHandler; AsyncRequestHandler( MPMCQueue * queue, - MPMCQueue> * msg_channel_, + std::vector * msg_channels_, const std::shared_ptr & context, const Request & req, const String & req_id) : rpc_context(context) , request(&req) , notify_queue(queue) - , msg_channel(msg_channel_) + , msg_channels(msg_channels_) , req_info(fmt::format("tunnel{}+{}", req.send_task_id, req.recv_task_id)) , log(Logger::get("ExchangeReceiver", req_id, req_info)) { @@ -260,13 +361,7 @@ class AsyncRequestHandler : public UnaryCallback for (size_t i = 0; i < read_packet_index; ++i) { auto & packet = packets[i]; - auto recv_msg = std::make_shared(); - recv_msg->packet = std::move(packet); - recv_msg->source_index = request->source_index; - recv_msg->req_info = req_info; - bool push_success = msg_channel->push(std::move(recv_msg)); - fiu_do_on(FailPoints::random_receiver_async_msg_push_failure_failpoint, push_success = false;); - if (!push_success) + if (!pushPacket(request->source_index, req_info, packet, *msg_channels, log)) return false; // can't reuse packet since it is sent to readers. packet = std::make_shared(); @@ -283,7 +378,7 @@ class AsyncRequestHandler : public UnaryCallback std::shared_ptr rpc_context; const Request * request; // won't be null MPMCQueue * notify_queue; // won't be null - MPMCQueue> * msg_channel; // won't be null + std::vector * msg_channels; // won't be null String req_info; bool meet_error = false; @@ -308,20 +403,32 @@ ExchangeReceiverBase::ExchangeReceiverBase( size_t source_num_, size_t max_streams_, const String & req_id, - const String & executor_id) + const String & executor_id, + uint64_t fine_grained_shuffle_stream_count_) : rpc_context(std::move(rpc_context_)) , source_num(source_num_) , max_streams(max_streams_) , max_buffer_size(std::max(batch_packet_count, std::max(source_num, max_streams_) * 2)) , thread_manager(newThreadManager()) - , msg_channel(max_buffer_size) , live_connections(source_num) , state(ExchangeReceiverState::NORMAL) , exc_log(Logger::get("ExchangeReceiver", req_id, executor_id)) , collected(false) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) { try { + if (enableFineGrainedShuffle(fine_grained_shuffle_stream_count_)) + { + for (size_t i = 0; i < max_streams_; ++i) + { + msg_channels.push_back(std::make_unique>>(max_buffer_size)); + } + } + else + { + msg_channels.push_back(std::make_unique>>(max_buffer_size)); + } rpc_context->fillSchema(schema); setUpConnection(); } @@ -358,14 +465,14 @@ template void ExchangeReceiverBase::cancel() { setEndState(ExchangeReceiverState::CANCELED); - msg_channel.cancel(); + cancelAllMsgChannels(); } template void ExchangeReceiverBase::close() { setEndState(ExchangeReceiverState::CLOSED); - msg_channel.finish(); + finishAllMsgChannels(); } template @@ -380,7 +487,12 @@ void ExchangeReceiverBase::setUpConnection() async_requests.push_back(std::move(req)); else { - thread_manager->schedule(true, "Receiver", [this, req = std::move(req)] { readLoop(req); }); + thread_manager->schedule(true, "Receiver", [this, req = std::move(req)] { + if (enableFineGrainedShuffle(fine_grained_shuffle_stream_count)) + readLoop(req); + else + readLoop(req); + }); ++thread_count; } } @@ -388,15 +500,21 @@ void ExchangeReceiverBase::setUpConnection() // TODO: reduce this thread in the future. if (!async_requests.empty()) { - thread_manager->schedule(true, "RecvReactor", [this, async_requests = std::move(async_requests)] { reactor(async_requests); }); + thread_manager->schedule(true, "RecvReactor", [this, async_requests = std::move(async_requests)] { + if (enableFineGrainedShuffle(fine_grained_shuffle_stream_count)) + reactor(async_requests); + else + reactor(async_requests); + }); ++thread_count; } } template +template void ExchangeReceiverBase::reactor(const std::vector & async_requests) { - using AsyncHandler = AsyncRequestHandler; + using AsyncHandler = AsyncRequestHandler; GET_METRIC(tiflash_thread_count, type_threads_of_receiver_reactor).Increment(); SCOPE_EXIT({ @@ -412,7 +530,7 @@ void ExchangeReceiverBase::reactor(const std::vector & asyn std::vector> handlers; handlers.reserve(alive_async_connections); for (const auto & req : async_requests) - handlers.emplace_back(std::make_unique(&ready_requests, &msg_channel, rpc_context, req, exc_log->identifier())); + handlers.emplace_back(std::make_unique(&ready_requests, &msg_channels, rpc_context, req, exc_log->identifier())); while (alive_async_connections > 0) { @@ -457,6 +575,7 @@ void ExchangeReceiverBase::reactor(const std::vector & asyn } template +template void ExchangeReceiverBase::readLoop(const Request & req) { GET_METRIC(tiflash_thread_count, type_threads_of_receiver_read_loop).Increment(); @@ -481,20 +600,15 @@ void ExchangeReceiverBase::readLoop(const Request & req) for (;;) { LOG_FMT_TRACE(log, "begin next "); - auto recv_msg = std::make_shared(); - recv_msg->packet = std::make_shared(); - recv_msg->req_info = req_info; - recv_msg->source_index = req.source_index; - bool success = reader->read(recv_msg->packet); + MPPDataPacketPtr packet = std::make_shared(); + bool success = reader->read(packet); if (!success) break; has_data = true; - if (recv_msg->packet->has_error()) - throw Exception("Exchange receiver meet error : " + recv_msg->packet->error().msg()); + if (packet->has_error()) + throw Exception("Exchange receiver meet error : " + packet->error().msg()); - bool push_success = msg_channel.push(std::move(recv_msg)); - fiu_do_on(FailPoints::random_receiver_sync_msg_push_failure_failpoint, push_success = false;); - if (!push_success) + if (!pushPacket(req.source_index, req_info, packet, msg_channels, log)) { meet_error = true; auto local_state = getState(); @@ -564,15 +678,15 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( assert(recv_msg != nullptr); DecodeDetail detail; - int chunk_size = recv_msg->packet->chunks_size(); - if (chunk_size == 0) + if (recv_msg->chunks.empty()) return detail; + // Record total packet size even if fine grained shuffle is enabled. detail.packet_bytes = recv_msg->packet->ByteSizeLong(); - /// ExchangeReceiverBase should receive chunks of TypeCHBlock - for (int i = 0; i < chunk_size; ++i) + + for (const String * chunk : recv_msg->chunks) { - Block block = CHBlockChunkCodec::decode(recv_msg->packet->chunks(i), header); + Block block = CHBlockChunkCodec::decode(*chunk, header); detail.rows += block.rows(); if (unlikely(block.rows() == 0)) continue; @@ -582,10 +696,15 @@ DecodeDetail ExchangeReceiverBase::decodeChunks( } template -ExchangeReceiverResult ExchangeReceiverBase::nextResult(std::queue & block_queue, const Block & header) +ExchangeReceiverResult ExchangeReceiverBase::nextResult(std::queue & block_queue, const Block & header, size_t stream_id) { + if (unlikely(stream_id >= msg_channels.size())) + { + LOG_FMT_ERROR(exc_log, "stream_id out of range, stream_id: {}, total_stream_count: {}", stream_id, msg_channels.size()); + return {nullptr, 0, "", true, "stream_id out of range", false}; + } std::shared_ptr recv_msg; - if (!msg_channel.pop(recv_msg)) + if (!msg_channels[stream_id]->pop(recv_msg)) { std::unique_lock lock(mu); @@ -607,29 +726,32 @@ ExchangeReceiverResult ExchangeReceiverBase::nextResult(std::queuepacket != nullptr); + assert(recv_msg != nullptr); ExchangeReceiverResult result; - if (recv_msg->packet->has_error()) + if (recv_msg->error_ptr != nullptr) { - result = {nullptr, recv_msg->source_index, recv_msg->req_info, true, recv_msg->packet->error().msg(), false}; + result = {nullptr, recv_msg->source_index, recv_msg->req_info, true, recv_msg->error_ptr->msg(), false}; } else { - if (!recv_msg->packet->data().empty()) /// the data of the last packet is serialized from tipb::SelectResponse including execution summaries. + if (recv_msg->resp_ptr != nullptr) /// the data of the last packet is serialized from tipb::SelectResponse including execution summaries. { - auto resp_ptr = std::make_shared(); - if (!resp_ptr->ParseFromString(recv_msg->packet->data())) + auto select_resp = std::make_shared(); + if (!select_resp->ParseFromString(*(recv_msg->resp_ptr))) { result = {nullptr, recv_msg->source_index, recv_msg->req_info, true, "decode error", false}; } else { - result = {resp_ptr, recv_msg->source_index, recv_msg->req_info, false, "", false}; - /// If mocking TiFlash as TiDB, here should decode chunks from resp_ptr. - if (!resp_ptr->chunks().empty()) + result = {select_resp, recv_msg->source_index, recv_msg->req_info, false, "", false}; + /// If mocking TiFlash as TiDB, here should decode chunks from select_resp. + if (!select_resp->chunks().empty()) { - assert(recv_msg->packet->chunks().empty()); - result.decode_detail = CoprocessorReader::decodeChunks(resp_ptr, block_queue, header, schema); + assert(recv_msg->chunks.empty()); + // Fine grained shuffle should only be enabled when sending data to TiFlash node. + // So all data should be encoded into MPPDataPacket.chunks. + RUNTIME_CHECK(!enableFineGrainedShuffle(fine_grained_shuffle_stream_count), Exception, "Data should not be encoded into tipb::SelectResponse.chunks when fine grained shuffle is enabled"); + result.decode_detail = CoprocessorReader::decodeChunks(select_resp, block_queue, header, schema); } } } @@ -637,7 +759,7 @@ ExchangeReceiverResult ExchangeReceiverBase::nextResult(std::queuesource_index, recv_msg->req_info, false, "", false}; } - if (!result.meet_error && !recv_msg->packet->chunks().empty()) + if (!result.meet_error && !recv_msg->chunks.empty()) { assert(result.decode_detail.rows == 0); result.decode_detail = decodeChunks(recv_msg, block_queue, header); @@ -699,7 +821,21 @@ void ExchangeReceiverBase::connectionDone( throw Exception("live_connections should not be less than 0!"); if (meet_error || copy_live_conn == 0) - msg_channel.finish(); + finishAllMsgChannels(); +} + +template +void ExchangeReceiverBase::finishAllMsgChannels() +{ + for (auto & msg_channel : msg_channels) + msg_channel->finish(); +} + +template +void ExchangeReceiverBase::cancelAllMsgChannels() +{ + for (auto & msg_channel : msg_channels) + msg_channel->cancel(); } /// Explicit template instantiations - to avoid code bloat in headers. diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.h b/dbms/src/Flash/Mpp/ExchangeReceiver.h index 830dc6241a9..708f133f226 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.h +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.h @@ -35,9 +35,28 @@ namespace DB { struct ReceivedMessage { - std::shared_ptr packet; - size_t source_index = 0; + size_t source_index; String req_info; + // shared_ptr is copied to make sure error_ptr, resp_ptr and chunks are valid. + const std::shared_ptr packet; + const mpp::Error * error_ptr; + const String * resp_ptr; + std::vector chunks; + + // Constructor that move chunks. + ReceivedMessage(size_t source_index_, + const String & req_info_, + const std::shared_ptr & packet_, + const mpp::Error * error_ptr_, + const String * resp_ptr_, + std::vector && chunks_) + : source_index(source_index_) + , req_info(req_info_) + , packet(packet_) + , error_ptr(error_ptr_) + , resp_ptr(resp_ptr_) + , chunks(chunks_) + {} }; struct ExchangeReceiverResult @@ -78,6 +97,7 @@ enum class ExchangeReceiverState CLOSED, }; +using MsgChannelPtr = std::unique_ptr>>; template class ExchangeReceiverBase @@ -92,7 +112,8 @@ class ExchangeReceiverBase size_t source_num_, size_t max_streams_, const String & req_id, - const String & executor_id); + const String & executor_id, + uint64_t fine_grained_shuffle_stream_count); ~ExchangeReceiverBase(); @@ -104,9 +125,11 @@ class ExchangeReceiverBase ExchangeReceiverResult nextResult( std::queue & block_queue, - const Block & header); + const Block & header, + size_t stream_id); size_t getSourceNum() const { return source_num; } + uint64_t getFineGrainedShuffleStreamCount() const { return fine_grained_shuffle_stream_count; } int computeNewThreadCount() const { return thread_count; } @@ -128,7 +151,10 @@ class ExchangeReceiverBase using Request = typename RPCContext::Request; void setUpConnection(); + // Template argument enable_fine_grained_shuffle will be setup properly in setUpConnection(). + template void readLoop(const Request & req); + template void reactor(const std::vector & async_requests); bool setEndState(ExchangeReceiverState new_state); @@ -139,12 +165,14 @@ class ExchangeReceiverBase std::queue & block_queue, const Block & header); - void connectionDone( bool meet_error, const String & local_err_msg, const LoggerPtr & log); + void finishAllMsgChannels(); + void cancelAllMsgChannels(); + std::shared_ptr rpc_context; const tipb::ExchangeReceiver pb_exchange_receiver; @@ -156,7 +184,7 @@ class ExchangeReceiverBase std::shared_ptr thread_manager; DAGSchema schema; - MPMCQueue> msg_channel; + std::vector msg_channels; std::mutex mu; /// should lock `mu` when visit these members @@ -168,6 +196,7 @@ class ExchangeReceiverBase bool collected = false; int thread_count = 0; + uint64_t fine_grained_shuffle_stream_count; }; class ExchangeReceiver : public ExchangeReceiverBase diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index da8f3034abc..7ddc6af361f 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -172,7 +172,8 @@ void MPPTask::initExchangeReceivers() executor.exchange_receiver().encoded_task_meta_size(), context->getMaxStreams(), log->identifier(), - executor_id); + executor_id, + executor.fine_grained_shuffle_stream_count()); if (status != RUNNING) throw Exception("exchange receiver map can not be initialized, because the task is not in running state"); diff --git a/dbms/src/Flash/tests/WindowTestUtil.h b/dbms/src/Flash/tests/WindowTestUtil.h index 3f4cb7d595f..b7385380419 100644 --- a/dbms/src/Flash/tests/WindowTestUtil.h +++ b/dbms/src/Flash/tests/WindowTestUtil.h @@ -39,9 +39,9 @@ inline void mockExecuteProject(std::shared_ptr & mock_ mock_interpreter->executeProject(pipeline, final_project); } -inline void mockExecuteWindowOrder(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, const tipb::Sort & sort) +inline void mockExecuteWindowOrder(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, const tipb::Sort & sort, uint64_t fine_grained_shuffle_stream_count) { - mock_interpreter->handleWindowOrder(pipeline, sort); + mock_interpreter->handleWindowOrder(pipeline, sort, ::DB::enableFineGrainedShuffle(fine_grained_shuffle_stream_count)); mock_interpreter->input_streams_vec[0] = pipeline.streams; NamesWithAliases final_project; for (const auto & column : (*mock_interpreter->analyzer).source_columns) @@ -51,16 +51,9 @@ inline void mockExecuteWindowOrder(std::shared_ptr & m mockExecuteProject(mock_interpreter, pipeline, final_project); } -inline void mockExecuteWindowOrder(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, const String & sort_json) +inline void mockExecuteWindow(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, const tipb::Window & window, uint64_t fine_grained_shuffle_stream_count) { - tipb::Sort sort; - ::google::protobuf::util::JsonStringToMessage(sort_json, &sort); - mockExecuteWindowOrder(mock_interpreter, pipeline, sort); -} - -inline void mockExecuteWindow(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, const tipb::Window & window) -{ - mock_interpreter->handleWindow(pipeline, window); + mock_interpreter->handleWindow(pipeline, window, ::DB::enableFineGrainedShuffle(fine_grained_shuffle_stream_count)); mock_interpreter->input_streams_vec[0] = pipeline.streams; NamesWithAliases final_project; for (const auto & column : (*mock_interpreter->analyzer).source_columns) @@ -70,12 +63,5 @@ inline void mockExecuteWindow(std::shared_ptr & mock_i mockExecuteProject(mock_interpreter, pipeline, final_project); } -inline void mockExecuteWindow(std::shared_ptr & mock_interpreter, DAGPipeline & pipeline, std::string window_json_str) -{ - tipb::Window window; - google::protobuf::util::JsonStringToMessage(window_json_str, &window); - mockExecuteWindow(mock_interpreter, pipeline, window); -} - } // namespace tests } // namespace DB diff --git a/dbms/src/Flash/tests/bench_exchange.cpp b/dbms/src/Flash/tests/bench_exchange.cpp index cbbdf060580..d6e3f3e825e 100644 --- a/dbms/src/Flash/tests/bench_exchange.cpp +++ b/dbms/src/Flash/tests/bench_exchange.cpp @@ -47,29 +47,46 @@ MockFixedRowsBlockInputStream::MockFixedRowsBlockInputStream(size_t total_rows_, , blocks(blocks_) {} -Block makeBlock(int row_num) +Block makeBlock(int row_num, bool skew) { - std::mt19937 mt(rd()); - std::uniform_int_distribution int64_dist; - std::uniform_int_distribution len_dist(10, 20); - std::uniform_int_distribution char_dist; - InferredDataVector> int64_vec; InferredDataVector> int64_vec2; - for (int i = 0; i < row_num; ++i) + InferredDataVector> string_vec; + + if (skew) { - int64_vec.emplace_back(int64_dist(mt)); - int64_vec2.emplace_back(int64_dist(mt)); + for (int i = 0; i < row_num; ++i) + { + int64_vec.emplace_back(100); + int64_vec2.emplace_back(100); + } + + for (int i = 0; i < row_num; ++i) + { + string_vec.push_back("abcdefg"); + } } - - InferredDataVector> string_vec; - for (int i = 0; i < row_num; ++i) + else { - int len = len_dist(mt); - String s; - for (int j = 0; j < len; ++j) - s.push_back(char_dist(mt)); - string_vec.push_back(std::move(s)); + std::mt19937 mt(rd()); + std::uniform_int_distribution int64_dist; + std::uniform_int_distribution len_dist(10, 20); + std::uniform_int_distribution char_dist; + + for (int i = 0; i < row_num; ++i) + { + int64_vec.emplace_back(int64_dist(mt)); + int64_vec2.emplace_back(int64_dist(mt)); + } + + for (int i = 0; i < row_num; ++i) + { + int len = len_dist(mt); + String s; + for (int j = 0; j < len; ++j) + s.push_back(char_dist(mt)); + string_vec.push_back(std::move(s)); + } } auto int64_data_type = makeDataType>(); @@ -82,11 +99,11 @@ Block makeBlock(int row_num) return Block({int64_column, string_column, int64_column2}); } -std::vector makeBlocks(int block_num, int row_num) +std::vector makeBlocks(int block_num, int row_num, bool skew) { std::vector blocks; for (int i = 0; i < block_num; ++i) - blocks.push_back(makeBlock(row_num)); + blocks.push_back(makeBlock(row_num, skew)); return blocks; } @@ -139,32 +156,10 @@ void printException(const Exception & e) << e.getStackTrace().toString() << std::endl; } -void sendPacket(const std::vector & packets, const PacketQueuePtr & queue, StopFlag & stop_flag) -{ - std::mt19937 mt(rd()); - std::uniform_int_distribution dist(0, packets.size() - 1); - - while (!stop_flag.load()) - { - int i = dist(mt); - queue->tryPush(packets[i], std::chrono::milliseconds(10)); - } - queue->finish(); -} - -void receivePacket(const PacketQueuePtr & queue) -{ - while (true) - { - PacketPtr packet; - if (!queue->pop(packet)) - break; - } -} - -ReceiverHelper::ReceiverHelper(int concurrency_, int source_num_) +ReceiverHelper::ReceiverHelper(int concurrency_, int source_num_, uint32_t fine_grained_shuffle_stream_count_) : concurrency(concurrency_) , source_num(source_num_) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) { pb_exchange_receiver.set_tp(tipb::Hash); for (int i = 0; i < source_num; ++i) @@ -198,16 +193,21 @@ MockExchangeReceiverPtr ReceiverHelper::buildReceiver() source_num, concurrency, "mock_req_id", - "mock_exchange_receiver_id"); + "mock_exchange_receiver_id", + fine_grained_shuffle_stream_count); } std::vector ReceiverHelper::buildExchangeReceiverStream() { auto receiver = buildReceiver(); std::vector streams(concurrency); + // NOTE: check if need fine_grained_shuffle_stream_count for (int i = 0; i < concurrency; ++i) { - streams[i] = std::make_shared(receiver, "mock_req_id", "mock_executor_id" + std::to_string(i)); + streams[i] = std::make_shared(receiver, + "mock_req_id", + "mock_executor_id" + std::to_string(i), + /*stream_id=*/enableFineGrainedShuffle(fine_grained_shuffle_stream_count) ? i : 0); } return streams; } @@ -230,10 +230,14 @@ void ReceiverHelper::finish() SenderHelper::SenderHelper( int source_num_, int concurrency_, + uint32_t fine_grained_shuffle_stream_count_, + int64_t fine_grained_shuffle_batch_size_, const std::vector & queues_, const std::vector & fields) : source_num(source_num_) , concurrency(concurrency_) + , fine_grained_shuffle_stream_count(fine_grained_shuffle_stream_count_) + , fine_grained_shuffle_batch_size(fine_grained_shuffle_batch_size_) , queues(queues_) { mpp::TaskMeta task_meta; @@ -277,17 +281,38 @@ BlockInputStreamPtr SenderHelper::buildUnionStream( for (int i = 0; i < concurrency; ++i) { BlockInputStreamPtr stream = std::make_shared(blocks, stop_flag); - std::unique_ptr response_writer( - new StreamingDAGResponseWriter( - tunnel_set, - {0, 1, 2}, - TiDB::TiDBCollators(3), - tipb::Hash, - -1, - -1, - true, - *dag_context)); - send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); + if (enableFineGrainedShuffle(fine_grained_shuffle_stream_count)) + { + std::unique_ptr response_writer( + new StreamingDAGResponseWriter( + tunnel_set, + {0, 1, 2}, + TiDB::TiDBCollators(3), + tipb::Hash, + -1, + -1, + true, + *dag_context, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size)); + send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); + } + else + { + std::unique_ptr response_writer( + new StreamingDAGResponseWriter( + tunnel_set, + {0, 1, 2}, + TiDB::TiDBCollators(3), + tipb::Hash, + -1, + -1, + true, + *dag_context, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size)); + send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); + } } return std::make_shared>(send_streams, BlockInputStreams{}, concurrency, /*req_id=*/""); @@ -299,17 +324,38 @@ BlockInputStreamPtr SenderHelper::buildUnionStream(size_t total_rows, const std: for (int i = 0; i < concurrency; ++i) { BlockInputStreamPtr stream = std::make_shared(total_rows / concurrency, blocks); - std::unique_ptr response_writer( - new StreamingDAGResponseWriter( - tunnel_set, - {0, 1, 2}, - TiDB::TiDBCollators(3), - tipb::Hash, - -1, - -1, - true, - *dag_context)); - send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); + if (enableFineGrainedShuffle(fine_grained_shuffle_stream_count)) + { + std::unique_ptr response_writer( + new StreamingDAGResponseWriter( + tunnel_set, + {0, 1, 2}, + TiDB::TiDBCollators(3), + tipb::Hash, + -1, + -1, + true, + *dag_context, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size)); + send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); + } + else + { + std::unique_ptr response_writer( + new StreamingDAGResponseWriter( + tunnel_set, + {0, 1, 2}, + TiDB::TiDBCollators(3), + tipb::Hash, + -1, + -1, + true, + *dag_context, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size)); + send_streams.push_back(std::make_shared(stream, std::move(response_writer), /*req_id=*/"")); + } } return std::make_shared>(send_streams, BlockInputStreams{}, concurrency, /*req_id=*/""); @@ -327,13 +373,12 @@ void SenderHelper::finish() void ExchangeBench::SetUp(const benchmark::State &) { - Poco::Logger::root().setLevel("error"); - DynamicThreadPool::global_instance = std::make_unique( /*fixed_thread_num=*/300, std::chrono::milliseconds(100000)); - input_blocks = makeBlocks(/*block_num=*/100, /*row_num=*/1024); + uniform_blocks = makeBlocks(/*block_num=*/100, /*row_num=*/1024); + skew_blocks = makeBlocks(/*block_num=*/100, /*row_num=*/1024, /*skew=*/true); try { @@ -348,7 +393,8 @@ void ExchangeBench::SetUp(const benchmark::State &) void ExchangeBench::TearDown(const benchmark::State &) { - input_blocks.clear(); + uniform_blocks.clear(); + skew_blocks.clear(); // NOTE: Must reset here, otherwise DynamicThreadPool::fixedWork() may core because metrics already destroyed. DynamicThreadPool::global_instance.reset(); } @@ -383,25 +429,38 @@ try const int concurrency = state.range(0); const int source_num = state.range(1); const int total_rows = state.range(2); + const int fine_grained_shuffle_stream_count = state.range(3); + const int fine_grained_shuffle_batch_size = state.range(4); Context context = TiFlashTestEnv::getContext(); for (auto _ : state) { - std::shared_ptr receiver_helper = std::make_shared(concurrency, source_num); + std::shared_ptr receiver_helper = std::make_shared(concurrency, source_num, fine_grained_shuffle_stream_count); BlockInputStreamPtr receiver_stream = receiver_helper->buildUnionStream(); std::shared_ptr sender_helper = std::make_shared(source_num, concurrency, + fine_grained_shuffle_stream_count, + fine_grained_shuffle_batch_size, receiver_helper->queues, receiver_helper->fields); - BlockInputStreamPtr sender_stream = sender_helper->buildUnionStream(total_rows, input_blocks); + BlockInputStreamPtr sender_stream = sender_helper->buildUnionStream(total_rows, uniform_blocks); runAndWait(receiver_helper, receiver_stream, sender_helper, sender_stream); } } CATCH BENCHMARK_REGISTER_F(ExchangeBench, basic_send_receive) - ->Args({8, 1, 1024 * 1000}); + ->Args({8, 1, 1024 * 1000, 0, 4096}) + ->Args({8, 1, 1024 * 1000, 4, 4096}) + ->Args({8, 1, 1024 * 1000, 8, 4096}) + ->Args({8, 1, 1024 * 1000, 16, 4096}) + ->Args({8, 1, 1024 * 1000, 32, 4096}) + ->Args({8, 1, 1024 * 1000, 8, 1}) + ->Args({8, 1, 1024 * 1000, 8, 1000}) + ->Args({8, 1, 1024 * 1000, 8, 10000}) + ->Args({8, 1, 1024 * 1000, 8, 100000}); + } // namespace tests } // namespace DB diff --git a/dbms/src/Flash/tests/bench_exchange.h b/dbms/src/Flash/tests/bench_exchange.h index 6b09e319613..d8300d45740 100644 --- a/dbms/src/Flash/tests/bench_exchange.h +++ b/dbms/src/Flash/tests/bench_exchange.h @@ -69,7 +69,9 @@ struct MockReceiverContext : queue(queue_) {} - void initialize() const {} + void initialize() const + { + } bool read(PacketPtr & packet [[maybe_unused]]) const { @@ -105,7 +107,8 @@ struct MockReceiverContext const std::vector & field_types_) : queues(queues_) , field_types(field_types_) - {} + { + } void fillSchema(DAGSchema & schema) const { @@ -220,8 +223,8 @@ struct MockFixedRowsBlockInputStream : public IProfilingBlockInputStream } }; -Block makeBlock(int row_num); -std::vector makeBlocks(int block_num, int row_num); +Block makeBlock(int row_num, bool skew = false); +std::vector makeBlocks(int block_num, int row_num, bool skew = false); mpp::MPPDataPacket makePacket(ChunkCodecStream & codec, int row_num); std::vector makePackets(ChunkCodecStream & codec, int packet_num, int row_num); std::vector makePacketQueues(int source_num, int queue_size); @@ -234,17 +237,17 @@ struct ReceiverHelper { const int concurrency; const int source_num; + const uint32_t fine_grained_shuffle_stream_count; tipb::ExchangeReceiver pb_exchange_receiver; std::vector fields; mpp::TaskMeta task_meta; std::vector queues; std::shared_ptr join_ptr; - explicit ReceiverHelper(int concurrency_, int source_num_); + explicit ReceiverHelper(int concurrency_, int source_num_, uint32_t fine_grained_shuffle_stream_count_); MockExchangeReceiverPtr buildReceiver(); std::vector buildExchangeReceiverStream(); BlockInputStreamPtr buildUnionStream(); - BlockInputStreamPtr buildUnionStreamWithHashJoinBuildStream(); void finish(); }; @@ -252,6 +255,8 @@ struct SenderHelper { const int source_num; const int concurrency; + const uint32_t fine_grained_shuffle_stream_count; + const int64_t fine_grained_shuffle_batch_size; std::vector queues; std::vector mock_writers; @@ -262,6 +267,8 @@ struct SenderHelper SenderHelper( int source_num_, int concurrency_, + uint32_t fine_grained_shuffle_stream_count_, + int64_t fine_grained_shuffle_batch_size_, const std::vector & queues_, const std::vector & fields); @@ -283,7 +290,8 @@ class ExchangeBench : public benchmark::Fixture std::shared_ptr & sender_helper, BlockInputStreamPtr sender_stream); - std::vector input_blocks; + std::vector uniform_blocks; + std::vector skew_blocks; }; diff --git a/dbms/src/Flash/tests/bench_window.cpp b/dbms/src/Flash/tests/bench_window.cpp index 356f544a836..75dc53b065b 100644 --- a/dbms/src/Flash/tests/bench_window.cpp +++ b/dbms/src/Flash/tests/bench_window.cpp @@ -24,9 +24,13 @@ class WindowFunctionBench : public ExchangeBench public: void SetUp(const benchmark::State & state) override { - // build tipb::Window and tipb::Sort. + // Using DAGRequestBuilder to build tipb::Window and tipb::Sort. // select row_number() over w1 from t1 window w1 as (partition by c1, c2, c3 order by c1, c2, c3); ExchangeBench::SetUp(state); + } + + static void setupPB(uint64_t fine_grained_shuffle_stream_count, tipb::Window & window, tipb::Sort & sort) + { MockColumnInfoVec columns{ {"c1", TiDB::TP::TypeLongLong}, {"c2", TiDB::TP::TypeString}, @@ -36,11 +40,12 @@ class WindowFunctionBench : public ExchangeBench DAGRequestBuilder builder(executor_index); builder .mockTable("test", "t1", columns) - .sort({{"c1", false}, {"c2", false}, {"c3", false}}, true) + .sort({{"c1", false}, {"c2", false}, {"c3", false}}, true, fine_grained_shuffle_stream_count) .window(RowNumber(), {{"c1", false}, {"c2", false}, {"c3", false}}, {{"c1", false}, {"c2", false}, {"c3", false}}, - buildDefaultRowsFrame()); + buildDefaultRowsFrame(), + fine_grained_shuffle_stream_count); tipb::DAGRequest req; MPPInfo mpp_info(0, -1, -1, {}, std::unordered_map>{}); builder.getRoot()->toTiPBExecutor(req.mutable_root_executor(), /*collator_id=*/0, mpp_info, TiFlashTestEnv::getContext()); @@ -50,13 +55,17 @@ class WindowFunctionBench : public ExchangeBench sort = window.child().sort(); } - void prepareWindowStream(Context & context, int concurrency, int source_num, int total_rows, const std::vector & blocks, BlockInputStreamPtr & sender_stream, BlockInputStreamPtr & receiver_stream, std::shared_ptr & sender_helper, std::shared_ptr & receiver_helper) const + static void prepareWindowStream(Context & context, int concurrency, int source_num, int total_rows, uint32_t fine_grained_shuffle_stream_count, uint64_t fine_grained_shuffle_batch_size, const std::vector & blocks, BlockInputStreamPtr & sender_stream, BlockInputStreamPtr & receiver_stream, std::shared_ptr & sender_helper, std::shared_ptr & receiver_helper, bool build_window = true) { + tipb::Window window; + tipb::Sort sort; + setupPB(fine_grained_shuffle_stream_count, window, sort); + DAGPipeline pipeline; - receiver_helper = std::make_shared(concurrency, source_num); + receiver_helper = std::make_shared(concurrency, source_num, fine_grained_shuffle_stream_count); pipeline.streams = receiver_helper->buildExchangeReceiverStream(); - sender_helper = std::make_shared(source_num, concurrency, receiver_helper->queues, receiver_helper->fields); + sender_helper = std::make_shared(source_num, concurrency, fine_grained_shuffle_stream_count, fine_grained_shuffle_batch_size, receiver_helper->queues, receiver_helper->fields); sender_stream = sender_helper->buildUnionStream(total_rows, blocks); context.setDAGContext(sender_helper->dag_context.get()); @@ -66,16 +75,16 @@ class WindowFunctionBench : public ExchangeBench NameAndTypePair("c3", makeNullable(std::make_shared()))}; auto mock_interpreter = mockInterpreter(context, source_columns, concurrency); mock_interpreter->input_streams_vec.push_back(pipeline.streams); - mockExecuteWindowOrder(mock_interpreter, pipeline, sort); - mockExecuteWindow(mock_interpreter, pipeline, window); + mockExecuteWindowOrder(mock_interpreter, pipeline, sort, fine_grained_shuffle_stream_count); + if (build_window) + { + mockExecuteWindow(mock_interpreter, pipeline, window, fine_grained_shuffle_stream_count); + } pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, 8192, 0, "mock_executor_id_squashing"); }); receiver_stream = std::make_shared>(pipeline.streams, BlockInputStreams{}, concurrency, /*req_id=*/""); } - - tipb::Window window; - tipb::Sort sort; }; BENCHMARK_DEFINE_F(WindowFunctionBench, basic_row_number) @@ -85,8 +94,15 @@ try const int concurrency = state.range(0); const int source_num = state.range(1); const int total_rows = state.range(2); + const int fine_grained_shuffle_stream_count = state.range(3); + const int fine_grained_shuffle_batch_size = state.range(4); + const bool skew = state.range(5); Context context = TiFlashTestEnv::getContext(); + std::vector * blocks = &uniform_blocks; + if (skew) + blocks = &skew_blocks; + for (auto _ : state) { std::shared_ptr sender_helper; @@ -94,14 +110,58 @@ try BlockInputStreamPtr sender_stream; BlockInputStreamPtr receiver_stream; - prepareWindowStream(context, concurrency, source_num, total_rows, input_blocks, sender_stream, receiver_stream, sender_helper, receiver_helper); + prepareWindowStream(context, concurrency, source_num, total_rows, fine_grained_shuffle_stream_count, fine_grained_shuffle_batch_size, *blocks, sender_stream, receiver_stream, sender_helper, receiver_helper); runAndWait(receiver_helper, receiver_stream, sender_helper, sender_stream); } } CATCH BENCHMARK_REGISTER_F(WindowFunctionBench, basic_row_number) - ->Args({8, 1, 1024 * 1000}); + ->Args({8, 1, 1024 * 1000, 0, 4096, false}) // Test fine_grained_shuffle_stream_count. + ->Args({8, 1, 1024 * 1000, 4, 4096, false}) + ->Args({8, 1, 1024 * 1000, 8, 4096, false}) + ->Args({8, 1, 1024 * 1000, 16, 4096, false}) + ->Args({8, 1, 1024 * 1000, 32, 4096, false}) + ->Args({8, 1, 1024 * 1000, 8, 1, false}) // Test fine_grained_shuffle_batch_size. + ->Args({8, 1, 1024 * 1000, 8, 1000, false}) + ->Args({8, 1, 1024 * 1000, 8, 10000, false}) + ->Args({8, 1, 1024 * 1000, 8, 100000, false}) + ->Args({8, 1, 1024 * 1000, 0, 4096, true}) // Test skew dataset. + ->Args({8, 1, 1024 * 1000, 4, 4096, true}) + ->Args({8, 1, 1024 * 1000, 8, 4096, true}) + ->Args({8, 1, 1024 * 1000, 16, 4096, true}); + +BENCHMARK_DEFINE_F(WindowFunctionBench, partial_sort_skew_dataset) +(benchmark::State & state) +try +{ + const int concurrency = state.range(0); + const int source_num = state.range(1); + const int total_rows = state.range(2); + const int fine_grained_shuffle_stream_count = state.range(3); + const int fine_grained_shuffle_batch_size = state.range(4); + Context context = TiFlashTestEnv::getContext(); + std::vector * blocks = &skew_blocks; + + for (auto _ : state) + { + std::shared_ptr sender_helper; + std::shared_ptr receiver_helper; + BlockInputStreamPtr sender_stream; + BlockInputStreamPtr receiver_stream; + + // Only build partial sort. + prepareWindowStream(context, concurrency, source_num, total_rows, fine_grained_shuffle_stream_count, fine_grained_shuffle_batch_size, *blocks, sender_stream, receiver_stream, sender_helper, receiver_helper, /*build_window=*/false); + + runAndWait(receiver_helper, receiver_stream, sender_helper, sender_stream); + } +} +CATCH +BENCHMARK_REGISTER_F(WindowFunctionBench, partial_sort_skew_dataset) + ->Args({1, 1, 1024 * 10000, 0, 4096}) // Test how much multiple-thread improves performance for partial sort. + ->Args({2, 1, 1024 * 10000, 0, 4096}) + ->Args({4, 1, 1024 * 10000, 0, 4096}) + ->Args({8, 1, 1024 * 10000, 0, 4096}); } // namespace tests } // namespace DB diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp index c583fbf35c6..53b260f9638 100644 --- a/dbms/src/Flash/tests/gtest_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_interpreter.cpp @@ -340,6 +340,78 @@ Union: } CATCH +TEST_F(InterpreterExecuteTest, FineGrainedShuffle) +try +{ + // fine-grained shuffle is enabled. + const uint64_t enable = 8; + const uint64_t disable = 0; + auto request = context + .receive("sender_1", enable) + .sort({{"s1", true}, {"s2", false}}, true, enable) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame(), enable) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + Window: , function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Expression: + MergeSorting: , limit = 0 + PartialSorting: : limit = 0 + Expression: + MockExchangeReceiver + )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + auto topn_request = context + .receive("sender_1") + .topN("s2", false, 10) + .build(context); + String topn_expected = R"( +Union: + SharedQuery x 10: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockExchangeReceiver + )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(topn_expected, topn_request, 10); + + // fine-grained shuffle is disabled. + request = context + .receive("sender_1", disable) + .sort({{"s1", true}, {"s2", false}}, true, disable) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame(), disable) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Expression: + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + Expression: + MockExchangeReceiver + )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + topn_request = context + .receive("sender_1") + .topN("s2", false, 10) + .build(context); + ASSERT_BLOCKINPUTSTREAM_EQAUL(topn_expected, topn_request, 10); +} +CATCH + TEST_F(InterpreterExecuteTest, Join) try { @@ -586,4 +658,4 @@ CreatingSets CATCH } // namespace tests -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/TestUtils/bench_dbms_main.cpp b/dbms/src/TestUtils/bench_dbms_main.cpp index 48bd02a71f7..092c45c35e2 100644 --- a/dbms/src/TestUtils/bench_dbms_main.cpp +++ b/dbms/src/TestUtils/bench_dbms_main.cpp @@ -20,6 +20,8 @@ int main(int argc, char * argv[]) { benchmark::Initialize(&argc, argv); DB::tests::TiFlashTestEnv::setupLogger(); + // Each time TiFlashTestEnv::getContext() is called, some log will print, it's annoying. + Poco::Logger::root().setLevel("error"); DB::tests::TiFlashTestEnv::initializeGlobalContext(); if (::benchmark::ReportUnrecognizedArguments(argc, argv)) return 1; diff --git a/dbms/src/TestUtils/mockExecutor.cpp b/dbms/src/TestUtils/mockExecutor.cpp index 49ae9787ea4..30d05786c9a 100644 --- a/dbms/src/TestUtils/mockExecutor.cpp +++ b/dbms/src/TestUtils/mockExecutor.cpp @@ -119,12 +119,12 @@ DAGRequestBuilder & DAGRequestBuilder::mockTable(const MockTableName & name, con return mockTable(name.first, name.second, columns); } -DAGRequestBuilder & DAGRequestBuilder::exchangeReceiver(const MockColumnInfoVec & columns) +DAGRequestBuilder & DAGRequestBuilder::exchangeReceiver(const MockColumnInfoVec & columns, uint64_t fine_grained_shuffle_stream_count) { - return buildExchangeReceiver(columns); + return buildExchangeReceiver(columns, fine_grained_shuffle_stream_count); } -DAGRequestBuilder & DAGRequestBuilder::buildExchangeReceiver(const MockColumnInfoVec & columns) +DAGRequestBuilder & DAGRequestBuilder::buildExchangeReceiver(const MockColumnInfoVec & columns, uint64_t fine_grained_shuffle_stream_count) { DAGSchema schema; for (const auto & column : columns) @@ -135,7 +135,7 @@ DAGRequestBuilder & DAGRequestBuilder::buildExchangeReceiver(const MockColumnInf schema.push_back({column.first, info}); } - root = compileExchangeReceiver(getExecutorIndex(), schema); + root = compileExchangeReceiver(getExecutorIndex(), schema, fine_grained_shuffle_stream_count); return *this; } @@ -266,45 +266,45 @@ DAGRequestBuilder & DAGRequestBuilder::buildAggregation(ASTPtr agg_funcs, ASTPtr return *this; } -DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByItem order_by, MockPartitionByItem partition_by, MockWindowFrame frame) +DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByItem order_by, MockPartitionByItem partition_by, MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count) { assert(root); auto window_func_list = std::make_shared(); window_func_list->children.push_back(window_func); - root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec({partition_by}), buildOrderByItemVec({order_by}), frame); + root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec({partition_by}), buildOrderByItemVec({order_by}), frame, fine_grained_shuffle_stream_count); return *this; } -DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame) +DAGRequestBuilder & DAGRequestBuilder::window(ASTPtr window_func, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count) { assert(root); auto window_func_list = std::make_shared(); window_func_list->children.push_back(window_func); - root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec(partition_by_vec), buildOrderByItemVec(order_by_vec), frame); + root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec(partition_by_vec), buildOrderByItemVec(order_by_vec), frame, fine_grained_shuffle_stream_count); return *this; } -DAGRequestBuilder & DAGRequestBuilder::window(MockAstVec window_funcs, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame) +DAGRequestBuilder & DAGRequestBuilder::window(MockAstVec window_funcs, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count) { assert(root); auto window_func_list = std::make_shared(); for (const auto & func : window_funcs) window_func_list->children.push_back(func); - root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec(partition_by_vec), buildOrderByItemVec(order_by_vec), frame); + root = compileWindow(root, getExecutorIndex(), window_func_list, buildOrderByItemVec(partition_by_vec), buildOrderByItemVec(order_by_vec), frame, fine_grained_shuffle_stream_count); return *this; } -DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItem order_by, bool is_partial_sort) +DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItem order_by, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count) { assert(root); - root = compileSort(root, getExecutorIndex(), buildOrderByItemVec({order_by}), is_partial_sort); + root = compileSort(root, getExecutorIndex(), buildOrderByItemVec({order_by}), is_partial_sort, fine_grained_shuffle_stream_count); return *this; } -DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItemVec order_by_vec, bool is_partial_sort) +DAGRequestBuilder & DAGRequestBuilder::sort(MockOrderByItemVec order_by_vec, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count) { assert(root); - root = compileSort(root, getExecutorIndex(), buildOrderByItemVec(order_by_vec), is_partial_sort); + root = compileSort(root, getExecutorIndex(), buildOrderByItemVec(order_by_vec), is_partial_sort, fine_grained_shuffle_stream_count); return *this; } @@ -368,9 +368,9 @@ DAGRequestBuilder MockDAGRequestContext::scan(String db_name, String table_name) return builder; } -DAGRequestBuilder MockDAGRequestContext::receive(String exchange_name) +DAGRequestBuilder MockDAGRequestContext::receive(String exchange_name, uint64_t fine_grained_shuffle_stream_count) { - auto builder = DAGRequestBuilder(index).exchangeReceiver(exchange_schemas[exchange_name]); + auto builder = DAGRequestBuilder(index).exchangeReceiver(exchange_schemas[exchange_name], fine_grained_shuffle_stream_count); receiver_source_task_ids_map[builder.getRoot()->name] = {}; // If don't have related columns, user must pass input columns as argument of executeStreams in order to run Executors Tests. // If user don't want to test executors, it will be safe to run Interpreter Tests. @@ -380,5 +380,4 @@ DAGRequestBuilder MockDAGRequestContext::receive(String exchange_name) } return builder; } - } // namespace DB::tests diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index 5f752e58da6..c034a8f86ca 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -66,7 +66,7 @@ class DAGRequestBuilder DAGRequestBuilder & mockTable(const String & db, const String & table, const MockColumnInfoVec & columns); DAGRequestBuilder & mockTable(const MockTableName & name, const MockColumnInfoVec & columns); - DAGRequestBuilder & exchangeReceiver(const MockColumnInfoVec & columns); + DAGRequestBuilder & exchangeReceiver(const MockColumnInfoVec & columns, uint64_t fine_grained_shuffle_stream_count = 0); DAGRequestBuilder & filter(ASTPtr filter_expr); @@ -93,16 +93,16 @@ class DAGRequestBuilder DAGRequestBuilder & aggregation(MockAstVec agg_funcs, MockAstVec group_by_exprs); // window - DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItem order_by, MockPartitionByItem partition_by, MockWindowFrame frame); - DAGRequestBuilder & window(MockAstVec window_funcs, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame); - DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame); - DAGRequestBuilder & sort(MockOrderByItem order_by, bool is_partial_sort); - DAGRequestBuilder & sort(MockOrderByItemVec order_by_vec, bool is_partial_sort); + DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItem order_by, MockPartitionByItem partition_by, MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count = 0); + DAGRequestBuilder & window(MockAstVec window_funcs, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count = 0); + DAGRequestBuilder & window(ASTPtr window_func, MockOrderByItemVec order_by_vec, MockPartitionByItemVec partition_by_vec, MockWindowFrame frame, uint64_t fine_grained_shuffle_stream_count = 0); + DAGRequestBuilder & sort(MockOrderByItem order_by, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count = 0); + DAGRequestBuilder & sort(MockOrderByItemVec order_by_vec, bool is_partial_sort, uint64_t fine_grained_shuffle_stream_count = 0); private: void initDAGRequest(tipb::DAGRequest & dag_request); DAGRequestBuilder & buildAggregation(ASTPtr agg_funcs, ASTPtr group_by_exprs); - DAGRequestBuilder & buildExchangeReceiver(const MockColumnInfoVec & columns); + DAGRequestBuilder & buildExchangeReceiver(const MockColumnInfoVec & columns, uint64_t fine_grained_shuffle_stream_count = 0); ExecutorPtr root; DAGProperties properties; @@ -139,7 +139,7 @@ class MockDAGRequestContext std::unordered_map & executorIdColumnsMap() { return executor_id_columns_map; } DAGRequestBuilder scan(String db_name, String table_name); - DAGRequestBuilder receive(String exchange_name); + DAGRequestBuilder receive(String exchange_name, uint64_t fine_grained_shuffle_stream_count = 0); private: size_t index; diff --git a/tests/fullstack-test/mpp/window.test b/tests/fullstack-test/mpp/window.test new file mode 100644 index 00000000000..698d39ef2ea --- /dev/null +++ b/tests/fullstack-test/mpp/window.test @@ -0,0 +1,32 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +mysql> drop table if exists test.t1; +mysql> create table test.t1(c1 int, c2 int); +mysql> insert into test.t1 values(1, 1),(2, 2),(3, 3),(1, 1),(2, 2),(3, 3),(4, 4); +mysql> alter table test.t1 set tiflash replica 1; +func> wait_table test t1 +mysql> use test; set @@tidb_isolation_read_engines='tiflash'; select c1, c2, row_number() over w2, row_number() over w1 from test.t1 window w1 as(partition by c1), w2 as (partition by c1, c2) order by 1, 2, 3, 4; ++------+------+----------------------+----------------------+ +| c1 | c2 | row_number() over w2 | row_number() over w1 | ++------+------+----------------------+----------------------+ +| 1 | 1 | 1 | 1 | +| 1 | 1 | 2 | 2 | +| 2 | 2 | 1 | 1 | +| 2 | 2 | 2 | 2 | +| 3 | 3 | 1 | 1 | +| 3 | 3 | 2 | 2 | +| 4 | 4 | 1 | 1 | ++------+------+----------------------+----------------------+ +mysql> drop table if exists test.t1; From 707fc6d9759446a44460af4a9e77d519c2268f97 Mon Sep 17 00:00:00 2001 From: Xuanyi Li Date: Sun, 10 Jul 2022 22:57:04 -0700 Subject: [PATCH 098/104] feat: pushdown get_format into TiFlash (#5269) close pingcap/tiflash#5115 --- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 2 +- dbms/src/Functions/FunctionsConversion.cpp | 1 + dbms/src/Functions/FunctionsConversion.h | 114 +++++++++++++ dbms/src/Functions/tests/gtest_get_format.cpp | 153 ++++++++++++++++++ tests/fullstack-test/expr/get_format.test | 60 +++++++ 5 files changed, 329 insertions(+), 1 deletion(-) create mode 100644 dbms/src/Functions/tests/gtest_get_format.cpp create mode 100644 tests/fullstack-test/expr/get_format.test diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index a4d491e5637..2003103a20a 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -513,7 +513,7 @@ const std::unordered_map scalar_func_map({ //{tipb::ScalarFuncSig::YearWeekWithMode, "cast"}, //{tipb::ScalarFuncSig::YearWeekWithoutMode, "cast"}, - //{tipb::ScalarFuncSig::GetFormat, "cast"}, + {tipb::ScalarFuncSig::GetFormat, "getFormat"}, {tipb::ScalarFuncSig::SysDateWithFsp, "sysDateWithFsp"}, {tipb::ScalarFuncSig::SysDateWithoutFsp, "sysDateWithoutFsp"}, //{tipb::ScalarFuncSig::CurrentDate, "cast"}, diff --git a/dbms/src/Functions/FunctionsConversion.cpp b/dbms/src/Functions/FunctionsConversion.cpp index 118574ed33d..0446f76bd51 100644 --- a/dbms/src/Functions/FunctionsConversion.cpp +++ b/dbms/src/Functions/FunctionsConversion.cpp @@ -240,6 +240,7 @@ void registerFunctionsConversion(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction>(); factory.registerFunction>(); factory.registerFunction>(); diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index ddf64a70ca1..e8333ceeeea 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -1751,6 +1751,120 @@ class FunctionDateFormat : public IFunction } }; +class FunctionGetFormat : public IFunction +{ +private: + static String get_format(const StringRef & time_type, const StringRef & location) + { + if (time_type == "DATE") + { + if (location == "USA") + return "%m.%d.%Y"; + else if (location == "JIS") + return "%Y-%m-%d"; + else if (location == "ISO") + return "%Y-%m-%d"; + else if (location == "EUR") + return "%d.%m.%Y"; + else if (location == "INTERNAL") + return "%Y%m%d"; + } + else if (time_type == "DATETIME" || time_type == "TIMESTAMP") + { + if (location == "USA") + return "%Y-%m-%d %H.%i.%s"; + else if (location == "JIS") + return "%Y-%m-%d %H:%i:%s"; + else if (location == "ISO") + return "%Y-%m-%d %H:%i:%s"; + else if (location == "EUR") + return "%Y-%m-%d %H.%i.%s"; + else if (location == "INTERNAL") + return "%Y%m%d%H%i%s"; + } + else if (time_type == "TIME") + { + if (location == "USA") + return "%h:%i:%s %p"; + else if (location == "JIS") + return "%H:%i:%s"; + else if (location == "ISO") + return "%H:%i:%s"; + else if (location == "EUR") + return "%H.%i.%s"; + else if (location == "INTERNAL") + return "%H%i%s"; + } + return ""; + } + +public: + static constexpr auto name = "getFormat"; + static FunctionPtr create(const Context &) { return std::make_shared(); }; + + String getName() const override { return name; } + + size_t getNumberOfArguments() const override { return 2; } + + DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override + { + if (!arguments[0].type->isString()) + throw Exception("First argument for function " + getName() + " must be String", ErrorCodes::ILLEGAL_COLUMN); + if (!arguments[1].type->isString()) + throw Exception("Second argument for function " + getName() + " must be String", ErrorCodes::ILLEGAL_COLUMN); + + return std::make_shared(); + } + + bool useDefaultImplementationForConstants() const override { return true; } + + /** + * @brief The first argument is designed as a MySQL reserved word. You would encounter a syntax error when wrap it around with quote in SQL. + * For example, select GET_FORMAT("DATE", "USA") will fail. Removing the quote can solve the problem. + * Thus the first argument should always be a ColumnConst. See details in the link below: + * https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_get-format + * + * @return ColumnNumbers + */ + ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {0}; } + + void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) const override + { + const auto * location_col = checkAndGetColumn(block.getByPosition(arguments[1]).column.get()); + assert(location_col); + size_t size = location_col->size(); + const auto & time_type_col = block.getByPosition(arguments[0]).column; + auto col_to = ColumnString::create(); + + if (time_type_col->isColumnConst()) + { + const auto & time_type_col_const = checkAndGetColumnConst(time_type_col.get()); + const auto & time_type = time_type_col_const->getValue(); + + ColumnString::Chars_t & data_to = col_to->getChars(); + ColumnString::Offsets & offsets_to = col_to->getOffsets(); + auto max_length = 18; + data_to.resize(size * max_length); + offsets_to.resize(size); + WriteBufferFromVector write_buffer(data_to); + for (size_t i = 0; i < size; ++i) + { + const auto & location = location_col->getDataAt(i); + const auto & result = get_format(StringRef(time_type), location); + write_buffer.write(result.c_str(), result.size()); + writeChar(0, write_buffer); + offsets_to[i] = write_buffer.count(); + } + data_to.resize(write_buffer.count()); + block.getByPosition(result).column = std::move(col_to); + } + else + { + throw Exception("First argument for function " + getName() + " must be String constant", ErrorCodes::ILLEGAL_COLUMN); + } + } +}; + struct NameStrToDateDate { static constexpr auto name = "strToDateDate"; diff --git a/dbms/src/Functions/tests/gtest_get_format.cpp b/dbms/src/Functions/tests/gtest_get_format.cpp new file mode 100644 index 00000000000..61a8d80e7b4 --- /dev/null +++ b/dbms/src/Functions/tests/gtest_get_format.cpp @@ -0,0 +1,153 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include +#include +#include +#include +#include +#include +#include + +#include +#include + + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wsign-compare" +#include + +#pragma GCC diagnostic pop + +namespace DB::tests +{ +class GetFormatTest : public DB::tests::FunctionTest +{ +public: + static constexpr auto funcName = "getFormat"; +}; + +TEST_F(GetFormatTest, testBoundary) +try +{ + // const(non-null), vector + // time_type is a const with non null value + // location is a vector containing null + ASSERT_COLUMN_EQ( + createColumn>({"%m.%d.%Y", {}}), + executeFunction( + funcName, + createConstColumn>(2, "DATE"), + createColumn>({"USA", {}}))); + + // const(null), vector + // time_type is a const with null value + // location is a vector containing null + ASSERT_COLUMN_EQ( + createConstColumn>(2, {}), + executeFunction( + funcName, + createConstColumn>(2, {}), + createColumn>({"USA", {}}))); + + // const(non-null), const(non-null) + // time_type is a const with non null value + // location is a const with non null value + ASSERT_COLUMN_EQ( + createConstColumn(2, "%m.%d.%Y"), + executeFunction( + funcName, + createConstColumn>(2, "DATE"), + createConstColumn>(2, "USA"))); + + // const(non-null), const(null) + // time_type is a const with non null value + // location is a const with null value + ASSERT_COLUMN_EQ( + createConstColumn>(2, {}), + executeFunction( + funcName, + createConstColumn>(2, "DATE"), + createConstColumn>(2, {}))); + + // The time_type is a system pre_defined macro, thus assume time_type column is const + // Throw an exception is time_type is not ColumnConst + ASSERT_THROW( + executeFunction( + funcName, + createColumn>({"DATE", "TIME"}), + createColumn>({"USA", {}})), + DB::Exception); +} +CATCH + +TEST_F(GetFormatTest, testMoreCases) +try +{ + // time_type: DATE + // all locations + ASSERT_COLUMN_EQ( + createColumn>({"%m.%d.%Y", "%Y-%m-%d", "%Y-%m-%d", "%d.%m.%Y", "%Y%m%d"}), + executeFunction( + funcName, + createConstColumn>(5, "DATE"), + createColumn>({"USA", "JIS", "ISO", "EUR", "INTERNAL"}))); + + // time_type: DATETIME + // all locations + ASSERT_COLUMN_EQ( + createColumn>({"%Y-%m-%d %H.%i.%s", "%Y-%m-%d %H:%i:%s", "%Y-%m-%d %H:%i:%s", "%Y-%m-%d %H.%i.%s", "%Y%m%d%H%i%s"}), + executeFunction( + funcName, + createConstColumn>(5, "DATETIME"), + createColumn>({"USA", "JIS", "ISO", "EUR", "INTERNAL"}))); + + // time_type: TIMESTAMP + // all locations + ASSERT_COLUMN_EQ( + createColumn>({"%Y-%m-%d %H.%i.%s", "%Y-%m-%d %H:%i:%s", "%Y-%m-%d %H:%i:%s", "%Y-%m-%d %H.%i.%s", "%Y%m%d%H%i%s"}), + executeFunction( + funcName, + createConstColumn>(5, "TIMESTAMP"), + createColumn>({"USA", "JIS", "ISO", "EUR", "INTERNAL"}))); + + // time_type: TIME + // all locations + ASSERT_COLUMN_EQ( + createColumn>({"%h:%i:%s %p", "%H:%i:%s", "%H:%i:%s", "%H.%i.%s", "%H%i%s"}), + executeFunction( + funcName, + createConstColumn>(5, "TIME"), + createColumn>({"USA", "JIS", "ISO", "EUR", "INTERNAL"}))); + + // the location is not in ("USA", "JIS", "ISO", "EUR", "INTERNAL") + ASSERT_COLUMN_EQ( + createColumn>({"", ""}), + executeFunction( + funcName, + createConstColumn>(2, "TIME"), + createColumn>({"CAN", ""}))); + + // the time_type is not in ("DATE", "DATETIME", "TIMESTAMP", "TIME") + ASSERT_COLUMN_EQ( + createColumn>({"", ""}), + executeFunction( + funcName, + createConstColumn>(2, "TIMEINUTC"), + createColumn>({"USA", "ISO"}))); +} +CATCH + +} // namespace DB::tests diff --git a/tests/fullstack-test/expr/get_format.test b/tests/fullstack-test/expr/get_format.test new file mode 100644 index 00000000000..5409302c10a --- /dev/null +++ b/tests/fullstack-test/expr/get_format.test @@ -0,0 +1,60 @@ +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +mysql> drop table if exists test.t; +mysql> create table test.t(location varchar(10)); +mysql> insert into test.t values('USA'), ('JIS'), ('ISO'), ('EUR'), ('INTERNAL'); +mysql> alter table test.t set tiflash replica 1; +func> wait_table test t +mysql> set @@tidb_enforce_mpp=1; set @@tidb_isolation_read_engines='tiflash'; select GET_FORMAT(DATE, location) from test.t; ++----------------------------+ +| GET_FORMAT(DATE, location) | ++----------------------------+ +| %m.%d.%Y | +| %Y-%m-%d | +| %Y-%m-%d | +| %d.%m.%Y | +| %Y%m%d | ++----------------------------+ +mysql> set @@tidb_enforce_mpp=1; set @@tidb_isolation_read_engines='tiflash'; select GET_FORMAT(DATETIME, location) from test.t; ++--------------------------------+ +| GET_FORMAT(DATETIME, location) | ++--------------------------------+ +| %Y-%m-%d %H.%i.%s | +| %Y-%m-%d %H:%i:%s | +| %Y-%m-%d %H:%i:%s | +| %Y-%m-%d %H.%i.%s | +| %Y%m%d%H%i%s | ++--------------------------------+ +mysql> set @@tidb_enforce_mpp=1; set @@tidb_isolation_read_engines='tiflash'; select GET_FORMAT(TIMESTAMP, location) from test.t; ++---------------------------------+ +| GET_FORMAT(TIMESTAMP, location) | ++---------------------------------+ +| %Y-%m-%d %H.%i.%s | +| %Y-%m-%d %H:%i:%s | +| %Y-%m-%d %H:%i:%s | +| %Y-%m-%d %H.%i.%s | +| %Y%m%d%H%i%s | ++---------------------------------+ +mysql> set @@tidb_enforce_mpp=1; set @@tidb_isolation_read_engines='tiflash'; select GET_FORMAT(TIME, location) from test.t; ++----------------------------+ +| GET_FORMAT(TIME, location) | ++----------------------------+ +| %h:%i:%s %p | +| %H:%i:%s | +| %H:%i:%s | +| %H.%i.%s | +| %H%i%s | ++----------------------------+ +mysql> drop table if exists test.t; From 2af78c51b22ff2aec97c24f9bc78a706d21661b3 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 11 Jul 2022 15:29:05 +0800 Subject: [PATCH 099/104] fix: format throw data truncated error (#5272) close pingcap/tiflash#4891 --- dbms/src/Functions/FunctionsString.cpp | 5 +- .../Functions/tests/gtest_strings_format.cpp | 19 +++---- tests/fullstack-test/expr/format.test | 49 +++++++++++++++++++ 3 files changed, 61 insertions(+), 12 deletions(-) diff --git a/dbms/src/Functions/FunctionsString.cpp b/dbms/src/Functions/FunctionsString.cpp index ad75c69a090..76022b983ad 100644 --- a/dbms/src/Functions/FunctionsString.cpp +++ b/dbms/src/Functions/FunctionsString.cpp @@ -4573,7 +4573,9 @@ class FormatImpl : public IFunction using NumberFieldType = typename NumberType::FieldType; using NumberColVec = std::conditional_t, ColumnDecimal, ColumnVector>; const auto * number_raw = block.getByPosition(arguments[0]).column.get(); + TiDBDecimalRoundInfo info{number_type, number_type}; + info.output_prec = info.output_prec < 65 ? info.output_prec + 1 : 65; return getPrecisionType(precision_base_type, [&](const auto & precision_type, bool) { using PrecisionType = std::decay_t; @@ -4723,10 +4725,11 @@ class FormatImpl : public IFunction static void format( T number, size_t max_num_decimals, - const TiDBDecimalRoundInfo & info, + TiDBDecimalRoundInfo & info, ColumnString::Chars_t & res_data, ColumnString::Offsets & res_offsets) { + info.output_scale = std::min(max_num_decimals, static_cast(info.input_scale)); auto round_number = round(number, max_num_decimals, info); std::string round_number_str = number2Str(round_number, info); std::string buffer = Format::apply(round_number_str, max_num_decimals); diff --git a/dbms/src/Functions/tests/gtest_strings_format.cpp b/dbms/src/Functions/tests/gtest_strings_format.cpp index 2d571a9bb1b..8f3b899316e 100644 --- a/dbms/src/Functions/tests/gtest_strings_format.cpp +++ b/dbms/src/Functions/tests/gtest_strings_format.cpp @@ -34,7 +34,7 @@ class StringFormat : public DB::tests::FunctionTest using FieldType = DecimalField; using NullableDecimal = Nullable; ASSERT_COLUMN_EQ( - createColumn>({"0.0000", "-0.0120", "0.0120", "12,332.1000", "12,332", "12,332", "12,332.300000000000000000000000000000", "-12,332.30000", "-1,000.0", "-333.33", {}}), + createColumn>({"0.0000", "-0.0120", "0.0120", "12,332.1000", "12,332", "12,332", "12,332.300000000000000000000000000000", "-12,332.30000", "-1,000.0", "-333.33", {}, "99,999.9999000000", "100,000.000", "100,000"}), executeFunction( func_name, createColumn( @@ -49,8 +49,11 @@ class StringFormat : public DB::tests::FunctionTest FieldType(static_cast(-123323000), 4), FieldType(static_cast(-9999999), 4), FieldType(static_cast(-3333330), 4), - FieldType(static_cast(0), 0)}), - createColumn>({4, 4, 4, 4, 0, -1, 31, 5, 1, 2, {}}))); + FieldType(static_cast(0), 0), + FieldType(static_cast(999999999), 4), + FieldType(static_cast(999999999), 4), + FieldType(static_cast(999999999), 4)}), + createColumn>({4, 4, 4, 4, 0, -1, 31, 5, 1, 2, {}, 10, 3, -5}))); ASSERT_COLUMN_EQ( createColumn>({"12,332.100", "-12,332.300", "-1,000.000", "-333.333"}), executeFunction( @@ -62,8 +65,6 @@ class StringFormat : public DB::tests::FunctionTest FieldType(static_cast(-9999999), 4), FieldType(static_cast(-3333330), 4)}), createConstColumn>(4, 3))); - /// known issue https://github.com/pingcap/tiflash/issues/4891 - /* ASSERT_COLUMN_EQ( createColumn>({"-999.9999", "-1,000", "-1,000", "-999.999900000000000000000000000000", "-999.99990", "-1,000.0", "-1,000.00"}), executeFunction( @@ -74,7 +75,7 @@ class StringFormat : public DB::tests::FunctionTest FieldType(static_cast(-9999999), 4)), createColumn>({4, 0, -1, 31, 5, 1, 2}))); ASSERT_COLUMN_EQ( - createConstColumn>(1, "-1,000.000"), + createConstColumn(1, "-1,000.000"), executeFunction( func_name, createConstColumn( @@ -82,7 +83,6 @@ class StringFormat : public DB::tests::FunctionTest 1, FieldType(static_cast(-9999999), 4)), createConstColumn>(1, 3))); - */ ASSERT_COLUMN_EQ( createColumn>({"12,332.1000", "12,332", "12,332.300000000000000000000000000000", "-12,332.30000", "-1,000.0", "-333.33", {}}), executeFunction( @@ -108,8 +108,6 @@ class StringFormat : public DB::tests::FunctionTest FieldType(static_cast(-9999999), 4), FieldType(static_cast(-3333330), 4)}), createConstColumn>(4, 3))); - /// known issue https://github.com/pingcap/tiflash/issues/4891 - /* ASSERT_COLUMN_EQ( createColumn>({"-999.9999", "-1,000", "-999.999900000000000000000000000000", "-999.99990", "-1,000.0", "-1,000.00"}), executeFunction( @@ -120,7 +118,7 @@ class StringFormat : public DB::tests::FunctionTest FieldType(static_cast(-9999999), 4)), createColumn>({4, 0, 31, 5, 1, 2}))); ASSERT_COLUMN_EQ( - createConstColumn>(1, "-1,000.000"), + createConstColumn(1, "-1,000.000"), executeFunction( func_name, createConstColumn( @@ -128,7 +126,6 @@ class StringFormat : public DB::tests::FunctionTest 1, FieldType(static_cast(-9999999), 4)), createConstColumn>(1, 3))); - */ } template diff --git a/tests/fullstack-test/expr/format.test b/tests/fullstack-test/expr/format.test index 8cea75d6914..719e30c974d 100644 --- a/tests/fullstack-test/expr/format.test +++ b/tests/fullstack-test/expr/format.test @@ -44,3 +44,52 @@ int_val 1,234.000 mysql> drop table if exists test.t + +mysql> create table test.t(id int, value decimal(65,4)) +mysql> alter table test.t set tiflash replica 1 +mysql> insert into test.t values(1,9999999999999999999999999999999999999999999999999999999999999.9999) + +func> wait_table test t + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,-3) as result from test.t +result +10,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000 + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,0) as result from test.t +result +10,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000 + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,3) as result from test.t +result +10,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000,000.000 + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,10) as result from test.t +result +9,999,999,999,999,999,999,999,999,999,999,999,999,999,999,999,999,999,999,999,999.9999000000 + + +mysql> drop table if exists test.t + +mysql> create table test.t(id int, value decimal(7,4)) +mysql> alter table test.t set tiflash replica 1 +mysql> insert into test.t values(1,999.9999) + +func> wait_table test t + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,-2) as result from test.t +result +1,000 + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,0) as result from test.t +result +1,000 + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,2) as result from test.t +result +1,000.00 + +mysql> set tidb_enforce_mpp=1; set tidb_isolation_read_engines='tiflash'; select format(value,10) as result from test.t +result +999.9999000000 + +mysql> drop table if exists test.t From cf7aa5ebcd317f69709939dc19f4b178b4107b19 Mon Sep 17 00:00:00 2001 From: xzhangxian1008 Date: Mon, 11 Jul 2022 16:05:05 +0800 Subject: [PATCH 100/104] Print content of columns for gtest (#5243) close pingcap/tiflash#5203 --- dbms/src/TestUtils/FunctionTestUtils.cpp | 56 ++++++++++++++++++ dbms/src/TestUtils/FunctionTestUtils.h | 6 +- .../TestUtils/tests/gtest_print_columns.cpp | 57 +++++++++++++++++++ 3 files changed, 118 insertions(+), 1 deletion(-) create mode 100644 dbms/src/TestUtils/tests/gtest_print_columns.cpp diff --git a/dbms/src/TestUtils/FunctionTestUtils.cpp b/dbms/src/TestUtils/FunctionTestUtils.cpp index 9fbf3c9691f..1c8b0242bfa 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.cpp +++ b/dbms/src/TestUtils/FunctionTestUtils.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include @@ -120,6 +121,7 @@ ::testing::AssertionResult blockEqual( { const auto & expected_col = expected.getByPosition(i); const auto & actual_col = actual.getByPosition(i); + auto cmp_res = columnEqual(expected_col, actual_col); if (!cmp_res) return cmp_res; @@ -375,9 +377,63 @@ ColumnWithTypeAndName toNullableDatetimeVec(String name, const std::vector>(data_type, vec), data_type, name, 0}; } +String getColumnsContent(const ColumnsWithTypeAndName & cols) +{ + if (cols.size() <= 0) + return ""; + return getColumnsContent(cols, 0, cols[0].column->size() - 1); +} + +String getColumnsContent(const ColumnsWithTypeAndName & cols, size_t begin, size_t end) +{ + const size_t col_num = cols.size(); + if (col_num <= 0) + return ""; + + const size_t col_size = cols[0].column->size(); + assert(begin <= end); + assert(col_size > end); + assert(col_size > begin); + + bool is_same = true; + + for (size_t i = 1; i < col_num; ++i) + { + if (cols[i].column->size() != col_size) + is_same = false; + } + + assert(is_same); /// Ensure the sizes of columns in cols are the same + + std::vector> col_content; + FmtBuffer fmt_buf; + for (size_t i = 0; i < col_num; ++i) + { + /// Push the column name + fmt_buf.append(fmt::format("{}: (", cols[i].name)); + for (size_t j = begin; j <= end; ++j) + col_content.push_back(std::make_pair(j, (*cols[i].column)[j].toString())); + + /// Add content + fmt_buf.joinStr( + col_content.begin(), + col_content.end(), + [](const auto & content, FmtBuffer & fmt_buf) { + fmt_buf.append(fmt::format("{}: {}", content.first, content.second)); + }, + ", "); + + fmt_buf.append(")\n"); + col_content.clear(); + } + + return fmt_buf.toString(); +} + ColumnsWithTypeAndName createColumns(const ColumnsWithTypeAndName & cols) { return cols; } + } // namespace tests } // namespace DB diff --git a/dbms/src/TestUtils/FunctionTestUtils.h b/dbms/src/TestUtils/FunctionTestUtils.h index ad01e2e8441..8680d1886b1 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.h +++ b/dbms/src/TestUtils/FunctionTestUtils.h @@ -514,13 +514,17 @@ ColumnWithTypeAndName createConstColumn( return createConstColumn(data_type_args, size, InferredFieldType(std::nullopt), name); } +String getColumnsContent(const ColumnsWithTypeAndName & cols); + +/// We can designate the range of columns printed with begin and end. range: [begin, end] +String getColumnsContent(const ColumnsWithTypeAndName & cols, size_t begin, size_t end); + // This wrapper function only serves to construct columns input for function-like macros, // since preprocessor recognizes `{col1, col2, col3}` as three arguments instead of one. // E.g. preprocessor does not allow us to write `ASSERT_COLUMNS_EQ_R({col1, col2, col3}, actual_cols)`, // but with this func we can write `ASSERT_COLUMNS_EQ_R(createColumns{col1, col2, col3}, actual_cols)` instead. ColumnsWithTypeAndName createColumns(const ColumnsWithTypeAndName & cols); - ::testing::AssertionResult dataTypeEqual( const DataTypePtr & expected, const DataTypePtr & actual); diff --git a/dbms/src/TestUtils/tests/gtest_print_columns.cpp b/dbms/src/TestUtils/tests/gtest_print_columns.cpp new file mode 100644 index 00000000000..50631fc4f4a --- /dev/null +++ b/dbms/src/TestUtils/tests/gtest_print_columns.cpp @@ -0,0 +1,57 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +#include +#include + +namespace DB +{ +namespace tests +{ + +class PrintColumnsTest : public DB::tests::ExecutorTest +{ +public: + using ColStringType = std::optional::FieldType>; + using ColInt32Type = std::optional::FieldType>; + using ColumnWithString = std::vector; + using ColumnWithInt32 = std::vector; + + void initializeContext() override + { + test_cols.push_back(toNullableVec("col1", ColumnWithInt32{36, 34, 32, 27, {}, {}})); + test_cols.push_back(toNullableVec("col2", ColumnWithString{"female", "male", "male", "female", "male", "female"})); + col_len = test_cols[0].column->size(); + } + + ColumnsWithTypeAndName test_cols; + size_t col_len; + const String result1{"col1: (0: Int64_36, 1: Int64_34, 2: Int64_32, 3: Int64_27, 4: NULL, 5: NULL)\ncol2: (0: 'female', 1: 'male', 2: 'male', 3: 'female', 4: 'male', 5: 'female')\n"}; + const String result2{"col1: (0: Int64_36, 1: Int64_34, 2: Int64_32, 3: Int64_27, 4: NULL, 5: NULL)\ncol2: (0: 'female', 1: 'male', 2: 'male', 3: 'female', 4: 'male', 5: 'female')\n"}; + const String result3{"col1: (0: Int64_36)\ncol2: (0: 'female')\n"}; + const String result4{"col1: (1: Int64_34, 2: Int64_32, 3: Int64_27, 4: NULL)\ncol2: (1: 'male', 2: 'male', 3: 'female', 4: 'male')\n"}; +}; + +TEST_F(PrintColumnsTest, SimpleTest) +try +{ + EXPECT_EQ(getColumnsContent(test_cols), result1); + EXPECT_EQ(getColumnsContent(test_cols, 0, col_len - 1), result2); + EXPECT_EQ(getColumnsContent(test_cols, 0, 0), result3); + EXPECT_EQ(getColumnsContent(test_cols, 1, col_len - 2), result4); +} +CATCH + +} // namespace tests +} // namespace DB From 5ddee14e15f65dfcfddef64874e50e7bbdec993c Mon Sep 17 00:00:00 2001 From: Schrodinger ZHU Yifan Date: Mon, 11 Jul 2022 16:49:05 +0800 Subject: [PATCH 101/104] *: also enable O3 for aarch64 (#5338) close pingcap/tiflash#5342 --- CMakeLists.txt | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/CMakeLists.txt b/CMakeLists.txt index f2ec9f3316b..2e33a127807 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -239,10 +239,8 @@ else () set (CMAKE_CXX_STANDARD_REQUIRED ON) endif () -if (NOT ARCH_ARM) - set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -O3") - set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3") -endif () +set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -O3") +set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3") option (DEBUG_WITHOUT_DEBUG_INFO "Set to ON to build dev target without debug info (remove flag `-g` in order to accelerate compiling speed and reduce target binary size)" OFF) if (DEBUG_WITHOUT_DEBUG_INFO) From 7a717b575050d97caf7f55166120d915725cf78f Mon Sep 17 00:00:00 2001 From: yibin Date: Mon, 11 Jul 2022 17:25:04 +0800 Subject: [PATCH 102/104] Add debug image build target for CentOS7 (#5344) close pingcap/tiflash#5343 --- release-centos7-llvm/Makefile | 4 ++++ release-centos7-llvm/scripts/build-debug.sh | 23 +++++++++++++++++++ .../scripts/build-tiflash-release.sh | 9 +++++++- 3 files changed, 35 insertions(+), 1 deletion(-) create mode 100755 release-centos7-llvm/scripts/build-debug.sh diff --git a/release-centos7-llvm/Makefile b/release-centos7-llvm/Makefile index 1b15df7ddc3..9c1bba42a53 100644 --- a/release-centos7-llvm/Makefile +++ b/release-centos7-llvm/Makefile @@ -23,6 +23,10 @@ image_tiflash_llvm_base_aarch64: build_tiflash_release_amd64: docker run --rm -v $(realpath ..):/build/tics hub.pingcap.net/tiflash/tiflash-llvm-base:amd64 /build/tics/release-centos7-llvm/scripts/build-release.sh +# Add build_tiflash_debug_amd64 target to enable FailPoints on x86. Since outputs are the same as release version, no new package targets added. +build_tiflash_debug_amd64: + docker run --rm -v $(realpath ..):/build/tics hub.pingcap.net/tiflash/tiflash-llvm-base:amd64 /build/tics/release-centos7-llvm/scripts/build-debug.sh + build_tiflash_ci_amd64: docker run --rm -v $(realpath ..):/build/tics hub.pingcap.net/tiflash/tiflash-llvm-base:amd64 /build/tics/release-centos7-llvm/scripts/build-tiflash-ci.sh diff --git a/release-centos7-llvm/scripts/build-debug.sh b/release-centos7-llvm/scripts/build-debug.sh new file mode 100755 index 00000000000..59dc9b86a54 --- /dev/null +++ b/release-centos7-llvm/scripts/build-debug.sh @@ -0,0 +1,23 @@ +#!/bin/bash +# Copyright 2022 PingCAP, Ltd. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + + +CMAKE_PREFIX_PATH=$1 + +set -ueox pipefail + +SCRIPTPATH="$( cd "$(dirname "$0")" ; pwd -P )" + +${SCRIPTPATH}/build-tiflash-release.sh "DEBUG" "${CMAKE_PREFIX_PATH}" diff --git a/release-centos7-llvm/scripts/build-tiflash-release.sh b/release-centos7-llvm/scripts/build-tiflash-release.sh index 42993b51afe..01ca00e8706 100755 --- a/release-centos7-llvm/scripts/build-tiflash-release.sh +++ b/release-centos7-llvm/scripts/build-tiflash-release.sh @@ -47,7 +47,13 @@ ENABLE_PCH=${ENABLE_PCH:-ON} INSTALL_DIR="${SRCPATH}/release-centos7-llvm/tiflash" rm -rf ${INSTALL_DIR} && mkdir -p ${INSTALL_DIR} -BUILD_DIR="${SRCPATH}/release-centos7-llvm/build-release" +if [ $CMAKE_BUILD_TYPE == "RELWITHDEBINFO" ]; then + BUILD_DIR="$SRCPATH/release-centos7-llvm/build-release" + ENABLE_FAILPOINTS="OFF" +else + BUILD_DIR="$SRCPATH/release-centos7-llvm/build-debug" + ENABLE_FAILPOINTS="ON" +fi rm -rf ${BUILD_DIR} && mkdir -p ${BUILD_DIR} && cd ${BUILD_DIR} cmake -S "${SRCPATH}" \ @@ -55,6 +61,7 @@ cmake -S "${SRCPATH}" \ -DCMAKE_BUILD_TYPE=${CMAKE_BUILD_TYPE} \ -DENABLE_TESTING=OFF \ -DENABLE_TESTS=OFF \ + -DENABLE_FAILPOINTS=${ENABLE_FAILPOINTS} \ -Wno-dev \ -DUSE_CCACHE=OFF \ -DRUN_HAVE_STD_REGEX=0 \ From 4619605bb980499ed321e58f64f927e2d52288fe Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 11 Jul 2022 18:09:05 +0800 Subject: [PATCH 103/104] *: mini refactor (#5326) close pingcap/tiflash#4739 --- .../DataStreams/TiRemoteBlockInputStream.h | 13 +- dbms/src/Debug/astToExecutor.cpp | 9 + dbms/src/Flash/Coprocessor/DAGContext.cpp | 7 + dbms/src/Flash/Coprocessor/DAGContext.h | 4 + .../Coprocessor/DAGExpressionAnalyzer.cpp | 32 ++- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 58 +++-- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 96 +------- .../Coprocessor/DAGQueryBlockInterpreter.h | 2 - .../Flash/Coprocessor/GenSchemaAndColumn.cpp | 13 +- .../Flash/Coprocessor/GenSchemaAndColumn.h | 4 +- .../Flash/Coprocessor/InterpreterUtils.cpp | 77 +++++++ dbms/src/Flash/Coprocessor/InterpreterUtils.h | 16 ++ dbms/src/Flash/tests/gtest_interpreter.cpp | 214 ++++++++---------- dbms/src/Interpreters/Settings.h | 10 +- dbms/src/TestUtils/mockExecutor.h | 1 + 15 files changed, 294 insertions(+), 262 deletions(-) diff --git a/dbms/src/DataStreams/TiRemoteBlockInputStream.h b/dbms/src/DataStreams/TiRemoteBlockInputStream.h index cfa3e95c440..c1afb1e9f4e 100644 --- a/dbms/src/DataStreams/TiRemoteBlockInputStream.h +++ b/dbms/src/DataStreams/TiRemoteBlockInputStream.h @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -182,21 +183,13 @@ class TiRemoteBlockInputStream : public IProfilingBlockInputStream , total_rows(0) , stream_id(stream_id_) { - // generate sample block - ColumnsWithTypeAndName columns; - for (auto & dag_col : remote_reader->getOutputSchema()) - { - auto tp = getDataTypeByColumnInfoForComputingLayer(dag_col.second); - ColumnWithTypeAndName col(tp, dag_col.first); - columns.emplace_back(col); - } - for (size_t i = 0; i < source_num; i++) + for (size_t i = 0; i < source_num; ++i) { execution_summaries_inited[i].store(false); } execution_summaries.resize(source_num); connection_profile_infos.resize(source_num); - sample_block = Block(columns); + sample_block = Block(getColumnWithTypeAndName(toNamesAndTypes(remote_reader->getOutputSchema()))); } Block getHeader() const override { return sample_block; } diff --git a/dbms/src/Debug/astToExecutor.cpp b/dbms/src/Debug/astToExecutor.cpp index e02dd3aa740..61f4474f919 100644 --- a/dbms/src/Debug/astToExecutor.cpp +++ b/dbms/src/Debug/astToExecutor.cpp @@ -170,6 +170,7 @@ std::unordered_map func_name_to_sig({ {"cast_decimal_datetime", tipb::ScalarFuncSig::CastDecimalAsTime}, {"cast_time_datetime", tipb::ScalarFuncSig::CastTimeAsTime}, {"cast_string_datetime", tipb::ScalarFuncSig::CastStringAsTime}, + {"concat", tipb::ScalarFuncSig::Concat}, {"round_int", tipb::ScalarFuncSig::RoundInt}, {"round_uint", tipb::ScalarFuncSig::RoundInt}, {"round_dec", tipb::ScalarFuncSig::RoundDec}, @@ -461,6 +462,14 @@ void functionToPB(const DAGSchema & input, ASTFunction * func, tipb::Expr * expr ft->set_collate(collator_id); break; } + case tipb::ScalarFuncSig::Concat: + { + expr->set_sig(it_sig->second); + auto * ft = expr->mutable_field_type(); + ft->set_tp(TiDB::TypeString); + ft->set_collate(collator_id); + break; + } case tipb::ScalarFuncSig::RoundInt: case tipb::ScalarFuncSig::RoundWithFracInt: { diff --git a/dbms/src/Flash/Coprocessor/DAGContext.cpp b/dbms/src/Flash/Coprocessor/DAGContext.cpp index ec0544c6ee4..1cf7a0d6c87 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.cpp +++ b/dbms/src/Flash/Coprocessor/DAGContext.cpp @@ -30,6 +30,8 @@ extern const int DIVIDED_BY_ZERO; extern const int INVALID_TIME; } // namespace ErrorCodes +const String enableFineGrainedShuffleExtraInfo = "enable fine grained shuffle"; + bool strictSqlMode(UInt64 sql_mode) { return sql_mode & TiDBSQLMode::STRICT_ALL_TABLES || sql_mode & TiDBSQLMode::STRICT_TRANS_TABLES; @@ -75,6 +77,11 @@ std::unordered_map & DAGContext::getProfileStreamsMap return profile_streams_map; } +void DAGContext::updateFinalConcurrency(size_t cur_streams_size, size_t streams_upper_limit) +{ + final_concurrency = std::min(std::max(final_concurrency, cur_streams_size), streams_upper_limit); +} + void DAGContext::initExecutorIdToJoinIdMap() { // only mpp task has join executor diff --git a/dbms/src/Flash/Coprocessor/DAGContext.h b/dbms/src/Flash/Coprocessor/DAGContext.h index 10190074a0f..7bfc67afcad 100644 --- a/dbms/src/Flash/Coprocessor/DAGContext.h +++ b/dbms/src/Flash/Coprocessor/DAGContext.h @@ -121,6 +121,8 @@ inline bool enableFineGrainedShuffle(uint64_t stream_count) return stream_count > 0; } +extern const String enableFineGrainedShuffleExtraInfo; + /// A context used to track the information that needs to be passed around during DAG planning. class DAGContext { @@ -308,6 +310,8 @@ class DAGContext return sql_mode & f; } + void updateFinalConcurrency(size_t cur_streams_size, size_t streams_upper_limit); + bool isTest() const { return is_test; } void setColumnsForTest(std::unordered_map & columns_for_test_map_) { columns_for_test_map = columns_for_test_map_; } ColumnsWithTypeAndName columnsForTest(String executor_id); diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index aa269469cdb..5fbd86e9762 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -1130,30 +1130,40 @@ NamesWithAliases DAGExpressionAnalyzer::appendFinalProjectForRootQueryBlock( const std::vector & output_offsets, const String & column_prefix, bool keep_session_timezone_info) +{ + auto & step = initAndGetLastStep(chain); + + NamesWithAliases final_project = buildFinalProjection(step.actions, schema, output_offsets, column_prefix, keep_session_timezone_info); + + for (const auto & name : final_project) + { + step.required_output.push_back(name.first); + } + return final_project; +} + +NamesWithAliases DAGExpressionAnalyzer::buildFinalProjection( + const ExpressionActionsPtr & actions, + const std::vector & schema, + const std::vector & output_offsets, + const String & column_prefix, + bool keep_session_timezone_info) { if (unlikely(output_offsets.empty())) - throw Exception("Root Query block without output_offsets", ErrorCodes::LOGICAL_ERROR); + throw Exception("DAGRequest without output_offsets", ErrorCodes::LOGICAL_ERROR); bool need_append_timezone_cast = !keep_session_timezone_info && !context.getTimezoneInfo().is_utc_timezone; auto [need_append_type_cast, need_append_type_cast_vec] = isCastRequiredForRootFinalProjection(schema, output_offsets); assert(need_append_type_cast_vec.size() == output_offsets.size()); - auto & step = initAndGetLastStep(chain); - if (need_append_timezone_cast || need_append_type_cast) { // after appendCastForRootFinalProjection, source_columns has been modified. - appendCastForRootFinalProjection(step.actions, schema, output_offsets, need_append_timezone_cast, need_append_type_cast_vec); + appendCastForRootFinalProjection(actions, schema, output_offsets, need_append_timezone_cast, need_append_type_cast_vec); } // generate project aliases from source_columns. - NamesWithAliases final_project = genRootFinalProjectAliases(column_prefix, output_offsets); - - for (const auto & name : final_project) - { - step.required_output.push_back(name.first); - } - return final_project; + return genRootFinalProjectAliases(column_prefix, output_offsets); } String DAGExpressionAnalyzer::alignReturnType( diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 046088ab2b2..63d35abe26d 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -102,6 +102,8 @@ class DAGExpressionAnalyzer : private boost::noncopyable ExpressionActionsChain & chain, const String & column_prefix) const; + NamesWithAliases genNonRootFinalProjectAliases(const String & column_prefix) const; + // Generate a project action for root DAGQueryBlock, // to keep the schema of Block and tidb-schema the same. NamesWithAliases appendFinalProjectForRootQueryBlock( @@ -111,6 +113,13 @@ class DAGExpressionAnalyzer : private boost::noncopyable const String & column_prefix, bool keep_session_timezone_info); + NamesWithAliases buildFinalProjection( + const ExpressionActionsPtr & actions, + const std::vector & schema, + const std::vector & output_offsets, + const String & column_prefix, + bool keep_session_timezone_info); + String getActions( const tipb::Expr & expr, const ExpressionActionsPtr & actions, @@ -153,17 +162,38 @@ class DAGExpressionAnalyzer : private boost::noncopyable const tipb::Window & window, size_t window_columns_start_index); -#ifndef DBMS_PUBLIC_GTEST -private: -#endif NamesAndTypes buildOrderColumns( const ExpressionActionsPtr & actions, const ::google::protobuf::RepeatedPtrField & order_by); + String buildFilterColumn( + const ExpressionActionsPtr & actions, + const std::vector & conditions); + + void buildAggFuncs( + const tipb::Aggregation & aggregation, + const ExpressionActionsPtr & actions, + AggregateDescriptions & aggregate_descriptions, + NamesAndTypes & aggregated_columns); + + void buildAggGroupBy( + const google::protobuf::RepeatedPtrField & group_by, + const ExpressionActionsPtr & actions, + AggregateDescriptions & aggregate_descriptions, + NamesAndTypes & aggregated_columns, + Names & aggregation_keys, + std::unordered_set & agg_key_set, + bool group_by_collation_sensitive, + TiDB::TiDBCollators & collators); + void appendCastAfterAgg( const ExpressionActionsPtr & actions, const tipb::Aggregation & agg); +#ifndef DBMS_PUBLIC_GTEST +private: +#endif + String buildTupleFunctionForGroupConcat( const tipb::Expr & expr, SortDescription & sort_desc, @@ -187,22 +217,6 @@ class DAGExpressionAnalyzer : private boost::noncopyable NamesAndTypes & aggregated_columns, bool empty_input_as_null); - void buildAggFuncs( - const tipb::Aggregation & aggregation, - const ExpressionActionsPtr & actions, - AggregateDescriptions & aggregate_descriptions, - NamesAndTypes & aggregated_columns); - - void buildAggGroupBy( - const google::protobuf::RepeatedPtrField & group_by, - const ExpressionActionsPtr & actions, - AggregateDescriptions & aggregate_descriptions, - NamesAndTypes & aggregated_columns, - Names & aggregation_keys, - std::unordered_set & agg_key_set, - bool group_by_collation_sensitive, - TiDB::TiDBCollators & collators); - void fillArgumentDetail( const ExpressionActionsPtr & actions, const tipb::Expr & arg, @@ -275,12 +289,6 @@ class DAGExpressionAnalyzer : private boost::noncopyable const ExpressionActionsPtr & actions, const String & column_name); - String buildFilterColumn( - const ExpressionActionsPtr & actions, - const std::vector & conditions); - - NamesWithAliases genNonRootFinalProjectAliases(const String & column_prefix) const; - NamesWithAliases genRootFinalProjectAliases( const String & column_prefix, const std::vector & output_offsets) const; diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 4714580fda0..764bf07f533 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -18,7 +18,6 @@ #include #include #include -#include #include #include #include @@ -57,10 +56,6 @@ namespace FailPoints { extern const char minimum_block_size_for_cross_join[]; } // namespace FailPoints -namespace -{ -const String enableFineGrainedShuffleExtraInfo = "enable fine grained shuffle"; -} DAGQueryBlockInterpreter::DAGQueryBlockInterpreter( Context & context_, @@ -272,7 +267,7 @@ void DAGQueryBlockInterpreter::handleJoin(const tipb::Join & join, DAGPipeline & size_t join_build_concurrency = settings.join_concurrent_build ? std::min(max_streams, build_pipeline.streams.size()) : 1; /// build side streams - executeExpression(build_pipeline, build_side_prepare_actions, "append join key and join filters for build side"); + executeExpression(build_pipeline, build_side_prepare_actions, log, "append join key and join filters for build side"); // add a HashJoinBuildBlockInputStream to build a shared hash table auto get_concurrency_build_index = JoinInterpreterHelper::concurrencyBuildIndexGenerator(join_build_concurrency); build_pipeline.transform([&](auto & stream) { @@ -288,7 +283,7 @@ void DAGQueryBlockInterpreter::handleJoin(const tipb::Join & join, DAGPipeline & join_ptr->init(right_query.source->getHeader(), join_build_concurrency); /// probe side streams - executeExpression(probe_pipeline, probe_side_prepare_actions, "append join key and join filters for probe side"); + executeExpression(probe_pipeline, probe_side_prepare_actions, log, "append join key and join filters for probe side"); NamesAndTypes source_columns; for (const auto & p : probe_pipeline.firstStream()->getHeader()) source_columns.emplace_back(p.name, p.type); @@ -354,7 +349,7 @@ void DAGQueryBlockInterpreter::executeWindow( WindowDescription & window_description, bool enable_fine_grained_shuffle) { - executeExpression(pipeline, window_description.before_window, "before window"); + executeExpression(pipeline, window_description.before_window, log, "before window"); if (enable_fine_grained_shuffle) { @@ -381,10 +376,7 @@ void DAGQueryBlockInterpreter::executeAggregation( AggregateDescriptions & aggregate_descriptions, bool is_final_agg) { - pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, expression_actions_ptr, log->identifier()); - stream->setExtraInfo("before aggregation"); - }); + executeExpression(pipeline, expression_actions_ptr, log, "before aggregation"); Block before_agg_header = pipeline.firstStream()->getHeader(); @@ -442,77 +434,15 @@ void DAGQueryBlockInterpreter::executeAggregation( } } -void DAGQueryBlockInterpreter::executeExpression(DAGPipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, const String & extra_info) -{ - if (!expressionActionsPtr->getActions().empty()) - { - pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, expressionActionsPtr, log->identifier()); - stream->setExtraInfo(extra_info); - }); - } -} - void DAGQueryBlockInterpreter::executeWindowOrder(DAGPipeline & pipeline, SortDescription sort_desc, bool enable_fine_grained_shuffle) { - orderStreams(pipeline, sort_desc, 0, enable_fine_grained_shuffle); + orderStreams(pipeline, max_streams, sort_desc, 0, enable_fine_grained_shuffle, context, log); } void DAGQueryBlockInterpreter::executeOrder(DAGPipeline & pipeline, const NamesAndTypes & order_columns) { Int64 limit = query_block.limit_or_topn->topn().limit(); - orderStreams(pipeline, getSortDescription(order_columns, query_block.limit_or_topn->topn().order_by()), limit, false); -} - -void DAGQueryBlockInterpreter::orderStreams(DAGPipeline & pipeline, SortDescription order_descr, Int64 limit, bool enable_fine_grained_shuffle) -{ - const Settings & settings = context.getSettingsRef(); - String extra_info; - if (enable_fine_grained_shuffle) - extra_info = enableFineGrainedShuffleExtraInfo; - - pipeline.transform([&](auto & stream) { - auto sorting_stream = std::make_shared(stream, order_descr, log->identifier(), limit); - - /// Limits on sorting - IProfilingBlockInputStream::LocalLimits limits; - limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL; - limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); - sorting_stream->setLimits(limits); - - stream = sorting_stream; - stream->setExtraInfo(extra_info); - }); - - if (enable_fine_grained_shuffle) - { - pipeline.transform([&](auto & stream) { - stream = std::make_shared( - stream, - order_descr, - settings.max_block_size, - limit, - settings.max_bytes_before_external_sort, - context.getTemporaryPath(), - log->identifier()); - stream->setExtraInfo(enableFineGrainedShuffleExtraInfo); - }); - } - else - { - /// If there are several streams, we merge them into one - executeUnion(pipeline, max_streams, log, false, "for partial order"); - - /// Merge the sorted blocks. - pipeline.firstStream() = std::make_shared( - pipeline.firstStream(), - order_descr, - settings.max_block_size, - limit, - settings.max_bytes_before_external_sort, - context.getTemporaryPath(), - log->identifier()); - } + orderStreams(pipeline, max_streams, getSortDescription(order_columns, query_block.limit_or_topn->topn().order_by()), limit, false, context, log); } void DAGQueryBlockInterpreter::recordProfileStreams(DAGPipeline & pipeline, const String & key) @@ -603,10 +533,7 @@ void DAGQueryBlockInterpreter::handleProjection(DAGPipeline & pipeline, const ti output_columns.emplace_back(alias, col.type); project_cols.emplace_back(col.name, alias); } - pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, chain.getLastActions(), log->identifier()); - stream->setExtraInfo("before projection"); - }); + executeExpression(pipeline, chain.getLastActions(), log, "before projection"); executeProject(pipeline, project_cols, "projection"); analyzer = std::make_unique(std::move(output_columns), context); } @@ -621,7 +548,7 @@ void DAGQueryBlockInterpreter::handleWindow(DAGPipeline & pipeline, const tipb:: DAGExpressionAnalyzer dag_analyzer(input_columns, context); WindowDescription window_description = dag_analyzer.buildWindowDescription(window); executeWindow(pipeline, window_description, enable_fine_grained_shuffle); - executeExpression(pipeline, window_description.after_window, "cast after window"); + executeExpression(pipeline, window_description.after_window, log, "cast after window"); analyzer = std::make_unique(window_description.after_window_columns, context); } @@ -733,7 +660,7 @@ void DAGQueryBlockInterpreter::executeImpl(DAGPipeline & pipeline) } if (res.before_order_and_select) { - executeExpression(pipeline, res.before_order_and_select, "before order and select"); + executeExpression(pipeline, res.before_order_and_select, log, "before order and select"); } if (!res.order_columns.empty()) @@ -769,10 +696,7 @@ void DAGQueryBlockInterpreter::executeProject(DAGPipeline & pipeline, NamesWithA if (project_cols.empty()) return; ExpressionActionsPtr project = generateProjectExpressionActions(pipeline.firstStream(), context, project_cols); - pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, project, log->identifier()); - stream->setExtraInfo(extra_info); - }); + executeExpression(pipeline, project, log, extra_info); } void DAGQueryBlockInterpreter::executeLimit(DAGPipeline & pipeline) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h index 532dceabce9..c449b37e360 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.h @@ -67,9 +67,7 @@ class DAGQueryBlockInterpreter void handleWindow(DAGPipeline & pipeline, const tipb::Window & window, bool enable_fine_grained_shuffle); void handleWindowOrder(DAGPipeline & pipeline, const tipb::Sort & window_sort, bool enable_fine_grained_shuffle); void executeWhere(DAGPipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, String & filter_column, const String & extra_info = ""); - void executeExpression(DAGPipeline & pipeline, const ExpressionActionsPtr & expressionActionsPtr, const String & extra_info = ""); void executeWindowOrder(DAGPipeline & pipeline, SortDescription sort_desc, bool enable_fine_grained_shuffle); - void orderStreams(DAGPipeline & pipeline, SortDescription order_descr, Int64 limit, bool enable_fine_grained_shuffle); void executeOrder(DAGPipeline & pipeline, const NamesAndTypes & order_columns); void executeLimit(DAGPipeline & pipeline); void executeWindow( diff --git a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp index be3475f714f..efb8a08f1d8 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.cpp @@ -54,4 +54,15 @@ ColumnsWithTypeAndName getColumnWithTypeAndName(const NamesAndTypes & names_and_ } return column_with_type_and_names; } -} // namespace DB \ No newline at end of file + +NamesAndTypes toNamesAndTypes(const DAGSchema & dag_schema) +{ + NamesAndTypes names_and_types; + for (const auto & col : dag_schema) + { + auto tp = getDataTypeByColumnInfoForComputingLayer(col.second); + names_and_types.emplace_back(col.first, tp); + } + return names_and_types; +} +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h index 617f69de925..96f202d800e 100644 --- a/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h +++ b/dbms/src/Flash/Coprocessor/GenSchemaAndColumn.h @@ -16,6 +16,7 @@ #include #include +#include #include #include @@ -23,4 +24,5 @@ namespace DB { NamesAndTypes genNamesAndTypes(const TiDBTableScan & table_scan); ColumnsWithTypeAndName getColumnWithTypeAndName(const NamesAndTypes & names_and_types); -} // namespace DB \ No newline at end of file +NamesAndTypes toNamesAndTypes(const DAGSchema & dag_schema); +} // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index 6415d36389b..002a06d07b9 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -12,8 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include +#include +#include #include #include +#include #include #include @@ -88,4 +92,77 @@ ExpressionActionsPtr generateProjectExpressionActions( project->add(ExpressionAction::project(project_cols)); return project; } + +void executeExpression( + DAGPipeline & pipeline, + const ExpressionActionsPtr & expr_actions, + const LoggerPtr & log, + const String & extra_info) +{ + if (expr_actions && !expr_actions->getActions().empty()) + { + pipeline.transform([&](auto & stream) { + stream = std::make_shared(stream, expr_actions, log->identifier()); + stream->setExtraInfo(extra_info); + }); + } +} + +void orderStreams( + DAGPipeline & pipeline, + size_t max_streams, + SortDescription order_descr, + Int64 limit, + bool enable_fine_grained_shuffle, + const Context & context, + const LoggerPtr & log) +{ + const Settings & settings = context.getSettingsRef(); + String extra_info; + if (enable_fine_grained_shuffle) + extra_info = enableFineGrainedShuffleExtraInfo; + + pipeline.transform([&](auto & stream) { + auto sorting_stream = std::make_shared(stream, order_descr, log->identifier(), limit); + + /// Limits on sorting + IProfilingBlockInputStream::LocalLimits limits; + limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL; + limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode); + sorting_stream->setLimits(limits); + + stream = sorting_stream; + stream->setExtraInfo(extra_info); + }); + + if (enable_fine_grained_shuffle) + { + pipeline.transform([&](auto & stream) { + stream = std::make_shared( + stream, + order_descr, + settings.max_block_size, + limit, + settings.max_bytes_before_external_sort, + context.getTemporaryPath(), + log->identifier()); + stream->setExtraInfo(enableFineGrainedShuffleExtraInfo); + }); + } + else + { + /// If there are several streams, we merge them into one + executeUnion(pipeline, max_streams, log, false, "for partial order"); + + /// Merge the sorted blocks. + pipeline.firstStream() = std::make_shared( + pipeline.firstStream(), + order_descr, + settings.max_block_size, + limit, + settings.max_bytes_before_external_sort, + context.getTemporaryPath(), + log->identifier()); + } +} } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index 5c4d4721d5e..bd64346718c 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -15,6 +15,7 @@ #pragma once #include +#include #include #include @@ -44,4 +45,19 @@ ExpressionActionsPtr generateProjectExpressionActions( const BlockInputStreamPtr & stream, const Context & context, const NamesWithAliases & project_cols); + +void executeExpression( + DAGPipeline & pipeline, + const ExpressionActionsPtr & expr_actions, + const LoggerPtr & log, + const String & extra_info = ""); + +void orderStreams( + DAGPipeline & pipeline, + size_t max_streams, + SortDescription order_descr, + Int64 limit, + bool enable_fine_grained_shuffle, + const Context & context, + const LoggerPtr & log); } // namespace DB diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp index 53b260f9638..75a0857465e 100644 --- a/dbms/src/Flash/tests/gtest_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_interpreter.cpp @@ -103,15 +103,12 @@ try Union: Expression x 10: Expression: - Expression: - Expression: - Expression: - Expression: + Expression: + Expression: + Expression: + Expression: Expression: - Expression: - Expression: - Expression: - MockTableScan)"; + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -125,16 +122,14 @@ Union: Union: Expression x 10: Expression: - Expression: - SharedQuery: - Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - Expression: - MockTableScan)"; + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -150,22 +145,18 @@ Union: Union: Expression x 10: Expression: - Expression: - Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - Expression: - SharedQuery: - Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - Expression: - MockTableScan)"; + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -188,27 +179,22 @@ Union: Limit x 10, limit = 10 Expression: Expression: - Expression: - Expression: - Expression: - Filter: - Expression: - Expression: - Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - Expression: - SharedQuery: - Expression: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - Expression: - MockTableScan)"; + Expression: + Expression: + Filter: + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + SharedQuery: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -222,15 +208,12 @@ Union: Union: Expression x 10: Expression: - Expression: - Expression: - Expression: - Expression: + Expression: + Expression: + Expression: + Expression: Expression: - Expression: - Expression: - Expression: - MockExchangeReceiver)"; + MockExchangeReceiver)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -246,15 +229,12 @@ Union: MockExchangeSender x 10 Expression: Expression: - Expression: - Expression: - Expression: - Expression: + Expression: + Expression: + Expression: + Expression: Expression: - Expression: - Expression: - Expression: - MockExchangeReceiver)"; + MockExchangeReceiver)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } } @@ -294,17 +274,16 @@ Union: Union: Expression x 10: Expression: - Expression: - Expression: - SharedQuery: - Expression: - Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} - Expression: - MergeSorting, limit = 0 - Union: - PartialSorting x 10: limit = 0 - Expression: - MockTableScan)"; + Expression: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Expression: + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -319,22 +298,20 @@ Union: Union: Expression x 10: Expression: - Expression: - Expression: - SharedQuery: - Expression: - Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} - Union: - Expression x 10: - Expression: - Expression: - SharedQuery: - Expression: - MergeSorting, limit = 0 - Union: - PartialSorting x 10: limit = 0 - Expression: - MockTableScan)"; + Expression: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Union: + Expression x 10: + Expression: + SharedQuery: + Expression: + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } } @@ -571,11 +548,10 @@ CreatingSets Expression x 10: SharedQuery: ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - HashJoinProbe: - Expression: - MockTableScan)"; + Expression x 10: + HashJoinProbe: + Expression: + MockTableScan)"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -601,15 +577,13 @@ CreatingSets Expression x 10: SharedQuery: ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - HashJoinProbe: - Expression: - Expression: - MockTableScan - Expression x 10: - Expression: - NonJoined: )"; + Expression x 10: + HashJoinProbe: + Expression: + Expression: + MockTableScan + Expression x 10: + NonJoined: )"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); } @@ -643,15 +617,13 @@ CreatingSets Expression: SharedQuery: ParallelAggregating, max_threads: 20, final: true - Expression x 20: - Expression: - HashJoinProbe: - Expression: - Expression: - MockExchangeReceiver - Expression x 20: - Expression: - NonJoined: )"; + Expression x 20: + HashJoinProbe: + Expression: + Expression: + MockExchangeReceiver + Expression x 20: + NonJoined: )"; ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 20); } } diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 9361e0525d2..add761c581d 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -209,7 +209,7 @@ struct Settings * Basically, limits are checked for each block (not every row). That is, the limits can be slightly violated. \ * Almost all limits apply only to SELECTs. \ * Almost all limits apply to each stream individually. \ - */ \ + */ \ \ M(SettingUInt64, max_rows_to_read, 0, "Limit on read rows from the most 'deep' sources. That is, only in the deepest subquery. When reading from a remote server, it " \ "is only checked on a remote server.") \ @@ -272,7 +272,7 @@ struct Settings M(SettingUInt64, dt_segment_delta_small_column_file_size, 8388608, "Determine whether a column file in delta is small or not. 8MB by default.") \ M(SettingUInt64, dt_segment_stable_pack_rows, DEFAULT_MERGE_BLOCK_SIZE, "Expected stable pack rows in DeltaTree Engine.") \ M(SettingFloat, dt_segment_wait_duration_factor, 1, "The factor of wait duration in a write stall.") \ - M(SettingUInt64, dt_bg_gc_check_interval, 60, "Background gc thread check interval, the unit is second.") \ + M(SettingUInt64, dt_bg_gc_check_interval, 60, "Background gc thread check interval, the unit is second.") \ M(SettingInt64, dt_bg_gc_max_segments_to_check_every_round, 100, "Max segments to check in every gc round, value less than or equal to 0 means gc no segments.") \ M(SettingFloat, dt_bg_gc_ratio_threhold_to_trigger_gc, 1.2, "Trigger segment's gc when the ratio of invalid version exceed this threhold. Values smaller than or equal to 1.0 means gc all " \ "segments") \ @@ -355,15 +355,15 @@ struct Settings M(SettingUInt64, elastic_threadpool_init_cap, 400, "The size of elastic thread pool.") \ M(SettingUInt64, elastic_threadpool_shrink_period_ms, 300000, "The shrink period(ms) of elastic thread pool.") \ M(SettingBool, enable_local_tunnel, true, "Enable local data transfer between local MPP tasks.") \ - M(SettingBool, enable_async_grpc_client, true, "Enable async grpc in MPP.") \ - M(SettingUInt64, grpc_completion_queue_pool_size, 0, "The size of gRPC completion queue pool. 0 means using hardware_concurrency.")\ + M(SettingBool, enable_async_grpc_client, true, "Enable async grpc in MPP.") \ + M(SettingUInt64, grpc_completion_queue_pool_size, 0, "The size of gRPC completion queue pool. 0 means using hardware_concurrency.") \ M(SettingBool, enable_async_server, true, "Enable async rpc server.") \ M(SettingUInt64, async_pollers_per_cq, 200, "grpc async pollers per cqs") \ M(SettingUInt64, async_cqs, 1, "grpc async cqs") \ M(SettingUInt64, preallocated_request_count_per_poller, 20, "grpc preallocated_request_count_per_poller") \ \ M(SettingUInt64, manual_compact_pool_size, 1, "The number of worker threads to handle manual compact requests.") \ - M(SettingUInt64, manual_compact_max_concurrency, 10, "Max concurrent tasks. It should be larger than pool size.") \ + M(SettingUInt64, manual_compact_max_concurrency, 10, "Max concurrent tasks. It should be larger than pool size.") \ M(SettingUInt64, manual_compact_more_until_ms, 60000, "Continuously compact more segments until reaching specified elapsed time. If 0 is specified, only one segment will be compacted each round.") // clang-format on diff --git a/dbms/src/TestUtils/mockExecutor.h b/dbms/src/TestUtils/mockExecutor.h index c034a8f86ca..8b5a6d300ff 100644 --- a/dbms/src/TestUtils/mockExecutor.h +++ b/dbms/src/TestUtils/mockExecutor.h @@ -166,6 +166,7 @@ MockWindowFrame buildDefaultRowsFrame(); #define col(name) buildColumn((name)) #define lit(field) buildLiteral((field)) +#define concat(expr1, expr2) makeASTFunction("concat", (expr1), (expr2)) #define eq(expr1, expr2) makeASTFunction("equals", (expr1), (expr2)) #define Not_eq(expr1, expr2) makeASTFunction("notEquals", (expr1), (expr2)) #define lt(expr1, expr2) makeASTFunction("less", (expr1), (expr2)) From b8a6178ea5bb423dc7d06e28a88c294c73d77b01 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Mon, 11 Jul 2022 23:21:53 +0800 Subject: [PATCH 104/104] f update for planner update for tests fix tests --- .../Coprocessor/DAGExpressionAnalyzer.cpp | 2 +- .../Flash/Coprocessor/DAGExpressionAnalyzer.h | 1 + dbms/src/Flash/Coprocessor/DAGUtils.cpp | 7 - dbms/src/Flash/Planner/Planner.cpp | 8 +- .../Planner/plans/PhysicalAggregation.cpp | 23 +- .../plans/PhysicalExchangeReceiver.cpp | 11 +- .../Planner/plans/PhysicalExchangeSender.cpp | 7 +- dbms/src/Flash/Planner/plans/PhysicalTopN.cpp | 2 +- .../Planner/tests/gtest_physical_plan.cpp | 2 +- dbms/src/Flash/tests/gtest_executor.cpp | 274 +++--- ..._interpreter.cpp => gtest_interpreter.cpp} | 8 +- dbms/src/Flash/tests/gtest_limit_executor.cpp | 36 +- .../Flash/tests/gtest_planner_interpreter.cpp | 827 ++++++++++++------ .../Flash/tests/gtest_projection_executor.cpp | 250 +++--- dbms/src/Flash/tests/gtest_topn_executor.cpp | 220 ++--- .../Functions/tests/gtest_strings_format.cpp | 6 - dbms/src/Interpreters/Settings.h | 2 + dbms/src/Server/main.cpp | 3 - dbms/src/TestUtils/ExecutorTestUtils.cpp | 7 +- dbms/src/TestUtils/ExecutorTestUtils.h | 13 + dbms/src/TestUtils/mockExecutor.cpp | 53 -- dbms/src/TiDB/Schema/SchemaBuilder.h | 35 +- 22 files changed, 1026 insertions(+), 771 deletions(-) rename dbms/src/Flash/tests/{gtest_qb_interpreter.cpp => gtest_interpreter.cpp} (99%) diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp index 00e66b7518d..5fbd86e9762 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.cpp @@ -1125,7 +1125,7 @@ std::pair DAGExpressionAnalyzer::isCastRequiredForRootFinalProjec } NamesWithAliases DAGExpressionAnalyzer::appendFinalProjectForRootQueryBlock( - const ExpressionActionsPtr & actions, + ExpressionActionsChain & chain, const std::vector & schema, const std::vector & output_offsets, const String & column_prefix, diff --git a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h index 6a2d026a478..c42312b95c3 100644 --- a/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h +++ b/dbms/src/Flash/Coprocessor/DAGExpressionAnalyzer.h @@ -199,6 +199,7 @@ class DAGExpressionAnalyzer : private boost::noncopyable #ifndef DBMS_PUBLIC_GTEST private: #endif + String buildTupleFunctionForGroupConcat( const tipb::Expr & expr, SortDescription & sort_desc, diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index 24cd7a33c0f..2003103a20a 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -1432,13 +1432,6 @@ tipb::EncodeType analyzeDAGEncodeType(DAGContext & dag_context) return tipb::EncodeType::TypeDefault; return encode_type; } -tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name) -{ - static std::unordered_map func_name_sig_map = getFuncNameToSigMap(); - if (func_name_sig_map.find(name) == func_name_sig_map.end()) - throw Exception(fmt::format("Unsupported function {}", name)); - return func_name_sig_map[name]; -} tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name) { diff --git a/dbms/src/Flash/Planner/Planner.cpp b/dbms/src/Flash/Planner/Planner.cpp index b798123de71..8aae14b9420 100644 --- a/dbms/src/Flash/Planner/Planner.cpp +++ b/dbms/src/Flash/Planner/Planner.cpp @@ -87,9 +87,15 @@ BlockInputStreams Planner::execute() bool Planner::isSupported(const DAGQueryBlock & query_block) { + /// todo support fine grained shuffle + static auto disable_fine_frained_shuffle = [](const DAGQueryBlock & query_block) { + return !enableFineGrainedShuffle(query_block.source->fine_grained_shuffle_stream_count()) + && (!query_block.exchange_sender || !enableFineGrainedShuffle(query_block.exchange_sender->fine_grained_shuffle_stream_count())); + }; return query_block.source && (query_block.source->tp() == tipb::ExecType::TypeProjection - || query_block.source->tp() == tipb::ExecType::TypeExchangeReceiver); + || query_block.source->tp() == tipb::ExecType::TypeExchangeReceiver) + && disable_fine_frained_shuffle(query_block); } DAGContext & Planner::dagContext() const diff --git a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp index 45e4586dd18..26a6fa574f2 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalAggregation.cpp @@ -106,34 +106,39 @@ void PhysicalAggregation::transformImpl(DAGPipeline & pipeline, Context & contex is_final_agg); /// If there are several sources, then we perform parallel aggregation - if (pipeline.streams.size() > 1) + if (pipeline.streams.size() > 1 || pipeline.streams_with_non_joined_data.size() > 1) { const Settings & settings = context.getSettingsRef(); - BlockInputStreamPtr stream_with_non_joined_data = combinedNonJoinedDataStream(pipeline, max_streams, log); - pipeline.firstStream() = std::make_shared( + BlockInputStreamPtr stream = std::make_shared( pipeline.streams, - stream_with_non_joined_data, + pipeline.streams_with_non_joined_data, params, context.getFileProvider(), true, max_streams, settings.aggregation_memory_efficient_merge_threads ? static_cast(settings.aggregation_memory_efficient_merge_threads) : static_cast(settings.max_threads), log->identifier()); + pipeline.streams.resize(1); + pipeline.streams_with_non_joined_data.clear(); + pipeline.firstStream() = std::move(stream); + // should record for agg before restore concurrency. See #3804. recordProfileStreams(pipeline, context); restoreConcurrency(pipeline, context.getDAGContext()->final_concurrency, log); } else { - BlockInputStreamPtr stream_with_non_joined_data = combinedNonJoinedDataStream(pipeline, max_streams, log); BlockInputStreams inputs; if (!pipeline.streams.empty()) inputs.push_back(pipeline.firstStream()); - else - pipeline.streams.resize(1); - if (stream_with_non_joined_data) - inputs.push_back(stream_with_non_joined_data); + + if (!pipeline.streams_with_non_joined_data.empty()) + inputs.push_back(pipeline.streams_with_non_joined_data.at(0)); + + pipeline.streams.resize(1); + pipeline.streams_with_non_joined_data.clear(); + pipeline.firstStream() = std::make_shared( std::make_shared(inputs, log->identifier()), params, diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp index ee40e42e1aa..ca87a85ab17 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeReceiver.cpp @@ -43,15 +43,14 @@ PhysicalPlanNodePtr PhysicalExchangeReceiver::build( const String & executor_id, const LoggerPtr & log) { - const auto & mpp_exchange_receiver_map = context.getDAGContext()->getMPPExchangeReceiverMap(); - - auto it = mpp_exchange_receiver_map.find(executor_id); - if (unlikely(it == mpp_exchange_receiver_map.end())) + auto mpp_exchange_receiver = context.getDAGContext()->getMPPExchangeReceiver(executor_id); + if (unlikely(mpp_exchange_receiver == nullptr)) throw TiFlashException( fmt::format("Can not find exchange receiver for {}", executor_id), Errors::Planner::Internal); + /// todo support fine grained shuffle + assert(!enableFineGrainedShuffle(mpp_exchange_receiver->getFineGrainedShuffleStreamCount())); - const auto & mpp_exchange_receiver = it->second; NamesAndTypes schema = toNamesAndTypes(mpp_exchange_receiver->getOutputSchema()); auto physical_exchange_receiver = std::make_shared( executor_id, @@ -69,7 +68,7 @@ void PhysicalExchangeReceiver::transformImpl(DAGPipeline & pipeline, Context & c auto & exchange_receiver_io_input_streams = dag_context.getInBoundIOInputStreamsMap()[executor_id]; for (size_t i = 0; i < max_streams; ++i) { - BlockInputStreamPtr stream = std::make_shared(mpp_exchange_receiver, log->identifier(), executor_id); + BlockInputStreamPtr stream = std::make_shared(mpp_exchange_receiver, log->identifier(), executor_id, /*stream_id=*/0); exchange_receiver_io_input_streams.push_back(stream); stream = std::make_shared(stream, 8192, 0, log->identifier()); stream->setExtraInfo("squashing after exchange receiver"); diff --git a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp index 373b04a3941..1f99656506e 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalExchangeSender.cpp @@ -55,10 +55,11 @@ void PhysicalExchangeSender::transformImpl(DAGPipeline & pipeline, Context & con RUNTIME_ASSERT(dag_context.isMPPTask() && dag_context.tunnel_set != nullptr, log, "exchange_sender only run in MPP"); + /// todo support fine grained shuffle int stream_id = 0; pipeline.transform([&](auto & stream) { // construct writer - std::unique_ptr response_writer = std::make_unique>( + std::unique_ptr response_writer = std::make_unique>( dag_context.tunnel_set, partition_col_ids, partition_col_collators, @@ -66,7 +67,9 @@ void PhysicalExchangeSender::transformImpl(DAGPipeline & pipeline, Context & con context.getSettingsRef().dag_records_per_chunk, context.getSettingsRef().batch_send_min_limit, stream_id++ == 0, /// only one stream needs to sending execution summaries for the last response - dag_context); + dag_context, + 0, + 0); stream = std::make_shared(stream, std::move(response_writer), log->identifier()); }); } diff --git a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp index d572435d645..a78dd350264 100644 --- a/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp +++ b/dbms/src/Flash/Planner/plans/PhysicalTopN.cpp @@ -62,7 +62,7 @@ void PhysicalTopN::transformImpl(DAGPipeline & pipeline, Context & context, size executeExpression(pipeline, before_sort_actions, log, "before TopN"); - orderStreams(pipeline, max_streams, order_descr, limit, context, log); + orderStreams(pipeline, max_streams, order_descr, limit, false, context, log); } void PhysicalTopN::finalize(const Names & parent_require) diff --git a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp index 7ad06fbeb04..45414b717df 100644 --- a/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp +++ b/dbms/src/Flash/Planner/tests/gtest_physical_plan.cpp @@ -69,7 +69,7 @@ class PhysicalPlanTestRunner : public DB::tests::ExecutorTest ASSERT_EQ(Poco::trim(expected_streams), Poco::trim(fb.toString())); } - readAndAssertBlock(final_stream, expect_columns); + ASSERT_COLUMNS_EQ_R(readBlock(final_stream), expect_columns); } LoggerPtr log = Logger::get("PhysicalPlanTestRunner", "test_physical_plan"); diff --git a/dbms/src/Flash/tests/gtest_executor.cpp b/dbms/src/Flash/tests/gtest_executor.cpp index d0e7b7e6c67..ee35af0c03d 100644 --- a/dbms/src/Flash/tests/gtest_executor.cpp +++ b/dbms/src/Flash/tests/gtest_executor.cpp @@ -64,157 +64,165 @@ class ExecutorTestRunner : public DB::tests::ExecutorTest TEST_F(ExecutorTestRunner, Filter) try { - auto request = context - .scan("test_db", "test_table") - .filter(eq(col("s1"), col("s2"))) - .build(context); - { - ASSERT_COLUMNS_EQ_R(executeStreams(request), - createColumns({toNullableVec({"banana"}), - toNullableVec({"banana"})})); - } - - request = context.receive("exchange1") - .filter(eq(col("s1"), col("s2"))) - .build(context); - { - ASSERT_COLUMNS_EQ_R(executeStreams(request), - createColumns({toNullableVec({"banana"}), - toNullableVec({"banana"})})); - } + wrapForDisEnablePlanner([&]() { + auto request = context + .scan("test_db", "test_table") + .filter(eq(col("s1"), col("s2"))) + .build(context); + { + ASSERT_COLUMNS_EQ_R(executeStreams(request), + createColumns({toNullableVec({"banana"}), + toNullableVec({"banana"})})); + } + + request = context.receive("exchange1") + .filter(eq(col("s1"), col("s2"))) + .build(context); + { + ASSERT_COLUMNS_EQ_R(executeStreams(request), + createColumns({toNullableVec({"banana"}), + toNullableVec({"banana"})})); + } + }); } CATCH TEST_F(ExecutorTestRunner, JoinWithTableScan) try { - auto request = context - .scan("test_db", "l_table") - .join(context.scan("test_db", "r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) - .topN("join_c", false, 2) - .build(context); - { - String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 2\n" - " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" - " table_scan_0 | {<0, String>, <1, String>}\n" - " table_scan_1 | {<0, String>, <1, String>}\n"; - ASSERT_DAGREQUEST_EQAUL(expected, request); - ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), - createColumns({toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})})); - - ASSERT_COLUMNS_EQ_R(executeStreams(request, 5), - createColumns({toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})})); - - ASSERT_COLUMNS_EQ_R(executeStreams(request), - createColumns({toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})})); - } - request = context - .scan("test_db", "l_table") - .join(context.scan("test_db", "r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) - .project({"s", "join_c"}) - .topN("join_c", false, 2) - .build(context); - { - String expected = "topn_4 | order_by: {(<1, String>, desc: false)}, limit: 2\n" - " project_3 | {<0, String>, <1, String>}\n" - " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" - " table_scan_0 | {<0, String>, <1, String>}\n" - " table_scan_1 | {<0, String>, <1, String>}\n"; - ASSERT_DAGREQUEST_EQAUL(expected, request); - ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), - createColumns({toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})})); - } - - request = context - .scan("test_db", "l_table") - .join(context.scan("test_db", "r_table_2"), {col("join_c")}, ASTTableJoin::Kind::Left) - .topN("join_c", false, 4) - .build(context); - { - String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 4\n" - " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" - " table_scan_0 | {<0, String>, <1, String>}\n" - " table_scan_1 | {<0, String>, <1, String>}\n"; - ASSERT_DAGREQUEST_EQAUL(expected, request); - ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), - createColumns({toNullableVec({"banana", "banana", "banana", "banana"}), - toNullableVec({"apple", "apple", "apple", "banana"}), - toNullableVec({"banana", "banana", "banana", {}}), - toNullableVec({"apple", "apple", "apple", {}})})); - ASSERT_COLUMNS_EQ_R(executeStreams(request, 3), - createColumns({toNullableVec({"banana", "banana", "banana", "banana"}), - toNullableVec({"apple", "apple", "apple", "banana"}), - toNullableVec({"banana", "banana", "banana", {}}), - toNullableVec({"apple", "apple", "apple", {}})})); - } + wrapForDisEnablePlanner([&]() { + auto request = context + .scan("test_db", "l_table") + .join(context.scan("test_db", "r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) + .topN("join_c", false, 2) + .build(context); + { + String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 2\n" + " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" + " table_scan_0 | {<0, String>, <1, String>}\n" + " table_scan_1 | {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); + + ASSERT_COLUMNS_EQ_R(executeStreams(request, 5), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); + + ASSERT_COLUMNS_EQ_R(executeStreams(request), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); + } + request = context + .scan("test_db", "l_table") + .join(context.scan("test_db", "r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) + .project({"s", "join_c"}) + .topN("join_c", false, 2) + .build(context); + { + String expected = "topn_4 | order_by: {(<1, String>, desc: false)}, limit: 2\n" + " project_3 | {<0, String>, <1, String>}\n" + " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" + " table_scan_0 | {<0, String>, <1, String>}\n" + " table_scan_1 | {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); + } + + request = context + .scan("test_db", "l_table") + .join(context.scan("test_db", "r_table_2"), {col("join_c")}, ASTTableJoin::Kind::Left) + .topN("join_c", false, 4) + .build(context); + { + String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 4\n" + " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" + " table_scan_0 | {<0, String>, <1, String>}\n" + " table_scan_1 | {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), + createColumns({toNullableVec({"banana", "banana", "banana", "banana"}), + toNullableVec({"apple", "apple", "apple", "banana"}), + toNullableVec({"banana", "banana", "banana", {}}), + toNullableVec({"apple", "apple", "apple", {}})})); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 3), + createColumns({toNullableVec({"banana", "banana", "banana", "banana"}), + toNullableVec({"apple", "apple", "apple", "banana"}), + toNullableVec({"banana", "banana", "banana", {}}), + toNullableVec({"apple", "apple", "apple", {}})})); + } + }); } CATCH TEST_F(ExecutorTestRunner, JoinWithExchangeReceiver) try { - auto request = context - .receive("exchange_l_table") - .join(context.receive("exchange_r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) - .topN("join_c", false, 2) - .build(context); - { - String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 2\n" - " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" - " exchange_receiver_0 | type:PassThrough, {<0, String>, <1, String>}\n" - " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; - ASSERT_DAGREQUEST_EQAUL(expected, request); - ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), - createColumns({toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})})); - - ASSERT_COLUMNS_EQ_R(executeStreams(request, 5), - createColumns({toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})})); - - ASSERT_COLUMNS_EQ_R(executeStreams(request), - createColumns({toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})})); - } + wrapForDisEnablePlanner([&]() { + auto request = context + .receive("exchange_l_table") + .join(context.receive("exchange_r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) + .topN("join_c", false, 2) + .build(context); + { + String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 2\n" + " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" + " exchange_receiver_0 | type:PassThrough, {<0, String>, <1, String>}\n" + " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); + + ASSERT_COLUMNS_EQ_R(executeStreams(request, 5), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); + + ASSERT_COLUMNS_EQ_R(executeStreams(request), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); + } + }); } CATCH TEST_F(ExecutorTestRunner, JoinWithTableScanAndReceiver) try { - auto request = context - .scan("test_db", "l_table") - .join(context.receive("exchange_r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) - .topN("join_c", false, 2) - .build(context); - { - String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 2\n" - " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" - " table_scan_0 | {<0, String>, <1, String>}\n" - " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; - ASSERT_DAGREQUEST_EQAUL(expected, request); - ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), - createColumns({toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"}), - toNullableVec({"banana", "banana"}), - toNullableVec({"apple", "banana"})})); - } + wrapForDisEnablePlanner([&]() { + auto request = context + .scan("test_db", "l_table") + .join(context.receive("exchange_r_table"), {col("join_c")}, ASTTableJoin::Kind::Left) + .topN("join_c", false, 2) + .build(context); + { + String expected = "topn_3 | order_by: {(<1, String>, desc: false)}, limit: 2\n" + " Join_2 | LeftOuterJoin, HashJoin. left_join_keys: {<0, String>}, right_join_keys: {<0, String>}\n" + " table_scan_0 | {<0, String>, <1, String>}\n" + " exchange_receiver_1 | type:PassThrough, {<0, String>, <1, String>}\n"; + ASSERT_DAGREQUEST_EQAUL(expected, request); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), + createColumns({toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"}), + toNullableVec({"banana", "banana"}), + toNullableVec({"apple", "banana"})})); + } + }); } CATCH diff --git a/dbms/src/Flash/tests/gtest_qb_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp similarity index 99% rename from dbms/src/Flash/tests/gtest_qb_interpreter.cpp rename to dbms/src/Flash/tests/gtest_interpreter.cpp index 00336cd8871..d681cfaf66b 100644 --- a/dbms/src/Flash/tests/gtest_qb_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_interpreter.cpp @@ -19,14 +19,14 @@ namespace DB { namespace tests { -class QBInterpreterExecuteTest : public DB::tests::ExecutorTest +class InterpreterExecuteTest : public DB::tests::ExecutorTest { public: void initializeContext() override { ExecutorTest::initializeContext(); - context.context.setSetting("enable_planner", "false"); + enablePlanner(false); context.addMockTable({"test_db", "test_table"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "test_table_1"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); @@ -38,7 +38,7 @@ class QBInterpreterExecuteTest : public DB::tests::ExecutorTest } }; -TEST_F(QBInterpreterExecuteTest, SingleQueryBlock) +TEST_F(InterpreterExecuteTest, SingleQueryBlock) try { auto request = context.scan("test_db", "test_table_1") @@ -92,7 +92,7 @@ Union: } CATCH -TEST_F(QBInterpreterExecuteTest, MultipleQueryBlockWithSource) +TEST_F(InterpreterExecuteTest, MultipleQueryBlockWithSource) try { auto request = context.scan("test_db", "test_table_1") diff --git a/dbms/src/Flash/tests/gtest_limit_executor.cpp b/dbms/src/Flash/tests/gtest_limit_executor.cpp index 47482540b39..a3ddf341525 100644 --- a/dbms/src/Flash/tests/gtest_limit_executor.cpp +++ b/dbms/src/Flash/tests/gtest_limit_executor.cpp @@ -50,26 +50,28 @@ class ExecutorLimitTestRunner : public DB::tests::ExecutorTest TEST_F(ExecutorLimitTestRunner, Limit) try { - std::shared_ptr request; - ColumnsWithTypeAndName expect_cols; + wrapForDisEnablePlanner([&]() { + std::shared_ptr request; + ColumnsWithTypeAndName expect_cols; - /// Check limit result with various parameters - const size_t col_data_num = col0.size(); - for (size_t limit_num = 0; limit_num <= col_data_num + 3; ++limit_num) - { - if (limit_num == col_data_num + 3) - limit_num = INT_MAX; - request = buildDAGRequest(limit_num); + /// Check limit result with various parameters + const size_t col_data_num = col0.size(); + for (size_t limit_num = 0; limit_num <= col_data_num + 3; ++limit_num) + { + if (limit_num == col_data_num + 3) + limit_num = INT_MAX; + request = buildDAGRequest(limit_num); - if (limit_num == 0) - expect_cols = {}; - else if (limit_num > col_data_num) - expect_cols = {toNullableVec(col_name, ColumnWithData(col0.begin(), col0.end()))}; - else - expect_cols = {toNullableVec(col_name, ColumnWithData(col0.begin(), col0.begin() + limit_num))}; + if (limit_num == 0) + expect_cols = {}; + else if (limit_num > col_data_num) + expect_cols = {toNullableVec(col_name, ColumnWithData(col0.begin(), col0.end()))}; + else + expect_cols = {toNullableVec(col_name, ColumnWithData(col0.begin(), col0.begin() + limit_num))}; - ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols); - } + ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols); + } + }); } CATCH diff --git a/dbms/src/Flash/tests/gtest_planner_interpreter.cpp b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp index 3840b858340..aa4950a2b22 100644 --- a/dbms/src/Flash/tests/gtest_planner_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_planner_interpreter.cpp @@ -26,19 +26,19 @@ class PlannerInterpreterExecuteTest : public DB::tests::ExecutorTest { ExecutorTest::initializeContext(); - context.context.setSetting("enable_planner", "true"); + enablePlanner(true); context.addMockTable({"test_db", "test_table"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "test_table_1"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "r_table"}, {{"r_a", TiDB::TP::TypeLong}, {"r_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "l_table"}, {{"l_a", TiDB::TP::TypeLong}, {"l_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); context.addExchangeRelationSchema("sender_1", {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); - context.addExchangeRelationSchema("sender_l", {{"l_a", TiDB::TP::TypeString}, {"l_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); - context.addExchangeRelationSchema("sender_r", {{"r_a", TiDB::TP::TypeString}, {"r_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + context.addExchangeRelationSchema("sender_l", {{"l_a", TiDB::TP::TypeLong}, {"l_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); + context.addExchangeRelationSchema("sender_r", {{"r_a", TiDB::TP::TypeLong}, {"r_b", TiDB::TP::TypeString}, {"join_c", TiDB::TP::TypeString}}); } }; -TEST_F(PlannerInterpreterExecuteTest, SimpleQuery) +TEST_F(PlannerInterpreterExecuteTest, SingleQueryBlock) try { auto request = context.scan("test_db", "test_table_1") @@ -92,282 +92,6 @@ Union: } CATCH -TEST_F(PlannerInterpreterExecuteTest, ComplexQuery) -try -{ - auto request = context.scan("test_db", "test_table_1") - .project({"s1", "s2", "s3"}) - .project({"s1", "s2"}) - .project("s1") - .build(context); - { - String expected = R"( -Union: - Expression x 10: - Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - MockTableScan)"; - ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - } - - request = context.scan("test_db", "test_table_1") - .project({"s1", "s2", "s3"}) - .topN({{"s1", true}, {"s2", false}}, 10) - .project({"s1", "s2"}) - .build(context); - { - String expected = R"( -Union: - Expression x 10: - Expression: - Expression: - SharedQuery: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - MockTableScan)"; - ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - } - - request = context.scan("test_db", "test_table_1") - .project({"s1", "s2", "s3"}) - .topN({{"s1", true}, {"s2", false}}, 10) - .project({"s1", "s2"}) - .aggregation({Max(col("s1"))}, {col("s1"), col("s2")}) - .project({"max(s1)", "s1", "s2"}) - .build(context); - { - String expected = R"( -Union: - Expression x 10: - Expression: - Expression: - Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - SharedQuery: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - MockTableScan)"; - ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - } - - request = context.scan("test_db", "test_table_1") - .project({"s1", "s2", "s3"}) - .topN({{"s1", true}, {"s2", false}}, 10) - .project({"s1", "s2"}) - .aggregation({Max(col("s1"))}, {col("s1"), col("s2")}) - .project({"max(s1)", "s1", "s2"}) - .filter(eq(col("s1"), col("s2"))) - .project({"max(s1)", "s1"}) - .limit(10) - .build(context); - { - String expected = R"( -Union: - Expression x 10: - SharedQuery: - Limit, limit = 10 - Union: - Limit x 10, limit = 10 - Expression: - Expression: - Filter - Expression: - Expression: - Expression: - SharedQuery: - ParallelAggregating, max_threads: 10, final: true - Expression x 10: - Expression: - SharedQuery: - MergeSorting, limit = 10 - Union: - PartialSorting x 10: limit = 10 - Expression: - Expression: - MockTableScan)"; - ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - } - - // Join Source. - DAGRequestBuilder table1 = context.scan("test_db", "r_table"); - DAGRequestBuilder table2 = context.scan("test_db", "l_table"); - DAGRequestBuilder table3 = context.scan("test_db", "r_table"); - DAGRequestBuilder table4 = context.scan("test_db", "l_table"); - - request = table1.join( - table2.join( - table3.join(table4, - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left) - .build(context); - { - String expected = R"( -CreatingSets - Union: - HashJoinBuildBlockInputStream x 10: , join_kind = Left - Expression: - Expression: - MockTableScan - Union x 2: - HashJoinBuildBlockInputStream x 10: , join_kind = Left - Expression: - Expression: - Expression: - HashJoinProbe: - Expression: - MockTableScan - Union: - Expression x 10: - Expression: - HashJoinProbe: - Expression: - MockTableScan)"; - ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - } - - request = context.receive("sender_1") - .project({"s1", "s2", "s3"}) - .project({"s1", "s2"}) - .project("s1") - .build(context); - { - String expected = R"( -Union: - Expression x 10: - Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - MockExchangeReceiver)"; - ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - } - - request = context.receive("sender_1") - .project({"s1", "s2", "s3"}) - .project({"s1", "s2"}) - .project("s1") - .exchangeSender(tipb::Broadcast) - .build(context); - { - String expected = R"( -Union: - MockExchangeSender x 10 - Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - Expression: - MockExchangeReceiver)"; - ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - } - - // only join + ExchangeReceiver - DAGRequestBuilder receiver1 = context.receive("sender_l"); - DAGRequestBuilder receiver2 = context.receive("sender_r"); - DAGRequestBuilder receiver3 = context.receive("sender_l"); - DAGRequestBuilder receiver4 = context.receive("sender_r"); - - request = receiver1.join( - receiver2.join( - receiver3.join(receiver4, - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left) - .build(context); - { - String expected = R"( -CreatingSets - Union: - HashJoinBuildBlockInputStream x 10: , join_kind = Left - Expression: - Expression: - MockExchangeReceiver - Union x 2: - HashJoinBuildBlockInputStream x 10: , join_kind = Left - Expression: - Expression: - Expression: - HashJoinProbe: - Expression: - MockExchangeReceiver - Union: - Expression x 10: - Expression: - HashJoinProbe: - Expression: - MockExchangeReceiver)"; - ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - } - - // join + receiver + sender - // TODO: Find a way to write the request easier. - DAGRequestBuilder receiver5 = context.receive("sender_l"); - DAGRequestBuilder receiver6 = context.receive("sender_r"); - DAGRequestBuilder receiver7 = context.receive("sender_l"); - DAGRequestBuilder receiver8 = context.receive("sender_r"); - request = receiver5.join( - receiver6.join( - receiver7.join(receiver8, - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left), - {col("join_c")}, - ASTTableJoin::Kind::Left) - .exchangeSender(tipb::PassThrough) - .build(context); - { - String expected = R"( -CreatingSets - Union: - HashJoinBuildBlockInputStream x 10: , join_kind = Left - Expression: - Expression: - MockExchangeReceiver - Union x 2: - HashJoinBuildBlockInputStream x 10: , join_kind = Left - Expression: - Expression: - Expression: - HashJoinProbe: - Expression: - MockExchangeReceiver - Union: - MockExchangeSender x 10 - Expression: - Expression: - HashJoinProbe: - Expression: - MockExchangeReceiver)"; - ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); - } -} -CATCH - TEST_F(PlannerInterpreterExecuteTest, ParallelQuery) try { @@ -673,5 +397,548 @@ CreatingSets } CATCH +TEST_F(PlannerInterpreterExecuteTest, MultipleQueryBlockWithSource) +try +{ + auto request = context.scan("test_db", "test_table_1") + .project({"s1", "s2", "s3"}) + .project({"s1", "s2"}) + .project({"s1"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .project({"s1", "s2", "s3"}) + .topN({{"s1", true}, {"s2", false}}, 10) + .project({"s1", "s2"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + Expression: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .project({"s1", "s2", "s3"}) + .topN({{"s1", true}, {"s2", false}}, 10) + .project({"s1", "s2"}) + .aggregation({Max(col("s1"))}, {col("s1"), col("s2")}) + .project({"max(s1)", "s1", "s2"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .project({"s1", "s2", "s3"}) + .topN({{"s1", true}, {"s2", false}}, 10) + .project({"s1", "s2"}) + .aggregation({Max(col("s1"))}, {col("s1"), col("s2")}) + .project({"max(s1)", "s1", "s2"}) + .filter(eq(col("s1"), col("s2"))) + .project({"max(s1)", "s1"}) + .limit(10) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + Limit, limit = 10 + Union: + Limit x 10, limit = 10 + Expression: + Expression: + Filter + Expression: + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + Expression: + SharedQuery: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + Expression: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.receive("sender_1") + .project({"s1", "s2", "s3"}) + .project({"s1", "s2"}) + .project({"s1"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.receive("sender_1") + .project({"s1", "s2", "s3"}) + .project({"s1", "s2"}) + .project({"s1"}) + .exchangeSender(tipb::Broadcast) + .build(context); + { + String expected = R"( +Union: + MockExchangeSender x 10 + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PlannerInterpreterExecuteTest, Window) +try +{ + auto request = context + .scan("test_db", "test_table") + .sort({{"s1", true}, {"s2", false}}, true) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame()) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Expression: + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table") + .sort({{"s1", true}, {"s2", false}}, true) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame()) + .project({"s1", "s2", "RowNumber()"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + Expression: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Expression: + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + request = context.scan("test_db", "test_table_1") + .sort({{"s1", true}, {"s2", false}}, true) + .project({"s1", "s2", "s3"}) + .window(RowNumber(), {"s1", true}, {"s1", false}, buildDefaultRowsFrame()) + .project({"s1", "s2", "s3", "RowNumber()"}) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + Expression: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Union: + Expression x 10: + Expression: + SharedQuery: + Expression: + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +/// todo support FineGrainedShuffle +/* +TEST_F(PlannerInterpreterExecuteTest, FineGrainedShuffle) +try +{ + // fine-grained shuffle is enabled. + const uint64_t enable = 8; + const uint64_t disable = 0; + auto request = context + .receive("sender_1", enable) + .sort({{"s1", true}, {"s2", false}}, true, enable) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame(), enable) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + Expression: + Window: , function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Expression: + MergeSorting: , limit = 0 + PartialSorting: : limit = 0 + Expression: + MockExchangeReceiver + )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + auto topn_request = context + .receive("sender_1") + .topN("s2", false, 10) + .build(context); + String topn_expected = R"( +Union: + SharedQuery x 10: + Expression: + MergeSorting, limit = 10 + Union: + PartialSorting x 10: limit = 10 + MockExchangeReceiver + )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(topn_expected, topn_request, 10); + + // fine-grained shuffle is disabled. + request = context + .receive("sender_1", disable) + .sort({{"s1", true}, {"s2", false}}, true, disable) + .window(RowNumber(), {"s1", true}, {"s2", false}, buildDefaultRowsFrame(), disable) + .build(context); + { + String expected = R"( +Union: + Expression x 10: + SharedQuery: + Expression: + Window, function: {row_number}, frame: {type: Rows, boundary_begin: Current, boundary_end: Current} + Expression: + MergeSorting, limit = 0 + Union: + PartialSorting x 10: limit = 0 + Expression: + MockExchangeReceiver + )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + topn_request = context + .receive("sender_1") + .topN("s2", false, 10) + .build(context); + ASSERT_BLOCKINPUTSTREAM_EQAUL(topn_expected, topn_request, 10); +} +CATCH +*/ + +TEST_F(PlannerInterpreterExecuteTest, Join) +try +{ + // TODO: Find a way to write the request easier. + { + // Join Source. + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + DAGRequestBuilder table3 = context.scan("test_db", "r_table"); + DAGRequestBuilder table4 = context.scan("test_db", "l_table"); + + auto request = table1.join( + table2.join( + table3.join(table4, + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + MockTableScan + Union x 2: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + Expression: + HashJoinProbe: + Expression: + MockTableScan + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + // only join + ExchangeReceiver + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r"); + DAGRequestBuilder receiver3 = context.receive("sender_l"); + DAGRequestBuilder receiver4 = context.receive("sender_r"); + + auto request = receiver1.join( + receiver2.join( + receiver3.join(receiver4, + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + MockExchangeReceiver + Union x 2: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver + Union: + Expression x 10: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + // join + receiver + sender + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r"); + DAGRequestBuilder receiver3 = context.receive("sender_l"); + DAGRequestBuilder receiver4 = context.receive("sender_r"); + + auto request = receiver1.join( + receiver2.join( + receiver3.join(receiver4, + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left), + {col("join_c")}, + ASTTableJoin::Kind::Left) + .exchangeSender(tipb::PassThrough) + .build(context); + + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + MockExchangeReceiver + Union x 2: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver + Union: + MockExchangeSender x 10 + Expression: + Expression: + HashJoinProbe: + Expression: + MockExchangeReceiver)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } +} +CATCH + +TEST_F(PlannerInterpreterExecuteTest, JoinThenAgg) +try +{ + { + // Left Join. + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + + auto request = table1.join( + table2, + {col("join_c")}, + ASTTableJoin::Kind::Left) + .aggregation({Max(col("r_a"))}, {col("join_c")}) + .build(context); + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Left + Expression: + Expression: + MockTableScan + Union: + Expression x 10: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + HashJoinProbe: + Expression: + MockTableScan)"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + // Right Join + DAGRequestBuilder table1 = context.scan("test_db", "r_table"); + DAGRequestBuilder table2 = context.scan("test_db", "l_table"); + + auto request = table1.join( + table2, + {col("join_c")}, + ASTTableJoin::Kind::Right) + .aggregation({Max(col("r_a"))}, {col("join_c")}) + .build(context); + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 10: , join_kind = Right + Expression: + Expression: + MockTableScan + Union: + Expression x 10: + SharedQuery: + ParallelAggregating, max_threads: 10, final: true + Expression x 10: + HashJoinProbe: + Expression: + Expression: + MockTableScan + Expression x 10: + NonJoined: )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 10); + } + + { + // Right join + receiver + sender + DAGRequestBuilder receiver1 = context.receive("sender_l"); + DAGRequestBuilder receiver2 = context.receive("sender_r"); + + auto request = receiver1.join( + receiver2, + {col("join_c")}, + ASTTableJoin::Kind::Right) + .aggregation({Sum(col("r_a"))}, {col("join_c")}) + .exchangeSender(tipb::PassThrough) + .limit(10) + .build(context); + String expected = R"( +CreatingSets + Union: + HashJoinBuildBlockInputStream x 20: , join_kind = Right + Expression: + Expression: + MockExchangeReceiver + Union: + MockExchangeSender x 20 + SharedQuery: + Limit, limit = 10 + Union: + Limit x 20, limit = 10 + Expression: + Expression: + SharedQuery: + ParallelAggregating, max_threads: 20, final: true + Expression x 20: + HashJoinProbe: + Expression: + Expression: + MockExchangeReceiver + Expression x 20: + NonJoined: )"; + ASSERT_BLOCKINPUTSTREAM_EQAUL(expected, request, 20); + } +} +CATCH + } // namespace tests } // namespace DB diff --git a/dbms/src/Flash/tests/gtest_projection_executor.cpp b/dbms/src/Flash/tests/gtest_projection_executor.cpp index 8443dedeb49..65b75bad9ab 100644 --- a/dbms/src/Flash/tests/gtest_projection_executor.cpp +++ b/dbms/src/Flash/tests/gtest_projection_executor.cpp @@ -85,138 +85,142 @@ class ExecutorProjectionTestRunner : public DB::tests::ExecutorTest TEST_F(ExecutorProjectionTestRunner, Projection) try { - /// Check single column - auto request = buildDAGRequest({col_names[4]}); - executeWithConcurrency(request, {toNullableVec(col_names[4], col4_sorted_asc)}); - - /// Check multi columns - request = buildDAGRequest({col_names[0], col_names[4]}); - executeWithConcurrency(request, - { - toNullableVec(col_names[0], col0_sorted_asc), - toNullableVec(col_names[4], col4_sorted_asc), - }); - - /// Check multi columns - request = buildDAGRequest({col_names[0], col_names[1], col_names[4]}); - executeWithConcurrency(request, - {toNullableVec(col_names[0], col0_sorted_asc), - toNullableVec(col_names[1], col1_sorted_asc), - toNullableVec(col_names[4], col4_sorted_asc)}); - - /// Check duplicate columns - request = buildDAGRequest({col_names[4], col_names[4], col_names[4]}); - executeWithConcurrency(request, - {toNullableVec(col_names[4], col4_sorted_asc), - toNullableVec(col_names[4], col4_sorted_asc), - toNullableVec(col_names[4], col4_sorted_asc)}); + wrapForDisEnablePlanner([&]() { + /// Check single column + auto request = buildDAGRequest({col_names[4]}); + executeWithConcurrency(request, {toNullableVec(col_names[4], col4_sorted_asc)}); + + /// Check multi columns + request = buildDAGRequest({col_names[0], col_names[4]}); + executeWithConcurrency(request, + { + toNullableVec(col_names[0], col0_sorted_asc), + toNullableVec(col_names[4], col4_sorted_asc), + }); + + /// Check multi columns + request = buildDAGRequest({col_names[0], col_names[1], col_names[4]}); + executeWithConcurrency(request, + {toNullableVec(col_names[0], col0_sorted_asc), + toNullableVec(col_names[1], col1_sorted_asc), + toNullableVec(col_names[4], col4_sorted_asc)}); + + /// Check duplicate columns + request = buildDAGRequest({col_names[4], col_names[4], col_names[4]}); + executeWithConcurrency(request, + {toNullableVec(col_names[4], col4_sorted_asc), + toNullableVec(col_names[4], col4_sorted_asc), + toNullableVec(col_names[4], col4_sorted_asc)}); - { - /// Check large number of columns - const size_t col_num = 100; - MockColumnNameVec projection_input; - ColumnsWithTypeAndName columns; - auto expect_column = toNullableVec(col_names[4], col4_sorted_asc); - - for (size_t i = 0; i < col_num; ++i) { - projection_input.push_back(col_names[4]); - columns.push_back(expect_column); + /// Check large number of columns + const size_t col_num = 100; + MockColumnNameVec projection_input; + ColumnsWithTypeAndName columns; + auto expect_column = toNullableVec(col_names[4], col4_sorted_asc); + + for (size_t i = 0; i < col_num; ++i) + { + projection_input.push_back(col_names[4]); + columns.push_back(expect_column); + } + + request = buildDAGRequest(projection_input); + executeWithConcurrency(request, columns); } - - request = buildDAGRequest(projection_input); - executeWithConcurrency(request, columns); - } + }); } CATCH TEST_F(ExecutorProjectionTestRunner, ProjectionFunction) try { - std::shared_ptr request; - - /// Test "equal" function - - /// Data type: TypeString - request = buildDAGRequest({eq(col(col_names[0]), col(col_names[0])), col(col_names[4])}); - executeWithConcurrency(request, - {toNullableVec({{}, 1, 1, 1, 1, 1, 1}), - toNullableVec(col_names[4], col4_sorted_asc)}); - - request = buildDAGRequest({eq(col(col_names[0]), col(col_names[1])), col(col_names[4])}); - executeWithConcurrency(request, - {toNullableVec({{}, 0, 1, 0, {}, 0, 0}), - toNullableVec(col_names[4], col4_sorted_asc)}); - - /// Data type: TypeLong - request = buildDAGRequest({eq(col(col_names[3]), col(col_names[4])), col(col_names[4])}); - executeWithConcurrency(request, - {toNullableVec({{}, 0, 0, 0, {}, 1, 0}), - toNullableVec(col_names[4], col4_sorted_asc)}); - - - /// Test "greater" function - - /// Data type: TypeString - request = buildDAGRequest({gt(col(col_names[0]), col(col_names[1])), col(col_names[4])}); - executeWithConcurrency(request, - {toNullableVec({{}, 0, 0, 0, {}, 0, 0}), - toNullableVec(col_names[4], col4_sorted_asc)}); - - request = buildDAGRequest({gt(col(col_names[1]), col(col_names[0])), col(col_names[4])}); - executeWithConcurrency(request, - {toNullableVec({{}, 1, 0, 1, {}, 1, 1}), - toNullableVec(col_names[4], col4_sorted_asc)}); - - /// Data type: TypeLong - request = buildDAGRequest({gt(col(col_names[3]), col(col_names[4])), col(col_names[4])}); - executeWithConcurrency(request, - {toNullableVec({{}, 0, 1, 1, {}, 0, 0}), - toNullableVec(col_names[4], col4_sorted_asc)}); - - request = buildDAGRequest({gt(col(col_names[4]), col(col_names[3])), col(col_names[4])}); - executeWithConcurrency(request, - {toNullableVec({{}, 1, 0, 0, {}, 0, 1}), - toNullableVec(col_names[4], col4_sorted_asc)}); - - - /// Test "and" function - - /// Data type: TypeString - request = buildDAGRequest({And(col(col_names[0]), col(col_names[0])), col(col_names[4])}); - executeWithConcurrency(request, - {toNullableVec({{}, 0, 0, 0, 0, 0, 0}), - toNullableVec(col_names[4], col4_sorted_asc)}); - - request = buildDAGRequest({And(col(col_names[0]), col(col_names[1])), col(col_names[4])}); - executeWithConcurrency(request, - {toNullableVec({0, 0, 0, 0, 0, 0, 0}), - toNullableVec(col_names[4], col4_sorted_asc)}); - - /// Data type: TypeLong - request = buildDAGRequest({And(col(col_names[3]), col(col_names[4])), col(col_names[4])}); - executeWithConcurrency(request, - {toNullableVec({{}, 1, 0, 0, {}, 1, 0}), - toNullableVec(col_names[4], col4_sorted_asc)}); - - /// Test "not" function - - /// Data type: TypeString - request = buildDAGRequest({NOT(col(col_names[0])), NOT(col(col_names[1])), NOT(col(col_names[2])), col(col_names[4])}); - executeWithConcurrency(request, - {toNullableVec({{}, 1, 1, 1, 1, 1, 1}), - toNullableVec({1, 1, 1, 1, {}, 1, 1}), - toNullableVec({1, {}, 1, 1, 1, 1, {}}), - toNullableVec(col_names[4], col4_sorted_asc)}); - - /// Data type: TypeLong - request = buildDAGRequest({NOT(col(col_names[3])), NOT(col(col_names[4])), col(col_names[4])}); - executeWithConcurrency(request, - {toNullableVec({{}, 0, 1, 0, {}, 0, 1}), - toNullableVec({{}, 0, 0, 1, 0, 0, 0}), - toNullableVec(col_names[4], col4_sorted_asc)}); - - /// TODO more functions... + wrapForDisEnablePlanner([&]() { + std::shared_ptr request; + + /// Test "equal" function + + /// Data type: TypeString + request = buildDAGRequest({eq(col(col_names[0]), col(col_names[0])), col(col_names[4])}); + executeWithConcurrency(request, + {toNullableVec({{}, 1, 1, 1, 1, 1, 1}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + request = buildDAGRequest({eq(col(col_names[0]), col(col_names[1])), col(col_names[4])}); + executeWithConcurrency(request, + {toNullableVec({{}, 0, 1, 0, {}, 0, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + /// Data type: TypeLong + request = buildDAGRequest({eq(col(col_names[3]), col(col_names[4])), col(col_names[4])}); + executeWithConcurrency(request, + {toNullableVec({{}, 0, 0, 0, {}, 1, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + + /// Test "greater" function + + /// Data type: TypeString + request = buildDAGRequest({gt(col(col_names[0]), col(col_names[1])), col(col_names[4])}); + executeWithConcurrency(request, + {toNullableVec({{}, 0, 0, 0, {}, 0, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + request = buildDAGRequest({gt(col(col_names[1]), col(col_names[0])), col(col_names[4])}); + executeWithConcurrency(request, + {toNullableVec({{}, 1, 0, 1, {}, 1, 1}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + /// Data type: TypeLong + request = buildDAGRequest({gt(col(col_names[3]), col(col_names[4])), col(col_names[4])}); + executeWithConcurrency(request, + {toNullableVec({{}, 0, 1, 1, {}, 0, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + request = buildDAGRequest({gt(col(col_names[4]), col(col_names[3])), col(col_names[4])}); + executeWithConcurrency(request, + {toNullableVec({{}, 1, 0, 0, {}, 0, 1}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + + /// Test "and" function + + /// Data type: TypeString + request = buildDAGRequest({And(col(col_names[0]), col(col_names[0])), col(col_names[4])}); + executeWithConcurrency(request, + {toNullableVec({{}, 0, 0, 0, 0, 0, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + request = buildDAGRequest({And(col(col_names[0]), col(col_names[1])), col(col_names[4])}); + executeWithConcurrency(request, + {toNullableVec({0, 0, 0, 0, 0, 0, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + /// Data type: TypeLong + request = buildDAGRequest({And(col(col_names[3]), col(col_names[4])), col(col_names[4])}); + executeWithConcurrency(request, + {toNullableVec({{}, 1, 0, 0, {}, 1, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + /// Test "not" function + + /// Data type: TypeString + request = buildDAGRequest({NOT(col(col_names[0])), NOT(col(col_names[1])), NOT(col(col_names[2])), col(col_names[4])}); + executeWithConcurrency(request, + {toNullableVec({{}, 1, 1, 1, 1, 1, 1}), + toNullableVec({1, 1, 1, 1, {}, 1, 1}), + toNullableVec({1, {}, 1, 1, 1, 1, {}}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + /// Data type: TypeLong + request = buildDAGRequest({NOT(col(col_names[3])), NOT(col(col_names[4])), col(col_names[4])}); + executeWithConcurrency(request, + {toNullableVec({{}, 0, 1, 0, {}, 0, 1}), + toNullableVec({{}, 0, 0, 1, 0, 0, 0}), + toNullableVec(col_names[4], col4_sorted_asc)}); + + /// TODO more functions... + }); } CATCH diff --git a/dbms/src/Flash/tests/gtest_topn_executor.cpp b/dbms/src/Flash/tests/gtest_topn_executor.cpp index 597ac9f279a..a6ba3183118 100644 --- a/dbms/src/Flash/tests/gtest_topn_executor.cpp +++ b/dbms/src/Flash/tests/gtest_topn_executor.cpp @@ -78,142 +78,146 @@ class ExecutorTopNTestRunner : public DB::tests::ExecutorTest TEST_F(ExecutorTopNTestRunner, TopN) try { - std::shared_ptr request; - std::vector expect_cols; + wrapForDisEnablePlanner([&]() { + std::shared_ptr request; + std::vector expect_cols; - { - /// Test single column - size_t col_data_num = col0.size(); - for (size_t i = 1; i <= 1; ++i) { - bool is_desc; - is_desc = static_cast(i); /// Set descent or ascent - if (is_desc) - sort(col0.begin(), col0.end(), std::greater()); /// Sort col0 for the following comparison - else - sort(col0.begin(), col0.end()); - - for (size_t limit_num = 0; limit_num <= col_data_num + 5; ++limit_num) + /// Test single column + size_t col_data_num = col0.size(); + for (size_t i = 1; i <= 1; ++i) { - request = buildDAGRequest(table_single_name, single_col_name, is_desc, limit_num); - - expect_cols.clear(); - if (limit_num == 0 || limit_num > col_data_num) - expect_cols.push_back({toNullableVec(single_col_name, ColumnWithString(col0.begin(), col0.end()))}); + bool is_desc; + is_desc = static_cast(i); /// Set descent or ascent + if (is_desc) + sort(col0.begin(), col0.end(), std::greater()); /// Sort col0 for the following comparison else - expect_cols.push_back({toNullableVec(single_col_name, ColumnWithString(col0.begin(), col0.begin() + limit_num))}); - - ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[0]); - ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), expect_cols[0]); - ASSERT_COLUMNS_EQ_R(executeStreams(request, 4), expect_cols[0]); - ASSERT_COLUMNS_EQ_R(executeStreams(request, 8), expect_cols[0]); + sort(col0.begin(), col0.end()); + + for (size_t limit_num = 0; limit_num <= col_data_num + 5; ++limit_num) + { + request = buildDAGRequest(table_single_name, single_col_name, is_desc, limit_num); + + expect_cols.clear(); + if (limit_num == 0 || limit_num > col_data_num) + expect_cols.push_back({toNullableVec(single_col_name, ColumnWithString(col0.begin(), col0.end()))}); + else + expect_cols.push_back({toNullableVec(single_col_name, ColumnWithString(col0.begin(), col0.begin() + limit_num))}); + + ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[0]); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 2), expect_cols[0]); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 4), expect_cols[0]); + ASSERT_COLUMNS_EQ_R(executeStreams(request, 8), expect_cols[0]); + } } } - } - { - /// Test multi-columns - expect_cols = {{toNullableVec(col_name[0], ColumnWithInt32{36, 34, 32, 27, {}, {}}), - toNullableVec(col_name[1], ColumnWithString{"female", "male", "male", "female", "male", "female"}), - toNullableVec(col_name[2], ColumnWithString{"china", "china", "usa", "usa", "china", "korea"}), - toNullableVec(col_name[3], ColumnWithInt32{900, -300, {}, 0, {}, 1300})}, - {toNullableVec(col_name[0], ColumnWithInt32{32, {}, 34, 27, 36, {}}), - toNullableVec(col_name[1], ColumnWithString{"male", "male", "male", "female", "female", "female"}), - toNullableVec(col_name[2], ColumnWithString{"usa", "china", "china", "usa", "china", "korea"}), - toNullableVec(col_name[3], ColumnWithInt32{{}, {}, -300, 0, 900, 1300})}, - {toNullableVec(col_name[0], ColumnWithInt32{34, {}, 32, 36, {}, 27}), - toNullableVec(col_name[1], ColumnWithString{"male", "male", "male", "female", "female", "female"}), - toNullableVec(col_name[2], ColumnWithString{"china", "china", "usa", "china", "korea", "usa"}), - toNullableVec(col_name[3], ColumnWithInt32{-300, {}, {}, 900, 1300, 0})}}; - - std::vector order_by_items{ - /// select * from clerk order by age DESC, gender DESC; - {MockOrderByItem(col_name[0], true), MockOrderByItem(col_name[1], true)}, - /// select * from clerk order by gender DESC, salary ASC; - {MockOrderByItem(col_name[1], true), MockOrderByItem(col_name[3], false)}, - /// select * from clerk order by gender DESC, country ASC, salary DESC; - {MockOrderByItem(col_name[1], true), MockOrderByItem(col_name[2], false), MockOrderByItem(col_name[3], true)}}; - - size_t test_num = expect_cols.size(); - - for (size_t i = 0; i < test_num; ++i) { - request = buildDAGRequest(table_name, order_by_items[i], 100); - ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[i]); + /// Test multi-columns + expect_cols = {{toNullableVec(col_name[0], ColumnWithInt32{36, 34, 32, 27, {}, {}}), + toNullableVec(col_name[1], ColumnWithString{"female", "male", "male", "female", "male", "female"}), + toNullableVec(col_name[2], ColumnWithString{"china", "china", "usa", "usa", "china", "korea"}), + toNullableVec(col_name[3], ColumnWithInt32{900, -300, {}, 0, {}, 1300})}, + {toNullableVec(col_name[0], ColumnWithInt32{32, {}, 34, 27, 36, {}}), + toNullableVec(col_name[1], ColumnWithString{"male", "male", "male", "female", "female", "female"}), + toNullableVec(col_name[2], ColumnWithString{"usa", "china", "china", "usa", "china", "korea"}), + toNullableVec(col_name[3], ColumnWithInt32{{}, {}, -300, 0, 900, 1300})}, + {toNullableVec(col_name[0], ColumnWithInt32{34, {}, 32, 36, {}, 27}), + toNullableVec(col_name[1], ColumnWithString{"male", "male", "male", "female", "female", "female"}), + toNullableVec(col_name[2], ColumnWithString{"china", "china", "usa", "china", "korea", "usa"}), + toNullableVec(col_name[3], ColumnWithInt32{-300, {}, {}, 900, 1300, 0})}}; + + std::vector order_by_items{ + /// select * from clerk order by age DESC, gender DESC; + {MockOrderByItem(col_name[0], true), MockOrderByItem(col_name[1], true)}, + /// select * from clerk order by gender DESC, salary ASC; + {MockOrderByItem(col_name[1], true), MockOrderByItem(col_name[3], false)}, + /// select * from clerk order by gender DESC, country ASC, salary DESC; + {MockOrderByItem(col_name[1], true), MockOrderByItem(col_name[2], false), MockOrderByItem(col_name[3], true)}}; + + size_t test_num = expect_cols.size(); + + for (size_t i = 0; i < test_num; ++i) + { + request = buildDAGRequest(table_name, order_by_items[i], 100); + ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[i]); + } } - } + }); } CATCH TEST_F(ExecutorTopNTestRunner, TopNFunction) try { - std::shared_ptr request; - std::vector expect_cols; - MockColumnNameVec output_projection{col_name[0], col_name[1], col_name[2], col_name[3]}; - MockAstVec func_projection; // Do function operation for topn - MockOrderByItemVec order_by_items; - ASTPtr col0_ast = col(col_name[0]); - ASTPtr col1_ast = col(col_name[1]); - ASTPtr col2_ast = col(col_name[2]); - ASTPtr col3_ast = col(col_name[3]); - ASTPtr func_ast; - - { - /// "and" function - expect_cols = {{toNullableVec(col_name[0], ColumnWithInt32{{}, {}, 32, 27, 36, 34}), - toNullableVec(col_name[1], ColumnWithString{"female", "male", "male", "female", "female", "male"}), - toNullableVec(col_name[2], ColumnWithString{"korea", "china", "usa", "usa", "china", "china"}), - toNullableVec(col_name[3], ColumnWithInt32{1300, {}, {}, 0, 900, -300})}}; + wrapForDisEnablePlanner([&]() { + std::shared_ptr request; + std::vector expect_cols; + MockColumnNameVec output_projection{col_name[0], col_name[1], col_name[2], col_name[3]}; + MockAstVec func_projection; // Do function operation for topn + MockOrderByItemVec order_by_items; + ASTPtr col0_ast = col(col_name[0]); + ASTPtr col1_ast = col(col_name[1]); + ASTPtr col2_ast = col(col_name[2]); + ASTPtr col3_ast = col(col_name[3]); + ASTPtr func_ast; { - /// select * from clerk order by age and salary ASC limit 100; - order_by_items = {MockOrderByItem("and(age, salary)", false)}; - func_ast = And(col(col_name[0]), col(col_name[3])); - func_projection = {col0_ast, col1_ast, col2_ast, col3_ast, func_ast}; + /// "and" function + expect_cols = {{toNullableVec(col_name[0], ColumnWithInt32{{}, {}, 32, 27, 36, 34}), + toNullableVec(col_name[1], ColumnWithString{"female", "male", "male", "female", "female", "male"}), + toNullableVec(col_name[2], ColumnWithString{"korea", "china", "usa", "usa", "china", "china"}), + toNullableVec(col_name[3], ColumnWithInt32{1300, {}, {}, 0, 900, -300})}}; - request = buildDAGRequest(table_name, order_by_items, 100, func_projection, output_projection); - ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[0]); - } - } + { + /// select * from clerk order by age and salary ASC limit 100; + order_by_items = {MockOrderByItem("and(age, salary)", false)}; + func_ast = And(col(col_name[0]), col(col_name[3])); + func_projection = {col0_ast, col1_ast, col2_ast, col3_ast, func_ast}; - { - /// "equal" function - expect_cols = {{toNullableVec(col_name[0], ColumnWithInt32{27, 36, 34, 32, {}, {}}), - toNullableVec(col_name[1], ColumnWithString{"female", "female", "male", "male", "female", "male"}), - toNullableVec(col_name[2], ColumnWithString{"usa", "china", "china", "usa", "korea", "china"}), - toNullableVec(col_name[3], ColumnWithInt32{0, 900, -300, {}, 1300, {}})}}; + request = buildDAGRequest(table_name, order_by_items, 100, func_projection, output_projection); + ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[0]); + } + } { - /// select age, salary from clerk order by age = salary DESC limit 100; - order_by_items = {MockOrderByItem("equals(age, salary)", true)}; - func_ast = eq(col(col_name[0]), col(col_name[3])); - func_projection = {col0_ast, col1_ast, col2_ast, col3_ast, func_ast}; + /// "equal" function + expect_cols = {{toNullableVec(col_name[0], ColumnWithInt32{27, 36, 34, 32, {}, {}}), + toNullableVec(col_name[1], ColumnWithString{"female", "female", "male", "male", "female", "male"}), + toNullableVec(col_name[2], ColumnWithString{"usa", "china", "china", "usa", "korea", "china"}), + toNullableVec(col_name[3], ColumnWithInt32{0, 900, -300, {}, 1300, {}})}}; - request = buildDAGRequest(table_name, order_by_items, 100, func_projection, output_projection); - ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[0]); - } - } + { + /// select age, salary from clerk order by age = salary DESC limit 100; + order_by_items = {MockOrderByItem("equals(age, salary)", true)}; + func_ast = eq(col(col_name[0]), col(col_name[3])); + func_projection = {col0_ast, col1_ast, col2_ast, col3_ast, func_ast}; - { - /// "greater" function - expect_cols = {{toNullableVec(col_name[0], ColumnWithInt32{{}, 32, {}, 36, 27, 34}), - toNullableVec(col_name[1], ColumnWithString{"female", "male", "male", "female", "female", "male"}), - toNullableVec(col_name[2], ColumnWithString{"korea", "usa", "china", "china", "usa", "china"}), - toNullableVec(col_name[3], ColumnWithInt32{1300, {}, {}, 900, 0, -300})}}; + request = buildDAGRequest(table_name, order_by_items, 100, func_projection, output_projection); + ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[0]); + } + } { - /// select age, gender, country, salary from clerk order by age > salary ASC limit 100; - order_by_items = {MockOrderByItem("greater(age, salary)", false)}; - func_ast = gt(col(col_name[0]), col(col_name[3])); - func_projection = {col0_ast, col1_ast, col2_ast, col3_ast, func_ast}; + /// "greater" function + expect_cols = {{toNullableVec(col_name[0], ColumnWithInt32{{}, 32, {}, 36, 27, 34}), + toNullableVec(col_name[1], ColumnWithString{"female", "male", "male", "female", "female", "male"}), + toNullableVec(col_name[2], ColumnWithString{"korea", "usa", "china", "china", "usa", "china"}), + toNullableVec(col_name[3], ColumnWithInt32{1300, {}, {}, 900, 0, -300})}}; + + { + /// select age, gender, country, salary from clerk order by age > salary ASC limit 100; + order_by_items = {MockOrderByItem("greater(age, salary)", false)}; + func_ast = gt(col(col_name[0]), col(col_name[3])); + func_projection = {col0_ast, col1_ast, col2_ast, col3_ast, func_ast}; - request = buildDAGRequest(table_name, order_by_items, 100, func_projection, output_projection); - ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[0]); + request = buildDAGRequest(table_name, order_by_items, 100, func_projection, output_projection); + ASSERT_COLUMNS_EQ_R(executeStreams(request), expect_cols[0]); + } } - } - /// TODO more functions... + /// TODO more functions... + }); } CATCH diff --git a/dbms/src/Functions/tests/gtest_strings_format.cpp b/dbms/src/Functions/tests/gtest_strings_format.cpp index 72e659e9ec8..8f3b899316e 100644 --- a/dbms/src/Functions/tests/gtest_strings_format.cpp +++ b/dbms/src/Functions/tests/gtest_strings_format.cpp @@ -65,8 +65,6 @@ class StringFormat : public DB::tests::FunctionTest FieldType(static_cast(-9999999), 4), FieldType(static_cast(-3333330), 4)}), createConstColumn>(4, 3))); - /// known issue https://github.com/pingcap/tiflash/issues/4891 - /* ASSERT_COLUMN_EQ( createColumn>({"-999.9999", "-1,000", "-1,000", "-999.999900000000000000000000000000", "-999.99990", "-1,000.0", "-1,000.00"}), executeFunction( @@ -85,7 +83,6 @@ class StringFormat : public DB::tests::FunctionTest 1, FieldType(static_cast(-9999999), 4)), createConstColumn>(1, 3))); - */ ASSERT_COLUMN_EQ( createColumn>({"12,332.1000", "12,332", "12,332.300000000000000000000000000000", "-12,332.30000", "-1,000.0", "-333.33", {}}), executeFunction( @@ -111,8 +108,6 @@ class StringFormat : public DB::tests::FunctionTest FieldType(static_cast(-9999999), 4), FieldType(static_cast(-3333330), 4)}), createConstColumn>(4, 3))); - /// known issue https://github.com/pingcap/tiflash/issues/4891 - /* ASSERT_COLUMN_EQ( createColumn>({"-999.9999", "-1,000", "-999.999900000000000000000000000000", "-999.99990", "-1,000.0", "-1,000.00"}), executeFunction( @@ -131,7 +126,6 @@ class StringFormat : public DB::tests::FunctionTest 1, FieldType(static_cast(-9999999), 4)), createConstColumn>(1, 3))); - */ } template diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 78a44eaf278..21366c336a8 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -361,10 +361,12 @@ struct Settings M(SettingUInt64, async_pollers_per_cq, 200, "grpc async pollers per cqs") \ M(SettingUInt64, async_cqs, 1, "grpc async cqs") \ M(SettingUInt64, preallocated_request_count_per_poller, 20, "grpc preallocated_request_count_per_poller") \ + \ M(SettingUInt64, manual_compact_pool_size, 1, "The number of worker threads to handle manual compact requests.") \ M(SettingUInt64, manual_compact_max_concurrency, 10, "Max concurrent tasks. It should be larger than pool size.") \ M(SettingUInt64, manual_compact_more_until_ms, 60000, "Continuously compact more segments until reaching specified elapsed time. If 0 is specified, only one segment will be compacted each round.") \ M(SettingBool, enable_planner, true, "Enable planner") + // clang-format on #define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) TYPE NAME{DEFAULT}; diff --git a/dbms/src/Server/main.cpp b/dbms/src/Server/main.cpp index bd09f9264c2..dbcaa4f38fc 100644 --- a/dbms/src/Server/main.cpp +++ b/dbms/src/Server/main.cpp @@ -44,9 +44,6 @@ #if ENABLE_TIFLASH_PAGECTL #include #endif -#if ENABLE_TIFLASH_PAGECTL -#include -#endif #include #include diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index 634e483abd2..763c74b45b6 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -103,6 +103,7 @@ Block mergeBlocks(Blocks blocks) actual_columns.push_back({std::move(actual_cols[i]), sample_block.getColumnsWithTypeAndName()[i].type, sample_block.getColumnsWithTypeAndName()[i].name, sample_block.getColumnsWithTypeAndName()[i].column_id}); return Block(actual_columns); } +} // namespace DB::ColumnsWithTypeAndName readBlock(BlockInputStreamPtr stream) { @@ -115,7 +116,11 @@ DB::ColumnsWithTypeAndName readBlock(BlockInputStreamPtr stream) stream->readSuffix(); return mergeBlocks(actual_blocks).getColumnsWithTypeAndName(); } -} // namespace + +void ExecutorTest::enablePlanner(bool is_enable) +{ + context.context.setSetting("enable_planner", is_enable ? "true" : "false"); +} DB::ColumnsWithTypeAndName ExecutorTest::executeStreams(const std::shared_ptr & request, std::unordered_map & source_columns_map, size_t concurrency) { diff --git a/dbms/src/TestUtils/ExecutorTestUtils.h b/dbms/src/TestUtils/ExecutorTestUtils.h index 59b829e04b5..54fed31f88d 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.h +++ b/dbms/src/TestUtils/ExecutorTestUtils.h @@ -28,6 +28,8 @@ void executeInterpreter(const std::shared_ptr & request, Conte ::testing::AssertionResult check_columns_equality(const ColumnsWithTypeAndName & expected, const ColumnsWithTypeAndName & actual, bool _restrict); +DB::ColumnsWithTypeAndName readBlock(BlockInputStreamPtr stream); + class ExecutorTest : public ::testing::Test { protected: @@ -49,6 +51,17 @@ class ExecutorTest : public ::testing::Test DAGContext & getDAGContext(); + /// for planner + void enablePlanner(bool is_enable); + template + void wrapForDisEnablePlanner(FF && ff) + { + enablePlanner(false); + ff(); + enablePlanner(true); + ff(); + } + static void dagRequestEqual(const String & expected_string, const std::shared_ptr & actual); void executeInterpreter(const String & expected_string, const std::shared_ptr & request, size_t concurrency); diff --git a/dbms/src/TestUtils/mockExecutor.cpp b/dbms/src/TestUtils/mockExecutor.cpp index 0595b7562f7..30d05786c9a 100644 --- a/dbms/src/TestUtils/mockExecutor.cpp +++ b/dbms/src/TestUtils/mockExecutor.cpp @@ -23,8 +23,6 @@ #include #include -#include - namespace DB::tests { ASTPtr buildColumn(const String & column_name) @@ -358,57 +356,6 @@ void MockDAGRequestContext::addExchangeReceiver(const String & name, MockColumnI addExchangeReceiverColumnData(name, columns); } -void MockDAGRequestContext::addMockTableColumnData(const String & db, const String & table, ColumnsWithTypeAndName columns) -{ - mock_table_columns[db + "." + table] = columns; -} - -void MockDAGRequestContext::addMockTableColumnData(const MockTableName & name, ColumnsWithTypeAndName columns) -{ - mock_table_columns[name.first + "." + name.second] = columns; -} - -void MockDAGRequestContext::addExchangeReceiverColumnData(const String & name, ColumnsWithTypeAndName columns) -{ - mock_exchange_columns[name] = columns; -} - -void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfoList & columnInfos, ColumnsWithTypeAndName columns) -{ - addMockTable(db, table, columnInfos); - addMockTableColumnData(db, table, columns); -} - -void MockDAGRequestContext::addMockTable(const String & db, const String & table, const MockColumnInfos & columnInfos, ColumnsWithTypeAndName columns) -{ - addMockTable(db, table, columnInfos); - addMockTableColumnData(db, table, columns); -} - -void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfoList & columnInfos, ColumnsWithTypeAndName columns) -{ - addMockTable(name, columnInfos); - addMockTableColumnData(name, columns); -} - -void MockDAGRequestContext::addMockTable(const MockTableName & name, const MockColumnInfos & columnInfos, ColumnsWithTypeAndName columns) -{ - addMockTable(name, columnInfos); - addMockTableColumnData(name, columns); -} - -void MockDAGRequestContext::addExchangeReceiver(const String & name, MockColumnInfos columnInfos, ColumnsWithTypeAndName columns) -{ - addExchangeRelationSchema(name, columnInfos); - addExchangeReceiverColumnData(name, columns); -} - -void MockDAGRequestContext::addExchangeReceiver(const String & name, MockColumnInfoList columnInfos, ColumnsWithTypeAndName columns) -{ - addExchangeRelationSchema(name, columnInfos); - addExchangeReceiverColumnData(name, columns); -} - DAGRequestBuilder MockDAGRequestContext::scan(String db_name, String table_name) { auto builder = DAGRequestBuilder(index).mockTable({db_name, table_name}, mock_tables[db_name + "." + table_name]); diff --git a/dbms/src/TiDB/Schema/SchemaBuilder.h b/dbms/src/TiDB/Schema/SchemaBuilder.h index 8446765f74a..827203a682f 100644 --- a/dbms/src/TiDB/Schema/SchemaBuilder.h +++ b/dbms/src/TiDB/Schema/SchemaBuilder.h @@ -55,39 +55,44 @@ struct SchemaBuilder bool applyCreateSchema(DatabaseID schema_id); - void applyCreateSchema(TiDB::DBInfoPtr db_info); + void applyCreateSchema(const TiDB::DBInfoPtr & db_info); - void applyCreateTable(TiDB::DBInfoPtr db_info, TableID table_id); + void applyCreateTable(const TiDB::DBInfoPtr & db_info, TableID table_id); - void applyCreateLogicalTable(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info); + void applyCreateLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info); - void applyCreatePhysicalTable(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info); + void applyCreatePhysicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info); - void applyDropTable(TiDB::DBInfoPtr db_info, TableID table_id); + void applyDropTable(const TiDB::DBInfoPtr & db_info, TableID table_id); /// Parameter schema_name should be mapped. void applyDropPhysicalTable(const String & db_name, TableID table_id); - void applyPartitionDiff(TiDB::DBInfoPtr db_info, TableID table_id); + void applyPartitionDiff(const TiDB::DBInfoPtr & db_info, TableID table_id); - void applyPartitionDiff(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info, ManageableStoragePtr storage); + void applyPartitionDiff(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); - void applyAlterTable(TiDB::DBInfoPtr db_info, TableID table_id); + void applyAlterTable(const TiDB::DBInfoPtr & db_info, TableID table_id); - void applyAlterLogicalTable(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info, ManageableStoragePtr storage); + void applyAlterLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); - void applyAlterPhysicalTable(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info, ManageableStoragePtr storage); + void applyAlterPhysicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); - void applyRenameTable(TiDB::DBInfoPtr new_db_info, TiDB::TableID table_id); + void applyRenameTable(const TiDB::DBInfoPtr & new_db_info, TiDB::TableID table_id); - void applyRenameLogicalTable(TiDB::DBInfoPtr new_db_info, TiDB::TableInfoPtr new_table_info, ManageableStoragePtr storage); + void applyRenameLogicalTable(const TiDB::DBInfoPtr & new_db_info, const TiDB::TableInfoPtr & new_table_info, const ManageableStoragePtr & storage); - void applyRenamePhysicalTable(TiDB::DBInfoPtr new_db_info, TiDB::TableInfo & new_table_info, ManageableStoragePtr storage); + void applyRenamePhysicalTable(const TiDB::DBInfoPtr & new_db_info, const TiDB::TableInfo & new_table_info, const ManageableStoragePtr & storage); void applyExchangeTablePartition(const SchemaDiff & diff); - void applySetTiFlashReplica(TiDB::DBInfoPtr db_info, TableID table_id); - void applySetTiFlashReplica(TiDB::DBInfoPtr db_info, TiDB::TableInfoPtr table_info, ManageableStoragePtr storage); + void applySetTiFlashReplica(const TiDB::DBInfoPtr & db_info, TableID table_id); + void applySetTiFlashReplicaOnLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); + void applySetTiFlashReplicaOnPhysicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); + + void applySetTiFlashMode(const TiDB::DBInfoPtr & db_info, TableID table_id); + void applySetTiFlashModeOnLogicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); + void applySetTiFlashModeOnPhysicalTable(const TiDB::DBInfoPtr & db_info, const TiDB::TableInfoPtr & table_info, const ManageableStoragePtr & storage); }; } // namespace DB