From 1d5b62cc18678ae2287f69ad608d927b9fd9fbab Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 3 Jan 2021 13:24:09 +0300 Subject: [PATCH 001/200] add simple optimization --- src/CMakeLists.txt | 2 +- src/Core/Settings.h | 1 + src/Interpreters/ConstraintMatcherVisitor.cpp | 1 + src/Interpreters/ConstraintMatcherVisitor.h | 48 +++++++++++++++++++ src/Interpreters/TreeOptimizer.cpp | 21 ++++++++ 5 files changed, 72 insertions(+), 1 deletion(-) create mode 100644 src/Interpreters/ConstraintMatcherVisitor.cpp create mode 100644 src/Interpreters/ConstraintMatcherVisitor.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 4e04f5607df..53232e742a7 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -63,7 +63,7 @@ add_subdirectory (Server) set(dbms_headers) -set(dbms_sources) +set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h) add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index b09e960da36..eea95a8b45a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -378,6 +378,7 @@ class IColumn; M(Bool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \ M(Bool, optimize_if_transform_strings_to_enum, false, "Replaces string-type arguments in If and Transform to enum. Disabled by default cause it could make inconsistent change in distributed query that would lead to its fail.", 0) \ M(Bool, optimize_monotonous_functions_in_order_by, true, "Replace monotonous function with its argument in ORDER BY", 0) \ + M(Bool, optimize_using_constraints, true, "Use constraints for query optimization", 0) \ M(Bool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \ M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \ M(Bool, deduplicate_blocks_in_dependent_materialized_views, false, "Should deduplicate blocks for materialized views if the block is not a duplicate for the table. Use true to always deduplicate in dependent tables.", 0) \ diff --git a/src/Interpreters/ConstraintMatcherVisitor.cpp b/src/Interpreters/ConstraintMatcherVisitor.cpp new file mode 100644 index 00000000000..61b1c436b3a --- /dev/null +++ b/src/Interpreters/ConstraintMatcherVisitor.cpp @@ -0,0 +1 @@ +#include "ConstraintMatcherVisitor.h" diff --git a/src/Interpreters/ConstraintMatcherVisitor.h b/src/Interpreters/ConstraintMatcherVisitor.h new file mode 100644 index 00000000000..246afcc91bd --- /dev/null +++ b/src/Interpreters/ConstraintMatcherVisitor.h @@ -0,0 +1,48 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +/// recursive traversal and check for optimizeGroupByFunctionKeys +struct ConstraintMatcher +{ + struct Data + { + std::unordered_map> constraints; + }; + + using Visitor = InDepthNodeVisitor; + + static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { return (node->as()) && (node->as()); } + + static bool alwaysTrue(const ASTPtr & node, Data & data) { + const auto it = data.constraints.find(node->getTreeHash().second); + if (it != std::end(data.constraints)) { + for (const auto & ast : it->second) { + if (node->getColumnName() == ast->getColumnName()) { + return true; + } + } + } + return false; + } + + static void visit(ASTPtr & ast, Data & data) + { + if (alwaysTrue(ast, data)) { + ast = std::make_shared(1); + } + } +}; + +using ConstraintMatcherVisitor = InDepthNodeVisitor; + +} diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index cee19c632fa..504ead7465a 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -506,6 +507,20 @@ void optimizeLimitBy(const ASTSelectQuery * select_query) elems = std::move(unique_elems); } +void optimizeWithConstraints(ASTPtr & query, const NameSet & /* source_columns_set */, + const std::vector & /* tables_with_columns */, + const StorageMetadataPtr & metadata_snapshot) +{ + ConstraintMatcherVisitor::Data constraint_data; + + for (const auto & constraint : metadata_snapshot->getConstraints().constraints) + { + constraint_data.constraints[constraint->getTreeHash().second].push_back(constraint); + } + + ConstraintMatcherVisitor(constraint_data).visit(query); +} + /// Remove duplicated columns from USING(...). void optimizeUsing(const ASTSelectQuery * select_query) { @@ -597,6 +612,12 @@ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & sou /// Push the predicate expression down to the subqueries. rewrite_subqueries = PredicateExpressionsOptimizer(context, tables_with_columns, settings).optimize(*select_query); + if (settings.optimize_using_constraints) + { + optimizeWithConstraints(select_query->refWhere(), source_columns_set, tables_with_columns, metadata_snapshot); + optimizeWithConstraints(select_query->refPrewhere(), source_columns_set, tables_with_columns, metadata_snapshot); + } + /// GROUP BY injective function elimination. optimizeGroupBy(select_query, source_columns_set, context); From 4c3e83ac57a9b0ae9a6d2c396be540982ac01beb Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 3 Jan 2021 18:02:00 +0300 Subject: [PATCH 002/200] fix --- .../CheckConstraintsBlockOutputStream.cpp | 7 ++--- .../CheckConstraintsBlockOutputStream.h | 2 +- src/Interpreters/ConstraintMatcherVisitor.h | 8 +++--- src/Interpreters/TreeOptimizer.cpp | 13 +++++---- src/Parsers/ASTConstraintDeclaration.cpp | 3 ++- src/Parsers/ASTConstraintDeclaration.h | 6 +++++ src/Parsers/ParserCreateQuery.cpp | 10 ++++++- src/Storages/ConstraintsDescription.cpp | 27 +++++++++++++++---- src/Storages/ConstraintsDescription.h | 9 ++++++- ..._constraints_simple_optimization.reference | 4 +++ .../01622_constraints_simple_optimization.sql | 21 +++++++++++++++ 11 files changed, 87 insertions(+), 23 deletions(-) create mode 100644 tests/queries/0_stateless/01622_constraints_simple_optimization.reference create mode 100644 tests/queries/0_stateless/01622_constraints_simple_optimization.sql diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index a967ee28502..c13402c7646 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -28,8 +28,8 @@ CheckConstraintsBlockOutputStream::CheckConstraintsBlockOutputStream( : table_id(table_id_), output(output_), header(header_), - constraints(constraints_), - expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList())) + constraints_to_check(constraints_.filterConstraints(ConstraintsDescription::ConstraintType::CHECK)), + expressions(constraints_.getExpressionsToCheck(context_, header.getNamesAndTypesList())) { } @@ -38,13 +38,14 @@ void CheckConstraintsBlockOutputStream::write(const Block & block) { if (block.rows() > 0) { + Block block_to_calculate = block; for (size_t i = 0; i < expressions.size(); ++i) { auto constraint_expr = expressions[i]; constraint_expr->execute(block_to_calculate); - auto * constraint_ptr = constraints.constraints[i]->as(); + auto * constraint_ptr = constraints_to_check[i]->as(); ColumnWithTypeAndName res_column = block_to_calculate.getByName(constraint_ptr->expr->getColumnName()); diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.h b/src/DataStreams/CheckConstraintsBlockOutputStream.h index a1be720f21e..9317d03b742 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.h +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.h @@ -34,7 +34,7 @@ private: StorageID table_id; BlockOutputStreamPtr output; Block header; - const ConstraintsDescription constraints; + const ASTs constraints_to_check; const ConstraintsExpressions expressions; size_t rows_written = 0; }; diff --git a/src/Interpreters/ConstraintMatcherVisitor.h b/src/Interpreters/ConstraintMatcherVisitor.h index 246afcc91bd..39f99f93ef1 100644 --- a/src/Interpreters/ConstraintMatcherVisitor.h +++ b/src/Interpreters/ConstraintMatcherVisitor.h @@ -7,11 +7,11 @@ #include #include #include -#include +#include namespace DB { -/// recursive traversal and check for optimizeGroupByFunctionKeys + struct ConstraintMatcher { struct Data @@ -21,7 +21,7 @@ struct ConstraintMatcher using Visitor = InDepthNodeVisitor; - static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { return (node->as()) && (node->as()); } + static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { return node->as() || node->as(); } static bool alwaysTrue(const ASTPtr & node, Data & data) { const auto it = data.constraints.find(node->getTreeHash().second); @@ -38,7 +38,7 @@ struct ConstraintMatcher static void visit(ASTPtr & ast, Data & data) { if (alwaysTrue(ast, data)) { - ast = std::make_shared(1); + ast = std::make_shared(static_cast(1)); } } }; diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 504ead7465a..0d1fdd69c95 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -507,7 +507,7 @@ void optimizeLimitBy(const ASTSelectQuery * select_query) elems = std::move(unique_elems); } -void optimizeWithConstraints(ASTPtr & query, const NameSet & /* source_columns_set */, +void optimizeWithConstraints(ASTSelectQuery * select_query, const NameSet & /* source_columns_set */, const std::vector & /* tables_with_columns */, const StorageMetadataPtr & metadata_snapshot) { @@ -515,10 +515,12 @@ void optimizeWithConstraints(ASTPtr & query, const NameSet & /* source_columns_s for (const auto & constraint : metadata_snapshot->getConstraints().constraints) { - constraint_data.constraints[constraint->getTreeHash().second].push_back(constraint); + const auto expr = constraint->as()->expr->clone(); + constraint_data.constraints[expr->getTreeHash().second].push_back(expr); } - ConstraintMatcherVisitor(constraint_data).visit(query); + if (select_query->where()) + ConstraintMatcherVisitor(constraint_data).visit(select_query->refWhere()); } /// Remove duplicated columns from USING(...). @@ -613,10 +615,7 @@ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & sou rewrite_subqueries = PredicateExpressionsOptimizer(context, tables_with_columns, settings).optimize(*select_query); if (settings.optimize_using_constraints) - { - optimizeWithConstraints(select_query->refWhere(), source_columns_set, tables_with_columns, metadata_snapshot); - optimizeWithConstraints(select_query->refPrewhere(), source_columns_set, tables_with_columns, metadata_snapshot); - } + optimizeWithConstraints(select_query, source_columns_set, tables_with_columns, metadata_snapshot); /// GROUP BY injective function elimination. optimizeGroupBy(select_query, source_columns_set, context); diff --git a/src/Parsers/ASTConstraintDeclaration.cpp b/src/Parsers/ASTConstraintDeclaration.cpp index 7d74837478c..2b895b85996 100644 --- a/src/Parsers/ASTConstraintDeclaration.cpp +++ b/src/Parsers/ASTConstraintDeclaration.cpp @@ -11,6 +11,7 @@ ASTPtr ASTConstraintDeclaration::clone() const auto res = std::make_shared(); res->name = name; + res->type = type; if (expr) res->set(res->expr, expr->clone()); @@ -21,7 +22,7 @@ ASTPtr ASTConstraintDeclaration::clone() const void ASTConstraintDeclaration::formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const { s.ostr << backQuoteIfNeed(name); - s.ostr << (s.hilite ? hilite_keyword : "") << " CHECK " << (s.hilite ? hilite_none : ""); + s.ostr << (s.hilite ? hilite_keyword : "") << (type == Type::CHECK ? " CHECK " : " ASSUME ") << (s.hilite ? hilite_none : ""); expr->formatImpl(s, state, frame); } diff --git a/src/Parsers/ASTConstraintDeclaration.h b/src/Parsers/ASTConstraintDeclaration.h index 3a8ad75f54b..bdba7bd577a 100644 --- a/src/Parsers/ASTConstraintDeclaration.h +++ b/src/Parsers/ASTConstraintDeclaration.h @@ -10,7 +10,13 @@ namespace DB class ASTConstraintDeclaration : public IAST { public: + enum class Type { + CHECK, + ASSUME, + }; + String name; + Type type; IAST * expr; String getID(char) const override { return "Constraint"; } diff --git a/src/Parsers/ParserCreateQuery.cpp b/src/Parsers/ParserCreateQuery.cpp index c77ddc5f180..1392f27e876 100644 --- a/src/Parsers/ParserCreateQuery.cpp +++ b/src/Parsers/ParserCreateQuery.cpp @@ -127,24 +127,32 @@ bool ParserIndexDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expe bool ParserConstraintDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { ParserKeyword s_check("CHECK"); + ParserKeyword s_assume("ASSUME"); ParserIdentifier name_p; ParserLogicalOrExpression expression_p; ASTPtr name; ASTPtr expr; + ASTConstraintDeclaration::Type type = ASTConstraintDeclaration::Type::CHECK; if (!name_p.parse(pos, name, expected)) return false; if (!s_check.ignore(pos, expected)) - return false; + { + if (s_assume.ignore(pos, expected)) + type = ASTConstraintDeclaration::Type::ASSUME; + else + return false; + } if (!expression_p.parse(pos, expr, expected)) return false; auto constraint = std::make_shared(); constraint->name = name->as().name(); + constraint->type = type; constraint->set(constraint->expr, expr); node = constraint; diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index e6315872a66..08612cd240f 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -41,18 +41,35 @@ ConstraintsDescription ConstraintsDescription::parse(const String & str) return res; } -ConstraintsExpressions ConstraintsDescription::getExpressions(const DB::Context & context, +ASTs ConstraintsDescription::filterConstraints(ConstraintType type) const +{ + auto constraint_type = (type == ConstraintType::CHECK ? ASTConstraintDeclaration::Type::CHECK : ASTConstraintDeclaration::Type::ASSUME); + ASTs res; + res.reserve(constraints.size()); + for (const auto & constraint : constraints) + { + if (constraint->as()->type == constraint_type) { + res.push_back(constraint); + } + } + return res; +} + +ConstraintsExpressions ConstraintsDescription::getExpressionsToCheck(const DB::Context & context, const DB::NamesAndTypesList & source_columns_) const { ConstraintsExpressions res; res.reserve(constraints.size()); for (const auto & constraint : constraints) { - // TreeRewriter::analyze has query as non-const argument so to avoid accidental query changes we clone it auto * constraint_ptr = constraint->as(); - ASTPtr expr = constraint_ptr->expr->clone(); - auto syntax_result = TreeRewriter(context).analyze(expr, source_columns_); - res.push_back(ExpressionAnalyzer(constraint_ptr->expr->clone(), syntax_result, context).getActions(false)); + if (constraint_ptr->type == ASTConstraintDeclaration::Type::CHECK) + { + // TreeRewriter::analyze has query as non-const argument so to avoid accidental query changes we clone it + ASTPtr expr = constraint_ptr->expr->clone(); + auto syntax_result = TreeRewriter(context).analyze(expr, source_columns_); + res.push_back(ExpressionAnalyzer(constraint_ptr->expr->clone(), syntax_result, context).getActions(false)); + } } return res; } diff --git a/src/Storages/ConstraintsDescription.h b/src/Storages/ConstraintsDescription.h index d6d2baefbd2..414392e0662 100644 --- a/src/Storages/ConstraintsDescription.h +++ b/src/Storages/ConstraintsDescription.h @@ -10,6 +10,11 @@ using ConstraintsExpressions = std::vector; struct ConstraintsDescription { + enum class ConstraintType { + CHECK, + ASSUME, + }; + std::vector constraints; ConstraintsDescription() = default; @@ -19,7 +24,9 @@ struct ConstraintsDescription static ConstraintsDescription parse(const String & str); - ConstraintsExpressions getExpressions(const Context & context, const NamesAndTypesList & source_columns_) const; + ASTs filterConstraints(ConstraintType type) const; + + ConstraintsExpressions getExpressionsToCheck(const Context & context, const NamesAndTypesList & source_columns_) const; ConstraintsDescription(const ConstraintsDescription & other); ConstraintsDescription & operator=(const ConstraintsDescription & other); diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference new file mode 100644 index 00000000000..e55c7091740 --- /dev/null +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -0,0 +1,4 @@ +4 +0 +3 +0 diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql new file mode 100644 index 00000000000..cedae1b0237 --- /dev/null +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -0,0 +1,21 @@ +DROP DATABASE IF EXISTS constraint_test; +DROP TABLE IF EXISTS constraint_test.constrained; + +SET optimize_using_constraints = 1; + +CREATE DATABASE constraint_test; +CREATE TABLE constraint_test.assumption (URL String, CONSTRAINT is_yandex ASSUME domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = TinyLog; + +--- Add wrong rows in order to check optimization +INSERT INTO constraint_test.assumption (URL) VALUES ('1'); +INSERT INTO constraint_test.assumption (URL) VALUES ('2'); +INSERT INTO constraint_test.assumption (URL) VALUES ('yandex.ru'); +INSERT INTO constraint_test.assumption (URL) VALUES ('3'); + +SELECT count() FROM constraint_test.assumption WHERE domainWithoutWWW(URL) = 'yandex.ru'; --- assumption -> 4 +SELECT count() FROM constraint_test.assumption WHERE NOT (domainWithoutWWW(URL) = 'yandex.ru'); --- assumption -> 0 +SELECT count() FROM constraint_test.assumption WHERE domainWithoutWWW(URL) != 'yandex.ru'; --- not optimized -> 3 +SELECT count() FROM constraint_test.assumption WHERE domainWithoutWWW(URL) = 'nothing'; --- not optimized -> 0 + +DROP TABLE constraint_test.assumption; +DROP DATABASE constraint_test; \ No newline at end of file From 92e4f1370e405f231a1bce2f3871cd797f07b27f Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Mon, 4 Jan 2021 23:55:32 +0300 Subject: [PATCH 003/200] CNF --- src/CMakeLists.txt | 2 +- src/Core/Settings.h | 1 + src/Interpreters/ConstraintMatcherVisitor.h | 9 +- src/Interpreters/TreeCNFConverter.cpp | 238 ++++++++++++++++++ src/Interpreters/TreeCNFConverter.h | 64 +++++ src/Interpreters/TreeOptimizer.cpp | 34 +-- .../WhereConstraintsOptimizer.cpp | 49 ++++ src/Interpreters/WhereConstraintsOptimizer.h | 34 +++ src/Storages/ConstraintsDescription.cpp | 16 +- src/Storages/ConstraintsDescription.h | 14 +- 10 files changed, 432 insertions(+), 29 deletions(-) create mode 100644 src/Interpreters/TreeCNFConverter.cpp create mode 100644 src/Interpreters/TreeCNFConverter.h create mode 100644 src/Interpreters/WhereConstraintsOptimizer.cpp create mode 100644 src/Interpreters/WhereConstraintsOptimizer.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 53232e742a7..d178d2356c6 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -63,7 +63,7 @@ add_subdirectory (Server) set(dbms_headers) -set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h) +set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h Interpreters/WhereConstraintsOptimizer.cpp Interpreters/WhereConstraintsOptimizer.h Interpreters/TreeCNFConverter.cpp Interpreters/TreeCNFConverter.h) add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index eea95a8b45a..d610d4e122a 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -372,6 +372,7 @@ class IColumn; M(UInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \ M(Bool, optimize_move_functions_out_of_any, true, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ + M(Bool, convert_query_to_cnf, true, "Convert SELECT query to CNF", 0) \ M(Bool, optimize_arithmetic_operations_in_aggregate_functions, true, "Move arithmetic operations out of aggregation functions", 0) \ M(Bool, optimize_duplicate_order_by_and_distinct, true, "Remove duplicate ORDER BY and DISTINCT if it's possible", 0) \ M(Bool, optimize_redundant_functions_in_order_by, true, "Remove functions from ORDER BY if its argument is also in ORDER BY", 0) \ diff --git a/src/Interpreters/ConstraintMatcherVisitor.h b/src/Interpreters/ConstraintMatcherVisitor.h index 39f99f93ef1..24f3e7037e5 100644 --- a/src/Interpreters/ConstraintMatcherVisitor.h +++ b/src/Interpreters/ConstraintMatcherVisitor.h @@ -23,7 +23,7 @@ struct ConstraintMatcher static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { return node->as() || node->as(); } - static bool alwaysTrue(const ASTPtr & node, Data & data) { + static std::optional getASTValue(const ASTPtr & node, Data & data) { const auto it = data.constraints.find(node->getTreeHash().second); if (it != std::end(data.constraints)) { for (const auto & ast : it->second) { @@ -32,14 +32,13 @@ struct ConstraintMatcher } } } - return false; + return std::nullopt; } static void visit(ASTPtr & ast, Data & data) { - if (alwaysTrue(ast, data)) { - ast = std::make_shared(static_cast(1)); - } + if (const auto always_value = getASTValue(ast, data); always_value) + ast = std::make_shared(static_cast(*always_value)); } }; diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp new file mode 100644 index 00000000000..c1814444435 --- /dev/null +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -0,0 +1,238 @@ +#include +#include +#include + +#include + +namespace DB +{ + +/// Splits AND(a, b, c) to AND(a, AND(b, c)) for AND/OR +void splitMultiLogic(ASTPtr & node) +{ + auto * func = node->as(); + + if (func && (func->name == "and" || func->name == "or")) + { + if (func->arguments->children.size() > 2) + { + ASTPtr res = func->arguments->children.front(); + for (size_t i = 1; i < func->arguments->children.size(); ++i) + { + res = makeASTFunction(func->name, res, func->arguments->children[i]); + } + node = std::move(res); + } + + auto * new_func = node->as(); + for (auto & child : new_func->arguments->children) + splitMultiLogic(child); + } +} + +/// Push NOT to leafs +void traversePushNot(ASTPtr & node, bool add_negation) +{ + auto * func = node->as(); + + if (func && (func->name == "and" || func->name == "or")) + { + if (add_negation) { + /// apply De Morgan's Law + node = makeASTFunction( + (func->name == "and" ? "or" : "and"), + func->arguments->children[0], + func->arguments->children[1]); + } + + auto * new_func = node->as(); + for (auto & child : new_func->arguments->children) + traversePushNot(child, add_negation); + } + else if (func && func->name == "not") + { + /// delete NOT + node = func->arguments->children[0]; + + traversePushNot(node, !add_negation); + } + else + { + if (add_negation) + node = makeASTFunction("not", node); + } +} + +void findOrs(ASTPtr & node, std::vector> & ors) +{ + auto * func = node->as(); + + if (func && func->name == "or") + ors.push_back(node); + + if (func) + { + for (auto & child : func->arguments->children) + findOrs(child, ors); + } +} + +/// Push Or inside And (actually pull AND to top) +void pushOr(ASTPtr & query) +{ + std::vector> ors; + findOrs(query, ors); + + while (!ors.empty()) + { + std::reference_wrapper or_node = ors.back(); + ors.pop_back(); + + auto * or_func = or_node.get()->as(); + ASSERT(or_func) + ASSERT(or_func->name == "or") + + /// find or upper than and + size_t and_node_id = or_func->arguments->children.size(); + for (size_t i = 0; i < or_func->arguments->children.size(); ++i) + { + auto & child = or_func->arguments->children[i]; + auto * and_func = child->as(); + if (and_func->name == "and") + { + and_node_id = i; + } + } + if (and_node_id == or_func->arguments->children.size()) + continue; + const size_t other_node_id = 1 - and_node_id; + + auto and_func = or_func->arguments->children[and_node_id]->as(); + ASSERT(and_func) + ASSERT(and_func->name == "and") + + auto a = or_func->arguments->children[other_node_id]; + auto b = and_func->arguments->children[0]; + auto c = and_func->arguments->children[1]; + + /// apply the distributive law ( a or (b and c) -> (a or b) and (a or c) ) + or_node.get() = makeASTFunction( + "and", + makeASTFunction("or", a, b), + makeASTFunction("or", a, c)); + + /// add new ors to stack + auto * new_func = or_node.get()->as(); + for (auto & new_or : new_func->arguments->children) + ors.push_back(new_or); + } +} + +/// transform ast into cnf groups +void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & and_group, CNFQuery::OrGroup & or_group) +{ + auto * func = node->as(); + if (func && func->name == "and") + { + for (auto & child : func->arguments->children) + { + CNFQuery::OrGroup group; + traverseCNF(child, and_group, group); + if (!group.empty()) + and_group.insert(std::move(group)); + } + } + else if (func && func->name == "or") + { + for (auto & child : func->arguments->children) + { + traverseCNF(child, and_group, or_group); + } + } + else + { + or_group.insert(node); + } +} + +void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & result) +{ + CNFQuery::OrGroup or_group; + traverseCNF(node, result, or_group); + if (!or_group.empty()) + result.insert(or_group); +} + +CNFQuery TreeCNFConverter::toCNF(const ASTPtr & query) +{ + auto cnf = query->clone(); + + splitMultiLogic(cnf); + traversePushNot(cnf, false); + pushOr(cnf); + + CNFQuery::AndGroup and_group; + traverseCNF(cnf, and_group); + + CNFQuery result{std::move(and_group)}; + + Poco::Logger::get("CNF CONVERSION").information("DONE: " + result.dump()); + return result; +} + +ASTPtr TreeCNFConverter::fromCNF(const CNFQuery & cnf) +{ + const auto & groups = cnf.getStatements(); + if (groups.empty()) + return nullptr; + + ASTs or_groups; + for (const auto & group : groups) + { + if (group.size() == 1) + or_groups.push_back(*group.begin()); + else if (group.size() > 1) + { + or_groups.push_back(makeASTFunction("or")); + auto * func = or_groups.back()->as(); + for (const auto & ast : group) + func->arguments->children.push_back(ast); + } + } + + if (or_groups.size() == 1) + return or_groups.front(); + + ASTPtr res = makeASTFunction("and"); + auto * func = res->as(); + for (const auto & group : or_groups) + func->arguments->children.push_back(group); + + return res; +} + +std::string CNFQuery::dump() const +{ + std::stringstream res; + bool first = true; + for (const auto & group : statements) + { + if (!first) + res << " AND "; + first = false; + res << "("; + bool first_in_group = true; + for (const auto & ast : group) + { + if (!first_in_group) + res << " OR "; + first_in_group = false; + res << ast->getColumnName(); + } + res << ")"; + } + + return res.str(); +} + +} diff --git a/src/Interpreters/TreeCNFConverter.h b/src/Interpreters/TreeCNFConverter.h new file mode 100644 index 00000000000..9bc27f2c1b4 --- /dev/null +++ b/src/Interpreters/TreeCNFConverter.h @@ -0,0 +1,64 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class CNFQuery +{ +public: + using OrGroup = std::set; // Add NOT container??? + using AndGroup = std::set; + + CNFQuery(AndGroup && statements_) + : statements(std::move(statements_)) {} + + template + void filterGroups(P predicate) { + AndGroup filtered; + for (const auto & or_group : statements) + { + if (predicate(or_group)) + filtered.insert(or_group); + } + std::swap(statements, filtered); + } + + template + void filterAtoms(P predicate) { + AndGroup filtered; + for (const auto & or_group : statements) + { + OrGroup filtered_group; + for (auto ast : or_group) { + if (predicate(ast)) + filtered_group.insert(ast); + } + if (!filtered_group.empty()) + filtered.insert(filtered_group); + } + std::swap(statements, filtered); + } + + const AndGroup & getStatements() const { return statements; } + + std::string dump() const; + +private: + AndGroup statements; +}; + +class TreeCNFConverter +{ +public: + + static CNFQuery toCNF(const ASTPtr & query); + + static ASTPtr fromCNF(const CNFQuery & cnf); +}; + +} diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 0d1fdd69c95..557b492d638 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -3,7 +3,8 @@ #include #include #include -#include +#include +#include #include #include #include @@ -507,20 +508,22 @@ void optimizeLimitBy(const ASTSelectQuery * select_query) elems = std::move(unique_elems); } -void optimizeWithConstraints(ASTSelectQuery * select_query, const NameSet & /* source_columns_set */, - const std::vector & /* tables_with_columns */, +/// Use constraints to get rid of useless parts of query +void optimizeWithConstraints(ASTSelectQuery * select_query, Aliases & aliases, const NameSet & source_columns_set, + const std::vector & tables_with_columns, const StorageMetadataPtr & metadata_snapshot) { - ConstraintMatcherVisitor::Data constraint_data; - - for (const auto & constraint : metadata_snapshot->getConstraints().constraints) - { - const auto expr = constraint->as()->expr->clone(); - constraint_data.constraints[expr->getTreeHash().second].push_back(expr); - } + WhereConstraintsOptimizer(select_query, aliases, source_columns_set, tables_with_columns, metadata_snapshot).perform(); +} +/// transform where to CNF for more convenient optimization +void convertQueryToCNF(ASTSelectQuery * select_query) +{ if (select_query->where()) - ConstraintMatcherVisitor(constraint_data).visit(select_query->refWhere()); + { + auto cnf_form = TreeCNFConverter::toCNF(select_query->where()); + select_query->refWhere() = TreeCNFConverter::fromCNF(cnf_form); + } } /// Remove duplicated columns from USING(...). @@ -611,12 +614,15 @@ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & sou if (settings.optimize_arithmetic_operations_in_aggregate_functions) optimizeAggregationFunctions(query); + if (settings.convert_query_to_cnf) + convertQueryToCNF(select_query); + + if (settings.convert_query_to_cnf && settings.optimize_using_constraints) + optimizeWithConstraints(select_query, aliases, source_columns_set, tables_with_columns, metadata_snapshot); + /// Push the predicate expression down to the subqueries. rewrite_subqueries = PredicateExpressionsOptimizer(context, tables_with_columns, settings).optimize(*select_query); - if (settings.optimize_using_constraints) - optimizeWithConstraints(select_query, source_columns_set, tables_with_columns, metadata_snapshot); - /// GROUP BY injective function elimination. optimizeGroupBy(select_query, source_columns_set, context); diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp new file mode 100644 index 00000000000..a91414444ce --- /dev/null +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -0,0 +1,49 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + +ConstraintMatcherVisitor::Data getConstraintData(const StorageMetadataPtr & metadata_snapshot) +{ + ConstraintMatcherVisitor::Data constraint_data; + + for (const auto & constraint : metadata_snapshot->getConstraints().filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) + { + const auto expr = constraint->as()->expr->clone(); + + constraint_data.constraints[expr->getTreeHash().second].push_back(expr); + } + + return constraint_data; +} + +WhereConstraintsOptimizer::WhereConstraintsOptimizer( + ASTSelectQuery * select_query_, + Aliases & /*aliases_*/, + const NameSet & /*source_columns_set_*/, + const std::vector & /*tables_with_columns_*/, + const StorageMetadataPtr & metadata_snapshot_) + : select_query(select_query_) + /* , aliases(aliases_) + , source_columns_set(source_columns_set_) + , tables_with_columns(tables_with_columns_)*/ + , metadata_snapshot(metadata_snapshot_) +{ +} + +void WhereConstraintsOptimizer::perform() +{ + auto constraint_data = getConstraintData(metadata_snapshot); + + /// TODO: split != -> NOT =, >= -> NOT <, ... + + if (select_query->where()) + ConstraintMatcherVisitor(constraint_data).visit(select_query->refWhere()); +} + +} diff --git a/src/Interpreters/WhereConstraintsOptimizer.h b/src/Interpreters/WhereConstraintsOptimizer.h new file mode 100644 index 00000000000..fe13c943ef4 --- /dev/null +++ b/src/Interpreters/WhereConstraintsOptimizer.h @@ -0,0 +1,34 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; + +class WhereConstraintsOptimizer final +{ +public: + WhereConstraintsOptimizer(ASTSelectQuery * select_query, Aliases & /* aliases */, const NameSet & /* source_columns_set */, + const std::vector & /* tables_with_columns */, + const StorageMetadataPtr & metadata_snapshot); + + void perform(); + +private: + ASTSelectQuery * select_query; + /*Aliases & aliases; + const NameSet & source_columns_set; + const std::vector & tables_with_columns;*/ + const StorageMetadataPtr & metadata_snapshot; +}; + +} diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index 08612cd240f..5336079aa21 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -41,14 +41,24 @@ ConstraintsDescription ConstraintsDescription::parse(const String & str) return res; } -ASTs ConstraintsDescription::filterConstraints(ConstraintType type) const +ASTs ConstraintsDescription::filterConstraints(ConstraintType selection) const { - auto constraint_type = (type == ConstraintType::CHECK ? ASTConstraintDeclaration::Type::CHECK : ASTConstraintDeclaration::Type::ASSUME); + const auto ast_to_decr_constraint_type = [](ASTConstraintDeclaration::Type constraint_type) -> UInt32 + { + switch (constraint_type) + { + case ASTConstraintDeclaration::Type::CHECK: + return static_cast(ConstraintType::CHECK); + case ASTConstraintDeclaration::Type::ASSUME: + return static_cast(ConstraintType::ASSUME); + } + }; + ASTs res; res.reserve(constraints.size()); for (const auto & constraint : constraints) { - if (constraint->as()->type == constraint_type) { + if ((ast_to_decr_constraint_type(constraint->as()->type) & static_cast(selection)) != 0) { res.push_back(constraint); } } diff --git a/src/Storages/ConstraintsDescription.h b/src/Storages/ConstraintsDescription.h index 414392e0662..3bdd9bc5503 100644 --- a/src/Storages/ConstraintsDescription.h +++ b/src/Storages/ConstraintsDescription.h @@ -10,11 +10,6 @@ using ConstraintsExpressions = std::vector; struct ConstraintsDescription { - enum class ConstraintType { - CHECK, - ASSUME, - }; - std::vector constraints; ConstraintsDescription() = default; @@ -24,7 +19,14 @@ struct ConstraintsDescription static ConstraintsDescription parse(const String & str); - ASTs filterConstraints(ConstraintType type) const; + enum class ConstraintType { + CHECK = 1, + ASSUME = 2, + ALWAYS_TRUE = CHECK | ASSUME, + ALL = CHECK | ASSUME, + }; + + ASTs filterConstraints(ConstraintType selection) const; ConstraintsExpressions getExpressionsToCheck(const Context & context, const NamesAndTypesList & source_columns_) const; From 10fecb91836934877b1b12442108ad2ff7361828 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Tue, 5 Jan 2021 23:51:19 +0300 Subject: [PATCH 004/200] more optimization --- src/Interpreters/TreeCNFConverter.cpp | 88 ++++++++++- src/Interpreters/TreeCNFConverter.h | 61 +++++++- src/Interpreters/TreeOptimizer.cpp | 6 + .../WhereConstraintsOptimizer.cpp | 137 ++++++++++++++++-- ..._constraints_simple_optimization.reference | 5 +- .../01622_constraints_simple_optimization.sql | 16 +- 6 files changed, 288 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index c1814444435..b9208a92d84 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -30,7 +30,7 @@ void splitMultiLogic(ASTPtr & node) } } -/// Push NOT to leafs +/// Push NOT to leafs, remove NOT NOT ... void traversePushNot(ASTPtr & node, bool add_negation) { auto * func = node->as(); @@ -196,7 +196,7 @@ ASTPtr TreeCNFConverter::fromCNF(const CNFQuery & cnf) or_groups.push_back(makeASTFunction("or")); auto * func = or_groups.back()->as(); for (const auto & ast : group) - func->arguments->children.push_back(ast); + func->arguments->children.push_back(ast->clone()); } } @@ -211,6 +211,90 @@ ASTPtr TreeCNFConverter::fromCNF(const CNFQuery & cnf) return res; } +void pullNotOut(ASTPtr & node) +{ + static const std::map inverse_relations = { + {"notEquals", "equals"}, + {"greaterOrEquals", "less"}, + {"greater", "lessOrEquals"}, + {"notIn", "in"}, + {"notLike", "like"}, + {"notEmpty", "empty"}, + }; + + auto * func = node->as(); + if (!func) + return; + if (auto it = inverse_relations.find(func->name); it != std::end(inverse_relations)) + { + /// inverse func + node = node->clone(); + auto * new_func = node->as(); + new_func->name = it->second; + /// add not + node = makeASTFunction("not", node); + } +} + +void pushNotIn(ASTPtr & node) +{ + static const std::map inverse_relations = { + {"equals", "notEquals"}, + {"less", "greaterOrEquals"}, + {"lessOrEquals", "greater"}, + {"in", "notIn"}, + {"like", "notLike"}, + {"empty", "notEmpty"}, + }; + + auto * func = node->as(); + if (!func) + return; + if (auto it = inverse_relations.find(func->name); it != std::end(inverse_relations)) + { + /// inverse func + node = node->clone(); + auto * new_func = node->as(); + new_func->name = it->second; + /// add not + node = makeASTFunction("not", node); + } +} + +CNFQuery & CNFQuery::pullNotOutFunctions() +{ + transformAtoms([](const ASTPtr & node) -> ASTPtr + { + auto * func = node->as(); + if (!func) + return node; + ASTPtr result = node->clone(); + if (func->name == "not") + pullNotOut(func->arguments->children.front()); + else + pullNotOut(result); + traversePushNot(result, false); + return result; + }); + return *this; +} + +CNFQuery & CNFQuery::pushNotInFuntions() +{ + transformAtoms([](const ASTPtr & node) -> ASTPtr + { + auto * func = node->as(); + if (!func) + return node; + ASTPtr result = node->clone(); + if (func->name == "not") + pushNotIn(func->arguments->children.front()); + traversePushNot(result, false); + return result; + }); + return *this; +} + std::string CNFQuery::dump() const { std::stringstream res; diff --git a/src/Interpreters/TreeCNFConverter.h b/src/Interpreters/TreeCNFConverter.h index 9bc27f2c1b4..6b25ddc8ff6 100644 --- a/src/Interpreters/TreeCNFConverter.h +++ b/src/Interpreters/TreeCNFConverter.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -14,11 +15,11 @@ public: using OrGroup = std::set; // Add NOT container??? using AndGroup = std::set; - CNFQuery(AndGroup && statements_) - : statements(std::move(statements_)) {} + CNFQuery(AndGroup && statements_) : statements(std::move(statements_)) { } template - void filterGroups(P predicate) { + CNFQuery & filterGroups(P predicate) /// delete always true groups + { AndGroup filtered; for (const auto & or_group : statements) { @@ -26,28 +27,78 @@ public: filtered.insert(or_group); } std::swap(statements, filtered); + return *this; } template - void filterAtoms(P predicate) { + CNFQuery & filterAtoms(P predicate) /// delete always false atoms + { AndGroup filtered; for (const auto & or_group : statements) { OrGroup filtered_group; - for (auto ast : or_group) { + for (auto ast : or_group) + { if (predicate(ast)) filtered_group.insert(ast); } if (!filtered_group.empty()) filtered.insert(filtered_group); + else + { + /// all atoms false -> group false -> CNF false + filtered.clear(); + filtered_group.clear(); + filtered_group.insert(std::make_shared(static_cast(0))); + filtered.insert(filtered_group); + std::swap(statements, filtered); + return *this; + } } std::swap(statements, filtered); + return *this; + } + + template + CNFQuery & transformGroups(F func) + { + AndGroup result; + for (const auto & group : statements) + { + auto new_group = func(group); + if (!new_group.empty()) + result.insert(std::move(new_group)); + } + std::swap(statements, result); + return *this; + } + + template + CNFQuery & transformAtoms(F func) + { + transformGroups([func](const OrGroup & group) -> OrGroup + { + OrGroup result; + for (const auto & ast : group) + { + auto new_ast = func(ast); + if (new_ast) + result.insert(std::move(new_ast)); + } + return result; + }); + return *this; } const AndGroup & getStatements() const { return statements; } std::string dump() const; + /// Converts != -> NOT =; <,>= -> (NOT) <; >,<= -> (NOT) <= for simpler matching + CNFQuery & pullNotOutFunctions(); + /// Revert pullNotOutFunctions actions + CNFQuery & pushNotInFuntions(); + private: AndGroup statements; }; diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 557b492d638..2c3d07072ed 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -514,6 +514,10 @@ void optimizeWithConstraints(ASTSelectQuery * select_query, Aliases & aliases, c const StorageMetadataPtr & metadata_snapshot) { WhereConstraintsOptimizer(select_query, aliases, source_columns_set, tables_with_columns, metadata_snapshot).perform(); + if (select_query->where()) + Poco::Logger::get("KEK").information(select_query->where()->dumpTree()); + else + Poco::Logger::get("KEK").information("NO WHERE"); } /// transform where to CNF for more convenient optimization @@ -524,6 +528,8 @@ void convertQueryToCNF(ASTSelectQuery * select_query) auto cnf_form = TreeCNFConverter::toCNF(select_query->where()); select_query->refWhere() = TreeCNFConverter::fromCNF(cnf_form); } + if (select_query->where()) + Poco::Logger::get("KEK").information(select_query->where()->dumpTree()); } /// Remove duplicated columns from USING(...). diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index a91414444ce..14ea9055623 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -7,16 +8,17 @@ namespace DB { - -ConstraintMatcherVisitor::Data getConstraintData(const StorageMetadataPtr & metadata_snapshot) +std::vector> getConstraintData(const StorageMetadataPtr & metadata_snapshot) { - ConstraintMatcherVisitor::Data constraint_data; + std::vector> constraint_data; - for (const auto & constraint : metadata_snapshot->getConstraints().filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) + for (const auto & constraint : + metadata_snapshot->getConstraints().filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) { - const auto expr = constraint->as()->expr->clone(); - - constraint_data.constraints[expr->getTreeHash().second].push_back(expr); + const auto cnf = TreeCNFConverter::toCNF(constraint->as()->expr->ptr()) + .pullNotOutFunctions(); + for (const auto & group : cnf.getStatements()) + constraint_data.emplace_back(std::begin(group), std::end(group)); } return constraint_data; @@ -29,21 +31,134 @@ WhereConstraintsOptimizer::WhereConstraintsOptimizer( const std::vector & /*tables_with_columns_*/, const StorageMetadataPtr & metadata_snapshot_) : select_query(select_query_) - /* , aliases(aliases_) + /* , aliases(aliases_) , source_columns_set(source_columns_set_) , tables_with_columns(tables_with_columns_)*/ , metadata_snapshot(metadata_snapshot_) { } +bool checkIfGroupAlwaysTrue(const CNFQuery::OrGroup & group, const std::vector> & constraints) +{ + /// TODO: this is temporary; need to write more effective search + /// TODO: go deeper into asts (a < b, a = b,...) with z3 or some visitor + for (const auto & constraint : constraints) /// one constraint in group is enough, + /// otherwise it's difficult to make judgements without using constraint solving (z3..) + { + bool group_always_true = true; + for (const auto & constraint_ast : constraint) + { + bool found_match = false; + for (const auto & group_ast : group) + { + bool match_means_ok = true; + ASTPtr a = constraint_ast; + ASTPtr b = group_ast; + + { + auto * func_a = a->as(); + if (func_a && func_a->name == "not") + { + a = func_a->arguments->children.front(); + match_means_ok ^= true; + } + } + { + auto * func_b = b->as(); + if (func_b && func_b->name == "not") + { + b = func_b->arguments->children.front(); + match_means_ok ^= true; + } + } + + if (a->getTreeHash() == b->getTreeHash() && + a->getColumnName() == b->getColumnName()) + { + found_match = match_means_ok; + break; + } + } + if (!found_match) + { + group_always_true = false; + break; + } + } + + if (group_always_true) + return true; + } + return false; +} + +bool checkIfAtomAlwaysFalse(const ASTPtr & atom, const std::vector> & constraints) +{ + /// TODO: more efficient matching + + for (const auto & constraint : constraints) + { + if (constraint.size() > 1) + continue; /// TMP; Too hard to do something at current time (without more powerful instruments) + + for (const auto & constraint_ast : constraint) + { + bool match_means_ok = true; + ASTPtr a = constraint_ast; + ASTPtr b = atom; + + { + auto * func_a = a->as(); + if (func_a && func_a->name == "not") + { + a = func_a->arguments->children.front(); + match_means_ok ^= true; + } + } + { + auto * func_b = b->as(); + if (func_b && func_b->name == "not") + { + b = func_b->arguments->children.front(); + match_means_ok ^= true; + } + } + + Poco::Logger::get("MATCHER a").information(a->dumpTree()); + Poco::Logger::get("MATCHER b").information(b->dumpTree()); + Poco::Logger::get("MATCHER a>>").information(a->getColumnName()); + Poco::Logger::get("MATCHER b>>" ).information(b->getColumnName()); + if (a->getTreeHash() == b->getTreeHash() && + a->getColumnName() == b->getColumnName()) + { + Poco::Logger::get("MATCH").information(std::to_string(static_cast(match_means_ok))); + return !match_means_ok; + } + } + } + + return false; +} + void WhereConstraintsOptimizer::perform() { auto constraint_data = getConstraintData(metadata_snapshot); - /// TODO: split != -> NOT =, >= -> NOT <, ... - if (select_query->where()) - ConstraintMatcherVisitor(constraint_data).visit(select_query->refWhere()); + { + auto cnf = TreeCNFConverter::toCNF(select_query->where()); + + cnf.pullNotOutFunctions() + .filterGroups([&constraint_data](const auto & group) + { return !checkIfGroupAlwaysTrue(group, constraint_data); }) /// remove always true functions in CNF + .filterAtoms([&constraint_data](const auto & ast) + { return !checkIfAtomAlwaysFalse(ast, constraint_data); }) /// TODO: remove always false atoms in CNF + .pushNotInFuntions(); + + //ConstraintMatcherVisitor(constraint_data).visit(select_query->refWhere()); + Poco::Logger::get("AFTER OPT").information(cnf.dump()); + select_query->setExpression(ASTSelectQuery::Expression::WHERE, TreeCNFConverter::fromCNF(cnf)); + } } } diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index e55c7091740..05243cf58eb 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -1,4 +1,7 @@ 4 0 -3 +0 +0 +4 +4 0 diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index cedae1b0237..d4f34a2dea7 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -4,18 +4,22 @@ DROP TABLE IF EXISTS constraint_test.constrained; SET optimize_using_constraints = 1; CREATE DATABASE constraint_test; -CREATE TABLE constraint_test.assumption (URL String, CONSTRAINT is_yandex ASSUME domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT is_utf8 CHECK isValidUTF8(URL)) ENGINE = TinyLog; +CREATE TABLE constraint_test.assumption (URL String, a Int32, CONSTRAINT c1 ASSUME domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT c2 ASSUME URL > 'zzz', CONSTRAINT c3 CHECK isValidUTF8(URL)) ENGINE = TinyLog; --- Add wrong rows in order to check optimization -INSERT INTO constraint_test.assumption (URL) VALUES ('1'); -INSERT INTO constraint_test.assumption (URL) VALUES ('2'); -INSERT INTO constraint_test.assumption (URL) VALUES ('yandex.ru'); -INSERT INTO constraint_test.assumption (URL) VALUES ('3'); +INSERT INTO constraint_test.assumption (URL, a) VALUES ('1', 1); +INSERT INTO constraint_test.assumption (URL, a) VALUES ('2', 2); +INSERT INTO constraint_test.assumption (URL, a) VALUES ('yandex.ru', 3); +INSERT INTO constraint_test.assumption (URL, a) VALUES ('3', 4); SELECT count() FROM constraint_test.assumption WHERE domainWithoutWWW(URL) = 'yandex.ru'; --- assumption -> 4 SELECT count() FROM constraint_test.assumption WHERE NOT (domainWithoutWWW(URL) = 'yandex.ru'); --- assumption -> 0 -SELECT count() FROM constraint_test.assumption WHERE domainWithoutWWW(URL) != 'yandex.ru'; --- not optimized -> 3 +SELECT count() FROM constraint_test.assumption WHERE domainWithoutWWW(URL) != 'yandex.ru'; --- assumption -> 0 SELECT count() FROM constraint_test.assumption WHERE domainWithoutWWW(URL) = 'nothing'; --- not optimized -> 0 +SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL > 'zzz'); ---> assumption -> 4 +SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL > 'zzz') OR (a = 10 AND a + 5 < 100); ---> assumption -> 4 +SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL = '111'); ---> assumption & no assumption -> 0 + DROP TABLE constraint_test.assumption; DROP DATABASE constraint_test; \ No newline at end of file From 106c00d19d58757ca6616e62885db8c8e810016a Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Wed, 6 Jan 2021 13:41:06 +0300 Subject: [PATCH 005/200] improvements --- src/Interpreters/TreeCNFConverter.h | 8 +- .../WhereConstraintsOptimizer.cpp | 117 ++++++++---------- 2 files changed, 59 insertions(+), 66 deletions(-) diff --git a/src/Interpreters/TreeCNFConverter.h b/src/Interpreters/TreeCNFConverter.h index 6b25ddc8ff6..a27df8bd2b9 100644 --- a/src/Interpreters/TreeCNFConverter.h +++ b/src/Interpreters/TreeCNFConverter.h @@ -18,12 +18,12 @@ public: CNFQuery(AndGroup && statements_) : statements(std::move(statements_)) { } template - CNFQuery & filterGroups(P predicate) /// delete always true groups + CNFQuery & filterAlwaysTrueGroups(P predicate_is_unknown) /// delete always true groups { AndGroup filtered; for (const auto & or_group : statements) { - if (predicate(or_group)) + if (predicate_is_unknown(or_group)) filtered.insert(or_group); } std::swap(statements, filtered); @@ -31,7 +31,7 @@ public: } template - CNFQuery & filterAtoms(P predicate) /// delete always false atoms + CNFQuery & filterAlwaysFalseAtoms(P predicate_is_unknown) /// delete always false atoms { AndGroup filtered; for (const auto & or_group : statements) @@ -39,7 +39,7 @@ public: OrGroup filtered_group; for (auto ast : or_group) { - if (predicate(ast)) + if (predicate_is_unknown(ast)) filtered_group.insert(ast); } if (!filtered_group.empty()) diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index 14ea9055623..dbd5216a1ce 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -1,7 +1,7 @@ #include #include -#include +#include #include #include #include @@ -16,7 +16,7 @@ std::vector> getConstraintData(const StorageMetadataPtr & me metadata_snapshot->getConstraints().filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) { const auto cnf = TreeCNFConverter::toCNF(constraint->as()->expr->ptr()) - .pullNotOutFunctions(); + .pullNotOutFunctions(); /// TODO: move prepare stage to ConstraintsDescription for (const auto & group : cnf.getStatements()) constraint_data.emplace_back(std::begin(group), std::end(group)); } @@ -38,6 +38,45 @@ WhereConstraintsOptimizer::WhereConstraintsOptimizer( { } +namespace +{ + enum class MatchState + { + FULL_MATCH, /// a = b + NOT_MATCH, /// a = not b + NONE, /// other + }; +} + +MatchState match(ASTPtr a, ASTPtr b) +{ + bool match_means_ok = true; + + { + auto * func_a = a->as(); + if (func_a && func_a->name == "not") + { + a = func_a->arguments->children.front(); + match_means_ok ^= true; + } + } + { + auto * func_b = b->as(); + if (func_b && func_b->name == "not") + { + b = func_b->arguments->children.front(); + match_means_ok ^= true; + } + } + + if (a->getTreeHash() == b->getTreeHash() && + a->getColumnName() == b->getColumnName()) + { + return match_means_ok ? MatchState::FULL_MATCH : MatchState::NOT_MATCH; + } + return MatchState::NONE; +} + bool checkIfGroupAlwaysTrue(const CNFQuery::OrGroup & group, const std::vector> & constraints) { /// TODO: this is temporary; need to write more effective search @@ -51,31 +90,11 @@ bool checkIfGroupAlwaysTrue(const CNFQuery::OrGroup & group, const std::vectoras(); - if (func_a && func_a->name == "not") - { - a = func_a->arguments->children.front(); - match_means_ok ^= true; - } - } - { - auto * func_b = b->as(); - if (func_b && func_b->name == "not") - { - b = func_b->arguments->children.front(); - match_means_ok ^= true; - } - } - - if (a->getTreeHash() == b->getTreeHash() && - a->getColumnName() == b->getColumnName()) - { - found_match = match_means_ok; + found_match = true; break; } } @@ -99,41 +118,14 @@ bool checkIfAtomAlwaysFalse(const ASTPtr & atom, const std::vector 1) - continue; /// TMP; Too hard to do something at current time (without more powerful instruments) + continue; /// TMP for (const auto & constraint_ast : constraint) { - bool match_means_ok = true; - ASTPtr a = constraint_ast; - ASTPtr b = atom; + const auto match_result = match(constraint_ast, atom); - { - auto * func_a = a->as(); - if (func_a && func_a->name == "not") - { - a = func_a->arguments->children.front(); - match_means_ok ^= true; - } - } - { - auto * func_b = b->as(); - if (func_b && func_b->name == "not") - { - b = func_b->arguments->children.front(); - match_means_ok ^= true; - } - } - - Poco::Logger::get("MATCHER a").information(a->dumpTree()); - Poco::Logger::get("MATCHER b").information(b->dumpTree()); - Poco::Logger::get("MATCHER a>>").information(a->getColumnName()); - Poco::Logger::get("MATCHER b>>" ).information(b->getColumnName()); - if (a->getTreeHash() == b->getTreeHash() && - a->getColumnName() == b->getColumnName()) - { - Poco::Logger::get("MATCH").information(std::to_string(static_cast(match_means_ok))); - return !match_means_ok; - } + if (match_result != MatchState::NONE) + return match_result == MatchState::NOT_MATCH; } } @@ -149,14 +141,15 @@ void WhereConstraintsOptimizer::perform() auto cnf = TreeCNFConverter::toCNF(select_query->where()); cnf.pullNotOutFunctions() - .filterGroups([&constraint_data](const auto & group) - { return !checkIfGroupAlwaysTrue(group, constraint_data); }) /// remove always true functions in CNF - .filterAtoms([&constraint_data](const auto & ast) - { return !checkIfAtomAlwaysFalse(ast, constraint_data); }) /// TODO: remove always false atoms in CNF + .filterAlwaysTrueGroups([&constraint_data](const auto & group) { /// remove always true groups from CNF + return !checkIfGroupAlwaysTrue(group, constraint_data); + }) + .filterAlwaysFalseAtoms([&constraint_data](const auto & ast) { /// remove always false atoms from CNF + return !checkIfAtomAlwaysFalse(ast, constraint_data); + }) .pushNotInFuntions(); - //ConstraintMatcherVisitor(constraint_data).visit(select_query->refWhere()); - Poco::Logger::get("AFTER OPT").information(cnf.dump()); + //Poco::Logger::get("AFTER OPT").information(cnf.dump()); select_query->setExpression(ASTSelectQuery::Expression::WHERE, TreeCNFConverter::fromCNF(cnf)); } } From 7e23baa1e303bcf52dea480c1135d4f9668d5e7d Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Wed, 6 Jan 2021 23:04:49 +0300 Subject: [PATCH 006/200] fix nullptr --- src/Interpreters/TreeOptimizer.cpp | 8 ++++---- src/Interpreters/WhereConstraintsOptimizer.cpp | 11 +++++------ 2 files changed, 9 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 2c3d07072ed..f66985c13ad 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -515,9 +515,9 @@ void optimizeWithConstraints(ASTSelectQuery * select_query, Aliases & aliases, c { WhereConstraintsOptimizer(select_query, aliases, source_columns_set, tables_with_columns, metadata_snapshot).perform(); if (select_query->where()) - Poco::Logger::get("KEK").information(select_query->where()->dumpTree()); + Poco::Logger::get("CNF").information(select_query->where()->dumpTree()); else - Poco::Logger::get("KEK").information("NO WHERE"); + Poco::Logger::get("CNF").information("NO WHERE"); } /// transform where to CNF for more convenient optimization @@ -525,11 +525,11 @@ void convertQueryToCNF(ASTSelectQuery * select_query) { if (select_query->where()) { - auto cnf_form = TreeCNFConverter::toCNF(select_query->where()); + auto cnf_form = TreeCNFConverter::toCNF(select_query->where()).pushNotInFuntions(); select_query->refWhere() = TreeCNFConverter::fromCNF(cnf_form); } if (select_query->where()) - Poco::Logger::get("KEK").information(select_query->where()->dumpTree()); + Poco::Logger::get("CNF").information(select_query->where()->dumpTree()); } /// Remove duplicated columns from USING(...). diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index dbd5216a1ce..3953e34f267 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -5,13 +5,13 @@ #include #include #include +#include namespace DB { std::vector> getConstraintData(const StorageMetadataPtr & metadata_snapshot) { std::vector> constraint_data; - for (const auto & constraint : metadata_snapshot->getConstraints().filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) { @@ -134,12 +134,11 @@ bool checkIfAtomAlwaysFalse(const ASTPtr & atom, const std::vectorwhere()) + if (select_query->where() && metadata_snapshot) { + const auto constraint_data = getConstraintData(metadata_snapshot); auto cnf = TreeCNFConverter::toCNF(select_query->where()); - + Poco::Logger::get("BEFORE OPT").information(cnf.dump()); cnf.pullNotOutFunctions() .filterAlwaysTrueGroups([&constraint_data](const auto & group) { /// remove always true groups from CNF return !checkIfGroupAlwaysTrue(group, constraint_data); @@ -149,7 +148,7 @@ void WhereConstraintsOptimizer::perform() }) .pushNotInFuntions(); - //Poco::Logger::get("AFTER OPT").information(cnf.dump()); + Poco::Logger::get("AFTER OPT").information(cnf.dump()); select_query->setExpression(ASTSelectQuery::Expression::WHERE, TreeCNFConverter::fromCNF(cnf)); } } From 7b2d8e1094e3b45b88d42764077c8ee11840a810 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 14 Feb 2021 17:47:15 +0300 Subject: [PATCH 007/200] more clones --- src/Interpreters/InterpreterSelectQuery.cpp | 2 ++ src/Interpreters/TreeCNFConverter.cpp | 18 +++++++++--------- src/Interpreters/TreeOptimizer.cpp | 11 +++++++++++ .../MergeTree/MergeTreeRangeReader.cpp | 8 ++++++++ .../MergeTree/MergeTreeWhereOptimizer.cpp | 13 ++++++++++++- 5 files changed, 42 insertions(+), 10 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 38cc19a00d6..6fcfc445d74 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -342,6 +342,8 @@ InterpreterSelectQuery::InterpreterSelectQuery( current_info.query = query_ptr; current_info.syntax_analyzer_result = syntax_analyzer_result; + log->information("TRY MOVE TO PREWHERE ---------------------"); + log->information(query.where()->dumpTree()); MergeTreeWhereOptimizer{current_info, *context, *merge_tree, metadata_snapshot, syntax_analyzer_result->requiredSourceColumns(), log}; } } diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index b9208a92d84..c547cdd63ac 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -16,10 +16,10 @@ void splitMultiLogic(ASTPtr & node) { if (func->arguments->children.size() > 2) { - ASTPtr res = func->arguments->children.front(); + ASTPtr res = func->arguments->children.front()->clone(); for (size_t i = 1; i < func->arguments->children.size(); ++i) { - res = makeASTFunction(func->name, res, func->arguments->children[i]); + res = makeASTFunction(func->name, res, func->arguments->children[i]->clone()); } node = std::move(res); } @@ -41,8 +41,8 @@ void traversePushNot(ASTPtr & node, bool add_negation) /// apply De Morgan's Law node = makeASTFunction( (func->name == "and" ? "or" : "and"), - func->arguments->children[0], - func->arguments->children[1]); + func->arguments->children[0]->clone(), + func->arguments->children[1]->clone()); } auto * new_func = node->as(); @@ -52,14 +52,14 @@ void traversePushNot(ASTPtr & node, bool add_negation) else if (func && func->name == "not") { /// delete NOT - node = func->arguments->children[0]; + node = func->arguments->children[0]->clone(); traversePushNot(node, !add_negation); } else { if (add_negation) - node = makeASTFunction("not", node); + node = makeASTFunction("not", node->clone()); } } @@ -118,8 +118,8 @@ void pushOr(ASTPtr & query) /// apply the distributive law ( a or (b and c) -> (a or b) and (a or c) ) or_node.get() = makeASTFunction( "and", - makeASTFunction("or", a, b), - makeASTFunction("or", a, c)); + makeASTFunction("or", a->clone(), b->clone()), + makeASTFunction("or", a->clone(), c->clone())); /// add new ors to stack auto * new_func = or_node.get()->as(); @@ -190,7 +190,7 @@ ASTPtr TreeCNFConverter::fromCNF(const CNFQuery & cnf) for (const auto & group : groups) { if (group.size() == 1) - or_groups.push_back(*group.begin()); + or_groups.push_back((*group.begin())->clone()); else if (group.size() > 1) { or_groups.push_back(makeASTFunction("or")); diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index f66985c13ad..7022d5a5002 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -626,6 +626,17 @@ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & sou if (settings.convert_query_to_cnf && settings.optimize_using_constraints) optimizeWithConstraints(select_query, aliases, source_columns_set, tables_with_columns, metadata_snapshot); + if (select_query->where()) + { + Poco::Logger::get("&&&&&&&&&&&&&&& WHERE").information(select_query->where()->getColumnName()); + Poco::Logger::get("&&&&&&&&&&&&&&& WHERE").information(select_query->where()->dumpTree()); + } + if (select_query->prewhere()) + { + Poco::Logger::get("&&&&&&&&&&&&&&& prewhere").information(select_query->prewhere()->getColumnName()); + Poco::Logger::get("&&&&&&&&&&&&&&& prewhere").information(select_query->prewhere()->dumpTree()); + } + /// Push the predicate expression down to the subqueries. rewrite_subqueries = PredicateExpressionsOptimizer(context, tables_with_columns, settings).optimize(*select_query); diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index c13146bd35c..5eec9b134de 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -495,6 +495,12 @@ MergeTreeRangeReader::MergeTreeRangeReader( , index_granularity(&(merge_tree_reader->data_part->index_granularity)), prev_reader(prev_reader_) , prewhere(prewhere_), last_reader_in_chain(last_reader_in_chain_), is_initialized(true) { + if (prewhere_) + { + Poco::Logger::get("PREWHERE").information(prewhere_->prewhere_column_name); + Poco::Logger::get("PREWHERE").information(prewhere_->prewhere_actions->dumpActions()); + Poco::Logger::get("PREWHERE rm?").information(std::to_string(prewhere_->remove_prewhere_column)); + } if (prev_reader) sample_block = prev_reader->getSampleBlock(); @@ -868,6 +874,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r /// If we need to filter in PREWHERE else if (prewhere->need_filter || result.need_filter) { + Poco::Logger::get("USE PREWHERE FILTER").information("++++++++++++++"); /// If there is a filter and without optimized if (result.getFilter() && last_reader_in_chain) { @@ -918,6 +925,7 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r /// Filter in WHERE instead else { + Poco::Logger::get("USE WHERE FILTER ").information("---------------------"); result.columns[prewhere_column_pos] = result.getFilterHolder()->convertToFullColumnIfConst(); if (getSampleBlock().getByName(prewhere->prewhere_column_name).type->isNullable()) result.columns[prewhere_column_pos] = makeNullable(std::move(result.columns[prewhere_column_pos])); diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 5d6b74cabe9..53e8c25a8a3 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -14,6 +14,7 @@ #include #include +#include namespace DB { @@ -40,6 +41,7 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( , block_with_constants{KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context)} , log{log_} { + Poco::Logger::get("kek").information("IN MT OPTIMIZER"); const auto & primary_key = metadata_snapshot->getPrimaryKey(); if (!primary_key.column_names.empty()) first_primary_key_column = primary_key.column_names[0]; @@ -190,7 +192,16 @@ void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const { if (!select.where() || select.prewhere()) return; - + if (select.where()) + { + Poco::Logger::get("MTPRWHERE WHE").information(select.where()->getColumnName()); + Poco::Logger::get("MTPRWHERE WHE").information(select.where()->dumpTree()); + } + if(select.prewhere()) + { + Poco::Logger::get("MTPRWHERE PRE").information(select.prewhere()->dumpTree()); + Poco::Logger::get("MTPRWHERE PRE").information(select.prewhere()->getColumnName()); + } Conditions where_conditions = analyze(select.where()); Conditions prewhere_conditions; From 1fb947b70b23bb3e26fe4bd904cbdca8776640d6 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 4 Mar 2021 15:11:43 +0300 Subject: [PATCH 008/200] impr --- src/CMakeLists.txt | 2 +- src/Interpreters/ComparisonGraph.cpp | 38 ++++++ src/Interpreters/ComparisonGraph.h | 76 ++++++++++++ src/Interpreters/TreeCNFConverter.cpp | 113 ++++++++++-------- src/Interpreters/TreeCNFConverter.h | 26 ++-- .../WhereConstraintsOptimizer.cpp | 77 +++++++----- src/Storages/ConstraintsDescription.h | 5 + 7 files changed, 247 insertions(+), 90 deletions(-) create mode 100644 src/Interpreters/ComparisonGraph.cpp create mode 100644 src/Interpreters/ComparisonGraph.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index d178d2356c6..f4709af1ad3 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -63,7 +63,7 @@ add_subdirectory (Server) set(dbms_headers) -set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h Interpreters/WhereConstraintsOptimizer.cpp Interpreters/WhereConstraintsOptimizer.h Interpreters/TreeCNFConverter.cpp Interpreters/TreeCNFConverter.h) +set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h Interpreters/WhereConstraintsOptimizer.cpp Interpreters/WhereConstraintsOptimizer.h Interpreters/TreeCNFConverter.cpp Interpreters/TreeCNFConverter.h Interpreters/ComparisonGraph.cpp Interpreters/ComparisonGraph.h) add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp new file mode 100644 index 00000000000..0a281848e59 --- /dev/null +++ b/src/Interpreters/ComparisonGraph.cpp @@ -0,0 +1,38 @@ +#include + +#include + +namespace DB +{ + +ComparisonGraph::ComparisonGraph(const std::vector & /*atomic_formulas*/) +{ +} + +std::vector ComparisonGraph::getEqual(const ASTPtr & ast) const +{ + const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash().second); + if (hash_it != std::end(graph.ast_hash_to_component)) + return {}; + const size_t index = hash_it->second; + //const auto vertex_it = std::find(std::begin(graph.vertexes[index].asts), std::end(graph.vertexes[index].asts), ast, ); + if (std::any_of( + std::cbegin(graph.vertexes[index].asts), + std::cend(graph.vertexes[index].asts), + [ast](const ASTPtr & constraint_ast) + { + return constraint_ast->getTreeHash() == ast->getTreeHash() && + constraint_ast->getColumnName() == ast->getColumnName(); + })) { + return graph.vertexes[index].asts; + } else { + return {}; + } +} + +ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAsts(const Graph & /*asts_graph*/) +{ + return {}; +} + +} diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h new file mode 100644 index 00000000000..5469e623786 --- /dev/null +++ b/src/Interpreters/ComparisonGraph.h @@ -0,0 +1,76 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class ComparisonGraph +{ +public: + ComparisonGraph(const std::vector & atomic_formulas); + + /// Works for string and num. + /// For other -- only eq. + enum class CompareResult + { + LESS, + LESS_OR_EQUAL, + EQUAL, + GREATER_OR_EQUAL, + GREATER, + //NOT_EQUAL, + UNKNOWN, + }; + + // TODO: implement + CompareResult compare(const ASTPtr & /*left*/, const ASTPtr & /*right*/) const { return CompareResult::UNKNOWN; } + + std::vector getEqual(const ASTPtr & ast) const; + + /// Find constants less and greater. + /// For int and double linear programming can be applied here. + // TODO: implement + ASTPtr getMax(const ASTPtr &) const { return nullptr; } // sup + ASTPtr getMin(const ASTPtr &) const { return nullptr; } // inf + +private: + /// strongly connected component + struct EqualComponent + { + std::vector asts; + }; + + /// TODO: move to diff for int and double: + /// LESS and LESS_OR_EQUAL with +const or 0 --- ok + /// with -const --- not ok + /// EQUAL is ok only for 0 + struct Edge + { + enum Type + { + LESS, + LESS_OR_EQUAL, + EQUAL, + }; + + Type type; + EqualComponent to; + }; + + struct Graph + { + std::unordered_map ast_hash_to_component; + std::vector vertexes; + std::vector> edges; + }; + + Graph BuildGraphFromAsts(const Graph & asts_graph); + + Graph graph; +}; + +} diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index c547cdd63ac..ad991b64299 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -149,9 +149,13 @@ void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & and_group, CNFQuery:: traverseCNF(child, and_group, or_group); } } + else if (func && func->name == "not") + { + or_group.insert(CNFQuery::AtomicFormula{true, func->arguments->children.front()}); + } else { - or_group.insert(node); + or_group.insert(CNFQuery::AtomicFormula{false, node}); } } @@ -190,13 +194,23 @@ ASTPtr TreeCNFConverter::fromCNF(const CNFQuery & cnf) for (const auto & group : groups) { if (group.size() == 1) - or_groups.push_back((*group.begin())->clone()); + { + if ((*group.begin()).negative) + or_groups.push_back(makeASTFunction("not", (*group.begin()).ast->clone())); + else + or_groups.push_back((*group.begin()).ast->clone()); + } else if (group.size() > 1) { or_groups.push_back(makeASTFunction("or")); auto * func = or_groups.back()->as(); - for (const auto & ast : group) - func->arguments->children.push_back(ast->clone()); + for (const auto & atom : group) + { + if ((*group.begin()).negative) + func->arguments->children.push_back(makeASTFunction("not", atom.ast->clone())); + else + func->arguments->children.push_back(atom.ast->clone()); + } } } @@ -211,7 +225,23 @@ ASTPtr TreeCNFConverter::fromCNF(const CNFQuery & cnf) return res; } -void pullNotOut(ASTPtr & node) +void pushPullNotInAtom(CNFQuery::AtomicFormula & atom, const std::map & inverse_relations) +{ + auto * func = atom.ast->as(); + if (!func) + return; + if (auto it = inverse_relations.find(func->name); it != std::end(inverse_relations)) + { + /// inverse func + atom.ast = atom.ast->clone(); + auto * new_func = atom.ast->as(); + new_func->name = it->second; + /// add not + atom.negative = !atom.negative; + } +} + +void pullNotOut(CNFQuery::AtomicFormula & atom) { static const std::map inverse_relations = { {"notEquals", "equals"}, @@ -222,22 +252,14 @@ void pullNotOut(ASTPtr & node) {"notEmpty", "empty"}, }; - auto * func = node->as(); - if (!func) - return; - if (auto it = inverse_relations.find(func->name); it != std::end(inverse_relations)) - { - /// inverse func - node = node->clone(); - auto * new_func = node->as(); - new_func->name = it->second; - /// add not - node = makeASTFunction("not", node); - } + pushPullNotInAtom(atom, inverse_relations); } -void pushNotIn(ASTPtr & node) +void pushNotIn(CNFQuery::AtomicFormula & atom) { + if (!atom.negative) + return; + static const std::map inverse_relations = { {"equals", "notEquals"}, {"less", "greaterOrEquals"}, @@ -245,51 +267,34 @@ void pushNotIn(ASTPtr & node) {"in", "notIn"}, {"like", "notLike"}, {"empty", "notEmpty"}, + {"notEquals", "equals"}, + {"greaterOrEquals", "less"}, + {"greater", "lessOrEquals"}, + {"notIn", "in"}, + {"notLike", "like"}, + {"notEmpty", "empty"}, }; - auto * func = node->as(); - if (!func) - return; - if (auto it = inverse_relations.find(func->name); it != std::end(inverse_relations)) - { - /// inverse func - node = node->clone(); - auto * new_func = node->as(); - new_func->name = it->second; - /// add not - node = makeASTFunction("not", node); - } + pushPullNotInAtom(atom, inverse_relations); } CNFQuery & CNFQuery::pullNotOutFunctions() { - transformAtoms([](const ASTPtr & node) -> ASTPtr - { - auto * func = node->as(); - if (!func) - return node; - ASTPtr result = node->clone(); - if (func->name == "not") - pullNotOut(func->arguments->children.front()); - else - pullNotOut(result); - traversePushNot(result, false); - return result; - }); + transformAtoms([](const AtomicFormula & atom) -> AtomicFormula + { + AtomicFormula result{atom.negative, atom.ast->clone()}; + pullNotOut(result); + return result; + }); return *this; } CNFQuery & CNFQuery::pushNotInFuntions() { - transformAtoms([](const ASTPtr & node) -> ASTPtr + transformAtoms([](const AtomicFormula & atom) -> AtomicFormula { - auto * func = node->as(); - if (!func) - return node; - ASTPtr result = node->clone(); - if (func->name == "not") - pushNotIn(func->arguments->children.front()); - traversePushNot(result, false); + AtomicFormula result{atom.negative, atom.ast->clone()}; + pushNotIn(result); return result; }); return *this; @@ -306,12 +311,14 @@ std::string CNFQuery::dump() const first = false; res << "("; bool first_in_group = true; - for (const auto & ast : group) + for (const auto & atom : group) { if (!first_in_group) res << " OR "; first_in_group = false; - res << ast->getColumnName(); + if (atom.negative) + res << " NOT "; + res << atom.ast->getColumnName(); } res << ")"; } diff --git a/src/Interpreters/TreeCNFConverter.h b/src/Interpreters/TreeCNFConverter.h index a27df8bd2b9..51c5ae3aa43 100644 --- a/src/Interpreters/TreeCNFConverter.h +++ b/src/Interpreters/TreeCNFConverter.h @@ -12,7 +12,19 @@ namespace DB class CNFQuery { public: - using OrGroup = std::set; // Add NOT container??? + struct AtomicFormula + { + bool negative = false; + ASTPtr ast; + + /// for set + bool operator<(const AtomicFormula & rhs) const + { + return ast == rhs.ast ? negative < rhs.negative : ast < rhs.ast; + } + }; + + using OrGroup = std::set; using AndGroup = std::set; CNFQuery(AndGroup && statements_) : statements(std::move(statements_)) { } @@ -46,10 +58,10 @@ public: filtered.insert(filtered_group); else { - /// all atoms false -> group false -> CNF false + /// all atoms false -> group false -> CNF false filtered.clear(); filtered_group.clear(); - filtered_group.insert(std::make_shared(static_cast(0))); + filtered_group.insert(AtomicFormula{false, std::make_shared(static_cast(0))}); filtered.insert(filtered_group); std::swap(statements, filtered); return *this; @@ -79,11 +91,11 @@ public: transformGroups([func](const OrGroup & group) -> OrGroup { OrGroup result; - for (const auto & ast : group) + for (const auto & atom : group) { - auto new_ast = func(ast); - if (new_ast) - result.insert(std::move(new_ast)); + auto new_atom = func(atom); + if (new_atom.ast) + result.insert(std::move(new_atom)); } return result; }); diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index 3953e34f267..b0ebaffdfed 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -9,9 +9,9 @@ namespace DB { -std::vector> getConstraintData(const StorageMetadataPtr & metadata_snapshot) +std::vector> getConstraintData(const StorageMetadataPtr & metadata_snapshot) { - std::vector> constraint_data; + std::vector> constraint_data; for (const auto & constraint : metadata_snapshot->getConstraints().filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) { @@ -24,6 +24,39 @@ std::vector> getConstraintData(const StorageMetadataPtr & me return constraint_data; } +std::vector getAtomicConstraintData(const StorageMetadataPtr & metadata_snapshot) +{ + std::vector constraint_data; + for (const auto & constraint : + metadata_snapshot->getConstraints().filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) + { + const auto cnf = TreeCNFConverter::toCNF(constraint->as()->expr->ptr()) + .pullNotOutFunctions(); /// TODO: move prepare stage to ConstraintsDescription + for (const auto & group : cnf.getStatements()) { + if (group.size() == 1) + constraint_data.push_back(*group.begin()); + } + } + + return constraint_data; +} + +std::vector> getEqualConstraintData(const StorageMetadataPtr & metadata_snapshot) +{ + std::vector> equal_constraints; + const std::vector atomic_constraints = getAtomicConstraintData(metadata_snapshot); + for (const auto & constraint : atomic_constraints) { + auto * func = constraint.ast->as(); + if (func && (func->name == "equal" && !constraint.negative)) + { + equal_constraints.emplace_back( + func->arguments->children[0], + func->arguments->children[1]); + } + } + return equal_constraints; +} + WhereConstraintsOptimizer::WhereConstraintsOptimizer( ASTSelectQuery * select_query_, Aliases & /*aliases_*/, @@ -48,37 +81,22 @@ namespace }; } -MatchState match(ASTPtr a, ASTPtr b) +MatchState match(CNFQuery::AtomicFormula a, CNFQuery::AtomicFormula b) { - bool match_means_ok = true; + bool match_means_ok = true ^ a.negative ^ b.negative; - { - auto * func_a = a->as(); - if (func_a && func_a->name == "not") - { - a = func_a->arguments->children.front(); - match_means_ok ^= true; - } - } - { - auto * func_b = b->as(); - if (func_b && func_b->name == "not") - { - b = func_b->arguments->children.front(); - match_means_ok ^= true; - } - } - - if (a->getTreeHash() == b->getTreeHash() && - a->getColumnName() == b->getColumnName()) + if (a.ast->getTreeHash() == b.ast->getTreeHash() && + a.ast->getColumnName() == b.ast->getColumnName()) { return match_means_ok ? MatchState::FULL_MATCH : MatchState::NOT_MATCH; } return MatchState::NONE; } -bool checkIfGroupAlwaysTrue(const CNFQuery::OrGroup & group, const std::vector> & constraints) +bool checkIfGroupAlwaysTrue(const CNFQuery::OrGroup & group, const std::vector> & constraints) { + /// TODO: constraints graph + /// TODO: this is temporary; need to write more effective search /// TODO: go deeper into asts (a < b, a = b,...) with z3 or some visitor for (const auto & constraint : constraints) /// one constraint in group is enough, @@ -111,7 +129,7 @@ bool checkIfGroupAlwaysTrue(const CNFQuery::OrGroup & group, const std::vector> & constraints) +bool checkIfAtomAlwaysFalse(const CNFQuery::AtomicFormula & atom, const std::vector> & constraints) { /// TODO: more efficient matching @@ -120,9 +138,9 @@ bool checkIfAtomAlwaysFalse(const ASTPtr & atom, const std::vector 1) continue; /// TMP - for (const auto & constraint_ast : constraint) + for (const auto & constraint_atoms : constraint) { - const auto match_result = match(constraint_ast, atom); + const auto match_result = match(constraint_atoms, atom); if (match_result != MatchState::NONE) return match_result == MatchState::NOT_MATCH; @@ -137,14 +155,15 @@ void WhereConstraintsOptimizer::perform() if (select_query->where() && metadata_snapshot) { const auto constraint_data = getConstraintData(metadata_snapshot); + Poco::Logger::get("BEFORE CNF ").information(select_query->where()->dumpTree()); auto cnf = TreeCNFConverter::toCNF(select_query->where()); Poco::Logger::get("BEFORE OPT").information(cnf.dump()); cnf.pullNotOutFunctions() .filterAlwaysTrueGroups([&constraint_data](const auto & group) { /// remove always true groups from CNF return !checkIfGroupAlwaysTrue(group, constraint_data); }) - .filterAlwaysFalseAtoms([&constraint_data](const auto & ast) { /// remove always false atoms from CNF - return !checkIfAtomAlwaysFalse(ast, constraint_data); + .filterAlwaysFalseAtoms([&constraint_data](const auto & atom) { /// remove always false atoms from CNF + return !checkIfAtomAlwaysFalse(atom, constraint_data); }) .pushNotInFuntions(); diff --git a/src/Storages/ConstraintsDescription.h b/src/Storages/ConstraintsDescription.h index 3bdd9bc5503..219dce89851 100644 --- a/src/Storages/ConstraintsDescription.h +++ b/src/Storages/ConstraintsDescription.h @@ -2,6 +2,7 @@ #include #include +#include namespace DB { @@ -11,6 +12,7 @@ using ConstraintsExpressions = std::vector; struct ConstraintsDescription { std::vector constraints; + std::vector cnf_constraints; ConstraintsDescription() = default; @@ -27,6 +29,9 @@ struct ConstraintsDescription }; ASTs filterConstraints(ConstraintType selection) const; + // TODO: перенести преобразование в КНФ + get constraitns + //ASTs filterAtomicConstraints(ConstraintType selection) const; + //ASTs filterEqualConstraints(ConstraintType selection) const; ConstraintsExpressions getExpressionsToCheck(const Context & context, const NamesAndTypesList & source_columns_) const; From dc1996239b88f6c1a224b762c0adc3c71a147722 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 4 Mar 2021 15:16:39 +0300 Subject: [PATCH 009/200] test --- .../01622_constraints_simple_optimization.reference | 1 + .../0_stateless/01622_constraints_simple_optimization.sql | 3 +++ 2 files changed, 4 insertions(+) diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index 05243cf58eb..e23dadb814a 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -4,4 +4,5 @@ 0 4 4 +4 0 diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index d4f34a2dea7..f5416a2a888 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -1,7 +1,9 @@ DROP DATABASE IF EXISTS constraint_test; DROP TABLE IF EXISTS constraint_test.constrained; +SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; +SET optimize_move_to_prewhere = 1; CREATE DATABASE constraint_test; CREATE TABLE constraint_test.assumption (URL String, a Int32, CONSTRAINT c1 ASSUME domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT c2 ASSUME URL > 'zzz', CONSTRAINT c3 CHECK isValidUTF8(URL)) ENGINE = TinyLog; @@ -18,6 +20,7 @@ SELECT count() FROM constraint_test.assumption WHERE domainWithoutWWW(URL) != 'y SELECT count() FROM constraint_test.assumption WHERE domainWithoutWWW(URL) = 'nothing'; --- not optimized -> 0 SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL > 'zzz'); ---> assumption -> 4 +SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND NOT URL <= 'zzz'); ---> assumption -> 4 SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL > 'zzz') OR (a = 10 AND a + 5 < 100); ---> assumption -> 4 SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL = '111'); ---> assumption & no assumption -> 0 From 06a774e179b23adbad748fd4ec8fba32e2e07ffb Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Fri, 5 Mar 2021 12:53:03 +0300 Subject: [PATCH 010/200] test for graph --- ..._constraints_simple_optimization.reference | 4 +++ .../01622_constraints_simple_optimization.sql | 28 +++++++++++++++++-- 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index e23dadb814a..ec3bb632b0d 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -6,3 +6,7 @@ 4 4 0 +4 +1 +1 +1 diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index f5416a2a888..0bf62e1c488 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -1,12 +1,13 @@ DROP DATABASE IF EXISTS constraint_test; -DROP TABLE IF EXISTS constraint_test.constrained; +DROP TABLE IF EXISTS constraint_test.assumption; +DROP TABLE IF EXISTS constraint_test.transitivity; SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET optimize_move_to_prewhere = 1; CREATE DATABASE constraint_test; -CREATE TABLE constraint_test.assumption (URL String, a Int32, CONSTRAINT c1 ASSUME domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT c2 ASSUME URL > 'zzz', CONSTRAINT c3 CHECK isValidUTF8(URL)) ENGINE = TinyLog; +CREATE TABLE constraint_test.assumption (URL String, a Int32, CONSTRAINT c1 ASSUME domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT c2 ASSUME URL > 'zzz' AND startsWith(URL, 'test') = True) ENGINE = TinyLog; --- Add wrong rows in order to check optimization INSERT INTO constraint_test.assumption (URL, a) VALUES ('1', 1); @@ -23,6 +24,27 @@ SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'y SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND NOT URL <= 'zzz'); ---> assumption -> 4 SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL > 'zzz') OR (a = 10 AND a + 5 < 100); ---> assumption -> 4 SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL = '111'); ---> assumption & no assumption -> 0 +SELECT count() FROM constraint_test.assumption WHERE (startsWith(URL, 'test') = True); ---> assumption -> 4 + +-- DROP TABLE constraint_test.assumption; + +CREATE TABLE constraint_test.transitivity (a Int64, b Int64, c Int64, d Int32, CONSTRAINT c1 ASSUME a = b AND c = d, CONSTRAINT c2 ASSUME b = c) ENGINE = TinyLog; + +INSERT INTO constraint_test.transitivity (a, b, c, d) VALUES (1, 2, 3, 4); + +SELECT count() FROM constraint_test.transitivity WHERE a = d; ---> assumption -> 1 + +DROP TABLE constraint_test.transitivity; + + +CREATE TABLE constraint_test.strong_connectivity (a String, b String, c String, d String, CONSTRAINT c1 ASSUME a <= b AND b <= c AND c <= d AND d <= a) ENGINE = TinyLog; + +INSERT INTO constraint_test.strong_connectivity (a, b, c, d) VALUES ('1', '2', '3', '4'); + +SELECT count() FROM constraint_test.strong_connectivity WHERE a = d; ---> assumption -> 1 +SELECT count() FROM constraint_test.strong_connectivity WHERE a = c AND b = d; ---> assumption -> 1 + +DROP TABLE constraint_test.strong_connectivity; + -DROP TABLE constraint_test.assumption; DROP DATABASE constraint_test; \ No newline at end of file From 31caff2113c0cd3f1501e1b49b541d4689b7f76f Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Fri, 5 Mar 2021 12:54:13 +0300 Subject: [PATCH 011/200] graph --- src/Interpreters/ComparisonGraph.cpp | 253 +++++++++++++++++- src/Interpreters/ComparisonGraph.h | 28 +- src/Interpreters/TreeCNFConverter.h | 2 + .../WhereConstraintsOptimizer.cpp | 105 ++++++-- 4 files changed, 358 insertions(+), 30 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 0a281848e59..3318da7e549 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -1,21 +1,148 @@ #include +#include +#include +#include #include +#include namespace DB { -ComparisonGraph::ComparisonGraph(const std::vector & /*atomic_formulas*/) +/// make function a < b or a <= b +ASTPtr ComparisonGraph::normalizeAtom(const ASTPtr & atom) const { + static const std::map inverse_relations = { + {"greaterOrEquals", "less"}, + {"greater", "lessOrEquals"}, + }; + + ASTPtr res = atom->clone(); + { + auto * func = res->as(); + if (func) + { + if (const auto it = inverse_relations.find(func->name); it != std::end(inverse_relations)) + { + res = makeASTFunction(it->second, func->arguments->children[1]->clone(), func->arguments->children[0]->clone()); + } + } + } + + return res; +} + +ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) +{ + static const std::map relation_to_enum = { + {"equals", Edge::Type::EQUAL}, + {"less", Edge::Type::LESS}, + {"lessOrEquals", Edge::Type::LESS_OR_EQUAL}, + }; + + Graph g; + for (const auto & atom_raw : atomic_formulas) { + const auto atom = normalizeAtom(atom_raw); + + const auto bad_term = std::numeric_limits::max(); + auto get_index = [](const ASTPtr & ast, Graph & asts_graph) -> std::size_t { + const auto it = asts_graph.ast_hash_to_component.find(ast->getTreeHash()); + if (it != std::end(asts_graph.ast_hash_to_component)) + { + if (!std::any_of( + std::cbegin(asts_graph.vertexes[it->second].asts), + std::cend(asts_graph.vertexes[it->second].asts), + [ast](const ASTPtr & constraint_ast) { + return constraint_ast->getTreeHash() == ast->getTreeHash() + && constraint_ast->getColumnName() == ast->getColumnName(); + })) + { + return bad_term; + } + + return it->second; + } + else + { + asts_graph.ast_hash_to_component[ast->getTreeHash()] = asts_graph.vertexes.size(); + asts_graph.vertexes.push_back(EqualComponent{{ast}}); + asts_graph.edges.emplace_back(); + return asts_graph.vertexes.size() - 1; + } + }; + + const auto * func = atom->as(); + if (func) + { + if (const auto it = relation_to_enum.find(func->name); it != std::end(relation_to_enum) && func->arguments->children.size() == 2) + { + const size_t index_left = get_index(func->arguments->children[0], g); + const size_t index_right = get_index(func->arguments->children[1], g); + + if (index_left != bad_term && index_right != bad_term) + { + Poco::Logger::get("Edges").information("GOOD: " + atom->dumpTree()); + Poco::Logger::get("Edges").information("left=" + std::to_string(index_left) + " right=" + std::to_string(index_right)); + Poco::Logger::get("Edges").information("sz=" + std::to_string(g.edges.size())); + g.edges[index_right].push_back(Edge{it->second, index_left}); + if (func->name == "equals") + { + Poco::Logger::get("Edges").information("right=" + std::to_string(index_left) + " left=" + std::to_string(index_right)); + g.edges[index_left].push_back(Edge{it->second, index_right}); + } + } + else + { + Poco::Logger::get("Edges").information("BAD: " + atom->dumpTree()); + } + } + } + } + + graph = BuildGraphFromAstsGraph(g); +} + +ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, const ASTPtr & right) const +{ + size_t start = 0; + size_t finish = 0; + { + /// TODO: check full ast + const auto it_left = graph.ast_hash_to_component.find(left->getTreeHash()); + const auto it_right = graph.ast_hash_to_component.find(right->getTreeHash()); + if (it_left == std::end(graph.ast_hash_to_component) || it_right == std::end(graph.ast_hash_to_component)) + { + Poco::Logger::get("Graph").information("not found"); + Poco::Logger::get("Graph").information(std::to_string(left->getTreeHash().second)); + Poco::Logger::get("Graph").information(std::to_string(right->getTreeHash().second)); + for (const auto & [hash, id] : graph.ast_hash_to_component) + { + Poco::Logger::get("Graph MAP").information(std::to_string(hash.second) + " " + std::to_string(id)); + } + return CompareResult::UNKNOWN; + } + else + { + start = it_left->second; + finish = it_right->second; + Poco::Logger::get("Graph").information("found:" + std::to_string(start) + " " + std::to_string(finish)); + } + } + + if (start == finish) + return CompareResult::EQUAL; + + /// TODO: precalculate in O(n^3) using Floyd–Warshall algorithm where < = -1 and =< = 0. + /// TODO: use it for less, greater and so on + return CompareResult::UNKNOWN; } std::vector ComparisonGraph::getEqual(const ASTPtr & ast) const { - const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash().second); + const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash()); if (hash_it != std::end(graph.ast_hash_to_component)) return {}; const size_t index = hash_it->second; - //const auto vertex_it = std::find(std::begin(graph.vertexes[index].asts), std::end(graph.vertexes[index].asts), ast, ); if (std::any_of( std::cbegin(graph.vertexes[index].asts), std::cend(graph.vertexes[index].asts), @@ -30,9 +157,125 @@ std::vector ComparisonGraph::getEqual(const ASTPtr & ast) const } } -ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAsts(const Graph & /*asts_graph*/) +void ComparisonGraph::dfsOrder(const Graph & asts_graph, size_t v, std::vector & visited, std::vector & order) const { - return {}; + visited[v] = true; + for (const auto & edge : asts_graph.edges[v]) + { + if (!visited[edge.to]) + { + dfsOrder(asts_graph, edge.to, visited, order); + } + } + order.push_back(v); +} + +ComparisonGraph::Graph ComparisonGraph::reverseGraph(const Graph & asts_graph) const +{ + Graph g; + g.ast_hash_to_component = asts_graph.ast_hash_to_component; + g.vertexes = asts_graph.vertexes; + g.edges.resize(g.vertexes.size()); + for (size_t v = 0; v < asts_graph.vertexes.size(); ++v) + { + for (const auto & edge : asts_graph.edges[v]) + { + g.edges[edge.to].push_back(Edge{edge.type, v}); + } + } + return asts_graph; +} + +void ComparisonGraph::dfsComponents( + const Graph & reversed_graph, size_t v, std::vector & components, const size_t not_visited, const size_t component) const +{ + components[v] = component; + for (const auto & edge : reversed_graph.edges[v]) + { + if (components[edge.to] == not_visited) + { + dfsComponents(reversed_graph, edge.to, components, not_visited, component); + } + } +} + +ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & asts_graph) const +{ + Poco::Logger::get("Graph").information("building"); + /// Find strongly connected component + const auto n = asts_graph.vertexes.size(); + + std::vector order; + { + std::vector visited(n, false); + for (size_t v = 0; v < n; ++v) + { + if (!visited[v]) + dfsOrder(asts_graph, v, visited, order); + } + } + + Poco::Logger::get("Graph").information("dfs1"); + + const auto not_visited = std::numeric_limits::max(); + std::vector components(n, not_visited); + size_t component = 0; + { + const Graph reversed_graph = reverseGraph(asts_graph); + for (const size_t v : order) + { + if (components[v] == not_visited) + { + dfsComponents(reversed_graph, v, components, not_visited, component); + ++component; + } + } + } + + Poco::Logger::get("Graph").information("dfs2"); + + Graph result; + result.vertexes.resize(component); + result.edges.resize(component); + for (const auto & [hash, index] : asts_graph.ast_hash_to_component) + { + result.ast_hash_to_component[hash] = components[index]; + result.vertexes[components[index]].asts.insert( + std::end(result.vertexes[components[index]].asts), + std::begin(asts_graph.vertexes[index].asts), + std::end(asts_graph.vertexes[index].asts)); // asts_graph has only one ast per vertex + } + + Poco::Logger::get("Graph").information("components: " + std::to_string(component)); + + for (size_t v = 0; v < n; ++v) + { + for (const auto & edge : asts_graph.edges[v]) + { + result.edges[components[v]].push_back(Edge{edge.type, components[edge.to]}); + } + // TODO: make edges unique (most strict) + } + + Poco::Logger::get("Graph").information("finish"); + + for (size_t v = 0; v < result.vertexes.size(); ++v) + { + std::stringstream s; + for (const auto & atom : result.vertexes[v].asts) + { + s << atom->getTreeHash().second << " "; + } + s << "|"; + for (const auto & atom : result.ast_hash_to_component) + { + s << atom.first.second << " -" << atom.second << " "; + } + + Poco::Logger::get("Graph").information(s.str()); + } + + return result; } } diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 5469e623786..522c3d4d4e8 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -22,20 +22,20 @@ public: EQUAL, GREATER_OR_EQUAL, GREATER, - //NOT_EQUAL, UNKNOWN, }; - // TODO: implement - CompareResult compare(const ASTPtr & /*left*/, const ASTPtr & /*right*/) const { return CompareResult::UNKNOWN; } + CompareResult compare(const ASTPtr & left, const ASTPtr & right) const; + CompareResult getExpectedCompare(const ASTPtr & ast) const; std::vector getEqual(const ASTPtr & ast) const; /// Find constants less and greater. /// For int and double linear programming can be applied here. // TODO: implement - ASTPtr getMax(const ASTPtr &) const { return nullptr; } // sup - ASTPtr getMin(const ASTPtr &) const { return nullptr; } // inf + //ASTPtr getMax(const ASTPtr &) const { return nullptr; } // sup + //ASTPtr getMin(const ASTPtr &) const { return nullptr; } // inf + //ASTPtr getEqualConst(const ASTPtr &) const { return nullptr; } // inf private: /// strongly connected component @@ -58,17 +58,29 @@ private: }; Type type; - EqualComponent to; + size_t to; }; struct Graph { - std::unordered_map ast_hash_to_component; + struct ASTHash { + size_t operator() (const IAST::Hash & hash) const { + return hash.first; + } + }; + + std::unordered_map ast_hash_to_component; std::vector vertexes; std::vector> edges; }; - Graph BuildGraphFromAsts(const Graph & asts_graph); + ASTPtr normalizeAtom(const ASTPtr & atom) const; + Graph BuildGraphFromAstsGraph(const Graph & asts_graph) const; + + Graph reverseGraph(const Graph & asts_graph) const; + void dfsOrder(const Graph & asts_graph, size_t v, std::vector & visited, std::vector & order) const; + void dfsComponents( + const Graph & reversed_graph, size_t v, std::vector & components, const size_t not_visited, const size_t component) const; Graph graph; }; diff --git a/src/Interpreters/TreeCNFConverter.h b/src/Interpreters/TreeCNFConverter.h index 51c5ae3aa43..5fc0ba36b6a 100644 --- a/src/Interpreters/TreeCNFConverter.h +++ b/src/Interpreters/TreeCNFConverter.h @@ -124,4 +124,6 @@ public: static ASTPtr fromCNF(const CNFQuery & cnf); }; +void pushNotIn(CNFQuery::AtomicFormula & atom); + } diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index b0ebaffdfed..3b6e8d09fbd 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -1,6 +1,8 @@ #include #include +#include +#include #include #include #include @@ -9,6 +11,13 @@ namespace DB { + +namespace ErrorCodes +{ +extern const int LOGICAL_ERROR; +} + + std::vector> getConstraintData(const StorageMetadataPtr & metadata_snapshot) { std::vector> constraint_data; @@ -31,7 +40,7 @@ std::vector getAtomicConstraintData(const StorageMetada metadata_snapshot->getConstraints().filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) { const auto cnf = TreeCNFConverter::toCNF(constraint->as()->expr->ptr()) - .pullNotOutFunctions(); /// TODO: move prepare stage to ConstraintsDescription + .pullNotOutFunctions(); for (const auto & group : cnf.getStatements()) { if (group.size() == 1) constraint_data.push_back(*group.begin()); @@ -41,20 +50,27 @@ std::vector getAtomicConstraintData(const StorageMetada return constraint_data; } -std::vector> getEqualConstraintData(const StorageMetadataPtr & metadata_snapshot) +ComparisonGraph getComparisonGraph(const StorageMetadataPtr & metadata_snapshot) { - std::vector> equal_constraints; + static const std::set relations = { + "equals", "less", "lessOrEquals", "greaterOrEquals", "greater"}; + + std::vector constraints_for_graph; + auto atomic_formulas = getAtomicConstraintData(metadata_snapshot); const std::vector atomic_constraints = getAtomicConstraintData(metadata_snapshot); - for (const auto & constraint : atomic_constraints) { - auto * func = constraint.ast->as(); - if (func && (func->name == "equal" && !constraint.negative)) + for (auto & atomic_formula : atomic_formulas) + { + pushNotIn(atomic_formula); + auto * func = atomic_formula.ast->as(); + if (func && relations.count(func->name)) { - equal_constraints.emplace_back( - func->arguments->children[0], - func->arguments->children[1]); + if (atomic_formula.negative) + throw Exception(": ", ErrorCodes::LOGICAL_ERROR); + constraints_for_graph.push_back(atomic_formula.ast); } } - return equal_constraints; + + return ComparisonGraph(constraints_for_graph); } WhereConstraintsOptimizer::WhereConstraintsOptimizer( @@ -93,7 +109,7 @@ MatchState match(CNFQuery::AtomicFormula a, CNFQuery::AtomicFormula b) return MatchState::NONE; } -bool checkIfGroupAlwaysTrue(const CNFQuery::OrGroup & group, const std::vector> & constraints) +bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const std::vector> & constraints) { /// TODO: constraints graph @@ -106,9 +122,9 @@ bool checkIfGroupAlwaysTrue(const CNFQuery::OrGroup & group, const std::vector> & constraints) +ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula & atom) +{ + static const std::map inverse_relations = { + {"equals", "notEquals"}, + {"less", "greaterOrEquals"}, + {"lessOrEquals", "greater"}, + {"notEquals", "equals"}, + {"greaterOrEquals", "less"}, + {"greater", "lessOrEquals"}, + }; + + static const std::map relation_to_compare = { + {"equals", ComparisonGraph::CompareResult::EQUAL}, + {"less", ComparisonGraph::CompareResult::LESS}, + {"lessOrEquals", ComparisonGraph::CompareResult::LESS_OR_EQUAL}, + {"notEquals", ComparisonGraph::CompareResult::UNKNOWN}, + {"greaterOrEquals", ComparisonGraph::CompareResult::GREATER_OR_EQUAL}, + {"greater", ComparisonGraph::CompareResult::GREATER}, + }; + + + const auto * func = atom.ast->as(); + if (func && inverse_relations.count(func->name)) + { + std::string function_name = func->name; + if (atom.negative) + { + function_name = inverse_relations.at(func->name); + } + return relation_to_compare.at(function_name); + } + return ComparisonGraph::CompareResult::UNKNOWN; +} + + +bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const ComparisonGraph & graph) +{ + for (const auto & atom : group) + { + const auto * func = atom.ast->as(); + if (func && func->arguments->children.size() == 2) + { + const auto expected = getExpectedCompare(atom); + const auto result = graph.compare(func->arguments->children[0], func->arguments->children[1]); + Poco::Logger::get("GRAPH REASON").information("neg: " + std::to_string(atom.negative)); + Poco::Logger::get("GRAPH REASON").information(atom.ast->dumpTree()); + Poco::Logger::get("GRAPH REASON").information(std::to_string(static_cast(expected)) + " " + std::to_string(static_cast(result))); + return expected != ComparisonGraph::CompareResult::UNKNOWN && expected == result; + } + } + return false; +} + + +bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const std::vector> & constraints) { /// TODO: more efficient matching @@ -155,15 +225,16 @@ void WhereConstraintsOptimizer::perform() if (select_query->where() && metadata_snapshot) { const auto constraint_data = getConstraintData(metadata_snapshot); + const auto compare_graph = getComparisonGraph(metadata_snapshot); Poco::Logger::get("BEFORE CNF ").information(select_query->where()->dumpTree()); auto cnf = TreeCNFConverter::toCNF(select_query->where()); Poco::Logger::get("BEFORE OPT").information(cnf.dump()); cnf.pullNotOutFunctions() - .filterAlwaysTrueGroups([&constraint_data](const auto & group) { /// remove always true groups from CNF - return !checkIfGroupAlwaysTrue(group, constraint_data); + .filterAlwaysTrueGroups([&constraint_data, &compare_graph](const auto & group) { /// remove always true groups from CNF + return !checkIfGroupAlwaysTrueFullMatch(group, constraint_data) && !checkIfGroupAlwaysTrueGraph(group, compare_graph); }) .filterAlwaysFalseAtoms([&constraint_data](const auto & atom) { /// remove always false atoms from CNF - return !checkIfAtomAlwaysFalse(atom, constraint_data); + return !checkIfAtomAlwaysFalseFullMatch(atom, constraint_data); }) .pushNotInFuntions(); From 5a2fa1aba717d7c2f43d01e664174acaecc31368 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Fri, 5 Mar 2021 13:33:06 +0300 Subject: [PATCH 012/200] fix --- src/Interpreters/WhereConstraintsOptimizer.cpp | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index 3b6e8d09fbd..1ea2b8458e9 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -192,7 +192,19 @@ bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const Comparis Poco::Logger::get("GRAPH REASON").information("neg: " + std::to_string(atom.negative)); Poco::Logger::get("GRAPH REASON").information(atom.ast->dumpTree()); Poco::Logger::get("GRAPH REASON").information(std::to_string(static_cast(expected)) + " " + std::to_string(static_cast(result))); - return expected != ComparisonGraph::CompareResult::UNKNOWN && expected == result; + + if (expected == ComparisonGraph::CompareResult::UNKNOWN) + return false; + + if (expected == result) + return true; + if (result == ComparisonGraph::CompareResult::EQUAL && + (expected == ComparisonGraph::CompareResult::LESS_OR_EQUAL || expected == ComparisonGraph::CompareResult::GREATER_OR_EQUAL)) + return true; + if (result == ComparisonGraph::CompareResult::LESS && expected == ComparisonGraph::CompareResult::LESS_OR_EQUAL) + return true; + if (result == ComparisonGraph::CompareResult::GREATER && expected == ComparisonGraph::CompareResult::GREATER_OR_EQUAL) + return true; } } return false; From 06454bd912b1bb0edc32125d61a6de1b65189ab4 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Fri, 5 Mar 2021 15:13:00 +0300 Subject: [PATCH 013/200] fix --- src/Interpreters/ComparisonGraph.cpp | 43 ++++++++++++++++++- src/Interpreters/ComparisonGraph.h | 3 +- .../WhereConstraintsOptimizer.cpp | 38 ++++++++++++++-- 3 files changed, 79 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 3318da7e549..4b58ff94adc 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -102,6 +102,38 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) graph = BuildGraphFromAstsGraph(g); } +std::pair ComparisonGraph::findPath(const size_t start, const size_t finish) const +{ + // min path : < = -1, =< = 0 + const auto inf = std::numeric_limits::max(); + const size_t n = graph.vertexes.size(); + std::vector dist(n, inf); + dist[start] = 0; + for (size_t k = 0; k < n; ++k) + { + bool has_relaxation = false; + for (size_t v = 0; v < n; ++v) + { + if (dist[v] == inf) + continue; + + for (const auto & edge : graph.edges[v]) + { + const int64_t weight = edge.type == Edge::Type::LESS ? -1 : 0; + if (dist[edge.to] > dist[v] + weight) + { + dist[edge.to] = dist[v] + weight; + has_relaxation = true; + } + } + } + + if (has_relaxation) + break; + } + return {dist[finish] != inf, dist[finish] < 0}; +} + ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, const ASTPtr & right) const { size_t start = 0; @@ -132,8 +164,17 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con if (start == finish) return CompareResult::EQUAL; - /// TODO: precalculate in O(n^3) using Floyd–Warshall algorithm where < = -1 and =< = 0. + /// TODO: precalculate using Floyd–Warshall O(n^3) algorithm where < = -1 and =< = 0. /// TODO: use it for less, greater and so on + + auto [has_path, is_strict] = findPath(start, finish); + if (has_path) + return is_strict ? CompareResult::GREATER : CompareResult::GREATER_OR_EQUAL; + + auto [has_path_r, is_strict_r] = findPath(finish, start); + if (has_path_r) + return is_strict_r ? CompareResult::LESS : CompareResult::LESS_OR_EQUAL; + return CompareResult::UNKNOWN; } diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 522c3d4d4e8..de1a9efaa07 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -26,7 +26,6 @@ public: }; CompareResult compare(const ASTPtr & left, const ASTPtr & right) const; - CompareResult getExpectedCompare(const ASTPtr & ast) const; std::vector getEqual(const ASTPtr & ast) const; @@ -82,6 +81,8 @@ private: void dfsComponents( const Graph & reversed_graph, size_t v, std::vector & components, const size_t not_visited, const size_t component) const; + std::pair findPath(const size_t start, const size_t finish) const; + Graph graph; }; diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index 1ea2b8458e9..bfd8e5e2a34 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -193,7 +193,7 @@ bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const Comparis Poco::Logger::get("GRAPH REASON").information(atom.ast->dumpTree()); Poco::Logger::get("GRAPH REASON").information(std::to_string(static_cast(expected)) + " " + std::to_string(static_cast(result))); - if (expected == ComparisonGraph::CompareResult::UNKNOWN) + if (expected == ComparisonGraph::CompareResult::UNKNOWN || result == ComparisonGraph::CompareResult::UNKNOWN) return false; if (expected == result) @@ -232,6 +232,38 @@ bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const return false; } +bool checkIfAtomAlwaysFalseGraph(const CNFQuery::AtomicFormula & atom, const ComparisonGraph & graph) +{ + const auto * func = atom.ast->as(); + if (func && func->arguments->children.size() == 2) + { + /// TODO: special support for != + const auto expected = getExpectedCompare(atom); + const auto result = graph.compare(func->arguments->children[0], func->arguments->children[1]); + Poco::Logger::get("GRAPH REASON F").information("neg: " + std::to_string(atom.negative)); + Poco::Logger::get("GRAPH REASON F").information(atom.ast->dumpTree()); + Poco::Logger::get("GRAPH REASON F").information(std::to_string(static_cast(expected)) + " " + std::to_string(static_cast(result))); + + if (expected == ComparisonGraph::CompareResult::UNKNOWN || result == ComparisonGraph::CompareResult::UNKNOWN) + return false; + + if (expected == result) + return false; + else if (result == ComparisonGraph::CompareResult::EQUAL && + (expected == ComparisonGraph::CompareResult::LESS_OR_EQUAL || expected == ComparisonGraph::CompareResult::GREATER_OR_EQUAL)) + return false; + else if (result == ComparisonGraph::CompareResult::LESS && expected == ComparisonGraph::CompareResult::LESS_OR_EQUAL) + return false; + else if (result == ComparisonGraph::CompareResult::GREATER && expected == ComparisonGraph::CompareResult::GREATER_OR_EQUAL) + return false; + else + return true; + } + + return false; +} + + void WhereConstraintsOptimizer::perform() { if (select_query->where() && metadata_snapshot) @@ -245,8 +277,8 @@ void WhereConstraintsOptimizer::perform() .filterAlwaysTrueGroups([&constraint_data, &compare_graph](const auto & group) { /// remove always true groups from CNF return !checkIfGroupAlwaysTrueFullMatch(group, constraint_data) && !checkIfGroupAlwaysTrueGraph(group, compare_graph); }) - .filterAlwaysFalseAtoms([&constraint_data](const auto & atom) { /// remove always false atoms from CNF - return !checkIfAtomAlwaysFalseFullMatch(atom, constraint_data); + .filterAlwaysFalseAtoms([&constraint_data, &compare_graph](const auto & atom) { /// remove always false atoms from CNF + return !checkIfAtomAlwaysFalseFullMatch(atom, constraint_data) && !checkIfAtomAlwaysFalseGraph(atom, compare_graph); }) .pushNotInFuntions(); From add3004d69f3237b0ccd36b96d8971e2672b129d Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Fri, 5 Mar 2021 15:13:25 +0300 Subject: [PATCH 014/200] fix --- ...622_constraints_simple_optimization.reference | 7 +++++++ .../01622_constraints_simple_optimization.sql | 16 +++++++++++++++- 2 files changed, 22 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index ec3bb632b0d..298b6ce1e12 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -10,3 +10,10 @@ 1 1 1 +0 +1 +1 +1 +1 +0 +0 diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index 0bf62e1c488..011d2709979 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -1,6 +1,7 @@ DROP DATABASE IF EXISTS constraint_test; DROP TABLE IF EXISTS constraint_test.assumption; DROP TABLE IF EXISTS constraint_test.transitivity; +DROP TABLE IF EXISTS constraint_test.transitivity2; SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; @@ -26,7 +27,7 @@ SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'y SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL = '111'); ---> assumption & no assumption -> 0 SELECT count() FROM constraint_test.assumption WHERE (startsWith(URL, 'test') = True); ---> assumption -> 4 --- DROP TABLE constraint_test.assumption; +DROP TABLE constraint_test.assumption; CREATE TABLE constraint_test.transitivity (a Int64, b Int64, c Int64, d Int32, CONSTRAINT c1 ASSUME a = b AND c = d, CONSTRAINT c2 ASSUME b = c) ENGINE = TinyLog; @@ -43,8 +44,21 @@ INSERT INTO constraint_test.strong_connectivity (a, b, c, d) VALUES ('1', '2', ' SELECT count() FROM constraint_test.strong_connectivity WHERE a = d; ---> assumption -> 1 SELECT count() FROM constraint_test.strong_connectivity WHERE a = c AND b = d; ---> assumption -> 1 +SELECT count() FROM constraint_test.strong_connectivity WHERE a < c OR b < d; ---> assumption -> 0 +SELECT count() FROM constraint_test.strong_connectivity WHERE a <= c OR b <= d; ---> assumption -> 1 DROP TABLE constraint_test.strong_connectivity; +CREATE TABLE constraint_test.transitivity2 (a String, b String, c String, d String, CONSTRAINT c1 ASSUME a > b AND b >= c AND c > d AND a >= d) ENGINE = TinyLog; + +INSERT INTO constraint_test.transitivity2 (a, b, c, d) VALUES ('1', '2', '3', '4'); + +SELECT count() FROM constraint_test.transitivity2 WHERE a > d; ---> assumption -> 1 +SELECT count() FROM constraint_test.transitivity2 WHERE a >= d; ---> assumption -> 1 +SELECT count() FROM constraint_test.transitivity2 WHERE d < a; ---> assumption -> 1 +SELECT count() FROM constraint_test.transitivity2 WHERE a < d; ---> assumption -> 0 +SELECT count() FROM constraint_test.transitivity2 WHERE a = d; ---> assumption -> 0 + +DROP TABLE constraint_test.transitivity2; DROP DATABASE constraint_test; \ No newline at end of file From d97e1a54f3fc86848ab83d7d7a639ee50481d474 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Fri, 5 Mar 2021 15:46:27 +0300 Subject: [PATCH 015/200] test const replace --- .../01622_constraints_simple_optimization.reference | 4 ++++ .../01622_constraints_simple_optimization.sql | 12 ++++++++++++ 2 files changed, 16 insertions(+) diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index 298b6ce1e12..d09aa7baef4 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -17,3 +17,7 @@ 1 0 0 +1 +0 +0 +1 diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index 011d2709979..148321f129f 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -61,4 +61,16 @@ SELECT count() FROM constraint_test.transitivity2 WHERE a = d; ---> assumption - DROP TABLE constraint_test.transitivity2; + +CREATE TABLE constraint_test.constants_repl (a Int64, b Int64, c Int64, d Int64, CONSTRAINT c1 ASSUME a - b = 10 AND c + d = 20) ENGINE = TinyLog; + +INSERT INTO constraint_test.constants_repl (a, b, c, d) VALUES (1, 2, 3, 4); + +SELECT count() FROM constraint_test.constants_repl WHERE a - b = 10; ---> assumption -> 1 +SELECT count() FROM constraint_test.constants_repl WHERE a - b < 0; ---> assumption -> 0 +SELECT count() FROM constraint_test.constants_repl WHERE a - b = c + d; ---> assumption -> 0 +SELECT count() FROM constraint_test.constants_repl WHERE (a - b) * 2 = c + d; ---> assumption -> 1 + +DROP TABLE constraint_test.constants_repl; + DROP DATABASE constraint_test; \ No newline at end of file From 7247490124641d383fd33e37087673e3cbecac5d Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Fri, 5 Mar 2021 15:46:42 +0300 Subject: [PATCH 016/200] const replace --- src/Interpreters/ComparisonGraph.cpp | 18 ++++++++++++- src/Interpreters/ComparisonGraph.h | 6 ++++- .../WhereConstraintsOptimizer.cpp | 27 +++++++++++++++++++ 3 files changed, 49 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 4b58ff94adc..a2116b0c6fa 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -181,7 +182,7 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con std::vector ComparisonGraph::getEqual(const ASTPtr & ast) const { const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash()); - if (hash_it != std::end(graph.ast_hash_to_component)) + if (hash_it == std::end(graph.ast_hash_to_component)) return {}; const size_t index = hash_it->second; if (std::any_of( @@ -198,6 +199,21 @@ std::vector ComparisonGraph::getEqual(const ASTPtr & ast) const } } +std::optional ComparisonGraph::getEqualConst(const ASTPtr & ast) const +{ + const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash()); + if (hash_it == std::end(graph.ast_hash_to_component)) + return std::nullopt; + const size_t index = hash_it->second; + for (const auto & term : graph.vertexes[index].asts) + { + const ASTLiteral * lit = term->as(); + if (lit) + return term; + } + return std::nullopt; +} + void ComparisonGraph::dfsOrder(const Graph & asts_graph, size_t v, std::vector & visited, std::vector & order) const { visited[v] = true; diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index de1a9efaa07..f76b491997d 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -8,6 +8,10 @@ namespace DB { +/* + * Graph of relations between terms in constraints. + * Allows to compare terms and get equal terms. + */ class ComparisonGraph { public: @@ -28,13 +32,13 @@ public: CompareResult compare(const ASTPtr & left, const ASTPtr & right) const; std::vector getEqual(const ASTPtr & ast) const; + std::optional getEqualConst(const ASTPtr & ast) const; /// Find constants less and greater. /// For int and double linear programming can be applied here. // TODO: implement //ASTPtr getMax(const ASTPtr &) const { return nullptr; } // sup //ASTPtr getMin(const ASTPtr &) const { return nullptr; } // inf - //ASTPtr getEqualConst(const ASTPtr &) const { return nullptr; } // inf private: /// strongly connected component diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index bfd8e5e2a34..64ff894bb51 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -263,6 +263,30 @@ bool checkIfAtomAlwaysFalseGraph(const CNFQuery::AtomicFormula & atom, const Com return false; } +void replaceToConstants(ASTPtr & term, const ComparisonGraph & graph) +{ + const auto equal_constant = graph.getEqualConst(term); + if (equal_constant) + { + term = (*equal_constant)->clone(); + } + else + { + for (auto & child : term->children) + replaceToConstants(child, graph); + } +} + +CNFQuery::AtomicFormula replaceTermsToConstants(const CNFQuery::AtomicFormula & atom, const ComparisonGraph & graph) +{ + CNFQuery::AtomicFormula result; + result.negative = atom.negative; + result.ast = atom.ast->clone(); + + replaceToConstants(result.ast, graph); + + return result; +} void WhereConstraintsOptimizer::perform() { @@ -280,6 +304,9 @@ void WhereConstraintsOptimizer::perform() .filterAlwaysFalseAtoms([&constraint_data, &compare_graph](const auto & atom) { /// remove always false atoms from CNF return !checkIfAtomAlwaysFalseFullMatch(atom, constraint_data) && !checkIfAtomAlwaysFalseGraph(atom, compare_graph); }) + .transformAtoms([&compare_graph](const auto & atom) { + return replaceTermsToConstants(atom, compare_graph); + }) .pushNotInFuntions(); Poco::Logger::get("AFTER OPT").information(cnf.dump()); From 7478159d868d7227827e359d114f423b2ac9c584 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sat, 6 Mar 2021 16:06:58 +0300 Subject: [PATCH 017/200] fix logs --- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index d5808bca1c5..cb107c5f7d6 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -525,11 +525,11 @@ MergeTreeRangeReader::MergeTreeRangeReader( , last_reader_in_chain(last_reader_in_chain_) , is_initialized(true) { - if (prewhere_) + if (prewhere_info_) { - Poco::Logger::get("PREWHERE").information(prewhere_->prewhere_column_name); - Poco::Logger::get("PREWHERE").information(prewhere_->prewhere_actions->dumpActions()); - Poco::Logger::get("PREWHERE rm?").information(std::to_string(prewhere_->remove_prewhere_column)); + Poco::Logger::get("PREWHERE").information(prewhere_info_->prewhere_column_name); + Poco::Logger::get("PREWHERE").information(prewhere_info_->prewhere_actions->dumpActions()); + Poco::Logger::get("PREWHERE rm?").information(std::to_string(prewhere_info_->remove_prewhere_column)); } if (prev_reader) sample_block = prev_reader->getSampleBlock(); From 08206ab20b08829aa8f6582e4e9c3c02ae949b23 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sat, 3 Apr 2021 15:12:45 +0300 Subject: [PATCH 018/200] fix --- src/Interpreters/ComparisonGraph.h | 6 +- .../WhereConstraintsOptimizer.cpp | 67 +------------------ src/Storages/ConstraintsDescription.cpp | 53 +++++++++++++++ src/Storages/ConstraintsDescription.h | 7 +- 4 files changed, 65 insertions(+), 68 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index f76b491997d..550c7080128 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -34,11 +34,11 @@ public: std::vector getEqual(const ASTPtr & ast) const; std::optional getEqualConst(const ASTPtr & ast) const; - /// Find constants less and greater. + /// Find constants lessOrEqual and greaterOrEqual. /// For int and double linear programming can be applied here. // TODO: implement - //ASTPtr getMax(const ASTPtr &) const { return nullptr; } // sup - //ASTPtr getMin(const ASTPtr &) const { return nullptr; } // inf + //ASTPtr getUpperBound(const ASTPtr &) const { return nullptr; } // sup + //ASTPtr getLowerBound(const ASTPtr &) const { return nullptr; } // inf private: /// strongly connected component diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index aefdaab8522..95e5fd35046 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -17,62 +17,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } - -std::vector> getConstraintData(const StorageMetadataPtr & metadata_snapshot) -{ - std::vector> constraint_data; - for (const auto & constraint : - metadata_snapshot->getConstraints().filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) - { - const auto cnf = TreeCNFConverter::toCNF(constraint->as()->expr->ptr()) - .pullNotOutFunctions(); /// TODO: move prepare stage to ConstraintsDescription - for (const auto & group : cnf.getStatements()) - constraint_data.emplace_back(std::begin(group), std::end(group)); - } - - return constraint_data; -} - -std::vector getAtomicConstraintData(const StorageMetadataPtr & metadata_snapshot) -{ - std::vector constraint_data; - for (const auto & constraint : - metadata_snapshot->getConstraints().filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) - { - const auto cnf = TreeCNFConverter::toCNF(constraint->as()->expr->ptr()) - .pullNotOutFunctions(); - for (const auto & group : cnf.getStatements()) { - if (group.size() == 1) - constraint_data.push_back(*group.begin()); - } - } - - return constraint_data; -} - -ComparisonGraph getComparisonGraph(const StorageMetadataPtr & metadata_snapshot) -{ - static const std::set relations = { - "equals", "less", "lessOrEquals", "greaterOrEquals", "greater"}; - - std::vector constraints_for_graph; - auto atomic_formulas = getAtomicConstraintData(metadata_snapshot); - const std::vector atomic_constraints = getAtomicConstraintData(metadata_snapshot); - for (auto & atomic_formula : atomic_formulas) - { - pushNotIn(atomic_formula); - auto * func = atomic_formula.ast->as(); - if (func && relations.count(func->name)) - { - if (atomic_formula.negative) - throw Exception(": ", ErrorCodes::LOGICAL_ERROR); - constraints_for_graph.push_back(atomic_formula.ast); - } - } - - return ComparisonGraph(constraints_for_graph); -} - WhereConstraintsOptimizer::WhereConstraintsOptimizer( ASTSelectQuery * select_query_, Aliases & /*aliases_*/, @@ -111,10 +55,7 @@ MatchState match(CNFQuery::AtomicFormula a, CNFQuery::AtomicFormula b) bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const std::vector> & constraints) { - /// TODO: this is temporary; need to write more effective search - /// TODO: go deeper into asts (a < b, a = b,...) with z3 or some visitor - for (const auto & constraint : constraints) /// one constraint in group is enough, - /// otherwise it's difficult to make judgements without using constraint solving (z3..) + for (const auto & constraint : constraints) { bool group_always_true = true; for (const auto & constraint_ast : constraint) @@ -211,8 +152,6 @@ bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const Comparis bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const std::vector> & constraints) { - /// TODO: more efficient matching - for (const auto & constraint : constraints) { if (constraint.size() > 1) @@ -290,8 +229,8 @@ void WhereConstraintsOptimizer::perform() { if (select_query->where() && metadata_snapshot) { - const auto constraint_data = getConstraintData(metadata_snapshot); - const auto compare_graph = getComparisonGraph(metadata_snapshot); + const auto constraint_data = metadata_snapshot->getConstraints().getConstraintData(); + const auto compare_graph = metadata_snapshot->getConstraints().getGraph(); Poco::Logger::get("BEFORE CNF ").information(select_query->where()->dumpTree()); auto cnf = TreeCNFConverter::toCNF(select_query->where()); Poco::Logger::get("BEFORE OPT").information(cnf.dump()); diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index 5336079aa21..f8de4036b6a 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include @@ -65,6 +66,58 @@ ASTs ConstraintsDescription::filterConstraints(ConstraintType selection) const return res; } +std::vector> ConstraintsDescription::getConstraintData() const +{ + std::vector> constraint_data; + for (const auto & constraint : filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) + { + const auto cnf = TreeCNFConverter::toCNF(constraint->as()->expr->ptr()) + .pullNotOutFunctions(); /// TODO: move prepare stage to ConstraintsDescription + for (const auto & group : cnf.getStatements()) + constraint_data.emplace_back(std::begin(group), std::end(group)); + } + + return constraint_data; +} + +std::vector ConstraintsDescription::getAtomicConstraintData() const +{ + std::vector constraint_data; + for (const auto & constraint : filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) + { + const auto cnf = TreeCNFConverter::toCNF(constraint->as()->expr->ptr()) + .pullNotOutFunctions(); + for (const auto & group : cnf.getStatements()) { + if (group.size() == 1) + constraint_data.push_back(*group.begin()); + } + } + + return constraint_data; +} + +ComparisonGraph ConstraintsDescription::getGraph() const +{ + static const std::set relations = { + "equals", "less", "lessOrEquals", "greaterOrEquals", "greater"}; + + std::vector constraints_for_graph; + auto atomic_formulas = getAtomicConstraintData(); + for (auto & atomic_formula : atomic_formulas) + { + pushNotIn(atomic_formula); + auto * func = atomic_formula.ast->as(); + if (func && relations.count(func->name)) + { + if (atomic_formula.negative) + throw Exception(": ", ErrorCodes::LOGICAL_ERROR); + constraints_for_graph.push_back(atomic_formula.ast); + } + } + + return ComparisonGraph(constraints_for_graph); +} + ConstraintsExpressions ConstraintsDescription::getExpressionsToCheck(const DB::Context & context, const DB::NamesAndTypesList & source_columns_) const { diff --git a/src/Storages/ConstraintsDescription.h b/src/Storages/ConstraintsDescription.h index 219dce89851..89c3f82ad84 100644 --- a/src/Storages/ConstraintsDescription.h +++ b/src/Storages/ConstraintsDescription.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -31,7 +32,11 @@ struct ConstraintsDescription ASTs filterConstraints(ConstraintType selection) const; // TODO: перенести преобразование в КНФ + get constraitns //ASTs filterAtomicConstraints(ConstraintType selection) const; - //ASTs filterEqualConstraints(ConstraintType selection) const; + + std::vector> getConstraintData() const; + std::vector getAtomicConstraintData() const; + + ComparisonGraph getGraph() const; ConstraintsExpressions getExpressionsToCheck(const Context & context, const NamesAndTypesList & source_columns_) const; From d868b52d37d1d6217588b86eb29d19483ab1d5ce Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sat, 3 Apr 2021 19:30:49 +0300 Subject: [PATCH 019/200] calc --- src/Interpreters/ComparisonGraph.cpp | 68 ++++++++++++++------------- src/Interpreters/ComparisonGraph.h | 10 ++++ src/Storages/ConstraintsDescription.h | 3 +- 3 files changed, 46 insertions(+), 35 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index a2116b0c6fa..b89bb57155f 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -82,13 +82,13 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) if (index_left != bad_term && index_right != bad_term) { - Poco::Logger::get("Edges").information("GOOD: " + atom->dumpTree()); - Poco::Logger::get("Edges").information("left=" + std::to_string(index_left) + " right=" + std::to_string(index_right)); - Poco::Logger::get("Edges").information("sz=" + std::to_string(g.edges.size())); + //Poco::Logger::get("Edges").information("GOOD: " + atom->dumpTree()); + //Poco::Logger::get("Edges").information("left=" + std::to_string(index_left) + " right=" + std::to_string(index_right)); + //Poco::Logger::get("Edges").information("sz=" + std::to_string(g.edges.size())); g.edges[index_right].push_back(Edge{it->second, index_left}); if (func->name == "equals") { - Poco::Logger::get("Edges").information("right=" + std::to_string(index_left) + " left=" + std::to_string(index_right)); + //Poco::Logger::get("Edges").information("right=" + std::to_string(index_left) + " left=" + std::to_string(index_right)); g.edges[index_left].push_back(Edge{it->second, index_right}); } } @@ -101,38 +101,16 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) } graph = BuildGraphFromAstsGraph(g); + dists = BuildDistsFromGraph(graph); } std::pair ComparisonGraph::findPath(const size_t start, const size_t finish) const { - // min path : < = -1, =< = 0 - const auto inf = std::numeric_limits::max(); - const size_t n = graph.vertexes.size(); - std::vector dist(n, inf); - dist[start] = 0; - for (size_t k = 0; k < n; ++k) - { - bool has_relaxation = false; - for (size_t v = 0; v < n; ++v) - { - if (dist[v] == inf) - continue; - - for (const auto & edge : graph.edges[v]) - { - const int64_t weight = edge.type == Edge::Type::LESS ? -1 : 0; - if (dist[edge.to] > dist[v] + weight) - { - dist[edge.to] = dist[v] + weight; - has_relaxation = true; - } - } - } - - if (has_relaxation) - break; - } - return {dist[finish] != inf, dist[finish] < 0}; + const auto it = dists.find(std::make_pair(start, finish)); + if (it == std::end(dists)) + return {false, false}; + else + return {true, it->second == Path::LESS}; } ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, const ASTPtr & right) const @@ -311,7 +289,7 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as { result.edges[components[v]].push_back(Edge{edge.type, components[edge.to]}); } - // TODO: make edges unique (most strict) + // TODO: make edges unique (left most strict) } Poco::Logger::get("Graph").information("finish"); @@ -335,4 +313,28 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as return result; } +std::map, ComparisonGraph::Path> ComparisonGraph::BuildDistsFromGraph(const Graph & g) const +{ + // min path : < = -1, =< = 0 + const auto inf = std::numeric_limits::max(); + const size_t n = graph.vertexes.size(); + std::vector> results(n, std::vector(n, inf)); + for (size_t v = 0; v < n; ++v) + for (const auto & edge : g.edges[v]) + results[v][edge.to] = (edge.type == Edge::LESS ? -1 : 0); + + for (size_t k = 0; k < n; ++k) + for (size_t v = 0; v < n; ++v) + for (size_t u = 0; u < n; ++u) + if (results[v][k] != inf && results[k][u] != inf) + results[v][u] = std::min(results[v][u], std::min(results[v][k], results[k][u])); + + std::map, Path> path; + for (size_t v = 0; v < n; ++v) + for (size_t u = 0; u < n; ++u) + if (results[v][u] != inf) + path[std::make_pair(v, u)] = (results[v][u] == -1 ? Path::LESS : Path::LESS_OR_EQUAL); + return path; +} + } diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 550c7080128..ffdc10e13e6 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -87,7 +88,16 @@ private: std::pair findPath(const size_t start, const size_t finish) const; + enum class Path + { + LESS, + LESS_OR_EQUAL, + }; + + std::map, Path> BuildDistsFromGraph(const Graph & g) const; + Graph graph; + std::map, Path> dists; }; } diff --git a/src/Storages/ConstraintsDescription.h b/src/Storages/ConstraintsDescription.h index 89c3f82ad84..6aee4423d93 100644 --- a/src/Storages/ConstraintsDescription.h +++ b/src/Storages/ConstraintsDescription.h @@ -15,6 +15,7 @@ struct ConstraintsDescription std::vector constraints; std::vector cnf_constraints; + // TODO: перенести преобразование в КНФ + get constraitns ConstraintsDescription() = default; bool empty() const { return constraints.empty(); } @@ -30,8 +31,6 @@ struct ConstraintsDescription }; ASTs filterConstraints(ConstraintType selection) const; - // TODO: перенести преобразование в КНФ + get constraitns - //ASTs filterAtomicConstraints(ConstraintType selection) const; std::vector> getConstraintData() const; std::vector getAtomicConstraintData() const; From 9ebc0cd731fe2eee5eae26d2f41b623a554a0b1d Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sat, 10 Apr 2021 18:47:50 +0300 Subject: [PATCH 020/200] consts --- src/Interpreters/ComparisonGraph.cpp | 72 ++++++++++++++++++++++----- src/Interpreters/ComparisonGraph.h | 11 +++- src/Storages/ConstraintsDescription.h | 2 +- src/Storages/SelectQueryInfo.h | 2 + 4 files changed, 71 insertions(+), 16 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index b89bb57155f..b47e35e2606 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -104,6 +104,10 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) dists = BuildDistsFromGraph(graph); } +/// resturns {is less, is strict} +/// {true, true} = < +/// {true, false} = =< +/// {false, ...} = ? std::pair ComparisonGraph::findPath(const size_t start, const size_t finish) const { const auto it = dists.find(std::make_pair(start, finish)); @@ -143,16 +147,13 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con if (start == finish) return CompareResult::EQUAL; - /// TODO: precalculate using Floyd–Warshall O(n^3) algorithm where < = -1 and =< = 0. - /// TODO: use it for less, greater and so on - auto [has_path, is_strict] = findPath(start, finish); if (has_path) return is_strict ? CompareResult::GREATER : CompareResult::GREATER_OR_EQUAL; - auto [has_path_r, is_strict_r] = findPath(finish, start); - if (has_path_r) - return is_strict_r ? CompareResult::LESS : CompareResult::LESS_OR_EQUAL; + auto [has_path_reverse, is_strict_reverse] = findPath(finish, start); + if (has_path_reverse) + return is_strict_reverse ? CompareResult::LESS : CompareResult::LESS_OR_EQUAL; return CompareResult::UNKNOWN; } @@ -177,19 +178,35 @@ std::vector ComparisonGraph::getEqual(const ASTPtr & ast) const } } +bool ComparisonGraph::EqualComponent::hasConstant() const { + return constant_index != -1; +} + +ASTPtr ComparisonGraph::EqualComponent::getConstant() const { + return asts[constant_index]; +} + +void ComparisonGraph::EqualComponent::buildConstants() { + constant_index = -1; + for (size_t i = 0; i < asts.size(); ++i) + { + if (asts[i]->as()) + { + constant_index = i; + return; + } + } +} + std::optional ComparisonGraph::getEqualConst(const ASTPtr & ast) const { const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash()); if (hash_it == std::end(graph.ast_hash_to_component)) return std::nullopt; const size_t index = hash_it->second; - for (const auto & term : graph.vertexes[index].asts) - { - const ASTLiteral * lit = term->as(); - if (lit) - return term; - } - return std::nullopt; + return graph.vertexes[index].hasConstant() + ? std::optional{graph.vertexes[index].getConstant()} + : std::nullopt; } void ComparisonGraph::dfsOrder(const Graph & asts_graph, size_t v, std::vector & visited, std::vector & order) const @@ -281,6 +298,12 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as std::end(asts_graph.vertexes[index].asts)); // asts_graph has only one ast per vertex } + /// Calculate constants + for (auto & vertex : result.vertexes) + { + vertex.buildConstants(); + } + Poco::Logger::get("Graph").information("components: " + std::to_string(component)); for (size_t v = 0; v < n; ++v) @@ -292,6 +315,29 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as // TODO: make edges unique (left most strict) } + for (size_t v = 0; v < result.vertexes.size(); ++v) + { + for (size_t u = 0; u < result.vertexes.size(); ++u) + { + if (v == u) + continue; + if (result.vertexes[v].hasConstant() && result.vertexes[u].hasConstant()) + { + const auto * left = result.vertexes[v].getConstant()->as(); + const auto * right = result.vertexes[u].getConstant()->as(); + Poco::Logger::get("Graph").information("kek"); + Poco::Logger::get("Graph").information(left->value.dump() + " " + right->value.dump()); + + /// Only less. Equal constant fields = equal literals so it was already considered above. + if (left->value > right->value) + { + result.edges[v].push_back(Edge{Edge::LESS, u}); + Poco::Logger::get("Graph").information("1111 > "); + } + } + } + } + Poco::Logger::get("Graph").information("finish"); for (size_t v = 0; v < result.vertexes.size(); ++v) diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index ffdc10e13e6..425403d4944 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -38,14 +38,19 @@ public: /// Find constants lessOrEqual and greaterOrEqual. /// For int and double linear programming can be applied here. // TODO: implement - //ASTPtr getUpperBound(const ASTPtr &) const { return nullptr; } // sup - //ASTPtr getLowerBound(const ASTPtr &) const { return nullptr; } // inf + ASTPtr getUpperBound(const ASTPtr &) const { return nullptr; } // sup + ASTPtr getLowerBound(const ASTPtr &) const { return nullptr; } // inf private: /// strongly connected component struct EqualComponent { std::vector asts; + ssize_t constant_index = -1; + + bool hasConstant() const; + ASTPtr getConstant() const; + void buildConstants(); }; /// TODO: move to diff for int and double: @@ -98,6 +103,8 @@ private: Graph graph; std::map, Path> dists; + //std::vector ast_lower_bound; + //std::vector ast_upper_bound; }; } diff --git a/src/Storages/ConstraintsDescription.h b/src/Storages/ConstraintsDescription.h index 6aee4423d93..73d3534976d 100644 --- a/src/Storages/ConstraintsDescription.h +++ b/src/Storages/ConstraintsDescription.h @@ -13,7 +13,7 @@ using ConstraintsExpressions = std::vector; struct ConstraintsDescription { std::vector constraints; - std::vector cnf_constraints; + //std::vector cnf_constraints; // TODO: перенести преобразование в КНФ + get constraitns ConstraintsDescription() = default; diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index fea9a7bad68..71e1ced0c25 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -127,6 +127,8 @@ struct SelectQueryInfo PrewhereInfoPtr prewhere_info; + ASTPtr index_hint; + ReadInOrderOptimizerPtr order_optimizer; /// Can be modified while reading from storage InputOrderInfoPtr input_order_info; From 541f80159331cfd396a263081f90dd60c1f5dc3b Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sat, 10 Apr 2021 18:48:53 +0300 Subject: [PATCH 021/200] fix --- .../01622_constraints_simple_optimization.reference | 2 ++ .../01622_constraints_simple_optimization.sql | 9 +++++++++ 2 files changed, 11 insertions(+) diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index d09aa7baef4..c75c43b0ff1 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -18,6 +18,8 @@ 0 0 1 +1 +1 0 0 1 diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index 148321f129f..78ee75106be 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -61,6 +61,15 @@ SELECT count() FROM constraint_test.transitivity2 WHERE a = d; ---> assumption - DROP TABLE constraint_test.transitivity2; +CREATE TABLE constraint_test.transitivity3 (a Int64, b Int64, c Int64, CONSTRAINT c1 ASSUME b > 10 AND 1 > a) ENGINE = TinyLog; + +INSERT INTO constraint_test.transitivity3 (a, b, c) VALUES (4, 0, 2); + +SELECT count() FROM constraint_test.transitivity3 WHERE a < b; ---> assumption -> 1 +SELECT count() FROM constraint_test.transitivity3 WHERE b >= a; ---> assumption -> 1 + +DROP TABLE constraint_test.transitivity3; + CREATE TABLE constraint_test.constants_repl (a Int64, b Int64, c Int64, d Int64, CONSTRAINT c1 ASSUME a - b = 10 AND c + d = 20) ENGINE = TinyLog; From 082f43d7f1450d2426f849616d7fd5a90336d898 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sat, 10 Apr 2021 23:46:53 +0300 Subject: [PATCH 022/200] lower/upper bound --- src/Interpreters/ComparisonGraph.cpp | 145 +++++++++++++++++++++++- src/Interpreters/ComparisonGraph.h | 11 +- src/Storages/ConstraintsDescription.cpp | 3 + 3 files changed, 148 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index b47e35e2606..da0bd25c99e 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include @@ -14,8 +15,8 @@ namespace DB ASTPtr ComparisonGraph::normalizeAtom(const ASTPtr & atom) const { static const std::map inverse_relations = { - {"greaterOrEquals", "less"}, - {"greater", "lessOrEquals"}, + {"greaterOrEquals", "lessOrEquals"}, + {"greater", "less"}, }; ASTPtr res = atom->clone(); @@ -82,9 +83,9 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) if (index_left != bad_term && index_right != bad_term) { - //Poco::Logger::get("Edges").information("GOOD: " + atom->dumpTree()); - //Poco::Logger::get("Edges").information("left=" + std::to_string(index_left) + " right=" + std::to_string(index_right)); - //Poco::Logger::get("Edges").information("sz=" + std::to_string(g.edges.size())); + Poco::Logger::get("Edges").information("GOOD: " + atom->dumpTree()); + Poco::Logger::get("Edges").information("left=" + std::to_string(index_left) + " right=" + std::to_string(index_right)); + Poco::Logger::get("Edges").information("sz=" + std::to_string(g.edges.size())); g.edges[index_right].push_back(Edge{it->second, index_left}); if (func->name == "equals") { @@ -102,6 +103,7 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) graph = BuildGraphFromAstsGraph(g); dists = BuildDistsFromGraph(graph); + std::tie(ast_const_lower_bound, ast_const_upper_bound) = buildConstBounds(); } /// resturns {is less, is strict} @@ -114,7 +116,12 @@ std::pair ComparisonGraph::findPath(const size_t start, const size_t if (it == std::end(dists)) return {false, false}; else + { + Poco::Logger::get("dists found").information(std::to_string(start) + " " + std::to_string(finish) + " : " + std::to_string(static_cast(it->second))); + Poco::Logger::get("dists found").information(graph.vertexes[start].asts.back()->dumpTree()); + Poco::Logger::get("dists found").information(graph.vertexes[finish].asts.back()->dumpTree()); return {true, it->second == Path::LESS}; + } } ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, const ASTPtr & right) const @@ -122,6 +129,11 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con size_t start = 0; size_t finish = 0; { + for (const auto & [k, k2] : dists) + { + Poco::Logger::get("dists").information(std::to_string(k.first) + "-" + std::to_string(k.second) + " : " + std::to_string(static_cast(k2))); + } + /// TODO: check full ast const auto it_left = graph.ast_hash_to_component.find(left->getTreeHash()); const auto it_right = graph.ast_hash_to_component.find(right->getTreeHash()); @@ -134,6 +146,38 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con { Poco::Logger::get("Graph MAP").information(std::to_string(hash.second) + " " + std::to_string(id)); } + { + const auto left_bound = getConstLowerBound(left); + const auto right_bound = getConstUpperBound(right); + if (left_bound && right_bound) + { + Poco::Logger::get("&&&&&&&").information(left_bound->first.dump() + " " + std::to_string(left_bound->second) + " | " + right_bound->first.dump() + " " + std::to_string(right_bound->second)); + if (left_bound->first < right_bound->first) + return CompareResult::UNKNOWN; + else if (left_bound->first > right_bound->first) + return CompareResult::GREATER; + else if (left_bound->second || right_bound->second) + return CompareResult::GREATER; + else + return CompareResult::GREATER_OR_EQUAL; + } + } + { + const auto left_bound = getConstUpperBound(left); + const auto right_bound = getConstLowerBound(right); + Poco::Logger::get("!!!!!!").information(left_bound->first.dump() + " " + std::to_string(left_bound->second) + " | " + right_bound->first.dump() + " " + std::to_string(right_bound->second)); + if (left_bound && right_bound) + { + if (left_bound->first > right_bound->first) + return CompareResult::UNKNOWN; + else if (left_bound->first < right_bound->first) + return CompareResult::LESS; + else if (left_bound->second || right_bound->second) + return CompareResult::LESS; + else + return CompareResult::LESS_OR_EQUAL; + } + } return CompareResult::UNKNOWN; } else @@ -209,6 +253,40 @@ std::optional ComparisonGraph::getEqualConst(const ASTPtr & ast) const : std::nullopt; } +std::optional> ComparisonGraph::getConstUpperBound(const ASTPtr & ast) const +{ + { + const auto * literal = ast->as(); + if (literal) + return std::make_pair(literal->value, false); + } + const auto it = graph.ast_hash_to_component.find(ast->getTreeHash()); + if (it == std::end(graph.ast_hash_to_component)) + return std::nullopt; + const size_t to = it->second; + const ssize_t from = ast_const_upper_bound[to]; + if (from == -1) + return std::nullopt; + return std::make_pair(graph.vertexes[from].getConstant()->as()->value, dists.at({from, to}) == Path::LESS); +} + +std::optional> ComparisonGraph::getConstLowerBound(const ASTPtr & ast) const +{ + { + const auto * literal = ast->as(); + if (literal) + return std::make_pair(literal->value, false); + } + const auto it = graph.ast_hash_to_component.find(ast->getTreeHash()); + if (it == std::end(graph.ast_hash_to_component)) + return std::nullopt; + const size_t from = it->second; + const ssize_t to = ast_const_lower_bound[from]; + if (to == -1) + return std::nullopt; + return std::make_pair(graph.vertexes[to].getConstant()->as()->value, dists.at({from, to}) == Path::LESS); +} + void ComparisonGraph::dfsOrder(const Graph & asts_graph, size_t v, std::vector & visited, std::vector & order) const { visited[v] = true; @@ -257,6 +335,13 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as /// Find strongly connected component const auto n = asts_graph.vertexes.size(); + for (size_t v = 0; v < n; ++v) { + Poco::LogStream{"kek"}.information() << "VERTEX " << v << " " << asts_graph.vertexes[v].asts.back()->dumpTree() << std::endl; + for (const auto & edge : asts_graph.edges[v]) { + Poco::LogStream{"kek"}.information() << "TO " << edge.to << " " << static_cast(edge.type) << std::endl; + } + } + std::vector order; { std::vector visited(n, false); @@ -340,6 +425,14 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as Poco::Logger::get("Graph").information("finish"); + + for (size_t v = 0; v < result.vertexes.size(); ++v) { + Poco::LogStream{"kekkek"}.information() << "VERTEX " << v << " " << result.vertexes[v].asts.back()->dumpTree() << std::endl; + for (const auto & edge : result.edges[v]) { + Poco::LogStream{"kekkek"}.information() << "TO " << edge.to << " " << static_cast(edge.type) << std::endl; + } + } + for (size_t v = 0; v < result.vertexes.size(); ++v) { std::stringstream s; @@ -366,8 +459,14 @@ std::map, ComparisonGraph::Path> ComparisonGraph::Buil const size_t n = graph.vertexes.size(); std::vector> results(n, std::vector(n, inf)); for (size_t v = 0; v < n; ++v) + { + results[v][v] = 0; for (const auto & edge : g.edges[v]) - results[v][edge.to] = (edge.type == Edge::LESS ? -1 : 0); + results[v][edge.to] = std::min(results[v][edge.to], static_cast(edge.type == Edge::LESS ? -1 : 0)); + } + for (size_t v = 0; v < n; ++v) + for (size_t u = 0; u < n; ++u) + Poco::LogStream{Poco::Logger::get("Graph ---=------------")}.information() << v << " " << u << " " << static_cast(results[v][u]) << std::endl; for (size_t k = 0; k < n; ++k) for (size_t v = 0; v < n; ++v) @@ -378,9 +477,43 @@ std::map, ComparisonGraph::Path> ComparisonGraph::Buil std::map, Path> path; for (size_t v = 0; v < n; ++v) for (size_t u = 0; u < n; ++u) + { + Poco::LogStream{Poco::Logger::get("Graph results-------------")}.information() << v << " " << u << " " << static_cast(results[v][u]) << std::endl; if (results[v][u] != inf) path[std::make_pair(v, u)] = (results[v][u] == -1 ? Path::LESS : Path::LESS_OR_EQUAL); + } return path; } +std::pair, std::vector> ComparisonGraph::buildConstBounds() const +{ + const size_t n = graph.vertexes.size(); + std::vector lower(n, -1); + std::vector upper(n, -1); + + auto get_value = [this](const size_t vertex) -> Field { + return graph.vertexes[vertex].getConstant()->as()->value; + }; + + for (const auto & [edge, path] : dists) + { + const auto [from, to] = edge; + if (graph.vertexes[to].hasConstant()) { + if (lower[from] == -1 + || get_value(lower[from]) > get_value(to) + || (get_value(lower[from]) >= get_value(to) && dists.at({from, to}) == Path::LESS)) + lower[from] = to; + } + if (graph.vertexes[from].hasConstant()) { + if (upper[to] == -1 + || get_value(upper[to]) < get_value(from) + || (get_value(upper[to]) <= get_value(from) && dists.at({from, to}) == Path::LESS)) + upper[to] = from; + } + } + + return {lower, upper}; +} + + } diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 425403d4944..a6b40fc148a 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -37,9 +37,9 @@ public: /// Find constants lessOrEqual and greaterOrEqual. /// For int and double linear programming can be applied here. - // TODO: implement - ASTPtr getUpperBound(const ASTPtr &) const { return nullptr; } // sup - ASTPtr getLowerBound(const ASTPtr &) const { return nullptr; } // inf + /// Returns: {constant, is strict less/greater} + std::optional> getConstUpperBound(const ASTPtr & ast) const; + std::optional> getConstLowerBound(const ASTPtr & ast) const; private: /// strongly connected component @@ -100,11 +100,12 @@ private: }; std::map, Path> BuildDistsFromGraph(const Graph & g) const; + std::pair, std::vector> buildConstBounds() const; Graph graph; std::map, Path> dists; - //std::vector ast_lower_bound; - //std::vector ast_upper_bound; + std::vector ast_const_lower_bound; + std::vector ast_const_upper_bound; }; } diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index f8de4036b6a..ad36cf6f50b 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -85,6 +85,7 @@ std::vector ConstraintsDescription::getAtomicConstraint std::vector constraint_data; for (const auto & constraint : filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) { + Poco::Logger::get("atomic_formula: initial:").information(constraint->as()->expr->ptr()->dumpTree()); const auto cnf = TreeCNFConverter::toCNF(constraint->as()->expr->ptr()) .pullNotOutFunctions(); for (const auto & group : cnf.getStatements()) { @@ -105,12 +106,14 @@ ComparisonGraph ConstraintsDescription::getGraph() const auto atomic_formulas = getAtomicConstraintData(); for (auto & atomic_formula : atomic_formulas) { + Poco::Logger::get("atomic_formula: before:").information(atomic_formula.ast->dumpTree() + " " + std::to_string(atomic_formula.negative)); pushNotIn(atomic_formula); auto * func = atomic_formula.ast->as(); if (func && relations.count(func->name)) { if (atomic_formula.negative) throw Exception(": ", ErrorCodes::LOGICAL_ERROR); + Poco::Logger::get("atomic_formula: after:").information(atomic_formula.ast->dumpTree() + " " + std::to_string(atomic_formula.negative)); constraints_for_graph.push_back(atomic_formula.ast); } } From d8b34f3c61099bd8092e490d548dc1cfb7240b96 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sat, 10 Apr 2021 23:47:28 +0300 Subject: [PATCH 023/200] fix --- ...01622_constraints_simple_optimization.reference | 7 +++++++ .../01622_constraints_simple_optimization.sql | 14 ++++++++++++++ 2 files changed, 21 insertions(+) diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index c75c43b0ff1..fe3be5091ca 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -23,3 +23,10 @@ 0 0 1 +1 +0 +1 +1 +0 +1 +0 diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index 78ee75106be..f1e85ee8685 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -82,4 +82,18 @@ SELECT count() FROM constraint_test.constants_repl WHERE (a - b) * 2 = c + d; -- DROP TABLE constraint_test.constants_repl; +CREATE TABLE constraint_test.constants (a Int64, b Int64, c Int64, CONSTRAINT c1 ASSUME b > 10 AND a >= 10) ENGINE = TinyLog; + +INSERT INTO constraint_test.constants (a, b, c) VALUES (0, 0, 0); + +SELECT count() FROM constraint_test.constants WHERE 9 < b; ---> assumption -> 1 +SELECT count() FROM constraint_test.constants WHERE 11 < b; ---> assumption -> 0 +SELECT count() FROM constraint_test.constants WHERE 10 <= b; ---> assumption -> 1 +SELECT count() FROM constraint_test.constants WHERE 9 < a; ---> assumption -> 1 +SELECT count() FROM constraint_test.constants WHERE 10 < a; ---> assumption -> 0 +SELECT count() FROM constraint_test.constants WHERE 9 <= a; ---> assumption -> 1 +SELECT count() FROM constraint_test.constants WHERE 11 <= a; ---> assumption -> 0 + +DROP TABLE constraint_test.constants; + DROP DATABASE constraint_test; \ No newline at end of file From e362383b7c127c4cc17851cbb26a42ee68a798d5 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sat, 24 Apr 2021 18:24:32 +0300 Subject: [PATCH 024/200] swap columns --- src/CMakeLists.txt | 2 +- src/Interpreters/TreeOptimizer.cpp | 15 ++- src/Interpreters/TreeOptimizer.h | 3 +- src/Interpreters/TreeRewriter.cpp | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- .../MergeTree/SubstituteColumnOptimizer.cpp | 113 ++++++++++++++++++ .../MergeTree/SubstituteColumnOptimizer.h | 38 ++++++ .../01623_constraints_column_swap.reference | 0 .../01623_constraints_column_swap.sql | 0 9 files changed, 171 insertions(+), 6 deletions(-) create mode 100644 src/Storages/MergeTree/SubstituteColumnOptimizer.cpp create mode 100644 src/Storages/MergeTree/SubstituteColumnOptimizer.h create mode 100644 tests/queries/0_stateless/01623_constraints_column_swap.reference create mode 100644 tests/queries/0_stateless/01623_constraints_column_swap.sql diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 031c205df6f..5416c955bfe 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -64,7 +64,7 @@ add_subdirectory (Coordination) set(dbms_headers) -set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h Interpreters/WhereConstraintsOptimizer.cpp Interpreters/WhereConstraintsOptimizer.h Interpreters/TreeCNFConverter.cpp Interpreters/TreeCNFConverter.h Interpreters/ComparisonGraph.cpp Interpreters/ComparisonGraph.h) +set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h Interpreters/WhereConstraintsOptimizer.cpp Interpreters/WhereConstraintsOptimizer.h Interpreters/TreeCNFConverter.cpp Interpreters/TreeCNFConverter.h Interpreters/ComparisonGraph.cpp Interpreters/ComparisonGraph.h Storages/MergeTree/SubstituteColumnOptimizer.cpp Storages/MergeTree/SubstituteColumnOptimizer.h) add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 53689f195ab..c1d30aaa5f4 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -521,6 +522,14 @@ void optimizeWithConstraints(ASTSelectQuery * select_query, Aliases & aliases, c Poco::Logger::get("CNF").information("NO WHERE"); } +void optimizeSubstituteColumn(ASTSelectQuery * select_query, Aliases & aliases, const NameSet & source_columns_set, + const std::vector & tables_with_columns, + const StorageMetadataPtr & metadata_snapshot, + const ConstStoragePtr & storage) +{ + SubstituteColumnOptimizer(select_query, aliases, source_columns_set, tables_with_columns, metadata_snapshot, storage).perform(); +} + /// transform where to CNF for more convenient optimization void convertQueryToCNF(ASTSelectQuery * select_query) { @@ -619,7 +628,7 @@ void TreeOptimizer::optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, const std::vector & tables_with_columns, const Context & context, const StorageMetadataPtr & metadata_snapshot, - bool & rewrite_subqueries) + const ConstStoragePtr & storage, bool & rewrite_subqueries) { const auto & settings = context.getSettingsRef(); @@ -637,8 +646,10 @@ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & sou convertQueryToCNF(select_query); if (settings.convert_query_to_cnf && settings.optimize_using_constraints) + { optimizeWithConstraints(select_query, aliases, source_columns_set, tables_with_columns, metadata_snapshot); - + optimizeSubstituteColumn(select_query, aliases, source_columns_set, tables_with_columns, metadata_snapshot, storage); + } if (select_query->where()) { Poco::Logger::get("&&&&&&&&&&&&&&& WHERE").information(select_query->where()->getColumnName()); diff --git a/src/Interpreters/TreeOptimizer.h b/src/Interpreters/TreeOptimizer.h index a10dfc57451..5fabe3563d9 100644 --- a/src/Interpreters/TreeOptimizer.h +++ b/src/Interpreters/TreeOptimizer.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -19,7 +20,7 @@ public: static void apply(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, const std::vector & tables_with_columns, const Context & context, const StorageMetadataPtr & metadata_snapshot, - bool & rewrite_subqueries); + const ConstStoragePtr & storage, bool & rewrite_subqueries); static void optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_to_multiif); }; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index 9318f87175a..c87219671f0 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -824,7 +824,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( /// Executing scalar subqueries - replacing them with constant values. executeScalarSubqueries(query, context, subquery_depth, result.scalars, select_options.only_analyze); - TreeOptimizer::apply(query, result.aliases, source_columns_set, tables_with_columns, context, result.metadata_snapshot, result.rewrite_subqueries); + TreeOptimizer::apply(query, result.aliases, source_columns_set, tables_with_columns, context, result.metadata_snapshot, result.storage, result.rewrite_subqueries); /// array_join_alias_to_name, array_join_result_to_source. getArrayJoinedColumns(query, result, select_query, result.source_columns, source_columns_set); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 96a3dba12f7..f3cced2efbe 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1461,6 +1461,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( { for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it) { + // TODO: Prepare columns and prewhere on part auto source_processor = std::make_shared( data, metadata_snapshot, @@ -1476,7 +1477,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( reader_settings, virt_columns, part_it->part_index_in_query); - + // TODO: Conversion pipes.emplace_back(std::move(source_processor)); } } @@ -1571,6 +1572,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( for (size_t i = 0; i < num_streams_for_lonely_parts; ++i) { + // TODO: repl columns auto source = std::make_shared( i, pool, min_marks_for_concurrent_read, max_block_size, settings.preferred_block_size_bytes, settings.preferred_max_column_in_block_size_bytes, diff --git a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp new file mode 100644 index 00000000000..7f349c5d66c --- /dev/null +++ b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp @@ -0,0 +1,113 @@ +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +namespace +{ +class SubstituteColumnMatcher +{ +public: + using Visitor = InDepthNodeVisitor; + + struct Data + { + const ComparisonGraph & graph; + ConstStoragePtr storage; + + Data(const ComparisonGraph & graph_, const ConstStoragePtr & storage_) + : graph(graph_), storage(storage_) + { + } + }; + + static void visit(ASTPtr & ast, Data & data) + { + const auto column_sizes = data.storage->getColumnSizes(); + + // like TreeRewriter + struct ColumnSizeTuple + { + size_t compressed_size; + size_t uncompressed_size; + const ASTPtr & ast; + + bool operator<(const ColumnSizeTuple & that) const + { + return std::tie(compressed_size, uncompressed_size) + < std::tie(that.compressed_size, that.uncompressed_size); + } + }; + + std::vector columns; + for (const auto & equal_ast : data.graph.getEqual(ast)) + { + if (const auto it = column_sizes.find(equal_ast->getColumnName()); it != std::end(column_sizes)) + columns.push_back({ + it->second.data_compressed, + it->second.data_uncompressed, + equal_ast}); + } + + if (!columns.empty()) + ast = std::min_element(std::begin(columns), std::end(columns))->ast->clone(); + } + + static bool needChildVisit(const ASTPtr &, const ASTPtr &) + { + return true; + } +}; + +using SubstituteColumnVisitor = SubstituteColumnMatcher::Visitor; +} + +SubstituteColumnOptimizer::SubstituteColumnOptimizer( + ASTSelectQuery * select_query_, + Aliases & /*aliases_*/, + const NameSet & /*source_columns_set_*/, + const std::vector & /*tables_with_columns_*/, + const StorageMetadataPtr & metadata_snapshot_, + const ConstStoragePtr & storage_) + : select_query(select_query_) + /* , aliases(aliases_) + , source_columns_set(source_columns_set_) + , tables_with_columns(tables_with_columns_)*/ + , metadata_snapshot(metadata_snapshot_) + , storage(storage_) +{ +} + +void SubstituteColumnOptimizer::perform() +{ + if (!storage) + return; + const auto compare_graph = metadata_snapshot->getConstraints().getGraph(); + SubstituteColumnVisitor::Data data(compare_graph, storage); + if (select_query->where()) + SubstituteColumnVisitor(data).visit(select_query->refWhere()); + if (select_query->prewhere()) + SubstituteColumnVisitor(data).visit(select_query->refPrewhere()); + if (select_query->select()) + SubstituteColumnVisitor(data).visit(select_query->refSelect()); + if (select_query->having()) + SubstituteColumnVisitor(data).visit(select_query->refHaving()); +} + +} diff --git a/src/Storages/MergeTree/SubstituteColumnOptimizer.h b/src/Storages/MergeTree/SubstituteColumnOptimizer.h new file mode 100644 index 00000000000..b7835385b45 --- /dev/null +++ b/src/Storages/MergeTree/SubstituteColumnOptimizer.h @@ -0,0 +1,38 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; + +class SubstituteColumnOptimizer +{ +public: + SubstituteColumnOptimizer( + ASTSelectQuery * select_query, + Aliases & /* aliases */, const NameSet & /* source_columns_set */, + const std::vector & /* tables_with_columns */, + const StorageMetadataPtr & /* metadata_snapshot */, + const ConstStoragePtr & storage); + + void perform(); + +private: + ASTSelectQuery * select_query; + /*Aliases & aliases; + const NameSet & source_columns_set; + const std::vector & tables_with_columns;*/ + const StorageMetadataPtr & metadata_snapshot; + ConstStoragePtr storage; +}; + +} diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.sql b/tests/queries/0_stateless/01623_constraints_column_swap.sql new file mode 100644 index 00000000000..e69de29bb2d From 4601623d6172907834e75ee8f241e7f9146f1193 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sat, 24 Apr 2021 18:25:15 +0300 Subject: [PATCH 025/200] test --- .../01623_constraints_column_swap.reference | 30 +++++++++++++++++++ .../01623_constraints_column_swap.sql | 29 ++++++++++++++++++ 2 files changed, 59 insertions(+) diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index e69de29bb2d..3c0278a5ccd 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -0,0 +1,30 @@ +SELECT + b + 10, + b + 3 +FROM constraint_test.test +WHERE b = 1 +SELECT + b + 10, + b + 3 +FROM constraint_test.test +WHERE b = 0 +SELECT + b + 10, + b + 3 +FROM constraint_test.test +WHERE b = 0 +SELECT + b + 10, + b + 3 +FROM constraint_test.test +WHERE b = 1 +SELECT + a, + a +FROM constraint_test.test +WHERE a = \'c\' +SELECT + a, + a +FROM constraint_test.test +WHERE a = \'c\' diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.sql b/tests/queries/0_stateless/01623_constraints_column_swap.sql index e69de29bb2d..47c52c9ad07 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.sql +++ b/tests/queries/0_stateless/01623_constraints_column_swap.sql @@ -0,0 +1,29 @@ +SET convert_query_to_cnf = 1; +SET optimize_using_constraints = 1; +SET optimize_move_to_prewhere = 1; + +DROP DATABASE IF EXISTS constraint_test; +DROP TABLE IF EXISTS constraint_test.test; + +CREATE DATABASE constraint_test; + +CREATE TABLE constraint_test.test (i Int64, a String, b UInt64, CONSTRAINT c1 ASSUME b = cityHash64(a)) ENGINE = MergeTree() ORDER BY i; +INSERT INTO constraint_test.test VALUES (1, 'cat', 1), (2, 'dog', 2); + +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM constraint_test.test WHERE cityHash64(a) = 1; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM constraint_test.test WHERE cityHash64(a) = 0; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM constraint_test.test WHERE b = 0; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM constraint_test.test WHERE b = 1; + +DROP TABLE constraint_test.test; + + +CREATE TABLE constraint_test.test (i Int64, a String, b String, CONSTRAINT c1 ASSUME a = substring(reverse(b), 1, 1)) ENGINE = MergeTree() ORDER BY i; +INSERT INTO constraint_test.test VALUES (1, 'c', 'aaaaaaaaaaaaaaaaaac'), (2, 'd', 'bbbbbbbbbbbbbbbd'); + +EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM constraint_test.test WHERE a = 'c'; +EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM constraint_test.test WHERE substring(reverse(b), 1, 1) = 'c'; + +DROP TABLE constraint_test.test; + +DROP DATABASE constraint_test; From 2896f9aa7563b5541bb57e66ba29f09c10a7cb36 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Mon, 26 Apr 2021 12:40:54 +0300 Subject: [PATCH 026/200] hypothesis basic --- src/CMakeLists.txt | 2 +- .../MergeTree/MergeTreeIndexHypothesis.cpp | 164 ++++++++++++++++++ .../MergeTree/MergeTreeIndexHypothesis.h | 107 ++++++++++++ src/Storages/MergeTree/MergeTreeIndices.cpp | 3 + src/Storages/MergeTree/MergeTreeIndices.h | 3 + .../01624_soft_constraints.reference | 1 + .../0_stateless/01624_soft_constraints.sql | 17 ++ 7 files changed, 296 insertions(+), 1 deletion(-) create mode 100644 src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp create mode 100644 src/Storages/MergeTree/MergeTreeIndexHypothesis.h create mode 100644 tests/queries/0_stateless/01624_soft_constraints.reference create mode 100644 tests/queries/0_stateless/01624_soft_constraints.sql diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 5416c955bfe..9052d6a6bf7 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -64,7 +64,7 @@ add_subdirectory (Coordination) set(dbms_headers) -set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h Interpreters/WhereConstraintsOptimizer.cpp Interpreters/WhereConstraintsOptimizer.h Interpreters/TreeCNFConverter.cpp Interpreters/TreeCNFConverter.h Interpreters/ComparisonGraph.cpp Interpreters/ComparisonGraph.h Storages/MergeTree/SubstituteColumnOptimizer.cpp Storages/MergeTree/SubstituteColumnOptimizer.h) +set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h Interpreters/WhereConstraintsOptimizer.cpp Interpreters/WhereConstraintsOptimizer.h Interpreters/TreeCNFConverter.cpp Interpreters/TreeCNFConverter.h Interpreters/ComparisonGraph.cpp Interpreters/ComparisonGraph.h Storages/MergeTree/SubstituteColumnOptimizer.cpp Storages/MergeTree/SubstituteColumnOptimizer.h Storages/MergeTree/MergeTreeIndexHypothesis.cpp Storages/MergeTree/MergeTreeIndexHypothesis.h) add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp new file mode 100644 index 00000000000..d834968d636 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp @@ -0,0 +1,164 @@ +#include + +#include +#include +#include + +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INCORRECT_QUERY; +} + + +MergeTreeIndexGranuleHypothesis::MergeTreeIndexGranuleHypothesis(const String & index_name_) + : index_name(index_name_), is_empty(true), met(false) +{ +} + +MergeTreeIndexGranuleHypothesis::MergeTreeIndexGranuleHypothesis(const String & index_name_, const bool met_) + : index_name(index_name_), is_empty(false), met(met_) +{ +} + +void MergeTreeIndexGranuleHypothesis::serializeBinary(WriteBuffer & ostr) const +{ + const auto & size_type = DataTypePtr(std::make_shared()); + size_type->serializeBinary(static_cast(met), ostr); +} + +void MergeTreeIndexGranuleHypothesis::deserializeBinary(ReadBuffer & istr) +{ + Field field_met; + const auto & size_type = DataTypePtr(std::make_shared()); + size_type->deserializeBinary(field_met, istr); + met = field_met.get(); + is_empty = false; +} + +MergeTreeIndexAggregatorHypothesis::MergeTreeIndexAggregatorHypothesis(const String & index_name_, const String & column_name_) + : index_name(index_name_), column_name(column_name_) +{ +} + +MergeTreeIndexGranulePtr MergeTreeIndexAggregatorHypothesis::getGranuleAndReset() +{ + const auto granule = std::make_shared(index_name, met); + met = true; + is_empty = true; + return granule; +} + +void MergeTreeIndexAggregatorHypothesis::update(const Block & block, size_t * pos, size_t limit) +{ + size_t rows_read = std::min(limit, block.rows() - *pos); + if (rows_read == 0) + return; + const auto & column = block.getByName(column_name).column->cut(*pos, rows_read); + + if (!column->hasEqualValues() || column->get64(0) == 0) + met = false; + + is_empty = false; + *pos += rows_read; +} + +MergeTreeIndexConditionHypothesis::MergeTreeIndexConditionHypothesis( + const String & index_name_, + const String & column_name_, + const SelectQueryInfo & query_, + const Context &) + : index_name(index_name_) + , column_name(column_name_) +{ + const auto & select = query_.query->as(); + + if (select.where() && select.prewhere()) + expression_ast = makeASTFunction( + "and", + select.where()->clone(), + select.prewhere()->clone()); + else if (select.where()) + expression_ast = select.where()->clone(); + else if (select.prewhere()) + expression_ast = select.prewhere()->clone(); +} + +std::pair MergeTreeIndexConditionHypothesis::mayBeTrue(const ASTPtr & ast, const bool value) const +{ + if (ast->getColumnName() == column_name) + return {value, !value}; + + auto * func = ast->as(); + if (!func) + return {true, true}; + auto & args = func->arguments->children; + if (func->name == "not") + { + const auto res = mayBeTrue(args[0], value); + return {res.second, res.first}; + } + /*else if (func->name == "or") + { + + } + else if (func->name == "and") + { + + }*/ + else + { + return {true, true}; + } +} + +bool MergeTreeIndexConditionHypothesis::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const +{ + if (idx_granule->empty()) + return true; + auto granule = std::dynamic_pointer_cast(idx_granule); + if (!granule) + throw Exception( + "Set index condition got a granule with the wrong type.", ErrorCodes::LOGICAL_ERROR); + return mayBeTrue(expression_ast, granule->met).first; +} + +MergeTreeIndexGranulePtr MergeTreeIndexHypothesis::createIndexGranule() const +{ + return std::make_shared(index.name); +} + +MergeTreeIndexAggregatorPtr MergeTreeIndexHypothesis::createIndexAggregator() const +{ + return std::make_shared(index.name, index.sample_block.getNames().front()); +} + +MergeTreeIndexConditionPtr MergeTreeIndexHypothesis::createIndexCondition( + const SelectQueryInfo & query, const Context & context) const +{ + return std::make_shared(index.name, index.sample_block.getNames().front(), query, context); +} + +bool MergeTreeIndexHypothesis::mayBenefitFromIndexForIn(const ASTPtr &) const +{ + return false; +} + +MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index) +{ + return std::make_shared(index); +} + +void hypothesisIndexValidator(const IndexDescription &, bool /*attach*/) +{ +} + + +} diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h new file mode 100644 index 00000000000..7b01cebe16d --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h @@ -0,0 +1,107 @@ +#pragma once + +#include +#include + +#include + +#include +#include + + +namespace DB +{ + +class MergeTreeIndexHyposesis; + +struct MergeTreeIndexGranuleHypothesis : public IMergeTreeIndexGranule +{ + explicit MergeTreeIndexGranuleHypothesis( + const String & index_name_); + + MergeTreeIndexGranuleHypothesis( + const String & index_name_, + const bool met_); + + void serializeBinary(WriteBuffer & ostr) const override; + void deserializeBinary(ReadBuffer & istr) override; + + bool empty() const override { return is_empty; } + + ~MergeTreeIndexGranuleHypothesis() override = default; + + String index_name; + bool is_empty = true; + bool met = true; +}; + + +struct MergeTreeIndexAggregatorHypothesis : IMergeTreeIndexAggregator +{ + explicit MergeTreeIndexAggregatorHypothesis( + const String & index_name_, const String & column_name_); + + ~MergeTreeIndexAggregatorHypothesis() override = default; + + bool empty() const override { return is_empty; } + + MergeTreeIndexGranulePtr getGranuleAndReset() override; + + void update(const Block & block, size_t * pos, size_t limit) override; + +private: + String index_name; + String column_name; + + bool met = true; + bool is_empty = true; +}; + + +class MergeTreeIndexConditionHypothesis : public IMergeTreeIndexCondition +{ +public: + MergeTreeIndexConditionHypothesis( + const String & index_name_, + const String & column_name_, + const SelectQueryInfo & query, + const Context & context); + + bool alwaysUnknownOrTrue() const override { return false; } + + bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const override; + + ~MergeTreeIndexConditionHypothesis() override = default; + +private: + std::pair mayBeTrue(const ASTPtr & ast, const bool value) const; + + String index_name; + + String column_name; + ASTPtr expression_ast; +}; + + +class MergeTreeIndexHypothesis : public IMergeTreeIndex +{ +public: + MergeTreeIndexHypothesis( + const IndexDescription & index_) + : IMergeTreeIndex(index_) + {} + + ~MergeTreeIndexHypothesis() override = default; + + MergeTreeIndexGranulePtr createIndexGranule() const override; + MergeTreeIndexAggregatorPtr createIndexAggregator() const override; + + MergeTreeIndexConditionPtr createIndexCondition( + const SelectQueryInfo & query, const Context & context) const override; + + bool mayBenefitFromIndexForIn(const ASTPtr & node) const override; + + size_t max_rows = 0; +}; + +} diff --git a/src/Storages/MergeTree/MergeTreeIndices.cpp b/src/Storages/MergeTree/MergeTreeIndices.cpp index b0f5b4d92f5..9d7e0cdfdbe 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.cpp +++ b/src/Storages/MergeTree/MergeTreeIndices.cpp @@ -98,6 +98,9 @@ MergeTreeIndexFactory::MergeTreeIndexFactory() registerCreator("bloom_filter", bloomFilterIndexCreatorNew); registerValidator("bloom_filter", bloomFilterIndexValidatorNew); + + registerCreator("hypothesis", hypothesisIndexCreator); + registerValidator("hypothesis", hypothesisIndexValidator); } MergeTreeIndexFactory & MergeTreeIndexFactory::instance() diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index c7b9dfb123e..7b7e96587ca 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -135,4 +135,7 @@ void bloomFilterIndexValidator(const IndexDescription & index, bool attach); MergeTreeIndexPtr bloomFilterIndexCreatorNew(const IndexDescription & index); void bloomFilterIndexValidatorNew(const IndexDescription & index, bool attach); +MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index); +void hypothesisIndexValidator(const IndexDescription & index, bool attach); + } diff --git a/tests/queries/0_stateless/01624_soft_constraints.reference b/tests/queries/0_stateless/01624_soft_constraints.reference new file mode 100644 index 00000000000..0cfbf08886f --- /dev/null +++ b/tests/queries/0_stateless/01624_soft_constraints.reference @@ -0,0 +1 @@ +2 diff --git a/tests/queries/0_stateless/01624_soft_constraints.sql b/tests/queries/0_stateless/01624_soft_constraints.sql new file mode 100644 index 00000000000..53a2d4ac9f6 --- /dev/null +++ b/tests/queries/0_stateless/01624_soft_constraints.sql @@ -0,0 +1,17 @@ +SET convert_query_to_cnf = 1; +SET optimize_using_constraints = 1; +SET optimize_move_to_prewhere = 1; + +DROP DATABASE IF EXISTS constraint_test; +DROP TABLE IF EXISTS constraint_test.test; + +CREATE DATABASE constraint_test; + +CREATE TABLE constraint_test.test (i UInt64, a UInt64, b UInt64, INDEX t (a = b) TYPE hypothesis GRANULARITY 1) ENGINE = MergeTree() ORDER BY i SETTINGS index_granularity=1; +INSERT INTO constraint_test.test VALUES (1, 1, 1), (2, 1, 2), (3, 2, 1), (4, 2, 2); + +SELECT count() FROM constraint_test.test WHERE a = b; + +DROP TABLE constraint_test.test; + +DROP DATABASE constraint_test; \ No newline at end of file From 5663db33b1166ff2d7eff08e1a1f0bad421e43c5 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Mon, 26 Apr 2021 14:26:54 +0300 Subject: [PATCH 027/200] impr --- src/Interpreters/ComparisonGraph.cpp | 30 +++- src/Interpreters/ComparisonGraph.h | 3 + .../MergeTree/SubstituteColumnOptimizer.cpp | 167 +++++++++++++----- 3 files changed, 152 insertions(+), 48 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index da0bd25c99e..aff272a76d2 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -203,25 +203,39 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con } std::vector ComparisonGraph::getEqual(const ASTPtr & ast) const +{ + const auto res = getComponentId(ast); + if (!res) + return {}; + else + return getComponent(res.value()); +} + +std::optional ComparisonGraph::getComponentId(const ASTPtr & ast) const { const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash()); if (hash_it == std::end(graph.ast_hash_to_component)) return {}; const size_t index = hash_it->second; if (std::any_of( - std::cbegin(graph.vertexes[index].asts), - std::cend(graph.vertexes[index].asts), - [ast](const ASTPtr & constraint_ast) - { - return constraint_ast->getTreeHash() == ast->getTreeHash() && - constraint_ast->getColumnName() == ast->getColumnName(); - })) { - return graph.vertexes[index].asts; + std::cbegin(graph.vertexes[index].asts), + std::cend(graph.vertexes[index].asts), + [ast](const ASTPtr & constraint_ast) + { + return constraint_ast->getTreeHash() == ast->getTreeHash() && + constraint_ast->getColumnName() == ast->getColumnName(); + })) { + return index; } else { return {}; } } +std::vector ComparisonGraph::getComponent(const std::size_t id) const +{ + return graph.vertexes[id].asts; +} + bool ComparisonGraph::EqualComponent::hasConstant() const { return constant_index != -1; } diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index a6b40fc148a..a47f9fbdf90 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -35,6 +35,9 @@ public: std::vector getEqual(const ASTPtr & ast) const; std::optional getEqualConst(const ASTPtr & ast) const; + std::optional getComponentId(const ASTPtr & ast) const; + std::vector getComponent(const std::size_t id) const; + /// Find constants lessOrEqual and greaterOrEqual. /// For int and double linear programming can be applied here. /// Returns: {constant, is strict less/greater} diff --git a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp index 7f349c5d66c..e832087bab1 100644 --- a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp +++ b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp @@ -1,10 +1,9 @@ #include -#include #include #include #include #include -#include +#include #include #include #include @@ -21,52 +20,118 @@ namespace ErrorCodes namespace { -class SubstituteColumnMatcher + +const String COMPONENT = "__constraint_component_"; + +class ComponentMatcher { public: - using Visitor = InDepthNodeVisitor; + using Visitor = InDepthNodeVisitor; struct Data { const ComparisonGraph & graph; - ConstStoragePtr storage; - Data(const ComparisonGraph & graph_, const ConstStoragePtr & storage_) - : graph(graph_), storage(storage_) + Data(const ComparisonGraph & graph_) + : graph(graph_) { } }; static void visit(ASTPtr & ast, Data & data) { - const auto column_sizes = data.storage->getColumnSizes(); + const auto id = data.graph.getComponentId(ast); + if (id) + ast = std::make_shared(COMPONENT + std::to_string(id.value())); + } - // like TreeRewriter - struct ColumnSizeTuple + static bool needChildVisit(const ASTPtr &, const ASTPtr &) + { + return true; + } +}; + +using ComponentVisitor = ComponentMatcher::Visitor; + + +class IdentifierSetMatcher +{ +public: + using Visitor = InDepthNodeVisitor; + + struct Data + { + std::unordered_set identifiers; + }; + + static void visit(ASTPtr & ast, Data & data) + { + const auto * identifier = ast->as(); + if (identifier) + data.identifiers.insert(identifier->name()); + } + + static bool needChildVisit(const ASTPtr &, const ASTPtr &) + { + return true; + } +}; + +using IdentifierSetVisitor = IdentifierSetMatcher::Visitor; + + +class SubstituteColumnMatcher +{ +public: + using Visitor = InDepthNodeVisitor; + + struct Data + { + const ComparisonGraph & graph; + const std::unordered_set & identifiers; + ConstStoragePtr storage; + + Data(const ComparisonGraph & graph_, + const std::unordered_set & identifiers_, + const ConstStoragePtr & storage_) + : graph(graph_) + , identifiers(identifiers_) + , storage(storage_) { - size_t compressed_size; - size_t uncompressed_size; - const ASTPtr & ast; - - bool operator<(const ColumnSizeTuple & that) const - { - return std::tie(compressed_size, uncompressed_size) - < std::tie(that.compressed_size, that.uncompressed_size); - } - }; - - std::vector columns; - for (const auto & equal_ast : data.graph.getEqual(ast)) - { - if (const auto it = column_sizes.find(equal_ast->getColumnName()); it != std::end(column_sizes)) - columns.push_back({ - it->second.data_compressed, - it->second.data_uncompressed, - equal_ast}); } + }; - if (!columns.empty()) - ast = std::min_element(std::begin(columns), std::end(columns))->ast->clone(); + static void visit(ASTPtr & ast, Data & data) + { + const auto * identifier = ast->as(); + if (identifier && identifier->name().starts_with(COMPONENT)) + { + const std::size_t id = std::stoll(identifier->name().substr(COMPONENT.size(), identifier->name().size())); + // like TreeRewriter + struct ColumnSizeTuple + { + size_t compressed_size; + size_t uncompressed_size; + const ASTPtr & ast; + + bool operator<(const ColumnSizeTuple & that) const + { + return std::tie(compressed_size, uncompressed_size) < std::tie(that.compressed_size, that.uncompressed_size); + } + }; + + const auto column_sizes = data.storage->getColumnSizes(); + + std::vector columns; + for (const auto & equal_ast : data.graph.getComponent(id)) + { + if (const auto it = column_sizes.find(equal_ast->getColumnName()); it != std::end(column_sizes)) + columns.push_back({it->second.data_compressed, it->second.data_uncompressed, equal_ast}); + } + + if (!columns.empty()) + ast = std::min_element(std::begin(columns), std::end(columns))->ast->clone(); + } } static bool needChildVisit(const ASTPtr &, const ASTPtr &) @@ -78,6 +143,7 @@ public: using SubstituteColumnVisitor = SubstituteColumnMatcher::Visitor; } + SubstituteColumnOptimizer::SubstituteColumnOptimizer( ASTSelectQuery * select_query_, Aliases & /*aliases_*/, @@ -99,15 +165,36 @@ void SubstituteColumnOptimizer::perform() if (!storage) return; const auto compare_graph = metadata_snapshot->getConstraints().getGraph(); - SubstituteColumnVisitor::Data data(compare_graph, storage); - if (select_query->where()) - SubstituteColumnVisitor(data).visit(select_query->refWhere()); - if (select_query->prewhere()) - SubstituteColumnVisitor(data).visit(select_query->refPrewhere()); - if (select_query->select()) - SubstituteColumnVisitor(data).visit(select_query->refSelect()); - if (select_query->having()) - SubstituteColumnVisitor(data).visit(select_query->refHaving()); + + auto run_for_all = [&](const auto func) { + if (select_query->where()) + func(select_query->refWhere()); + if (select_query->prewhere()) + func(select_query->refPrewhere()); + if (select_query->select()) + func(select_query->refSelect()); + if (select_query->having()) + func(select_query->refHaving()); + }; + + ComponentVisitor::Data component_data(compare_graph); + IdentifierSetVisitor::Data identifier_data; + auto preprocess = [&](ASTPtr & ast) { + ComponentVisitor(component_data).visit(ast); + IdentifierSetVisitor(identifier_data).visit(ast); + }; + + auto process = [&](ASTPtr & ast) { + SubstituteColumnVisitor::Data substitute_data(compare_graph, identifier_data.identifiers, storage); + SubstituteColumnVisitor(substitute_data).visit(ast); + }; + + ASTPtr old_query = select_query->clone(); + + run_for_all(preprocess); + run_for_all(process); + + } } From 4650e36d05c4e5847f1048d8789a9d7ae146c42f Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Mon, 26 Apr 2021 17:19:18 +0300 Subject: [PATCH 028/200] improve constrdescr --- src/Interpreters/InterpreterCreateQuery.cpp | 6 ++-- src/Storages/AlterCommands.cpp | 24 ++++++++----- src/Storages/ConstraintsDescription.cpp | 35 +++++++++++++++++-- src/Storages/ConstraintsDescription.h | 23 ++++++++---- .../MergeTree/registerStorageMergeTree.cpp | 4 ++- 5 files changed, 70 insertions(+), 22 deletions(-) diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index f8bcbf02ab4..c6afe3d05e0 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -354,7 +354,7 @@ ASTPtr InterpreterCreateQuery::formatConstraints(const ConstraintsDescription & { auto res = std::make_shared(); - for (const auto & constraint : constraints.constraints) + for (const auto & constraint : constraints.getConstraints()) res->children.push_back(constraint->clone()); return res; @@ -489,9 +489,11 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( ConstraintsDescription InterpreterCreateQuery::getConstraintsDescription(const ASTExpressionList * constraints) { ConstraintsDescription res; + auto constraints_data = res.getConstraints(); if (constraints) for (const auto & constraint : constraints->children) - res.constraints.push_back(std::dynamic_pointer_cast(constraint->clone())); + constraints_data.push_back(std::dynamic_pointer_cast(constraint->clone())); + res.updateConstraints(constraints_data); return res; } diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 7043a32760b..19fe490cfee 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -462,9 +462,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con } else if (type == ADD_CONSTRAINT) { + auto constraints = metadata.constraints.getConstraints(); if (std::any_of( - metadata.constraints.constraints.cbegin(), - metadata.constraints.constraints.cend(), + constraints.cbegin(), + constraints.cend(), [this](const ASTPtr & constraint_ast) { return constraint_ast->as().name == constraint_name; @@ -476,28 +477,31 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con ErrorCodes::ILLEGAL_COLUMN); } - auto insert_it = metadata.constraints.constraints.end(); + auto insert_it = constraints.end(); - metadata.constraints.constraints.emplace(insert_it, std::dynamic_pointer_cast(constraint_decl)); + constraints.emplace(insert_it, std::dynamic_pointer_cast(constraint_decl)); + metadata.constraints.updateConstraints(constraints); } else if (type == DROP_CONSTRAINT) { + auto constraints = metadata.constraints.getConstraints(); auto erase_it = std::find_if( - metadata.constraints.constraints.begin(), - metadata.constraints.constraints.end(), + constraints.begin(), + constraints.end(), [this](const ASTPtr & constraint_ast) { return constraint_ast->as().name == constraint_name; }); - if (erase_it == metadata.constraints.constraints.end()) + if (erase_it == constraints.end()) { if (if_exists) return; throw Exception("Wrong constraint name. Cannot find constraint `" + constraint_name + "` to drop.", ErrorCodes::BAD_ARGUMENTS); } - metadata.constraints.constraints.erase(erase_it); + constraints.erase(erase_it); + metadata.constraints.updateConstraints(constraints); } else if (type == MODIFY_TTL) { @@ -543,8 +547,10 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, const Context & con if (metadata.table_ttl.definition_ast) rename_visitor.visit(metadata.table_ttl.definition_ast); - for (auto & constraint : metadata.constraints.constraints) + auto constraints_data = metadata.constraints.getConstraints(); + for (auto & constraint : constraints_data) rename_visitor.visit(constraint); + metadata.constraints.updateConstraints(constraints_data); if (metadata.isSortingKeyDefined()) rename_visitor.visit(metadata.sorting_key.definition_ast); diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index ad36cf6f50b..37af5efcc6e 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -66,7 +66,7 @@ ASTs ConstraintsDescription::filterConstraints(ConstraintType selection) const return res; } -std::vector> ConstraintsDescription::getConstraintData() const +std::vector> ConstraintsDescription::buildConstraintData() const { std::vector> constraint_data; for (const auto & constraint : filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) @@ -97,7 +97,7 @@ std::vector ConstraintsDescription::getAtomicConstraint return constraint_data; } -ComparisonGraph ConstraintsDescription::getGraph() const +std::unique_ptr ConstraintsDescription::buildGraph() const { static const std::set relations = { "equals", "less", "lessOrEquals", "greaterOrEquals", "greater"}; @@ -118,7 +118,7 @@ ComparisonGraph ConstraintsDescription::getGraph() const } } - return ComparisonGraph(constraints_for_graph); + return std::make_unique(constraints_for_graph); } ConstraintsExpressions ConstraintsDescription::getExpressionsToCheck(const DB::Context & context, @@ -140,11 +140,33 @@ ConstraintsExpressions ConstraintsDescription::getExpressionsToCheck(const DB::C return res; } +const ComparisonGraph & ConstraintsDescription::getGraph() const +{ + return *graph; +} + +const std::vector> & ConstraintsDescription::getConstraintData() const +{ + return cnf_constraints; +} + +const std::vector & ConstraintsDescription::getConstraints() const +{ + return constraints; +} + +void ConstraintsDescription::updateConstraints(const std::vector & constraints_) +{ + constraints = constraints_; + update(); +} + ConstraintsDescription::ConstraintsDescription(const ConstraintsDescription & other) { constraints.reserve(other.constraints.size()); for (const auto & constraint : other.constraints) constraints.emplace_back(constraint->clone()); + update(); } ConstraintsDescription & ConstraintsDescription::operator=(const ConstraintsDescription & other) @@ -152,7 +174,14 @@ ConstraintsDescription & ConstraintsDescription::operator=(const ConstraintsDesc constraints.resize(other.constraints.size()); for (size_t i = 0; i < constraints.size(); ++i) constraints[i] = other.constraints[i]->clone(); + update(); return *this; } +void ConstraintsDescription::update() +{ + cnf_constraints = buildConstraintData(); + graph = buildGraph(); +} + } diff --git a/src/Storages/ConstraintsDescription.h b/src/Storages/ConstraintsDescription.h index 73d3534976d..e942a221db4 100644 --- a/src/Storages/ConstraintsDescription.h +++ b/src/Storages/ConstraintsDescription.h @@ -12,11 +12,8 @@ using ConstraintsExpressions = std::vector; struct ConstraintsDescription { - std::vector constraints; - //std::vector cnf_constraints; - - // TODO: перенести преобразование в КНФ + get constraitns - ConstraintsDescription() = default; +public: + ConstraintsDescription() { update(); } bool empty() const { return constraints.empty(); } String toString() const; @@ -32,15 +29,27 @@ struct ConstraintsDescription ASTs filterConstraints(ConstraintType selection) const; - std::vector> getConstraintData() const; + const std::vector & getConstraints() const; + void updateConstraints(const std::vector & constraints); + + const std::vector> & getConstraintData() const; std::vector getAtomicConstraintData() const; - ComparisonGraph getGraph() const; + const ComparisonGraph & getGraph() const; ConstraintsExpressions getExpressionsToCheck(const Context & context, const NamesAndTypesList & source_columns_) const; ConstraintsDescription(const ConstraintsDescription & other); ConstraintsDescription & operator=(const ConstraintsDescription & other); + +private: + std::vector> buildConstraintData() const; + std::unique_ptr buildGraph() const; + void update(); + + std::vector constraints; + std::vector> cnf_constraints; + std::unique_ptr graph; }; } diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 6cde5245735..b32cd439175 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -658,9 +658,11 @@ static StoragePtr create(const StorageFactory::Arguments & args) for (auto & index : args.query.columns_list->indices->children) metadata.secondary_indices.push_back(IndexDescription::getIndexFromAST(index, args.columns, args.context)); + auto constraints = metadata.constraints.getConstraints(); if (args.query.columns_list && args.query.columns_list->constraints) for (auto & constraint : args.query.columns_list->constraints->children) - metadata.constraints.constraints.push_back(constraint); + constraints.push_back(constraint); + metadata.constraints.updateConstraints(constraints); auto column_ttl_asts = args.columns.getColumnTTLs(); for (const auto & [name, ast] : column_ttl_asts) From c5fbc8b175f8e9231f706bfbecbd4710a342f2df Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Tue, 27 Apr 2021 13:57:59 +0300 Subject: [PATCH 029/200] small fixes --- src/DataStreams/CheckConstraintsBlockOutputStream.cpp | 2 +- src/Interpreters/TreeOptimizer.h | 3 --- src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp | 8 ++++---- src/Storages/MergeTree/MergeTreeIndexHypothesis.h | 4 ++-- 4 files changed, 7 insertions(+), 10 deletions(-) diff --git a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp index 5df6dd2d23d..1360d80a844 100644 --- a/src/DataStreams/CheckConstraintsBlockOutputStream.cpp +++ b/src/DataStreams/CheckConstraintsBlockOutputStream.cpp @@ -32,7 +32,7 @@ CheckConstraintsBlockOutputStream::CheckConstraintsBlockOutputStream( output(output_), header(header_), constraints_to_check(constraints_.filterConstraints(ConstraintsDescription::ConstraintType::CHECK)), - expressions(constraints_.getExpressionsToCheck(context_, header.getNamesAndTypesList())) + expressions(constraints_.getExpressions(context_, header.getNamesAndTypesList())) { } diff --git a/src/Interpreters/TreeOptimizer.h b/src/Interpreters/TreeOptimizer.h index 762702bc1ec..e19f7c99f5c 100644 --- a/src/Interpreters/TreeOptimizer.h +++ b/src/Interpreters/TreeOptimizer.h @@ -3,11 +3,8 @@ #include #include #include -<<<<<<< HEAD #include -======= #include ->>>>>>> upstream/master namespace DB { diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp index d834968d636..a34af89f063 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp @@ -31,14 +31,14 @@ MergeTreeIndexGranuleHypothesis::MergeTreeIndexGranuleHypothesis(const String & void MergeTreeIndexGranuleHypothesis::serializeBinary(WriteBuffer & ostr) const { const auto & size_type = DataTypePtr(std::make_shared()); - size_type->serializeBinary(static_cast(met), ostr); + size_type->getDefaultSerialization()->serializeBinary(static_cast(met), ostr); } void MergeTreeIndexGranuleHypothesis::deserializeBinary(ReadBuffer & istr) { Field field_met; const auto & size_type = DataTypePtr(std::make_shared()); - size_type->deserializeBinary(field_met, istr); + size_type->getDefaultSerialization()->deserializeBinary(field_met, istr); met = field_met.get(); is_empty = false; } @@ -74,7 +74,7 @@ MergeTreeIndexConditionHypothesis::MergeTreeIndexConditionHypothesis( const String & index_name_, const String & column_name_, const SelectQueryInfo & query_, - const Context &) + ContextPtr) : index_name(index_name_) , column_name(column_name_) { @@ -141,7 +141,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexHypothesis::createIndexAggregator() co } MergeTreeIndexConditionPtr MergeTreeIndexHypothesis::createIndexCondition( - const SelectQueryInfo & query, const Context & context) const + const SelectQueryInfo & query, ContextPtr context) const { return std::make_shared(index.name, index.sample_block.getNames().front(), query, context); } diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h index 7b01cebe16d..ae11b710aa0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h @@ -65,7 +65,7 @@ public: const String & index_name_, const String & column_name_, const SelectQueryInfo & query, - const Context & context); + ContextPtr context); bool alwaysUnknownOrTrue() const override { return false; } @@ -97,7 +97,7 @@ public: MergeTreeIndexAggregatorPtr createIndexAggregator() const override; MergeTreeIndexConditionPtr createIndexCondition( - const SelectQueryInfo & query, const Context & context) const override; + const SelectQueryInfo & query, ContextPtr context) const override; bool mayBenefitFromIndexForIn(const ASTPtr & node) const override; From ad0d0ad1db4dfeb004752240232b8723d96d599c Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Wed, 28 Apr 2021 20:35:51 +0300 Subject: [PATCH 030/200] map to index --- src/CMakeLists.txt | 2 +- .../AddIndexConstraintsOptimizer.cpp | 189 ++++++++++++++++++ .../AddIndexConstraintsOptimizer.h | 31 +++ src/Interpreters/ComparisonGraph.cpp | 12 ++ src/Interpreters/ComparisonGraph.h | 2 + src/Interpreters/TreeCNFConverter.h | 26 ++- .../WhereConstraintsOptimizer.cpp | 6 +- src/Storages/SelectQueryInfo.h | 2 - .../01625_constraints_index_append.reference | 15 ++ .../01625_constraints_index_append.sql | 20 ++ 10 files changed, 299 insertions(+), 6 deletions(-) create mode 100644 src/Interpreters/AddIndexConstraintsOptimizer.cpp create mode 100644 src/Interpreters/AddIndexConstraintsOptimizer.h create mode 100644 tests/queries/0_stateless/01625_constraints_index_append.reference create mode 100644 tests/queries/0_stateless/01625_constraints_index_append.sql diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index a8cc56d4e56..16cdd989964 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -69,7 +69,7 @@ add_subdirectory (Coordination) set(dbms_headers) -set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h Interpreters/WhereConstraintsOptimizer.cpp Interpreters/WhereConstraintsOptimizer.h Interpreters/TreeCNFConverter.cpp Interpreters/TreeCNFConverter.h Interpreters/ComparisonGraph.cpp Interpreters/ComparisonGraph.h Storages/MergeTree/SubstituteColumnOptimizer.cpp Storages/MergeTree/SubstituteColumnOptimizer.h Storages/MergeTree/MergeTreeIndexHypothesis.cpp Storages/MergeTree/MergeTreeIndexHypothesis.h) +set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h Interpreters/WhereConstraintsOptimizer.cpp Interpreters/WhereConstraintsOptimizer.h Interpreters/TreeCNFConverter.cpp Interpreters/TreeCNFConverter.h Interpreters/ComparisonGraph.cpp Interpreters/ComparisonGraph.h Storages/MergeTree/SubstituteColumnOptimizer.cpp Storages/MergeTree/SubstituteColumnOptimizer.h Storages/MergeTree/MergeTreeIndexHypothesis.cpp Storages/MergeTree/MergeTreeIndexHypothesis.h Interpreters/AddIndexConstraintsOptimizer.cpp Interpreters/AddIndexConstraintsOptimizer.h) add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) diff --git a/src/Interpreters/AddIndexConstraintsOptimizer.cpp b/src/Interpreters/AddIndexConstraintsOptimizer.cpp new file mode 100644 index 00000000000..3c3c1043c0a --- /dev/null +++ b/src/Interpreters/AddIndexConstraintsOptimizer.cpp @@ -0,0 +1,189 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +AddIndexConstraintsOptimizer::AddIndexConstraintsOptimizer( + const StorageMetadataPtr & metadata_snapshot_) + : metadata_snapshot(metadata_snapshot_) +{ +} + +namespace +{ + + bool hasIndexColumns(const ASTPtr & ast, const std::unordered_set & primary_key_set) + { + const auto * identifier = ast->as(); + if (identifier && primary_key_set.contains(identifier->name())) + return true; + for (auto & child : ast->children) + if (hasIndexColumns(child, primary_key_set)) + return true; + return false; + } + + bool onlyIndexColumns(const ASTPtr & ast, const std::unordered_set & primary_key_set) + { + const auto * identifier = ast->as(); + if (identifier && !primary_key_set.contains(identifier->name())) + return false; + for (auto & child : ast->children) + if (!onlyIndexColumns(child, primary_key_set)) + return false; + return true; + } + + bool onlyConstants(const ASTPtr & ast) + { + const auto * identifier = ast->as(); + if (identifier) + return false; + for (auto & child : ast->children) + if (!onlyConstants(child)) + return false; + return true; + } + + const std::unordered_map & getRelationMap() + { + const static std::unordered_map relations = { + {"equals", ComparisonGraph::CompareResult::EQUAL}, + {"less", ComparisonGraph::CompareResult::LESS}, + {"lessOrEquals", ComparisonGraph::CompareResult::LESS_OR_EQUAL}, + {"greaterOrEquals", ComparisonGraph::CompareResult::GREATER_OR_EQUAL}, + {"greater", ComparisonGraph::CompareResult::GREATER}, + }; + return relations; + } + + const std::unordered_map & getReverseRelationMap() + { + const static std::unordered_map relations = { + {ComparisonGraph::CompareResult::EQUAL, "equals"}, + {ComparisonGraph::CompareResult::LESS, "less"}, + {ComparisonGraph::CompareResult::LESS_OR_EQUAL, "lessOrEquals"}, + {ComparisonGraph::CompareResult::GREATER_OR_EQUAL, "greaterOrEquals"}, + {ComparisonGraph::CompareResult::GREATER, "greater"}, + }; + return relations; + } + + bool canBeSequence(const ComparisonGraph::CompareResult left, const ComparisonGraph::CompareResult right) + { + using CR = ComparisonGraph::CompareResult; + if (left == CR::UNKNOWN || right == CR::UNKNOWN) + return false; + if ((left == CR::GREATER || left == CR::GREATER_OR_EQUAL) && (right == CR::LESS || right == CR::LESS_OR_EQUAL)) + return false; + if ((right == CR::GREATER || right == CR::GREATER_OR_EQUAL) && (left == CR::LESS || left == CR::LESS_OR_EQUAL)) + return false; + return true; + } + + ComparisonGraph::CompareResult mostStrict(const ComparisonGraph::CompareResult left, const ComparisonGraph::CompareResult right) + { + using CR = ComparisonGraph::CompareResult; + if (left == CR::LESS || left == CR::GREATER) + return left; + if (right == CR::LESS || right == CR::GREATER) + return right; + if (left == CR::LESS_OR_EQUAL || left == CR::GREATER_OR_EQUAL) + return left; + if (right == CR::LESS_OR_EQUAL || right == CR::GREATER_OR_EQUAL) + return right; + if (left == CR::EQUAL) + return left; + if (right == CR::EQUAL) + return right; + return left; + } + + /// Create OR-group for index_hint + CNFQuery::OrGroup createIndexHintGroup( + const CNFQuery::OrGroup & group, + const ComparisonGraph & graph, + const ASTs & primary_key_only_asts) + { + CNFQuery::OrGroup result; + for (const auto & atom : group) + { + Poco::Logger::get("INDEX_HINT_CREATE").information("CHECK"); + const auto * func = atom.ast->as(); + if (func && func->arguments->children.size() == 2 && getRelationMap().contains(func->name)) { + auto check_and_insert = [&](const size_t index, const ComparisonGraph::CompareResult need_result) -> bool { + if (!onlyConstants(func->arguments->children[1 - index])) + return false; + + for (const auto & primary_key_ast : primary_key_only_asts) + { + ComparisonGraph::CompareResult actual_result; + if (index == 0) + actual_result = graph.compare(primary_key_ast, func->arguments->children[index]); + else + actual_result = graph.compare(func->arguments->children[index], primary_key_ast); + + if (canBeSequence(need_result, actual_result)) + { + Poco::Logger::get("INDEX_HINT_CREATE").information(func->arguments->children[index]->getColumnName() + " " + primary_key_ast->getColumnName()); + ASTPtr helper_ast = func->clone(); + auto * helper_func = helper_ast->as(); + helper_func->name = getReverseRelationMap().at(mostStrict(need_result, actual_result)); + helper_func->arguments->children[index] = primary_key_ast->clone(); + result.insert(CNFQuery::AtomicFormula{atom.negative, helper_ast}); + return true; + } + } + return false; + }; + + if (!check_and_insert(1, getRelationMap().at(func->name)) && !check_and_insert(0, getRelationMap().at(func->name))) + return {}; + } + } + + return result; + } +} + +void AddIndexConstraintsOptimizer::perform(CNFQuery & cnf_query) +{ + const auto primary_key = metadata_snapshot->getColumnsRequiredForPrimaryKey(); + const auto & graph = metadata_snapshot->getConstraints().getGraph(); + const std::unordered_set primary_key_set(std::begin(primary_key), std::end(primary_key)); + + ASTs primary_key_only_asts; + for (const auto & vertex : graph.getVertexes()) + for (const auto & ast : vertex) + if (hasIndexColumns(ast, primary_key_set) && onlyIndexColumns(ast, primary_key_set)) + primary_key_only_asts.push_back(ast); + + CNFQuery::AndGroup and_group; + cnf_query.iterateGroups([&and_group, &graph, &primary_key_only_asts](const auto & or_group) { + auto add_group = createIndexHintGroup(or_group, graph, primary_key_only_asts); + if (!add_group.empty()) + and_group.emplace(std::move(add_group)); + }); + if (!and_group.empty()) + { + CNFQuery::OrGroup new_or_group; + new_or_group.insert(CNFQuery::AtomicFormula{false, makeASTFunction("indexHint", TreeCNFConverter::fromCNF(CNFQuery(std::move(and_group))))}); + cnf_query.appendGroup(CNFQuery::AndGroup{new_or_group}); + } +} + +} diff --git a/src/Interpreters/AddIndexConstraintsOptimizer.h b/src/Interpreters/AddIndexConstraintsOptimizer.h new file mode 100644 index 00000000000..3ec7b2f3594 --- /dev/null +++ b/src/Interpreters/AddIndexConstraintsOptimizer.h @@ -0,0 +1,31 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; + +class AddIndexConstraintsOptimizer final +{ +public: + AddIndexConstraintsOptimizer( + const StorageMetadataPtr & metadata_snapshot); + + void perform(CNFQuery & cnf_query); + +private: + const StorageMetadataPtr & metadata_snapshot; +}; + +} diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index da0bd25c99e..93f13322916 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -316,6 +316,18 @@ ComparisonGraph::Graph ComparisonGraph::reverseGraph(const Graph & asts_graph) c return asts_graph; } +std::vector ComparisonGraph::getVertexes() const +{ + std::vector result; + for (const auto & vertex : graph.vertexes) + { + result.emplace_back(); + for (const auto & ast : vertex.asts) + result.back().push_back(ast); + } + return result; +} + void ComparisonGraph::dfsComponents( const Graph & reversed_graph, size_t v, std::vector & components, const size_t not_visited, const size_t component) const { diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index a6b40fc148a..40f86c8ceee 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -41,6 +41,8 @@ public: std::optional> getConstUpperBound(const ASTPtr & ast) const; std::optional> getConstLowerBound(const ASTPtr & ast) const; + std::vector getVertexes() const; + private: /// strongly connected component struct EqualComponent diff --git a/src/Interpreters/TreeCNFConverter.h b/src/Interpreters/TreeCNFConverter.h index 5fc0ba36b6a..1398f0314b6 100644 --- a/src/Interpreters/TreeCNFConverter.h +++ b/src/Interpreters/TreeCNFConverter.h @@ -20,7 +20,16 @@ public: /// for set bool operator<(const AtomicFormula & rhs) const { - return ast == rhs.ast ? negative < rhs.negative : ast < rhs.ast; + return ast->getTreeHash() == rhs.ast->getTreeHash() + ? negative < rhs.negative + : ast->getTreeHash() < rhs.ast->getTreeHash(); + } + + bool operator==(const AtomicFormula & rhs) const + { + return negative == rhs.negative && + ast->getTreeHash() == rhs.ast->getTreeHash() && + ast->getColumnName() == rhs.ast->getColumnName(); } }; @@ -71,6 +80,21 @@ public: return *this; } + template + CNFQuery & iterateGroups(F func) + { + for (const auto & group : statements) + func(group); + return *this; + } + + CNFQuery & appendGroup(AndGroup&& and_group) + { + for (auto && or_group : and_group) + statements.emplace(std::move(or_group)); + return *this; + } + template CNFQuery & transformGroups(F func) { diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index 95e5fd35046..75a2380986c 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -229,8 +230,8 @@ void WhereConstraintsOptimizer::perform() { if (select_query->where() && metadata_snapshot) { - const auto constraint_data = metadata_snapshot->getConstraints().getConstraintData(); - const auto compare_graph = metadata_snapshot->getConstraints().getGraph(); + const auto & constraint_data = metadata_snapshot->getConstraints().getConstraintData(); + const auto & compare_graph = metadata_snapshot->getConstraints().getGraph(); Poco::Logger::get("BEFORE CNF ").information(select_query->where()->dumpTree()); auto cnf = TreeCNFConverter::toCNF(select_query->where()); Poco::Logger::get("BEFORE OPT").information(cnf.dump()); @@ -245,6 +246,7 @@ void WhereConstraintsOptimizer::perform() return replaceTermsToConstants(atom, compare_graph); }) .pushNotInFuntions(); + AddIndexConstraintsOptimizer(metadata_snapshot).perform(cnf); Poco::Logger::get("AFTER OPT").information(cnf.dump()); select_query->setExpression(ASTSelectQuery::Expression::WHERE, TreeCNFConverter::fromCNF(cnf)); diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index f017b73f2ba..b4ac07c612a 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -131,8 +131,6 @@ struct SelectQueryInfo PrewhereInfoPtr prewhere_info; - ASTPtr index_hint; - ReadInOrderOptimizerPtr order_optimizer; /// Can be modified while reading from storage InputOrderInfoPtr input_order_info; diff --git a/tests/queries/0_stateless/01625_constraints_index_append.reference b/tests/queries/0_stateless/01625_constraints_index_append.reference new file mode 100644 index 00000000000..867f65bb426 --- /dev/null +++ b/tests/queries/0_stateless/01625_constraints_index_append.reference @@ -0,0 +1,15 @@ +SELECT i +FROM constraint_test.test +PREWHERE a = 0 +WHERE (a = 0) AND indexHint((i + 40) > 0) +SELECT i +FROM constraint_test.test +PREWHERE a < 0 +SELECT i +FROM constraint_test.test +PREWHERE a >= 0 +WHERE (a >= 0) AND indexHint((i + 40) > 0) +SELECT i +FROM constraint_test.test +PREWHERE (2 * b) < 100 +WHERE ((2 * b) < 100) AND indexHint(i < 100) diff --git a/tests/queries/0_stateless/01625_constraints_index_append.sql b/tests/queries/0_stateless/01625_constraints_index_append.sql new file mode 100644 index 00000000000..8d668d1a306 --- /dev/null +++ b/tests/queries/0_stateless/01625_constraints_index_append.sql @@ -0,0 +1,20 @@ +SET convert_query_to_cnf = 1; +SET optimize_using_constraints = 1; +SET optimize_move_to_prewhere = 1; + +DROP DATABASE IF EXISTS constraint_test; +DROP TABLE IF EXISTS constraint_test.test; + +CREATE DATABASE constraint_test; + +CREATE TABLE constraint_test.test (i Int64, a UInt32, b UInt64, CONSTRAINT c1 ASSUME i <= 2 * b AND i + 40 > a) ENGINE = MergeTree() ORDER BY i; +INSERT INTO constraint_test.test VALUES (1, 10, 1), (2, 20, 2); + +EXPLAIN SYNTAX SELECT i FROM constraint_test.test WHERE a = 0; +EXPLAIN SYNTAX SELECT i FROM constraint_test.test WHERE a < 0; +EXPLAIN SYNTAX SELECT i FROM constraint_test.test WHERE a >= 0; +EXPLAIN SYNTAX SELECT i FROM constraint_test.test WHERE 2 * b < 100; + +DROP TABLE constraint_test.test; + +DROP DATABASE constraint_test; \ No newline at end of file From 460bff4d895711fc9cad13e8e32c3e505fda3317 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Wed, 28 Apr 2021 20:48:29 +0300 Subject: [PATCH 031/200] fix logs --- src/Interpreters/ComparisonGraph.cpp | 24 +++++++++++------------- 1 file changed, 11 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 93f13322916..44f8856e191 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -83,7 +83,7 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) if (index_left != bad_term && index_right != bad_term) { - Poco::Logger::get("Edges").information("GOOD: " + atom->dumpTree()); + //Poco::Logger::get("Edges").information("GOOD: " + atom->dumpTree()); Poco::Logger::get("Edges").information("left=" + std::to_string(index_left) + " right=" + std::to_string(index_right)); Poco::Logger::get("Edges").information("sz=" + std::to_string(g.edges.size())); g.edges[index_right].push_back(Edge{it->second, index_left}); @@ -95,7 +95,7 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) } else { - Poco::Logger::get("Edges").information("BAD: " + atom->dumpTree()); + //Poco::Logger::get("Edges").information("BAD: " + atom->dumpTree()); } } } @@ -118,8 +118,8 @@ std::pair ComparisonGraph::findPath(const size_t start, const size_t else { Poco::Logger::get("dists found").information(std::to_string(start) + " " + std::to_string(finish) + " : " + std::to_string(static_cast(it->second))); - Poco::Logger::get("dists found").information(graph.vertexes[start].asts.back()->dumpTree()); - Poco::Logger::get("dists found").information(graph.vertexes[finish].asts.back()->dumpTree()); + //Poco::Logger::get("dists found").information(graph.vertexes[start].asts.back()->dumpTree()); + //Poco::Logger::get("dists found").information(graph.vertexes[finish].asts.back()->dumpTree()); return {true, it->second == Path::LESS}; } } @@ -151,7 +151,7 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con const auto right_bound = getConstUpperBound(right); if (left_bound && right_bound) { - Poco::Logger::get("&&&&&&&").information(left_bound->first.dump() + " " + std::to_string(left_bound->second) + " | " + right_bound->first.dump() + " " + std::to_string(right_bound->second)); + //Poco::Logger::get("&&&&&&&").information(left_bound->first.dump() + " " + std::to_string(left_bound->second) + " | " + right_bound->first.dump() + " " + std::to_string(right_bound->second)); if (left_bound->first < right_bound->first) return CompareResult::UNKNOWN; else if (left_bound->first > right_bound->first) @@ -165,7 +165,7 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con { const auto left_bound = getConstUpperBound(left); const auto right_bound = getConstLowerBound(right); - Poco::Logger::get("!!!!!!").information(left_bound->first.dump() + " " + std::to_string(left_bound->second) + " | " + right_bound->first.dump() + " " + std::to_string(right_bound->second)); + //Poco::Logger::get("!!!!!!").information(left_bound->first.dump() + " " + std::to_string(left_bound->second) + " | " + right_bound->first.dump() + " " + std::to_string(right_bound->second)); if (left_bound && right_bound) { if (left_bound->first > right_bound->first) @@ -348,9 +348,9 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as const auto n = asts_graph.vertexes.size(); for (size_t v = 0; v < n; ++v) { - Poco::LogStream{"kek"}.information() << "VERTEX " << v << " " << asts_graph.vertexes[v].asts.back()->dumpTree() << std::endl; + //Poco::LogStream{"kek"}.information() << "VERTEX " << v << " " << asts_graph.vertexes[v].asts.back()->dumpTree() << std::endl; for (const auto & edge : asts_graph.edges[v]) { - Poco::LogStream{"kek"}.information() << "TO " << edge.to << " " << static_cast(edge.type) << std::endl; + //Poco::LogStream{"kek"}.information() << "TO " << edge.to << " " << static_cast(edge.type) << std::endl; } } @@ -422,14 +422,12 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as { const auto * left = result.vertexes[v].getConstant()->as(); const auto * right = result.vertexes[u].getConstant()->as(); - Poco::Logger::get("Graph").information("kek"); - Poco::Logger::get("Graph").information(left->value.dump() + " " + right->value.dump()); + //Poco::Logger::get("Graph").information(left->value.dump() + " " + right->value.dump()); /// Only less. Equal constant fields = equal literals so it was already considered above. if (left->value > right->value) { result.edges[v].push_back(Edge{Edge::LESS, u}); - Poco::Logger::get("Graph").information("1111 > "); } } } @@ -438,12 +436,12 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as Poco::Logger::get("Graph").information("finish"); - for (size_t v = 0; v < result.vertexes.size(); ++v) { + /* for (size_t v = 0; v < result.vertexes.size(); ++v) { Poco::LogStream{"kekkek"}.information() << "VERTEX " << v << " " << result.vertexes[v].asts.back()->dumpTree() << std::endl; for (const auto & edge : result.edges[v]) { Poco::LogStream{"kekkek"}.information() << "TO " << edge.to << " " << static_cast(edge.type) << std::endl; } - } + }*/ for (size_t v = 0; v < result.vertexes.size(); ++v) { From 464d779da178ff0fb3de0bc3fff2892279fed965 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 29 Apr 2021 21:12:51 +0300 Subject: [PATCH 032/200] optimization columns --- src/Interpreters/ComparisonGraph.cpp | 4 +- .../MergeTree/SubstituteColumnOptimizer.cpp | 249 +++++++++++++----- 2 files changed, 184 insertions(+), 69 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index a3efa63ee9f..57d1c398d05 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -361,12 +361,12 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as /// Find strongly connected component const auto n = asts_graph.vertexes.size(); - for (size_t v = 0; v < n; ++v) { + /*for (size_t v = 0; v < n; ++v) { //Poco::LogStream{"kek"}.information() << "VERTEX " << v << " " << asts_graph.vertexes[v].asts.back()->dumpTree() << std::endl; for (const auto & edge : asts_graph.edges[v]) { //Poco::LogStream{"kek"}.information() << "TO " << edge.to << " " << static_cast(edge.type) << std::endl; } - } + }*/ std::vector order; { diff --git a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp index e832087bab1..15974adbb0a 100644 --- a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp +++ b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp @@ -22,6 +22,8 @@ namespace { const String COMPONENT = "__constraint_component_"; +constexpr UInt64 COLUMN_PENALTY = 0; //1024 * 1024 * 10; +//constexpr size_t MAX_COMPONENTS_FOR_BRUTEFORCE = 10; class ComponentMatcher { @@ -31,9 +33,10 @@ public: struct Data { const ComparisonGraph & graph; + std::set & components; - Data(const ComparisonGraph & graph_) - : graph(graph_) + Data(const ComparisonGraph & graph_, std::set & components_) + : graph(graph_), components(components_) { } }; @@ -42,7 +45,10 @@ public: { const auto id = data.graph.getComponentId(ast); if (id) + { ast = std::make_shared(COMPONENT + std::to_string(id.value())); + data.components.insert(id.value()); + } } static bool needChildVisit(const ASTPtr &, const ASTPtr &) @@ -54,32 +60,58 @@ public: using ComponentVisitor = ComponentMatcher::Visitor; -class IdentifierSetMatcher +void collectIdentifiers(const ASTPtr & ast, std::unordered_set & identifiers) { -public: - using Visitor = InDepthNodeVisitor; - - struct Data + const auto * identifier = ast->as(); + if (identifier) + identifiers.insert(identifier->name()); + else { - std::unordered_set identifiers; - }; + for (const auto & child : ast->children) + collectIdentifiers(child, identifiers); + } +} - static void visit(ASTPtr & ast, Data & data) +struct ColumnPrice +{ + size_t compressed_size; + size_t uncompressed_size; + + ColumnPrice(const size_t compressed_size_, const size_t uncompressed_size_) + : compressed_size(compressed_size_) + , uncompressed_size(uncompressed_size_) + {} + + ColumnPrice() + : ColumnPrice(0, 0) + {} + + bool operator<(const ColumnPrice & that) const { - const auto * identifier = ast->as(); - if (identifier) - data.identifiers.insert(identifier->name()); + return std::tie(compressed_size, uncompressed_size) < std::tie(that.compressed_size, that.uncompressed_size); } - static bool needChildVisit(const ASTPtr &, const ASTPtr &) + ColumnPrice operator+(ColumnPrice that) const { - return true; + that += *this; + return that; + } + + ColumnPrice & operator+=(const ColumnPrice & that) + { + compressed_size += that.compressed_size; + uncompressed_size += that.uncompressed_size; + return *this; + } + + ColumnPrice & operator-=(const ColumnPrice & that) + { + compressed_size -= that.compressed_size; + uncompressed_size -= that.uncompressed_size; + return *this; } }; -using IdentifierSetVisitor = IdentifierSetMatcher::Visitor; - - class SubstituteColumnMatcher { public: @@ -87,51 +119,14 @@ public: struct Data { - const ComparisonGraph & graph; - const std::unordered_set & identifiers; - ConstStoragePtr storage; - - Data(const ComparisonGraph & graph_, - const std::unordered_set & identifiers_, - const ConstStoragePtr & storage_) - : graph(graph_) - , identifiers(identifiers_) - , storage(storage_) - { - } + std::unordered_map id_to_expression_map; }; static void visit(ASTPtr & ast, Data & data) { const auto * identifier = ast->as(); - if (identifier && identifier->name().starts_with(COMPONENT)) - { - const std::size_t id = std::stoll(identifier->name().substr(COMPONENT.size(), identifier->name().size())); - // like TreeRewriter - struct ColumnSizeTuple - { - size_t compressed_size; - size_t uncompressed_size; - const ASTPtr & ast; - - bool operator<(const ColumnSizeTuple & that) const - { - return std::tie(compressed_size, uncompressed_size) < std::tie(that.compressed_size, that.uncompressed_size); - } - }; - - const auto column_sizes = data.storage->getColumnSizes(); - - std::vector columns; - for (const auto & equal_ast : data.graph.getComponent(id)) - { - if (const auto it = column_sizes.find(equal_ast->getColumnName()); it != std::end(column_sizes)) - columns.push_back({it->second.data_compressed, it->second.data_uncompressed, equal_ast}); - } - - if (!columns.empty()) - ast = std::min_element(std::begin(columns), std::end(columns))->ast->clone(); - } + if (identifier && data.id_to_expression_map.contains(identifier->name())) + ast = data.id_to_expression_map.at(identifier->name())->clone(); } static bool needChildVisit(const ASTPtr &, const ASTPtr &) @@ -141,6 +136,71 @@ public: }; using SubstituteColumnVisitor = SubstituteColumnMatcher::Visitor; + +ColumnPrice calculatePrice( + const std::unordered_map & column_prices, + std::unordered_set identifiers) +{ + ColumnPrice result(0, 0); + for (const auto & ident : identifiers) + result = result + column_prices.at(ident); + return result; +} + +// TODO: branch-and-bound +void bruteforce( + const ComparisonGraph & graph, + const std::vector & components, + size_t current_component, + const std::unordered_map & column_prices, + ColumnPrice current_price, + std::vector & expressions_stack, + ColumnPrice & min_price, + std::vector & min_expressions) +{ + if (current_component == components.size()) + { + Poco::Logger::get("New price").information(std::to_string(current_price.compressed_size)); + for (const auto & ast : expressions_stack) + Poco::Logger::get("AST").information(ast->getColumnName()); + if (current_price < min_price) + { + min_price = current_price; + min_expressions = expressions_stack; + Poco::Logger::get("New price").information("UPDATE"); + } + } + else + { + for (const auto & ast : graph.getComponent(components[current_component])) + { + std::unordered_set identifiers; + collectIdentifiers(ast, identifiers); + ColumnPrice expression_price = calculatePrice(column_prices, identifiers); + Poco::Logger::get("EXPRPRICE").information( ast->getColumnName()+ " " + std::to_string(expression_price.compressed_size)); + + expressions_stack.push_back(ast); + current_price += expression_price; + + std::unordered_map new_prices(column_prices); + for (const auto & identifier : identifiers) + new_prices[identifier] = ColumnPrice(0, 0); + + bruteforce(graph, + components, + current_component + 1, + new_prices, + current_price, + expressions_stack, + min_price, + min_expressions); + + current_price -= expression_price; + expressions_stack.pop_back(); + } + } +} + } @@ -164,6 +224,13 @@ void SubstituteColumnOptimizer::perform() { if (!storage) return; + const auto column_sizes = storage->getColumnSizes(); + if (column_sizes.empty()) + { + Poco::Logger::get("SubstituteColumnOptimizer").information("skip: column sizes not available"); + return; + } + const auto compare_graph = metadata_snapshot->getConstraints().getGraph(); auto run_for_all = [&](const auto func) { @@ -177,24 +244,72 @@ void SubstituteColumnOptimizer::perform() func(select_query->refHaving()); }; - ComponentVisitor::Data component_data(compare_graph); - IdentifierSetVisitor::Data identifier_data; + std::set components; + ComponentVisitor::Data component_data(compare_graph, components); + std::unordered_set identifiers; auto preprocess = [&](ASTPtr & ast) { ComponentVisitor(component_data).visit(ast); - IdentifierSetVisitor(identifier_data).visit(ast); + collectIdentifiers(ast, identifiers); + Poco::Logger::get("kek").information(ast->dumpTree()); }; + run_for_all(preprocess); + + const auto primary_key = metadata_snapshot->getColumnsRequiredForPrimaryKey(); + const std::unordered_set primary_key_set(std::begin(primary_key), std::end(primary_key)); + std::unordered_map column_prices; + for (const auto & [column_name, column_size] : column_sizes) + { + column_prices[column_name] = ColumnPrice( + column_size.data_compressed + COLUMN_PENALTY, column_size.data_uncompressed); + Poco::Logger::get("COLUMNS").information(column_name + " " + std::to_string(column_size.data_compressed) + " " + std::to_string(column_size.data_uncompressed)); + } + for (const auto & column_name : primary_key) + { + column_prices[column_name] = ColumnPrice(0, 0); + Poco::Logger::get("PK COLUMNS").information(column_name + " " + std::to_string(column_prices[column_name].compressed_size)); + } + for (const auto & column_name : identifiers) + { + column_prices[column_name] = ColumnPrice(0, 0); + Poco::Logger::get("ID COLUMNS").information(column_name + " " + std::to_string(column_prices[column_name].compressed_size)); + } + + std::unordered_map id_to_expression_map; + std::vector components_list; + for (const UInt64 component : components) + if (compare_graph.getComponent(component).size() == 1) + id_to_expression_map[COMPONENT + std::to_string(component)] = compare_graph.getComponent(component).front(); + else + components_list.push_back(component); + + /*if (components_list.size() > MAX_COMPONENTS_FOR_BRUTEFORCE) + { + Poco::Logger::get("SubstituteColumnOptimizer").information("skip: too many components for bruteforce"); + return; + }*/ + + std::vector expressions_stack; + ColumnPrice min_price(std::numeric_limits::max(), std::numeric_limits::max()); + std::vector min_expressions; + bruteforce(compare_graph, + components_list, + 0, + column_prices, + ColumnPrice(0, 0), + expressions_stack, + min_price, + min_expressions); + + for (size_t i = 0; i < min_expressions.size(); ++i) + id_to_expression_map[COMPONENT + std::to_string(components_list[i])] = min_expressions[i]; + auto process = [&](ASTPtr & ast) { - SubstituteColumnVisitor::Data substitute_data(compare_graph, identifier_data.identifiers, storage); + SubstituteColumnVisitor::Data substitute_data{id_to_expression_map}; SubstituteColumnVisitor(substitute_data).visit(ast); }; - ASTPtr old_query = select_query->clone(); - - run_for_all(preprocess); run_for_all(process); - - } } From 4ddabcbc5f6e38d9df8303bf79cfea350c8f9c24 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 29 Apr 2021 21:13:32 +0300 Subject: [PATCH 033/200] test --- .../01623_constraints_column_swap.reference | 16 ++++++++++++++++ .../01623_constraints_column_swap.sql | 9 +++++++-- 2 files changed, 23 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index 3c0278a5ccd..e334c6f1c4e 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -18,6 +18,19 @@ SELECT b + 3 FROM constraint_test.test WHERE b = 1 +SELECT b + 10 +FROM constraint_test.test +WHERE b = 0 +SELECT + cityHash64(a) + 10, + a +FROM constraint_test.test +WHERE cityHash64(a) = 0 +SELECT + cityHash64(a) + 10, + a +FROM constraint_test.test +WHERE cityHash64(a) = 0 SELECT a, a @@ -28,3 +41,6 @@ SELECT a FROM constraint_test.test WHERE a = \'c\' +SELECT a +FROM constraint_test.test +WHERE a = \'c\' diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.sql b/tests/queries/0_stateless/01623_constraints_column_swap.sql index 47c52c9ad07..eb3188f885a 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.sql +++ b/tests/queries/0_stateless/01623_constraints_column_swap.sql @@ -9,20 +9,25 @@ CREATE DATABASE constraint_test; CREATE TABLE constraint_test.test (i Int64, a String, b UInt64, CONSTRAINT c1 ASSUME b = cityHash64(a)) ENGINE = MergeTree() ORDER BY i; INSERT INTO constraint_test.test VALUES (1, 'cat', 1), (2, 'dog', 2); +INSERT INTO constraint_test.test SELECT number AS i, format('test {} kek {}', toString(number), toString(number + 10)) AS a, 1 AS b FROM system.numbers LIMIT 1000000; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM constraint_test.test WHERE cityHash64(a) = 1; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM constraint_test.test WHERE cityHash64(a) = 0; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM constraint_test.test WHERE b = 0; EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM constraint_test.test WHERE b = 1; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10 FROM constraint_test.test WHERE cityHash64(a) = 0; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, a FROM constraint_test.test WHERE cityHash64(a) = 0; +EXPLAIN SYNTAX SELECT b + 10, a FROM constraint_test.test WHERE b = 0; + DROP TABLE constraint_test.test; - CREATE TABLE constraint_test.test (i Int64, a String, b String, CONSTRAINT c1 ASSUME a = substring(reverse(b), 1, 1)) ENGINE = MergeTree() ORDER BY i; -INSERT INTO constraint_test.test VALUES (1, 'c', 'aaaaaaaaaaaaaaaaaac'), (2, 'd', 'bbbbbbbbbbbbbbbd'); +INSERT INTO constraint_test.test SELECT number AS i, toString(number) AS a, format('test {} kek {}', toString(number), toString(number + 10)) b FROM system.numbers LIMIT 1000000; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM constraint_test.test WHERE a = 'c'; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM constraint_test.test WHERE substring(reverse(b), 1, 1) = 'c'; +EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) FROM constraint_test.test WHERE substring(reverse(b), 1, 1) = 'c'; DROP TABLE constraint_test.test; From c8e84928b9006112307469333f9b271ed7c8ce6c Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 29 Apr 2021 21:54:54 +0300 Subject: [PATCH 034/200] fix --- src/Storages/MergeTree/SubstituteColumnOptimizer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp index 15974adbb0a..1104e1078f3 100644 --- a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp +++ b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp @@ -22,7 +22,7 @@ namespace { const String COMPONENT = "__constraint_component_"; -constexpr UInt64 COLUMN_PENALTY = 0; //1024 * 1024 * 10; +constexpr UInt64 COLUMN_PENALTY = 10 * 1024 * 1024; //constexpr size_t MAX_COMPONENTS_FOR_BRUTEFORCE = 10; class ComponentMatcher From e565bc47a81f419df2cf0eaf97509274d6129317 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Fri, 30 Apr 2021 12:34:00 +0300 Subject: [PATCH 035/200] settings off --- src/Core/Settings.h | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 941609a8afe..50adbadb183 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -394,14 +394,16 @@ class IColumn; M(Bool, optimize_move_functions_out_of_any, false, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \ M(Bool, optimize_normalize_count_variants, true, "Rewrite aggregate functions that semantically equals to count() as count().", 0) \ M(Bool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() functions.", 0) \ - M(Bool, convert_query_to_cnf, true, "Convert SELECT query to CNF", 0) \ + M(Bool, convert_query_to_cnf, false, "Convert SELECT query to CNF", 0) \ M(Bool, optimize_arithmetic_operations_in_aggregate_functions, true, "Move arithmetic operations out of aggregation functions", 0) \ M(Bool, optimize_duplicate_order_by_and_distinct, true, "Remove duplicate ORDER BY and DISTINCT if it's possible", 0) \ M(Bool, optimize_redundant_functions_in_order_by, true, "Remove functions from ORDER BY if its argument is also in ORDER BY", 0) \ M(Bool, optimize_if_chain_to_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \ M(Bool, optimize_if_transform_strings_to_enum, false, "Replaces string-type arguments in If and Transform to enum. Disabled by default cause it could make inconsistent change in distributed query that would lead to its fail.", 0) \ M(Bool, optimize_monotonous_functions_in_order_by, true, "Replace monotonous function with its argument in ORDER BY", 0) \ - M(Bool, optimize_using_constraints, true, "Use constraints for query optimization", 0) \ + M(Bool, optimize_using_constraints, false, "Use constraints for query optimization", 0) \ + M(Bool, optimize_substitute_columns, false, "Use constraints for column substitution", 0) \ + M(Bool, optimize_append_index, false, "Use constraints in order to append index condition (indexHint)", 0) \ M(Bool, normalize_function_names, true, "Normalize function names to their canonical names", 0) \ M(Bool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \ M(Bool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \ From 564a48464252d249d6278aab52b00c3ab8c3f4bc Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sun, 2 May 2021 22:16:40 +0300 Subject: [PATCH 036/200] fix --- src/CMakeLists.txt | 2 +- src/Interpreters/ComparisonGraph.cpp | 48 +++++ src/Interpreters/ComparisonGraph.h | 8 +- src/Interpreters/TreeCNFConverter.h | 2 +- .../WhereConstraintsOptimizer.cpp | 38 +--- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 174 +++++++++++++++- .../MergeTree/MergeTreeDataSelectExecutor.h | 12 ++ .../MergeTree/MergeTreeIndexHypothesis.cpp | 68 +------ .../MergeTree/MergeTreeIndexHypothesis.h | 37 +--- .../MergeTreeIndexMergedCondition.cpp | 186 ++++++++++++++++++ .../MergeTree/MergeTreeIndexMergedCondition.h | 47 +++++ src/Storages/MergeTree/MergeTreeIndices.h | 4 + 12 files changed, 487 insertions(+), 139 deletions(-) create mode 100644 src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp create mode 100644 src/Storages/MergeTree/MergeTreeIndexMergedCondition.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 16cdd989964..f6f4d5b6199 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -69,7 +69,7 @@ add_subdirectory (Coordination) set(dbms_headers) -set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h Interpreters/WhereConstraintsOptimizer.cpp Interpreters/WhereConstraintsOptimizer.h Interpreters/TreeCNFConverter.cpp Interpreters/TreeCNFConverter.h Interpreters/ComparisonGraph.cpp Interpreters/ComparisonGraph.h Storages/MergeTree/SubstituteColumnOptimizer.cpp Storages/MergeTree/SubstituteColumnOptimizer.h Storages/MergeTree/MergeTreeIndexHypothesis.cpp Storages/MergeTree/MergeTreeIndexHypothesis.h Interpreters/AddIndexConstraintsOptimizer.cpp Interpreters/AddIndexConstraintsOptimizer.h) +set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h Interpreters/WhereConstraintsOptimizer.cpp Interpreters/WhereConstraintsOptimizer.h Interpreters/TreeCNFConverter.cpp Interpreters/TreeCNFConverter.h Interpreters/ComparisonGraph.cpp Interpreters/ComparisonGraph.h Storages/MergeTree/SubstituteColumnOptimizer.cpp Storages/MergeTree/SubstituteColumnOptimizer.h Storages/MergeTree/MergeTreeIndexHypothesis.cpp Storages/MergeTree/MergeTreeIndexHypothesis.h Interpreters/AddIndexConstraintsOptimizer.cpp Interpreters/AddIndexConstraintsOptimizer.h Storages/MergeTree/MergeTreeIndexMergedCondition.cpp Storages/MergeTree/MergeTreeIndexMergedCondition.h) add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 57d1c398d05..8a4d9f0e0be 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -202,6 +202,54 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con return CompareResult::UNKNOWN; } +bool ComparisonGraph::isPossibleCompare(const CompareResult expected, const ASTPtr & left, const ASTPtr & right) const +{ + const auto result = compare(left, right); + + if (expected == CompareResult::UNKNOWN || result == CompareResult::UNKNOWN) + { + Poco::Logger::get("isPossibleCompare").information("unknonw"); + return true; + } + if (expected == result) + return true; + + static const std::set> possible_pairs = { + {CompareResult::EQUAL, CompareResult::LESS_OR_EQUAL}, + {CompareResult::EQUAL, CompareResult::GREATER_OR_EQUAL}, + {CompareResult::LESS_OR_EQUAL, CompareResult::LESS}, + {CompareResult::LESS_OR_EQUAL, CompareResult::EQUAL}, + {CompareResult::GREATER_OR_EQUAL, CompareResult::GREATER}, + {CompareResult::GREATER_OR_EQUAL, CompareResult::EQUAL}, + {CompareResult::LESS, CompareResult::LESS}, + {CompareResult::LESS, CompareResult::LESS_OR_EQUAL}, + {CompareResult::GREATER, CompareResult::GREATER}, + {CompareResult::GREATER, CompareResult::GREATER_OR_EQUAL}, + }; + + return possible_pairs.contains({expected, result}); +} + +bool ComparisonGraph::isAlwaysCompare(const CompareResult expected, const ASTPtr & left, const ASTPtr & right) const +{ + const auto result = compare(left, right); + + if (expected == CompareResult::UNKNOWN || result == CompareResult::UNKNOWN) + return false; + if (expected == result) + return true; + + static const std::set> possible_pairs = { + {CompareResult::LESS_OR_EQUAL, CompareResult::LESS}, + {CompareResult::LESS_OR_EQUAL, CompareResult::EQUAL}, + {CompareResult::GREATER_OR_EQUAL, CompareResult::GREATER}, + {CompareResult::GREATER_OR_EQUAL, CompareResult::EQUAL}, + }; + + return possible_pairs.contains({expected, result}); +} + + std::vector ComparisonGraph::getEqual(const ASTPtr & ast) const { const auto res = getComponentId(ast); diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 6eb396d2879..9fee991c399 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -18,8 +18,6 @@ class ComparisonGraph public: ComparisonGraph(const std::vector & atomic_formulas); - /// Works for string and num. - /// For other -- only eq. enum class CompareResult { LESS, @@ -32,6 +30,12 @@ public: CompareResult compare(const ASTPtr & left, const ASTPtr & right) const; + /// It's possible that left right + bool isPossibleCompare(const CompareResult expected, const ASTPtr & left, const ASTPtr & right) const; + + /// It's always true that left right + bool isAlwaysCompare(const CompareResult expected, const ASTPtr & left, const ASTPtr & right) const; + std::vector getEqual(const ASTPtr & ast) const; std::optional getEqualConst(const ASTPtr & ast) const; diff --git a/src/Interpreters/TreeCNFConverter.h b/src/Interpreters/TreeCNFConverter.h index 1398f0314b6..af5659af678 100644 --- a/src/Interpreters/TreeCNFConverter.h +++ b/src/Interpreters/TreeCNFConverter.h @@ -81,7 +81,7 @@ public: } template - CNFQuery & iterateGroups(F func) + const CNFQuery & iterateGroups(F func) const { for (const auto & group : statements) func(group); diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index 75a2380986c..d3034ae4316 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -128,23 +128,7 @@ bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const Comparis if (func && func->arguments->children.size() == 2) { const auto expected = getExpectedCompare(atom); - const auto result = graph.compare(func->arguments->children[0], func->arguments->children[1]); - Poco::Logger::get("GRAPH REASON").information("neg: " + std::to_string(atom.negative)); - Poco::Logger::get("GRAPH REASON").information(atom.ast->dumpTree()); - Poco::Logger::get("GRAPH REASON").information(std::to_string(static_cast(expected)) + " " + std::to_string(static_cast(result))); - - if (expected == ComparisonGraph::CompareResult::UNKNOWN || result == ComparisonGraph::CompareResult::UNKNOWN) - return false; - - if (expected == result) - return true; - if (result == ComparisonGraph::CompareResult::EQUAL && - (expected == ComparisonGraph::CompareResult::LESS_OR_EQUAL || expected == ComparisonGraph::CompareResult::GREATER_OR_EQUAL)) - return true; - if (result == ComparisonGraph::CompareResult::LESS && expected == ComparisonGraph::CompareResult::LESS_OR_EQUAL) - return true; - if (result == ComparisonGraph::CompareResult::GREATER && expected == ComparisonGraph::CompareResult::GREATER_OR_EQUAL) - return true; + return graph.isAlwaysCompare(expected, func->arguments->children[0], func->arguments->children[1]); } } return false; @@ -177,25 +161,7 @@ bool checkIfAtomAlwaysFalseGraph(const CNFQuery::AtomicFormula & atom, const Com { /// TODO: special support for != const auto expected = getExpectedCompare(atom); - const auto result = graph.compare(func->arguments->children[0], func->arguments->children[1]); - Poco::Logger::get("GRAPH REASON F").information("neg: " + std::to_string(atom.negative)); - Poco::Logger::get("GRAPH REASON F").information(atom.ast->dumpTree()); - Poco::Logger::get("GRAPH REASON F").information(std::to_string(static_cast(expected)) + " " + std::to_string(static_cast(result))); - - if (expected == ComparisonGraph::CompareResult::UNKNOWN || result == ComparisonGraph::CompareResult::UNKNOWN) - return false; - - if (expected == result) - return false; - else if (result == ComparisonGraph::CompareResult::EQUAL && - (expected == ComparisonGraph::CompareResult::LESS_OR_EQUAL || expected == ComparisonGraph::CompareResult::GREATER_OR_EQUAL)) - return false; - else if (result == ComparisonGraph::CompareResult::LESS && expected == ComparisonGraph::CompareResult::LESS_OR_EQUAL) - return false; - else if (result == ComparisonGraph::CompareResult::GREATER && expected == ComparisonGraph::CompareResult::GREATER_OR_EQUAL) - return false; - else - return true; + return !graph.isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1]); } return false; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 8245364d87a..ae692439038 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -611,14 +611,52 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( { } }; + + struct MergedDataSkippingIndexAndCondition + { + std::vector indices; + MergeTreeIndexMergedConditionPtr condition; + std::atomic total_granules{0}; + std::atomic granules_dropped{0}; + std::atomic total_parts{0}; + std::atomic parts_dropped{0}; + + MergedDataSkippingIndexAndCondition(MergeTreeIndexMergedConditionPtr condition_) + : condition(condition_) + { + } + + void addIndex(const MergeTreeIndexPtr & index) + { + indices.push_back(index); + condition->addIndex(indices.back()); + } + }; + std::list useful_indices; + std::unordered_map> merged_indices; for (const auto & index : metadata_snapshot->getSecondaryIndices()) { auto index_helper = MergeTreeIndexFactory::instance().get(index); - auto condition = index_helper->createIndexCondition(query_info, context); - if (!condition->alwaysUnknownOrTrue()) - useful_indices.emplace_back(index_helper, condition); + if (index_helper->isMergeable()) + { + if (!merged_indices.contains(index_helper->getGranularity())) + { + merged_indices.emplace( + index_helper->getGranularity(), + std::make_shared( + std::make_shared(query_info, context, index_helper->getGranularity()))); + merged_indices.at(index_helper->getGranularity())->condition->addConstraints(metadata_snapshot->getConstraints()); + } + merged_indices.at(index_helper->getGranularity())->addIndex(index_helper); + } + else + { + auto condition = index_helper->createIndexCondition(query_info, context); + if (!condition->alwaysUnknownOrTrue()) + useful_indices.emplace_back(index_helper, condition); + } } if (settings.force_data_skipping_indices.changed) @@ -718,6 +756,29 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( index_and_condition.parts_dropped.fetch_add(1, std::memory_order_relaxed); } + for (auto & [granularity, indices_and_condition] : merged_indices) + { + if (ranges.ranges.empty()) + break; + + indices_and_condition->total_parts.fetch_add(1, std::memory_order_relaxed); + + size_t total_granules = 0; + size_t granules_dropped = 0; + ranges.ranges = filterMarksUsingMergedIndex( + indices_and_condition->indices, indices_and_condition->condition, + part, ranges.ranges, + settings, reader_settings, + total_granules, granules_dropped, + log); + + indices_and_condition->total_granules.fetch_add(total_granules, std::memory_order_relaxed); + indices_and_condition->granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); + + if (ranges.ranges.empty()) + indices_and_condition->parts_dropped.fetch_add(1, std::memory_order_relaxed); + } + if (!ranges.ranges.empty()) { if (limits.max_rows || leaf_limits.max_rows) @@ -811,6 +872,23 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( .num_granules_after = index_and_condition.total_granules - index_and_condition.granules_dropped}); } + for (const auto & [granularity, index_and_condition] : merged_indices) + { + const auto & index_name = "Merged"; + LOG_DEBUG(log, "Index {} has dropped {}/{} granules.", + backQuote(index_name), + index_and_condition->granules_dropped, index_and_condition->total_granules); + + std::string description = "MERGED GRANULARITY " + std::to_string(granularity); + + index_stats->emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::Skip, + .name = index_name, + .description = std::move(description), + .num_parts_after = index_and_condition->total_parts - index_and_condition->parts_dropped, + .num_granules_after = index_and_condition->total_granules - index_and_condition->granules_dropped}); + } + LOG_DEBUG(log, "Selected {}/{} parts by partition key, {} parts by primary key, {}/{} marks by primary key, {} marks to read from {} ranges", parts.size(), total_parts, parts_with_ranges.size(), sum_marks_pk.load(std::memory_order_relaxed), @@ -1865,6 +1943,96 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingIndex( return res; } +MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingMergedIndex( + MergeTreeIndices indices, + MergeTreeIndexMergedConditionPtr condition, + MergeTreeData::DataPartPtr part, + const MarkRanges & ranges, + const Settings & settings, + const MergeTreeReaderSettings & reader_settings, + size_t & total_granules, + size_t & granules_dropped, + Poco::Logger * log) +{ + for (const auto & index_helper : indices) + { + if (!part->volume->getDisk()->exists(part->getFullRelativePath() + index_helper->getFileName() + ".idx")) + { + LOG_DEBUG(log, "File for index {} does not exist. Skipping it.", backQuote(index_helper->index.name)); + return ranges; + } + } + + auto index_granularity = indices.front()->index.granularity; + + const size_t min_marks_for_seek = roundRowsOrBytesToMarks( + settings.merge_tree_min_rows_for_seek, + settings.merge_tree_min_bytes_for_seek, + part->index_granularity_info.fixed_index_granularity, + part->index_granularity_info.index_granularity_bytes); + + size_t marks_count = part->getMarksCount(); + size_t final_mark = part->index_granularity.hasFinalMark(); + size_t index_marks_count = (marks_count - final_mark + index_granularity - 1) / index_granularity; + + std::vector readers; + for (const auto & index_helper : indices) + { + readers.emplace_back( + index_helper, + part, + index_marks_count, + ranges, + reader_settings); + } + + MarkRanges res; + + /// Some granules can cover two or more ranges, + /// this variable is stored to avoid reading the same granule twice. + MergeTreeIndexGranules granules(indices.size(), nullptr); + bool granules_filled = false; + size_t last_index_mark = 0; + for (const auto & range : ranges) + { + MarkRange index_range( + range.begin / index_granularity, + (range.end + index_granularity - 1) / index_granularity); + + if (last_index_mark != index_range.begin || !granules_filled) + for (auto & reader : readers) + reader.seek(index_range.begin); + + total_granules += index_range.end - index_range.begin; + + for (size_t index_mark = index_range.begin; index_mark < index_range.end; ++index_mark) + { + if (index_mark != index_range.begin || !granules_filled || last_index_mark != index_range.begin) + for (size_t i = 0; i < readers.size(); ++i) + granules[i] = readers[i].read(); + + MarkRange data_range( + std::max(range.begin, index_mark * index_granularity), + std::min(range.end, (index_mark + 1) * index_granularity)); + + if (!condition->mayBeTrueOnGranule(granules)) + { + ++granules_dropped; + continue; + } + + if (res.empty() || res.back().end - data_range.begin > min_marks_for_seek) + res.push_back(data_range); + else + res.back().end = data_range.end; + } + + last_index_mark = index_range.end - 1; + } + + return res; +} + void MergeTreeDataSelectExecutor::selectPartsToRead( MergeTreeData::DataPartsVector & parts, const std::unordered_set & part_values, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index d7193fbfbfa..8ad885e4b29 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -6,6 +6,7 @@ #include #include #include +#include namespace DB @@ -125,6 +126,17 @@ private: size_t & granules_dropped, Poco::Logger * log); + static MarkRanges filterMarksUsingMergedIndex( + MergeTreeIndices index_helper, + MergeTreeIndexMergedConditionPtr condition, + MergeTreeData::DataPartPtr part, + const MarkRanges & ranges, + const Settings & settings, + const MergeTreeReaderSettings & reader_settings, + size_t & total_granules, + size_t & granules_dropped, + Poco::Logger * log); + struct PartFilterCounters { size_t num_initial_selected_parts = 0; diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp index a34af89f063..263d9057159 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp @@ -70,66 +70,6 @@ void MergeTreeIndexAggregatorHypothesis::update(const Block & block, size_t * po *pos += rows_read; } -MergeTreeIndexConditionHypothesis::MergeTreeIndexConditionHypothesis( - const String & index_name_, - const String & column_name_, - const SelectQueryInfo & query_, - ContextPtr) - : index_name(index_name_) - , column_name(column_name_) -{ - const auto & select = query_.query->as(); - - if (select.where() && select.prewhere()) - expression_ast = makeASTFunction( - "and", - select.where()->clone(), - select.prewhere()->clone()); - else if (select.where()) - expression_ast = select.where()->clone(); - else if (select.prewhere()) - expression_ast = select.prewhere()->clone(); -} - -std::pair MergeTreeIndexConditionHypothesis::mayBeTrue(const ASTPtr & ast, const bool value) const -{ - if (ast->getColumnName() == column_name) - return {value, !value}; - - auto * func = ast->as(); - if (!func) - return {true, true}; - auto & args = func->arguments->children; - if (func->name == "not") - { - const auto res = mayBeTrue(args[0], value); - return {res.second, res.first}; - } - /*else if (func->name == "or") - { - - } - else if (func->name == "and") - { - - }*/ - else - { - return {true, true}; - } -} - -bool MergeTreeIndexConditionHypothesis::mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const -{ - if (idx_granule->empty()) - return true; - auto granule = std::dynamic_pointer_cast(idx_granule); - if (!granule) - throw Exception( - "Set index condition got a granule with the wrong type.", ErrorCodes::LOGICAL_ERROR); - return mayBeTrue(expression_ast, granule->met).first; -} - MergeTreeIndexGranulePtr MergeTreeIndexHypothesis::createIndexGranule() const { return std::make_shared(index.name); @@ -141,9 +81,9 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexHypothesis::createIndexAggregator() co } MergeTreeIndexConditionPtr MergeTreeIndexHypothesis::createIndexCondition( - const SelectQueryInfo & query, ContextPtr context) const + const SelectQueryInfo &, ContextPtr) const { - return std::make_shared(index.name, index.sample_block.getNames().front(), query, context); + return nullptr; } bool MergeTreeIndexHypothesis::mayBenefitFromIndexForIn(const ASTPtr &) const @@ -156,8 +96,10 @@ MergeTreeIndexPtr hypothesisIndexCreator(const IndexDescription & index) return std::make_shared(index); } -void hypothesisIndexValidator(const IndexDescription &, bool /*attach*/) +void hypothesisIndexValidator(const IndexDescription & index, bool /*attach*/) { + if (index.expression_list_ast->children.size() != 1) + throw Exception("Hypothesis index needs exactly one expression", ErrorCodes::LOGICAL_ERROR); } diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h index ae11b710aa0..1989cab6a8e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h @@ -2,12 +2,7 @@ #include #include - -#include - #include -#include - namespace DB { @@ -30,7 +25,7 @@ struct MergeTreeIndexGranuleHypothesis : public IMergeTreeIndexGranule ~MergeTreeIndexGranuleHypothesis() override = default; - String index_name; + const String & index_name; bool is_empty = true; bool met = true; }; @@ -50,39 +45,13 @@ struct MergeTreeIndexAggregatorHypothesis : IMergeTreeIndexAggregator void update(const Block & block, size_t * pos, size_t limit) override; private: - String index_name; + const String & index_name; String column_name; bool met = true; bool is_empty = true; }; - -class MergeTreeIndexConditionHypothesis : public IMergeTreeIndexCondition -{ -public: - MergeTreeIndexConditionHypothesis( - const String & index_name_, - const String & column_name_, - const SelectQueryInfo & query, - ContextPtr context); - - bool alwaysUnknownOrTrue() const override { return false; } - - bool mayBeTrueOnGranule(MergeTreeIndexGranulePtr idx_granule) const override; - - ~MergeTreeIndexConditionHypothesis() override = default; - -private: - std::pair mayBeTrue(const ASTPtr & ast, const bool value) const; - - String index_name; - - String column_name; - ASTPtr expression_ast; -}; - - class MergeTreeIndexHypothesis : public IMergeTreeIndex { public: @@ -93,6 +62,8 @@ public: ~MergeTreeIndexHypothesis() override = default; + bool isMergeable() const override { return true; } + MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexAggregatorPtr createIndexAggregator() const override; diff --git a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp new file mode 100644 index 00000000000..fd4640000aa --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp @@ -0,0 +1,186 @@ +#include + +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; + extern const int INCORRECT_QUERY; +} + +MergeTreeIndexMergedCondition::MergeTreeIndexMergedCondition( + const SelectQueryInfo & query_, + ContextPtr /*context_*/, + const size_t granularity_) + : granularity(granularity_) +{ + const auto & select = query_.query->as(); + + if (select.where() && select.prewhere()) + expression_ast = makeASTFunction( + "and", + select.where()->clone(), + select.prewhere()->clone()); + else if (select.where()) + expression_ast = select.where()->clone(); + else if (select.prewhere()) + expression_ast = select.prewhere()->clone(); + + expression_cnf = std::make_unique(TreeCNFConverter::toCNF(expression_ast)); +} + +void MergeTreeIndexMergedCondition::addIndex(const MergeTreeIndexPtr & index) +{ + if (!index->isMergeable() || index->getGranularity() != granularity) + throw Exception("Index can not be merged", + ErrorCodes::LOGICAL_ERROR); + + const auto hypothesis_index = std::dynamic_pointer_cast(index); + if (!hypothesis_index) + throw Exception( + "Only hypothesis index is supported here.", ErrorCodes::LOGICAL_ERROR); + + static const std::set relations = { + "equals", "less", "lessOrEquals", "greaterOrEquals", "greater"}; + + // TODO: move to index hypothesis + std::vector compare_hypotheses_data; + std::vector hypotheses_data; + const auto cnf = TreeCNFConverter::toCNF(hypothesis_index->index.expression_list_ast->children.front()).pullNotOutFunctions(); + for (const auto & group : cnf.getStatements()) { + hypotheses_data.push_back(group); + if (group.size() == 1) + { + CNFQuery::AtomicFormula atom = *group.begin(); + pushNotIn(atom); + if (atom.negative) + throw Exception("negative atom", ErrorCodes::LOGICAL_ERROR); + + auto * func = atom.ast->as(); + if (func && relations.count(func->name)) + compare_hypotheses_data.push_back(atom.ast); + } + } + index_to_compare_atomic_hypotheses.push_back(compare_hypotheses_data); + index_to_atomic_hypotheses.push_back(hypotheses_data); +} + +void MergeTreeIndexMergedCondition::addConstraints(const ConstraintsDescription & constraints_description) +{ + auto atomic_constraints_data = constraints_description.getAtomicConstraintData(); + for (auto & atom : atomic_constraints_data) + { + pushNotIn(atom); + atomic_constraints.push_back(atom.ast); + } +} + +namespace +{ + +ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula & atom) +{ + static const std::map inverse_relations = { + {"equals", "notEquals"}, + {"less", "greaterOrEquals"}, + {"lessOrEquals", "greater"}, + {"notEquals", "equals"}, + {"greaterOrEquals", "less"}, + {"greater", "lessOrEquals"}, + }; + + static const std::map relation_to_compare = { + {"equals", ComparisonGraph::CompareResult::EQUAL}, + {"less", ComparisonGraph::CompareResult::LESS}, + {"lessOrEquals", ComparisonGraph::CompareResult::LESS_OR_EQUAL}, + {"notEquals", ComparisonGraph::CompareResult::UNKNOWN}, + {"greaterOrEquals", ComparisonGraph::CompareResult::GREATER_OR_EQUAL}, + {"greater", ComparisonGraph::CompareResult::GREATER}, + }; + + + const auto * func = atom.ast->as(); + if (func && inverse_relations.count(func->name)) + { + std::string function_name = func->name; + if (atom.negative) + function_name = inverse_relations.at(func->name); + return relation_to_compare.at(function_name); + } + return ComparisonGraph::CompareResult::UNKNOWN; +} + +} + +bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const +{ + std::vector values; + for (const auto & index_granule : granules) + { + const auto granule = std::dynamic_pointer_cast(index_granule); + if (!granule) + throw Exception("Only hypothesis index is supported here.", ErrorCodes::LOGICAL_ERROR); + values.push_back(granule->met); + } + const auto & graph = getGraph(values); + + bool always_false = false; + expression_cnf->iterateGroups( + [&](const CNFQuery::OrGroup & or_group) + { + if (always_false) + return; + + for (auto atom : or_group) + { + pushNotIn(atom); + Poco::Logger::get("KEK").information(atom.ast->dumpTree()); + const auto * func = atom.ast->as(); + if (func && func->arguments->children.size() == 2) + { + const auto expected = getExpectedCompare(atom); + if (graph.isPossibleCompare( + expected, + func->arguments->children[0], + func->arguments->children[1])) + { + return; + } + } + } + always_false = true; + }); + return !always_false; +} + +std::unique_ptr MergeTreeIndexMergedCondition::buildGraph(const std::vector & values) const +{ + Poco::Logger::get("MergeTreeIndexMergedCondition").information("New graph"); + std::vector active_atomic_formulas(atomic_constraints); + for (size_t i = 0; i < values.size(); ++i) + { + if (values[i]) + active_atomic_formulas.insert( + std::end(active_atomic_formulas), + std::begin(index_to_compare_atomic_hypotheses[i]), + std::end(index_to_compare_atomic_hypotheses[i])); + } + return std::make_unique(active_atomic_formulas); +} + +const ComparisonGraph & MergeTreeIndexMergedCondition::getGraph(const std::vector & values) const +{ + if (!graphCache.contains(values)) + graphCache[values] = buildGraph(values); + return *graphCache.at(values); +} + +} diff --git a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h new file mode 100644 index 00000000000..3048a9d6bdc --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h @@ -0,0 +1,47 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +/* + * IndexCondition checking several indexes at the same time. + * Works only for hypotheses. (will also support minmax soon). + */ +class MergeTreeIndexMergedCondition +{ +public: + MergeTreeIndexMergedCondition( + const SelectQueryInfo & query, + ContextPtr context, + const size_t granularity); + + void addIndex(const MergeTreeIndexPtr & index); + void addConstraints(const ConstraintsDescription & constraints_description); + + bool alwaysUnknownOrTrue() const { return false; } // TODO: replace < -> <=, > -> >= and assume all hypotheses are true + check path exists + bool mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const; + + //TODO: add constraints +private: + std::unique_ptr buildGraph(const std::vector & values) const; + const ComparisonGraph & getGraph(const std::vector & values) const; + + const size_t granularity; + ASTPtr expression_ast; + std::unique_ptr expression_cnf; + + mutable std::unordered_map, std::unique_ptr> graphCache; + + std::vector> index_to_compare_atomic_hypotheses; + std::vector> index_to_atomic_hypotheses; + std::vector atomic_constraints; +}; + +using MergeTreeIndexMergedConditionPtr = std::shared_ptr; +using MergeTreeIndexMergedConditions = std::vector; + +} diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index 7e5cb4156cb..319be92960d 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -62,6 +62,7 @@ public: }; using MergeTreeIndexConditionPtr = std::shared_ptr; +using MergeTreeIndexConditions = std::vector; struct IMergeTreeIndex @@ -75,6 +76,9 @@ struct IMergeTreeIndex /// gets filename without extension String getFileName() const { return INDEX_FILE_PREFIX + index.name; } + size_t getGranularity() const { return index.granularity; } + + virtual bool isMergeable() const { return false; } /// Checks whether the column is in data skipping index. virtual bool mayBenefitFromIndexForIn(const ASTPtr & node) const = 0; From cdb0d86e2bb1bde43675a0433a69cbb0e55ae331 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Mon, 3 May 2021 22:08:26 +0300 Subject: [PATCH 037/200] improve hypothesis --- src/Interpreters/ComparisonGraph.cpp | 9 +- src/Interpreters/ComparisonGraph.h | 1 + .../MergeTreeIndexMergedCondition.cpp | 96 +++++++++++++++---- .../MergeTree/MergeTreeIndexMergedCondition.h | 2 +- .../01624_soft_constraints.reference | 9 +- .../0_stateless/01624_soft_constraints.sh | 49 ++++++++++ .../0_stateless/01624_soft_constraints.sql | 17 ---- 7 files changed, 146 insertions(+), 37 deletions(-) create mode 100755 tests/queries/0_stateless/01624_soft_constraints.sh delete mode 100644 tests/queries/0_stateless/01624_soft_constraints.sql diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 8a4d9f0e0be..37f9b99a15e 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -191,11 +191,11 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con if (start == finish) return CompareResult::EQUAL; - auto [has_path, is_strict] = findPath(start, finish); + const auto [has_path, is_strict] = findPath(start, finish); if (has_path) return is_strict ? CompareResult::GREATER : CompareResult::GREATER_OR_EQUAL; - auto [has_path_reverse, is_strict_reverse] = findPath(finish, start); + const auto [has_path_reverse, is_strict_reverse] = findPath(finish, start); if (has_path_reverse) return is_strict_reverse ? CompareResult::LESS : CompareResult::LESS_OR_EQUAL; @@ -279,6 +279,11 @@ std::optional ComparisonGraph::getComponentId(const ASTPtr & ast) c } } +bool ComparisonGraph::hasPath(const size_t left, const size_t right) const +{ + return findPath(left, right).first || findPath(right, left).first; +} + std::vector ComparisonGraph::getComponent(const std::size_t id) const { return graph.vertexes[id].asts; diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 9fee991c399..5450a69bd4b 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -41,6 +41,7 @@ public: std::optional getComponentId(const ASTPtr & ast) const; std::vector getComponent(const std::size_t id) const; + bool hasPath(const size_t left, const size_t right) const; /// Find constants lessOrEqual and greaterOrEqual. /// For int and double linear programming can be applied here. diff --git a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp index fd4640000aa..5f9c2807469 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp @@ -55,10 +55,11 @@ void MergeTreeIndexMergedCondition::addIndex(const MergeTreeIndexPtr & index) std::vector compare_hypotheses_data; std::vector hypotheses_data; const auto cnf = TreeCNFConverter::toCNF(hypothesis_index->index.expression_list_ast->children.front()).pullNotOutFunctions(); - for (const auto & group : cnf.getStatements()) { - hypotheses_data.push_back(group); + for (const auto & group : cnf.getStatements()) + { if (group.size() == 1) { + hypotheses_data.push_back(group); CNFQuery::AtomicFormula atom = *group.begin(); pushNotIn(atom); if (atom.negative) @@ -120,6 +121,53 @@ ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula } +/// Replaces < -> <=, > -> >= and assumes that all hypotheses are true then checks if path exists +bool MergeTreeIndexMergedCondition::alwaysUnknownOrTrue() const { + std::vector active_atomic_formulas(atomic_constraints); + for (size_t i = 0; i < index_to_compare_atomic_hypotheses.size(); ++i) + { + active_atomic_formulas.insert( + std::end(active_atomic_formulas), + std::begin(index_to_compare_atomic_hypotheses[i]), + std::end(index_to_compare_atomic_hypotheses[i])); + } + + /// transform active formulas + for (auto & formula : active_atomic_formulas) + { + formula = formula->clone(); /// do all operations with copy + auto * func = formula->as(); + if (func && func->name == "less") + func->name = "lessOrEquals"; + if (func && func->name == "greater") + func->name = "greaterOrEquals"; + } + + const auto weak_graph = std::make_unique(active_atomic_formulas); + + bool useless = true; + expression_cnf->iterateGroups( + [&](const CNFQuery::OrGroup & or_group) + { + for (auto atom : or_group) + { + pushNotIn(atom); + const auto * func = atom.ast->as(); + if (func && func->arguments->children.size() == 2) + { + const auto left = weak_graph->getComponentId(func->arguments->children[0]); + const auto right = weak_graph->getComponentId(func->arguments->children[1]); + if (left && right && weak_graph->hasPath(left.value(), right.value())) + { + useless = false; + return; + } + } + } + }); + return useless; +} + bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const { std::vector values; @@ -141,20 +189,36 @@ bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranu for (auto atom : or_group) { - pushNotIn(atom); - Poco::Logger::get("KEK").information(atom.ast->dumpTree()); - const auto * func = atom.ast->as(); - if (func && func->arguments->children.size() == 2) - { - const auto expected = getExpectedCompare(atom); - if (graph.isPossibleCompare( - expected, - func->arguments->children[0], - func->arguments->children[1])) - { - return; - } - } + pushNotIn(atom); + Poco::Logger::get("KEK").information(atom.ast->dumpTree()); + const auto * func = atom.ast->as(); + if (func && func->arguments->children.size() == 2) + { + const auto expected = getExpectedCompare(atom); + if (graph.isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1])) + { + /// If graph failed use matching. + /// We don't need to check constraints. + return; + } + } + } + for (auto atom : or_group) + { + pushNotIn(atom); + for (size_t i = 0; i < values.size(); ++i) + if (values[i]) + for (const auto & hypothesis_or_group : index_to_atomic_hypotheses[i]) + { + if (hypothesis_or_group.size() == 1) + { + const auto & hypothesis_atom = *std::begin(hypothesis_or_group); + if (atom.ast->getTreeHash() == hypothesis_atom.ast->getTreeHash()) + { + return; + } + } + } } always_false = true; }); diff --git a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h index 3048a9d6bdc..204e6f01ea2 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h +++ b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h @@ -22,7 +22,7 @@ public: void addIndex(const MergeTreeIndexPtr & index); void addConstraints(const ConstraintsDescription & constraints_description); - bool alwaysUnknownOrTrue() const { return false; } // TODO: replace < -> <=, > -> >= and assume all hypotheses are true + check path exists + bool alwaysUnknownOrTrue() const; bool mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const; //TODO: add constraints diff --git a/tests/queries/0_stateless/01624_soft_constraints.reference b/tests/queries/0_stateless/01624_soft_constraints.reference index 0cfbf08886f..f0a866816d6 100644 --- a/tests/queries/0_stateless/01624_soft_constraints.reference +++ b/tests/queries/0_stateless/01624_soft_constraints.reference @@ -1 +1,8 @@ -2 + "rows_read": 4, + "rows_read": 2, + "rows_read": 4, + "rows_read": 2, + "rows_read": 2, + "rows_read": 2, + "rows_read": 4, + "rows_read": 2, diff --git a/tests/queries/0_stateless/01624_soft_constraints.sh b/tests/queries/0_stateless/01624_soft_constraints.sh new file mode 100755 index 00000000000..30f77446b49 --- /dev/null +++ b/tests/queries/0_stateless/01624_soft_constraints.sh @@ -0,0 +1,49 @@ +#!/usr/bin/env bash + +set -e + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +SETTINGS="SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET optimize_move_to_prewhere = 1;" + +$CLICKHOUSE_CLIENT -n --query=" +DROP DATABASE IF EXISTS constraint_test; +DROP TABLE IF EXISTS constraint_test.test; +CREATE DATABASE constraint_test; +CREATE TABLE constraint_test.test ( + i UInt64, + a UInt64, + b UInt64, + c Float64, + INDEX t (a < b) TYPE hypothesis GRANULARITY 1, + INDEX t2 (b <= c) TYPE hypothesis GRANULARITY 1 +) ENGINE = MergeTree() ORDER BY i SETTINGS index_granularity=1; +" + +$CLICKHOUSE_CLIENT --query="INSERT INTO constraint_test.test VALUES +(0, 1, 2, 2), +(1, 2, 1, 2), +(2, 2, 2, 1), +(3, 1, 2, 3)" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE b > a FORMAT JSON" | grep "rows_read" # 4 + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE b <= a FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE b >= a FORMAT JSON" | grep "rows_read" # 4 + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE b = a FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE c < a FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE c = a FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE c > a FORMAT JSON" | grep "rows_read" # 4 + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE c < a FORMAT JSON" | grep "rows_read" + +$CLICKHOUSE_CLIENT -n --query=" +DROP TABLE constraint_test.test; +DROP DATABASE constraint_test;" diff --git a/tests/queries/0_stateless/01624_soft_constraints.sql b/tests/queries/0_stateless/01624_soft_constraints.sql deleted file mode 100644 index 53a2d4ac9f6..00000000000 --- a/tests/queries/0_stateless/01624_soft_constraints.sql +++ /dev/null @@ -1,17 +0,0 @@ -SET convert_query_to_cnf = 1; -SET optimize_using_constraints = 1; -SET optimize_move_to_prewhere = 1; - -DROP DATABASE IF EXISTS constraint_test; -DROP TABLE IF EXISTS constraint_test.test; - -CREATE DATABASE constraint_test; - -CREATE TABLE constraint_test.test (i UInt64, a UInt64, b UInt64, INDEX t (a = b) TYPE hypothesis GRANULARITY 1) ENGINE = MergeTree() ORDER BY i SETTINGS index_granularity=1; -INSERT INTO constraint_test.test VALUES (1, 1, 1), (2, 1, 2), (3, 2, 1), (4, 2, 2); - -SELECT count() FROM constraint_test.test WHERE a = b; - -DROP TABLE constraint_test.test; - -DROP DATABASE constraint_test; \ No newline at end of file From 62d33f93c0110985371532f0d42f43e34cb2abbe Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Tue, 4 May 2021 13:47:23 +0300 Subject: [PATCH 038/200] != support --- .../AddIndexConstraintsOptimizer.cpp | 4 +- src/Interpreters/ComparisonGraph.cpp | 110 +++++++++--------- src/Interpreters/ComparisonGraph.h | 5 + .../WhereConstraintsOptimizer.cpp | 12 +- .../MergeTreeIndexMergedCondition.cpp | 17 ++- ..._constraints_simple_optimization.reference | 1 + .../01622_constraints_simple_optimization.sql | 1 + .../01623_constraints_column_swap.reference | 20 ++-- .../01623_constraints_column_swap.sql | 42 +++---- .../01624_soft_constraints.reference | 8 ++ .../0_stateless/01624_soft_constraints.sh | 84 ++++++++++--- .../01625_constraints_index_append.reference | 8 +- .../01625_constraints_index_append.sql | 22 ++-- 13 files changed, 214 insertions(+), 120 deletions(-) diff --git a/src/Interpreters/AddIndexConstraintsOptimizer.cpp b/src/Interpreters/AddIndexConstraintsOptimizer.cpp index 3c3c1043c0a..63a24eba094 100644 --- a/src/Interpreters/AddIndexConstraintsOptimizer.cpp +++ b/src/Interpreters/AddIndexConstraintsOptimizer.cpp @@ -86,7 +86,7 @@ namespace bool canBeSequence(const ComparisonGraph::CompareResult left, const ComparisonGraph::CompareResult right) { using CR = ComparisonGraph::CompareResult; - if (left == CR::UNKNOWN || right == CR::UNKNOWN) + if (left == CR::UNKNOWN || right == CR::UNKNOWN || left == CR::NOT_EQUAL || right == CR::NOT_EQUAL) return false; if ((left == CR::GREATER || left == CR::GREATER_OR_EQUAL) && (right == CR::LESS || right == CR::LESS_OR_EQUAL)) return false; @@ -110,7 +110,7 @@ namespace return left; if (right == CR::EQUAL) return right; - return left; + return CR::UNKNOWN; } /// Create OR-group for index_hint diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 37f9b99a15e..25a1a346886 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -76,26 +76,24 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) const auto * func = atom->as(); if (func) { - if (const auto it = relation_to_enum.find(func->name); it != std::end(relation_to_enum) && func->arguments->children.size() == 2) - { - const size_t index_left = get_index(func->arguments->children[0], g); - const size_t index_right = get_index(func->arguments->children[1], g); + const size_t index_left = get_index(func->arguments->children[0], g); + const size_t index_right = get_index(func->arguments->children[1], g); - if (index_left != bad_term && index_right != bad_term) + if (index_left != bad_term && index_right != bad_term) + { + if (const auto it = relation_to_enum.find(func->name); + it != std::end(relation_to_enum) && func->arguments->children.size() == 2) { - //Poco::Logger::get("Edges").information("GOOD: " + atom->dumpTree()); - Poco::Logger::get("Edges").information("left=" + std::to_string(index_left) + " right=" + std::to_string(index_right)); - Poco::Logger::get("Edges").information("sz=" + std::to_string(g.edges.size())); g.edges[index_right].push_back(Edge{it->second, index_left}); if (func->name == "equals") { - //Poco::Logger::get("Edges").information("right=" + std::to_string(index_left) + " left=" + std::to_string(index_right)); g.edges[index_left].push_back(Edge{it->second, index_right}); } } - else + else if (func->name == "notEquals") { - //Poco::Logger::get("Edges").information("BAD: " + atom->dumpTree()); + not_equal.emplace(index_left, index_right); + not_equal.emplace(index_right, index_left); } } } @@ -117,10 +115,10 @@ std::pair ComparisonGraph::findPath(const size_t start, const size_t return {false, false}; else { - Poco::Logger::get("dists found").information(std::to_string(start) + " " + std::to_string(finish) + " : " + std::to_string(static_cast(it->second))); + //Poco::Logger::get("dists found").information(std::to_string(start) + " " + std::to_string(finish) + " : " + std::to_string(static_cast(it->second))); //Poco::Logger::get("dists found").information(graph.vertexes[start].asts.back()->dumpTree()); //Poco::Logger::get("dists found").information(graph.vertexes[finish].asts.back()->dumpTree()); - return {true, it->second == Path::LESS}; + return {true, it->second == Path::LESS || not_equal.contains({start, finish})}; } } @@ -129,11 +127,6 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con size_t start = 0; size_t finish = 0; { - for (const auto & [k, k2] : dists) - { - Poco::Logger::get("dists").information(std::to_string(k.first) + "-" + std::to_string(k.second) + " : " + std::to_string(static_cast(k2))); - } - /// TODO: check full ast const auto it_left = graph.ast_hash_to_component.find(left->getTreeHash()); const auto it_right = graph.ast_hash_to_component.find(right->getTreeHash()); @@ -142,16 +135,15 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con Poco::Logger::get("Graph").information("not found"); Poco::Logger::get("Graph").information(std::to_string(left->getTreeHash().second)); Poco::Logger::get("Graph").information(std::to_string(right->getTreeHash().second)); - for (const auto & [hash, id] : graph.ast_hash_to_component) + /*for (const auto & [hash, id] : graph.ast_hash_to_component) { Poco::Logger::get("Graph MAP").information(std::to_string(hash.second) + " " + std::to_string(id)); - } + }*/ { const auto left_bound = getConstLowerBound(left); const auto right_bound = getConstUpperBound(right); if (left_bound && right_bound) { - //Poco::Logger::get("&&&&&&&").information(left_bound->first.dump() + " " + std::to_string(left_bound->second) + " | " + right_bound->first.dump() + " " + std::to_string(right_bound->second)); if (left_bound->first < right_bound->first) return CompareResult::UNKNOWN; else if (left_bound->first > right_bound->first) @@ -165,7 +157,6 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con { const auto left_bound = getConstUpperBound(left); const auto right_bound = getConstLowerBound(right); - //Poco::Logger::get("!!!!!!").information(left_bound->first.dump() + " " + std::to_string(left_bound->second) + " | " + right_bound->first.dump() + " " + std::to_string(right_bound->second)); if (left_bound && right_bound) { if (left_bound->first > right_bound->first) @@ -199,6 +190,9 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con if (has_path_reverse) return is_strict_reverse ? CompareResult::LESS : CompareResult::LESS_OR_EQUAL; + if (not_equal.contains({start, finish})) + return CompareResult::NOT_EQUAL; + return CompareResult::UNKNOWN; } @@ -219,12 +213,20 @@ bool ComparisonGraph::isPossibleCompare(const CompareResult expected, const ASTP {CompareResult::EQUAL, CompareResult::GREATER_OR_EQUAL}, {CompareResult::LESS_OR_EQUAL, CompareResult::LESS}, {CompareResult::LESS_OR_EQUAL, CompareResult::EQUAL}, + {CompareResult::LESS_OR_EQUAL, CompareResult::NOT_EQUAL}, {CompareResult::GREATER_OR_EQUAL, CompareResult::GREATER}, {CompareResult::GREATER_OR_EQUAL, CompareResult::EQUAL}, + {CompareResult::GREATER_OR_EQUAL, CompareResult::NOT_EQUAL}, {CompareResult::LESS, CompareResult::LESS}, {CompareResult::LESS, CompareResult::LESS_OR_EQUAL}, + {CompareResult::LESS, CompareResult::NOT_EQUAL}, {CompareResult::GREATER, CompareResult::GREATER}, {CompareResult::GREATER, CompareResult::GREATER_OR_EQUAL}, + {CompareResult::GREATER, CompareResult::NOT_EQUAL}, + {CompareResult::NOT_EQUAL, CompareResult::LESS}, + {CompareResult::NOT_EQUAL, CompareResult::GREATER}, + {CompareResult::NOT_EQUAL, CompareResult::LESS_OR_EQUAL}, + {CompareResult::NOT_EQUAL, CompareResult::GREATER_OR_EQUAL}, }; return possible_pairs.contains({expected, result}); @@ -244,6 +246,8 @@ bool ComparisonGraph::isAlwaysCompare(const CompareResult expected, const ASTPtr {CompareResult::LESS_OR_EQUAL, CompareResult::EQUAL}, {CompareResult::GREATER_OR_EQUAL, CompareResult::GREATER}, {CompareResult::GREATER_OR_EQUAL, CompareResult::EQUAL}, + {CompareResult::NOT_EQUAL, CompareResult::GREATER}, + {CompareResult::NOT_EQUAL, CompareResult::LESS}, }; return possible_pairs.contains({expected, result}); @@ -309,6 +313,35 @@ void ComparisonGraph::EqualComponent::buildConstants() { } } +ComparisonGraph::CompareResult ComparisonGraph::getCompareResult(const std::string & name) +{ + static const std::unordered_map relation_to_compare = { + {"equals", CompareResult::EQUAL}, + {"notEquals", CompareResult::NOT_EQUAL}, + {"less", CompareResult::LESS}, + {"lessOrEquals", CompareResult::LESS_OR_EQUAL}, + {"greaterOrEquals", CompareResult::GREATER_OR_EQUAL}, + {"greater", CompareResult::GREATER}, + }; + + const auto it = relation_to_compare.find(name); + return it == std::end(relation_to_compare) ? CompareResult::UNKNOWN : it->second; +} + +ComparisonGraph::CompareResult ComparisonGraph::inverseCompareResult(const CompareResult result) +{ + static const std::unordered_map inverse_relations = { + {CompareResult::NOT_EQUAL, CompareResult::EQUAL}, + {CompareResult::EQUAL, CompareResult::NOT_EQUAL}, + {CompareResult::GREATER_OR_EQUAL, CompareResult::LESS}, + {CompareResult::GREATER, CompareResult::LESS_OR_EQUAL}, + {CompareResult::LESS, CompareResult::GREATER_OR_EQUAL}, + {CompareResult::LESS_OR_EQUAL, CompareResult::GREATER}, + {CompareResult::UNKNOWN, CompareResult::UNKNOWN}, + }; + return inverse_relations.at(result); +} + std::optional ComparisonGraph::getEqualConst(const ASTPtr & ast) const { const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash()); @@ -414,13 +447,6 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as /// Find strongly connected component const auto n = asts_graph.vertexes.size(); - /*for (size_t v = 0; v < n; ++v) { - //Poco::LogStream{"kek"}.information() << "VERTEX " << v << " " << asts_graph.vertexes[v].asts.back()->dumpTree() << std::endl; - for (const auto & edge : asts_graph.edges[v]) { - //Poco::LogStream{"kek"}.information() << "TO " << edge.to << " " << static_cast(edge.type) << std::endl; - } - }*/ - std::vector order; { std::vector visited(n, false); @@ -431,8 +457,6 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as } } - Poco::Logger::get("Graph").information("dfs1"); - const auto not_visited = std::numeric_limits::max(); std::vector components(n, not_visited); size_t component = 0; @@ -448,8 +472,6 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as } } - Poco::Logger::get("Graph").information("dfs2"); - Graph result; result.vertexes.resize(component); result.edges.resize(component); @@ -489,7 +511,6 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as { const auto * left = result.vertexes[v].getConstant()->as(); const auto * right = result.vertexes[u].getConstant()->as(); - //Poco::Logger::get("Graph").information(left->value.dump() + " " + right->value.dump()); /// Only less. Equal constant fields = equal literals so it was already considered above. if (left->value > right->value) @@ -500,17 +521,7 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as } } - Poco::Logger::get("Graph").information("finish"); - - - /* for (size_t v = 0; v < result.vertexes.size(); ++v) { - Poco::LogStream{"kekkek"}.information() << "VERTEX " << v << " " << result.vertexes[v].asts.back()->dumpTree() << std::endl; - for (const auto & edge : result.edges[v]) { - Poco::LogStream{"kekkek"}.information() << "TO " << edge.to << " " << static_cast(edge.type) << std::endl; - } - }*/ - - for (size_t v = 0; v < result.vertexes.size(); ++v) + /*for (size_t v = 0; v < result.vertexes.size(); ++v) { std::stringstream s; for (const auto & atom : result.vertexes[v].asts) @@ -524,7 +535,7 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as } Poco::Logger::get("Graph").information(s.str()); - } + }*/ return result; } @@ -541,9 +552,6 @@ std::map, ComparisonGraph::Path> ComparisonGraph::Buil for (const auto & edge : g.edges[v]) results[v][edge.to] = std::min(results[v][edge.to], static_cast(edge.type == Edge::LESS ? -1 : 0)); } - for (size_t v = 0; v < n; ++v) - for (size_t u = 0; u < n; ++u) - Poco::LogStream{Poco::Logger::get("Graph ---=------------")}.information() << v << " " << u << " " << static_cast(results[v][u]) << std::endl; for (size_t k = 0; k < n; ++k) for (size_t v = 0; v < n; ++v) @@ -554,11 +562,9 @@ std::map, ComparisonGraph::Path> ComparisonGraph::Buil std::map, Path> path; for (size_t v = 0; v < n; ++v) for (size_t u = 0; u < n; ++u) - { - Poco::LogStream{Poco::Logger::get("Graph results-------------")}.information() << v << " " << u << " " << static_cast(results[v][u]) << std::endl; if (results[v][u] != inf) path[std::make_pair(v, u)] = (results[v][u] == -1 ? Path::LESS : Path::LESS_OR_EQUAL); - } + return path; } diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 5450a69bd4b..101294d3e6b 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -25,9 +25,13 @@ public: EQUAL, GREATER_OR_EQUAL, GREATER, + NOT_EQUAL, UNKNOWN, }; + static CompareResult getCompareResult(const std::string & name); + static CompareResult inverseCompareResult(const CompareResult result); + CompareResult compare(const ASTPtr & left, const ASTPtr & right) const; /// It's possible that left right @@ -114,6 +118,7 @@ private: Graph graph; std::map, Path> dists; + std::set> not_equal; std::vector ast_const_lower_bound; std::vector ast_const_upper_bound; }; diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index d3034ae4316..1d53eab5768 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -87,7 +87,7 @@ bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const std: ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula & atom) { - static const std::map inverse_relations = { + /*static const std::map inverse_relations = { {"equals", "notEquals"}, {"less", "greaterOrEquals"}, {"lessOrEquals", "greater"}, @@ -116,6 +116,16 @@ ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula } return relation_to_compare.at(function_name); } + return ComparisonGraph::CompareResult::UNKNOWN;*/ + + const auto * func = atom.ast->as(); + if (func) + { + auto expected = ComparisonGraph::getCompareResult(func->name); + if (atom.negative) + expected = ComparisonGraph::inverseCompareResult(expected); + return expected; + } return ComparisonGraph::CompareResult::UNKNOWN; } diff --git a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp index 5f9c2807469..e32514c81a0 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp @@ -49,7 +49,7 @@ void MergeTreeIndexMergedCondition::addIndex(const MergeTreeIndexPtr & index) "Only hypothesis index is supported here.", ErrorCodes::LOGICAL_ERROR); static const std::set relations = { - "equals", "less", "lessOrEquals", "greaterOrEquals", "greater"}; + "equals", "notEquals", "less", "lessOrEquals", "greaterOrEquals", "greater"}; // TODO: move to index hypothesis std::vector compare_hypotheses_data; @@ -89,7 +89,7 @@ namespace ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula & atom) { - static const std::map inverse_relations = { + /*static const std::map inverse_relations = { {"equals", "notEquals"}, {"less", "greaterOrEquals"}, {"lessOrEquals", "greater"}, @@ -116,6 +116,15 @@ ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula function_name = inverse_relations.at(func->name); return relation_to_compare.at(function_name); } + return ComparisonGraph::CompareResult::UNKNOWN;*/ + const auto * func = atom.ast->as(); + if (func) + { + auto expected = ComparisonGraph::getCompareResult(func->name); + if (atom.negative) + expected = ComparisonGraph::inverseCompareResult(expected); + return expected; + } return ComparisonGraph::CompareResult::UNKNOWN; } @@ -203,7 +212,7 @@ bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranu } } } - for (auto atom : or_group) + /*for (auto atom : or_group) { pushNotIn(atom); for (size_t i = 0; i < values.size(); ++i) @@ -219,7 +228,7 @@ bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranu } } } - } + }*/ always_false = true; }); return !always_false; diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index fe3be5091ca..0e65f1e557d 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -20,6 +20,7 @@ 1 1 1 +1 0 0 1 diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index f1e85ee8685..e9370dc8492 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -58,6 +58,7 @@ SELECT count() FROM constraint_test.transitivity2 WHERE a >= d; ---> assumption SELECT count() FROM constraint_test.transitivity2 WHERE d < a; ---> assumption -> 1 SELECT count() FROM constraint_test.transitivity2 WHERE a < d; ---> assumption -> 0 SELECT count() FROM constraint_test.transitivity2 WHERE a = d; ---> assumption -> 0 +SELECT count() FROM constraint_test.transitivity2 WHERE a != d; ---> assumption -> 1 DROP TABLE constraint_test.transitivity2; diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index e334c6f1c4e..f210d403d68 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -1,46 +1,46 @@ SELECT b + 10, b + 3 -FROM constraint_test.test +FROM column_swap_test.test WHERE b = 1 SELECT b + 10, b + 3 -FROM constraint_test.test +FROM column_swap_test.test WHERE b = 0 SELECT b + 10, b + 3 -FROM constraint_test.test +FROM column_swap_test.test WHERE b = 0 SELECT b + 10, b + 3 -FROM constraint_test.test +FROM column_swap_test.test WHERE b = 1 SELECT b + 10 -FROM constraint_test.test +FROM column_swap_test.test WHERE b = 0 SELECT cityHash64(a) + 10, a -FROM constraint_test.test +FROM column_swap_test.test WHERE cityHash64(a) = 0 SELECT cityHash64(a) + 10, a -FROM constraint_test.test +FROM column_swap_test.test WHERE cityHash64(a) = 0 SELECT a, a -FROM constraint_test.test +FROM column_swap_test.test WHERE a = \'c\' SELECT a, a -FROM constraint_test.test +FROM column_swap_test.test WHERE a = \'c\' SELECT a -FROM constraint_test.test +FROM column_swap_test.test WHERE a = \'c\' diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.sql b/tests/queries/0_stateless/01623_constraints_column_swap.sql index eb3188f885a..3f4cce71894 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.sql +++ b/tests/queries/0_stateless/01623_constraints_column_swap.sql @@ -2,33 +2,33 @@ SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET optimize_move_to_prewhere = 1; -DROP DATABASE IF EXISTS constraint_test; -DROP TABLE IF EXISTS constraint_test.test; +DROP DATABASE IF EXISTS column_swap_test; +DROP TABLE IF EXISTS column_swap_test.test; -CREATE DATABASE constraint_test; +CREATE DATABASE column_swap_test; -CREATE TABLE constraint_test.test (i Int64, a String, b UInt64, CONSTRAINT c1 ASSUME b = cityHash64(a)) ENGINE = MergeTree() ORDER BY i; -INSERT INTO constraint_test.test VALUES (1, 'cat', 1), (2, 'dog', 2); -INSERT INTO constraint_test.test SELECT number AS i, format('test {} kek {}', toString(number), toString(number + 10)) AS a, 1 AS b FROM system.numbers LIMIT 1000000; +CREATE TABLE column_swap_test.test (i Int64, a String, b UInt64, CONSTRAINT c1 ASSUME b = cityHash64(a)) ENGINE = MergeTree() ORDER BY i; +INSERT INTO column_swap_test.test VALUES (1, 'cat', 1), (2, 'dog', 2); +INSERT INTO column_swap_test.test SELECT number AS i, format('test {} kek {}', toString(number), toString(number + 10)) AS a, 1 AS b FROM system.numbers LIMIT 1000000; -EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM constraint_test.test WHERE cityHash64(a) = 1; -EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM constraint_test.test WHERE cityHash64(a) = 0; -EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM constraint_test.test WHERE b = 0; -EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM constraint_test.test WHERE b = 1; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test.test WHERE cityHash64(a) = 1; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test.test WHERE cityHash64(a) = 0; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test.test WHERE b = 0; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test.test WHERE b = 1; -EXPLAIN SYNTAX SELECT cityHash64(a) + 10 FROM constraint_test.test WHERE cityHash64(a) = 0; -EXPLAIN SYNTAX SELECT cityHash64(a) + 10, a FROM constraint_test.test WHERE cityHash64(a) = 0; -EXPLAIN SYNTAX SELECT b + 10, a FROM constraint_test.test WHERE b = 0; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10 FROM column_swap_test.test WHERE cityHash64(a) = 0; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, a FROM column_swap_test.test WHERE cityHash64(a) = 0; +EXPLAIN SYNTAX SELECT b + 10, a FROM column_swap_test.test WHERE b = 0; -DROP TABLE constraint_test.test; +DROP TABLE column_swap_test.test; -CREATE TABLE constraint_test.test (i Int64, a String, b String, CONSTRAINT c1 ASSUME a = substring(reverse(b), 1, 1)) ENGINE = MergeTree() ORDER BY i; -INSERT INTO constraint_test.test SELECT number AS i, toString(number) AS a, format('test {} kek {}', toString(number), toString(number + 10)) b FROM system.numbers LIMIT 1000000; +CREATE TABLE column_swap_test.test (i Int64, a String, b String, CONSTRAINT c1 ASSUME a = substring(reverse(b), 1, 1)) ENGINE = MergeTree() ORDER BY i; +INSERT INTO column_swap_test.test SELECT number AS i, toString(number) AS a, format('test {} kek {}', toString(number), toString(number + 10)) b FROM system.numbers LIMIT 1000000; -EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM constraint_test.test WHERE a = 'c'; -EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM constraint_test.test WHERE substring(reverse(b), 1, 1) = 'c'; -EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) FROM constraint_test.test WHERE substring(reverse(b), 1, 1) = 'c'; +EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test.test WHERE a = 'c'; +EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test.test WHERE substring(reverse(b), 1, 1) = 'c'; +EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) FROM column_swap_test.test WHERE substring(reverse(b), 1, 1) = 'c'; -DROP TABLE constraint_test.test; +DROP TABLE column_swap_test.test; -DROP DATABASE constraint_test; +DROP DATABASE column_swap_test; diff --git a/tests/queries/0_stateless/01624_soft_constraints.reference b/tests/queries/0_stateless/01624_soft_constraints.reference index f0a866816d6..fef1b3c48a7 100644 --- a/tests/queries/0_stateless/01624_soft_constraints.reference +++ b/tests/queries/0_stateless/01624_soft_constraints.reference @@ -6,3 +6,11 @@ "rows_read": 2, "rows_read": 4, "rows_read": 2, + "rows_read": 4, + "rows_read": 4, + "rows_read": 1, + "rows_read": 4, + "rows_read": 3, + "rows_read": 4, + "rows_read": 4, + "rows_read": 3, diff --git a/tests/queries/0_stateless/01624_soft_constraints.sh b/tests/queries/0_stateless/01624_soft_constraints.sh index 30f77446b49..45eb122d05b 100755 --- a/tests/queries/0_stateless/01624_soft_constraints.sh +++ b/tests/queries/0_stateless/01624_soft_constraints.sh @@ -9,10 +9,13 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) SETTINGS="SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET optimize_move_to_prewhere = 1;" $CLICKHOUSE_CLIENT -n --query=" -DROP DATABASE IF EXISTS constraint_test; -DROP TABLE IF EXISTS constraint_test.test; -CREATE DATABASE constraint_test; -CREATE TABLE constraint_test.test ( +DROP DATABASE IF EXISTS hypothesis_test; +DROP TABLE IF EXISTS hypothesis_test.test; +DROP TABLE IF EXISTS hypothesis_test.test2; +DROP TABLE IF EXISTS hypothesis_test.test3; + +CREATE DATABASE hypothesis_test; +CREATE TABLE hypothesis_test.test ( i UInt64, a UInt64, b UInt64, @@ -22,28 +25,79 @@ CREATE TABLE constraint_test.test ( ) ENGINE = MergeTree() ORDER BY i SETTINGS index_granularity=1; " -$CLICKHOUSE_CLIENT --query="INSERT INTO constraint_test.test VALUES +$CLICKHOUSE_CLIENT --query="INSERT INTO hypothesis_test.test VALUES (0, 1, 2, 2), (1, 2, 1, 2), (2, 2, 2, 1), (3, 1, 2, 3)" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE b > a FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE b > a FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE b <= a FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE b <= a FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE b >= a FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE b >= a FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE b = a FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE b = a FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE c < a FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE c < a FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE c = a FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE c = a FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE c > a FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE c > a FORMAT JSON" | grep "rows_read" # 4 + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE c < a FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM constraint_test.test WHERE c < a FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT -n --query=" -DROP TABLE constraint_test.test; -DROP DATABASE constraint_test;" +CREATE TABLE hypothesis_test.test2 ( + i UInt64, + a UInt64, + b UInt64, + INDEX t (a != b) TYPE hypothesis GRANULARITY 1 +) ENGINE = MergeTree() ORDER BY i SETTINGS index_granularity=1; +" + +$CLICKHOUSE_CLIENT --query="INSERT INTO hypothesis_test.test2 VALUES +(0, 1, 2), +(1, 2, 1), +(2, 2, 2), +(3, 1, 0)" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test2 WHERE a < b FORMAT JSON" | grep "rows_read" # 4 + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test2 WHERE a <= b FORMAT JSON" | grep "rows_read" # 4 + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test2 WHERE a = b FORMAT JSON" | grep "rows_read" # 1 + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test2 WHERE a != b FORMAT JSON" | grep "rows_read" # 4 + + +$CLICKHOUSE_CLIENT -n --query=" +CREATE TABLE hypothesis_test.test3 ( + i UInt64, + a UInt64, + b UInt64, + INDEX t (a = b) TYPE hypothesis GRANULARITY 1 +) ENGINE = MergeTree() ORDER BY i SETTINGS index_granularity=1; +" + +$CLICKHOUSE_CLIENT --query="INSERT INTO hypothesis_test.test3 VALUES +(0, 1, 2), +(1, 2, 1), +(2, 2, 2), +(3, 1, 0)" + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test3 WHERE a < b FORMAT JSON" | grep "rows_read" # 3 + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test3 WHERE a <= b FORMAT JSON" | grep "rows_read" # 4 + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test3 WHERE a = b FORMAT JSON" | grep "rows_read" # 4 + +$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test3 WHERE a != b FORMAT JSON" | grep "rows_read" # 3 + + +$CLICKHOUSE_CLIENT -n --query=" +DROP TABLE hypothesis_test.test; +DROP TABLE hypothesis_test.test2; +DROP TABLE hypothesis_test.test3; +DROP DATABASE hypothesis_test;" diff --git a/tests/queries/0_stateless/01625_constraints_index_append.reference b/tests/queries/0_stateless/01625_constraints_index_append.reference index 867f65bb426..1f2bccf9f28 100644 --- a/tests/queries/0_stateless/01625_constraints_index_append.reference +++ b/tests/queries/0_stateless/01625_constraints_index_append.reference @@ -1,15 +1,15 @@ SELECT i -FROM constraint_test.test +FROM index_append_test.test PREWHERE a = 0 WHERE (a = 0) AND indexHint((i + 40) > 0) SELECT i -FROM constraint_test.test +FROM index_append_test.test PREWHERE a < 0 SELECT i -FROM constraint_test.test +FROM index_append_test.test PREWHERE a >= 0 WHERE (a >= 0) AND indexHint((i + 40) > 0) SELECT i -FROM constraint_test.test +FROM index_append_test.test PREWHERE (2 * b) < 100 WHERE ((2 * b) < 100) AND indexHint(i < 100) diff --git a/tests/queries/0_stateless/01625_constraints_index_append.sql b/tests/queries/0_stateless/01625_constraints_index_append.sql index 8d668d1a306..28c18334369 100644 --- a/tests/queries/0_stateless/01625_constraints_index_append.sql +++ b/tests/queries/0_stateless/01625_constraints_index_append.sql @@ -2,19 +2,19 @@ SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET optimize_move_to_prewhere = 1; -DROP DATABASE IF EXISTS constraint_test; -DROP TABLE IF EXISTS constraint_test.test; +DROP DATABASE IF EXISTS index_append_test; +DROP TABLE IF EXISTS index_append_test.test; -CREATE DATABASE constraint_test; +CREATE DATABASE index_append_test; -CREATE TABLE constraint_test.test (i Int64, a UInt32, b UInt64, CONSTRAINT c1 ASSUME i <= 2 * b AND i + 40 > a) ENGINE = MergeTree() ORDER BY i; -INSERT INTO constraint_test.test VALUES (1, 10, 1), (2, 20, 2); +CREATE TABLE index_append_test.test (i Int64, a UInt32, b UInt64, CONSTRAINT c1 ASSUME i <= 2 * b AND i + 40 > a) ENGINE = MergeTree() ORDER BY i; +INSERT INTO index_append_test.test VALUES (1, 10, 1), (2, 20, 2); -EXPLAIN SYNTAX SELECT i FROM constraint_test.test WHERE a = 0; -EXPLAIN SYNTAX SELECT i FROM constraint_test.test WHERE a < 0; -EXPLAIN SYNTAX SELECT i FROM constraint_test.test WHERE a >= 0; -EXPLAIN SYNTAX SELECT i FROM constraint_test.test WHERE 2 * b < 100; +EXPLAIN SYNTAX SELECT i FROM index_append_test.test WHERE a = 0; +EXPLAIN SYNTAX SELECT i FROM index_append_test.test WHERE a < 0; +EXPLAIN SYNTAX SELECT i FROM index_append_test.test WHERE a >= 0; +EXPLAIN SYNTAX SELECT i FROM index_append_test.test WHERE 2 * b < 100; -DROP TABLE constraint_test.test; +DROP TABLE index_append_test.test; -DROP DATABASE constraint_test; \ No newline at end of file +DROP DATABASE index_append_test; \ No newline at end of file From 06cc3610751647bbac73bd30522688ae9ed5d668 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Tue, 4 May 2021 14:07:43 +0300 Subject: [PATCH 039/200] rm unused --- src/Interpreters/ComparisonGraph.cpp | 23 ---------- .../WhereConstraintsOptimizer.cpp | 31 ------------- .../MergeTreeIndexMergedCondition.cpp | 45 ------------------- 3 files changed, 99 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 25a1a346886..2e5703dd58f 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -115,9 +115,6 @@ std::pair ComparisonGraph::findPath(const size_t start, const size_t return {false, false}; else { - //Poco::Logger::get("dists found").information(std::to_string(start) + " " + std::to_string(finish) + " : " + std::to_string(static_cast(it->second))); - //Poco::Logger::get("dists found").information(graph.vertexes[start].asts.back()->dumpTree()); - //Poco::Logger::get("dists found").information(graph.vertexes[finish].asts.back()->dumpTree()); return {true, it->second == Path::LESS || not_equal.contains({start, finish})}; } } @@ -135,10 +132,6 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con Poco::Logger::get("Graph").information("not found"); Poco::Logger::get("Graph").information(std::to_string(left->getTreeHash().second)); Poco::Logger::get("Graph").information(std::to_string(right->getTreeHash().second)); - /*for (const auto & [hash, id] : graph.ast_hash_to_component) - { - Poco::Logger::get("Graph MAP").information(std::to_string(hash.second) + " " + std::to_string(id)); - }*/ { const auto left_bound = getConstLowerBound(left); const auto right_bound = getConstUpperBound(right); @@ -521,22 +514,6 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as } } - /*for (size_t v = 0; v < result.vertexes.size(); ++v) - { - std::stringstream s; - for (const auto & atom : result.vertexes[v].asts) - { - s << atom->getTreeHash().second << " "; - } - s << "|"; - for (const auto & atom : result.ast_hash_to_component) - { - s << atom.first.second << " -" << atom.second << " "; - } - - Poco::Logger::get("Graph").information(s.str()); - }*/ - return result; } diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index 1d53eab5768..a548eae2eed 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -87,37 +87,6 @@ bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const std: ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula & atom) { - /*static const std::map inverse_relations = { - {"equals", "notEquals"}, - {"less", "greaterOrEquals"}, - {"lessOrEquals", "greater"}, - {"notEquals", "equals"}, - {"greaterOrEquals", "less"}, - {"greater", "lessOrEquals"}, - }; - - static const std::map relation_to_compare = { - {"equals", ComparisonGraph::CompareResult::EQUAL}, - {"less", ComparisonGraph::CompareResult::LESS}, - {"lessOrEquals", ComparisonGraph::CompareResult::LESS_OR_EQUAL}, - {"notEquals", ComparisonGraph::CompareResult::UNKNOWN}, - {"greaterOrEquals", ComparisonGraph::CompareResult::GREATER_OR_EQUAL}, - {"greater", ComparisonGraph::CompareResult::GREATER}, - }; - - - const auto * func = atom.ast->as(); - if (func && inverse_relations.count(func->name)) - { - std::string function_name = func->name; - if (atom.negative) - { - function_name = inverse_relations.at(func->name); - } - return relation_to_compare.at(function_name); - } - return ComparisonGraph::CompareResult::UNKNOWN;*/ - const auto * func = atom.ast->as(); if (func) { diff --git a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp index e32514c81a0..8352e1c0ff4 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp @@ -89,34 +89,6 @@ namespace ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula & atom) { - /*static const std::map inverse_relations = { - {"equals", "notEquals"}, - {"less", "greaterOrEquals"}, - {"lessOrEquals", "greater"}, - {"notEquals", "equals"}, - {"greaterOrEquals", "less"}, - {"greater", "lessOrEquals"}, - }; - - static const std::map relation_to_compare = { - {"equals", ComparisonGraph::CompareResult::EQUAL}, - {"less", ComparisonGraph::CompareResult::LESS}, - {"lessOrEquals", ComparisonGraph::CompareResult::LESS_OR_EQUAL}, - {"notEquals", ComparisonGraph::CompareResult::UNKNOWN}, - {"greaterOrEquals", ComparisonGraph::CompareResult::GREATER_OR_EQUAL}, - {"greater", ComparisonGraph::CompareResult::GREATER}, - }; - - - const auto * func = atom.ast->as(); - if (func && inverse_relations.count(func->name)) - { - std::string function_name = func->name; - if (atom.negative) - function_name = inverse_relations.at(func->name); - return relation_to_compare.at(function_name); - } - return ComparisonGraph::CompareResult::UNKNOWN;*/ const auto * func = atom.ast->as(); if (func) { @@ -212,23 +184,6 @@ bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranu } } } - /*for (auto atom : or_group) - { - pushNotIn(atom); - for (size_t i = 0; i < values.size(); ++i) - if (values[i]) - for (const auto & hypothesis_or_group : index_to_atomic_hypotheses[i]) - { - if (hypothesis_or_group.size() == 1) - { - const auto & hypothesis_atom = *std::begin(hypothesis_or_group); - if (atom.ast->getTreeHash() == hypothesis_atom.ast->getTreeHash()) - { - return; - } - } - } - }*/ always_false = true; }); return !always_false; From 125498c97de2e3383632410b1eaf0ebf42a149fb Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Tue, 4 May 2021 17:34:00 +0300 Subject: [PATCH 040/200] fix --- .../MergeTree/SubstituteColumnOptimizer.cpp | 98 +++++++++++-------- .../01623_constraints_column_swap.reference | 41 ++++---- .../01623_constraints_column_swap.sql | 1 + .../01625_constraints_index_append.reference | 8 +- 4 files changed, 85 insertions(+), 63 deletions(-) diff --git a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp index 1104e1078f3..8dcea9b852e 100644 --- a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp +++ b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp @@ -21,9 +21,9 @@ namespace ErrorCodes namespace { -const String COMPONENT = "__constraint_component_"; +const String COMPONENT = "__aorLwT30aH_comp"; +const String COMPONENT_SEPARATOR = "_"; constexpr UInt64 COLUMN_PENALTY = 10 * 1024 * 1024; -//constexpr size_t MAX_COMPONENTS_FOR_BRUTEFORCE = 10; class ComponentMatcher { @@ -34,19 +34,33 @@ public: { const ComparisonGraph & graph; std::set & components; + std::unordered_map & old_name; + std::unordered_map & component; + UInt64 & current_id; - Data(const ComparisonGraph & graph_, std::set & components_) - : graph(graph_), components(components_) + Data(const ComparisonGraph & graph_, + std::set & components_, + std::unordered_map & old_name_, + std::unordered_map & component_, + UInt64 & current_id_) + : graph(graph_) + , components(components_) + , old_name(old_name_) + , component(component_) + , current_id(current_id_) { } }; static void visit(ASTPtr & ast, Data & data) { - const auto id = data.graph.getComponentId(ast); + const auto id = data.graph.getComponentId(ast); if (id) { - ast = std::make_shared(COMPONENT + std::to_string(id.value())); + const String name = COMPONENT + std::to_string(id.value()) + COMPONENT_SEPARATOR + std::to_string(++data.current_id); + data.old_name[name] = ast->getAliasOrColumnName(); + data.component[name] = id.value(); + ast = std::make_shared(name); data.components.insert(id.value()); } } @@ -119,14 +133,23 @@ public: struct Data { - std::unordered_map id_to_expression_map; + std::unordered_map id_to_expression_map; + std::unordered_map name_to_component_id; + std::unordered_map old_name; + bool is_select; }; static void visit(ASTPtr & ast, Data & data) { const auto * identifier = ast->as(); - if (identifier && data.id_to_expression_map.contains(identifier->name())) - ast = data.id_to_expression_map.at(identifier->name())->clone(); + if (identifier && data.name_to_component_id.contains(identifier->name())) + { + const auto name = identifier->name(); + const auto component_id = data.name_to_component_id.at(name); + ast = data.id_to_expression_map.at(component_id)->clone(); + if (data.is_select) + ast->setAlias(data.old_name.at(name)); + } } static bool needChildVisit(const ASTPtr &, const ASTPtr &) @@ -160,14 +183,10 @@ void bruteforce( { if (current_component == components.size()) { - Poco::Logger::get("New price").information(std::to_string(current_price.compressed_size)); - for (const auto & ast : expressions_stack) - Poco::Logger::get("AST").information(ast->getColumnName()); if (current_price < min_price) { min_price = current_price; min_expressions = expressions_stack; - Poco::Logger::get("New price").information("UPDATE"); } } else @@ -177,7 +196,6 @@ void bruteforce( std::unordered_set identifiers; collectIdentifiers(ast, identifiers); ColumnPrice expression_price = calculatePrice(column_prices, identifiers); - Poco::Logger::get("EXPRPRICE").information( ast->getColumnName()+ " " + std::to_string(expression_price.compressed_size)); expressions_stack.push_back(ast); current_price += expression_price; @@ -233,24 +251,37 @@ void SubstituteColumnOptimizer::perform() const auto compare_graph = metadata_snapshot->getConstraints().getGraph(); + // Fill aliases + if (select_query->select()) + { + auto * list = select_query->refSelect()->as(); + if (!list) + throw Exception("Bad select list.", ErrorCodes::LOGICAL_ERROR); + for (ASTPtr & ast : list->children) + ast->setAlias(ast->getAliasOrColumnName()); + } + auto run_for_all = [&](const auto func) { if (select_query->where()) - func(select_query->refWhere()); + func(select_query->refWhere(), false); if (select_query->prewhere()) - func(select_query->refPrewhere()); + func(select_query->refPrewhere(), false); if (select_query->select()) - func(select_query->refSelect()); + func(select_query->refSelect(), true); if (select_query->having()) - func(select_query->refHaving()); + func(select_query->refHaving(), false); }; std::set components; - ComponentVisitor::Data component_data(compare_graph, components); + std::unordered_map old_name; + std::unordered_map name_to_component; + UInt64 counter_id = 0; + ComponentVisitor::Data component_data( + compare_graph, components, old_name, name_to_component, counter_id); std::unordered_set identifiers; - auto preprocess = [&](ASTPtr & ast) { + auto preprocess = [&](ASTPtr & ast, bool) { ComponentVisitor(component_data).visit(ast); collectIdentifiers(ast, identifiers); - Poco::Logger::get("kek").information(ast->dumpTree()); }; run_for_all(preprocess); @@ -259,36 +290,21 @@ void SubstituteColumnOptimizer::perform() const std::unordered_set primary_key_set(std::begin(primary_key), std::end(primary_key)); std::unordered_map column_prices; for (const auto & [column_name, column_size] : column_sizes) - { column_prices[column_name] = ColumnPrice( column_size.data_compressed + COLUMN_PENALTY, column_size.data_uncompressed); - Poco::Logger::get("COLUMNS").information(column_name + " " + std::to_string(column_size.data_compressed) + " " + std::to_string(column_size.data_uncompressed)); - } for (const auto & column_name : primary_key) - { column_prices[column_name] = ColumnPrice(0, 0); - Poco::Logger::get("PK COLUMNS").information(column_name + " " + std::to_string(column_prices[column_name].compressed_size)); - } for (const auto & column_name : identifiers) - { column_prices[column_name] = ColumnPrice(0, 0); - Poco::Logger::get("ID COLUMNS").information(column_name + " " + std::to_string(column_prices[column_name].compressed_size)); - } - std::unordered_map id_to_expression_map; + std::unordered_map id_to_expression_map; std::vector components_list; for (const UInt64 component : components) if (compare_graph.getComponent(component).size() == 1) - id_to_expression_map[COMPONENT + std::to_string(component)] = compare_graph.getComponent(component).front(); + id_to_expression_map[component] = compare_graph.getComponent(component).front(); else components_list.push_back(component); - /*if (components_list.size() > MAX_COMPONENTS_FOR_BRUTEFORCE) - { - Poco::Logger::get("SubstituteColumnOptimizer").information("skip: too many components for bruteforce"); - return; - }*/ - std::vector expressions_stack; ColumnPrice min_price(std::numeric_limits::max(), std::numeric_limits::max()); std::vector min_expressions; @@ -302,10 +318,10 @@ void SubstituteColumnOptimizer::perform() min_expressions); for (size_t i = 0; i < min_expressions.size(); ++i) - id_to_expression_map[COMPONENT + std::to_string(components_list[i])] = min_expressions[i]; + id_to_expression_map[components_list[i]] = min_expressions[i]; - auto process = [&](ASTPtr & ast) { - SubstituteColumnVisitor::Data substitute_data{id_to_expression_map}; + auto process = [&](ASTPtr & ast, bool is_select) { + SubstituteColumnVisitor::Data substitute_data{id_to_expression_map, name_to_component, old_name, is_select}; SubstituteColumnVisitor(substitute_data).visit(ast); }; diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index f210d403d68..4b558ecbddb 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -1,46 +1,51 @@ SELECT - b + 10, - b + 3 + (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, + (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test.test WHERE b = 1 SELECT - b + 10, - b + 3 + (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, + (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test.test WHERE b = 0 SELECT - b + 10, - b + 3 + (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, + (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test.test WHERE b = 0 SELECT - b + 10, - b + 3 + (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, + (b AS b) + 3 AS `plus(b, 3)` FROM column_swap_test.test WHERE b = 1 -SELECT b + 10 +SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)` FROM column_swap_test.test WHERE b = 0 SELECT - cityHash64(a) + 10, - a + (cityHash64(a) AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, + a AS a FROM column_swap_test.test WHERE cityHash64(a) = 0 SELECT - cityHash64(a) + 10, - a + (cityHash64(a) AS b) + 10 AS `plus(b, 10)`, + a AS a FROM column_swap_test.test WHERE cityHash64(a) = 0 SELECT - a, - a + a AS `substring(reverse(b), 1, 1)`, + a AS a FROM column_swap_test.test WHERE a = \'c\' SELECT - a, - a + a AS `substring(reverse(b), 1, 1)`, + a AS a FROM column_swap_test.test WHERE a = \'c\' -SELECT a +SELECT + a AS t1, + a AS t2 +FROM column_swap_test.test +WHERE a = \'c\' +SELECT a AS `substring(reverse(b), 1, 1)` FROM column_swap_test.test WHERE a = \'c\' diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.sql b/tests/queries/0_stateless/01623_constraints_column_swap.sql index 3f4cce71894..96073007505 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.sql +++ b/tests/queries/0_stateless/01623_constraints_column_swap.sql @@ -27,6 +27,7 @@ INSERT INTO column_swap_test.test SELECT number AS i, toString(number) AS a, for EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test.test WHERE a = 'c'; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test.test WHERE substring(reverse(b), 1, 1) = 'c'; +EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) AS t1, a AS t2 FROM column_swap_test.test WHERE substring(reverse(b), 1, 1) = 'c'; EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) FROM column_swap_test.test WHERE substring(reverse(b), 1, 1) = 'c'; DROP TABLE column_swap_test.test; diff --git a/tests/queries/0_stateless/01625_constraints_index_append.reference b/tests/queries/0_stateless/01625_constraints_index_append.reference index 1f2bccf9f28..e52b87fbd88 100644 --- a/tests/queries/0_stateless/01625_constraints_index_append.reference +++ b/tests/queries/0_stateless/01625_constraints_index_append.reference @@ -1,15 +1,15 @@ -SELECT i +SELECT i AS i FROM index_append_test.test PREWHERE a = 0 WHERE (a = 0) AND indexHint((i + 40) > 0) -SELECT i +SELECT i AS i FROM index_append_test.test PREWHERE a < 0 -SELECT i +SELECT i AS i FROM index_append_test.test PREWHERE a >= 0 WHERE (a >= 0) AND indexHint((i + 40) > 0) -SELECT i +SELECT i AS i FROM index_append_test.test PREWHERE (2 * b) < 100 WHERE ((2 * b) < 100) AND indexHint(i < 100) From 496d8ff46cfa6012964027532be3145cd30e2fea Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Tue, 4 May 2021 21:43:58 +0300 Subject: [PATCH 041/200] improvements --- .../AddIndexConstraintsOptimizer.cpp | 15 ++- src/Interpreters/ComparisonGraph.cpp | 107 ++++++++++-------- src/Interpreters/ComparisonGraph.h | 10 +- src/Interpreters/ConstraintMatcherVisitor.cpp | 37 ++++++ src/Interpreters/ConstraintMatcherVisitor.h | 29 +---- src/Interpreters/TreeCNFConverter.cpp | 6 +- src/Interpreters/TreeOptimizer.cpp | 12 +- .../WhereConstraintsOptimizer.cpp | 23 ++-- src/Interpreters/WhereConstraintsOptimizer.h | 9 +- src/Parsers/ASTConstraintDeclaration.h | 3 +- src/Storages/ConstraintsDescription.cpp | 7 +- src/Storages/ConstraintsDescription.h | 3 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 5 + .../MergeTree/MergeTreeIndexHypothesis.cpp | 2 +- .../MergeTreeIndexMergedCondition.cpp | 4 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 4 +- .../MergeTree/SubstituteColumnOptimizer.cpp | 19 +--- .../MergeTree/SubstituteColumnOptimizer.h | 11 +- .../0_stateless/01624_soft_constraints.sh | 60 +++++----- 19 files changed, 190 insertions(+), 176 deletions(-) diff --git a/src/Interpreters/AddIndexConstraintsOptimizer.cpp b/src/Interpreters/AddIndexConstraintsOptimizer.cpp index 63a24eba094..06a9e59d297 100644 --- a/src/Interpreters/AddIndexConstraintsOptimizer.cpp +++ b/src/Interpreters/AddIndexConstraintsOptimizer.cpp @@ -12,10 +12,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} AddIndexConstraintsOptimizer::AddIndexConstraintsOptimizer( const StorageMetadataPtr & metadata_snapshot_) @@ -124,8 +120,10 @@ namespace { Poco::Logger::get("INDEX_HINT_CREATE").information("CHECK"); const auto * func = atom.ast->as(); - if (func && func->arguments->children.size() == 2 && getRelationMap().contains(func->name)) { - auto check_and_insert = [&](const size_t index, const ComparisonGraph::CompareResult need_result) -> bool { + if (func && func->arguments->children.size() == 2 && getRelationMap().contains(func->name)) + { + auto check_and_insert = [&](const size_t index, const ComparisonGraph::CompareResult need_result) -> bool + { if (!onlyConstants(func->arguments->children[1 - index])) return false; @@ -167,13 +165,14 @@ void AddIndexConstraintsOptimizer::perform(CNFQuery & cnf_query) const std::unordered_set primary_key_set(std::begin(primary_key), std::end(primary_key)); ASTs primary_key_only_asts; - for (const auto & vertex : graph.getVertexes()) + for (const auto & vertex : graph.getVertices()) for (const auto & ast : vertex) if (hasIndexColumns(ast, primary_key_set) && onlyIndexColumns(ast, primary_key_set)) primary_key_only_asts.push_back(ast); CNFQuery::AndGroup and_group; - cnf_query.iterateGroups([&and_group, &graph, &primary_key_only_asts](const auto & or_group) { + cnf_query.iterateGroups([&and_group, &graph, &primary_key_only_asts](const auto & or_group) + { auto add_group = createIndexHintGroup(or_group, graph, primary_key_only_asts); if (!add_group.empty()) and_group.emplace(std::move(add_group)); diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 2e5703dd58f..83a887ce2bc 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -36,25 +36,29 @@ ASTPtr ComparisonGraph::normalizeAtom(const ASTPtr & atom) const ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) { - static const std::map relation_to_enum = { + static const std::unordered_map relation_to_enum = + { {"equals", Edge::Type::EQUAL}, {"less", Edge::Type::LESS}, {"lessOrEquals", Edge::Type::LESS_OR_EQUAL}, }; Graph g; - for (const auto & atom_raw : atomic_formulas) { + for (const auto & atom_raw : atomic_formulas) + { const auto atom = normalizeAtom(atom_raw); const auto bad_term = std::numeric_limits::max(); - auto get_index = [](const ASTPtr & ast, Graph & asts_graph) -> std::size_t { + auto get_index = [](const ASTPtr & ast, Graph & asts_graph) -> std::size_t + { const auto it = asts_graph.ast_hash_to_component.find(ast->getTreeHash()); if (it != std::end(asts_graph.ast_hash_to_component)) { if (!std::any_of( - std::cbegin(asts_graph.vertexes[it->second].asts), - std::cend(asts_graph.vertexes[it->second].asts), - [ast](const ASTPtr & constraint_ast) { + std::cbegin(asts_graph.vertices[it->second].asts), + std::cend(asts_graph.vertices[it->second].asts), + [ast](const ASTPtr & constraint_ast) + { return constraint_ast->getTreeHash() == ast->getTreeHash() && constraint_ast->getColumnName() == ast->getColumnName(); })) @@ -66,10 +70,10 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) } else { - asts_graph.ast_hash_to_component[ast->getTreeHash()] = asts_graph.vertexes.size(); - asts_graph.vertexes.push_back(EqualComponent{{ast}}); + asts_graph.ast_hash_to_component[ast->getTreeHash()] = asts_graph.vertices.size(); + asts_graph.vertices.push_back(EqualComponent{{ast}}); asts_graph.edges.emplace_back(); - return asts_graph.vertexes.size() - 1; + return asts_graph.vertices.size() - 1; } }; @@ -263,15 +267,18 @@ std::optional ComparisonGraph::getComponentId(const ASTPtr & ast) c return {}; const size_t index = hash_it->second; if (std::any_of( - std::cbegin(graph.vertexes[index].asts), - std::cend(graph.vertexes[index].asts), + std::cbegin(graph.vertices[index].asts), + std::cend(graph.vertices[index].asts), [ast](const ASTPtr & constraint_ast) { return constraint_ast->getTreeHash() == ast->getTreeHash() && constraint_ast->getColumnName() == ast->getColumnName(); - })) { + })) + { return index; - } else { + } + else + { return {}; } } @@ -283,18 +290,21 @@ bool ComparisonGraph::hasPath(const size_t left, const size_t right) const std::vector ComparisonGraph::getComponent(const std::size_t id) const { - return graph.vertexes[id].asts; + return graph.vertices[id].asts; } -bool ComparisonGraph::EqualComponent::hasConstant() const { +bool ComparisonGraph::EqualComponent::hasConstant() const +{ return constant_index != -1; } -ASTPtr ComparisonGraph::EqualComponent::getConstant() const { +ASTPtr ComparisonGraph::EqualComponent::getConstant() const +{ return asts[constant_index]; } -void ComparisonGraph::EqualComponent::buildConstants() { +void ComparisonGraph::EqualComponent::buildConstants() +{ constant_index = -1; for (size_t i = 0; i < asts.size(); ++i) { @@ -308,7 +318,8 @@ void ComparisonGraph::EqualComponent::buildConstants() { ComparisonGraph::CompareResult ComparisonGraph::getCompareResult(const std::string & name) { - static const std::unordered_map relation_to_compare = { + static const std::unordered_map relation_to_compare = + { {"equals", CompareResult::EQUAL}, {"notEquals", CompareResult::NOT_EQUAL}, {"less", CompareResult::LESS}, @@ -323,7 +334,8 @@ ComparisonGraph::CompareResult ComparisonGraph::getCompareResult(const std::stri ComparisonGraph::CompareResult ComparisonGraph::inverseCompareResult(const CompareResult result) { - static const std::unordered_map inverse_relations = { + static const std::unordered_map inverse_relations = + { {CompareResult::NOT_EQUAL, CompareResult::EQUAL}, {CompareResult::EQUAL, CompareResult::NOT_EQUAL}, {CompareResult::GREATER_OR_EQUAL, CompareResult::LESS}, @@ -341,8 +353,8 @@ std::optional ComparisonGraph::getEqualConst(const ASTPtr & ast) const if (hash_it == std::end(graph.ast_hash_to_component)) return std::nullopt; const size_t index = hash_it->second; - return graph.vertexes[index].hasConstant() - ? std::optional{graph.vertexes[index].getConstant()} + return graph.vertices[index].hasConstant() + ? std::optional{graph.vertices[index].getConstant()} : std::nullopt; } @@ -360,7 +372,7 @@ std::optional> ComparisonGraph::getConstUpperBound(const const ssize_t from = ast_const_upper_bound[to]; if (from == -1) return std::nullopt; - return std::make_pair(graph.vertexes[from].getConstant()->as()->value, dists.at({from, to}) == Path::LESS); + return std::make_pair(graph.vertices[from].getConstant()->as()->value, dists.at({from, to}) == Path::LESS); } std::optional> ComparisonGraph::getConstLowerBound(const ASTPtr & ast) const @@ -377,7 +389,7 @@ std::optional> ComparisonGraph::getConstLowerBound(const const ssize_t to = ast_const_lower_bound[from]; if (to == -1) return std::nullopt; - return std::make_pair(graph.vertexes[to].getConstant()->as()->value, dists.at({from, to}) == Path::LESS); + return std::make_pair(graph.vertices[to].getConstant()->as()->value, dists.at({from, to}) == Path::LESS); } void ComparisonGraph::dfsOrder(const Graph & asts_graph, size_t v, std::vector & visited, std::vector & order) const @@ -397,9 +409,9 @@ ComparisonGraph::Graph ComparisonGraph::reverseGraph(const Graph & asts_graph) c { Graph g; g.ast_hash_to_component = asts_graph.ast_hash_to_component; - g.vertexes = asts_graph.vertexes; - g.edges.resize(g.vertexes.size()); - for (size_t v = 0; v < asts_graph.vertexes.size(); ++v) + g.vertices = asts_graph.vertices; + g.edges.resize(g.vertices.size()); + for (size_t v = 0; v < asts_graph.vertices.size(); ++v) { for (const auto & edge : asts_graph.edges[v]) { @@ -409,10 +421,10 @@ ComparisonGraph::Graph ComparisonGraph::reverseGraph(const Graph & asts_graph) c return asts_graph; } -std::vector ComparisonGraph::getVertexes() const +std::vector ComparisonGraph::getVertices() const { std::vector result; - for (const auto & vertex : graph.vertexes) + for (const auto & vertex : graph.vertices) { result.emplace_back(); for (const auto & ast : vertex.asts) @@ -438,7 +450,7 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as { Poco::Logger::get("Graph").information("building"); /// Find strongly connected component - const auto n = asts_graph.vertexes.size(); + const auto n = asts_graph.vertices.size(); std::vector order; { @@ -466,19 +478,19 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as } Graph result; - result.vertexes.resize(component); + result.vertices.resize(component); result.edges.resize(component); for (const auto & [hash, index] : asts_graph.ast_hash_to_component) { result.ast_hash_to_component[hash] = components[index]; - result.vertexes[components[index]].asts.insert( - std::end(result.vertexes[components[index]].asts), - std::begin(asts_graph.vertexes[index].asts), - std::end(asts_graph.vertexes[index].asts)); // asts_graph has only one ast per vertex + result.vertices[components[index]].asts.insert( + std::end(result.vertices[components[index]].asts), + std::begin(asts_graph.vertices[index].asts), + std::end(asts_graph.vertices[index].asts)); // asts_graph has only one ast per vertex } /// Calculate constants - for (auto & vertex : result.vertexes) + for (auto & vertex : result.vertices) { vertex.buildConstants(); } @@ -494,16 +506,16 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as // TODO: make edges unique (left most strict) } - for (size_t v = 0; v < result.vertexes.size(); ++v) + for (size_t v = 0; v < result.vertices.size(); ++v) { - for (size_t u = 0; u < result.vertexes.size(); ++u) + for (size_t u = 0; u < result.vertices.size(); ++u) { if (v == u) continue; - if (result.vertexes[v].hasConstant() && result.vertexes[u].hasConstant()) + if (result.vertices[v].hasConstant() && result.vertices[u].hasConstant()) { - const auto * left = result.vertexes[v].getConstant()->as(); - const auto * right = result.vertexes[u].getConstant()->as(); + const auto * left = result.vertices[v].getConstant()->as(); + const auto * right = result.vertices[u].getConstant()->as(); /// Only less. Equal constant fields = equal literals so it was already considered above. if (left->value > right->value) @@ -521,7 +533,7 @@ std::map, ComparisonGraph::Path> ComparisonGraph::Buil { // min path : < = -1, =< = 0 const auto inf = std::numeric_limits::max(); - const size_t n = graph.vertexes.size(); + const size_t n = graph.vertices.size(); std::vector> results(n, std::vector(n, inf)); for (size_t v = 0; v < n; ++v) { @@ -547,24 +559,27 @@ std::map, ComparisonGraph::Path> ComparisonGraph::Buil std::pair, std::vector> ComparisonGraph::buildConstBounds() const { - const size_t n = graph.vertexes.size(); + const size_t n = graph.vertices.size(); std::vector lower(n, -1); std::vector upper(n, -1); - auto get_value = [this](const size_t vertex) -> Field { - return graph.vertexes[vertex].getConstant()->as()->value; + auto get_value = [this](const size_t vertex) -> Field + { + return graph.vertices[vertex].getConstant()->as()->value; }; for (const auto & [edge, path] : dists) { const auto [from, to] = edge; - if (graph.vertexes[to].hasConstant()) { + if (graph.vertices[to].hasConstant()) + { if (lower[from] == -1 || get_value(lower[from]) > get_value(to) || (get_value(lower[from]) >= get_value(to) && dists.at({from, to}) == Path::LESS)) lower[from] = to; } - if (graph.vertexes[from].hasConstant()) { + if (graph.vertices[from].hasConstant()) + { if (upper[to] == -1 || get_value(upper[to]) < get_value(from) || (get_value(upper[to]) <= get_value(from) && dists.at({from, to}) == Path::LESS)) diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 101294d3e6b..55d0e3c6d27 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -53,7 +53,7 @@ public: std::optional> getConstUpperBound(const ASTPtr & ast) const; std::optional> getConstLowerBound(const ASTPtr & ast) const; - std::vector getVertexes() const; + std::vector getVertices() const; private: /// strongly connected component @@ -86,14 +86,16 @@ private: struct Graph { - struct ASTHash { - size_t operator() (const IAST::Hash & hash) const { + struct ASTHash + { + size_t operator() (const IAST::Hash & hash) const + { return hash.first; } }; std::unordered_map ast_hash_to_component; - std::vector vertexes; + std::vector vertices; std::vector> edges; }; diff --git a/src/Interpreters/ConstraintMatcherVisitor.cpp b/src/Interpreters/ConstraintMatcherVisitor.cpp index 61b1c436b3a..b3050ea9eed 100644 --- a/src/Interpreters/ConstraintMatcherVisitor.cpp +++ b/src/Interpreters/ConstraintMatcherVisitor.cpp @@ -1 +1,38 @@ #include "ConstraintMatcherVisitor.h" + +#include +#include +#include +#include + +namespace DB +{ + +bool ConstraintMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &) +{ + return node->as() || node->as(); +} + +std::optional ConstraintMatcher::getASTValue(const ASTPtr & node, Data & data) +{ + const auto it = data.constraints.find(node->getTreeHash().second); + if (it != std::end(data.constraints)) + { + for (const auto & ast : it->second) + { + if (node->getColumnName() == ast->getColumnName()) + { + return true; + } + } + } + return std::nullopt; +} + +void ConstraintMatcher::visit(ASTPtr & ast, Data & data) +{ + if (const auto always_value = getASTValue(ast, data); always_value) + ast = std::make_shared(static_cast(*always_value)); +} + +} diff --git a/src/Interpreters/ConstraintMatcherVisitor.h b/src/Interpreters/ConstraintMatcherVisitor.h index 24f3e7037e5..be0abe91fec 100644 --- a/src/Interpreters/ConstraintMatcherVisitor.h +++ b/src/Interpreters/ConstraintMatcherVisitor.h @@ -1,13 +1,6 @@ #pragma once #include -#include -#include -#include -#include -#include -#include -#include namespace DB { @@ -21,25 +14,9 @@ struct ConstraintMatcher using Visitor = InDepthNodeVisitor; - static bool needChildVisit(const ASTPtr & node, const ASTPtr &) { return node->as() || node->as(); } - - static std::optional getASTValue(const ASTPtr & node, Data & data) { - const auto it = data.constraints.find(node->getTreeHash().second); - if (it != std::end(data.constraints)) { - for (const auto & ast : it->second) { - if (node->getColumnName() == ast->getColumnName()) { - return true; - } - } - } - return std::nullopt; - } - - static void visit(ASTPtr & ast, Data & data) - { - if (const auto always_value = getASTValue(ast, data); always_value) - ast = std::make_shared(static_cast(*always_value)); - } + static bool needChildVisit(const ASTPtr & node, const ASTPtr &); + static std::optional getASTValue(const ASTPtr & node, Data & data); + static void visit(ASTPtr & ast, Data & data); }; using ConstraintMatcherVisitor = InDepthNodeVisitor; diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index ad991b64299..fac3007f394 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -1,7 +1,6 @@ #include #include #include - #include namespace DB @@ -37,7 +36,8 @@ void traversePushNot(ASTPtr & node, bool add_negation) if (func && (func->name == "and" || func->name == "or")) { - if (add_negation) { + if (add_negation) + { /// apply De Morgan's Law node = makeASTFunction( (func->name == "and" ? "or" : "and"), @@ -302,7 +302,7 @@ CNFQuery & CNFQuery::pushNotInFuntions() std::string CNFQuery::dump() const { - std::stringstream res; + WriteBufferFromOwnString res; bool first = true; for (const auto & group : statements) { diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 92ca45259eb..6637792f8da 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -511,23 +511,23 @@ void optimizeLimitBy(const ASTSelectQuery * select_query) } /// Use constraints to get rid of useless parts of query -void optimizeWithConstraints(ASTSelectQuery * select_query, Aliases & aliases, const NameSet & source_columns_set, - const std::vector & tables_with_columns, +void optimizeWithConstraints(ASTSelectQuery * select_query, Aliases & /*aliases*/, const NameSet & /*source_columns_set*/, + const std::vector & /*tables_with_columns*/, const StorageMetadataPtr & metadata_snapshot) { - WhereConstraintsOptimizer(select_query, aliases, source_columns_set, tables_with_columns, metadata_snapshot).perform(); + WhereConstraintsOptimizer(select_query, metadata_snapshot).perform(); if (select_query->where()) Poco::Logger::get("CNF").information(select_query->where()->dumpTree()); else Poco::Logger::get("CNF").information("NO WHERE"); } -void optimizeSubstituteColumn(ASTSelectQuery * select_query, Aliases & aliases, const NameSet & source_columns_set, - const std::vector & tables_with_columns, +void optimizeSubstituteColumn(ASTSelectQuery * select_query, Aliases & /*aliases*/, const NameSet & /*source_columns_set*/, + const std::vector & /*tables_with_columns*/, const StorageMetadataPtr & metadata_snapshot, const ConstStoragePtr & storage) { - SubstituteColumnOptimizer(select_query, aliases, source_columns_set, tables_with_columns, metadata_snapshot, storage).perform(); + SubstituteColumnOptimizer(select_query, metadata_snapshot, storage).perform(); } /// transform where to CNF for more convenient optimization diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index a548eae2eed..bfdc88f34f2 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include #include @@ -13,21 +12,10 @@ namespace DB { -namespace ErrorCodes -{ -extern const int LOGICAL_ERROR; -} - WhereConstraintsOptimizer::WhereConstraintsOptimizer( ASTSelectQuery * select_query_, - Aliases & /*aliases_*/, - const NameSet & /*source_columns_set_*/, - const std::vector & /*tables_with_columns_*/, const StorageMetadataPtr & metadata_snapshot_) : select_query(select_query_) - /* , aliases(aliases_) - , source_columns_set(source_columns_set_) - , tables_with_columns(tables_with_columns_)*/ , metadata_snapshot(metadata_snapshot_) { } @@ -181,13 +169,18 @@ void WhereConstraintsOptimizer::perform() auto cnf = TreeCNFConverter::toCNF(select_query->where()); Poco::Logger::get("BEFORE OPT").information(cnf.dump()); cnf.pullNotOutFunctions() - .filterAlwaysTrueGroups([&constraint_data, &compare_graph](const auto & group) { /// remove always true groups from CNF + .filterAlwaysTrueGroups([&constraint_data, &compare_graph](const auto & group) + { + /// remove always true groups from CNF return !checkIfGroupAlwaysTrueFullMatch(group, constraint_data) && !checkIfGroupAlwaysTrueGraph(group, compare_graph); }) - .filterAlwaysFalseAtoms([&constraint_data, &compare_graph](const auto & atom) { /// remove always false atoms from CNF + .filterAlwaysFalseAtoms([&constraint_data, &compare_graph](const auto & atom) + { + /// remove always false atoms from CNF return !checkIfAtomAlwaysFalseFullMatch(atom, constraint_data) && !checkIfAtomAlwaysFalseGraph(atom, compare_graph); }) - .transformAtoms([&compare_graph](const auto & atom) { + .transformAtoms([&compare_graph](const auto & atom) + { return replaceTermsToConstants(atom, compare_graph); }) .pushNotInFuntions(); diff --git a/src/Interpreters/WhereConstraintsOptimizer.h b/src/Interpreters/WhereConstraintsOptimizer.h index fe13c943ef4..8a58aa41fa5 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.h +++ b/src/Interpreters/WhereConstraintsOptimizer.h @@ -17,17 +17,14 @@ using StorageMetadataPtr = std::shared_ptr; class WhereConstraintsOptimizer final { public: - WhereConstraintsOptimizer(ASTSelectQuery * select_query, Aliases & /* aliases */, const NameSet & /* source_columns_set */, - const std::vector & /* tables_with_columns */, - const StorageMetadataPtr & metadata_snapshot); + WhereConstraintsOptimizer( + ASTSelectQuery * select_query, + const StorageMetadataPtr & metadata_snapshot); void perform(); private: ASTSelectQuery * select_query; - /*Aliases & aliases; - const NameSet & source_columns_set; - const std::vector & tables_with_columns;*/ const StorageMetadataPtr & metadata_snapshot; }; diff --git a/src/Parsers/ASTConstraintDeclaration.h b/src/Parsers/ASTConstraintDeclaration.h index bdba7bd577a..5db8df43a76 100644 --- a/src/Parsers/ASTConstraintDeclaration.h +++ b/src/Parsers/ASTConstraintDeclaration.h @@ -10,7 +10,8 @@ namespace DB class ASTConstraintDeclaration : public IAST { public: - enum class Type { + enum class Type + { CHECK, ASSUME, }; diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index 4830ad2cd9e..042d7a344e5 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -53,13 +53,15 @@ ASTs ConstraintsDescription::filterConstraints(ConstraintType selection) const case ASTConstraintDeclaration::Type::ASSUME: return static_cast(ConstraintType::ASSUME); } + throw Exception("Unknown constraint type.", ErrorCodes::LOGICAL_ERROR); }; ASTs res; res.reserve(constraints.size()); for (const auto & constraint : constraints) { - if ((ast_to_decr_constraint_type(constraint->as()->type) & static_cast(selection)) != 0) { + if ((ast_to_decr_constraint_type(constraint->as()->type) & static_cast(selection)) != 0) + { res.push_back(constraint); } } @@ -88,7 +90,8 @@ std::vector ConstraintsDescription::getAtomicConstraint Poco::Logger::get("atomic_formula: initial:").information(constraint->as()->expr->ptr()->dumpTree()); const auto cnf = TreeCNFConverter::toCNF(constraint->as()->expr->ptr()) .pullNotOutFunctions(); - for (const auto & group : cnf.getStatements()) { + for (const auto & group : cnf.getStatements()) + { if (group.size() == 1) constraint_data.push_back(*group.begin()); } diff --git a/src/Storages/ConstraintsDescription.h b/src/Storages/ConstraintsDescription.h index 8f8cb750c52..00b5f6b6e4b 100644 --- a/src/Storages/ConstraintsDescription.h +++ b/src/Storages/ConstraintsDescription.h @@ -20,7 +20,8 @@ public: static ConstraintsDescription parse(const String & str); - enum class ConstraintType { + enum class ConstraintType + { CHECK = 1, ASSUME = 2, ALWAYS_TRUE = CHECK | ASSUME, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ae692439038..a2dbc6fc47f 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -2008,8 +2008,13 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingMergedIndex( for (size_t index_mark = index_range.begin; index_mark < index_range.end; ++index_mark) { if (index_mark != index_range.begin || !granules_filled || last_index_mark != index_range.begin) + { for (size_t i = 0; i < readers.size(); ++i) + { granules[i] = readers[i].read(); + granules_filled = true; + } + } MarkRange data_range( std::max(range.begin, index_mark * index_granularity), diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp index 263d9057159..57ef75824d4 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp @@ -83,7 +83,7 @@ MergeTreeIndexAggregatorPtr MergeTreeIndexHypothesis::createIndexAggregator() co MergeTreeIndexConditionPtr MergeTreeIndexHypothesis::createIndexCondition( const SelectQueryInfo &, ContextPtr) const { - return nullptr; + throw Exception("Not supported", ErrorCodes::LOGICAL_ERROR); } bool MergeTreeIndexHypothesis::mayBenefitFromIndexForIn(const ASTPtr &) const diff --git a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp index 8352e1c0ff4..82e745bce1a 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp @@ -13,7 +13,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int INCORRECT_QUERY; } MergeTreeIndexMergedCondition::MergeTreeIndexMergedCondition( @@ -103,7 +102,8 @@ ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula } /// Replaces < -> <=, > -> >= and assumes that all hypotheses are true then checks if path exists -bool MergeTreeIndexMergedCondition::alwaysUnknownOrTrue() const { +bool MergeTreeIndexMergedCondition::alwaysUnknownOrTrue() const +{ std::vector active_atomic_formulas(atomic_constraints); for (size_t i = 0; i < index_to_compare_atomic_hypotheses.size(); ++i) { diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index d28e4ecae90..0734ec33f67 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -189,8 +189,8 @@ void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const if (select.where()) { - Poco::Logger::get("MTPRWHERE WHE").information(select.where()->getColumnName()); - Poco::Logger::get("MTPRWHERE WHE").information(select.where()->dumpTree()); + Poco::Logger::get("MTPRWHERE WHERE").information(select.where()->getColumnName()); + Poco::Logger::get("MTPRWHERE WHERE").information(select.where()->dumpTree()); } if(select.prewhere()) { diff --git a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp index 8dcea9b852e..13535360fd8 100644 --- a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp +++ b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp @@ -13,10 +13,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} namespace { @@ -224,15 +220,9 @@ void bruteforce( SubstituteColumnOptimizer::SubstituteColumnOptimizer( ASTSelectQuery * select_query_, - Aliases & /*aliases_*/, - const NameSet & /*source_columns_set_*/, - const std::vector & /*tables_with_columns_*/, const StorageMetadataPtr & metadata_snapshot_, const ConstStoragePtr & storage_) : select_query(select_query_) - /* , aliases(aliases_) - , source_columns_set(source_columns_set_) - , tables_with_columns(tables_with_columns_)*/ , metadata_snapshot(metadata_snapshot_) , storage(storage_) { @@ -261,7 +251,8 @@ void SubstituteColumnOptimizer::perform() ast->setAlias(ast->getAliasOrColumnName()); } - auto run_for_all = [&](const auto func) { + auto run_for_all = [&](const auto func) + { if (select_query->where()) func(select_query->refWhere(), false); if (select_query->prewhere()) @@ -279,7 +270,8 @@ void SubstituteColumnOptimizer::perform() ComponentVisitor::Data component_data( compare_graph, components, old_name, name_to_component, counter_id); std::unordered_set identifiers; - auto preprocess = [&](ASTPtr & ast, bool) { + auto preprocess = [&](ASTPtr & ast, bool) + { ComponentVisitor(component_data).visit(ast); collectIdentifiers(ast, identifiers); }; @@ -320,7 +312,8 @@ void SubstituteColumnOptimizer::perform() for (size_t i = 0; i < min_expressions.size(); ++i) id_to_expression_map[components_list[i]] = min_expressions[i]; - auto process = [&](ASTPtr & ast, bool is_select) { + auto process = [&](ASTPtr & ast, bool is_select) + { SubstituteColumnVisitor::Data substitute_data{id_to_expression_map, name_to_component, old_name, is_select}; SubstituteColumnVisitor(substitute_data).visit(ast); }; diff --git a/src/Storages/MergeTree/SubstituteColumnOptimizer.h b/src/Storages/MergeTree/SubstituteColumnOptimizer.h index b7835385b45..6fc7c215afd 100644 --- a/src/Storages/MergeTree/SubstituteColumnOptimizer.h +++ b/src/Storages/MergeTree/SubstituteColumnOptimizer.h @@ -18,19 +18,14 @@ class SubstituteColumnOptimizer { public: SubstituteColumnOptimizer( - ASTSelectQuery * select_query, - Aliases & /* aliases */, const NameSet & /* source_columns_set */, - const std::vector & /* tables_with_columns */, - const StorageMetadataPtr & /* metadata_snapshot */, - const ConstStoragePtr & storage); + ASTSelectQuery * select_query, + const StorageMetadataPtr & metadata_snapshot, + const ConstStoragePtr & storage); void perform(); private: ASTSelectQuery * select_query; - /*Aliases & aliases; - const NameSet & source_columns_set; - const std::vector & tables_with_columns;*/ const StorageMetadataPtr & metadata_snapshot; ConstStoragePtr storage; }; diff --git a/tests/queries/0_stateless/01624_soft_constraints.sh b/tests/queries/0_stateless/01624_soft_constraints.sh index 45eb122d05b..3625d4f0e5f 100755 --- a/tests/queries/0_stateless/01624_soft_constraints.sh +++ b/tests/queries/0_stateless/01624_soft_constraints.sh @@ -6,9 +6,10 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -SETTINGS="SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET optimize_move_to_prewhere = 1;" +SETTINGS="SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET optimize_move_to_prewhere = 1" $CLICKHOUSE_CLIENT -n --query=" +$SETTINGS; DROP DATABASE IF EXISTS hypothesis_test; DROP TABLE IF EXISTS hypothesis_test.test; DROP TABLE IF EXISTS hypothesis_test.test2; @@ -25,30 +26,28 @@ CREATE TABLE hypothesis_test.test ( ) ENGINE = MergeTree() ORDER BY i SETTINGS index_granularity=1; " -$CLICKHOUSE_CLIENT --query="INSERT INTO hypothesis_test.test VALUES -(0, 1, 2, 2), -(1, 2, 1, 2), -(2, 2, 2, 1), -(3, 1, 2, 3)" +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; +INSERT INTO hypothesis_test.test VALUES (0, 1, 2, 2), (1, 2, 1, 2), (2, 2, 2, 1), (3, 1, 2, 3)" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE b > a FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE b > a FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE b <= a FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE b <= a FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE b >= a FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE b >= a FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE b = a FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE b = a FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE c < a FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE c < a FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE c = a FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE c = a FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE c > a FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE c > a FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test WHERE c < a FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE c < a FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT -n --query=" +$SETTINGS; CREATE TABLE hypothesis_test.test2 ( i UInt64, a UInt64, @@ -57,22 +56,20 @@ CREATE TABLE hypothesis_test.test2 ( ) ENGINE = MergeTree() ORDER BY i SETTINGS index_granularity=1; " -$CLICKHOUSE_CLIENT --query="INSERT INTO hypothesis_test.test2 VALUES -(0, 1, 2), -(1, 2, 1), -(2, 2, 2), -(3, 1, 0)" +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; +INSERT INTO hypothesis_test.test2 VALUES (0, 1, 2), (1, 2, 1), (2, 2, 2), (3, 1, 0)" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test2 WHERE a < b FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test2 WHERE a < b FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test2 WHERE a <= b FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test2 WHERE a <= b FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test2 WHERE a = b FORMAT JSON" | grep "rows_read" # 1 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test2 WHERE a = b FORMAT JSON" | grep "rows_read" # 1 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test2 WHERE a != b FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test2 WHERE a != b FORMAT JSON" | grep "rows_read" # 4 $CLICKHOUSE_CLIENT -n --query=" +$SETTINGS; CREATE TABLE hypothesis_test.test3 ( i UInt64, a UInt64, @@ -81,22 +78,21 @@ CREATE TABLE hypothesis_test.test3 ( ) ENGINE = MergeTree() ORDER BY i SETTINGS index_granularity=1; " -$CLICKHOUSE_CLIENT --query="INSERT INTO hypothesis_test.test3 VALUES -(0, 1, 2), -(1, 2, 1), -(2, 2, 2), -(3, 1, 0)" +$CLICKHOUSE_CLIENT -n --query=" +$SETTINGS; +INSERT INTO hypothesis_test.test3 VALUES (0, 1, 2), (1, 2, 1), (2, 2, 2), (3, 1, 0)" -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test3 WHERE a < b FORMAT JSON" | grep "rows_read" # 3 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test.test3 WHERE a < b FORMAT JSON" | grep "rows_read" # 3 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test3 WHERE a <= b FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test.test3 WHERE a <= b FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test3 WHERE a = b FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test.test3 WHERE a = b FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT --query="SELECT count() FROM hypothesis_test.test3 WHERE a != b FORMAT JSON" | grep "rows_read" # 3 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test.test3 WHERE a != b FORMAT JSON" | grep "rows_read" # 3 $CLICKHOUSE_CLIENT -n --query=" +$SETTINGS; DROP TABLE hypothesis_test.test; DROP TABLE hypothesis_test.test2; DROP TABLE hypothesis_test.test3; From 449e11f1ea3ad8fee6a7a4cdacc6a5f5ed1e840e Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Tue, 4 May 2021 22:18:37 +0300 Subject: [PATCH 042/200] settings --- src/CMakeLists.txt | 2 +- src/Interpreters/TreeOptimizer.cpp | 20 ++++++++++++------- .../WhereConstraintsOptimizer.cpp | 8 ++++++-- src/Interpreters/WhereConstraintsOptimizer.h | 4 +++- .../01622_constraints_simple_optimization.sql | 2 ++ .../01623_constraints_column_swap.sql | 2 ++ .../0_stateless/01624_soft_constraints.sh | 2 +- .../01625_constraints_index_append.sql | 2 ++ 8 files changed, 30 insertions(+), 12 deletions(-) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index f6f4d5b6199..69a84dbeb2c 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -69,7 +69,7 @@ add_subdirectory (Coordination) set(dbms_headers) -set(dbms_sources Interpreters/ConstraintMatcherVisitor.cpp Interpreters/ConstraintMatcherVisitor.h Interpreters/WhereConstraintsOptimizer.cpp Interpreters/WhereConstraintsOptimizer.h Interpreters/TreeCNFConverter.cpp Interpreters/TreeCNFConverter.h Interpreters/ComparisonGraph.cpp Interpreters/ComparisonGraph.h Storages/MergeTree/SubstituteColumnOptimizer.cpp Storages/MergeTree/SubstituteColumnOptimizer.h Storages/MergeTree/MergeTreeIndexHypothesis.cpp Storages/MergeTree/MergeTreeIndexHypothesis.h Interpreters/AddIndexConstraintsOptimizer.cpp Interpreters/AddIndexConstraintsOptimizer.h Storages/MergeTree/MergeTreeIndexMergedCondition.cpp Storages/MergeTree/MergeTreeIndexMergedCondition.h) +set(dbms_sources) add_headers_and_sources(clickhouse_common_io Common) add_headers_and_sources(clickhouse_common_io Common/HashTable) diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 6637792f8da..c82c908998d 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -511,18 +511,23 @@ void optimizeLimitBy(const ASTSelectQuery * select_query) } /// Use constraints to get rid of useless parts of query -void optimizeWithConstraints(ASTSelectQuery * select_query, Aliases & /*aliases*/, const NameSet & /*source_columns_set*/, - const std::vector & /*tables_with_columns*/, - const StorageMetadataPtr & metadata_snapshot) +void optimizeWithConstraints(ASTSelectQuery * select_query, + Aliases & /*aliases*/, + const NameSet & /*source_columns_set*/, + const std::vector & /*tables_with_columns*/, + const StorageMetadataPtr & metadata_snapshot, + const bool optimize_append_index) { - WhereConstraintsOptimizer(select_query, metadata_snapshot).perform(); + WhereConstraintsOptimizer(select_query, metadata_snapshot, optimize_append_index).perform(); if (select_query->where()) Poco::Logger::get("CNF").information(select_query->where()->dumpTree()); else Poco::Logger::get("CNF").information("NO WHERE"); } -void optimizeSubstituteColumn(ASTSelectQuery * select_query, Aliases & /*aliases*/, const NameSet & /*source_columns_set*/, +void optimizeSubstituteColumn(ASTSelectQuery * select_query, + Aliases & /*aliases*/, + const NameSet & /*source_columns_set*/, const std::vector & /*tables_with_columns*/, const StorageMetadataPtr & metadata_snapshot, const ConstStoragePtr & storage) @@ -647,8 +652,9 @@ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & sou if (settings.convert_query_to_cnf && settings.optimize_using_constraints) { - optimizeWithConstraints(select_query, aliases, source_columns_set, tables_with_columns, metadata_snapshot); - optimizeSubstituteColumn(select_query, aliases, source_columns_set, tables_with_columns, metadata_snapshot, storage); + optimizeWithConstraints(select_query, aliases, source_columns_set, tables_with_columns, metadata_snapshot, settings.optimize_append_index); + if (settings.optimize_substitute_columns) + optimizeSubstituteColumn(select_query, aliases, source_columns_set, tables_with_columns, metadata_snapshot, storage); } if (select_query->where()) { diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index bfdc88f34f2..13e09cd844c 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -14,9 +14,11 @@ namespace DB WhereConstraintsOptimizer::WhereConstraintsOptimizer( ASTSelectQuery * select_query_, - const StorageMetadataPtr & metadata_snapshot_) + const StorageMetadataPtr & metadata_snapshot_, + const bool optimize_append_index_) : select_query(select_query_) , metadata_snapshot(metadata_snapshot_) + , optimize_append_index(optimize_append_index_) { } @@ -184,7 +186,9 @@ void WhereConstraintsOptimizer::perform() return replaceTermsToConstants(atom, compare_graph); }) .pushNotInFuntions(); - AddIndexConstraintsOptimizer(metadata_snapshot).perform(cnf); + + if (optimize_append_index) + AddIndexConstraintsOptimizer(metadata_snapshot).perform(cnf); Poco::Logger::get("AFTER OPT").information(cnf.dump()); select_query->setExpression(ASTSelectQuery::Expression::WHERE, TreeCNFConverter::fromCNF(cnf)); diff --git a/src/Interpreters/WhereConstraintsOptimizer.h b/src/Interpreters/WhereConstraintsOptimizer.h index 8a58aa41fa5..75160983a76 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.h +++ b/src/Interpreters/WhereConstraintsOptimizer.h @@ -19,13 +19,15 @@ class WhereConstraintsOptimizer final public: WhereConstraintsOptimizer( ASTSelectQuery * select_query, - const StorageMetadataPtr & metadata_snapshot); + const StorageMetadataPtr & metadata_snapshot, + const bool optimize_append_index_); void perform(); private: ASTSelectQuery * select_query; const StorageMetadataPtr & metadata_snapshot; + bool optimize_append_index; }; } diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index e9370dc8492..0ab71cd8161 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -6,6 +6,8 @@ DROP TABLE IF EXISTS constraint_test.transitivity2; SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET optimize_move_to_prewhere = 1; +SET optimize_substitute_columns = 1; +SET optimize_append_index = 1; CREATE DATABASE constraint_test; CREATE TABLE constraint_test.assumption (URL String, a Int32, CONSTRAINT c1 ASSUME domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT c2 ASSUME URL > 'zzz' AND startsWith(URL, 'test') = True) ENGINE = TinyLog; diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.sql b/tests/queries/0_stateless/01623_constraints_column_swap.sql index 96073007505..a6c5c5b6bc5 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.sql +++ b/tests/queries/0_stateless/01623_constraints_column_swap.sql @@ -1,6 +1,8 @@ SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET optimize_move_to_prewhere = 1; +SET optimize_substitute_columns = 1; +SET optimize_append_index = 1; DROP DATABASE IF EXISTS column_swap_test; DROP TABLE IF EXISTS column_swap_test.test; diff --git a/tests/queries/0_stateless/01624_soft_constraints.sh b/tests/queries/0_stateless/01624_soft_constraints.sh index 3625d4f0e5f..d27d51079ae 100755 --- a/tests/queries/0_stateless/01624_soft_constraints.sh +++ b/tests/queries/0_stateless/01624_soft_constraints.sh @@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CURDIR"/../shell_config.sh -SETTINGS="SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET optimize_move_to_prewhere = 1" +SETTINGS="SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET optimize_move_to_prewhere = 1; SET optimize_substitute_columns = 1; SET optimize_append_index = 1" $CLICKHOUSE_CLIENT -n --query=" $SETTINGS; diff --git a/tests/queries/0_stateless/01625_constraints_index_append.sql b/tests/queries/0_stateless/01625_constraints_index_append.sql index 28c18334369..7d75c57218a 100644 --- a/tests/queries/0_stateless/01625_constraints_index_append.sql +++ b/tests/queries/0_stateless/01625_constraints_index_append.sql @@ -1,6 +1,8 @@ SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET optimize_move_to_prewhere = 1; +SET optimize_substitute_columns = 1; +SET optimize_append_index = 1; DROP DATABASE IF EXISTS index_append_test; DROP TABLE IF EXISTS index_append_test.test; From 3be49eeedb8834f3384331f30a9b79aacb88b773 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Tue, 4 May 2021 22:31:32 +0300 Subject: [PATCH 043/200] fix tests --- .../01622_constraints_simple_optimization.sql | 124 +++++++++--------- .../01623_constraints_column_swap.reference | 22 ++-- .../01623_constraints_column_swap.sql | 43 +++--- .../0_stateless/01624_soft_constraints.sh | 59 ++++----- .../01625_constraints_index_append.reference | 8 +- .../01625_constraints_index_append.sql | 21 ++- 6 files changed, 130 insertions(+), 147 deletions(-) diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index 0ab71cd8161..2fd9e883b1f 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -1,7 +1,6 @@ -DROP DATABASE IF EXISTS constraint_test; -DROP TABLE IF EXISTS constraint_test.assumption; -DROP TABLE IF EXISTS constraint_test.transitivity; -DROP TABLE IF EXISTS constraint_test.transitivity2; +DROP TABLE IF EXISTS constraint_test_assumption; +DROP TABLE IF EXISTS constraint_test_transitivity; +DROP TABLE IF EXISTS constraint_test_transitivity2; SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; @@ -9,94 +8,91 @@ SET optimize_move_to_prewhere = 1; SET optimize_substitute_columns = 1; SET optimize_append_index = 1; -CREATE DATABASE constraint_test; -CREATE TABLE constraint_test.assumption (URL String, a Int32, CONSTRAINT c1 ASSUME domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT c2 ASSUME URL > 'zzz' AND startsWith(URL, 'test') = True) ENGINE = TinyLog; +CREATE TABLE constraint_test_assumption (URL String, a Int32, CONSTRAINT c1 ASSUME domainWithoutWWW(URL) = 'yandex.ru', CONSTRAINT c2 ASSUME URL > 'zzz' AND startsWith(URL, 'test') = True) ENGINE = TinyLog; --- Add wrong rows in order to check optimization -INSERT INTO constraint_test.assumption (URL, a) VALUES ('1', 1); -INSERT INTO constraint_test.assumption (URL, a) VALUES ('2', 2); -INSERT INTO constraint_test.assumption (URL, a) VALUES ('yandex.ru', 3); -INSERT INTO constraint_test.assumption (URL, a) VALUES ('3', 4); +INSERT INTO constraint_test_assumption (URL, a) VALUES ('1', 1); +INSERT INTO constraint_test_assumption (URL, a) VALUES ('2', 2); +INSERT INTO constraint_test_assumption (URL, a) VALUES ('yandex.ru', 3); +INSERT INTO constraint_test_assumption (URL, a) VALUES ('3', 4); -SELECT count() FROM constraint_test.assumption WHERE domainWithoutWWW(URL) = 'yandex.ru'; --- assumption -> 4 -SELECT count() FROM constraint_test.assumption WHERE NOT (domainWithoutWWW(URL) = 'yandex.ru'); --- assumption -> 0 -SELECT count() FROM constraint_test.assumption WHERE domainWithoutWWW(URL) != 'yandex.ru'; --- assumption -> 0 -SELECT count() FROM constraint_test.assumption WHERE domainWithoutWWW(URL) = 'nothing'; --- not optimized -> 0 +SELECT count() FROM constraint_test_assumption WHERE domainWithoutWWW(URL) = 'yandex.ru'; --- assumption -> 4 +SELECT count() FROM constraint_test_assumption WHERE NOT (domainWithoutWWW(URL) = 'yandex.ru'); --- assumption -> 0 +SELECT count() FROM constraint_test_assumption WHERE domainWithoutWWW(URL) != 'yandex.ru'; --- assumption -> 0 +SELECT count() FROM constraint_test_assumption WHERE domainWithoutWWW(URL) = 'nothing'; --- not optimized -> 0 -SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL > 'zzz'); ---> assumption -> 4 -SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND NOT URL <= 'zzz'); ---> assumption -> 4 -SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL > 'zzz') OR (a = 10 AND a + 5 < 100); ---> assumption -> 4 -SELECT count() FROM constraint_test.assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL = '111'); ---> assumption & no assumption -> 0 -SELECT count() FROM constraint_test.assumption WHERE (startsWith(URL, 'test') = True); ---> assumption -> 4 +SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL > 'zzz'); ---> assumption -> 4 +SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND NOT URL <= 'zzz'); ---> assumption -> 4 +SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL > 'zzz') OR (a = 10 AND a + 5 < 100); ---> assumption -> 4 +SELECT count() FROM constraint_test_assumption WHERE (domainWithoutWWW(URL) = 'yandex.ru' AND URL = '111'); ---> assumption & no assumption -> 0 +SELECT count() FROM constraint_test_assumption WHERE (startsWith(URL, 'test') = True); ---> assumption -> 4 -DROP TABLE constraint_test.assumption; +DROP TABLE constraint_test_assumption; -CREATE TABLE constraint_test.transitivity (a Int64, b Int64, c Int64, d Int32, CONSTRAINT c1 ASSUME a = b AND c = d, CONSTRAINT c2 ASSUME b = c) ENGINE = TinyLog; +CREATE TABLE constraint_test_transitivity (a Int64, b Int64, c Int64, d Int32, CONSTRAINT c1 ASSUME a = b AND c = d, CONSTRAINT c2 ASSUME b = c) ENGINE = TinyLog; -INSERT INTO constraint_test.transitivity (a, b, c, d) VALUES (1, 2, 3, 4); +INSERT INTO constraint_test_transitivity (a, b, c, d) VALUES (1, 2, 3, 4); -SELECT count() FROM constraint_test.transitivity WHERE a = d; ---> assumption -> 1 +SELECT count() FROM constraint_test_transitivity WHERE a = d; ---> assumption -> 1 -DROP TABLE constraint_test.transitivity; +DROP TABLE constraint_test_transitivity; -CREATE TABLE constraint_test.strong_connectivity (a String, b String, c String, d String, CONSTRAINT c1 ASSUME a <= b AND b <= c AND c <= d AND d <= a) ENGINE = TinyLog; +CREATE TABLE constraint_test_strong_connectivity (a String, b String, c String, d String, CONSTRAINT c1 ASSUME a <= b AND b <= c AND c <= d AND d <= a) ENGINE = TinyLog; -INSERT INTO constraint_test.strong_connectivity (a, b, c, d) VALUES ('1', '2', '3', '4'); +INSERT INTO constraint_test_strong_connectivity (a, b, c, d) VALUES ('1', '2', '3', '4'); -SELECT count() FROM constraint_test.strong_connectivity WHERE a = d; ---> assumption -> 1 -SELECT count() FROM constraint_test.strong_connectivity WHERE a = c AND b = d; ---> assumption -> 1 -SELECT count() FROM constraint_test.strong_connectivity WHERE a < c OR b < d; ---> assumption -> 0 -SELECT count() FROM constraint_test.strong_connectivity WHERE a <= c OR b <= d; ---> assumption -> 1 +SELECT count() FROM constraint_test_strong_connectivity WHERE a = d; ---> assumption -> 1 +SELECT count() FROM constraint_test_strong_connectivity WHERE a = c AND b = d; ---> assumption -> 1 +SELECT count() FROM constraint_test_strong_connectivity WHERE a < c OR b < d; ---> assumption -> 0 +SELECT count() FROM constraint_test_strong_connectivity WHERE a <= c OR b <= d; ---> assumption -> 1 -DROP TABLE constraint_test.strong_connectivity; +DROP TABLE constraint_test_strong_connectivity; -CREATE TABLE constraint_test.transitivity2 (a String, b String, c String, d String, CONSTRAINT c1 ASSUME a > b AND b >= c AND c > d AND a >= d) ENGINE = TinyLog; +CREATE TABLE constraint_test_transitivity2 (a String, b String, c String, d String, CONSTRAINT c1 ASSUME a > b AND b >= c AND c > d AND a >= d) ENGINE = TinyLog; -INSERT INTO constraint_test.transitivity2 (a, b, c, d) VALUES ('1', '2', '3', '4'); +INSERT INTO constraint_test_transitivity2 (a, b, c, d) VALUES ('1', '2', '3', '4'); -SELECT count() FROM constraint_test.transitivity2 WHERE a > d; ---> assumption -> 1 -SELECT count() FROM constraint_test.transitivity2 WHERE a >= d; ---> assumption -> 1 -SELECT count() FROM constraint_test.transitivity2 WHERE d < a; ---> assumption -> 1 -SELECT count() FROM constraint_test.transitivity2 WHERE a < d; ---> assumption -> 0 -SELECT count() FROM constraint_test.transitivity2 WHERE a = d; ---> assumption -> 0 -SELECT count() FROM constraint_test.transitivity2 WHERE a != d; ---> assumption -> 1 +SELECT count() FROM constraint_test_transitivity2 WHERE a > d; ---> assumption -> 1 +SELECT count() FROM constraint_test_transitivity2 WHERE a >= d; ---> assumption -> 1 +SELECT count() FROM constraint_test_transitivity2 WHERE d < a; ---> assumption -> 1 +SELECT count() FROM constraint_test_transitivity2 WHERE a < d; ---> assumption -> 0 +SELECT count() FROM constraint_test_transitivity2 WHERE a = d; ---> assumption -> 0 +SELECT count() FROM constraint_test_transitivity2 WHERE a != d; ---> assumption -> 1 -DROP TABLE constraint_test.transitivity2; +DROP TABLE constraint_test_transitivity2; -CREATE TABLE constraint_test.transitivity3 (a Int64, b Int64, c Int64, CONSTRAINT c1 ASSUME b > 10 AND 1 > a) ENGINE = TinyLog; +CREATE TABLE constraint_test_transitivity3 (a Int64, b Int64, c Int64, CONSTRAINT c1 ASSUME b > 10 AND 1 > a) ENGINE = TinyLog; -INSERT INTO constraint_test.transitivity3 (a, b, c) VALUES (4, 0, 2); +INSERT INTO constraint_test_transitivity3 (a, b, c) VALUES (4, 0, 2); -SELECT count() FROM constraint_test.transitivity3 WHERE a < b; ---> assumption -> 1 -SELECT count() FROM constraint_test.transitivity3 WHERE b >= a; ---> assumption -> 1 +SELECT count() FROM constraint_test_transitivity3 WHERE a < b; ---> assumption -> 1 +SELECT count() FROM constraint_test_transitivity3 WHERE b >= a; ---> assumption -> 1 -DROP TABLE constraint_test.transitivity3; +DROP TABLE constraint_test_transitivity3; -CREATE TABLE constraint_test.constants_repl (a Int64, b Int64, c Int64, d Int64, CONSTRAINT c1 ASSUME a - b = 10 AND c + d = 20) ENGINE = TinyLog; +CREATE TABLE constraint_test_constants_repl (a Int64, b Int64, c Int64, d Int64, CONSTRAINT c1 ASSUME a - b = 10 AND c + d = 20) ENGINE = TinyLog; -INSERT INTO constraint_test.constants_repl (a, b, c, d) VALUES (1, 2, 3, 4); +INSERT INTO constraint_test_constants_repl (a, b, c, d) VALUES (1, 2, 3, 4); -SELECT count() FROM constraint_test.constants_repl WHERE a - b = 10; ---> assumption -> 1 -SELECT count() FROM constraint_test.constants_repl WHERE a - b < 0; ---> assumption -> 0 -SELECT count() FROM constraint_test.constants_repl WHERE a - b = c + d; ---> assumption -> 0 -SELECT count() FROM constraint_test.constants_repl WHERE (a - b) * 2 = c + d; ---> assumption -> 1 +SELECT count() FROM constraint_test_constants_repl WHERE a - b = 10; ---> assumption -> 1 +SELECT count() FROM constraint_test_constants_repl WHERE a - b < 0; ---> assumption -> 0 +SELECT count() FROM constraint_test_constants_repl WHERE a - b = c + d; ---> assumption -> 0 +SELECT count() FROM constraint_test_constants_repl WHERE (a - b) * 2 = c + d; ---> assumption -> 1 -DROP TABLE constraint_test.constants_repl; +DROP TABLE constraint_test_constants_repl; -CREATE TABLE constraint_test.constants (a Int64, b Int64, c Int64, CONSTRAINT c1 ASSUME b > 10 AND a >= 10) ENGINE = TinyLog; +CREATE TABLE constraint_test_constants (a Int64, b Int64, c Int64, CONSTRAINT c1 ASSUME b > 10 AND a >= 10) ENGINE = TinyLog; -INSERT INTO constraint_test.constants (a, b, c) VALUES (0, 0, 0); +INSERT INTO constraint_test_constants (a, b, c) VALUES (0, 0, 0); -SELECT count() FROM constraint_test.constants WHERE 9 < b; ---> assumption -> 1 -SELECT count() FROM constraint_test.constants WHERE 11 < b; ---> assumption -> 0 -SELECT count() FROM constraint_test.constants WHERE 10 <= b; ---> assumption -> 1 -SELECT count() FROM constraint_test.constants WHERE 9 < a; ---> assumption -> 1 -SELECT count() FROM constraint_test.constants WHERE 10 < a; ---> assumption -> 0 -SELECT count() FROM constraint_test.constants WHERE 9 <= a; ---> assumption -> 1 -SELECT count() FROM constraint_test.constants WHERE 11 <= a; ---> assumption -> 0 +SELECT count() FROM constraint_test_constants WHERE 9 < b; ---> assumption -> 1 +SELECT count() FROM constraint_test_constants WHERE 11 < b; ---> assumption -> 0 +SELECT count() FROM constraint_test_constants WHERE 10 <= b; ---> assumption -> 1 +SELECT count() FROM constraint_test_constants WHERE 9 < a; ---> assumption -> 1 +SELECT count() FROM constraint_test_constants WHERE 10 < a; ---> assumption -> 0 +SELECT count() FROM constraint_test_constants WHERE 9 <= a; ---> assumption -> 1 +SELECT count() FROM constraint_test_constants WHERE 11 <= a; ---> assumption -> 0 -DROP TABLE constraint_test.constants; - -DROP DATABASE constraint_test; \ No newline at end of file +DROP TABLE constraint_test_constants; diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index 4b558ecbddb..f35246a59cd 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -1,51 +1,51 @@ SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` -FROM column_swap_test.test +FROM column_swap_test_test WHERE b = 1 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` -FROM column_swap_test.test +FROM column_swap_test_test WHERE b = 0 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` -FROM column_swap_test.test +FROM column_swap_test_test WHERE b = 0 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, (b AS b) + 3 AS `plus(b, 3)` -FROM column_swap_test.test +FROM column_swap_test_test WHERE b = 1 SELECT (b AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)` -FROM column_swap_test.test +FROM column_swap_test_test WHERE b = 0 SELECT (cityHash64(a) AS `cityHash64(a)`) + 10 AS `plus(cityHash64(a), 10)`, a AS a -FROM column_swap_test.test +FROM column_swap_test_test WHERE cityHash64(a) = 0 SELECT (cityHash64(a) AS b) + 10 AS `plus(b, 10)`, a AS a -FROM column_swap_test.test +FROM column_swap_test_test WHERE cityHash64(a) = 0 SELECT a AS `substring(reverse(b), 1, 1)`, a AS a -FROM column_swap_test.test +FROM column_swap_test_test WHERE a = \'c\' SELECT a AS `substring(reverse(b), 1, 1)`, a AS a -FROM column_swap_test.test +FROM column_swap_test_test WHERE a = \'c\' SELECT a AS t1, a AS t2 -FROM column_swap_test.test +FROM column_swap_test_test WHERE a = \'c\' SELECT a AS `substring(reverse(b), 1, 1)` -FROM column_swap_test.test +FROM column_swap_test_test WHERE a = \'c\' diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.sql b/tests/queries/0_stateless/01623_constraints_column_swap.sql index a6c5c5b6bc5..ee48cfbbd3e 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.sql +++ b/tests/queries/0_stateless/01623_constraints_column_swap.sql @@ -4,34 +4,29 @@ SET optimize_move_to_prewhere = 1; SET optimize_substitute_columns = 1; SET optimize_append_index = 1; -DROP DATABASE IF EXISTS column_swap_test; -DROP TABLE IF EXISTS column_swap_test.test; +DROP TABLE IF EXISTS column_swap_test_test; -CREATE DATABASE column_swap_test; +CREATE TABLE column_swap_test_test (i Int64, a String, b UInt64, CONSTRAINT c1 ASSUME b = cityHash64(a)) ENGINE = MergeTree() ORDER BY i; +INSERT INTO column_swap_test_test VALUES (1, 'cat', 1), (2, 'dog', 2); +INSERT INTO column_swap_test_test SELECT number AS i, format('test {} kek {}', toString(number), toString(number + 10)) AS a, 1 AS b FROM system.numbers LIMIT 1000000; -CREATE TABLE column_swap_test.test (i Int64, a String, b UInt64, CONSTRAINT c1 ASSUME b = cityHash64(a)) ENGINE = MergeTree() ORDER BY i; -INSERT INTO column_swap_test.test VALUES (1, 'cat', 1), (2, 'dog', 2); -INSERT INTO column_swap_test.test SELECT number AS i, format('test {} kek {}', toString(number), toString(number + 10)) AS a, 1 AS b FROM system.numbers LIMIT 1000000; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 1; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE cityHash64(a) = 0; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 0; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test_test WHERE b = 1; -EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test.test WHERE cityHash64(a) = 1; -EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test.test WHERE cityHash64(a) = 0; -EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test.test WHERE b = 0; -EXPLAIN SYNTAX SELECT cityHash64(a) + 10, b + 3 FROM column_swap_test.test WHERE b = 1; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10 FROM column_swap_test_test WHERE cityHash64(a) = 0; +EXPLAIN SYNTAX SELECT cityHash64(a) + 10, a FROM column_swap_test_test WHERE cityHash64(a) = 0; +EXPLAIN SYNTAX SELECT b + 10, a FROM column_swap_test_test WHERE b = 0; -EXPLAIN SYNTAX SELECT cityHash64(a) + 10 FROM column_swap_test.test WHERE cityHash64(a) = 0; -EXPLAIN SYNTAX SELECT cityHash64(a) + 10, a FROM column_swap_test.test WHERE cityHash64(a) = 0; -EXPLAIN SYNTAX SELECT b + 10, a FROM column_swap_test.test WHERE b = 0; +DROP TABLE column_swap_test_test; -DROP TABLE column_swap_test.test; +CREATE TABLE column_swap_test_test (i Int64, a String, b String, CONSTRAINT c1 ASSUME a = substring(reverse(b), 1, 1)) ENGINE = MergeTree() ORDER BY i; +INSERT INTO column_swap_test_test SELECT number AS i, toString(number) AS a, format('test {} kek {}', toString(number), toString(number + 10)) b FROM system.numbers LIMIT 1000000; -CREATE TABLE column_swap_test.test (i Int64, a String, b String, CONSTRAINT c1 ASSUME a = substring(reverse(b), 1, 1)) ENGINE = MergeTree() ORDER BY i; -INSERT INTO column_swap_test.test SELECT number AS i, toString(number) AS a, format('test {} kek {}', toString(number), toString(number + 10)) b FROM system.numbers LIMIT 1000000; +EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE a = 'c'; +EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; +EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) AS t1, a AS t2 FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; +EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; -EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test.test WHERE a = 'c'; -EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1), a FROM column_swap_test.test WHERE substring(reverse(b), 1, 1) = 'c'; -EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) AS t1, a AS t2 FROM column_swap_test.test WHERE substring(reverse(b), 1, 1) = 'c'; -EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) FROM column_swap_test.test WHERE substring(reverse(b), 1, 1) = 'c'; - -DROP TABLE column_swap_test.test; - -DROP DATABASE column_swap_test; +DROP TABLE column_swap_test_test; diff --git a/tests/queries/0_stateless/01624_soft_constraints.sh b/tests/queries/0_stateless/01624_soft_constraints.sh index d27d51079ae..944a4e4234f 100755 --- a/tests/queries/0_stateless/01624_soft_constraints.sh +++ b/tests/queries/0_stateless/01624_soft_constraints.sh @@ -10,13 +10,11 @@ SETTINGS="SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET $CLICKHOUSE_CLIENT -n --query=" $SETTINGS; -DROP DATABASE IF EXISTS hypothesis_test; -DROP TABLE IF EXISTS hypothesis_test.test; -DROP TABLE IF EXISTS hypothesis_test.test2; -DROP TABLE IF EXISTS hypothesis_test.test3; +DROP TABLE IF EXISTS hypothesis_test_test; +DROP TABLE IF EXISTS hypothesis_test_test2; +DROP TABLE IF EXISTS hypothesis_test_test3; -CREATE DATABASE hypothesis_test; -CREATE TABLE hypothesis_test.test ( +CREATE TABLE hypothesis_test_test ( i UInt64, a UInt64, b UInt64, @@ -27,28 +25,28 @@ CREATE TABLE hypothesis_test.test ( " $CLICKHOUSE_CLIENT -n --query="$SETTINGS; -INSERT INTO hypothesis_test.test VALUES (0, 1, 2, 2), (1, 2, 1, 2), (2, 2, 2, 1), (3, 1, 2, 3)" +INSERT INTO hypothesis_test_test VALUES (0, 1, 2, 2), (1, 2, 1, 2), (2, 2, 2, 1), (3, 1, 2, 3)" -$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE b > a FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE b > a FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE b <= a FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE b <= a FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE b >= a FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE b >= a FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE b = a FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE b = a FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE c < a FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE c < a FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE c = a FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE c = a FORMAT JSON" | grep "rows_read" -$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE c > a FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE c > a FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test WHERE c < a FORMAT JSON" | grep "rows_read" +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test WHERE c < a FORMAT JSON" | grep "rows_read" $CLICKHOUSE_CLIENT -n --query=" $SETTINGS; -CREATE TABLE hypothesis_test.test2 ( +CREATE TABLE hypothesis_test_test2 ( i UInt64, a UInt64, b UInt64, @@ -57,20 +55,20 @@ CREATE TABLE hypothesis_test.test2 ( " $CLICKHOUSE_CLIENT -n --query="$SETTINGS; -INSERT INTO hypothesis_test.test2 VALUES (0, 1, 2), (1, 2, 1), (2, 2, 2), (3, 1, 0)" +INSERT INTO hypothesis_test_test2 VALUES (0, 1, 2), (1, 2, 1), (2, 2, 2), (3, 1, 0)" -$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test2 WHERE a < b FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test2 WHERE a < b FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test2 WHERE a <= b FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test2 WHERE a <= b FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test2 WHERE a = b FORMAT JSON" | grep "rows_read" # 1 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test2 WHERE a = b FORMAT JSON" | grep "rows_read" # 1 -$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test.test2 WHERE a != b FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS; SELECT count() FROM hypothesis_test_test2 WHERE a != b FORMAT JSON" | grep "rows_read" # 4 $CLICKHOUSE_CLIENT -n --query=" $SETTINGS; -CREATE TABLE hypothesis_test.test3 ( +CREATE TABLE hypothesis_test_test3 ( i UInt64, a UInt64, b UInt64, @@ -80,20 +78,19 @@ CREATE TABLE hypothesis_test.test3 ( $CLICKHOUSE_CLIENT -n --query=" $SETTINGS; -INSERT INTO hypothesis_test.test3 VALUES (0, 1, 2), (1, 2, 1), (2, 2, 2), (3, 1, 0)" +INSERT INTO hypothesis_test_test3 VALUES (0, 1, 2), (1, 2, 1), (2, 2, 2), (3, 1, 0)" -$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test.test3 WHERE a < b FORMAT JSON" | grep "rows_read" # 3 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test_test3 WHERE a < b FORMAT JSON" | grep "rows_read" # 3 -$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test.test3 WHERE a <= b FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test_test3 WHERE a <= b FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test.test3 WHERE a = b FORMAT JSON" | grep "rows_read" # 4 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test_test3 WHERE a = b FORMAT JSON" | grep "rows_read" # 4 -$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test.test3 WHERE a != b FORMAT JSON" | grep "rows_read" # 3 +$CLICKHOUSE_CLIENT -n --query="$SETTINGS;SELECT count() FROM hypothesis_test_test3 WHERE a != b FORMAT JSON" | grep "rows_read" # 3 $CLICKHOUSE_CLIENT -n --query=" $SETTINGS; -DROP TABLE hypothesis_test.test; -DROP TABLE hypothesis_test.test2; -DROP TABLE hypothesis_test.test3; -DROP DATABASE hypothesis_test;" +DROP TABLE hypothesis_test_test; +DROP TABLE hypothesis_test_test2; +DROP TABLE hypothesis_test_test3;" diff --git a/tests/queries/0_stateless/01625_constraints_index_append.reference b/tests/queries/0_stateless/01625_constraints_index_append.reference index e52b87fbd88..0df5c429d9e 100644 --- a/tests/queries/0_stateless/01625_constraints_index_append.reference +++ b/tests/queries/0_stateless/01625_constraints_index_append.reference @@ -1,15 +1,15 @@ SELECT i AS i -FROM index_append_test.test +FROM index_append_test_test PREWHERE a = 0 WHERE (a = 0) AND indexHint((i + 40) > 0) SELECT i AS i -FROM index_append_test.test +FROM index_append_test_test PREWHERE a < 0 SELECT i AS i -FROM index_append_test.test +FROM index_append_test_test PREWHERE a >= 0 WHERE (a >= 0) AND indexHint((i + 40) > 0) SELECT i AS i -FROM index_append_test.test +FROM index_append_test_test PREWHERE (2 * b) < 100 WHERE ((2 * b) < 100) AND indexHint(i < 100) diff --git a/tests/queries/0_stateless/01625_constraints_index_append.sql b/tests/queries/0_stateless/01625_constraints_index_append.sql index 7d75c57218a..fbffc9c7f10 100644 --- a/tests/queries/0_stateless/01625_constraints_index_append.sql +++ b/tests/queries/0_stateless/01625_constraints_index_append.sql @@ -4,19 +4,14 @@ SET optimize_move_to_prewhere = 1; SET optimize_substitute_columns = 1; SET optimize_append_index = 1; -DROP DATABASE IF EXISTS index_append_test; -DROP TABLE IF EXISTS index_append_test.test; +DROP TABLE IF EXISTS index_append_test_test; -CREATE DATABASE index_append_test; +CREATE TABLE index_append_test_test (i Int64, a UInt32, b UInt64, CONSTRAINT c1 ASSUME i <= 2 * b AND i + 40 > a) ENGINE = MergeTree() ORDER BY i; +INSERT INTO index_append_test_test VALUES (1, 10, 1), (2, 20, 2); -CREATE TABLE index_append_test.test (i Int64, a UInt32, b UInt64, CONSTRAINT c1 ASSUME i <= 2 * b AND i + 40 > a) ENGINE = MergeTree() ORDER BY i; -INSERT INTO index_append_test.test VALUES (1, 10, 1), (2, 20, 2); +EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a = 0; +EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a < 0; +EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE a >= 0; +EXPLAIN SYNTAX SELECT i FROM index_append_test_test WHERE 2 * b < 100; -EXPLAIN SYNTAX SELECT i FROM index_append_test.test WHERE a = 0; -EXPLAIN SYNTAX SELECT i FROM index_append_test.test WHERE a < 0; -EXPLAIN SYNTAX SELECT i FROM index_append_test.test WHERE a >= 0; -EXPLAIN SYNTAX SELECT i FROM index_append_test.test WHERE 2 * b < 100; - -DROP TABLE index_append_test.test; - -DROP DATABASE index_append_test; \ No newline at end of file +DROP TABLE index_append_test_test; From d274d4a5793db343d6289f619bd01b4332d813fa Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Wed, 5 May 2021 00:36:30 +0300 Subject: [PATCH 044/200] fix --- src/Interpreters/TreeCNFConverter.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index fac3007f394..2824c088f15 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -98,7 +98,7 @@ void pushOr(ASTPtr & query) { auto & child = or_func->arguments->children[i]; auto * and_func = child->as(); - if (and_func->name == "and") + if (and_func && and_func->name == "and") { and_node_id = i; } From 1d269570290a493ae5a1fb9f51f154d9d0ab528b Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Wed, 5 May 2021 11:51:25 +0300 Subject: [PATCH 045/200] a AND not a --- src/Interpreters/ConstraintMatcherVisitor.cpp | 38 -------------- src/Interpreters/ConstraintMatcherVisitor.h | 24 --------- src/Interpreters/TreeCNFConverter.cpp | 50 +++++++++++++++++++ src/Interpreters/TreeCNFConverter.h | 3 ++ .../WhereConstraintsOptimizer.cpp | 6 +-- ..._constraints_simple_optimization.reference | 6 +++ .../01622_constraints_simple_optimization.sql | 4 ++ 7 files changed, 66 insertions(+), 65 deletions(-) delete mode 100644 src/Interpreters/ConstraintMatcherVisitor.cpp delete mode 100644 src/Interpreters/ConstraintMatcherVisitor.h diff --git a/src/Interpreters/ConstraintMatcherVisitor.cpp b/src/Interpreters/ConstraintMatcherVisitor.cpp deleted file mode 100644 index b3050ea9eed..00000000000 --- a/src/Interpreters/ConstraintMatcherVisitor.cpp +++ /dev/null @@ -1,38 +0,0 @@ -#include "ConstraintMatcherVisitor.h" - -#include -#include -#include -#include - -namespace DB -{ - -bool ConstraintMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &) -{ - return node->as() || node->as(); -} - -std::optional ConstraintMatcher::getASTValue(const ASTPtr & node, Data & data) -{ - const auto it = data.constraints.find(node->getTreeHash().second); - if (it != std::end(data.constraints)) - { - for (const auto & ast : it->second) - { - if (node->getColumnName() == ast->getColumnName()) - { - return true; - } - } - } - return std::nullopt; -} - -void ConstraintMatcher::visit(ASTPtr & ast, Data & data) -{ - if (const auto always_value = getASTValue(ast, data); always_value) - ast = std::make_shared(static_cast(*always_value)); -} - -} diff --git a/src/Interpreters/ConstraintMatcherVisitor.h b/src/Interpreters/ConstraintMatcherVisitor.h deleted file mode 100644 index be0abe91fec..00000000000 --- a/src/Interpreters/ConstraintMatcherVisitor.h +++ /dev/null @@ -1,24 +0,0 @@ -#pragma once - -#include - -namespace DB -{ - -struct ConstraintMatcher -{ - struct Data - { - std::unordered_map> constraints; - }; - - using Visitor = InDepthNodeVisitor; - - static bool needChildVisit(const ASTPtr & node, const ASTPtr &); - static std::optional getASTValue(const ASTPtr & node, Data & data); - static void visit(ASTPtr & ast, Data & data); -}; - -using ConstraintMatcherVisitor = InDepthNodeVisitor; - -} diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index 2824c088f15..f1b34194b9c 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -38,6 +38,7 @@ void traversePushNot(ASTPtr & node, bool add_negation) { if (add_negation) { + ASSERT(func->arguments->size() == 2) /// apply De Morgan's Law node = makeASTFunction( (func->name == "and" ? "or" : "and"), @@ -51,6 +52,7 @@ void traversePushNot(ASTPtr & node, bool add_negation) } else if (func && func->name == "not") { + ASSERT(func->arguments->size() == 1) /// delete NOT node = func->arguments->children[0]->clone(); @@ -91,6 +93,7 @@ void pushOr(ASTPtr & query) auto * or_func = or_node.get()->as(); ASSERT(or_func) ASSERT(or_func->name == "or") + ASSERT(or_func->arguments->children.size() == 2) /// find or upper than and size_t and_node_id = or_func->arguments->children.size(); @@ -110,6 +113,7 @@ void pushOr(ASTPtr & query) auto and_func = or_func->arguments->children[and_node_id]->as(); ASSERT(and_func) ASSERT(and_func->name == "and") + ASSERT(and_func->arguments->children.size() == 2) auto a = or_func->arguments->children[other_node_id]; auto b = and_func->arguments->children[0]; @@ -300,6 +304,52 @@ CNFQuery & CNFQuery::pushNotInFuntions() return *this; } +namespace +{ + CNFQuery::AndGroup reduceOnce(const CNFQuery::AndGroup & groups) + { + CNFQuery::AndGroup result; + for (const CNFQuery::OrGroup & group : groups) + { + CNFQuery::OrGroup copy(group); + bool inserted = false; + for (const CNFQuery::AtomicFormula & atom : group) + { + copy.erase(atom); + CNFQuery::AtomicFormula negative_atom(atom); + negative_atom.negative = !atom.negative; + copy.insert(negative_atom); + + if (groups.contains(copy)) + { + copy.erase(negative_atom); + result.insert(copy); + inserted = true; + break; + } + + copy.erase(negative_atom); + copy.insert(atom); + } + if (!inserted) + result.insert(group); + } + return result; + } +} + +CNFQuery & CNFQuery::reduce() +{ + while (true) + { + AndGroup new_statements = reduceOnce(statements); + if (statements == new_statements) + return *this; + else + statements = new_statements; + } +} + std::string CNFQuery::dump() const { WriteBufferFromOwnString res; diff --git a/src/Interpreters/TreeCNFConverter.h b/src/Interpreters/TreeCNFConverter.h index af5659af678..7dfb2e511ba 100644 --- a/src/Interpreters/TreeCNFConverter.h +++ b/src/Interpreters/TreeCNFConverter.h @@ -135,6 +135,9 @@ public: /// Revert pullNotOutFunctions actions CNFQuery & pushNotInFuntions(); + /// (a OR b OR ...) AND (NOT a OR b OR ...) -> (b OR ...) + CNFQuery & reduce(); + private: AndGroup statements; }; diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index 13e09cd844c..fc635544b1d 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -36,8 +36,7 @@ MatchState match(CNFQuery::AtomicFormula a, CNFQuery::AtomicFormula b) { bool match_means_ok = true ^ a.negative ^ b.negative; - if (a.ast->getTreeHash() == b.ast->getTreeHash() && - a.ast->getColumnName() == b.ast->getColumnName()) + if (a.ast->getTreeHash() == b.ast->getTreeHash()) { return match_means_ok ? MatchState::FULL_MATCH : MatchState::NOT_MATCH; } @@ -109,7 +108,7 @@ bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const for (const auto & constraint : constraints) { if (constraint.size() > 1) - continue; /// TMP + continue; for (const auto & constraint_atoms : constraint) { @@ -185,6 +184,7 @@ void WhereConstraintsOptimizer::perform() { return replaceTermsToConstants(atom, compare_graph); }) + .reduce() .pushNotInFuntions(); if (optimize_append_index) diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index 0e65f1e557d..8fb281e43db 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -31,3 +31,9 @@ 0 1 0 +SELECT count() +FROM constraint_test_constants +WHERE (c > 100) OR (b > 100) +SELECT count() +FROM constraint_test_constants +WHERE c > 100 diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index 2fd9e883b1f..7eed0c660f2 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -95,4 +95,8 @@ SELECT count() FROM constraint_test_constants WHERE 10 < a; ---> assumption -> 0 SELECT count() FROM constraint_test_constants WHERE 9 <= a; ---> assumption -> 1 SELECT count() FROM constraint_test_constants WHERE 11 <= a; ---> assumption -> 0 +-- A AND NOT A +EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100); +EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100); + DROP TABLE constraint_test_constants; From cfdd84b3fcd2adf9a8bb251aa98b4a14367017b6 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Wed, 5 May 2021 14:17:49 +0300 Subject: [PATCH 046/200] improve matching --- .../WhereConstraintsOptimizer.cpp | 54 +++++++------------ src/Storages/ConstraintsDescription.cpp | 26 ++++++++- src/Storages/ConstraintsDescription.h | 16 +++++- 3 files changed, 58 insertions(+), 38 deletions(-) diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index fc635544b1d..578c895bdbb 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -43,33 +43,20 @@ MatchState match(CNFQuery::AtomicFormula a, CNFQuery::AtomicFormula b) return MatchState::NONE; } -bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const std::vector> & constraints) +bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const ConstraintsDescription & constraints_description) { - for (const auto & constraint : constraints) + for (const auto & atom : group) { - bool group_always_true = true; - for (const auto & constraint_ast : constraint) + const auto constraint_atom_ids = constraints_description.getAtomIds(atom.ast); + if (constraint_atom_ids) { - bool found_match = false; - for (const auto & atom_ast : group) + for (const auto & constraint_atom : constraints_description.getAtomsById(constraint_atom_ids.value())) { - const auto match_result = match(constraint_ast, atom_ast); - + const auto match_result = match(constraint_atom, atom); if (match_result == MatchState::FULL_MATCH) - { - found_match = true; - break; - } - } - if (!found_match) - { - group_always_true = false; - break; + return true; } } - - if (group_always_true) - return true; } return false; } @@ -103,19 +90,16 @@ bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const Comparis } -bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const std::vector> & constraints) +bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const ConstraintsDescription & constraints_description) { - for (const auto & constraint : constraints) + const auto constraint_atom_ids = constraints_description.getAtomIds(atom.ast); + if (constraint_atom_ids) { - if (constraint.size() > 1) - continue; - - for (const auto & constraint_atoms : constraint) + for (const auto & constraint_atom : constraints_description.getAtomsById(constraint_atom_ids.value())) { - const auto match_result = match(constraint_atoms, atom); - - if (match_result != MatchState::NONE) - return match_result == MatchState::NOT_MATCH; + const auto match_result = match(constraint_atom, atom); + if (match_result == MatchState::NOT_MATCH) + return true; } } @@ -164,21 +148,21 @@ void WhereConstraintsOptimizer::perform() { if (select_query->where() && metadata_snapshot) { - const auto & constraint_data = metadata_snapshot->getConstraints().getConstraintData(); + //const auto & constraint_data = metadata_snapshot->getConstraints().getConstraintData(); const auto & compare_graph = metadata_snapshot->getConstraints().getGraph(); Poco::Logger::get("BEFORE CNF ").information(select_query->where()->dumpTree()); auto cnf = TreeCNFConverter::toCNF(select_query->where()); Poco::Logger::get("BEFORE OPT").information(cnf.dump()); cnf.pullNotOutFunctions() - .filterAlwaysTrueGroups([&constraint_data, &compare_graph](const auto & group) + .filterAlwaysTrueGroups([&compare_graph, this](const auto & group) { /// remove always true groups from CNF - return !checkIfGroupAlwaysTrueFullMatch(group, constraint_data) && !checkIfGroupAlwaysTrueGraph(group, compare_graph); + return !checkIfGroupAlwaysTrueFullMatch(group, metadata_snapshot->getConstraints()) && !checkIfGroupAlwaysTrueGraph(group, compare_graph); }) - .filterAlwaysFalseAtoms([&constraint_data, &compare_graph](const auto & atom) + .filterAlwaysFalseAtoms([&compare_graph, this](const auto & atom) { /// remove always false atoms from CNF - return !checkIfAtomAlwaysFalseFullMatch(atom, constraint_data) && !checkIfAtomAlwaysFalseGraph(atom, compare_graph); + return !checkIfAtomAlwaysFalseFullMatch(atom, metadata_snapshot->getConstraints()) && !checkIfAtomAlwaysFalseGraph(atom, compare_graph); }) .transformAtoms([&compare_graph](const auto & atom) { diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index 042d7a344e5..e14a325a0bc 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -89,7 +89,7 @@ std::vector ConstraintsDescription::getAtomicConstraint { Poco::Logger::get("atomic_formula: initial:").information(constraint->as()->expr->ptr()->dumpTree()); const auto cnf = TreeCNFConverter::toCNF(constraint->as()->expr->ptr()) - .pullNotOutFunctions(); + .pullNotOutFunctions(); for (const auto & group : cnf.getStatements()) { if (group.size() == 1) @@ -158,6 +158,22 @@ const std::vector & ConstraintsDescription::getConstraints() const return constraints; } +std::optional ConstraintsDescription::getAtomIds(const ASTPtr & ast) const +{ + const auto hash = ast->getTreeHash(); + if (ast_to_atom_ids.contains(hash)) + return ast_to_atom_ids.at(hash); + return std::nullopt; +} + +std::vector ConstraintsDescription::getAtomsById(const ConstraintsDescription::AtomIds & ids) const +{ + std::vector result; + for (const auto & id : ids) + result.push_back(cnf_constraints[id.and_group][id.atom]); + return result; +} + void ConstraintsDescription::updateConstraints(const std::vector & constraints_) { constraints = constraints_; @@ -184,6 +200,14 @@ ConstraintsDescription & ConstraintsDescription::operator=(const ConstraintsDesc void ConstraintsDescription::update() { cnf_constraints = buildConstraintData(); + ast_to_atom_ids.clear(); + for (size_t i = 0; i < cnf_constraints.size(); ++i) + { + for (size_t j = 0; j < cnf_constraints[i].size(); ++j) + { + ast_to_atom_ids[cnf_constraints[i][j].ast->getTreeHash()].push_back({i, j}); + } + } graph = buildGraph(); } diff --git a/src/Storages/ConstraintsDescription.h b/src/Storages/ConstraintsDescription.h index 00b5f6b6e4b..d08b26fc7c2 100644 --- a/src/Storages/ConstraintsDescription.h +++ b/src/Storages/ConstraintsDescription.h @@ -15,6 +15,9 @@ struct ConstraintsDescription public: ConstraintsDescription() { update(); } + ConstraintsDescription(const ConstraintsDescription & other); + ConstraintsDescription & operator=(const ConstraintsDescription & other); + bool empty() const { return constraints.empty(); } String toString() const; @@ -40,8 +43,16 @@ public: ConstraintsExpressions getExpressions(ContextPtr context, const NamesAndTypesList & source_columns_) const; - ConstraintsDescription(const ConstraintsDescription & other); - ConstraintsDescription & operator=(const ConstraintsDescription & other); + struct AtomId + { + size_t and_group; + size_t atom; + }; + + using AtomIds = std::vector; + + std::optional getAtomIds(const ASTPtr & ast) const; + std::vector getAtomsById(const AtomIds & ids) const; private: std::vector> buildConstraintData() const; @@ -50,6 +61,7 @@ private: std::vector constraints; std::vector> cnf_constraints; + std::map ast_to_atom_ids; std::unique_ptr graph; }; From 8f0e4f43bdc38bf977b0782d7036e181b143335e Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Wed, 5 May 2021 14:28:02 +0300 Subject: [PATCH 047/200] remove subsets --- src/Interpreters/TreeCNFConverter.cpp | 35 +++++++++++++++++++ ..._constraints_simple_optimization.reference | 5 +++ .../01622_constraints_simple_optimization.sql | 2 ++ 3 files changed, 42 insertions(+) diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index f1b34194b9c..749e1cc54f3 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -336,6 +336,38 @@ namespace } return result; } + + bool isSubset(const CNFQuery::OrGroup & left, const CNFQuery::OrGroup & right) + { + if (left.size() > right.size()) + return false; + for (const auto & elem : left) + if (!right.contains(elem)) + return false; + return true; + } + + CNFQuery::AndGroup filterSubsets(const CNFQuery::AndGroup & groups) + { + CNFQuery::AndGroup result; + for (const CNFQuery::OrGroup & group : groups) + { + bool insert = true; + + for (const CNFQuery::OrGroup & other_group : groups) + { + if (isSubset(other_group, group) && group != other_group) + { + insert = false; + break; + } + } + + if (insert) + result.insert(group); + } + return result; + } } CNFQuery & CNFQuery::reduce() @@ -344,7 +376,10 @@ CNFQuery & CNFQuery::reduce() { AndGroup new_statements = reduceOnce(statements); if (statements == new_statements) + { + statements = filterSubsets(statements); return *this; + } else statements = new_statements; } diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index 8fb281e43db..ec997c2386a 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -37,3 +37,8 @@ WHERE (c > 100) OR (b > 100) SELECT count() FROM constraint_test_constants WHERE c > 100 +SELECT count() +FROM constraint_test_constants +WHERE c > 100 +SELECT count() +FROM constraint_test_constants diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index 7eed0c660f2..20ad26049a5 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -98,5 +98,7 @@ SELECT count() FROM constraint_test_constants WHERE 11 <= a; ---> assumption -> -- A AND NOT A EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100); EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100); +EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c > 100); +EXPLAIN SYNTAX SELECT count() FROM constraint_test_constants WHERE (a > 100 OR b > 100 OR c > 100) AND (a <= 100 OR b > 100 OR c > 100) AND (NOT b > 100 OR c > 100) AND (c <= 100); DROP TABLE constraint_test_constants; From 37800a10579e7cf5bda98f3f2c0a60a04dea19ae Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Wed, 5 May 2021 16:29:28 +0300 Subject: [PATCH 048/200] fix --- src/Interpreters/TreeCNFConverter.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index 749e1cc54f3..9e0e2e795a3 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -39,6 +39,8 @@ void traversePushNot(ASTPtr & node, bool add_negation) if (add_negation) { ASSERT(func->arguments->size() == 2) + if (func->arguments->children.size() != 2) + throw Exception("Bad AND or OR function.", ErrorCodes::LOGICAL_ERROR); /// apply De Morgan's Law node = makeASTFunction( (func->name == "and" ? "or" : "and"), @@ -53,6 +55,8 @@ void traversePushNot(ASTPtr & node, bool add_negation) else if (func && func->name == "not") { ASSERT(func->arguments->size() == 1) + if (func->arguments->children.size() != 1) + throw Exception("Bad NOT function.", ErrorCodes::LOGICAL_ERROR); /// delete NOT node = func->arguments->children[0]->clone(); @@ -94,6 +98,8 @@ void pushOr(ASTPtr & query) ASSERT(or_func) ASSERT(or_func->name == "or") ASSERT(or_func->arguments->children.size() == 2) + if (or_func->arguments->children.size() != 2) + throw Exception("Bad OR function.", ErrorCodes::LOGICAL_ERROR); /// find or upper than and size_t and_node_id = or_func->arguments->children.size(); @@ -114,6 +120,8 @@ void pushOr(ASTPtr & query) ASSERT(and_func) ASSERT(and_func->name == "and") ASSERT(and_func->arguments->children.size() == 2) + if (and_func->arguments->children.size() != 2) + throw Exception("Bad AND function.", ErrorCodes::LOGICAL_ERROR); auto a = or_func->arguments->children[other_node_id]; auto b = and_func->arguments->children[0]; From 4bf554735094df18bcc0558ace2481171e4673a6 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 6 May 2021 11:29:24 +0300 Subject: [PATCH 049/200] fix --- src/Interpreters/ComparisonGraph.cpp | 28 +++++++++---------- src/Interpreters/ComparisonGraph.h | 14 +++++----- src/Interpreters/TreeCNFConverter.cpp | 14 ++++------ src/Storages/ConstraintsDescription.cpp | 17 +++++++---- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.h | 2 +- .../MergeTree/MergeTreeIndexHypothesis.cpp | 1 - .../MergeTreeIndexMergedCondition.cpp | 18 +++++++----- .../MergeTree/MergeTreeWhereOptimizer.cpp | 2 +- .../MergeTree/SubstituteColumnOptimizer.cpp | 6 +++- 10 files changed, 56 insertions(+), 48 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 83a887ce2bc..4f0f6b919a8 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -12,7 +12,7 @@ namespace DB { /// make function a < b or a <= b -ASTPtr ComparisonGraph::normalizeAtom(const ASTPtr & atom) const +ASTPtr ComparisonGraph::normalizeAtom(const ASTPtr & atom) { static const std::map inverse_relations = { {"greaterOrEquals", "lessOrEquals"}, @@ -46,7 +46,7 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) Graph g; for (const auto & atom_raw : atomic_formulas) { - const auto atom = normalizeAtom(atom_raw); + const auto atom = ComparisonGraph::normalizeAtom(atom_raw); const auto bad_term = std::numeric_limits::max(); auto get_index = [](const ASTPtr & ast, Graph & asts_graph) -> std::size_t @@ -103,12 +103,12 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) } } - graph = BuildGraphFromAstsGraph(g); - dists = BuildDistsFromGraph(graph); + graph = ComparisonGraph::BuildGraphFromAstsGraph(g); + dists = ComparisonGraph::BuildDistsFromGraph(graph); std::tie(ast_const_lower_bound, ast_const_upper_bound) = buildConstBounds(); } -/// resturns {is less, is strict} +/// returns {is less, is strict} /// {true, true} = < /// {true, false} = =< /// {false, ...} = ? @@ -199,7 +199,7 @@ bool ComparisonGraph::isPossibleCompare(const CompareResult expected, const ASTP if (expected == CompareResult::UNKNOWN || result == CompareResult::UNKNOWN) { - Poco::Logger::get("isPossibleCompare").information("unknonw"); + Poco::Logger::get("isPossibleCompare").information("unknown"); return true; } if (expected == result) @@ -392,7 +392,7 @@ std::optional> ComparisonGraph::getConstLowerBound(const return std::make_pair(graph.vertices[to].getConstant()->as()->value, dists.at({from, to}) == Path::LESS); } -void ComparisonGraph::dfsOrder(const Graph & asts_graph, size_t v, std::vector & visited, std::vector & order) const +void ComparisonGraph::dfsOrder(const Graph & asts_graph, size_t v, std::vector & visited, std::vector & order) { visited[v] = true; for (const auto & edge : asts_graph.edges[v]) @@ -405,7 +405,7 @@ void ComparisonGraph::dfsOrder(const Graph & asts_graph, size_t v, std::vector ComparisonGraph::getVertices() const } void ComparisonGraph::dfsComponents( - const Graph & reversed_graph, size_t v, std::vector & components, const size_t not_visited, const size_t component) const + const Graph & reversed_graph, size_t v, std::vector & components, const size_t not_visited, const size_t component) { components[v] = component; for (const auto & edge : reversed_graph.edges[v]) @@ -446,7 +446,7 @@ void ComparisonGraph::dfsComponents( } } -ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & asts_graph) const +ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & asts_graph) { Poco::Logger::get("Graph").information("building"); /// Find strongly connected component @@ -458,7 +458,7 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as for (size_t v = 0; v < n; ++v) { if (!visited[v]) - dfsOrder(asts_graph, v, visited, order); + ComparisonGraph::dfsOrder(asts_graph, v, visited, order); } } @@ -471,7 +471,7 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as { if (components[v] == not_visited) { - dfsComponents(reversed_graph, v, components, not_visited, component); + ComparisonGraph::dfsComponents(reversed_graph, v, components, not_visited, component); ++component; } } @@ -529,11 +529,11 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as return result; } -std::map, ComparisonGraph::Path> ComparisonGraph::BuildDistsFromGraph(const Graph & g) const +std::map, ComparisonGraph::Path> ComparisonGraph::BuildDistsFromGraph(const Graph & g) { // min path : < = -1, =< = 0 const auto inf = std::numeric_limits::max(); - const size_t n = graph.vertices.size(); + const size_t n = g.vertices.size(); std::vector> results(n, std::vector(n, inf)); for (size_t v = 0; v < n; ++v) { diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 55d0e3c6d27..0c832c7ac85 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -99,13 +99,13 @@ private: std::vector> edges; }; - ASTPtr normalizeAtom(const ASTPtr & atom) const; - Graph BuildGraphFromAstsGraph(const Graph & asts_graph) const; + static ASTPtr normalizeAtom(const ASTPtr & atom); + static Graph BuildGraphFromAstsGraph(const Graph & asts_graph); - Graph reverseGraph(const Graph & asts_graph) const; - void dfsOrder(const Graph & asts_graph, size_t v, std::vector & visited, std::vector & order) const; - void dfsComponents( - const Graph & reversed_graph, size_t v, std::vector & components, const size_t not_visited, const size_t component) const; + static Graph reverseGraph(const Graph & asts_graph); + static void dfsOrder(const Graph & asts_graph, size_t v, std::vector & visited, std::vector & order); + static void dfsComponents( + const Graph & reversed_graph, size_t v, std::vector & components, const size_t not_visited, const size_t component); std::pair findPath(const size_t start, const size_t finish) const; @@ -115,7 +115,7 @@ private: LESS_OR_EQUAL, }; - std::map, Path> BuildDistsFromGraph(const Graph & g) const; + static std::map, Path> BuildDistsFromGraph(const Graph & g); std::pair, std::vector> buildConstBounds() const; Graph graph; diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index 9e0e2e795a3..8c731cb5a7b 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -5,6 +5,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} /// Splits AND(a, b, c) to AND(a, AND(b, c)) for AND/OR void splitMultiLogic(ASTPtr & node) @@ -38,7 +42,6 @@ void traversePushNot(ASTPtr & node, bool add_negation) { if (add_negation) { - ASSERT(func->arguments->size() == 2) if (func->arguments->children.size() != 2) throw Exception("Bad AND or OR function.", ErrorCodes::LOGICAL_ERROR); /// apply De Morgan's Law @@ -54,7 +57,6 @@ void traversePushNot(ASTPtr & node, bool add_negation) } else if (func && func->name == "not") { - ASSERT(func->arguments->size() == 1) if (func->arguments->children.size() != 1) throw Exception("Bad NOT function.", ErrorCodes::LOGICAL_ERROR); /// delete NOT @@ -95,9 +97,6 @@ void pushOr(ASTPtr & query) ors.pop_back(); auto * or_func = or_node.get()->as(); - ASSERT(or_func) - ASSERT(or_func->name == "or") - ASSERT(or_func->arguments->children.size() == 2) if (or_func->arguments->children.size() != 2) throw Exception("Bad OR function.", ErrorCodes::LOGICAL_ERROR); @@ -116,10 +115,7 @@ void pushOr(ASTPtr & query) continue; const size_t other_node_id = 1 - and_node_id; - auto and_func = or_func->arguments->children[and_node_id]->as(); - ASSERT(and_func) - ASSERT(and_func->name == "and") - ASSERT(and_func->arguments->children.size() == 2) + const auto * and_func = or_func->arguments->children[and_node_id]->as(); if (and_func->arguments->children.size() != 2) throw Exception("Bad AND function.", ErrorCodes::LOGICAL_ERROR); diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index e14a325a0bc..7fc5d8fc1cf 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -14,6 +14,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} String ConstraintsDescription::toString() const { @@ -107,17 +111,18 @@ std::unique_ptr ConstraintsDescription::buildGraph() const std::vector constraints_for_graph; auto atomic_formulas = getAtomicConstraintData(); - for (auto & atomic_formula : atomic_formulas) + for (const auto & atomic_formula : atomic_formulas) { Poco::Logger::get("atomic_formula: before:").information(atomic_formula.ast->dumpTree() + " " + std::to_string(atomic_formula.negative)); - pushNotIn(atomic_formula); - auto * func = atomic_formula.ast->as(); + CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()}; + pushNotIn(atom); + auto * func = atom.ast->as(); if (func && relations.count(func->name)) { - if (atomic_formula.negative) + if (atom.negative) throw Exception(": ", ErrorCodes::LOGICAL_ERROR); - Poco::Logger::get("atomic_formula: after:").information(atomic_formula.ast->dumpTree() + " " + std::to_string(atomic_formula.negative)); - constraints_for_graph.push_back(atomic_formula.ast); + Poco::Logger::get("atomic_formula: after:").information(atom.ast->dumpTree() + " " + std::to_string(atom.negative)); + constraints_for_graph.push_back(atom.ast); } } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index a2dbc6fc47f..b25c5035a1b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -621,7 +621,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( std::atomic total_parts{0}; std::atomic parts_dropped{0}; - MergedDataSkippingIndexAndCondition(MergeTreeIndexMergedConditionPtr condition_) + explicit MergedDataSkippingIndexAndCondition(MergeTreeIndexMergedConditionPtr condition_) : condition(condition_) { } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 8ad885e4b29..713a9430376 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -127,7 +127,7 @@ private: Poco::Logger * log); static MarkRanges filterMarksUsingMergedIndex( - MergeTreeIndices index_helper, + MergeTreeIndices indices, MergeTreeIndexMergedConditionPtr condition, MergeTreeData::DataPartPtr part, const MarkRanges & ranges, diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp index 57ef75824d4..20354ab19de 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp @@ -14,7 +14,6 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int INCORRECT_QUERY; } diff --git a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp index 82e745bce1a..f2c02604292 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp @@ -59,7 +59,8 @@ void MergeTreeIndexMergedCondition::addIndex(const MergeTreeIndexPtr & index) if (group.size() == 1) { hypotheses_data.push_back(group); - CNFQuery::AtomicFormula atom = *group.begin(); + CNFQuery::AtomicFormula atomic_formula = *group.begin(); + CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()}; pushNotIn(atom); if (atom.negative) throw Exception("negative atom", ErrorCodes::LOGICAL_ERROR); @@ -76,8 +77,9 @@ void MergeTreeIndexMergedCondition::addIndex(const MergeTreeIndexPtr & index) void MergeTreeIndexMergedCondition::addConstraints(const ConstraintsDescription & constraints_description) { auto atomic_constraints_data = constraints_description.getAtomicConstraintData(); - for (auto & atom : atomic_constraints_data) + for (const auto & atomic_formula : atomic_constraints_data) { + CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()}; pushNotIn(atom); atomic_constraints.push_back(atom.ast); } @@ -105,12 +107,12 @@ ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula bool MergeTreeIndexMergedCondition::alwaysUnknownOrTrue() const { std::vector active_atomic_formulas(atomic_constraints); - for (size_t i = 0; i < index_to_compare_atomic_hypotheses.size(); ++i) + for (const auto & hypothesis : index_to_compare_atomic_hypotheses) { active_atomic_formulas.insert( std::end(active_atomic_formulas), - std::begin(index_to_compare_atomic_hypotheses[i]), - std::end(index_to_compare_atomic_hypotheses[i])); + std::begin(hypothesis), + std::end(hypothesis)); } /// transform active formulas @@ -130,8 +132,9 @@ bool MergeTreeIndexMergedCondition::alwaysUnknownOrTrue() const expression_cnf->iterateGroups( [&](const CNFQuery::OrGroup & or_group) { - for (auto atom : or_group) + for (const auto & atomic_formula : or_group) { + CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()}; pushNotIn(atom); const auto * func = atom.ast->as(); if (func && func->arguments->children.size() == 2) @@ -168,8 +171,9 @@ bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranu if (always_false) return; - for (auto atom : or_group) + for (const auto & atomic_formula : or_group) { + CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()}; pushNotIn(atom); Poco::Logger::get("KEK").information(atom.ast->dumpTree()); const auto * func = atom.ast->as(); diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 0734ec33f67..3b45faab77b 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -192,7 +192,7 @@ void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const Poco::Logger::get("MTPRWHERE WHERE").information(select.where()->getColumnName()); Poco::Logger::get("MTPRWHERE WHERE").information(select.where()->dumpTree()); } - if(select.prewhere()) + if (select.prewhere()) { Poco::Logger::get("MTPRWHERE PRE").information(select.prewhere()->dumpTree()); Poco::Logger::get("MTPRWHERE PRE").information(select.prewhere()->getColumnName()); diff --git a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp index 13535360fd8..d92b68564bc 100644 --- a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp +++ b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp @@ -13,6 +13,10 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} namespace { @@ -140,7 +144,7 @@ public: const auto * identifier = ast->as(); if (identifier && data.name_to_component_id.contains(identifier->name())) { - const auto name = identifier->name(); + const auto & name = identifier->name(); const auto component_id = data.name_to_component_id.at(name); ast = data.id_to_expression_map.at(component_id)->clone(); if (data.is_select) From 3225c7186c96fb1657f57bc7acbe03fb92066c04 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 6 May 2021 23:36:18 +0300 Subject: [PATCH 050/200] fix --- src/Storages/MergeTree/SubstituteColumnOptimizer.cpp | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp index d92b68564bc..83a52daf3e4 100644 --- a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp +++ b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp @@ -144,11 +144,12 @@ public: const auto * identifier = ast->as(); if (identifier && data.name_to_component_id.contains(identifier->name())) { - const auto & name = identifier->name(); + const String & name = identifier->name(); const auto component_id = data.name_to_component_id.at(name); - ast = data.id_to_expression_map.at(component_id)->clone(); + auto new_ast = data.id_to_expression_map.at(component_id)->clone(); if (data.is_select) - ast->setAlias(data.old_name.at(name)); + new_ast->setAlias(data.old_name.at(name)); + ast = new_ast; } } From cdf78828da6196dead7baf8269d7d60bad15fb19 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Fri, 7 May 2021 13:49:05 +0300 Subject: [PATCH 051/200] fix --- src/Interpreters/ComparisonGraph.cpp | 2 ++ src/Interpreters/TreeCNFConverter.cpp | 12 +++++++++--- src/Storages/ConstraintsDescription.cpp | 7 +++++++ src/Storages/MergeTree/SubstituteColumnOptimizer.cpp | 2 +- 4 files changed, 19 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 4f0f6b919a8..401abf97531 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -36,6 +36,8 @@ ASTPtr ComparisonGraph::normalizeAtom(const ASTPtr & atom) ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) { + if (atomic_formulas.empty()) + return; static const std::unordered_map relation_to_enum = { {"equals", Edge::Type::EQUAL}, diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index 8c731cb5a7b..aefedacb741 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -8,6 +8,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int INCORRECT_QUERY; } /// Splits AND(a, b, c) to AND(a, AND(b, c)) for AND/OR @@ -17,6 +18,9 @@ void splitMultiLogic(ASTPtr & node) if (func && (func->name == "and" || func->name == "or")) { + if (func->arguments->children.size() < 2) + throw Exception("Bad logical function", ErrorCodes::INCORRECT_QUERY); + if (func->arguments->children.size() > 2) { ASTPtr res = func->arguments->children.front()->clone(); @@ -58,7 +62,7 @@ void traversePushNot(ASTPtr & node, bool add_negation) else if (func && func->name == "not") { if (func->arguments->children.size() != 1) - throw Exception("Bad NOT function.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Bad NOT function.", ErrorCodes::INCORRECT_QUERY); /// delete NOT node = func->arguments->children[0]->clone(); @@ -98,7 +102,7 @@ void pushOr(ASTPtr & query) auto * or_func = or_node.get()->as(); if (or_func->arguments->children.size() != 2) - throw Exception("Bad OR function.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Bad OR function", ErrorCodes::LOGICAL_ERROR); /// find or upper than and size_t and_node_id = or_func->arguments->children.size(); @@ -117,7 +121,7 @@ void pushOr(ASTPtr & query) const auto * and_func = or_func->arguments->children[and_node_id]->as(); if (and_func->arguments->children.size() != 2) - throw Exception("Bad AND function.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Bad AND function", ErrorCodes::LOGICAL_ERROR); auto a = or_func->arguments->children[other_node_id]; auto b = and_func->arguments->children[0]; @@ -159,6 +163,8 @@ void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & and_group, CNFQuery:: } else if (func && func->name == "not") { + if (func->arguments->children.size() != 1) + throw Exception("Bad NOT function", ErrorCodes::INCORRECT_QUERY); or_group.insert(CNFQuery::AtomicFormula{true, func->arguments->children.front()}); } else diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index 7fc5d8fc1cf..b06896a4ff0 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -204,6 +204,13 @@ ConstraintsDescription & ConstraintsDescription::operator=(const ConstraintsDesc void ConstraintsDescription::update() { + if (constraints.empty()) + { + cnf_constraints.clear(); + ast_to_atom_ids.clear(); + graph = std::make_unique(std::vector()); + return; + } cnf_constraints = buildConstraintData(); ast_to_atom_ids.clear(); for (size_t i = 0; i < cnf_constraints.size(); ++i) diff --git a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp index 83a52daf3e4..5c38b268098 100644 --- a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp +++ b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp @@ -244,7 +244,7 @@ void SubstituteColumnOptimizer::perform() return; } - const auto compare_graph = metadata_snapshot->getConstraints().getGraph(); + const auto & compare_graph = metadata_snapshot->getConstraints().getGraph(); // Fill aliases if (select_query->select()) From 8e4dc3aa253954b38d5e334168a5e7af65933013 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Sat, 8 May 2021 12:19:18 +0300 Subject: [PATCH 052/200] fix --- src/Interpreters/ComparisonGraph.cpp | 27 ++++++++++++------- src/Interpreters/TreeCNFConverter.cpp | 7 ++++- .../WhereConstraintsOptimizer.cpp | 16 ++++++++--- .../MergeTree/SubstituteColumnOptimizer.cpp | 26 +++++++++++++----- ..._constraints_simple_optimization.reference | 2 +- .../01622_constraints_simple_optimization.sql | 2 +- 6 files changed, 57 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 401abf97531..42ab659c649 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -38,8 +38,7 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) { if (atomic_formulas.empty()) return; - static const std::unordered_map relation_to_enum = - { + static const std::unordered_map relation_to_enum = { {"equals", Edge::Type::EQUAL}, {"less", Edge::Type::LESS}, {"lessOrEquals", Edge::Type::LESS_OR_EQUAL}, @@ -51,16 +50,14 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) const auto atom = ComparisonGraph::normalizeAtom(atom_raw); const auto bad_term = std::numeric_limits::max(); - auto get_index = [](const ASTPtr & ast, Graph & asts_graph) -> std::size_t - { + auto get_index = [](const ASTPtr & ast, Graph & asts_graph) -> std::size_t { const auto it = asts_graph.ast_hash_to_component.find(ast->getTreeHash()); if (it != std::end(asts_graph.ast_hash_to_component)) { if (!std::any_of( std::cbegin(asts_graph.vertices[it->second].asts), std::cend(asts_graph.vertices[it->second].asts), - [ast](const ASTPtr & constraint_ast) - { + [ast](const ASTPtr & constraint_ast) { return constraint_ast->getTreeHash() == ast->getTreeHash() && constraint_ast->getColumnName() == ast->getColumnName(); })) @@ -80,7 +77,7 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) }; const auto * func = atom->as(); - if (func) + if (func && func->arguments->children.size() == 2) { const size_t index_left = get_index(func->arguments->children[0], g); const size_t index_right = get_index(func->arguments->children[1], g); @@ -98,8 +95,7 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) } else if (func->name == "notEquals") { - not_equal.emplace(index_left, index_right); - not_equal.emplace(index_right, index_left); + /// Do nothing. } } } @@ -108,8 +104,19 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) graph = ComparisonGraph::BuildGraphFromAstsGraph(g); dists = ComparisonGraph::BuildDistsFromGraph(graph); std::tie(ast_const_lower_bound, ast_const_upper_bound) = buildConstBounds(); -} + for (const auto & atom_raw : atomic_formulas) + { + const auto atom = ComparisonGraph::normalizeAtom(atom_raw); + if (const auto * func = atom->as(); func) + { + auto index_left = graph.ast_hash_to_component.at(func->arguments->children[0]->getTreeHash()); + auto index_right = graph.ast_hash_to_component.at(func->arguments->children[1]->getTreeHash()); + not_equal.emplace(index_left, index_right); + not_equal.emplace(index_right, index_left); + } + } +} /// returns {is less, is strict} /// {true, true} = < /// {true, false} = =< diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index aefedacb741..b41939789a6 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -23,7 +23,7 @@ void splitMultiLogic(ASTPtr & node) if (func->arguments->children.size() > 2) { - ASTPtr res = func->arguments->children.front()->clone(); + ASTPtr res = func->arguments->children[0]->clone(); for (size_t i = 1; i < func->arguments->children.size(); ++i) { res = makeASTFunction(func->name, res, func->arguments->children[i]->clone()); @@ -32,6 +32,7 @@ void splitMultiLogic(ASTPtr & node) } auto * new_func = node->as(); + Poco::Logger::get("new_func_children").information(std::to_string(new_func->arguments->children.size())); for (auto & child : new_func->arguments->children) splitMultiLogic(child); } @@ -121,7 +122,11 @@ void pushOr(ASTPtr & query) const auto * and_func = or_func->arguments->children[and_node_id]->as(); if (and_func->arguments->children.size() != 2) + { + Poco::Logger::get("$$$$$").information(and_func->name); + Poco::Logger::get("CHILDREN: ").information(std::to_string(and_func->arguments->children.size())); throw Exception("Bad AND function", ErrorCodes::LOGICAL_ERROR); + } auto a = or_func->arguments->children[other_node_id]; auto b = and_func->arguments->children[0]; diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index 578c895bdbb..5b0182effb1 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -45,16 +45,24 @@ MatchState match(CNFQuery::AtomicFormula a, CNFQuery::AtomicFormula b) bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const ConstraintsDescription & constraints_description) { + const auto & constraints_data = constraints_description.getConstraintData(); + std::vector found(constraints_data.size(), 0); + for (size_t i = 0; i < constraints_data.size(); ++i) + found[i] = constraints_data[i].size(); + for (const auto & atom : group) { const auto constraint_atom_ids = constraints_description.getAtomIds(atom.ast); if (constraint_atom_ids) { - for (const auto & constraint_atom : constraints_description.getAtomsById(constraint_atom_ids.value())) + const auto constraint_atoms = constraints_description.getAtomsById(constraint_atom_ids.value()); + for (size_t i = 0; i < constraint_atoms.size(); ++i) { - const auto match_result = match(constraint_atom, atom); - if (match_result == MatchState::FULL_MATCH) - return true; + if (match(constraint_atoms[i], atom) == MatchState::FULL_MATCH) + { + if ((--found[(*constraint_atom_ids)[i].and_group]) == 0) + return true; + } } } } diff --git a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp index 5c38b268098..415d67d8b78 100644 --- a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp +++ b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp @@ -24,6 +24,7 @@ namespace const String COMPONENT = "__aorLwT30aH_comp"; const String COMPONENT_SEPARATOR = "_"; constexpr UInt64 COLUMN_PENALTY = 10 * 1024 * 1024; +constexpr Int64 INDEX_PRICE = -1'000'000'000'000'000'000; class ComponentMatcher { @@ -88,10 +89,10 @@ void collectIdentifiers(const ASTPtr & ast, std::unordered_set & identif struct ColumnPrice { - size_t compressed_size; - size_t uncompressed_size; + Int64 compressed_size; + Int64 uncompressed_size; - ColumnPrice(const size_t compressed_size_, const size_t uncompressed_size_) + ColumnPrice(const Int64 compressed_size_, const Int64 uncompressed_size_) : compressed_size(compressed_size_) , uncompressed_size(uncompressed_size_) {} @@ -145,10 +146,16 @@ public: if (identifier && data.name_to_component_id.contains(identifier->name())) { const String & name = identifier->name(); + //Poco::Logger::get("NAME").information(name); const auto component_id = data.name_to_component_id.at(name); + //Poco::Logger::get("COMP").information(std::to_string(component_id)); auto new_ast = data.id_to_expression_map.at(component_id)->clone(); + //Poco::Logger::get("NEW_AST").information(new_ast->dumpTree()); if (data.is_select) + { new_ast->setAlias(data.old_name.at(name)); + //Poco::Logger::get("OLD").information(data.old_name.at(name)); + } ast = new_ast; } } @@ -188,6 +195,7 @@ void bruteforce( { min_price = current_price; min_expressions = expressions_stack; + //Poco::Logger::get("PRICE").information("UPDATE"); } } else @@ -290,7 +298,7 @@ void SubstituteColumnOptimizer::perform() column_prices[column_name] = ColumnPrice( column_size.data_compressed + COLUMN_PENALTY, column_size.data_uncompressed); for (const auto & column_name : primary_key) - column_prices[column_name] = ColumnPrice(0, 0); + column_prices[column_name] = ColumnPrice(INDEX_PRICE, INDEX_PRICE); for (const auto & column_name : identifiers) column_prices[column_name] = ColumnPrice(0, 0); @@ -303,7 +311,7 @@ void SubstituteColumnOptimizer::perform() components_list.push_back(component); std::vector expressions_stack; - ColumnPrice min_price(std::numeric_limits::max(), std::numeric_limits::max()); + ColumnPrice min_price(std::numeric_limits::max(), std::numeric_limits::max()); std::vector min_expressions; bruteforce(compare_graph, components_list, @@ -314,9 +322,15 @@ void SubstituteColumnOptimizer::perform() min_price, min_expressions); - for (size_t i = 0; i < min_expressions.size(); ++i) + for (size_t i = 0; i < components_list.size(); ++i) id_to_expression_map[components_list[i]] = min_expressions[i]; + /*Poco::Logger::get("comp list").information("CL"); + for (const auto id : components_list) + Poco::Logger::get("comp list").information(std::to_string(id)); + for (const auto & [k, v] : id_to_expression_map) + Poco::Logger::get("id2expr").information(std::to_string(k) + " " + v->dumpTree());*/ + auto process = [&](ASTPtr & ast, bool is_select) { SubstituteColumnVisitor::Data substitute_data{id_to_expression_map, name_to_component, old_name, is_select}; diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index ec997c2386a..133771a5d47 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -28,7 +28,7 @@ 0 1 1 -0 +1 1 0 SELECT count() diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index 20ad26049a5..d4ef006e668 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -91,7 +91,7 @@ SELECT count() FROM constraint_test_constants WHERE 9 < b; ---> assumption -> 1 SELECT count() FROM constraint_test_constants WHERE 11 < b; ---> assumption -> 0 SELECT count() FROM constraint_test_constants WHERE 10 <= b; ---> assumption -> 1 SELECT count() FROM constraint_test_constants WHERE 9 < a; ---> assumption -> 1 -SELECT count() FROM constraint_test_constants WHERE 10 < a; ---> assumption -> 0 +SELECT count() FROM constraint_test_constants WHERE 10 < a; ---> assumption -> 1 SELECT count() FROM constraint_test_constants WHERE 9 <= a; ---> assumption -> 1 SELECT count() FROM constraint_test_constants WHERE 11 <= a; ---> assumption -> 0 From 24499fc31ac803fb30bd367535eb6b1c8d0051ee Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Mon, 10 May 2021 13:27:47 +0300 Subject: [PATCH 053/200] fix --- src/Interpreters/ComparisonGraph.cpp | 9 ++++++--- src/Interpreters/TreeCNFConverter.cpp | 2 +- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 42ab659c649..8a316a4109c 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -38,7 +38,8 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) { if (atomic_formulas.empty()) return; - static const std::unordered_map relation_to_enum = { + static const std::unordered_map relation_to_enum = + { {"equals", Edge::Type::EQUAL}, {"less", Edge::Type::LESS}, {"lessOrEquals", Edge::Type::LESS_OR_EQUAL}, @@ -50,14 +51,16 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) const auto atom = ComparisonGraph::normalizeAtom(atom_raw); const auto bad_term = std::numeric_limits::max(); - auto get_index = [](const ASTPtr & ast, Graph & asts_graph) -> std::size_t { + auto get_index = [](const ASTPtr & ast, Graph & asts_graph) -> std::size_t + { const auto it = asts_graph.ast_hash_to_component.find(ast->getTreeHash()); if (it != std::end(asts_graph.ast_hash_to_component)) { if (!std::any_of( std::cbegin(asts_graph.vertices[it->second].asts), std::cend(asts_graph.vertices[it->second].asts), - [ast](const ASTPtr & constraint_ast) { + [ast](const ASTPtr & constraint_ast) + { return constraint_ast->getTreeHash() == ast->getTreeHash() && constraint_ast->getColumnName() == ast->getColumnName(); })) diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index b41939789a6..649e043b15f 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -83,7 +83,7 @@ void findOrs(ASTPtr & node, std::vector> & ors) if (func && func->name == "or") ors.push_back(node); - if (func) + if (func && (func->name == "and" || func->name == "not")) { for (auto & child : func->arguments->children) findOrs(child, ors); From af55e75e8da8450b6f415a49ff60dbcf58a1422f Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Wed, 19 May 2021 22:07:27 +0300 Subject: [PATCH 054/200] fix CNF --- src/Interpreters/TreeCNFConverter.cpp | 80 ++++++++----------- .../0_stateless/01626_cnf_test.reference | 18 +++++ tests/queries/0_stateless/01626_cnf_test.sql | 14 ++++ 3 files changed, 65 insertions(+), 47 deletions(-) create mode 100644 tests/queries/0_stateless/01626_cnf_test.reference create mode 100644 tests/queries/0_stateless/01626_cnf_test.sql diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index 649e043b15f..8f7ece90d89 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -28,7 +28,7 @@ void splitMultiLogic(ASTPtr & node) { res = makeASTFunction(func->name, res, func->arguments->children[i]->clone()); } - node = std::move(res); + node = res; } auto * new_func = node->as(); @@ -36,6 +36,11 @@ void splitMultiLogic(ASTPtr & node) for (auto & child : new_func->arguments->children) splitMultiLogic(child); } + else if (func->name == "not") + { + for (auto & child : func->arguments->children) + splitMultiLogic(child); + } } /// Push NOT to leafs, remove NOT NOT ... @@ -76,72 +81,44 @@ void traversePushNot(ASTPtr & node, bool add_negation) } } -void findOrs(ASTPtr & node, std::vector> & ors) -{ +/// Push Or inside And (actually pull AND to top) +void traversePushOr(ASTPtr & node) { auto * func = node->as(); - if (func && func->name == "or") - ors.push_back(node); - - if (func && (func->name == "and" || func->name == "not")) + if (func && (func->name == "or" || func->name == "and")) { for (auto & child : func->arguments->children) - findOrs(child, ors); + traversePushOr(child); } -} -/// Push Or inside And (actually pull AND to top) -void pushOr(ASTPtr & query) -{ - std::vector> ors; - findOrs(query, ors); - - while (!ors.empty()) + if (func && func->name == "or") { - std::reference_wrapper or_node = ors.back(); - ors.pop_back(); - - auto * or_func = or_node.get()->as(); - if (or_func->arguments->children.size() != 2) - throw Exception("Bad OR function", ErrorCodes::LOGICAL_ERROR); - - /// find or upper than and - size_t and_node_id = or_func->arguments->children.size(); - for (size_t i = 0; i < or_func->arguments->children.size(); ++i) + size_t and_node_id = func->arguments->children.size(); + for (size_t i = 0; i < func->arguments->children.size(); ++i) { - auto & child = or_func->arguments->children[i]; + auto & child = func->arguments->children[i]; auto * and_func = child->as(); if (and_func && and_func->name == "and") { and_node_id = i; } } - if (and_node_id == or_func->arguments->children.size()) - continue; + if (and_node_id == func->arguments->children.size()) + return; const size_t other_node_id = 1 - and_node_id; - const auto * and_func = or_func->arguments->children[and_node_id]->as(); - if (and_func->arguments->children.size() != 2) - { - Poco::Logger::get("$$$$$").information(and_func->name); - Poco::Logger::get("CHILDREN: ").information(std::to_string(and_func->arguments->children.size())); - throw Exception("Bad AND function", ErrorCodes::LOGICAL_ERROR); - } - - auto a = or_func->arguments->children[other_node_id]; + const auto * and_func = func->arguments->children[and_node_id]->as(); + auto a = func->arguments->children[other_node_id]; auto b = and_func->arguments->children[0]; auto c = and_func->arguments->children[1]; /// apply the distributive law ( a or (b and c) -> (a or b) and (a or c) ) - or_node.get() = makeASTFunction( - "and", - makeASTFunction("or", a->clone(), b->clone()), - makeASTFunction("or", a->clone(), c->clone())); + node = makeASTFunction( + "and", + makeASTFunction("or", a->clone(), b), + makeASTFunction("or", a, c)); - /// add new ors to stack - auto * new_func = or_node.get()->as(); - for (auto & new_or : new_func->arguments->children) - ors.push_back(new_or); + traversePushOr(node); } } @@ -191,8 +168,17 @@ CNFQuery TreeCNFConverter::toCNF(const ASTPtr & query) auto cnf = query->clone(); splitMultiLogic(cnf); + Poco::Logger::get("CNF CONVERSION").information("SPLIT:" + cnf->dumpTree()); + Poco::Logger::get("CNF CONVERSION").information("SPLIT:" + cnf->getColumnName()); traversePushNot(cnf, false); - pushOr(cnf); + + Poco::Logger::get("PUSH NOT").information("SPLIT:" + cnf->dumpTree()); + Poco::Logger::get("PUSH NOT").information("SPLIT:" + cnf->getColumnName()); + + traversePushOr(cnf); + + Poco::Logger::get("PUSH OR").information("SPLIT:" + cnf->dumpTree()); + Poco::Logger::get("PUSH OR").information("SPLIT:" + cnf->getColumnName()); CNFQuery::AndGroup and_group; traverseCNF(cnf, and_group); diff --git a/tests/queries/0_stateless/01626_cnf_test.reference b/tests/queries/0_stateless/01626_cnf_test.reference new file mode 100644 index 00000000000..b8de3d3a57c --- /dev/null +++ b/tests/queries/0_stateless/01626_cnf_test.reference @@ -0,0 +1,18 @@ +SELECT i +FROM cnf_test +WHERE (i <= 2) AND (i <= 1) +SELECT i +FROM cnf_test +WHERE (i <= 2) OR (i <= 1) +SELECT i +FROM cnf_test +WHERE ((i > 4) OR (i > 1) OR (i > 6)) AND ((i > 4) OR (i > 1) OR (i > 5)) AND ((i > 4) OR (i > 6) OR (i > 2)) AND ((i > 4) OR (i > 2) OR (i > 5)) AND ((i > 1) OR (i > 6) OR (i > 3)) AND ((i > 1) OR (i > 3) OR (i > 5)) AND ((i > 6) OR (i > 2) OR (i > 3)) AND ((i > 2) OR (i > 3) OR (i > 5)) +SELECT i +FROM cnf_test +WHERE ((i <= 5) OR (i <= 2) OR (i <= 3)) AND ((i <= 5) OR (i <= 2) OR (i <= 4)) AND ((i <= 5) OR (i <= 3) OR (i <= 1)) AND ((i <= 5) OR (i <= 4) OR (i <= 1)) AND ((i <= 2) OR (i <= 3) OR (i <= 6)) AND ((i <= 2) OR (i <= 4) OR (i <= 6)) AND ((i <= 3) OR (i <= 1) OR (i <= 6)) AND ((i <= 4) OR (i <= 1) OR (i <= 6)) +SELECT i +FROM cnf_test +WHERE ((i > 4) OR (i > 1) OR (i > 6)) AND ((i > 4) OR (i > 1) OR (i > 5)) AND ((i > 4) OR (i > 6) OR (i > 2)) AND ((i > 4) OR (i > 6) OR (i > 7)) AND ((i > 4) OR (i > 2) OR (i > 5)) AND ((i > 4) OR (i > 7) OR (i > 5)) AND ((i > 1) OR (i > 8) OR (i > 6)) AND ((i > 1) OR (i > 8) OR (i > 5)) AND ((i > 1) OR (i > 6) OR (i > 3)) AND ((i > 1) OR (i > 3) OR (i > 5)) AND ((i > 8) OR (i > 6) OR (i > 2)) AND ((i > 8) OR (i > 6) OR (i > 7)) AND ((i > 8) OR (i > 2) OR (i > 5)) AND ((i > 8) OR (i > 7) OR (i > 5)) AND ((i > 6) OR (i > 2) OR (i > 3)) AND ((i > 6) OR (i > 3) OR (i > 7)) AND ((i > 2) OR (i > 3) OR (i > 5)) AND ((i > 3) OR (i > 7) OR (i > 5)) +SELECT i +FROM cnf_test +WHERE ((i > 4) OR (i > 8) OR (i > 3)) AND (i <= 5) AND ((i > 1) OR (i > 2) OR (i > 7)) AND (i <= 6) diff --git a/tests/queries/0_stateless/01626_cnf_test.sql b/tests/queries/0_stateless/01626_cnf_test.sql new file mode 100644 index 00000000000..e014441cbb3 --- /dev/null +++ b/tests/queries/0_stateless/01626_cnf_test.sql @@ -0,0 +1,14 @@ +SET convert_query_to_cnf = 1; + +CREATE TABLE cnf_test (i Int64) ENGINE = MergeTree() ORDER BY i; + +EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE NOT ((i > 1) OR (i > 2)); +EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE NOT ((i > 1) AND (i > 2)); + +EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE ((i > 1) AND (i > 2)) OR ((i > 3) AND (i > 4)) OR ((i > 5) AND (i > 6)); + +EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE NOT (((i > 1) OR (i > 2)) AND ((i > 3) OR (i > 4)) AND ((i > 5) OR (i > 6))); + +EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE ((i > 1) AND (i > 2) AND (i > 7)) OR ((i > 3) AND (i > 4) AND (i > 8)) OR ((i > 5) AND (i > 6)); + +EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE ((i > 1) OR (i > 2) OR (i > 7)) AND ((i > 3) OR (i > 4) OR (i > 8)) AND NOT ((i > 5) OR (i > 6)); From 44b996df4980059b24e5a735e8e1bac656b39aa0 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Wed, 19 May 2021 22:40:38 +0300 Subject: [PATCH 055/200] fix --- .../AddIndexConstraintsOptimizer.cpp | 2 -- src/Interpreters/ComparisonGraph.cpp | 8 +------- src/Interpreters/TreeCNFConverter.cpp | 15 ++------------- src/Interpreters/TreeOptimizer.cpp | 16 ---------------- src/Interpreters/WhereConstraintsOptimizer.cpp | 6 ++---- .../MergeTree/MergeTreeIndexMergedCondition.cpp | 9 ++++++--- .../MergeTree/MergeTreeIndexMergedCondition.h | 2 +- src/Storages/MergeTree/MergeTreeRangeReader.cpp | 8 -------- .../MergeTree/MergeTreeWhereOptimizer.cpp | 12 ------------ .../MergeTree/SubstituteColumnOptimizer.cpp | 11 ----------- 10 files changed, 12 insertions(+), 77 deletions(-) diff --git a/src/Interpreters/AddIndexConstraintsOptimizer.cpp b/src/Interpreters/AddIndexConstraintsOptimizer.cpp index 06a9e59d297..c3151ead34e 100644 --- a/src/Interpreters/AddIndexConstraintsOptimizer.cpp +++ b/src/Interpreters/AddIndexConstraintsOptimizer.cpp @@ -118,7 +118,6 @@ namespace CNFQuery::OrGroup result; for (const auto & atom : group) { - Poco::Logger::get("INDEX_HINT_CREATE").information("CHECK"); const auto * func = atom.ast->as(); if (func && func->arguments->children.size() == 2 && getRelationMap().contains(func->name)) { @@ -137,7 +136,6 @@ namespace if (canBeSequence(need_result, actual_result)) { - Poco::Logger::get("INDEX_HINT_CREATE").information(func->arguments->children[index]->getColumnName() + " " + primary_key_ast->getColumnName()); ASTPtr helper_ast = func->clone(); auto * helper_func = helper_ast->as(); helper_func->name = getReverseRelationMap().at(mostStrict(need_result, actual_result)); diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 8a316a4109c..7abb8f8f001 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -145,9 +145,6 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con const auto it_right = graph.ast_hash_to_component.find(right->getTreeHash()); if (it_left == std::end(graph.ast_hash_to_component) || it_right == std::end(graph.ast_hash_to_component)) { - Poco::Logger::get("Graph").information("not found"); - Poco::Logger::get("Graph").information(std::to_string(left->getTreeHash().second)); - Poco::Logger::get("Graph").information(std::to_string(right->getTreeHash().second)); { const auto left_bound = getConstLowerBound(left); const auto right_bound = getConstUpperBound(right); @@ -184,7 +181,6 @@ ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, con { start = it_left->second; finish = it_right->second; - Poco::Logger::get("Graph").information("found:" + std::to_string(start) + " " + std::to_string(finish)); } } @@ -211,7 +207,6 @@ bool ComparisonGraph::isPossibleCompare(const CompareResult expected, const ASTP if (expected == CompareResult::UNKNOWN || result == CompareResult::UNKNOWN) { - Poco::Logger::get("isPossibleCompare").information("unknown"); return true; } if (expected == result) @@ -460,7 +455,6 @@ void ComparisonGraph::dfsComponents( ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & asts_graph) { - Poco::Logger::get("Graph").information("building"); /// Find strongly connected component const auto n = asts_graph.vertices.size(); @@ -507,7 +501,7 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as vertex.buildConstants(); } - Poco::Logger::get("Graph").information("components: " + std::to_string(component)); + Poco::Logger::get("ComparisonGraph").information("components: " + std::to_string(component)); for (size_t v = 0; v < n; ++v) { diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index 8f7ece90d89..bdb1d5d490b 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -32,11 +32,10 @@ void splitMultiLogic(ASTPtr & node) } auto * new_func = node->as(); - Poco::Logger::get("new_func_children").information(std::to_string(new_func->arguments->children.size())); for (auto & child : new_func->arguments->children) splitMultiLogic(child); } - else if (func->name == "not") + else if (func && func->name == "not") { for (auto & child : func->arguments->children) splitMultiLogic(child); @@ -168,24 +167,14 @@ CNFQuery TreeCNFConverter::toCNF(const ASTPtr & query) auto cnf = query->clone(); splitMultiLogic(cnf); - Poco::Logger::get("CNF CONVERSION").information("SPLIT:" + cnf->dumpTree()); - Poco::Logger::get("CNF CONVERSION").information("SPLIT:" + cnf->getColumnName()); traversePushNot(cnf, false); - - Poco::Logger::get("PUSH NOT").information("SPLIT:" + cnf->dumpTree()); - Poco::Logger::get("PUSH NOT").information("SPLIT:" + cnf->getColumnName()); - traversePushOr(cnf); - - Poco::Logger::get("PUSH OR").information("SPLIT:" + cnf->dumpTree()); - Poco::Logger::get("PUSH OR").information("SPLIT:" + cnf->getColumnName()); - CNFQuery::AndGroup and_group; traverseCNF(cnf, and_group); CNFQuery result{std::move(and_group)}; - Poco::Logger::get("CNF CONVERSION").information("DONE: " + result.dump()); + Poco::Logger::get("TreeCNFConverter").information("Converted to CNF: " + result.dump()); return result; } diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index c82c908998d..73e7cc624c9 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -519,10 +519,6 @@ void optimizeWithConstraints(ASTSelectQuery * select_query, const bool optimize_append_index) { WhereConstraintsOptimizer(select_query, metadata_snapshot, optimize_append_index).perform(); - if (select_query->where()) - Poco::Logger::get("CNF").information(select_query->where()->dumpTree()); - else - Poco::Logger::get("CNF").information("NO WHERE"); } void optimizeSubstituteColumn(ASTSelectQuery * select_query, @@ -543,8 +539,6 @@ void convertQueryToCNF(ASTSelectQuery * select_query) auto cnf_form = TreeCNFConverter::toCNF(select_query->where()).pushNotInFuntions(); select_query->refWhere() = TreeCNFConverter::fromCNF(cnf_form); } - if (select_query->where()) - Poco::Logger::get("CNF").information(select_query->where()->dumpTree()); } /// Remove duplicated columns from USING(...). @@ -656,16 +650,6 @@ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & sou if (settings.optimize_substitute_columns) optimizeSubstituteColumn(select_query, aliases, source_columns_set, tables_with_columns, metadata_snapshot, storage); } - if (select_query->where()) - { - Poco::Logger::get("&&&&&&&&&&&&&&& WHERE").information(select_query->where()->getColumnName()); - Poco::Logger::get("&&&&&&&&&&&&&&& WHERE").information(select_query->where()->dumpTree()); - } - if (select_query->prewhere()) - { - Poco::Logger::get("&&&&&&&&&&&&&&& prewhere").information(select_query->prewhere()->getColumnName()); - Poco::Logger::get("&&&&&&&&&&&&&&& prewhere").information(select_query->prewhere()->dumpTree()); - } /// Push the predicate expression down to the subqueries. rewrite_subqueries = PredicateExpressionsOptimizer(context, tables_with_columns, settings).optimize(*select_query); diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index 5b0182effb1..595d39c406d 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -156,11 +156,9 @@ void WhereConstraintsOptimizer::perform() { if (select_query->where() && metadata_snapshot) { - //const auto & constraint_data = metadata_snapshot->getConstraints().getConstraintData(); const auto & compare_graph = metadata_snapshot->getConstraints().getGraph(); - Poco::Logger::get("BEFORE CNF ").information(select_query->where()->dumpTree()); auto cnf = TreeCNFConverter::toCNF(select_query->where()); - Poco::Logger::get("BEFORE OPT").information(cnf.dump()); + Poco::Logger::get("WhereConstraintsOptimizer").information("Before optimization: " + cnf.dump()); cnf.pullNotOutFunctions() .filterAlwaysTrueGroups([&compare_graph, this](const auto & group) { @@ -182,7 +180,7 @@ void WhereConstraintsOptimizer::perform() if (optimize_append_index) AddIndexConstraintsOptimizer(metadata_snapshot).perform(cnf); - Poco::Logger::get("AFTER OPT").information(cnf.dump()); + Poco::Logger::get("WhereConstraintsOptimizer").information("After optimization: " + cnf.dump()); select_query->setExpression(ASTSelectQuery::Expression::WHERE, TreeCNFConverter::fromCNF(cnf)); } } diff --git a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp index 450e6035a25..6d960c2acf3 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp @@ -163,6 +163,10 @@ bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranu throw Exception("Only hypothesis index is supported here.", ErrorCodes::LOGICAL_ERROR); values.push_back(granule->met); } + + if (const auto it = answerCache.find(values); it != std::end(answerCache)) + return it->second; + const auto & graph = getGraph(values); bool always_false = false; @@ -176,7 +180,6 @@ bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranu { CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()}; pushNotIn(atom); - Poco::Logger::get("KEK").information(atom.ast->dumpTree()); const auto * func = atom.ast->as(); if (func && func->arguments->children.size() == 2) { @@ -191,12 +194,12 @@ bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranu } always_false = true; }); + answerCache[values] = !always_false; return !always_false; } std::unique_ptr MergeTreeIndexMergedCondition::buildGraph(const std::vector & values) const { - Poco::Logger::get("MergeTreeIndexMergedCondition").information("New graph"); std::vector active_atomic_formulas(atomic_constraints); for (size_t i = 0; i < values.size(); ++i) { @@ -208,7 +211,7 @@ std::unique_ptr MergeTreeIndexMergedCondition::buildGraph(const } return std::make_unique(active_atomic_formulas); } - +// something strange)) const ComparisonGraph & MergeTreeIndexMergedCondition::getGraph(const std::vector & values) const { if (!graphCache.contains(values)) diff --git a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h index 204e6f01ea2..9a760797eb3 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h +++ b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h @@ -25,7 +25,6 @@ public: bool alwaysUnknownOrTrue() const; bool mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const; - //TODO: add constraints private: std::unique_ptr buildGraph(const std::vector & values) const; const ComparisonGraph & getGraph(const std::vector & values) const; @@ -35,6 +34,7 @@ private: std::unique_ptr expression_cnf; mutable std::unordered_map, std::unique_ptr> graphCache; + mutable std::unordered_map, bool> answerCache; std::vector> index_to_compare_atomic_hypotheses; std::vector> index_to_atomic_hypotheses; diff --git a/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/src/Storages/MergeTree/MergeTreeRangeReader.cpp index c0bfbd21e0e..bef67f0ade2 100644 --- a/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -529,12 +529,6 @@ MergeTreeRangeReader::MergeTreeRangeReader( , last_reader_in_chain(last_reader_in_chain_) , is_initialized(true) { - if (prewhere_info_) - { - Poco::Logger::get("PREWHERE").information(prewhere_info_->prewhere_column_name); - Poco::Logger::get("PREWHERE").information(prewhere_info_->prewhere_actions->dumpActions()); - Poco::Logger::get("PREWHERE rm?").information(std::to_string(prewhere_info_->remove_prewhere_column)); - } if (prev_reader) sample_block = prev_reader->getSampleBlock(); @@ -989,7 +983,6 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r /// If we need to filter in PREWHERE else if (prewhere_info->need_filter || result.need_filter || prewhere_info->row_level_filter) { - Poco::Logger::get("USE PREWHERE FILTER").information("++++++++++++++"); /// If there is a filter and without optimized if (result.getFilter() && last_reader_in_chain) { @@ -1045,7 +1038,6 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r /// Filter in WHERE instead else { - Poco::Logger::get("USE WHERE FILTER ").information("---------------------"); result.columns[prewhere_column_pos] = result.getFilterHolder()->convertToFullColumnIfConst(); if (getSampleBlock().getByName(prewhere_info->prewhere_column_name).type->isNullable()) result.columns[prewhere_column_pos] = makeNullable(std::move(result.columns[prewhere_column_pos])); diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 3b45faab77b..a819a1670ca 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -44,7 +44,6 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer( , log{log_} , column_sizes{std::move(column_sizes_)} { - Poco::Logger::get("kek").information("IN MT OPTIMIZER"); const auto & primary_key = metadata_snapshot->getPrimaryKey(); if (!primary_key.column_names.empty()) first_primary_key_column = primary_key.column_names[0]; @@ -187,17 +186,6 @@ void MergeTreeWhereOptimizer::optimize(ASTSelectQuery & select) const if (!select.where() || select.prewhere()) return; - if (select.where()) - { - Poco::Logger::get("MTPRWHERE WHERE").information(select.where()->getColumnName()); - Poco::Logger::get("MTPRWHERE WHERE").information(select.where()->dumpTree()); - } - if (select.prewhere()) - { - Poco::Logger::get("MTPRWHERE PRE").information(select.prewhere()->dumpTree()); - Poco::Logger::get("MTPRWHERE PRE").information(select.prewhere()->getColumnName()); - } - Conditions where_conditions = analyze(select.where(), select.final()); Conditions prewhere_conditions; diff --git a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp index 415d67d8b78..cd4f0560a0c 100644 --- a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp +++ b/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp @@ -146,15 +146,11 @@ public: if (identifier && data.name_to_component_id.contains(identifier->name())) { const String & name = identifier->name(); - //Poco::Logger::get("NAME").information(name); const auto component_id = data.name_to_component_id.at(name); - //Poco::Logger::get("COMP").information(std::to_string(component_id)); auto new_ast = data.id_to_expression_map.at(component_id)->clone(); - //Poco::Logger::get("NEW_AST").information(new_ast->dumpTree()); if (data.is_select) { new_ast->setAlias(data.old_name.at(name)); - //Poco::Logger::get("OLD").information(data.old_name.at(name)); } ast = new_ast; } @@ -195,7 +191,6 @@ void bruteforce( { min_price = current_price; min_expressions = expressions_stack; - //Poco::Logger::get("PRICE").information("UPDATE"); } } else @@ -325,12 +320,6 @@ void SubstituteColumnOptimizer::perform() for (size_t i = 0; i < components_list.size(); ++i) id_to_expression_map[components_list[i]] = min_expressions[i]; - /*Poco::Logger::get("comp list").information("CL"); - for (const auto id : components_list) - Poco::Logger::get("comp list").information(std::to_string(id)); - for (const auto & [k, v] : id_to_expression_map) - Poco::Logger::get("id2expr").information(std::to_string(k) + " " + v->dumpTree());*/ - auto process = [&](ASTPtr & ast, bool is_select) { SubstituteColumnVisitor::Data substitute_data{id_to_expression_map, name_to_component, old_name, is_select}; From 90144f40c58296fb5bbb212456b40bbff340ad73 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Wed, 19 May 2021 22:43:02 +0300 Subject: [PATCH 056/200] style --- src/Interpreters/TreeCNFConverter.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index bdb1d5d490b..b60eaa28e0a 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -81,7 +81,8 @@ void traversePushNot(ASTPtr & node, bool add_negation) } /// Push Or inside And (actually pull AND to top) -void traversePushOr(ASTPtr & node) { +void traversePushOr(ASTPtr & node) +{ auto * func = node->as(); if (func && (func->name == "or" || func->name == "and")) From 135b0ed69db740d8c5a3cd189e542b8efb818469 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Thu, 20 May 2021 23:27:32 +0300 Subject: [PATCH 057/200] fix --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 26 +++++++------------ 1 file changed, 10 insertions(+), 16 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 5e2c83c72cd..46cb5957d28 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1116,17 +1116,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( .num_granules_after = index_and_condition.total_granules - index_and_condition.granules_dropped}); } - LOG_DEBUG( - log, - "Selected {}/{} parts by partition key, {} parts by primary key, {}/{} marks by primary key, {} marks to read from {} ranges", - parts.size(), - total_parts, - parts_with_ranges.size(), - sum_marks_pk.load(std::memory_order_relaxed), - total_marks_pk.load(std::memory_order_relaxed), - sum_marks, - sum_ranges); - for (const auto & [granularity, index_and_condition] : merged_indices) { const auto & index_name = "Merged"; @@ -1144,11 +1133,16 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( .num_granules_after = index_and_condition->total_granules - index_and_condition->granules_dropped}); } - LOG_DEBUG(log, "Selected {}/{} parts by partition key, {} parts by primary key, {}/{} marks by primary key, {} marks to read from {} ranges", - parts.size(), total_parts, parts_with_ranges.size(), - sum_marks_pk.load(std::memory_order_relaxed), - total_marks_pk.load(std::memory_order_relaxed), - sum_marks, sum_ranges); + LOG_DEBUG( + log, + "Selected {}/{} pay, {} parts by primary key, {}/{} marks by primary key, {} marks to read from {} ranges", + parts.size(), + total_parts, + parts_with_ranges.size(), + sum_marks_pk.load(std::memory_order_relaxed), + total_marks_pk.load(std::memory_order_relaxed), + sum_marks, + sum_ranges); } if (cache) From dd343f087e64116188ac1931adf36d68bc633c65 Mon Sep 17 00:00:00 2001 From: Nikita Vasilev Date: Fri, 21 May 2021 19:23:33 +0300 Subject: [PATCH 058/200] fix --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataSelectExecutor.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 46cb5957d28..90d035aa5a5 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1135,7 +1135,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( LOG_DEBUG( log, - "Selected {}/{} pay, {} parts by primary key, {}/{} marks by primary key, {} marks to read from {} ranges", + "Selected {}/{} parts by partition key, {} parts by primary key, {}/{} marks by primary key, {} marks to read from {} ranges", parts.size(), total_parts, parts_with_ranges.size(), diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index e71bb083b78..52a53ab1177 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -24,7 +24,7 @@ struct MergeTreeDataSelectSamplingData struct MergeTreeDataSelectCache { RangesInDataParts parts_with_ranges; - MergeTreeDataSelectSamplingData sampling; + MergeTreeDataSelectSamplingData sampling{}; std::unique_ptr index_stats; size_t sum_marks = 0; size_t sum_ranges = 0; From 24d3209f6042806e18b1ad07d99c5278a8c9cb21 Mon Sep 17 00:00:00 2001 From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Date: Wed, 20 Oct 2021 15:08:50 +0300 Subject: [PATCH 059/200] Update table.md --- .../sql-reference/statements/create/table.md | 32 +++++++++++++++++-- 1 file changed, 29 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 5334532a54f..09096bf9baf 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -252,21 +252,47 @@ CREATE TABLE codec_example ENGINE = MergeTree() ``` - + +**Example** + +```sql +CREATE TABLE mytable +( + x String Codec(AES_128_GCM_SIV) +) +ENGINE=MergeTree ORDER BY x; +``` + +!!!note "Note" + If compression needs to be applied, it must be explicitly specified. Otherwise, only encryption will be applied to data. + +**Example** + +```sql +CREATE TABLE mytable +( + x String Codec(Delta, LZ4, AES_128_GCM_SIV) +) +ENGINE=MergeTree ORDER BY x; +``` + ## Temporary Tables {#temporary-tables} ClickHouse supports temporary tables which have the following characteristics: From 963ece5d6a057fa9ef2adc3b5acc8607c25c61c1 Mon Sep 17 00:00:00 2001 From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Date: Wed, 20 Oct 2021 15:13:12 +0300 Subject: [PATCH 060/200] Update table.md --- docs/en/sql-reference/statements/create/table.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/statements/create/table.md b/docs/en/sql-reference/statements/create/table.md index 09096bf9baf..d64642704f5 100644 --- a/docs/en/sql-reference/statements/create/table.md +++ b/docs/en/sql-reference/statements/create/table.md @@ -277,7 +277,7 @@ CREATE TABLE mytable ( x String Codec(AES_128_GCM_SIV) ) -ENGINE=MergeTree ORDER BY x; +ENGINE = MergeTree ORDER BY x; ``` !!!note "Note" @@ -290,7 +290,7 @@ CREATE TABLE mytable ( x String Codec(Delta, LZ4, AES_128_GCM_SIV) ) -ENGINE=MergeTree ORDER BY x; +ENGINE = MergeTree ORDER BY x; ``` ## Temporary Tables {#temporary-tables} From 5608e25ed423926fcfab29aba915f4e68993f57f Mon Sep 17 00:00:00 2001 From: Filatenkov Artur <58165623+FArthur-cmd@users.noreply.github.com> Date: Wed, 20 Oct 2021 17:09:48 +0300 Subject: [PATCH 061/200] Update settings.md --- .../en/operations/server-configuration-parameters/settings.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 19567ec29fb..af68e9a02be 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -105,7 +105,7 @@ Loading key from the environment variable: ```xml - + ``` @@ -118,7 +118,7 @@ Each of these methods can be applied for multiple keys: 00112233445566778899aabbccddeeff - + 1 From c65b3846726036c26200109ac427bb532aef56f3 Mon Sep 17 00:00:00 2001 From: qieqieplus Date: Tue, 2 Nov 2021 11:11:48 +0800 Subject: [PATCH 062/200] use func alias for untuple names --- src/Interpreters/ActionsVisitor.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index e358696fa40..4dadc8e0f49 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -693,6 +693,7 @@ ASTs ActionsMatcher::doUntuple(const ASTFunction * function, ActionsMatcher::Dat ASTs columns; size_t tid = 0; + auto func_alias = function->tryGetAlias(); for (const auto & name [[maybe_unused]] : tuple_type->getElementNames()) { auto tuple_ast = function->arguments->children[0]; @@ -703,7 +704,8 @@ ASTs ActionsMatcher::doUntuple(const ASTFunction * function, ActionsMatcher::Dat visit(*literal, literal, data); auto func = makeASTFunction("tupleElement", tuple_ast, literal); - + if (!func_alias.empty()) + func->setAlias(func_alias + toString(tid)); auto function_builder = FunctionFactory::instance().get(func->name, data.getContext()); data.addFunction(function_builder, {tuple_name_type->name, literal->getColumnName()}, func->getColumnName()); From 825f37d22217038236015d64ca583fba311746de Mon Sep 17 00:00:00 2001 From: qieqieplus Date: Tue, 2 Nov 2021 12:01:01 +0800 Subject: [PATCH 063/200] add test sql --- tests/queries/0_stateless/02113_untuple_func_alias.reference | 2 ++ tests/queries/0_stateless/02113_untuple_func_alias.sql | 2 ++ 2 files changed, 4 insertions(+) create mode 100644 tests/queries/0_stateless/02113_untuple_func_alias.reference create mode 100644 tests/queries/0_stateless/02113_untuple_func_alias.sql diff --git a/tests/queries/0_stateless/02113_untuple_func_alias.reference b/tests/queries/0_stateless/02113_untuple_func_alias.reference new file mode 100644 index 00000000000..f78b11058b5 --- /dev/null +++ b/tests/queries/0_stateless/02113_untuple_func_alias.reference @@ -0,0 +1,2 @@ +ut1 ut2 ut3 ut4 ut21 ut22 ut23 ut24 +1 2 3 \N \N 3 2 1 diff --git a/tests/queries/0_stateless/02113_untuple_func_alias.sql b/tests/queries/0_stateless/02113_untuple_func_alias.sql new file mode 100644 index 00000000000..d39e6626d48 --- /dev/null +++ b/tests/queries/0_stateless/02113_untuple_func_alias.sql @@ -0,0 +1,2 @@ +SELECT untuple((1, 2, 3, b)) AS `ut`, untuple((NULL, 3, 2, a)) AS `ut2` +FROM (SELECT 1 AS a, NULL AS b) FORMAT TSVWithNames; From 79bb4fdba4dd038198af5420084921bc0f8caf9f Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 2 Nov 2021 18:48:40 +0300 Subject: [PATCH 064/200] Fix --- src/Functions/FunctionSQLJSON.h | 15 ++++-- .../01889_sql_json_functions.reference | 50 ++++++++++++++++++- .../0_stateless/01889_sql_json_functions.sql | 1 + 3 files changed, 62 insertions(+), 4 deletions(-) diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index aaed980c62a..a4099b0b436 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -260,12 +260,21 @@ public: } if (status == VisitorStatus::Exhausted) - { return false; - } std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM - out << current_element.getElement(); + if (current_element.isString()) + { + std::string_view element = current_element.getString(); + if (element.starts_with('\"') && element.size() > 1) + out << element.substr(1, element.size() - 1); + else + out << element; + } + else + { + out << current_element.getElement(); + } auto output_str = out.str(); ColumnString & col_str = assert_cast(dest); col_str.insertData(output_str.data(), output_str.size()); diff --git a/tests/queries/0_stateless/01889_sql_json_functions.reference b/tests/queries/0_stateless/01889_sql_json_functions.reference index 593f2fb2d20..c4e855a125f 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.reference +++ b/tests/queries/0_stateless/01889_sql_json_functions.reference @@ -1,43 +1,91 @@ +-- { echo } +SELECT '--JSON_VALUE--'; --JSON_VALUE-- +SELECT JSON_VALUE('{"hello":1}', '$'); -- root is a complex object => default value (empty string) +SELECT JSON_VALUE('{"hello":1}', '$.hello'); 1 +SELECT JSON_VALUE('{"hello":1.2}', '$.hello'); 1.2 +SELECT JSON_VALUE('{"hello":true}', '$.hello'); true -"world" +SELECT JSON_VALUE('{"hello":"world"}', '$.hello'); +world +SELECT JSON_VALUE('{"hello":null}', '$.hello'); null +SELECT JSON_VALUE('{"hello":["world","world2"]}', '$.hello'); +SELECT JSON_VALUE('{"hello":{"world":"!"}}', '$.hello'); +SELECT JSON_VALUE('{hello:world}', '$.hello'); -- invalid json => default value (empty string) +SELECT JSON_VALUE('', '$.hello'); +SELECT '--JSON_QUERY--'; --JSON_QUERY-- +SELECT JSON_QUERY('{"hello":1}', '$'); [{"hello":1}] +SELECT JSON_QUERY('{"hello":1}', '$.hello'); [1] +SELECT JSON_QUERY('{"hello":1.2}', '$.hello'); [1.2] +SELECT JSON_QUERY('{"hello":true}', '$.hello'); [true] +SELECT JSON_QUERY('{"hello":"world"}', '$.hello'); ["world"] +SELECT JSON_QUERY('{"hello":null}', '$.hello'); [null] +SELECT JSON_QUERY('{"hello":["world","world2"]}', '$.hello'); [["world","world2"]] +SELECT JSON_QUERY('{"hello":{"world":"!"}}', '$.hello'); [{"world":"!"}] +SELECT JSON_QUERY( '{hello:{"world":"!"}}}', '$.hello'); -- invalid json => default value (empty string) +SELECT JSON_QUERY('', '$.hello'); +SELECT JSON_QUERY('{"array":[[0, 1, 2, 3, 4, 5], [0, -1, -2, -3, -4, -5]]}', '$.array[*][0 to 2, 4]'); [0, 1, 4, 0, -1, -4] +SELECT '--JSON_EXISTS--'; --JSON_EXISTS-- +SELECT JSON_EXISTS('{"hello":1}', '$'); 1 +SELECT JSON_EXISTS('', '$'); 0 +SELECT JSON_EXISTS('{}', '$'); 1 +SELECT JSON_EXISTS('{"hello":1}', '$.hello'); 1 +SELECT JSON_EXISTS('{"hello":1,"world":2}', '$.world'); 1 +SELECT JSON_EXISTS('{"hello":{"world":1}}', '$.world'); 0 +SELECT JSON_EXISTS('{"hello":{"world":1}}', '$.hello.world'); 1 +SELECT JSON_EXISTS('{hello:world}', '$.hello'); -- invalid json => default value (zero integer) 0 +SELECT JSON_EXISTS('', '$.hello'); 0 +SELECT JSON_EXISTS('{"hello":["world"]}', '$.hello[*]'); 1 +SELECT JSON_EXISTS('{"hello":["world"]}', '$.hello[0]'); 1 +SELECT JSON_EXISTS('{"hello":["world"]}', '$.hello[1]'); 0 +SELECT JSON_EXISTS('{"a":[{"b":1},{"c":2}]}', '$.a[*].b'); 1 +SELECT JSON_EXISTS('{"a":[{"b":1},{"c":2}]}', '$.a[*].f'); 0 +SELECT JSON_EXISTS('{"a":[[{"b":1}, {"g":1}],[{"h":1},{"y":1}]]}', '$.a[*][0].h'); 1 +SELECT '--MANY ROWS--'; --MANY ROWS-- +DROP TABLE IF EXISTS 01889_sql_json; +CREATE TABLE 01889_sql_json (id UInt8, json String) ENGINE = MergeTree ORDER BY id; +INSERT INTO 01889_sql_json(id, json) VALUES(0, '{"name":"Ivan","surname":"Ivanov","friends":["Vasily","Kostya","Artyom"]}'); +INSERT INTO 01889_sql_json(id, json) VALUES(1, '{"name":"Katya","surname":"Baltica","friends":["Tihon","Ernest","Innokentiy"]}'); +INSERT INTO 01889_sql_json(id, json) VALUES(2, '{"name":"Vitali","surname":"Brown","friends":["Katya","Anatoliy","Ivan","Oleg"]}'); +SELECT id, JSON_QUERY(json, '$.friends[0 to 2]') FROM 01889_sql_json ORDER BY id; 0 ["Vasily", "Kostya"] 1 ["Tihon", "Ernest"] 2 ["Katya", "Anatoliy"] +DROP TABLE 01889_sql_json; diff --git a/tests/queries/0_stateless/01889_sql_json_functions.sql b/tests/queries/0_stateless/01889_sql_json_functions.sql index 087f029e635..09f692e29a3 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.sql +++ b/tests/queries/0_stateless/01889_sql_json_functions.sql @@ -1,5 +1,6 @@ -- Tags: no-fasttest +-- { echo } SELECT '--JSON_VALUE--'; SELECT JSON_VALUE('{"hello":1}', '$'); -- root is a complex object => default value (empty string) SELECT JSON_VALUE('{"hello":1}', '$.hello'); From b1c4a48d949b1b7c87e32dbc10b730db1902aaf5 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 3 Nov 2021 14:40:35 +0300 Subject: [PATCH 065/200] Update test --- .../01889_sql_json_functions.reference | 93 +++++++++++++++++++ 1 file changed, 93 insertions(+) diff --git a/tests/queries/0_stateless/01889_sql_json_functions.reference b/tests/queries/0_stateless/01889_sql_json_functions.reference index e69de29bb2d..544d6f5134f 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.reference +++ b/tests/queries/0_stateless/01889_sql_json_functions.reference @@ -0,0 +1,93 @@ +-- { echo } +SELECT '--JSON_VALUE--'; +--JSON_VALUE-- +SELECT JSON_VALUE('{"hello":1}', '$'); -- root is a complex object => default value (empty string) + +SELECT JSON_VALUE('{"hello":1}', '$.hello'); +1 +SELECT JSON_VALUE('{"hello":1.2}', '$.hello'); +1.2 +SELECT JSON_VALUE('{"hello":true}', '$.hello'); +true +SELECT JSON_VALUE('{"hello":"world"}', '$.hello'); +world +SELECT JSON_VALUE('{"hello":null}', '$.hello'); +null +SELECT JSON_VALUE('{"hello":["world","world2"]}', '$.hello'); + +SELECT JSON_VALUE('{"hello":{"world":"!"}}', '$.hello'); + +SELECT JSON_VALUE('{hello:world}', '$.hello'); -- invalid json => default value (empty string) + +SELECT JSON_VALUE('', '$.hello'); + +SELECT JSON_VALUE('{"foo foo":"bar"}', '$."foo foo"'); +bar +SELECT '--JSON_QUERY--'; +--JSON_QUERY-- +SELECT JSON_QUERY('{"hello":1}', '$'); +[{"hello":1}] +SELECT JSON_QUERY('{"hello":1}', '$.hello'); +[1] +SELECT JSON_QUERY('{"hello":1.2}', '$.hello'); +[1.2] +SELECT JSON_QUERY('{"hello":true}', '$.hello'); +[true] +SELECT JSON_QUERY('{"hello":"world"}', '$.hello'); +["world"] +SELECT JSON_QUERY('{"hello":null}', '$.hello'); +[null] +SELECT JSON_QUERY('{"hello":["world","world2"]}', '$.hello'); +[["world","world2"]] +SELECT JSON_QUERY('{"hello":{"world":"!"}}', '$.hello'); +[{"world":"!"}] +SELECT JSON_QUERY( '{hello:{"world":"!"}}}', '$.hello'); -- invalid json => default value (empty string) + +SELECT JSON_QUERY('', '$.hello'); + +SELECT JSON_QUERY('{"array":[[0, 1, 2, 3, 4, 5], [0, -1, -2, -3, -4, -5]]}', '$.array[*][0 to 2, 4]'); +[0, 1, 4, 0, -1, -4] +SELECT '--JSON_EXISTS--'; +--JSON_EXISTS-- +SELECT JSON_EXISTS('{"hello":1}', '$'); +1 +SELECT JSON_EXISTS('', '$'); +0 +SELECT JSON_EXISTS('{}', '$'); +1 +SELECT JSON_EXISTS('{"hello":1}', '$.hello'); +1 +SELECT JSON_EXISTS('{"hello":1,"world":2}', '$.world'); +1 +SELECT JSON_EXISTS('{"hello":{"world":1}}', '$.world'); +0 +SELECT JSON_EXISTS('{"hello":{"world":1}}', '$.hello.world'); +1 +SELECT JSON_EXISTS('{hello:world}', '$.hello'); -- invalid json => default value (zero integer) +0 +SELECT JSON_EXISTS('', '$.hello'); +0 +SELECT JSON_EXISTS('{"hello":["world"]}', '$.hello[*]'); +1 +SELECT JSON_EXISTS('{"hello":["world"]}', '$.hello[0]'); +1 +SELECT JSON_EXISTS('{"hello":["world"]}', '$.hello[1]'); +0 +SELECT JSON_EXISTS('{"a":[{"b":1},{"c":2}]}', '$.a[*].b'); +1 +SELECT JSON_EXISTS('{"a":[{"b":1},{"c":2}]}', '$.a[*].f'); +0 +SELECT JSON_EXISTS('{"a":[[{"b":1}, {"g":1}],[{"h":1},{"y":1}]]}', '$.a[*][0].h'); +1 +SELECT '--MANY ROWS--'; +--MANY ROWS-- +DROP TABLE IF EXISTS 01889_sql_json; +CREATE TABLE 01889_sql_json (id UInt8, json String) ENGINE = MergeTree ORDER BY id; +INSERT INTO 01889_sql_json(id, json) VALUES(0, '{"name":"Ivan","surname":"Ivanov","friends":["Vasily","Kostya","Artyom"]}'); +INSERT INTO 01889_sql_json(id, json) VALUES(1, '{"name":"Katya","surname":"Baltica","friends":["Tihon","Ernest","Innokentiy"]}'); +INSERT INTO 01889_sql_json(id, json) VALUES(2, '{"name":"Vitali","surname":"Brown","friends":["Katya","Anatoliy","Ivan","Oleg"]}'); +SELECT id, JSON_QUERY(json, '$.friends[0 to 2]') FROM 01889_sql_json ORDER BY id; +0 ["Vasily", "Kostya"] +1 ["Tihon", "Ernest"] +2 ["Katya", "Anatoliy"] +DROP TABLE 01889_sql_json; From 2dd4393ca1ed69a0b64bb16a65d2bf2d6e08e59a Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 2 Nov 2021 16:40:41 +0300 Subject: [PATCH 066/200] Small refactoring in formats --- src/Client/ClientBase.cpp | 4 +- src/Functions/formatRow.cpp | 2 + src/Processors/Formats/IOutputFormat.cpp | 13 ++-- src/Processors/Formats/IOutputFormat.h | 61 +++++++++++-------- src/Processors/Formats/IRowOutputFormat.cpp | 5 -- src/Processors/Formats/IRowOutputFormat.h | 48 ++++++--------- .../Formats/Impl/ArrowBlockInputFormat.h | 3 +- .../Formats/Impl/ArrowBlockOutputFormat.h | 9 +-- .../Formats/Impl/AvroRowInputFormat.h | 13 ++-- .../Formats/Impl/AvroRowOutputFormat.cpp | 4 +- .../Formats/Impl/AvroRowOutputFormat.h | 3 +- .../Formats/Impl/BinaryRowOutputFormat.h | 6 +- .../Formats/Impl/CSVRowInputFormat.h | 3 +- .../Formats/Impl/CSVRowOutputFormat.cpp | 2 +- .../Formats/Impl/CSVRowOutputFormat.h | 15 +++-- .../Formats/Impl/CapnProtoRowInputFormat.h | 2 +- .../Formats/Impl/CapnProtoRowOutputFormat.h | 2 +- .../Formats/Impl/JSONAsStringRowInputFormat.h | 5 +- .../Impl/JSONCompactEachRowRowInputFormat.h | 2 +- .../JSONCompactEachRowRowOutputFormat.cpp | 2 +- .../Impl/JSONCompactEachRowRowOutputFormat.h | 5 +- .../Formats/Impl/JSONCompactRowOutputFormat.h | 2 +- .../Formats/Impl/JSONEachRowRowInputFormat.h | 4 +- .../Formats/Impl/JSONEachRowRowOutputFormat.h | 2 +- .../JSONEachRowWithProgressRowOutputFormat.h | 5 +- .../Formats/Impl/JSONRowOutputFormat.h | 39 ++++++------ .../Formats/Impl/LineAsStringRowInputFormat.h | 3 +- .../Formats/Impl/MarkdownRowOutputFormat.h | 5 +- .../Formats/Impl/MsgPackRowInputFormat.h | 3 +- .../Formats/Impl/MsgPackRowOutputFormat.h | 2 +- .../Formats/Impl/MySQLOutputFormat.cpp | 9 +-- .../Formats/Impl/MySQLOutputFormat.h | 8 +-- .../Impl/ODBCDriver2BlockOutputFormat.cpp | 7 --- .../Impl/ODBCDriver2BlockOutputFormat.h | 18 ++---- .../Formats/Impl/ORCBlockOutputFormat.h | 3 +- .../Impl/ParallelFormattingOutputFormat.cpp | 4 +- .../Impl/ParallelFormattingOutputFormat.h | 5 +- .../Formats/Impl/ParallelParsingInputFormat.h | 4 +- .../Formats/Impl/ParquetBlockInputFormat.h | 4 +- .../Formats/Impl/ParquetBlockOutputFormat.h | 5 +- .../Formats/Impl/PostgreSQLOutputFormat.cpp | 10 +-- .../Formats/Impl/PostgreSQLOutputFormat.h | 6 +- .../Formats/Impl/PrettyBlockOutputFormat.h | 2 +- .../Impl/PrettyCompactBlockOutputFormat.h | 3 +- .../Impl/PrettySpaceBlockOutputFormat.h | 2 +- .../Formats/Impl/ProtobufRowInputFormat.h | 2 +- .../Formats/Impl/ProtobufRowOutputFormat.h | 5 +- .../Formats/Impl/RawBLOBRowInputFormat.h | 4 +- .../Formats/Impl/RawBLOBRowOutputFormat.h | 1 + .../Formats/Impl/RegexpRowInputFormat.h | 4 +- .../Formats/Impl/TSKVRowInputFormat.h | 6 +- .../Formats/Impl/TSKVRowOutputFormat.h | 2 +- .../Formats/Impl/TabSeparatedRowInputFormat.h | 6 +- .../Impl/TabSeparatedRowOutputFormat.cpp | 2 +- .../Impl/TabSeparatedRowOutputFormat.h | 14 ++--- .../Impl/TemplateBlockOutputFormat.cpp | 12 +--- .../Formats/Impl/TemplateBlockOutputFormat.h | 7 +-- .../Formats/Impl/TemplateRowInputFormat.h | 7 +-- .../Formats/Impl/ValuesRowOutputFormat.h | 2 +- .../Formats/Impl/VerticalRowOutputFormat.h | 4 +- .../Formats/Impl/XMLRowOutputFormat.h | 2 +- .../Formats/RowInputFormatWithNamesAndTypes.h | 5 +- src/Server/GRPCServer.cpp | 9 +-- src/Storages/HDFS/StorageHDFS.cpp | 8 +-- src/Storages/Kafka/KafkaBlockOutputStream.cpp | 2 +- src/Storages/RabbitMQ/RabbitMQSink.cpp | 2 +- src/Storages/StorageFile.cpp | 18 ++---- src/Storages/StorageS3.cpp | 8 +-- src/Storages/StorageURL.cpp | 8 +-- src/Storages/StorageURL.h | 2 - .../01553_settings_early_apply.sql | 4 ++ .../0_stateless/02113_format_row.reference | 20 ++++++ .../queries/0_stateless/02113_format_row.sql | 5 ++ 73 files changed, 245 insertions(+), 290 deletions(-) create mode 100644 tests/queries/0_stateless/02113_format_row.reference create mode 100644 tests/queries/0_stateless/02113_format_row.sql diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index ebc6c7d3107..880c867b053 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -391,7 +391,7 @@ void ClientBase::initBlockOutputStream(const Block & block, ASTPtr parsed_query) output_format = global_context->getOutputFormat( current_format, out_file_buf ? *out_file_buf : *out_buf, block); - output_format->doWritePrefix(); + output_format->setAutoFlush(); } } @@ -671,7 +671,7 @@ void ClientBase::onEndOfStream() progress_indication.clearProgressOutput(); if (output_format) - output_format->doWriteSuffix(); + output_format->finish(); resetOutput(); diff --git a/src/Functions/formatRow.cpp b/src/Functions/formatRow.cpp index 3f9d3e782d7..ccd013123d7 100644 --- a/src/Functions/formatRow.cpp +++ b/src/Functions/formatRow.cpp @@ -77,6 +77,8 @@ public: if (!dynamic_cast(out.get())) throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot turn rows into a {} format strings. {} function supports only row output formats", format_name, getName()); + /// Don't write prefix if any. + out->doNotWritePrefix(); out->write(arg_columns); return col_str; } diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index 0f94622b7c6..a0b39f20b96 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -65,11 +65,7 @@ static Chunk prepareTotals(Chunk chunk) void IOutputFormat::work() { - if (!prefix_written) - { - doWritePrefix(); - prefix_written = true; - } + writePrefixIfNot(); if (finished && !finalized) { @@ -110,10 +106,17 @@ void IOutputFormat::flush() void IOutputFormat::write(const Block & block) { + writePrefixIfNot(); consume(Chunk(block.getColumns(), block.rows())); if (auto_flush) flush(); } +void IOutputFormat::finish() +{ + writePrefixIfNot(); + finalize(); +} + } diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index ba4dcee6f70..225971a8fe6 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -25,28 +25,6 @@ class IOutputFormat : public IProcessor public: enum PortKind { Main = 0, Totals = 1, Extremes = 2 }; -protected: - WriteBuffer & out; - - Chunk current_chunk; - PortKind current_block_kind = PortKind::Main; - bool has_input = false; - bool finished = false; - bool finalized = false; - - /// Flush data on each consumed chunk. This is intended for interactive applications to output data as soon as it's ready. - bool auto_flush = false; - - RowsBeforeLimitCounterPtr rows_before_limit_counter; - - friend class ParallelFormattingOutputFormat; - - virtual void consume(Chunk) = 0; - virtual void consumeTotals(Chunk) {} - virtual void consumeExtremes(Chunk) {} - virtual void finalize() {} - -public: IOutputFormat(const Block & header_, WriteBuffer & out_); Status prepare() override; @@ -77,8 +55,7 @@ public: void write(const Block & block); - virtual void doWritePrefix() {} - virtual void doWriteSuffix() { finalize(); } + virtual void finish(); virtual bool expectMaterializedColumns() const { return true; } @@ -88,11 +65,43 @@ public: size_t getResultRows() const { return result_rows; } size_t getResultBytes() const { return result_bytes; } + void doNotWritePrefix() { need_write_prefix = false; } + +protected: + friend class ParallelFormattingOutputFormat; + + virtual void consume(Chunk) = 0; + virtual void consumeTotals(Chunk) {} + virtual void consumeExtremes(Chunk) {} + virtual void finalize() {} + virtual void writePrefix() {} + + void writePrefixIfNot() + { + if (need_write_prefix) + { + writePrefix(); + need_write_prefix = false; + } + } + + WriteBuffer & out; + + Chunk current_chunk; + PortKind current_block_kind = PortKind::Main; + bool has_input = false; + bool finished = false; + bool finalized = false; + + /// Flush data on each consumed chunk. This is intended for interactive applications to output data as soon as it's ready. + bool auto_flush = false; + bool need_write_prefix = true; + + RowsBeforeLimitCounterPtr rows_before_limit_counter; + private: /// Counters for consumed chunks. Are used for QueryLog. size_t result_rows = 0; size_t result_bytes = 0; - - bool prefix_written = false; }; } diff --git a/src/Processors/Formats/IRowOutputFormat.cpp b/src/Processors/Formats/IRowOutputFormat.cpp index 6b7a9a46eaa..410d2d07d35 100644 --- a/src/Processors/Formats/IRowOutputFormat.cpp +++ b/src/Processors/Formats/IRowOutputFormat.cpp @@ -22,8 +22,6 @@ IRowOutputFormat::IRowOutputFormat(const Block & header, WriteBuffer & out_, con void IRowOutputFormat::consume(DB::Chunk chunk) { - writePrefixIfNot(); - auto num_rows = chunk.getNumRows(); const auto & columns = chunk.getColumns(); @@ -43,7 +41,6 @@ void IRowOutputFormat::consume(DB::Chunk chunk) void IRowOutputFormat::consumeTotals(DB::Chunk chunk) { - writePrefixIfNot(); writeSuffixIfNot(); auto num_rows = chunk.getNumRows(); @@ -59,7 +56,6 @@ void IRowOutputFormat::consumeTotals(DB::Chunk chunk) void IRowOutputFormat::consumeExtremes(DB::Chunk chunk) { - writePrefixIfNot(); writeSuffixIfNot(); auto num_rows = chunk.getNumRows(); @@ -76,7 +72,6 @@ void IRowOutputFormat::consumeExtremes(DB::Chunk chunk) void IRowOutputFormat::finalize() { - writePrefixIfNot(); writeSuffixIfNot(); writeLastSuffix(); } diff --git a/src/Processors/Formats/IRowOutputFormat.h b/src/Processors/Formats/IRowOutputFormat.h index 18575419cd0..2e658c452f6 100644 --- a/src/Processors/Formats/IRowOutputFormat.h +++ b/src/Processors/Formats/IRowOutputFormat.h @@ -26,41 +26,13 @@ class IRowOutputFormat : public IOutputFormat public: using Params = RowOutputFormatParams; -private: - bool prefix_written = false; - bool suffix_written = false; - protected: - DataTypes types; - Serializations serializations; - Params params; - - bool first_row = true; - + IRowOutputFormat(const Block & header, WriteBuffer & out_, const Params & params_); void consume(Chunk chunk) override; void consumeTotals(Chunk chunk) override; void consumeExtremes(Chunk chunk) override; void finalize() override; - void writePrefixIfNot() - { - if (!prefix_written) - writePrefix(); - - prefix_written = true; - } - - void writeSuffixIfNot() - { - if (!suffix_written) - writeSuffix(); - - suffix_written = true; - } - -public: - IRowOutputFormat(const Block & header, WriteBuffer & out_, const Params & params_); - /** Write a row. * Default implementation calls methods to write single values and delimiters * (except delimiter between rows (writeRowBetweenDelimiter())). @@ -78,7 +50,7 @@ public: virtual void writeRowStartDelimiter() {} /// delimiter before each row virtual void writeRowEndDelimiter() {} /// delimiter after each row virtual void writeRowBetweenDelimiter() {} /// delimiter between rows - virtual void writePrefix() {} /// delimiter before resultset + virtual void writePrefix() override {} /// delimiter before resultset virtual void writeSuffix() {} /// delimiter after resultset virtual void writeBeforeTotals() {} virtual void writeAfterTotals() {} @@ -86,6 +58,22 @@ public: virtual void writeAfterExtremes() {} virtual void writeLastSuffix() {} /// Write something after resultset, totals end extremes. + DataTypes types; + Serializations serializations; + Params params; + + bool first_row = true; + +private: + void writeSuffixIfNot() + { + if (!suffix_written) + writeSuffix(); + + suffix_written = true; + } + + bool suffix_written = false; }; } diff --git a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h index 1136937e65b..44e18e3f852 100644 --- a/src/Processors/Formats/Impl/ArrowBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockInputFormat.h @@ -24,10 +24,9 @@ public: String getName() const override { return "ArrowBlockInputFormat"; } -protected: +private: Chunk generate() override; -private: // Whether to use ArrowStream format bool stream; // This field is only used for ArrowStream format diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h index 154292d838f..4534575e735 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h @@ -21,19 +21,20 @@ public: ArrowBlockOutputFormat(WriteBuffer & out_, const Block & header_, bool stream_, const FormatSettings & format_settings_); String getName() const override { return "ArrowBlockOutputFormat"; } - void consume(Chunk) override; - void finalize() override; String getContentType() const override { return "application/octet-stream"; } private: + void consume(Chunk) override; + void finalize() override; + + void prepareWriter(const std::shared_ptr & schema); + bool stream; const FormatSettings format_settings; std::shared_ptr arrow_ostream; std::shared_ptr writer; std::unique_ptr ch_column_to_arrow_column; - - void prepareWriter(const std::shared_ptr & schema); }; } diff --git a/src/Processors/Formats/Impl/AvroRowInputFormat.h b/src/Processors/Formats/Impl/AvroRowInputFormat.h index 5617b4a7661..2de11178e96 100644 --- a/src/Processors/Formats/Impl/AvroRowInputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowInputFormat.h @@ -107,12 +107,13 @@ class AvroRowInputFormat : public IRowInputFormat { public: AvroRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_); - bool readRow(MutableColumns & columns, RowReadExtension & ext) override; - void readPrefix() override; String getName() const override { return "AvroRowInputFormat"; } private: + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + void readPrefix() override; + std::unique_ptr file_reader_ptr; std::unique_ptr deserializer_ptr; bool allow_missing_fields; @@ -128,14 +129,16 @@ class AvroConfluentRowInputFormat : public IRowInputFormat { public: AvroConfluentRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_); - virtual bool readRow(MutableColumns & columns, RowReadExtension & ext) override; String getName() const override { return "AvroConfluentRowInputFormat"; } class SchemaRegistry; -protected: + +private: + virtual bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + bool allowSyncAfterError() const override { return true; } void syncAfterError() override; -private: + std::shared_ptr schema_registry; using SchemaId = uint32_t; std::unordered_map deserializer_cache; diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp index 467738f49e8..fd7b2404c77 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.cpp @@ -428,7 +428,6 @@ void AvroRowOutputFormat::consumeImpl(DB::Chunk chunk) auto num_rows = chunk.getNumRows(); const auto & columns = chunk.getColumns(); - writePrefixIfNot(); for (size_t row = 0; row < num_rows; ++row) { write(columns, row); @@ -447,7 +446,7 @@ void AvroRowOutputFormat::consumeImplWithCallback(DB::Chunk chunk) /// used by WriteBufferToKafkaProducer to obtain auxiliary data /// from the starting row of a file - writePrefix(); + writePrefixIfNot(); for (size_t row_in_file = 0; row_in_file < settings.avro.output_rows_in_file && row < num_rows; ++row, ++row_in_file) @@ -457,6 +456,7 @@ void AvroRowOutputFormat::consumeImplWithCallback(DB::Chunk chunk) file_writer_ptr->flush(); writeSuffix(); + need_write_prefix = true; params.callback(columns, current_row); } diff --git a/src/Processors/Formats/Impl/AvroRowOutputFormat.h b/src/Processors/Formats/Impl/AvroRowOutputFormat.h index a3e8493f757..b5583406cb8 100644 --- a/src/Processors/Formats/Impl/AvroRowOutputFormat.h +++ b/src/Processors/Formats/Impl/AvroRowOutputFormat.h @@ -51,12 +51,13 @@ public: void consume(Chunk) override; String getName() const override { return "AvroRowOutputFormat"; } + +private: void write(const Columns & columns, size_t row_num) override; void writeField(const IColumn &, const ISerialization &, size_t) override {} virtual void writePrefix() override; virtual void writeSuffix() override; -private: FormatSettings settings; AvroSerializer serializer; std::unique_ptr file_writer_ptr; diff --git a/src/Processors/Formats/Impl/BinaryRowOutputFormat.h b/src/Processors/Formats/Impl/BinaryRowOutputFormat.h index 36a62098b75..0edfd4bfcf8 100644 --- a/src/Processors/Formats/Impl/BinaryRowOutputFormat.h +++ b/src/Processors/Formats/Impl/BinaryRowOutputFormat.h @@ -21,12 +21,12 @@ public: String getName() const override { return "BinaryRowOutputFormat"; } + String getContentType() const override { return "application/octet-stream"; } + +private: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writePrefix() override; - String getContentType() const override { return "application/octet-stream"; } - -protected: bool with_names; bool with_types; }; diff --git a/src/Processors/Formats/Impl/CSVRowInputFormat.h b/src/Processors/Formats/Impl/CSVRowInputFormat.h index f239464485a..d7c557b58d8 100644 --- a/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -25,12 +25,13 @@ public: String getName() const override { return "CSVRowInputFormat"; } +private: bool allowSyncAfterError() const override { return true; } void syncAfterError() override; -private: bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override; bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override; + bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override { return *pos != '\n' && *pos != '\r' && *pos != format_settings.csv.delimiter && *pos != ' ' && *pos != '\t'; diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp index b300928e569..790994cb240 100644 --- a/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CSVRowOutputFormat.cpp @@ -31,7 +31,7 @@ void CSVRowOutputFormat::writeLine(const std::vector & values) } } -void CSVRowOutputFormat::doWritePrefix() +void CSVRowOutputFormat::writePrefix() { const auto & sample = getPort(PortKind::Main).getHeader(); diff --git a/src/Processors/Formats/Impl/CSVRowOutputFormat.h b/src/Processors/Formats/Impl/CSVRowOutputFormat.h index 7f5d90203ea..dd9c2179f19 100644 --- a/src/Processors/Formats/Impl/CSVRowOutputFormat.h +++ b/src/Processors/Formats/Impl/CSVRowOutputFormat.h @@ -24,14 +24,6 @@ public: String getName() const override { return "CSVRowOutputFormat"; } - void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; - void writeFieldDelimiter() override; - void writeRowEndDelimiter() override; - void writeBeforeTotals() override; - void writeBeforeExtremes() override; - - void doWritePrefix() override; - /// https://www.iana.org/assignments/media-types/text/csv String getContentType() const override { @@ -39,6 +31,13 @@ public: } private: + void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowEndDelimiter() override; + void writeBeforeTotals() override; + void writeBeforeExtremes() override; + + void writePrefix() override; void writeLine(const std::vector & values); bool with_names; diff --git a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h index fc30cf11237..4c0f34d70a3 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h +++ b/src/Processors/Formats/Impl/CapnProtoRowInputFormat.h @@ -26,9 +26,9 @@ public: String getName() const override { return "CapnProtoRowInputFormat"; } +private: bool readRow(MutableColumns & columns, RowReadExtension &) override; -private: kj::Array readMessage(); std::shared_ptr parser; diff --git a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h index 0f321071d62..6e27426f2cc 100644 --- a/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h +++ b/src/Processors/Formats/Impl/CapnProtoRowOutputFormat.h @@ -35,11 +35,11 @@ public: String getName() const override { return "CapnProtoRowOutputFormat"; } +private: void write(const Columns & columns, size_t row_num) override; void writeField(const IColumn &, const ISerialization &, size_t) override { } -private: Names column_names; DataTypes column_types; capnp::StructSchema schema; diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h index c70d9efb178..96ad60b3fab 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.h @@ -18,14 +18,15 @@ class JSONAsStringRowInputFormat : public IRowInputFormat public: JSONAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_); - bool readRow(MutableColumns & columns, RowReadExtension & ext) override; String getName() const override { return "JSONAsStringRowInputFormat"; } void resetParser() override; +private: + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + void readPrefix() override; void readSuffix() override; -private: void readJSONObject(IColumn & column); PeekableReadBuffer buf; diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h index 373eb04f06c..e01a4f49b30 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h @@ -31,10 +31,10 @@ public: String getName() const override { return "JSONCompactEachRowRowInputFormat"; } +private: bool allowSyncAfterError() const override { return true; } void syncAfterError() override; -private: bool parseRowStartWithDiagnosticInfo(WriteBuffer & out) override; bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override; bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override; diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp index cdff7ff2070..c4645e0d63d 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.cpp @@ -81,7 +81,7 @@ void JSONCompactEachRowRowOutputFormat::writeLine(const std::vector & va writeRowEndDelimiter(); } -void JSONCompactEachRowRowOutputFormat::doWritePrefix() +void JSONCompactEachRowRowOutputFormat::writePrefix() { const auto & header = getPort(PortKind::Main).getHeader(); diff --git a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h index aa12ba7e809..6cb78bab49d 100644 --- a/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h @@ -26,7 +26,8 @@ public: String getName() const override { return "JSONCompactEachRowRowOutputFormat"; } - void doWritePrefix() override; +private: + void writePrefix() override; void writeTotals(const Columns & columns, size_t row_num) override; @@ -35,12 +36,10 @@ public: void writeRowStartDelimiter() override; void writeRowEndDelimiter() override; -protected: void consumeTotals(Chunk) override; /// No extremes. void consumeExtremes(Chunk) override {} -private: void writeLine(const std::vector & values); FormatSettings settings; diff --git a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h index 9bb433c50b1..961bd569d39 100644 --- a/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONCompactRowOutputFormat.h @@ -25,6 +25,7 @@ public: String getName() const override { return "JSONCompactRowOutputFormat"; } +private: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writeFieldDelimiter() override; void writeRowStartDelimiter() override; @@ -33,7 +34,6 @@ public: void writeBeforeTotals() override; void writeAfterTotals() override; -protected: void writeExtremesElement(const char * title, const Columns & columns, size_t row_num) override; void writeTotalsField(const IColumn & column, const ISerialization & serialization, size_t row_num) override diff --git a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h index 29a6ce6ecb8..9810f2dc765 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowInputFormat.h @@ -28,16 +28,16 @@ public: bool yield_strings_); String getName() const override { return "JSONEachRowRowInputFormat"; } + void resetParser() override; +private: void readPrefix() override; void readSuffix() override; bool readRow(MutableColumns & columns, RowReadExtension & ext) override; bool allowSyncAfterError() const override { return true; } void syncAfterError() override; - void resetParser() override; -private: const String & columnName(size_t i) const; size_t columnIndex(const StringRef & name, size_t key_index); bool advanceToNextKey(size_t key_index); diff --git a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h index 10b15f3e7b2..23fb506c220 100644 --- a/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowRowOutputFormat.h @@ -23,6 +23,7 @@ public: String getName() const override { return "JSONEachRowRowOutputFormat"; } +protected: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writeFieldDelimiter() override; void writeRowStartDelimiter() override; @@ -31,7 +32,6 @@ public: void writePrefix() override; void writeSuffix() override; -protected: /// No totals and extremes. void consumeTotals(Chunk) override {} void consumeExtremes(Chunk) override {} diff --git a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.h b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.h index 3062d664199..3eac61c4109 100644 --- a/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONEachRowWithProgressRowOutputFormat.h @@ -9,11 +9,12 @@ class JSONEachRowWithProgressRowOutputFormat : public JSONEachRowRowOutputFormat public: using JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat; - void writeRowStartDelimiter() override; - void writeRowEndDelimiter() override; void onProgress(const Progress & value) override; private: + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + Progress progress; }; diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.h b/src/Processors/Formats/Impl/JSONRowOutputFormat.h index 75d4aa5d201..757cdf01a35 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.h +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.h @@ -25,6 +25,25 @@ public: String getName() const override { return "JSONRowOutputFormat"; } + void onProgress(const Progress & value) override; + + String getContentType() const override { return "application/json; charset=UTF-8"; } + + void flush() override + { + ostr->next(); + + if (validating_ostr) + out.next(); + } + + void setRowsBeforeLimit(size_t rows_before_limit_) override + { + applied_limit = true; + rows_before_limit = rows_before_limit_; + } + +protected: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writeFieldDelimiter() override; void writeRowStartDelimiter() override; @@ -44,25 +63,6 @@ public: void writeLastSuffix() override; - void flush() override - { - ostr->next(); - - if (validating_ostr) - out.next(); - } - - void setRowsBeforeLimit(size_t rows_before_limit_) override - { - applied_limit = true; - rows_before_limit = rows_before_limit_; - } - - void onProgress(const Progress & value) override; - - String getContentType() const override { return "application/json; charset=UTF-8"; } - -protected: virtual void writeTotalsField(const IColumn & column, const ISerialization & serialization, size_t row_num); virtual void writeExtremesElement(const char * title, const Columns & columns, size_t row_num); virtual void writeTotalsFieldDelimiter() { writeFieldDelimiter(); } @@ -70,7 +70,6 @@ protected: void writeRowsBeforeLimitAtLeast(); void writeStatistics(); - std::unique_ptr validating_ostr; /// Validates UTF-8 sequences, replaces bad sequences with replacement character. WriteBuffer * ostr; diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h index 7c0187bc3ff..1a6c6247558 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.h @@ -17,11 +17,12 @@ class LineAsStringRowInputFormat : public IRowInputFormat public: LineAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_); - bool readRow(MutableColumns & columns, RowReadExtension & ext) override; String getName() const override { return "LineAsStringRowInputFormat"; } void resetParser() override; private: + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + void readLineObject(IColumn & column); }; diff --git a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h index 0b2a4dd0b23..7a2aaf86f7d 100644 --- a/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h +++ b/src/Processors/Formats/Impl/MarkdownRowOutputFormat.h @@ -14,6 +14,9 @@ class MarkdownRowOutputFormat : public IRowOutputFormat public: MarkdownRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_); + String getName() const override { return "MarkdownRowOutputFormat"; } + +private: /// Write higher part of markdown table like this: /// |columnName1|columnName2|...|columnNameN| /// |:-:|:-:|...|:-:| @@ -29,9 +32,7 @@ public: void writeRowEndDelimiter() override ; void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; - String getName() const override { return "MarkdownRowOutputFormat"; } -protected: const FormatSettings format_settings; }; diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h index fa5c2e74584..d2d500a4480 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.h +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.h @@ -59,11 +59,12 @@ class MsgPackRowInputFormat : public IRowInputFormat public: MsgPackRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_); - bool readRow(MutableColumns & columns, RowReadExtension & ext) override; String getName() const override { return "MagPackRowInputFormat"; } void resetParser() override; private: + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + bool readObject(); PeekableReadBuffer buf; diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h index 2766eb144e4..17d055818e9 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.h @@ -22,11 +22,11 @@ public: String getName() const override { return "MsgPackRowOutputFormat"; } +private: void write(const Columns & columns, size_t row_num) override; void writeField(const IColumn &, const ISerialization &, size_t) override {} void serializeField(const IColumn & column, DataTypePtr data_type, size_t row_num); -private: msgpack::packer packer; }; diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index 0b366244611..cdbac9f06fb 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -31,13 +31,8 @@ void MySQLOutputFormat::setContext(ContextPtr context_) context = context_; } -void MySQLOutputFormat::initialize() +void MySQLOutputFormat::writePrefix() { - if (initialized) - return; - - initialized = true; - const auto & header = getPort(PortKind::Main).getHeader(); data_types = header.getDataTypes(); @@ -66,8 +61,6 @@ void MySQLOutputFormat::initialize() void MySQLOutputFormat::consume(Chunk chunk) { - initialize(); - for (size_t i = 0; i < chunk.getNumRows(); i++) { ProtocolText::ResultSetRow row_packet(serializations, chunk.getColumns(), i); diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.h b/src/Processors/Formats/Impl/MySQLOutputFormat.h index a8e1ada3d6a..6f4337486e2 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.h +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.h @@ -26,15 +26,13 @@ public: void setContext(ContextPtr context_); - void consume(Chunk) override; - void finalize() override; void flush() override; - void doWritePrefix() override { initialize(); } private: - void initialize(); + void consume(Chunk) override; + void finalize() override; + void writePrefix() override; - bool initialized = false; uint32_t client_capabilities = 0; uint8_t * sequence_id = nullptr; uint8_t dummy_sequence_id = 0; diff --git a/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp b/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp index 0e486715c98..a82285c1c19 100644 --- a/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.cpp @@ -64,21 +64,14 @@ void ODBCDriver2BlockOutputFormat::write(Chunk chunk, PortKind port_kind) void ODBCDriver2BlockOutputFormat::consume(Chunk chunk) { - writePrefixIfNot(); write(std::move(chunk), PortKind::Main); } void ODBCDriver2BlockOutputFormat::consumeTotals(Chunk chunk) { - writePrefixIfNot(); write(std::move(chunk), PortKind::Totals); } -void ODBCDriver2BlockOutputFormat::finalize() -{ - writePrefixIfNot(); -} - void ODBCDriver2BlockOutputFormat::writePrefix() { const auto & header = getPort(PortKind::Main).getHeader(); diff --git a/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.h b/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.h index 4545e429cc2..de6ea22dfd7 100644 --- a/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.h @@ -24,30 +24,20 @@ public: String getName() const override { return "ODBCDriver2BlockOutputFormat"; } - void consume(Chunk) override; - void consumeTotals(Chunk) override; - void finalize() override; - std::string getContentType() const override { return "application/octet-stream"; } private: + void consume(Chunk) override; + void consumeTotals(Chunk) override; + void writePrefix() override; + const FormatSettings format_settings; - bool prefix_written = false; - - void writePrefixIfNot() - { - if (!prefix_written) - writePrefix(); - - prefix_written = true; - } void writeRow(const Serializations & serializations, const Columns & columns, size_t row_idx, std::string & buffer); void write(Chunk chunk, PortKind port_kind); - void writePrefix(); }; diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h index c131d724450..fcdc0e9c6d6 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h @@ -37,10 +37,11 @@ public: ORCBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_); String getName() const override { return "ORCBlockOutputFormat"; } + +private: void consume(Chunk chunk) override; void finalize() override; -private: ORC_UNIQUE_PTR getORCType(const DataTypePtr & type, const std::string & column_name); /// ConvertFunc is needed for type UInt8, because firstly UInt8 (char8_t) must be diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp index f7723e3f1d2..c7b7a4d2814 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp @@ -171,7 +171,7 @@ namespace DB { case ProcessingUnitType::START : { - formatter->doWritePrefix(); + formatter->writePrefix(); break; } case ProcessingUnitType::PLAIN : @@ -191,7 +191,7 @@ namespace DB } case ProcessingUnitType::FINALIZE : { - formatter->doWriteSuffix(); + formatter->finalize(); break; } } diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h index 65f4a329505..0f8bff9b7d9 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h @@ -95,7 +95,7 @@ public: need_flush = true; } - void doWritePrefix() override + void writePrefix() override { addChunk(Chunk{}, ProcessingUnitType::START, /*can_throw_exception*/ true); } @@ -114,7 +114,7 @@ public: return internal_formatter_creator(buffer)->getContentType(); } -protected: +private: void consume(Chunk chunk) override final { addChunk(std::move(chunk), ProcessingUnitType::PLAIN, /*can_throw_exception*/ true); @@ -132,7 +132,6 @@ protected: void finalize() override; -private: InternalFormatterCreator internal_formatter_creator; /// Status to synchronize multiple threads. diff --git a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h index 3355b7334dc..264beba8589 100644 --- a/src/Processors/Formats/Impl/ParallelParsingInputFormat.h +++ b/src/Processors/Formats/Impl/ParallelParsingInputFormat.h @@ -117,7 +117,7 @@ public: String getName() const override final { return "ParallelParsingBlockInputFormat"; } -protected: +private: Chunk generate() override final; @@ -137,8 +137,6 @@ protected: finishAndWait(); } -private: - class InternalParser { public: diff --git a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h index bae380f2c80..472aec66da3 100644 --- a/src/Processors/Formats/Impl/ParquetBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockInputFormat.h @@ -23,13 +23,11 @@ public: String getName() const override { return "ParquetBlockInputFormat"; } -protected: +private: Chunk generate() override; -private: void prepareReader(); -private: std::unique_ptr file_reader; int row_group_total = 0; // indices of columns to read from Parquet file diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h index 80d11b540b1..f58db3bd202 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h @@ -30,12 +30,13 @@ public: ParquetBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_); String getName() const override { return "ParquetBlockOutputFormat"; } - void consume(Chunk) override; - void finalize() override; String getContentType() const override { return "application/octet-stream"; } private: + void consume(Chunk) override; + void finalize() override; + const FormatSettings format_settings; std::unique_ptr file_writer; diff --git a/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp b/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp index a5f92e41da4..f46488fd0a8 100644 --- a/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PostgreSQLOutputFormat.cpp @@ -11,12 +11,8 @@ PostgreSQLOutputFormat::PostgreSQLOutputFormat(WriteBuffer & out_, const Block & { } -void PostgreSQLOutputFormat::doWritePrefix() +void PostgreSQLOutputFormat::writePrefix() { - if (initialized) - return; - - initialized = true; const auto & header = getPort(PortKind::Main).getHeader(); auto data_types = header.getDataTypes(); @@ -37,8 +33,6 @@ void PostgreSQLOutputFormat::doWritePrefix() void PostgreSQLOutputFormat::consume(Chunk chunk) { - doWritePrefix(); - for (size_t i = 0; i != chunk.getNumRows(); ++i) { const Columns & columns = chunk.getColumns(); @@ -61,8 +55,6 @@ void PostgreSQLOutputFormat::consume(Chunk chunk) } } -void PostgreSQLOutputFormat::finalize() {} - void PostgreSQLOutputFormat::flush() { message_transport.flush(); diff --git a/src/Processors/Formats/Impl/PostgreSQLOutputFormat.h b/src/Processors/Formats/Impl/PostgreSQLOutputFormat.h index 257fbdff341..f5fd55530b9 100644 --- a/src/Processors/Formats/Impl/PostgreSQLOutputFormat.h +++ b/src/Processors/Formats/Impl/PostgreSQLOutputFormat.h @@ -17,13 +17,11 @@ public: String getName() const override {return "PostgreSQLOutputFormat";} - void doWritePrefix() override; - void consume(Chunk) override; - void finalize() override; void flush() override; private: - bool initialized = false; + void writePrefix() override; + void consume(Chunk) override; FormatSettings format_settings; PostgreSQLProtocol::Messaging::MessageTransport message_transport; diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h index 02b438d2571..8d317277251 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h @@ -22,13 +22,13 @@ public: String getName() const override { return "PrettyBlockOutputFormat"; } +protected: void consume(Chunk) override; void consumeTotals(Chunk) override; void consumeExtremes(Chunk) override; void finalize() override; -protected: size_t total_rows = 0; size_t terminal_width = 0; bool suffix_written = false; diff --git a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h index 96344397a0c..1779a20e122 100644 --- a/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h @@ -16,7 +16,7 @@ public: PrettyCompactBlockOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & format_settings_, bool mono_block_); String getName() const override { return "PrettyCompactBlockOutputFormat"; } -protected: +private: void write(const Chunk & chunk, PortKind port_kind) override; void writeHeader(const Block & block, const Widths & max_widths, const Widths & name_widths); void writeBottom(const Widths & max_widths); @@ -28,7 +28,6 @@ protected: const WidthsPerColumn & widths, const Widths & max_widths); -private: bool mono_block; /// For mono_block == true only Chunk mono_chunk; diff --git a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.h index a041d324fd3..b3090497783 100644 --- a/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettySpaceBlockOutputFormat.h @@ -16,7 +16,7 @@ public: String getName() const override { return "PrettySpaceBlockOutputFormat"; } -protected: +private: void write(const Chunk & chunk, PortKind port_kind) override; void writeSuffix() override; }; diff --git a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h index 553a2dfd7f0..6f465e3f0b8 100644 --- a/src/Processors/Formats/Impl/ProtobufRowInputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowInputFormat.h @@ -32,11 +32,11 @@ public: String getName() const override { return "ProtobufRowInputFormat"; } +private: bool readRow(MutableColumns & columns, RowReadExtension &) override; bool allowSyncAfterError() const override; void syncAfterError() override; -private: std::unique_ptr reader; std::vector missing_column_indices; std::unique_ptr serializer; diff --git a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h index 5323aa56323..97b727842a7 100644 --- a/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ProtobufRowOutputFormat.h @@ -39,11 +39,12 @@ public: String getName() const override { return "ProtobufRowOutputFormat"; } - void write(const Columns & columns, size_t row_num) override; - void writeField(const IColumn &, const ISerialization &, size_t) override {} std::string getContentType() const override { return "application/octet-stream"; } private: + void write(const Columns & columns, size_t row_num) override; + void writeField(const IColumn &, const ISerialization &, size_t) override {} + std::unique_ptr writer; std::unique_ptr serializer; const bool allow_multiple_rows; diff --git a/src/Processors/Formats/Impl/RawBLOBRowInputFormat.h b/src/Processors/Formats/Impl/RawBLOBRowInputFormat.h index fd2c849687a..343af9f4068 100644 --- a/src/Processors/Formats/Impl/RawBLOBRowInputFormat.h +++ b/src/Processors/Formats/Impl/RawBLOBRowInputFormat.h @@ -16,8 +16,10 @@ class RawBLOBRowInputFormat : public IRowInputFormat public: RawBLOBRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_); - bool readRow(MutableColumns & columns, RowReadExtension &) override; String getName() const override { return "RawBLOBRowInputFormat"; } + +private: + bool readRow(MutableColumns & columns, RowReadExtension &) override; }; } diff --git a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h index 7a29c62e4d8..2c34595c1a4 100644 --- a/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h +++ b/src/Processors/Formats/Impl/RawBLOBRowOutputFormat.h @@ -34,6 +34,7 @@ public: String getName() const override { return "RawBLOBRowOutputFormat"; } +private: void writeField(const IColumn & column, const ISerialization &, size_t row_num) override; }; diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/src/Processors/Formats/Impl/RegexpRowInputFormat.h index 0cd8778e499..f8923d9c217 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.h +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -30,11 +30,11 @@ public: RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_); String getName() const override { return "RegexpRowInputFormat"; } - - bool readRow(MutableColumns & columns, RowReadExtension & ext) override; void resetParser() override; private: + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + bool readField(size_t index, MutableColumns & columns); void readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext); static ColumnFormat stringToFormat(const String & format); diff --git a/src/Processors/Formats/Impl/TSKVRowInputFormat.h b/src/Processors/Formats/Impl/TSKVRowInputFormat.h index bc537158d9b..7d732bae691 100644 --- a/src/Processors/Formats/Impl/TSKVRowInputFormat.h +++ b/src/Processors/Formats/Impl/TSKVRowInputFormat.h @@ -27,14 +27,14 @@ public: String getName() const override { return "TSKVRowInputFormat"; } + void resetParser() override; + +private: void readPrefix() override; bool readRow(MutableColumns & columns, RowReadExtension &) override; bool allowSyncAfterError() const override { return true; } void syncAfterError() override; - void resetParser() override; - -private: const FormatSettings format_settings; /// Buffer for the read from the stream the field name. Used when you have to copy it. diff --git a/src/Processors/Formats/Impl/TSKVRowOutputFormat.h b/src/Processors/Formats/Impl/TSKVRowOutputFormat.h index 24c4e5ca866..980e36c7e25 100644 --- a/src/Processors/Formats/Impl/TSKVRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TSKVRowOutputFormat.h @@ -18,10 +18,10 @@ public: String getName() const override { return "TSKVRowOutputFormat"; } +private: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writeRowEndDelimiter() override; -protected: NamesAndTypes fields; size_t field_number = 0; }; diff --git a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h index 11a788bc900..6e2e283e792 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h @@ -21,12 +21,10 @@ public: String getName() const override { return "TabSeparatedRowInputFormat"; } +private: bool allowSyncAfterError() const override { return true; } void syncAfterError() override; -private: - bool is_raw; - bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; @@ -48,6 +46,8 @@ private: bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override; bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override; bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override { return *pos != '\n' && *pos != '\t'; } + + bool is_raw; }; } diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index df0c19ad409..5d87f5a0b14 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -30,7 +30,7 @@ void TabSeparatedRowOutputFormat::writeLine(const std::vector & values) } } -void TabSeparatedRowOutputFormat::doWritePrefix() +void TabSeparatedRowOutputFormat::writePrefix() { const auto & header = getPort(PortKind::Main).getHeader(); diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h index 7dcc6529f1c..eeada54d74e 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h @@ -29,24 +29,22 @@ public: String getName() const override { return "TabSeparatedRowOutputFormat"; } + /// https://www.iana.org/assignments/media-types/text/tab-separated-values + String getContentType() const override { return "text/tab-separated-values; charset=UTF-8"; } + +protected: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writeFieldDelimiter() override; void writeRowEndDelimiter() override; void writeBeforeTotals() override; void writeBeforeExtremes() override; - void doWritePrefix() override; - - /// https://www.iana.org/assignments/media-types/text/tab-separated-values - String getContentType() const override { return "text/tab-separated-values; charset=UTF-8"; } - -private: + void writePrefix() override; void writeLine(const std::vector & values); + bool with_names; bool with_types; bool is_raw; - -protected: const FormatSettings format_settings; }; diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp index db5db4701a9..dae3398a36c 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -147,8 +147,6 @@ template void TemplateBlockOutputFormat::writeValue(U v void TemplateBlockOutputFormat::consume(Chunk chunk) { - doWritePrefix(); - size_t rows = chunk.getNumRows(); for (size_t i = 0; i < rows; ++i) @@ -161,13 +159,9 @@ void TemplateBlockOutputFormat::consume(Chunk chunk) } } -void TemplateBlockOutputFormat::doWritePrefix() +void TemplateBlockOutputFormat::writePrefix() { - if (need_write_prefix) - { - writeString(format.delimiters.front(), out); - need_write_prefix = false; - } + writeString(format.delimiters.front(), out); } void TemplateBlockOutputFormat::finalize() @@ -175,8 +169,6 @@ void TemplateBlockOutputFormat::finalize() if (finalized) return; - doWritePrefix(); - size_t parts = format.format_idx_to_column_idx.size(); for (size_t i = 0; i < parts; ++i) diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h index 0d41b8888d4..224a8d20176 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h @@ -20,8 +20,6 @@ public: String getName() const override { return "TemplateBlockOutputFormat"; } - void doWritePrefix() override; - void setRowsBeforeLimit(size_t rows_before_limit_) override { rows_before_limit = rows_before_limit_; rows_before_limit_set = true; } void onProgress(const Progress & progress_) override { progress.incrementPiecewiseAtomically(progress_); } @@ -40,7 +38,8 @@ public: static ResultsetPart stringToResultsetPart(const String & part); -protected: +private: + void writePrefix() override; void consume(Chunk chunk) override; void consumeTotals(Chunk chunk) override { totals = std::move(chunk); } void consumeExtremes(Chunk chunk) override { extremes = std::move(chunk); } @@ -50,7 +49,6 @@ protected: void serializeField(const IColumn & column, const ISerialization & serialization, size_t row_num, ColumnFormat format); template void writeValue(U value, ColumnFormat col_format); -protected: const FormatSettings settings; Serializations serializations; @@ -65,7 +63,6 @@ protected: Stopwatch watch; size_t row_count = 0; - bool need_write_prefix = true; std::string row_between_delimiter; }; diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.h b/src/Processors/Formats/Impl/TemplateRowInputFormat.h index 322f8570ab7..428bef40fcf 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.h +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.h @@ -22,6 +22,9 @@ public: String getName() const override { return "TemplateRowInputFormat"; } + void resetParser() override; + +private: bool readRow(MutableColumns & columns, RowReadExtension & extra) override; void readPrefix() override; @@ -29,9 +32,6 @@ public: bool allowSyncAfterError() const override; void syncAfterError() override; - void resetParser() override; - -private: bool deserializeField(const DataTypePtr & type, const SerializationPtr & serialization, IColumn & column, size_t file_column); @@ -51,7 +51,6 @@ private: void skipToNextDelimiterOrEof(const String & delimiter); -private: PeekableReadBuffer buf; const DataTypes data_types; diff --git a/src/Processors/Formats/Impl/ValuesRowOutputFormat.h b/src/Processors/Formats/Impl/ValuesRowOutputFormat.h index 493ce458b1e..8d89854d43c 100644 --- a/src/Processors/Formats/Impl/ValuesRowOutputFormat.h +++ b/src/Processors/Formats/Impl/ValuesRowOutputFormat.h @@ -19,13 +19,13 @@ public: String getName() const override { return "ValuesRowOutputFormat"; } +private: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writeFieldDelimiter() override; void writeRowStartDelimiter() override; void writeRowEndDelimiter() override; void writeRowBetweenDelimiter() override; -private: const FormatSettings format_settings; }; diff --git a/src/Processors/Formats/Impl/VerticalRowOutputFormat.h b/src/Processors/Formats/Impl/VerticalRowOutputFormat.h index 9e89f677f87..075c943cd76 100644 --- a/src/Processors/Formats/Impl/VerticalRowOutputFormat.h +++ b/src/Processors/Formats/Impl/VerticalRowOutputFormat.h @@ -22,6 +22,7 @@ public: String getName() const override { return "VerticalRowOutputFormat"; } +private: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writeRowStartDelimiter() override; void writeRowBetweenDelimiter() override; @@ -34,8 +35,7 @@ public: void writeBeforeTotals() override; void writeBeforeExtremes() override; -protected: - virtual void writeValue(const IColumn & column, const ISerialization & serialization, size_t row_num) const; + void writeValue(const IColumn & column, const ISerialization & serialization, size_t row_num) const; /// For totals and extremes. void writeSpecialRow(const Columns & columns, size_t row_num, const char * title); diff --git a/src/Processors/Formats/Impl/XMLRowOutputFormat.h b/src/Processors/Formats/Impl/XMLRowOutputFormat.h index 8ca4721c459..7f08fc12bd1 100644 --- a/src/Processors/Formats/Impl/XMLRowOutputFormat.h +++ b/src/Processors/Formats/Impl/XMLRowOutputFormat.h @@ -20,6 +20,7 @@ public: String getName() const override { return "XMLRowOutputFormat"; } +private: void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; void writeRowStartDelimiter() override; void writeRowEndDelimiter() override; @@ -54,7 +55,6 @@ public: String getContentType() const override { return "application/xml; charset=UTF-8"; } -protected: void writeExtremesElement(const char * title, const Columns & columns, size_t row_num); void writeRowsBeforeLimitAtLeast(); void writeStatistics(); diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index 0fd83238f5f..1cd349b8183 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -30,8 +30,6 @@ public: const Params & params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_); - bool readRow(MutableColumns & columns, RowReadExtension & ext) override; - void readPrefix() override; void resetParser() override; protected: @@ -67,6 +65,9 @@ protected: DataTypes data_types; private: + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; + void readPrefix() override; + bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override; void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override; diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index ba2644e0fba..7d68482ed78 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1090,7 +1090,6 @@ namespace write_buffer.emplace(*result.mutable_output()); output_format_processor = query_context->getOutputFormat(output_format, *write_buffer, header); - output_format_processor->doWritePrefix(); Stopwatch after_send_progress; /// Unless the input() function is used we are not going to receive input data anymore. @@ -1169,7 +1168,7 @@ namespace executor->execute(); } - output_format_processor->doWriteSuffix(); + output_format_processor->finish(); } void Call::finishQuery() @@ -1380,9 +1379,8 @@ namespace WriteBufferFromString buf{*result.mutable_totals()}; auto format = query_context->getOutputFormat(output_format, buf, totals); - format->doWritePrefix(); format->write(materializeBlock(totals)); - format->doWriteSuffix(); + format->finish(); } void Call::addExtremesToResult(const Block & extremes) @@ -1392,9 +1390,8 @@ namespace WriteBufferFromString buf{*result.mutable_extremes()}; auto format = query_context->getOutputFormat(output_format, buf, extremes); - format->doWritePrefix(); format->write(materializeBlock(extremes)); - format->doWriteSuffix(); + format->finish(); } void Call::addProfileInfoToResult(const ProfileInfo & info) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index b46668a233b..e2db5109585 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -210,11 +210,6 @@ public: void consume(Chunk chunk) override { - if (is_first_chunk) - { - writer->doWritePrefix(); - is_first_chunk = false; - } writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } @@ -222,7 +217,7 @@ public: { try { - writer->doWriteSuffix(); + writer->finish(); writer->flush(); write_buf->sync(); write_buf->finalize(); @@ -237,7 +232,6 @@ public: private: std::unique_ptr write_buf; OutputFormatPtr writer; - bool is_first_chunk = true; }; diff --git a/src/Storages/Kafka/KafkaBlockOutputStream.cpp b/src/Storages/Kafka/KafkaBlockOutputStream.cpp index d3c51fef9b7..79163ff8d17 100644 --- a/src/Storages/Kafka/KafkaBlockOutputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockOutputStream.cpp @@ -42,7 +42,7 @@ void KafkaSink::consume(Chunk chunk) void KafkaSink::onFinish() { if (format) - format->doWriteSuffix(); + format->finish(); //flush(); if (buffer) diff --git a/src/Storages/RabbitMQ/RabbitMQSink.cpp b/src/Storages/RabbitMQ/RabbitMQSink.cpp index 2b8d5ab3810..fc3dccb57fa 100644 --- a/src/Storages/RabbitMQ/RabbitMQSink.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSink.cpp @@ -49,7 +49,7 @@ void RabbitMQSink::consume(Chunk chunk) void RabbitMQSink::onFinish() { - format->doWriteSuffix(); + format->finish(); if (buffer) buffer->updateMaxWait(); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index 4a1eac2a39e..af46e2d3ba1 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -621,26 +621,21 @@ public: naked_buffer = std::make_unique(paths[0], DBMS_DEFAULT_BUFFER_SIZE, flags); } - /// In case of CSVWithNames we have already written prefix. - if (naked_buffer->size()) - prefix_written = true; + /// In case of formats with prefixes if file is not empty we have already written prefix. + bool do_not_write_prefix = naked_buffer->size(); write_buf = wrapWriteBufferWithCompressionMethod(std::move(naked_buffer), compression_method, 3); writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format_name, *write_buf, metadata_snapshot->getSampleBlock(), context, {}, format_settings); + + if (do_not_write_prefix) + writer->doNotWritePrefix(); } String getName() const override { return "StorageFileSink"; } - void onStart() override - { - if (!prefix_written) - writer->doWritePrefix(); - prefix_written = true; - } - void consume(Chunk chunk) override { writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); @@ -648,7 +643,7 @@ public: void onFinish() override { - writer->doWriteSuffix(); + writer->finish(); } // void flush() override @@ -662,7 +657,6 @@ private: std::unique_ptr write_buf; OutputFormatPtr writer; - bool prefix_written{false}; int table_fd; bool use_table_fd; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 80011cde077..99391963fd1 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -321,11 +321,6 @@ public: void consume(Chunk chunk) override { - if (is_first_chunk) - { - writer->doWritePrefix(); - is_first_chunk = false; - } writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } @@ -333,7 +328,7 @@ public: { try { - writer->doWriteSuffix(); + writer->finish(); writer->flush(); write_buf->finalize(); } @@ -350,7 +345,6 @@ private: std::optional format_settings; std::unique_ptr write_buf; OutputFormatPtr writer; - bool is_first_chunk = true; }; diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 66033f7a7d6..6882d1acf0e 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -226,18 +226,12 @@ StorageURLSink::StorageURLSink( void StorageURLSink::consume(Chunk chunk) { - if (is_first_chunk) - { - writer->doWritePrefix(); - is_first_chunk = false; - } - writer->write(getHeader().cloneWithColumns(chunk.detachColumns())); } void StorageURLSink::onFinish() { - writer->doWriteSuffix(); + writer->finish(); writer->flush(); write_buf->finalize(); } diff --git a/src/Storages/StorageURL.h b/src/Storages/StorageURL.h index a79100c8d70..cf72352a183 100644 --- a/src/Storages/StorageURL.h +++ b/src/Storages/StorageURL.h @@ -110,8 +110,6 @@ public: private: std::unique_ptr write_buf; OutputFormatPtr writer; - - bool is_first_chunk = true; }; class StorageURL : public shared_ptr_helper, public IStorageURLBase diff --git a/tests/queries/0_stateless/01553_settings_early_apply.sql b/tests/queries/0_stateless/01553_settings_early_apply.sql index 48178c38f33..13bb2d30d4d 100644 --- a/tests/queries/0_stateless/01553_settings_early_apply.sql +++ b/tests/queries/0_stateless/01553_settings_early_apply.sql @@ -1,5 +1,9 @@ select * from numbers(100) settings max_result_rows = 1; -- { serverError 396 } select * from numbers(100) FORMAT JSON settings max_result_rows = 1; -- { serverError 396 } +select * from numbers(100) FORMAT TSVWithNamesAndTypes settings max_result_rows = 1; -- { serverError 396 } +select * from numbers(100) FORMAT CSVWithNamesAndTypes settings max_result_rows = 1; -- { serverError 396 } +select * from numbers(100) FORMAT JSONCompactEachRowWithNamesAndTypes settings max_result_rows = 1; -- { serverError 396 } +select * from numbers(100) FORMAT XML settings max_result_rows = 1; -- { serverError 396 } SET max_result_rows = 1; select * from numbers(10); -- { serverError 396 } diff --git a/tests/queries/0_stateless/02113_format_row.reference b/tests/queries/0_stateless/02113_format_row.reference new file mode 100644 index 00000000000..0ac3a15b115 --- /dev/null +++ b/tests/queries/0_stateless/02113_format_row.reference @@ -0,0 +1,20 @@ +0\t1970-01-01\n +1\t1970-01-02\n +2\t1970-01-03\n +3\t1970-01-04\n +4\t1970-01-05\n +0,"1970-01-01"\n +1,"1970-01-02"\n +2,"1970-01-03"\n +3,"1970-01-04"\n +4,"1970-01-05"\n +["0", "1970-01-01"]\n +["1", "1970-01-02"]\n +["2", "1970-01-03"]\n +["3", "1970-01-04"]\n +["4", "1970-01-05"]\n +\t\t\n\t\t\t0\n\t\t\t1970-01-01\n\t\t\n +\t\t\n\t\t\t1\n\t\t\t1970-01-02\n\t\t\n +\t\t\n\t\t\t2\n\t\t\t1970-01-03\n\t\t\n +\t\t\n\t\t\t3\n\t\t\t1970-01-04\n\t\t\n +\t\t\n\t\t\t4\n\t\t\t1970-01-05\n\t\t\n diff --git a/tests/queries/0_stateless/02113_format_row.sql b/tests/queries/0_stateless/02113_format_row.sql new file mode 100644 index 00000000000..93ee6d0f1dd --- /dev/null +++ b/tests/queries/0_stateless/02113_format_row.sql @@ -0,0 +1,5 @@ +select formatRow('TSVWithNamesAndTypes', number, toDate(number)) from numbers(5); +select formatRow('CSVWithNamesAndTypes', number, toDate(number)) from numbers(5); +select formatRow('JSONCompactEachRowWithNamesAndTypes', number, toDate(number)) from numbers(5); +select formatRow('XML', number, toDate(number)) from numbers(5); + From 824c5b27da9eccb239d216456599601108dea293 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 4 Nov 2021 17:11:01 +0300 Subject: [PATCH 067/200] Add test with emoji --- tests/queries/0_stateless/01889_sql_json_functions.reference | 4 ++++ tests/queries/0_stateless/01889_sql_json_functions.sql | 2 ++ 2 files changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/01889_sql_json_functions.reference b/tests/queries/0_stateless/01889_sql_json_functions.reference index 544d6f5134f..1e463034b32 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.reference +++ b/tests/queries/0_stateless/01889_sql_json_functions.reference @@ -23,6 +23,10 @@ SELECT JSON_VALUE('', '$.hello'); SELECT JSON_VALUE('{"foo foo":"bar"}', '$."foo foo"'); bar +SELECT JSON_VALUE('{"smiley":"\\uD83D\\uDE0A"}', '$.smiley'); +😊 +SELECT JSON_VALUE('{"hello":"\\uD83C\\uDF3A \\uD83C\\uDF38 \\uD83C\\uDF37 Hello, World \\uD83C\\uDF37 \\uD83C\\uDF38 \\uD83C\\uDF3A"}', '$.hello'); +🌺 🌸 🌷 Hello, World 🌷 🌸 🌺 SELECT '--JSON_QUERY--'; --JSON_QUERY-- SELECT JSON_QUERY('{"hello":1}', '$'); diff --git a/tests/queries/0_stateless/01889_sql_json_functions.sql b/tests/queries/0_stateless/01889_sql_json_functions.sql index a06d005c2ff..41df6b83fa6 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.sql +++ b/tests/queries/0_stateless/01889_sql_json_functions.sql @@ -13,6 +13,8 @@ SELECT JSON_VALUE('{"hello":{"world":"!"}}', '$.hello'); SELECT JSON_VALUE('{hello:world}', '$.hello'); -- invalid json => default value (empty string) SELECT JSON_VALUE('', '$.hello'); SELECT JSON_VALUE('{"foo foo":"bar"}', '$."foo foo"'); +SELECT JSON_VALUE('{"smiley":"\\uD83D\\uDE0A"}', '$.smiley'); +SELECT JSON_VALUE('{"hello":"\\uD83C\\uDF3A \\uD83C\\uDF38 \\uD83C\\uDF37 Hello, World \\uD83C\\uDF37 \\uD83C\\uDF38 \\uD83C\\uDF3A"}', '$.hello'); SELECT '--JSON_QUERY--'; SELECT JSON_QUERY('{"hello":1}', '$'); From ab44123f49496f9d1bb2020c74b33c09a3496ccf Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 8 Nov 2021 13:00:42 +0300 Subject: [PATCH 068/200] Fix --- src/Functions/FunctionSQLJSON.h | 20 ++++++++++--------- .../01889_sql_json_functions.reference | 8 ++++++-- .../0_stateless/01889_sql_json_functions.sql | 4 +++- 3 files changed, 20 insertions(+), 12 deletions(-) diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index a4099b0b436..db28663c64f 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -18,6 +18,8 @@ #include #include #include +#include +#include #include #include "config_functions.h" @@ -263,21 +265,21 @@ public: return false; std::stringstream out; // STYLE_CHECK_ALLOW_STD_STRING_STREAM + out << current_element.getElement(); + auto output_str = out.str(); + ColumnString & col_str = assert_cast(dest); + if (current_element.isString()) { - std::string_view element = current_element.getString(); - if (element.starts_with('\"') && element.size() > 1) - out << element.substr(1, element.size() - 1); - else - out << element; + ReadBufferFromString buf(output_str); + String unquoted_output_str; + readJSONString(unquoted_output_str, buf); + col_str.insertData(unquoted_output_str.data(), unquoted_output_str.size()); } else { - out << current_element.getElement(); + col_str.insertData(output_str.data(), output_str.size()); } - auto output_str = out.str(); - ColumnString & col_str = assert_cast(dest); - col_str.insertData(output_str.data(), output_str.size()); return true; } }; diff --git a/tests/queries/0_stateless/01889_sql_json_functions.reference b/tests/queries/0_stateless/01889_sql_json_functions.reference index 1e463034b32..4ed6a9ee611 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.reference +++ b/tests/queries/0_stateless/01889_sql_json_functions.reference @@ -23,10 +23,14 @@ SELECT JSON_VALUE('', '$.hello'); SELECT JSON_VALUE('{"foo foo":"bar"}', '$."foo foo"'); bar -SELECT JSON_VALUE('{"smiley":"\\uD83D\\uDE0A"}', '$.smiley'); -😊 SELECT JSON_VALUE('{"hello":"\\uD83C\\uDF3A \\uD83C\\uDF38 \\uD83C\\uDF37 Hello, World \\uD83C\\uDF37 \\uD83C\\uDF38 \\uD83C\\uDF3A"}', '$.hello'); 🌺 🌸 🌷 Hello, World 🌷 🌸 🌺 +SELECT JSON_VALUE('{"a":"Hello \\"World\\" \\\\"}', '$.a'); +Hello "World" \\ +select JSON_VALUE('{"a":"\\n\\u0000"}', '$.a'); +\n\0 +select JSON_VALUE('{"a":"\\u263a"}', '$.a'); +☺ SELECT '--JSON_QUERY--'; --JSON_QUERY-- SELECT JSON_QUERY('{"hello":1}', '$'); diff --git a/tests/queries/0_stateless/01889_sql_json_functions.sql b/tests/queries/0_stateless/01889_sql_json_functions.sql index 41df6b83fa6..967e21df787 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.sql +++ b/tests/queries/0_stateless/01889_sql_json_functions.sql @@ -13,8 +13,10 @@ SELECT JSON_VALUE('{"hello":{"world":"!"}}', '$.hello'); SELECT JSON_VALUE('{hello:world}', '$.hello'); -- invalid json => default value (empty string) SELECT JSON_VALUE('', '$.hello'); SELECT JSON_VALUE('{"foo foo":"bar"}', '$."foo foo"'); -SELECT JSON_VALUE('{"smiley":"\\uD83D\\uDE0A"}', '$.smiley'); SELECT JSON_VALUE('{"hello":"\\uD83C\\uDF3A \\uD83C\\uDF38 \\uD83C\\uDF37 Hello, World \\uD83C\\uDF37 \\uD83C\\uDF38 \\uD83C\\uDF3A"}', '$.hello'); +SELECT JSON_VALUE('{"a":"Hello \\"World\\" \\\\"}', '$.a'); +select JSON_VALUE('{"a":"\\n\\u0000"}', '$.a'); +select JSON_VALUE('{"a":"\\u263a"}', '$.a'); SELECT '--JSON_QUERY--'; SELECT JSON_QUERY('{"hello":1}', '$'); From 3a981de64309edf608c833433827bd52ddae91b1 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 9 Nov 2021 10:57:37 +0800 Subject: [PATCH 069/200] ISSUES-30801: Remove not like function into RPNElement --- src/Storages/MergeTree/KeyCondition.cpp | 21 ------------------- .../01891_not_like_partition_prune.reference | 6 ++++++ .../01891_not_like_partition_prune.sql | 9 ++++++++ 3 files changed, 15 insertions(+), 21 deletions(-) create mode 100644 tests/queries/0_stateless/01891_not_like_partition_prune.reference create mode 100644 tests/queries/0_stateless/01891_not_like_partition_prune.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index dda7f235d97..f8f6bfff0ae 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -276,27 +276,6 @@ const KeyCondition::AtomMap KeyCondition::atom_map return true; } }, - { - "notLike", - [] (RPNElement & out, const Field & value) - { - if (value.getType() != Field::Types::String) - return false; - - String prefix = extractFixedPrefixFromLikePattern(value.get()); - if (prefix.empty()) - return false; - - String right_bound = firstStringThatIsGreaterThanAllStringsWithPrefix(prefix); - - out.function = RPNElement::FUNCTION_NOT_IN_RANGE; - out.range = !right_bound.empty() - ? Range(prefix, true, right_bound, false) - : Range::createLeftBounded(prefix, true); - - return true; - } - }, { "startsWith", [] (RPNElement & out, const Field & value) diff --git a/tests/queries/0_stateless/01891_not_like_partition_prune.reference b/tests/queries/0_stateless/01891_not_like_partition_prune.reference new file mode 100644 index 00000000000..9dcd05ad5b7 --- /dev/null +++ b/tests/queries/0_stateless/01891_not_like_partition_prune.reference @@ -0,0 +1,6 @@ +1.1 +1 +1.2 +1.12 +1 +1.1 diff --git a/tests/queries/0_stateless/01891_not_like_partition_prune.sql b/tests/queries/0_stateless/01891_not_like_partition_prune.sql new file mode 100644 index 00000000000..6a1842e4a03 --- /dev/null +++ b/tests/queries/0_stateless/01891_not_like_partition_prune.sql @@ -0,0 +1,9 @@ +drop table if exists test; + +create table test (a String) Engine MergeTree order by a partition by a; +insert into test values('1'), ('1.1'), ('1.2'), ('1.12'); + +select * from test where a like '1%1'; +select * from test where a not like '1%1'; +select * from test where a not like '1%2'; +drop table test; From 45cab9f3b094ebcb58b213f38550622f0fd1598c Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 9 Nov 2021 21:31:31 +0800 Subject: [PATCH 070/200] Add stable order by tests result --- src/Storages/MergeTree/KeyCondition.cpp | 13 +++++++++++-- .../01891_not_like_partition_prune.reference | 2 +- .../0_stateless/01891_not_like_partition_prune.sql | 6 +++--- 3 files changed, 15 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index f8f6bfff0ae..de5fc6913bf 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -51,7 +51,10 @@ String Range::toString() const } -/// Example: for `Hello\_World% ...` string it returns `Hello_World`, and for `%test%` returns an empty string. +/// Returns fixed_prefix only if the last char is '%', '_' +/// Example: +/// for `Hello\_World% ...` string it returns `Hello_World` +/// for `%test%` returns an empty string. static String extractFixedPrefixFromLikePattern(const String & like_pattern) { String fixed_prefix; @@ -65,7 +68,13 @@ static String extractFixedPrefixFromLikePattern(const String & like_pattern) case '%': [[fallthrough]]; case '_': - return fixed_prefix; + if (pos + 1 == end) { + return fixed_prefix; + } + else + { + return ""; + } case '\\': ++pos; diff --git a/tests/queries/0_stateless/01891_not_like_partition_prune.reference b/tests/queries/0_stateless/01891_not_like_partition_prune.reference index 9dcd05ad5b7..249697548b7 100644 --- a/tests/queries/0_stateless/01891_not_like_partition_prune.reference +++ b/tests/queries/0_stateless/01891_not_like_partition_prune.reference @@ -1,6 +1,6 @@ 1.1 1 -1.2 1.12 +1.2 1 1.1 diff --git a/tests/queries/0_stateless/01891_not_like_partition_prune.sql b/tests/queries/0_stateless/01891_not_like_partition_prune.sql index 6a1842e4a03..5346a7f08a8 100644 --- a/tests/queries/0_stateless/01891_not_like_partition_prune.sql +++ b/tests/queries/0_stateless/01891_not_like_partition_prune.sql @@ -3,7 +3,7 @@ drop table if exists test; create table test (a String) Engine MergeTree order by a partition by a; insert into test values('1'), ('1.1'), ('1.2'), ('1.12'); -select * from test where a like '1%1'; -select * from test where a not like '1%1'; -select * from test where a not like '1%2'; +select * from test where a like '1%1' order by a; +select * from test where a not like '1%1' order by a; +select * from test where a not like '1%2' order by a; drop table test; From d8aaac36ec6679ab4cc76d563f5a03459465ca1a Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 9 Nov 2021 21:34:55 +0800 Subject: [PATCH 071/200] Add stable order by tests result --- src/Storages/MergeTree/KeyCondition.cpp | 13 ++----------- 1 file changed, 2 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index de5fc6913bf..f8f6bfff0ae 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -51,10 +51,7 @@ String Range::toString() const } -/// Returns fixed_prefix only if the last char is '%', '_' -/// Example: -/// for `Hello\_World% ...` string it returns `Hello_World` -/// for `%test%` returns an empty string. +/// Example: for `Hello\_World% ...` string it returns `Hello_World`, and for `%test%` returns an empty string. static String extractFixedPrefixFromLikePattern(const String & like_pattern) { String fixed_prefix; @@ -68,13 +65,7 @@ static String extractFixedPrefixFromLikePattern(const String & like_pattern) case '%': [[fallthrough]]; case '_': - if (pos + 1 == end) { - return fixed_prefix; - } - else - { - return ""; - } + return fixed_prefix; case '\\': ++pos; From 1c5869d3ef40995773b26ccada701a738a98fa86 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Tue, 9 Nov 2021 22:15:35 +0300 Subject: [PATCH 072/200] Add some server settings MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Задокументировал несколько серверных настроек. --- .../settings.md | 55 ++++++++++++++++++- docs/en/operations/settings/settings.md | 11 ++++ 2 files changed, 65 insertions(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 6ec0d122e6a..9ce1571be09 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1436,4 +1436,57 @@ To add an LDAP server as a remote user directory of users that are not defined l ``` -[Original article](https://clickhouse.com/docs/en/operations/server_configuration_parameters/settings/) +## total_memory_profiler_step {#total-memory-profiler-step} + +Simple server-wide memory profiler. Collects a stack trace at every peak allocation step (in bytes). Data will be stored in the [system.trace_log](../../operations/system-tables/trace_log.md) system table with `query_id` equal to an empty string. + +Possible values: + +- Positive integer. +- 0 — Disabled. + +Default value: `4194304`. + +## total_memory_tracker_sample_probability {#total-memory-tracker-sample-probability} + +Collects random allocations and deallocations and writes them in the [system.trace_log](../../operations/system-tables/trace_log.md) system table with `trace_type` equal to a `MemorySample`. The probability is for every alloc/free, regardless of the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds the untracked memory limit (default value is `4` MiB). It can be lowered if [total_memory_profiler_step](#total-memory-profiler-step) is lowered. You can to set `total_memory_profiler_step` equal to `1` for extra fine-grained sampling. + +Possible values: + +- Positive integer. +- 0 — Disabled. + +Default value: `0`. + +## mmap_cache_size {#mmap-cache-size} + +Cache size in bytes for mapped files. This setting allows to avoid frequent open/[mmap/munmap](https://en.wikipedia.org/wiki/Mmap)/close calls (which are very expensive due to consequent page faults) and to reuse mappings from several threads and queries. The setting value is the number of mapped regions (usually equal to the number of mapped files). The amount of data in mapped files can be monitored in [system.metrics](../../operations/system-tables/metrics.md), [system.metric_log](../../operations/system-tables/metric_log.md) system tables by the `MMappedFiles` and `MMappedFileBytes` metrics, in [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md), [system.asynchronous_metrics_log](../../operations/system-tables/asynchronous_metric_log.md) by the `MMapCacheCells` metric, and also in [system.events](../../operations/system-tables/events.md), [system.processes](../../operations/system-tables/processes.md), [system.query_log](../../operations/system-tables/query_log.md), [system.query_thread_log](../../operations/system-tables/query_thread_log.md), [system.query_views_log](../../operations/system-tables/query_views_log.md) by the `CreatedReadBufferMMap`, `CreatedReadBufferMMapFailed`, `MMappedFileCacheHits`, `MMappedFileCacheMisses` events. Note that the amount of data in mapped files does not consume memory directly and is not accounted in query or server memory usage — because this memory can be discarded similar to OS page cache. The cache is dropped (the files are closed) automatically on the removal of old parts in tables of the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) family, also it can be dropped manually by the `SYSTEM DROP MMAP CACHE` query. + +Possible values: + +- Positive integer. +- 0 — Disabled. + +Default value: `1000`. + +## compiled_expression_cache_size {#compiled-expression-cache-size} + +Cache size in bytes for [compiled expressions](../../operations/caches.md). + +Possible values: + +- Positive integer. +- 0 — Disabled. + +Default value: `134217728`. + +## compiled_expression_cache_elements_size {#compiled_expression_cache_elements_size} + +Cache size in elements for [compiled expressions](../../operations/caches.md). + +Possible values: + +- Positive integer. +- 0 — Disabled. + +Default value: `10000`. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 92f397b2c39..0dceb3c2b9f 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3989,3 +3989,14 @@ Possible values: - 0 — Timeout disabled. Default value: `0`. + +## min_bytes_to_use_mmap_io {#min-bytes-to-use-mmap-io} + +Allows to read big files without copying data from kernel to userspace. Recommended threshold is about 64 MB, because [mmap/munmap](https://en.wikipedia.org/wiki/Mmap) is slow. It makes sense only for large files and helps only if data reside in the page cache. + +Possible values: + +- Positive integer. +- 0 — Big files read with copying data from kernel to userspace. + +Default value: `0`. From 27d98b86179f309e580689c9ab84c0e63e592dea Mon Sep 17 00:00:00 2001 From: karnevil13 Date: Wed, 10 Nov 2021 02:20:59 +0300 Subject: [PATCH 073/200] Updated gui.md, replicated.md & formats.md --- .../en/engines/database-engines/replicated.md | 2 + docs/en/interfaces/formats.md | 4 +- docs/en/interfaces/third-party/gui.md | 14 +++++++ .../ru/engines/database-engines/replicated.md | 4 +- docs/ru/interfaces/formats.md | 4 +- docs/ru/interfaces/third-party/gui.md | 38 +++++++++++++++++++ 6 files changed, 61 insertions(+), 5 deletions(-) diff --git a/docs/en/engines/database-engines/replicated.md b/docs/en/engines/database-engines/replicated.md index 5ce73604207..bdc17d32393 100644 --- a/docs/en/engines/database-engines/replicated.md +++ b/docs/en/engines/database-engines/replicated.md @@ -35,6 +35,8 @@ The [system.clusters](../../operations/system-tables/clusters.md) system table c When creating a new replica of the database, this replica creates tables by itself. If the replica has been unavailable for a long time and has lagged behind the replication log — it checks its local metadata with the current metadata in ZooKeeper, moves the extra tables with data to a separate non-replicated database (so as not to accidentally delete anything superfluous), creates the missing tables, updates the table names if they have been renamed. The data is replicated at the `ReplicatedMergeTree` level, i.e. if the table is not replicated, the data will not be replicated (the database is responsible only for metadata). +[`ALTER TABLE ATTACH|FETCH|DROP|DROP DETACHED|DETACH PARTITION|PART`](../../sql-reference/statements/alter/partition.md) queries are allowed but not replicated. The database engine will only add/fetch/remove the partition/part to the current replica. However, if the table itself uses a Replicated table engine, then the data will be replicated after using `ATTACH`. + ## Usage Example {#usage-example} Creating a cluster with three hosts: diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index d0e5c44b4f7..c2ad74dee63 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -164,8 +164,8 @@ This format is also available under the name `TSVRaw`. ## TabSeparatedWithNames {#tabseparatedwithnames} Differs from the `TabSeparated` format in that the column names are written in the first row. -During parsing, the first row is completely ignored. You can’t use column names to determine their position or to check their correctness. -(Support for parsing the header row may be added in the future.) + +During parsing, the first row is expected to contain the column names. You can use column names to determine their position and to check their correctness. This format is also available under the name `TSVWithNames`. diff --git a/docs/en/interfaces/third-party/gui.md b/docs/en/interfaces/third-party/gui.md index 06474499ccd..593019bfb2e 100644 --- a/docs/en/interfaces/third-party/gui.md +++ b/docs/en/interfaces/third-party/gui.md @@ -129,6 +129,20 @@ Features: - Support monitor (processor, connection, query) - Support migrate data +### Bytebase {#bytebase} + +[Bytebase](https://bytebase.com) is a web-based, open source schema change and version control tool for teams. It supports various databases including ClickHouse. + +Features: + +- Schema review between developers and DBAs. +- Database-as-Code, version control the schema in VCS such GitLab and trigger the deployment upon code commit. +- Streamlined deployment with per-environment policy. +- Full migration history. +- Schema drift detection. +- Backup and restore. +- RBAC. + ## Commercial {#commercial} ### DataGrip {#datagrip} diff --git a/docs/ru/engines/database-engines/replicated.md b/docs/ru/engines/database-engines/replicated.md index b4850a8dafd..9b4e6918633 100644 --- a/docs/ru/engines/database-engines/replicated.md +++ b/docs/ru/engines/database-engines/replicated.md @@ -34,7 +34,9 @@ DDL-запросы с базой данных `Replicated` работают по В системной таблице [system.clusters](../../operations/system-tables/clusters.md) есть кластер с именем, как у реплицируемой базы, который состоит из всех реплик базы. Этот кластер обновляется автоматически при создании/удалении реплик, и его можно использовать для [Distributed](../../engines/table-engines/special/distributed.md#distributed) таблиц. - При создании новой реплики базы, эта реплика сама создаёт таблицы. Если реплика долго была недоступна и отстала от лога репликации — она сверяет свои локальные метаданные с актуальными метаданными в ZooKeeper, перекладывает лишние таблицы с данными в отдельную нереплицируемую базу (чтобы случайно не удалить что-нибудь лишнее), создаёт недостающие таблицы, обновляет имена таблиц, если были переименования. Данные реплицируются на уровне `ReplicatedMergeTree`, т.е. если таблица не реплицируемая, то данные реплицироваться не будут (база отвечает только за метаданные). +При создании новой реплики базы, эта реплика сама создаёт таблицы. Если реплика долго была недоступна и отстала от лога репликации — она сверяет свои локальные метаданные с актуальными метаданными в ZooKeeper, перекладывает лишние таблицы с данными в отдельную нереплицируемую базу (чтобы случайно не удалить что-нибудь лишнее), создаёт недостающие таблицы, обновляет имена таблиц, если были переименования. Данные реплицируются на уровне `ReplicatedMergeTree`, т.е. если таблица не реплицируемая, то данные реплицироваться не будут (база отвечает только за метаданные). + +Запросы [`ALTER TABLE ATTACH|FETCH|DROP|DROP DETACHED|DETACH PARTITION|PART`](../../sql-reference/statements/alter/partition.md) допустимы, но не реплицируются. Движок базы данных может только добавить/извлечь/удалить раздел или часть нынешней реплики. Однако если сама таблица использует движок реплицируемой таблицы, тогда данные будут реплицированы после применения `ATTACH`. ## Примеры использования {#usage-example} diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index d4cfa8a053c..6580bf4b13b 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -163,8 +163,8 @@ SELECT * FROM nestedt FORMAT TSV ## TabSeparatedWithNames {#tabseparatedwithnames} Отличается от формата `TabSeparated` тем, что в первой строке пишутся имена столбцов. -При парсинге, первая строка полностью игнорируется. Вы не можете использовать имена столбцов, чтобы указать их порядок расположения, или чтобы проверить их корректность. -(Поддержка обработки заголовка при парсинге может быть добавлена в будущем.) + +При парсинге первая строка должна содержать имена столбцов. Вы можете использовать имена столбцов, чтобы указать их порядок расположения, или чтобы проверить их корректность. Этот формат также доступен под именем `TSVWithNames`. diff --git a/docs/ru/interfaces/third-party/gui.md b/docs/ru/interfaces/third-party/gui.md index 8e987fab554..0a3589db1d3 100644 --- a/docs/ru/interfaces/third-party/gui.md +++ b/docs/ru/interfaces/third-party/gui.md @@ -69,6 +69,14 @@ toc_title: "Визуальные интерфейсы от сторонних р - Проводник по базе данных. - Инструменты визуализации, позволяющие представить данные в различных формах. +### Grafana {#grafana} + +[Grafana](https://grafana.com/grafana/plugins/vertamedia-clickhouse-datasource) — платформа для мониторинга и визуализации. + +"С помощью Grafana вы можете делать запросы, визуализировать, получать уведомления и разбираться в метриках, где бы они ни хранились. Создавайте, исследуйте, делитесь дашбордами с командой и прививайте культуру принятия решений на основе данных. Мы пользуемся доверием и любовью пользователей" — grafana.com. + +Плагин источника данных ClickHouse поддерживает ClickHouse в качестве бэкенд базы данных. + ### DBeaver {#dbeaver} [DBeaver](https://dbeaver.io/) - универсальный desktop клиент баз данных с поддержкой ClickHouse. @@ -109,6 +117,36 @@ toc_title: "Визуальные интерфейсы от сторонних р [MindsDB](https://mindsdb.com/) — это продукт с открытым исходным кодом, реализующий слой искусственного интеллекта (Artificial Intelligence, AI) для различных СУБД, в том числе для ClickHouse. MindsDB облегчает процессы создания, обучения и развертывания современных моделей машинного обучения. Графический пользовательский интерфейс MindsDB Studio позволяет обучать новые модели на основе данных в БД, интерпретировать сделанные моделями прогнозы, выявлять потенциальные ошибки в данных, визуализировать и оценивать достоверность моделей с помощью функции Explainable AI, так чтобы вы могли быстрее адаптировать и настраивать ваши модели машинного обучения. +### DBM {#dbm} + +[DBM](https://dbm.incubator.edurt.io/) DBM — инструмент для визуального менеджмента в ClickHouse! + +Основные возможности: + +- Поддержка истории запросов (пагинация, очистка и т.д.) +- Поддержка отдельных секций запросов +- Поддержка остановки запроса +- Поддержка управления таблицами (метаданные, удаление, предпросмотр) +- Поддержка управления базами данных (удаление, создание) +- Поддержка пользовательских запросов +- Поддержка управления различными источниками данных (проверка соединения, мониторинг) +- Поддержка монитора (процессор, соединение, запрос) +- Поддержка миграции данных + +### Bytebase {#bytebase} + +[Bytebase](https://bytebase.com) — сетевой инструмент для смены схем и контроля версий с открытым исходным кодом для работы в команде. Поддерживает различные базы данных, в том числе ClickHouse. + +Основные возможности: + +- Проверка схемы для разработчиков и администраторов баз данных. +- "База данных в виде кода", хранение схемы базы данных с помощью системы контроля версий, например, GitLab, а также активация развертывания по коммиту. +- Рациональное развертывание и соответствующая среда. +- Полная история миграций. +- Определение смещения схемы. +- Резервное копирование и восстановление. +- Управление доступом на основе ролей. + ## Коммерческие {#commercial} ### DataGrip {#datagrip} From 03b64c062972683229f703760adb28da9495ecdd Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 10 Nov 2021 20:24:01 +0300 Subject: [PATCH 074/200] Update server settings MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Обновил описание. --- .../server-configuration-parameters/settings.md | 16 ++++++---------- docs/en/operations/settings/settings.md | 4 ++-- 2 files changed, 8 insertions(+), 12 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 9ce1571be09..73454055356 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1438,55 +1438,51 @@ To add an LDAP server as a remote user directory of users that are not defined l ## total_memory_profiler_step {#total-memory-profiler-step} -Simple server-wide memory profiler. Collects a stack trace at every peak allocation step (in bytes). Data will be stored in the [system.trace_log](../../operations/system-tables/trace_log.md) system table with `query_id` equal to an empty string. +Sets the memory size in bytes for a stack trace at every peak allocation step. The data is stored in the [system.trace_log](../../operations/system-tables/trace_log.md) system table with `query_id` equal to an empty string. Possible values: - Positive integer. -- 0 — Disabled. Default value: `4194304`. ## total_memory_tracker_sample_probability {#total-memory-tracker-sample-probability} -Collects random allocations and deallocations and writes them in the [system.trace_log](../../operations/system-tables/trace_log.md) system table with `trace_type` equal to a `MemorySample`. The probability is for every alloc/free, regardless of the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds the untracked memory limit (default value is `4` MiB). It can be lowered if [total_memory_profiler_step](#total-memory-profiler-step) is lowered. You can to set `total_memory_profiler_step` equal to `1` for extra fine-grained sampling. +Allows to collect random allocations and deallocations and writes them in the [system.trace_log](../../operations/system-tables/trace_log.md) system table with `trace_type` equal to a `MemorySample` with the specified probability. The probability is for every allocation or deallocations, regardless of the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds the untracked memory limit (default value is `4` MiB). It can be lowered if [total_memory_profiler_step](#total-memory-profiler-step) is lowered. You can to set `total_memory_profiler_step` equal to `1` for extra fine-grained sampling. Possible values: - Positive integer. -- 0 — Disabled. +- 0 — Writing of random allocations and deallocations in the `system.trace_log` system table is disabled. Default value: `0`. ## mmap_cache_size {#mmap-cache-size} -Cache size in bytes for mapped files. This setting allows to avoid frequent open/[mmap/munmap](https://en.wikipedia.org/wiki/Mmap)/close calls (which are very expensive due to consequent page faults) and to reuse mappings from several threads and queries. The setting value is the number of mapped regions (usually equal to the number of mapped files). The amount of data in mapped files can be monitored in [system.metrics](../../operations/system-tables/metrics.md), [system.metric_log](../../operations/system-tables/metric_log.md) system tables by the `MMappedFiles` and `MMappedFileBytes` metrics, in [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md), [system.asynchronous_metrics_log](../../operations/system-tables/asynchronous_metric_log.md) by the `MMapCacheCells` metric, and also in [system.events](../../operations/system-tables/events.md), [system.processes](../../operations/system-tables/processes.md), [system.query_log](../../operations/system-tables/query_log.md), [system.query_thread_log](../../operations/system-tables/query_thread_log.md), [system.query_views_log](../../operations/system-tables/query_views_log.md) by the `CreatedReadBufferMMap`, `CreatedReadBufferMMapFailed`, `MMappedFileCacheHits`, `MMappedFileCacheMisses` events. Note that the amount of data in mapped files does not consume memory directly and is not accounted in query or server memory usage — because this memory can be discarded similar to OS page cache. The cache is dropped (the files are closed) automatically on the removal of old parts in tables of the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) family, also it can be dropped manually by the `SYSTEM DROP MMAP CACHE` query. +Sets the cache size in bytes for mapped files. This setting allows to avoid frequent open/[mmap/munmap](https://en.wikipedia.org/wiki/Mmap)/close calls (which are very expensive due to consequent page faults) and to reuse mappings from several threads and queries. The setting value is the number of mapped regions (usually equal to the number of mapped files). The amount of data in mapped files can be monitored in [system.metrics](../../operations/system-tables/metrics.md), [system.metric_log](../../operations/system-tables/metric_log.md) system tables by the `MMappedFiles` and `MMappedFileBytes` metrics, in [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md), [system.asynchronous_metrics_log](../../operations/system-tables/asynchronous_metric_log.md) by the `MMapCacheCells` metric, and also in [system.events](../../operations/system-tables/events.md), [system.processes](../../operations/system-tables/processes.md), [system.query_log](../../operations/system-tables/query_log.md), [system.query_thread_log](../../operations/system-tables/query_thread_log.md), [system.query_views_log](../../operations/system-tables/query_views_log.md) by the `CreatedReadBufferMMap`, `CreatedReadBufferMMapFailed`, `MMappedFileCacheHits`, `MMappedFileCacheMisses` events. Note that the amount of data in mapped files does not consume memory directly and is not accounted in query or server memory usage — because this memory can be discarded similar to OS page cache. The cache is dropped (the files are closed) automatically on the removal of old parts in tables of the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) family, also it can be dropped manually by the `SYSTEM DROP MMAP CACHE` query. Possible values: - Positive integer. -- 0 — Disabled. Default value: `1000`. ## compiled_expression_cache_size {#compiled-expression-cache-size} -Cache size in bytes for [compiled expressions](../../operations/caches.md). +Sets the cache size in bytes for [compiled expressions](../../operations/caches.md). Possible values: - Positive integer. -- 0 — Disabled. Default value: `134217728`. ## compiled_expression_cache_elements_size {#compiled_expression_cache_elements_size} -Cache size in elements for [compiled expressions](../../operations/caches.md). +Sets the cache size in elements for [compiled expressions](../../operations/caches.md). Possible values: - Positive integer. -- 0 — Disabled. Default value: `10000`. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 0dceb3c2b9f..5398bf8fe34 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -3992,11 +3992,11 @@ Default value: `0`. ## min_bytes_to_use_mmap_io {#min-bytes-to-use-mmap-io} -Allows to read big files without copying data from kernel to userspace. Recommended threshold is about 64 MB, because [mmap/munmap](https://en.wikipedia.org/wiki/Mmap) is slow. It makes sense only for large files and helps only if data reside in the page cache. +This is an experimental setting. Sets the minimum amount of memory for reading large files without copying data from the kernel to userspace. Recommended threshold is about 64 MB, because [mmap/munmap](https://en.wikipedia.org/wiki/Mmap) is slow. It makes sense only for large files and helps only if data reside in the page cache. Possible values: - Positive integer. -- 0 — Big files read with copying data from kernel to userspace. +- 0 — Big files read with only copying data from kernel to userspace. Default value: `0`. From e5661b9b62643be673b3b8802e4f3a470dc11704 Mon Sep 17 00:00:00 2001 From: avogar Date: Thu, 11 Nov 2021 21:09:21 +0300 Subject: [PATCH 075/200] Minor change --- src/Client/ClientBase.cpp | 2 +- src/Processors/Formats/IOutputFormat.cpp | 6 +++--- src/Processors/Formats/IOutputFormat.h | 4 ++-- src/Processors/Formats/IRowOutputFormat.cpp | 2 +- src/Processors/Formats/IRowOutputFormat.h | 2 +- src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/ArrowBlockOutputFormat.h | 2 +- src/Processors/Formats/Impl/MySQLOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/MySQLOutputFormat.h | 2 +- src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/ORCBlockOutputFormat.h | 2 +- .../Formats/Impl/ParallelFormattingOutputFormat.cpp | 4 ++-- .../Formats/Impl/ParallelFormattingOutputFormat.h | 2 +- src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/ParquetBlockOutputFormat.h | 2 +- src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/PrettyBlockOutputFormat.h | 2 +- src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp | 2 +- src/Processors/Formats/Impl/TemplateBlockOutputFormat.h | 2 +- src/Processors/Formats/LazyOutputFormat.h | 2 +- src/Server/GRPCServer.cpp | 6 +++--- src/Storages/HDFS/StorageHDFS.cpp | 2 +- src/Storages/Kafka/KafkaBlockOutputStream.cpp | 2 +- src/Storages/RabbitMQ/RabbitMQSink.cpp | 2 +- src/Storages/StorageFile.cpp | 2 +- src/Storages/StorageS3.cpp | 2 +- src/Storages/StorageURL.cpp | 2 +- 27 files changed, 33 insertions(+), 33 deletions(-) diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 996f9f53ea3..61bcacf4a42 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -677,7 +677,7 @@ void ClientBase::onEndOfStream() progress_indication.clearProgressOutput(); if (output_format) - output_format->finish(); + output_format->finalize(); resetOutput(); diff --git a/src/Processors/Formats/IOutputFormat.cpp b/src/Processors/Formats/IOutputFormat.cpp index a0b39f20b96..d8273878717 100644 --- a/src/Processors/Formats/IOutputFormat.cpp +++ b/src/Processors/Formats/IOutputFormat.cpp @@ -72,7 +72,7 @@ void IOutputFormat::work() if (rows_before_limit_counter && rows_before_limit_counter->hasAppliedLimit()) setRowsBeforeLimit(rows_before_limit_counter->get()); - finalize(); + finalizeImpl(); finalized = true; return; } @@ -113,10 +113,10 @@ void IOutputFormat::write(const Block & block) flush(); } -void IOutputFormat::finish() +void IOutputFormat::finalize() { writePrefixIfNot(); - finalize(); + finalizeImpl(); } } diff --git a/src/Processors/Formats/IOutputFormat.h b/src/Processors/Formats/IOutputFormat.h index 225971a8fe6..40dcaf75b53 100644 --- a/src/Processors/Formats/IOutputFormat.h +++ b/src/Processors/Formats/IOutputFormat.h @@ -55,7 +55,7 @@ public: void write(const Block & block); - virtual void finish(); + void finalize(); virtual bool expectMaterializedColumns() const { return true; } @@ -73,7 +73,7 @@ protected: virtual void consume(Chunk) = 0; virtual void consumeTotals(Chunk) {} virtual void consumeExtremes(Chunk) {} - virtual void finalize() {} + virtual void finalizeImpl() {} virtual void writePrefix() {} void writePrefixIfNot() diff --git a/src/Processors/Formats/IRowOutputFormat.cpp b/src/Processors/Formats/IRowOutputFormat.cpp index 410d2d07d35..299c6c35fad 100644 --- a/src/Processors/Formats/IRowOutputFormat.cpp +++ b/src/Processors/Formats/IRowOutputFormat.cpp @@ -70,7 +70,7 @@ void IRowOutputFormat::consumeExtremes(DB::Chunk chunk) writeAfterExtremes(); } -void IRowOutputFormat::finalize() +void IRowOutputFormat::finalizeImpl() { writeSuffixIfNot(); writeLastSuffix(); diff --git a/src/Processors/Formats/IRowOutputFormat.h b/src/Processors/Formats/IRowOutputFormat.h index 2e658c452f6..11e690c33ee 100644 --- a/src/Processors/Formats/IRowOutputFormat.h +++ b/src/Processors/Formats/IRowOutputFormat.h @@ -31,7 +31,7 @@ protected: void consume(Chunk chunk) override; void consumeTotals(Chunk chunk) override; void consumeExtremes(Chunk chunk) override; - void finalize() override; + void finalizeImpl() override; /** Write a row. * Default implementation calls methods to write single values and delimiters diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp index 4404749adb6..692f17f843a 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.cpp @@ -50,7 +50,7 @@ void ArrowBlockOutputFormat::consume(Chunk chunk) "Error while writing a table: {}", status.ToString()); } -void ArrowBlockOutputFormat::finalize() +void ArrowBlockOutputFormat::finalizeImpl() { if (!writer) { diff --git a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h index 4534575e735..ab5a0e7351a 100644 --- a/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ArrowBlockOutputFormat.h @@ -26,7 +26,7 @@ public: private: void consume(Chunk) override; - void finalize() override; + void finalizeImpl() override; void prepareWriter(const std::shared_ptr & schema); diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index cdbac9f06fb..ab5d5e8399b 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -68,7 +68,7 @@ void MySQLOutputFormat::consume(Chunk chunk) } } -void MySQLOutputFormat::finalize() +void MySQLOutputFormat::finalizeImpl() { size_t affected_rows = 0; std::string human_readable_info; diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.h b/src/Processors/Formats/Impl/MySQLOutputFormat.h index 6f4337486e2..9481ef67070 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.h +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.h @@ -30,7 +30,7 @@ public: private: void consume(Chunk) override; - void finalize() override; + void finalizeImpl() override; void writePrefix() override; uint32_t client_capabilities = 0; diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp index 0a1eb9c6836..4c8f6ab2c54 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.cpp @@ -495,7 +495,7 @@ void ORCBlockOutputFormat::consume(Chunk chunk) writer->add(*batch); } -void ORCBlockOutputFormat::finalize() +void ORCBlockOutputFormat::finalizeImpl() { if (!writer) prepareWriter(); diff --git a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h index fcdc0e9c6d6..2ffee597e8f 100644 --- a/src/Processors/Formats/Impl/ORCBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockOutputFormat.h @@ -40,7 +40,7 @@ public: private: void consume(Chunk chunk) override; - void finalize() override; + void finalizeImpl() override; ORC_UNIQUE_PTR getORCType(const DataTypePtr & type, const std::string & column_name); diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp index c7b7a4d2814..30bfae4972d 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.cpp @@ -4,7 +4,7 @@ namespace DB { - void ParallelFormattingOutputFormat::finalize() + void ParallelFormattingOutputFormat::finalizeImpl() { need_flush = true; IOutputFormat::finalized = true; @@ -191,7 +191,7 @@ namespace DB } case ProcessingUnitType::FINALIZE : { - formatter->finalize(); + formatter->finalizeImpl(); break; } } diff --git a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h index 0f8bff9b7d9..50f2ea9dc1b 100644 --- a/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h +++ b/src/Processors/Formats/Impl/ParallelFormattingOutputFormat.h @@ -130,7 +130,7 @@ private: addChunk(std::move(extremes), ProcessingUnitType::EXTREMES, /*can_throw_exception*/ true); } - void finalize() override; + void finalizeImpl() override; InternalFormatterCreator internal_formatter_creator; diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp index 416d1b80988..a10858ee668 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.cpp @@ -60,7 +60,7 @@ void ParquetBlockOutputFormat::consume(Chunk chunk) throw Exception{"Error while writing a table: " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION}; } -void ParquetBlockOutputFormat::finalize() +void ParquetBlockOutputFormat::finalizeImpl() { if (!file_writer) { diff --git a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h index f58db3bd202..dee25ee1aa4 100644 --- a/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/ParquetBlockOutputFormat.h @@ -35,7 +35,7 @@ public: private: void consume(Chunk) override; - void finalize() override; + void finalizeImpl() override; const FormatSettings format_settings; diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp index 2bae0c6bd63..66ddaa616cd 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.cpp @@ -401,7 +401,7 @@ void PrettyBlockOutputFormat::writeSuffix() } } -void PrettyBlockOutputFormat::finalize() +void PrettyBlockOutputFormat::finalizeImpl() { writeSuffixIfNot(); } diff --git a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h index 8d317277251..597c82e8166 100644 --- a/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/PrettyBlockOutputFormat.h @@ -27,7 +27,7 @@ protected: void consumeTotals(Chunk) override; void consumeExtremes(Chunk) override; - void finalize() override; + void finalizeImpl() override; size_t total_rows = 0; size_t terminal_width = 0; diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp index dae3398a36c..fa55f233c7a 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -164,7 +164,7 @@ void TemplateBlockOutputFormat::writePrefix() writeString(format.delimiters.front(), out); } -void TemplateBlockOutputFormat::finalize() +void TemplateBlockOutputFormat::finalizeImpl() { if (finalized) return; diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h index 224a8d20176..d24870b0931 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h @@ -43,7 +43,7 @@ private: void consume(Chunk chunk) override; void consumeTotals(Chunk chunk) override { totals = std::move(chunk); } void consumeExtremes(Chunk chunk) override { extremes = std::move(chunk); } - void finalize() override; + void finalizeImpl() override; void writeRow(const Chunk & chunk, size_t row_num); void serializeField(const IColumn & column, const ISerialization & serialization, size_t row_num, ColumnFormat format); diff --git a/src/Processors/Formats/LazyOutputFormat.h b/src/Processors/Formats/LazyOutputFormat.h index 123d393a871..b539a8494c7 100644 --- a/src/Processors/Formats/LazyOutputFormat.h +++ b/src/Processors/Formats/LazyOutputFormat.h @@ -34,7 +34,7 @@ public: queue.clearAndFinish(); } - void finalize() override + void finalizeImpl() override { queue.finish(); } diff --git a/src/Server/GRPCServer.cpp b/src/Server/GRPCServer.cpp index 7d68482ed78..a518b8e9005 100644 --- a/src/Server/GRPCServer.cpp +++ b/src/Server/GRPCServer.cpp @@ -1168,7 +1168,7 @@ namespace executor->execute(); } - output_format_processor->finish(); + output_format_processor->finalize(); } void Call::finishQuery() @@ -1380,7 +1380,7 @@ namespace WriteBufferFromString buf{*result.mutable_totals()}; auto format = query_context->getOutputFormat(output_format, buf, totals); format->write(materializeBlock(totals)); - format->finish(); + format->finalize(); } void Call::addExtremesToResult(const Block & extremes) @@ -1391,7 +1391,7 @@ namespace WriteBufferFromString buf{*result.mutable_extremes()}; auto format = query_context->getOutputFormat(output_format, buf, extremes); format->write(materializeBlock(extremes)); - format->finish(); + format->finalize(); } void Call::addProfileInfoToResult(const ProfileInfo & info) diff --git a/src/Storages/HDFS/StorageHDFS.cpp b/src/Storages/HDFS/StorageHDFS.cpp index e2db5109585..08ab7573f32 100644 --- a/src/Storages/HDFS/StorageHDFS.cpp +++ b/src/Storages/HDFS/StorageHDFS.cpp @@ -217,7 +217,7 @@ public: { try { - writer->finish(); + writer->finalize(); writer->flush(); write_buf->sync(); write_buf->finalize(); diff --git a/src/Storages/Kafka/KafkaBlockOutputStream.cpp b/src/Storages/Kafka/KafkaBlockOutputStream.cpp index 79163ff8d17..aeaee04c506 100644 --- a/src/Storages/Kafka/KafkaBlockOutputStream.cpp +++ b/src/Storages/Kafka/KafkaBlockOutputStream.cpp @@ -42,7 +42,7 @@ void KafkaSink::consume(Chunk chunk) void KafkaSink::onFinish() { if (format) - format->finish(); + format->finalize(); //flush(); if (buffer) diff --git a/src/Storages/RabbitMQ/RabbitMQSink.cpp b/src/Storages/RabbitMQ/RabbitMQSink.cpp index fc3dccb57fa..3e7ced43cbc 100644 --- a/src/Storages/RabbitMQ/RabbitMQSink.cpp +++ b/src/Storages/RabbitMQ/RabbitMQSink.cpp @@ -49,7 +49,7 @@ void RabbitMQSink::consume(Chunk chunk) void RabbitMQSink::onFinish() { - format->finish(); + format->finalize(); if (buffer) buffer->updateMaxWait(); diff --git a/src/Storages/StorageFile.cpp b/src/Storages/StorageFile.cpp index af46e2d3ba1..fd7a3c77241 100644 --- a/src/Storages/StorageFile.cpp +++ b/src/Storages/StorageFile.cpp @@ -643,7 +643,7 @@ public: void onFinish() override { - writer->finish(); + writer->finalize(); } // void flush() override diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index 99391963fd1..3f08dee62b6 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -328,7 +328,7 @@ public: { try { - writer->finish(); + writer->finalize(); writer->flush(); write_buf->finalize(); } diff --git a/src/Storages/StorageURL.cpp b/src/Storages/StorageURL.cpp index 6882d1acf0e..4dcecf22568 100644 --- a/src/Storages/StorageURL.cpp +++ b/src/Storages/StorageURL.cpp @@ -231,7 +231,7 @@ void StorageURLSink::consume(Chunk chunk) void StorageURLSink::onFinish() { - writer->finish(); + writer->finalize(); writer->flush(); write_buf->finalize(); } From 7b88c956c83f1aa5a9ae94edfc9f44627fd60c1e Mon Sep 17 00:00:00 2001 From: liyang830 Date: Fri, 12 Nov 2021 18:22:55 +0800 Subject: [PATCH 076/200] fix: quota limit was not reached, but the limit was exceeded --- src/Access/EnabledQuota.cpp | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/src/Access/EnabledQuota.cpp b/src/Access/EnabledQuota.cpp index d776b2b851a..cf78bfd0475 100644 --- a/src/Access/EnabledQuota.cpp +++ b/src/Access/EnabledQuota.cpp @@ -52,9 +52,7 @@ struct EnabledQuota::Impl return end; } - /// We reset counters only if the interval's end has been calculated before. - /// If it hasn't we just calculate the interval's end for the first time and don't reset counters yet. - bool need_reset_counters = (end_loaded.count() != 0); + bool need_reset_counters = false; do { @@ -66,7 +64,12 @@ struct EnabledQuota::Impl UInt64 n = static_cast((current_time - end + duration) / duration); end = end + duration * n; if (end_of_interval.compare_exchange_strong(end_loaded, end.time_since_epoch())) + { + /// We reset counters only if the interval's end has been calculated before. + /// If it hasn't we just calculate the interval's end for the first time and don't reset counters yet. + need_reset_counters = true; break; + } end = std::chrono::system_clock::time_point{end_loaded}; } while (current_time >= end); From 398e164e46df4b0a79ec62c4e9445b646c22f95a Mon Sep 17 00:00:00 2001 From: Alexey Date: Fri, 12 Nov 2021 20:02:26 +0000 Subject: [PATCH 077/200] en draft (EXISTS) --- docs/en/sql-reference/operators/exists.md | 29 +++++++++++++++++++ docs/en/sql-reference/operators/index.md | 2 +- .../sql-reference/statements/select/where.md | 4 +++ 3 files changed, 34 insertions(+), 1 deletion(-) create mode 100644 docs/en/sql-reference/operators/exists.md diff --git a/docs/en/sql-reference/operators/exists.md b/docs/en/sql-reference/operators/exists.md new file mode 100644 index 00000000000..929da871655 --- /dev/null +++ b/docs/en/sql-reference/operators/exists.md @@ -0,0 +1,29 @@ +# EXISTS {#exists-operator} + +The `EXISTS` operator tests how many records are in the result of a subquery. If it is empty, then the operator returns `0`. Otherwise it returns `1`. + +!!! warning "Warning" + References to main query tables and columns are not supported in a subquery. + +**Syntax** + +```sql +WHERE EXISTS(subquery) +``` + +**Example** + +Query: + +``` sql +SELECT 'Exists' WHERE EXISTS (SELECT * FROM numbers(10) WHERE number < 2); +SELECT 'Empty subquery' WHERE EXISTS (SELECT * FROM numbers(10) WHERE number > 12); +``` + +The first query returns one row. The second query does not return rows because the result of the subquery is empty. + +``` text +┌─'Exists'─┐ +│ Exists │ +└──────────┘ +``` diff --git a/docs/en/sql-reference/operators/index.md b/docs/en/sql-reference/operators/index.md index efbc600fdd4..1c59b25fc63 100644 --- a/docs/en/sql-reference/operators/index.md +++ b/docs/en/sql-reference/operators/index.md @@ -71,7 +71,7 @@ For tuple subtraction: [tupleMinus](../../sql-reference/functions/tuple-function ## Operators for Working with Data Sets {#operators-for-working-with-data-sets} -*See [IN operators](../../sql-reference/operators/in.md).* +See [IN operators](../../sql-reference/operators/in.md) and [EXISTS](../../sql-reference/operators/exists.md) operator. `a IN ...` – The `in(a, b)` function. diff --git a/docs/en/sql-reference/statements/select/where.md b/docs/en/sql-reference/statements/select/where.md index 69505a51db4..ffc9a38b5fb 100644 --- a/docs/en/sql-reference/statements/select/where.md +++ b/docs/en/sql-reference/statements/select/where.md @@ -12,3 +12,7 @@ If there is a `WHERE` clause, it must contain an expression with the `UInt8` typ !!! note "Note" There’s a filtering optimization called [prewhere](../../../sql-reference/statements/select/prewhere.md). + +**See Also** + +- [EXISTS](../../../sql-reference/operators/exists.md) operator From b611a709b97f2ffa576e80ce9f8786d7ca6b42cc Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Fri, 12 Nov 2021 23:40:05 +0300 Subject: [PATCH 078/200] Update docs/en/operations/server-configuration-parameters/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/en/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 73454055356..671291ca729 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1448,7 +1448,7 @@ Default value: `4194304`. ## total_memory_tracker_sample_probability {#total-memory-tracker-sample-probability} -Allows to collect random allocations and deallocations and writes them in the [system.trace_log](../../operations/system-tables/trace_log.md) system table with `trace_type` equal to a `MemorySample` with the specified probability. The probability is for every allocation or deallocations, regardless of the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds the untracked memory limit (default value is `4` MiB). It can be lowered if [total_memory_profiler_step](#total-memory-profiler-step) is lowered. You can to set `total_memory_profiler_step` equal to `1` for extra fine-grained sampling. +Allows to collect random allocations and deallocations and writes them in the [system.trace_log](../../operations/system-tables/trace_log.md) system table with `trace_type` equal to a `MemorySample` with the specified probability. The probability is for every allocation or deallocations, regardless of the size of the allocation. Note that sampling happens only when the amount of untracked memory exceeds the untracked memory limit (default value is `4` MiB). It can be lowered if [total_memory_profiler_step](#total-memory-profiler-step) is lowered. You can set `total_memory_profiler_step` equal to `1` for extra fine-grained sampling. Possible values: From ba0d57b90e9c1c81d6b53090f968b1a3136c6d3f Mon Sep 17 00:00:00 2001 From: Alexey Date: Sat, 13 Nov 2021 19:46:38 +0000 Subject: [PATCH 079/200] Added example to WHERE --- .../sql-reference/statements/select/where.md | 20 ++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/where.md b/docs/en/sql-reference/statements/select/where.md index ffc9a38b5fb..f9b0b54f403 100644 --- a/docs/en/sql-reference/statements/select/where.md +++ b/docs/en/sql-reference/statements/select/where.md @@ -11,7 +11,25 @@ If there is a `WHERE` clause, it must contain an expression with the `UInt8` typ `WHERE` expression is evaluated on the ability to use indexes and partition pruning, if the underlying table engine supports that. !!! note "Note" - There’s a filtering optimization called [prewhere](../../../sql-reference/statements/select/prewhere.md). + There is a filtering optimization called [prewhere](../../../sql-reference/statements/select/prewhere.md). + +**Example** + +To find numbers that are multiples of 3 and are greater than 10 execute the following query on the [numbers table](../../../sql-reference/table-functions/numbers.md): + +``` sql +SELECT number FROM numbers(20) WHERE (number > 10) AND (number % 3 == 0); +``` + +Result: + +``` text +┌─number─┐ +│ 12 │ +│ 15 │ +│ 18 │ +└────────┘ +``` **See Also** From 40952d53acde1140067a5316130ab5ff86cf4398 Mon Sep 17 00:00:00 2001 From: Alexey Date: Sat, 13 Nov 2021 19:47:50 +0000 Subject: [PATCH 080/200] Small text change Removed See Also EXISTS from WHERE --- docs/en/sql-reference/operators/exists.md | 2 +- docs/en/sql-reference/statements/select/where.md | 4 ---- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/docs/en/sql-reference/operators/exists.md b/docs/en/sql-reference/operators/exists.md index 929da871655..20472314cab 100644 --- a/docs/en/sql-reference/operators/exists.md +++ b/docs/en/sql-reference/operators/exists.md @@ -20,7 +20,7 @@ SELECT 'Exists' WHERE EXISTS (SELECT * FROM numbers(10) WHERE number < 2); SELECT 'Empty subquery' WHERE EXISTS (SELECT * FROM numbers(10) WHERE number > 12); ``` -The first query returns one row. The second query does not return rows because the result of the subquery is empty. +The first query returns one row while the second query does not return rows because the result of the subquery is empty: ``` text ┌─'Exists'─┐ diff --git a/docs/en/sql-reference/statements/select/where.md b/docs/en/sql-reference/statements/select/where.md index f9b0b54f403..236a5f53950 100644 --- a/docs/en/sql-reference/statements/select/where.md +++ b/docs/en/sql-reference/statements/select/where.md @@ -30,7 +30,3 @@ Result: │ 18 │ └────────┘ ``` - -**See Also** - -- [EXISTS](../../../sql-reference/operators/exists.md) operator From 301e403d22bdb2a621a1aba63e4d341beb83cc9d Mon Sep 17 00:00:00 2001 From: Alexey Date: Sat, 13 Nov 2021 20:00:00 +0000 Subject: [PATCH 081/200] link from EXISTS to WHERE --- docs/en/sql-reference/operators/exists.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/en/sql-reference/operators/exists.md b/docs/en/sql-reference/operators/exists.md index 20472314cab..c1d718e80ad 100644 --- a/docs/en/sql-reference/operators/exists.md +++ b/docs/en/sql-reference/operators/exists.md @@ -2,6 +2,8 @@ The `EXISTS` operator tests how many records are in the result of a subquery. If it is empty, then the operator returns `0`. Otherwise it returns `1`. +`EXISTS` can be used in a [WHERE](../../sql-reference/statements/select/where.md) clause. + !!! warning "Warning" References to main query tables and columns are not supported in a subquery. From e30b0339c744943d7996e61d974b4f7f303f5d27 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sat, 13 Nov 2021 23:14:02 +0300 Subject: [PATCH 082/200] Translate to Russian MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык. --- .../settings.md | 8 +-- .../settings.md | 51 +++++++++++++++++++ docs/ru/operations/settings/settings.md | 10 ++++ 3 files changed, 65 insertions(+), 4 deletions(-) diff --git a/docs/en/operations/server-configuration-parameters/settings.md b/docs/en/operations/server-configuration-parameters/settings.md index 671291ca729..9c35a994509 100644 --- a/docs/en/operations/server-configuration-parameters/settings.md +++ b/docs/en/operations/server-configuration-parameters/settings.md @@ -1438,7 +1438,7 @@ To add an LDAP server as a remote user directory of users that are not defined l ## total_memory_profiler_step {#total-memory-profiler-step} -Sets the memory size in bytes for a stack trace at every peak allocation step. The data is stored in the [system.trace_log](../../operations/system-tables/trace_log.md) system table with `query_id` equal to an empty string. +Sets the memory size (in bytes) for a stack trace at every peak allocation step. The data is stored in the [system.trace_log](../../operations/system-tables/trace_log.md) system table with `query_id` equal to an empty string. Possible values: @@ -1459,7 +1459,7 @@ Default value: `0`. ## mmap_cache_size {#mmap-cache-size} -Sets the cache size in bytes for mapped files. This setting allows to avoid frequent open/[mmap/munmap](https://en.wikipedia.org/wiki/Mmap)/close calls (which are very expensive due to consequent page faults) and to reuse mappings from several threads and queries. The setting value is the number of mapped regions (usually equal to the number of mapped files). The amount of data in mapped files can be monitored in [system.metrics](../../operations/system-tables/metrics.md), [system.metric_log](../../operations/system-tables/metric_log.md) system tables by the `MMappedFiles` and `MMappedFileBytes` metrics, in [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md), [system.asynchronous_metrics_log](../../operations/system-tables/asynchronous_metric_log.md) by the `MMapCacheCells` metric, and also in [system.events](../../operations/system-tables/events.md), [system.processes](../../operations/system-tables/processes.md), [system.query_log](../../operations/system-tables/query_log.md), [system.query_thread_log](../../operations/system-tables/query_thread_log.md), [system.query_views_log](../../operations/system-tables/query_views_log.md) by the `CreatedReadBufferMMap`, `CreatedReadBufferMMapFailed`, `MMappedFileCacheHits`, `MMappedFileCacheMisses` events. Note that the amount of data in mapped files does not consume memory directly and is not accounted in query or server memory usage — because this memory can be discarded similar to OS page cache. The cache is dropped (the files are closed) automatically on the removal of old parts in tables of the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) family, also it can be dropped manually by the `SYSTEM DROP MMAP CACHE` query. +Sets the cache size (in bytes) for mapped files. This setting allows to avoid frequent open/[mmap/munmap](https://en.wikipedia.org/wiki/Mmap)/close calls (which are very expensive due to consequent page faults) and to reuse mappings from several threads and queries. The setting value is the number of mapped regions (usually equal to the number of mapped files). The amount of data in mapped files can be monitored in [system.metrics](../../operations/system-tables/metrics.md), [system.metric_log](../../operations/system-tables/metric_log.md) system tables by the `MMappedFiles` and `MMappedFileBytes` metrics, in [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md), [system.asynchronous_metrics_log](../../operations/system-tables/asynchronous_metric_log.md) by the `MMapCacheCells` metric, and also in [system.events](../../operations/system-tables/events.md), [system.processes](../../operations/system-tables/processes.md), [system.query_log](../../operations/system-tables/query_log.md), [system.query_thread_log](../../operations/system-tables/query_thread_log.md), [system.query_views_log](../../operations/system-tables/query_views_log.md) by the `CreatedReadBufferMMap`, `CreatedReadBufferMMapFailed`, `MMappedFileCacheHits`, `MMappedFileCacheMisses` events. Note that the amount of data in mapped files does not consume memory directly and is not accounted in query or server memory usage — because this memory can be discarded similar to OS page cache. The cache is dropped (the files are closed) automatically on the removal of old parts in tables of the [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md) family, also it can be dropped manually by the `SYSTEM DROP MMAP CACHE` query. Possible values: @@ -1469,7 +1469,7 @@ Default value: `1000`. ## compiled_expression_cache_size {#compiled-expression-cache-size} -Sets the cache size in bytes for [compiled expressions](../../operations/caches.md). +Sets the cache size (in bytes) for [compiled expressions](../../operations/caches.md). Possible values: @@ -1479,7 +1479,7 @@ Default value: `134217728`. ## compiled_expression_cache_elements_size {#compiled_expression_cache_elements_size} -Sets the cache size in elements for [compiled expressions](../../operations/caches.md). +Sets the cache size (in elements) for [compiled expressions](../../operations/caches.md). Possible values: diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index c8e3dadae97..01a61f5e789 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1422,3 +1422,54 @@ ClickHouse использует ZooKeeper для хранения метадан ``` + +## total_memory_profiler_step {#total-memory-profiler-step} + +Задает размер памяти (в байтах) для трассировки стека на каждом шаге выделения максимума памяти. Данные хранятся в системной таблице [system.trace_log](../../operations/system-tables/trace_log.md) с `query_id`, равным пустой строке. + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: `4194304`. + +## total_memory_tracker_sample_probability {#total-memory-tracker-sample-probability} + +Позволяет собирать случайные выделения и высвобождения памяти и записывать их в системную таблицу [system.trace_log](../../operations/system-tables/trace_log.md) с `trace_type`, равным equal `MemorySample` с указанной вероятностью. Вероятность касается каждого выделения или освобождения памяти, независимо от размера выделения. Обратите внимание, что выборка происходит только тогда, когда объем неотслеживаемой памяти превышает лимит неотслеживаемой памяти (значение по умолчанию: `4` MiB). Значение настройки может быть уменьшено, если значение настройки [total_memory_profiler_step](#total-memory-profiler-step) уменьшено. Вы можете установить значение настройки `total_memory_profiler_step`, равным `1` для особой детализованной выборки. + +Возможные значения: + +- Положительное целое число. +- 0 — запись случайных выделений и высвобождений памяти в системную таблицу `system.trace_log` отключена. + +Значение по умолчанию: `0`. + +## mmap_cache_size {#mmap-cache-size} + +Задает размер кэша (в байтах) для сопоставленных файлов. Эта настройка позволяет избежать частых открытых/[mmap/munmap](https://en.wikipedia.org/wiki/Mmap)/закрытых вызовов (которые очень дорогостоящие из-за последующих ошибок страниц) и повторного использования сопоставления из нескольких потоков и запросов. Значение настройки — это количество сопоставленных областей (обычно равно количеству сопоставленных файлов). Объем данных в сопоставленных файлах можно отслеживать в системных таблицах [system.metrics](../../operations/system-tables/metrics.md), [system.metric_log](../../operations/system-tables/metric_log.md) по метрикам `MMappedFiles` и `MMappedFileBytes`, в таблицах [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md), [system.asynchronous_metrics_log](../../operations/system-tables/asynchronous_metric_log.md) по метрике `MMapCacheCells`, а также в [system.events](../../operations/system-tables/events.md), [system.processes](../../operations/system-tables/processes.md), [system.query_log](../../operations/system-tables/query_log.md), [system.query_thread_log](../../operations/system-tables/query_thread_log.md), [system.query_views_log](../../operations/system-tables/query_views_log.md) по событиям `CreatedReadBufferMMap`, `CreatedReadBufferMMapFailed`, `MMappedFileCacheHits`, `MMappedFileCacheMisses`. Обратите внимание, что объем данных в сопоставленных файлах не потребляет память напрямую и не учитывается в запросе или использовании памяти сервера, поскольку эта память может быть удалена аналогично кэшу страниц ОС. Кэш удаляется (т.е. файлы закрываются) автоматически при удалении старых кусков в таблицах семейства [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md), также его можно удалить вручную с помощью запроса `SYSTEM DROP MMAP CACHE`. + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: `1000`. + +## compiled_expression_cache_size {#compiled-expression-cache-size} + +Задает размер кэша (в байтах) для [скомпилированных выражений](../../operations/caches.md). + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: `134217728`. + +## compiled_expression_cache_elements_size {#compiled_expression_cache_elements_size} + +Задает размер кэша (в элементах) для [скомпилированных выражений](../../operations/caches.md). + +Возможные значения: + +- Положительное целое число. + +Значение по умолчанию: `10000`. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index e3f5f2e002d..56b97deb7e9 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3791,3 +3791,13 @@ SELECT * FROM positional_arguments ORDER BY 2,3; Значение по умолчанию: `0`. +## min_bytes_to_use_mmap_io {#min-bytes-to-use-mmap-io} + +Это экспериментальная настройка. Устанавливает минимальный объем памяти для чтения больших файлов без копирования данных из ядра в пространство пользователей. Рекомендуемый лимит составляет около 64 MB, поскольку [mmap/munmap](https://en.wikipedia.org/wiki/Mmap) работает медленно. Это имеет смысл только для больших файлов и помогает только в том случае, если данные находятся в кэше страниц. + +Возможные значения: + +- Положительное целое число. +- 0 — большие файлы считываются только с копированием данных из ядра в пространство пользователей. + +Значение по умолчанию: `0`. From 881cbfa391b84e024ada3137b778a269fa9aa00e Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Sun, 14 Nov 2021 15:19:29 +0300 Subject: [PATCH 083/200] Update settings.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил перевод. --- .../operations/server-configuration-parameters/settings.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 01a61f5e789..e55e73db581 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1435,18 +1435,18 @@ ClickHouse использует ZooKeeper для хранения метадан ## total_memory_tracker_sample_probability {#total-memory-tracker-sample-probability} -Позволяет собирать случайные выделения и высвобождения памяти и записывать их в системную таблицу [system.trace_log](../../operations/system-tables/trace_log.md) с `trace_type`, равным equal `MemorySample` с указанной вероятностью. Вероятность касается каждого выделения или освобождения памяти, независимо от размера выделения. Обратите внимание, что выборка происходит только тогда, когда объем неотслеживаемой памяти превышает лимит неотслеживаемой памяти (значение по умолчанию: `4` MiB). Значение настройки может быть уменьшено, если значение настройки [total_memory_profiler_step](#total-memory-profiler-step) уменьшено. Вы можете установить значение настройки `total_memory_profiler_step`, равным `1` для особой детализованной выборки. +Позволяет собирать случайные выделения и освобождения памяти и записывать их в системную таблицу [system.trace_log](../../operations/system-tables/trace_log.md) с `trace_type`, равным `MemorySample`, с указанной вероятностью. Вероятность касается каждого выделения или освобождения памяти, независимо от размера выделения. Обратите внимание, что выборка происходит только тогда, когда объем неотслеживаемой памяти превышает лимит неотслеживаемой памяти (значение по умолчанию: `4` MiB). Значение настройки может быть уменьшено, если значение настройки [total_memory_profiler_step](#total-memory-profiler-step) уменьшено. Вы можете установить значение настройки `total_memory_profiler_step`, равным `1`, для особой детализованной выборки. Возможные значения: - Положительное целое число. -- 0 — запись случайных выделений и высвобождений памяти в системную таблицу `system.trace_log` отключена. +- 0 — запись случайных выделений и освобождений памяти в системную таблицу `system.trace_log` отключена. Значение по умолчанию: `0`. ## mmap_cache_size {#mmap-cache-size} -Задает размер кэша (в байтах) для сопоставленных файлов. Эта настройка позволяет избежать частых открытых/[mmap/munmap](https://en.wikipedia.org/wiki/Mmap)/закрытых вызовов (которые очень дорогостоящие из-за последующих ошибок страниц) и повторного использования сопоставления из нескольких потоков и запросов. Значение настройки — это количество сопоставленных областей (обычно равно количеству сопоставленных файлов). Объем данных в сопоставленных файлах можно отслеживать в системных таблицах [system.metrics](../../operations/system-tables/metrics.md), [system.metric_log](../../operations/system-tables/metric_log.md) по метрикам `MMappedFiles` и `MMappedFileBytes`, в таблицах [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md), [system.asynchronous_metrics_log](../../operations/system-tables/asynchronous_metric_log.md) по метрике `MMapCacheCells`, а также в [system.events](../../operations/system-tables/events.md), [system.processes](../../operations/system-tables/processes.md), [system.query_log](../../operations/system-tables/query_log.md), [system.query_thread_log](../../operations/system-tables/query_thread_log.md), [system.query_views_log](../../operations/system-tables/query_views_log.md) по событиям `CreatedReadBufferMMap`, `CreatedReadBufferMMapFailed`, `MMappedFileCacheHits`, `MMappedFileCacheMisses`. Обратите внимание, что объем данных в сопоставленных файлах не потребляет память напрямую и не учитывается в запросе или использовании памяти сервера, поскольку эта память может быть удалена аналогично кэшу страниц ОС. Кэш удаляется (т.е. файлы закрываются) автоматически при удалении старых кусков в таблицах семейства [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md), также его можно удалить вручную с помощью запроса `SYSTEM DROP MMAP CACHE`. +Задает размер кэша (в байтах) для сопоставленных файлов. Эта настройка позволяет избежать частых открытых/[mmap/munmap](https://en.wikipedia.org/wiki/Mmap)/закрытых вызовов (очень дорогостоящие из-за последующих ошибок страниц) и повторного использования сопоставления из нескольких потоков и запросов. Значение настройки — это количество сопоставленных областей (обычно равно количеству сопоставленных файлов). Объем данных в сопоставленных файлах можно отслеживать в системных таблицах [system.metrics](../../operations/system-tables/metrics.md), [system.metric_log](../../operations/system-tables/metric_log.md) по метрикам `MMappedFiles` и `MMappedFileBytes`, в таблицах [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md), [system.asynchronous_metrics_log](../../operations/system-tables/asynchronous_metric_log.md) по метрике `MMapCacheCells`, а также в [system.events](../../operations/system-tables/events.md), [system.processes](../../operations/system-tables/processes.md), [system.query_log](../../operations/system-tables/query_log.md), [system.query_thread_log](../../operations/system-tables/query_thread_log.md), [system.query_views_log](../../operations/system-tables/query_views_log.md) по событиям `CreatedReadBufferMMap`, `CreatedReadBufferMMapFailed`, `MMappedFileCacheHits`, `MMappedFileCacheMisses`. Обратите внимание, что объем данных в сопоставленных файлах не потребляет память напрямую и не учитывается в запросе или использовании памяти сервера, поскольку эта память может быть удалена аналогично кэшу страниц ОС. Кэш удаляется (т.е. файлы закрываются) автоматически при удалении старых кусков в таблицах семейства [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md), также его можно удалить вручную с помощью запроса `SYSTEM DROP MMAP CACHE`. Возможные значения: From b6aa781c9b1e7b94b0721e7c6f7bd95b468a45dc Mon Sep 17 00:00:00 2001 From: qieqieplus Date: Mon, 15 Nov 2021 14:47:34 +0800 Subject: [PATCH 084/200] add dot separator --- src/Interpreters/ActionsVisitor.cpp | 2 +- tests/queries/0_stateless/02113_untuple_func_alias.reference | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 4dadc8e0f49..d5eb701e2aa 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -705,7 +705,7 @@ ASTs ActionsMatcher::doUntuple(const ASTFunction * function, ActionsMatcher::Dat auto func = makeASTFunction("tupleElement", tuple_ast, literal); if (!func_alias.empty()) - func->setAlias(func_alias + toString(tid)); + func->setAlias(func_alias + "." + toString(tid)); auto function_builder = FunctionFactory::instance().get(func->name, data.getContext()); data.addFunction(function_builder, {tuple_name_type->name, literal->getColumnName()}, func->getColumnName()); diff --git a/tests/queries/0_stateless/02113_untuple_func_alias.reference b/tests/queries/0_stateless/02113_untuple_func_alias.reference index f78b11058b5..9985391d522 100644 --- a/tests/queries/0_stateless/02113_untuple_func_alias.reference +++ b/tests/queries/0_stateless/02113_untuple_func_alias.reference @@ -1,2 +1,2 @@ -ut1 ut2 ut3 ut4 ut21 ut22 ut23 ut24 +ut.1 ut.2 ut.3 ut.4 ut2.1 ut2.2 ut2.3 ut2.4 1 2 3 \N \N 3 2 1 From 5e9710a26ed9eee571be5bd274df0eef59b20476 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Mon, 8 Nov 2021 10:59:43 +0300 Subject: [PATCH 085/200] Attempt to resolve nullptr in STS credentials provider for S3. --- src/IO/S3Common.cpp | 160 +++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 159 insertions(+), 1 deletion(-) diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 89e1542d946..bca6b7e95f9 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -14,12 +14,14 @@ # include # include # include +# include # include # include # include # include # include # include +# include # include # include @@ -30,6 +32,8 @@ # include # include +# include + namespace { @@ -361,6 +365,156 @@ private: Poco::Logger * logger; }; +class AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider : public Aws::Auth::AWSCredentialsProvider +{ + /// See STSAssumeRoleWebIdentityCredentialsProvider. + +public: + AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider(DB::S3::PocoHTTPClientConfiguration & aws_client_configuration) + : logger(&Poco::Logger::get("AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider")) + + { + // check environment variables + String tmp_region = Aws::Environment::GetEnv("AWS_DEFAULT_REGION"); + role_arn = Aws::Environment::GetEnv("AWS_ROLE_ARN"); + token_file = Aws::Environment::GetEnv("AWS_WEB_IDENTITY_TOKEN_FILE"); + session_name = Aws::Environment::GetEnv("AWS_ROLE_SESSION_NAME"); + + // check profile_config if either m_roleArn or m_tokenFile is not loaded from environment variable + // region source is not enforced, but we need it to construct sts endpoint, if we can't find from environment, we should check if it's set in config file. + if (role_arn.empty() || token_file.empty() || tmp_region.empty()) + { + auto profile = Aws::Config::GetCachedConfigProfile(Aws::Auth::GetConfigProfileName()); + if (tmp_region.empty()) + { + tmp_region = profile.GetRegion(); + } + // If either of these two were not found from environment, use whatever found for all three in config file + if (role_arn.empty() || token_file.empty()) + { + role_arn = profile.GetRoleArn(); + token_file = profile.GetValue("web_identity_token_file"); + session_name = profile.GetValue("role_session_name"); + } + } + + if (token_file.empty()) + { + LOG_WARNING(logger, "Token file must be specified to use STS AssumeRole web identity creds provider."); + return; // No need to do further constructing + } + else + { + LOG_DEBUG(logger, "Resolved token_file from profile_config or environment variable to be {}", token_file); + } + + if (role_arn.empty()) + { + LOG_WARNING(logger, "RoleArn must be specified to use STS AssumeRole web identity creds provider."); + return; // No need to do further constructing + } + else + { + LOG_DEBUG(logger, "Resolved role_arn from profile_config or environment variable to be {}", role_arn); + } + + if (tmp_region.empty()) + { + tmp_region = Aws::Region::US_EAST_1; + } + else + { + LOG_DEBUG(logger, "Resolved region from profile_config or environment variable to be {}", tmp_region); + } + + if (session_name.empty()) + { + session_name = Aws::Utils::UUID::RandomUUID(); + } + else + { + LOG_DEBUG(logger, "Resolved session_name from profile_config or environment variable to be {}", session_name); + } + + aws_client_configuration.scheme = Aws::Http::Scheme::HTTPS; + aws_client_configuration.region = tmp_region; + + std::vector retryable_errors; + retryable_errors.push_back("IDPCommunicationError"); + retryable_errors.push_back("InvalidIdentityToken"); + + aws_client_configuration.retryStrategy = std::make_shared( + retryable_errors, /* maxRetries = */3); + + client = std::make_unique(aws_client_configuration); + initialized = true; + LOG_INFO(logger, "Creating STS AssumeRole with web identity creds provider."); + } + + Aws::Auth::AWSCredentials GetAWSCredentials() override + { + // A valid client means required information like role arn and token file were constructed correctly. + // We can use this provider to load creds, otherwise, we can just return empty creds. + if (!initialized) + { + return Aws::Auth::AWSCredentials(); + } + RefreshIfExpired(); + Aws::Utils::Threading::ReaderLockGuard guard(m_reloadLock); + return credentials; + } + +protected: + void Reload() override + { + LOG_INFO(logger, "Credentials have expired, attempting to renew from STS."); + + std::ifstream token_stream(token_file.data()); + if (token_stream) + { + String token_string((std::istreambuf_iterator(token_stream)), std::istreambuf_iterator()); + token = token_string; + } + else + { + LOG_INFO(logger, "Can't open token file: {}", token_file); + return; + } + Aws::Internal::STSCredentialsClient::STSAssumeRoleWithWebIdentityRequest request{session_name, role_arn, token}; + + auto result = client->GetAssumeRoleWithWebIdentityCredentials(request); + LOG_TRACE(logger, "Successfully retrieved credentials with AWS_ACCESS_KEY: {}", result.creds.GetAWSAccessKeyId()); + credentials = result.creds; + } + +private: + void RefreshIfExpired() + { + Aws::Utils::Threading::ReaderLockGuard guard(m_reloadLock); + if (!credentials.IsExpiredOrEmpty()) + { + return; + } + + guard.UpgradeToWriterLock(); + if (!credentials.IsExpiredOrEmpty()) // double-checked lock to avoid refreshing twice + { + return; + } + + Reload(); + } + + std::unique_ptr client; + Aws::Auth::AWSCredentials credentials; + Aws::String role_arn; + Aws::String token_file; + Aws::String session_name; + Aws::String token; + bool initialized = false; + Poco::Logger * logger; +}; + class S3CredentialsProviderChain : public Aws::Auth::AWSCredentialsProviderChain { public: @@ -381,7 +535,11 @@ public: AddProvider(std::make_shared()); AddProvider(std::make_shared()); AddProvider(std::make_shared()); - AddProvider(std::make_shared()); + + { + DB::S3::PocoHTTPClientConfiguration aws_client_configuration = DB::S3::ClientFactory::instance().createClientConfiguration(configuration.region, configuration.remote_host_filter, configuration.s3_max_redirects); + AddProvider(std::make_shared(aws_client_configuration)); + } /// ECS TaskRole Credentials only available when ENVIRONMENT VARIABLE is set. const auto relative_uri = Aws::Environment::GetEnv(AWS_ECS_CONTAINER_CREDENTIALS_RELATIVE_URI); From da76c4053c11b62f7e54755b86577cc64284995b Mon Sep 17 00:00:00 2001 From: lehasm Date: Mon, 15 Nov 2021 22:49:22 +0300 Subject: [PATCH 086/200] Update docs/en/sql-reference/operators/exists.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/en/sql-reference/operators/exists.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/operators/exists.md b/docs/en/sql-reference/operators/exists.md index c1d718e80ad..45ea584c3e6 100644 --- a/docs/en/sql-reference/operators/exists.md +++ b/docs/en/sql-reference/operators/exists.md @@ -1,6 +1,6 @@ # EXISTS {#exists-operator} -The `EXISTS` operator tests how many records are in the result of a subquery. If it is empty, then the operator returns `0`. Otherwise it returns `1`. +The `EXISTS` operator checks how many records are in the result of a subquery. If it is empty, then the operator returns `0`. Otherwise, it returns `1`. `EXISTS` can be used in a [WHERE](../../sql-reference/statements/select/where.md) clause. From a5dd4fccd40bfa749fa3c3172b5d3b7017c14303 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Mon, 15 Nov 2021 22:52:40 +0300 Subject: [PATCH 087/200] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index e55e73db581..30fc46e07ae 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1446,7 +1446,7 @@ ClickHouse использует ZooKeeper для хранения метадан ## mmap_cache_size {#mmap-cache-size} -Задает размер кэша (в байтах) для сопоставленных файлов. Эта настройка позволяет избежать частых открытых/[mmap/munmap](https://en.wikipedia.org/wiki/Mmap)/закрытых вызовов (очень дорогостоящие из-за последующих ошибок страниц) и повторного использования сопоставления из нескольких потоков и запросов. Значение настройки — это количество сопоставленных областей (обычно равно количеству сопоставленных файлов). Объем данных в сопоставленных файлах можно отслеживать в системных таблицах [system.metrics](../../operations/system-tables/metrics.md), [system.metric_log](../../operations/system-tables/metric_log.md) по метрикам `MMappedFiles` и `MMappedFileBytes`, в таблицах [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md), [system.asynchronous_metrics_log](../../operations/system-tables/asynchronous_metric_log.md) по метрике `MMapCacheCells`, а также в [system.events](../../operations/system-tables/events.md), [system.processes](../../operations/system-tables/processes.md), [system.query_log](../../operations/system-tables/query_log.md), [system.query_thread_log](../../operations/system-tables/query_thread_log.md), [system.query_views_log](../../operations/system-tables/query_views_log.md) по событиям `CreatedReadBufferMMap`, `CreatedReadBufferMMapFailed`, `MMappedFileCacheHits`, `MMappedFileCacheMisses`. Обратите внимание, что объем данных в сопоставленных файлах не потребляет память напрямую и не учитывается в запросе или использовании памяти сервера, поскольку эта память может быть удалена аналогично кэшу страниц ОС. Кэш удаляется (т.е. файлы закрываются) автоматически при удалении старых кусков в таблицах семейства [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md), также его можно удалить вручную с помощью запроса `SYSTEM DROP MMAP CACHE`. +Задает размер кеша (в байтах) для сопоставленных файлов. Эта настройка позволяет избежать частых открытых/[mmap/munmap](https://en.wikipedia.org/wiki/Mmap)/закрытых вызовов (очень дорогостоящие из-за последующих ошибок страниц) и повторного использования сопоставления из нескольких потоков и запросов. Значение настройки — это количество сопоставленных областей (обычно равно количеству сопоставленных файлов). Объем данных в сопоставленных файлах можно отслеживать в системных таблицах [system.metrics](../../operations/system-tables/metrics.md), [system.metric_log](../../operations/system-tables/metric_log.md) по метрикам `MMappedFiles` и `MMappedFileBytes`, в таблицах [system.asynchronous_metrics](../../operations/system-tables/asynchronous_metrics.md), [system.asynchronous_metrics_log](../../operations/system-tables/asynchronous_metric_log.md) по метрике `MMapCacheCells`, а также в [system.events](../../operations/system-tables/events.md), [system.processes](../../operations/system-tables/processes.md), [system.query_log](../../operations/system-tables/query_log.md), [system.query_thread_log](../../operations/system-tables/query_thread_log.md), [system.query_views_log](../../operations/system-tables/query_views_log.md) по событиям `CreatedReadBufferMMap`, `CreatedReadBufferMMapFailed`, `MMappedFileCacheHits`, `MMappedFileCacheMisses`. Обратите внимание, что объем данных в сопоставленных файлах не потребляет память напрямую и не учитывается в запросе или использовании памяти сервера, поскольку эта память может быть удалена аналогично кешу страниц ОС. Кеш удаляется (т.е. файлы закрываются) автоматически при удалении старых кусков в таблицах семейства [MergeTree](../../engines/table-engines/mergetree-family/mergetree.md), также его можно удалить вручную с помощью запроса `SYSTEM DROP MMAP CACHE`. Возможные значения: From eccb40da1ec9b908c0fe54305cebd01893148322 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Mon, 15 Nov 2021 22:52:48 +0300 Subject: [PATCH 088/200] Update docs/ru/operations/settings/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/operations/settings/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 56b97deb7e9..0fea824cc93 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -3793,7 +3793,7 @@ SELECT * FROM positional_arguments ORDER BY 2,3; ## min_bytes_to_use_mmap_io {#min-bytes-to-use-mmap-io} -Это экспериментальная настройка. Устанавливает минимальный объем памяти для чтения больших файлов без копирования данных из ядра в пространство пользователей. Рекомендуемый лимит составляет около 64 MB, поскольку [mmap/munmap](https://en.wikipedia.org/wiki/Mmap) работает медленно. Это имеет смысл только для больших файлов и помогает только в том случае, если данные находятся в кэше страниц. +Это экспериментальная настройка. Устанавливает минимальный объем памяти для чтения больших файлов без копирования данных из ядра в пространство пользователей. Рекомендуемый лимит составляет около 64 MB, поскольку [mmap/munmap](https://en.wikipedia.org/wiki/Mmap) работает медленно. Это имеет смысл только для больших файлов и помогает только в том случае, если данные находятся в кеше страниц. Возможные значения: From 553c97bd3ad87f249aa392ea60ca539ac9174831 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Mon, 15 Nov 2021 22:52:56 +0300 Subject: [PATCH 089/200] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index 30fc46e07ae..bf3a93f732d 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1466,7 +1466,7 @@ ClickHouse использует ZooKeeper для хранения метадан ## compiled_expression_cache_elements_size {#compiled_expression_cache_elements_size} -Задает размер кэша (в элементах) для [скомпилированных выражений](../../operations/caches.md). +Задает размер кеша (в элементах) для [скомпилированных выражений](../../operations/caches.md). Возможные значения: From 9d087b7e033b17c6d330541e2ceb700517541a7c Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Mon, 15 Nov 2021 22:53:03 +0300 Subject: [PATCH 090/200] Update docs/ru/operations/server-configuration-parameters/settings.md Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/operations/server-configuration-parameters/settings.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/server-configuration-parameters/settings.md b/docs/ru/operations/server-configuration-parameters/settings.md index bf3a93f732d..fe95f149105 100644 --- a/docs/ru/operations/server-configuration-parameters/settings.md +++ b/docs/ru/operations/server-configuration-parameters/settings.md @@ -1456,7 +1456,7 @@ ClickHouse использует ZooKeeper для хранения метадан ## compiled_expression_cache_size {#compiled-expression-cache-size} -Задает размер кэша (в байтах) для [скомпилированных выражений](../../operations/caches.md). +Задает размер кеша (в байтах) для [скомпилированных выражений](../../operations/caches.md). Возможные значения: From e42607834427114204c667c32a0c1ee4375b5d33 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Nov 2021 11:59:38 +0300 Subject: [PATCH 091/200] Trying to debug integration tests --- .github/workflows/main.yml | 2352 ++++++++++++++-------------- tests/ci/integration_test_check.py | 3 +- tests/integration/ci-runner.py | 19 +- tests/integration/runner | 6 +- 4 files changed, 1199 insertions(+), 1181 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 6725ea07bd6..00b6812787a 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -189,84 +189,84 @@ jobs: ########################################################################################## ##################################### ORDINARY BUILDS #################################### ########################################################################################## - BuilderDebRelease: - needs: DockerHubPush - if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} - runs-on: [self-hosted, builder] - steps: - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/images_path - - name: Check out repository code - uses: actions/checkout@v2 - with: - submodules: 'recursive' - fetch-depth: 0 # otherwise we will have no info about contributors - - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NUMBER: 0 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER - - name: Upload build URLs to artifacts - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - BuilderBinRelease: - needs: DockerHubPush - if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} - runs-on: [self-hosted, builder] - steps: - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/images_path - - name: Check out repository code - uses: actions/checkout@v2 - with: - submodules: 'recursive' - fetch-depth: 0 # otherwise we will have no info about contributors - - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NUMBER: 9 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER - - name: Upload build URLs to artifacts - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH +# BuilderDebRelease: +# needs: DockerHubPush +# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} +# runs-on: [self-hosted, builder] +# steps: +# - name: Download changed images +# uses: actions/download-artifact@v2 +# with: +# name: changed_images +# path: ${{ runner.temp }}/images_path +# - name: Check out repository code +# uses: actions/checkout@v2 +# with: +# submodules: 'recursive' +# fetch-depth: 0 # otherwise we will have no info about contributors +# - name: Build +# env: +# TEMP_PATH: ${{runner.temp}}/build_check +# IMAGES_PATH: ${{runner.temp}}/images_path +# REPO_COPY: ${{runner.temp}}/build_check/ClickHouse +# CACHES_PATH: ${{runner.temp}}/../ccaches +# CHECK_NAME: 'ClickHouse build check (actions)' +# BUILD_NUMBER: 0 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER +# - name: Upload build URLs to artifacts +# uses: actions/upload-artifact@v2 +# with: +# name: ${{ env.BUILD_NAME }} +# path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# BuilderBinRelease: +# needs: DockerHubPush +# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} +# runs-on: [self-hosted, builder] +# steps: +# - name: Download changed images +# uses: actions/download-artifact@v2 +# with: +# name: changed_images +# path: ${{ runner.temp }}/images_path +# - name: Check out repository code +# uses: actions/checkout@v2 +# with: +# submodules: 'recursive' +# fetch-depth: 0 # otherwise we will have no info about contributors +# - name: Build +# env: +# TEMP_PATH: ${{runner.temp}}/build_check +# IMAGES_PATH: ${{runner.temp}}/images_path +# REPO_COPY: ${{runner.temp}}/build_check/ClickHouse +# CACHES_PATH: ${{runner.temp}}/../ccaches +# CHECK_NAME: 'ClickHouse build check (actions)' +# BUILD_NUMBER: 9 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER +# - name: Upload build URLs to artifacts +# uses: actions/upload-artifact@v2 +# with: +# name: ${{ env.BUILD_NAME }} +# path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH BuilderDebAsan: needs: DockerHubPush if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} @@ -306,911 +306,628 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH - BuilderDebUBsan: - needs: DockerHubPush - if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} - runs-on: [self-hosted, builder] - steps: - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/images_path - - name: Check out repository code - uses: actions/checkout@v2 - with: - submodules: 'recursive' - fetch-depth: 0 # otherwise we will have no info about contributors - - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NUMBER: 4 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER - - name: Upload build URLs to artifacts - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - BuilderDebTsan: - needs: DockerHubPush - if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} - runs-on: [self-hosted, builder] - steps: - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/images_path - - name: Check out repository code - uses: actions/checkout@v2 - with: - submodules: 'recursive' - fetch-depth: 0 # otherwise we will have no info about contributors - - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NUMBER: 5 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER - - name: Upload build URLs to artifacts - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - BuilderDebMsan: - needs: DockerHubPush - if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} - runs-on: [self-hosted, builder] - steps: - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/images_path - - name: Check out repository code - uses: actions/checkout@v2 - with: - submodules: 'recursive' - fetch-depth: 0 # otherwise we will have no info about contributors - - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NUMBER: 6 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER - - name: Upload build URLs to artifacts - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - BuilderDebDebug: - needs: DockerHubPush - if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} - runs-on: [self-hosted, builder] - steps: - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/images_path - - name: Check out repository code - uses: actions/checkout@v2 - with: - submodules: 'recursive' - fetch-depth: 0 # otherwise we will have no info about contributors - - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse build check (actions)' - BUILD_NUMBER: 7 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER - - name: Upload build URLs to artifacts - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH -########################################################################################## -##################################### SPECIAL BUILDS ##################################### -########################################################################################## - BuilderDebSplitted: - needs: DockerHubPush - if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} - runs-on: [self-hosted, builder] - steps: - - name: Download changed images - uses: actions/download-artifact@v2 - with: - name: changed_images - path: ${{ runner.temp }}/images_path - - name: Check out repository code - uses: actions/checkout@v2 - with: - submodules: 'recursive' - fetch-depth: 0 # otherwise we will have no info about contributors - - name: Build - env: - TEMP_PATH: ${{runner.temp}}/build_check - IMAGES_PATH: ${{runner.temp}}/images_path - REPO_COPY: ${{runner.temp}}/build_check/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - CHECK_NAME: 'ClickHouse special build check (actions)' - BUILD_NUMBER: 1 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER - - name: Upload build URLs to artifacts - uses: actions/upload-artifact@v2 - with: - name: ${{ env.BUILD_NAME }} - path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH -############################################################################################ -##################################### BUILD REPORTER ####################################### -############################################################################################ - BuilderReport: - needs: - - BuilderDebRelease - - BuilderBinRelease - - BuilderDebAsan - - BuilderDebTsan - - BuilderDebUBsan - - BuilderDebMsan - - BuilderDebDebug - - BuilderDebSplitted - runs-on: [self-hosted, style-checker] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Report Builder - env: - TEMP_PATH: ${{runner.temp}}/report_check - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'ClickHouse build check (actions)' - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cd $GITHUB_WORKSPACE/tests/ci - python3 build_report_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH -############################################################################################## -########################### FUNCTIONAl STATELESS TESTS ####################################### -############################################################################################## - FunctionalStatelessTestRelease: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (release, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - FunctionalStatelessTestAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (address, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - FunctionalStatelessTestTsan: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/stateless_tsan/ClickHouse - KILL_TIMEOUT: 10800 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - FunctionalStatelessTestUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_ubsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (ubsan, actions)' - REPO_COPY: ${{runner.temp}}/stateless_ubsan/ClickHouse - KILL_TIMEOUT: 10800 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - FunctionalStatelessTestMsan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_memory - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (memory, actions)' - REPO_COPY: ${{runner.temp}}/stateless_memory/ClickHouse - KILL_TIMEOUT: 10800 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - FunctionalStatelessTestDebug: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests (debug, actions)' - REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse - KILL_TIMEOUT: 10800 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - FunctionalStatelessTestFlakyCheck: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateless_flaky_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateless tests flaky check (address, actions)' - REPO_COPY: ${{runner.temp}}/stateless_flaky_asan/ClickHouse - KILL_TIMEOUT: 3600 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH -############################################################################################## -############################ FUNCTIONAl STATEFUL TESTS ####################################### -############################################################################################## - FunctionalStatefulTestRelease: - needs: [BuilderDebRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (release, actions)' - REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT: 3600 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - FunctionalStatefulTestAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (address, actions)' - REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT: 3600 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - FunctionalStatefulTestTsan: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (thread, actions)' - REPO_COPY: ${{runner.temp}}/stateful_tsan/ClickHouse - KILL_TIMEOUT: 3600 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - FunctionalStatefulTestMsan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_msan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (memory, actions)' - REPO_COPY: ${{runner.temp}}/stateful_msan/ClickHouse - KILL_TIMEOUT: 3600 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - FunctionalStatefulTestUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_ubsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (ubsan, actions)' - REPO_COPY: ${{runner.temp}}/stateful_ubsan/ClickHouse - KILL_TIMEOUT: 3600 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - FunctionalStatefulTestDebug: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Functional test - env: - TEMP_PATH: ${{runner.temp}}/stateful_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stateful tests (debug, actions)' - REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse - KILL_TIMEOUT: 3600 - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH -############################################################################################## -######################################### STRESS TESTS ####################################### -############################################################################################## - StressTestAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_thread - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (address, actions)' - REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - StressTestTsan: - needs: [BuilderDebTsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_thread - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (thread, actions)' - REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - StressTestMsan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_memory - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (memory, actions)' - REPO_COPY: ${{runner.temp}}/stress_memory/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - StressTestUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, stress-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_undefined - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (undefined, actions)' - REPO_COPY: ${{runner.temp}}/stress_undefined/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - StressTestDebug: - needs: [BuilderDebDebug] - runs-on: [self-hosted, stress-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Stress test - env: - TEMP_PATH: ${{runner.temp}}/stress_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Stress test (debug, actions)' - REPO_COPY: ${{runner.temp}}/stress_debug/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 stress_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH -############################################################################################## -##################################### AST FUZZERS ############################################ -############################################################################################## - ASTFuzzerTestAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - env: - TEMP_PATH: ${{runner.temp}}/ast_fuzzer_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'AST fuzzer (ASan, actions)' - REPO_COPY: ${{runner.temp}}/ast_fuzzer_asan/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - ASTFuzzerTestTsan: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - env: - TEMP_PATH: ${{runner.temp}}/ast_fuzzer_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'AST fuzzer (TSan, actions)' - REPO_COPY: ${{runner.temp}}/ast_fuzzer_tsan/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - ASTFuzzerTestUBSan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - env: - TEMP_PATH: ${{runner.temp}}/ast_fuzzer_ubsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'AST fuzzer (UBSan, actions)' - REPO_COPY: ${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - ASTFuzzerTestMSan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - env: - TEMP_PATH: ${{runner.temp}}/ast_fuzzer_msan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'AST fuzzer (MSan, actions)' - REPO_COPY: ${{runner.temp}}/ast_fuzzer_msan/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - ASTFuzzerTestDebug: - needs: [BuilderDebDebug] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fuzzer - env: - TEMP_PATH: ${{runner.temp}}/ast_fuzzer_debug - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'AST fuzzer (debug, actions)' - REPO_COPY: ${{runner.temp}}/ast_fuzzer_debug/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 ast_fuzzer_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH -# IntegrationTestsAsan: +# BuilderDebUBsan: +# needs: DockerHubPush +# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} +# runs-on: [self-hosted, builder] +# steps: +# - name: Download changed images +# uses: actions/download-artifact@v2 +# with: +# name: changed_images +# path: ${{ runner.temp }}/images_path +# - name: Check out repository code +# uses: actions/checkout@v2 +# with: +# submodules: 'recursive' +# fetch-depth: 0 # otherwise we will have no info about contributors +# - name: Build +# env: +# TEMP_PATH: ${{runner.temp}}/build_check +# IMAGES_PATH: ${{runner.temp}}/images_path +# REPO_COPY: ${{runner.temp}}/build_check/ClickHouse +# CACHES_PATH: ${{runner.temp}}/../ccaches +# CHECK_NAME: 'ClickHouse build check (actions)' +# BUILD_NUMBER: 4 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER +# - name: Upload build URLs to artifacts +# uses: actions/upload-artifact@v2 +# with: +# name: ${{ env.BUILD_NAME }} +# path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# BuilderDebTsan: +# needs: DockerHubPush +# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} +# runs-on: [self-hosted, builder] +# steps: +# - name: Download changed images +# uses: actions/download-artifact@v2 +# with: +# name: changed_images +# path: ${{ runner.temp }}/images_path +# - name: Check out repository code +# uses: actions/checkout@v2 +# with: +# submodules: 'recursive' +# fetch-depth: 0 # otherwise we will have no info about contributors +# - name: Build +# env: +# TEMP_PATH: ${{runner.temp}}/build_check +# IMAGES_PATH: ${{runner.temp}}/images_path +# REPO_COPY: ${{runner.temp}}/build_check/ClickHouse +# CACHES_PATH: ${{runner.temp}}/../ccaches +# CHECK_NAME: 'ClickHouse build check (actions)' +# BUILD_NUMBER: 5 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER +# - name: Upload build URLs to artifacts +# uses: actions/upload-artifact@v2 +# with: +# name: ${{ env.BUILD_NAME }} +# path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# BuilderDebMsan: +# needs: DockerHubPush +# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} +# runs-on: [self-hosted, builder] +# steps: +# - name: Download changed images +# uses: actions/download-artifact@v2 +# with: +# name: changed_images +# path: ${{ runner.temp }}/images_path +# - name: Check out repository code +# uses: actions/checkout@v2 +# with: +# submodules: 'recursive' +# fetch-depth: 0 # otherwise we will have no info about contributors +# - name: Build +# env: +# TEMP_PATH: ${{runner.temp}}/build_check +# IMAGES_PATH: ${{runner.temp}}/images_path +# REPO_COPY: ${{runner.temp}}/build_check/ClickHouse +# CACHES_PATH: ${{runner.temp}}/../ccaches +# CHECK_NAME: 'ClickHouse build check (actions)' +# BUILD_NUMBER: 6 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER +# - name: Upload build URLs to artifacts +# uses: actions/upload-artifact@v2 +# with: +# name: ${{ env.BUILD_NAME }} +# path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# BuilderDebDebug: +# needs: DockerHubPush +# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} +# runs-on: [self-hosted, builder] +# steps: +# - name: Download changed images +# uses: actions/download-artifact@v2 +# with: +# name: changed_images +# path: ${{ runner.temp }}/images_path +# - name: Check out repository code +# uses: actions/checkout@v2 +# with: +# submodules: 'recursive' +# fetch-depth: 0 # otherwise we will have no info about contributors +# - name: Build +# env: +# TEMP_PATH: ${{runner.temp}}/build_check +# IMAGES_PATH: ${{runner.temp}}/images_path +# REPO_COPY: ${{runner.temp}}/build_check/ClickHouse +# CACHES_PATH: ${{runner.temp}}/../ccaches +# CHECK_NAME: 'ClickHouse build check (actions)' +# BUILD_NUMBER: 7 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER +# - name: Upload build URLs to artifacts +# uses: actions/upload-artifact@v2 +# with: +# name: ${{ env.BUILD_NAME }} +# path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +########################################################################################### +###################################### SPECIAL BUILDS ##################################### +########################################################################################### +# BuilderDebSplitted: +# needs: DockerHubPush +# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} +# runs-on: [self-hosted, builder] +# steps: +# - name: Download changed images +# uses: actions/download-artifact@v2 +# with: +# name: changed_images +# path: ${{ runner.temp }}/images_path +# - name: Check out repository code +# uses: actions/checkout@v2 +# with: +# submodules: 'recursive' +# fetch-depth: 0 # otherwise we will have no info about contributors +# - name: Build +# env: +# TEMP_PATH: ${{runner.temp}}/build_check +# IMAGES_PATH: ${{runner.temp}}/images_path +# REPO_COPY: ${{runner.temp}}/build_check/ClickHouse +# CACHES_PATH: ${{runner.temp}}/../ccaches +# CHECK_NAME: 'ClickHouse special build check (actions)' +# BUILD_NUMBER: 1 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER +# - name: Upload build URLs to artifacts +# uses: actions/upload-artifact@v2 +# with: +# name: ${{ env.BUILD_NAME }} +# path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +############################################################################################# +###################################### BUILD REPORTER ####################################### +############################################################################################# +# BuilderReport: +# needs: +# - BuilderDebRelease +# - BuilderBinRelease +# - BuilderDebAsan +# - BuilderDebTsan +# - BuilderDebUBsan +# - BuilderDebMsan +# - BuilderDebDebug +# - BuilderDebSplitted +# runs-on: [self-hosted, style-checker] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Report Builder +# env: +# TEMP_PATH: ${{runner.temp}}/report_check +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'ClickHouse build check (actions)' +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cd $GITHUB_WORKSPACE/tests/ci +# python3 build_report_check.py "$CHECK_NAME" +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +############################################################################################### +############################ FUNCTIONAl STATELESS TESTS ####################################### +############################################################################################### +# FunctionalStatelessTestRelease: +# needs: [BuilderDebRelease] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Functional test +# env: +# TEMP_PATH: ${{runner.temp}}/stateless_debug +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stateless tests (release, actions)' +# REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse +# KILL_TIMEOUT: 10800 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# FunctionalStatelessTestAsan: +# needs: [BuilderDebAsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Functional test +# env: +# TEMP_PATH: ${{runner.temp}}/stateless_debug +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stateless tests (address, actions)' +# REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse +# KILL_TIMEOUT: 10800 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# FunctionalStatelessTestTsan: +# needs: [BuilderDebTsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Functional test +# env: +# TEMP_PATH: ${{runner.temp}}/stateless_tsan +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stateless tests (thread, actions)' +# REPO_COPY: ${{runner.temp}}/stateless_tsan/ClickHouse +# KILL_TIMEOUT: 10800 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# FunctionalStatelessTestUBsan: +# needs: [BuilderDebUBsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Functional test +# env: +# TEMP_PATH: ${{runner.temp}}/stateless_ubsan +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stateless tests (ubsan, actions)' +# REPO_COPY: ${{runner.temp}}/stateless_ubsan/ClickHouse +# KILL_TIMEOUT: 10800 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# FunctionalStatelessTestMsan: +# needs: [BuilderDebMsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Functional test +# env: +# TEMP_PATH: ${{runner.temp}}/stateless_memory +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stateless tests (memory, actions)' +# REPO_COPY: ${{runner.temp}}/stateless_memory/ClickHouse +# KILL_TIMEOUT: 10800 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# FunctionalStatelessTestDebug: +# needs: [BuilderDebDebug] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Functional test +# env: +# TEMP_PATH: ${{runner.temp}}/stateless_debug +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stateless tests (debug, actions)' +# REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse +# KILL_TIMEOUT: 10800 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# FunctionalStatelessTestFlakyCheck: +# needs: [BuilderDebAsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Functional test +# env: +# TEMP_PATH: ${{runner.temp}}/stateless_flaky_asan +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stateless tests flaky check (address, actions)' +# REPO_COPY: ${{runner.temp}}/stateless_flaky_asan/ClickHouse +# KILL_TIMEOUT: 3600 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +############################################################################################### +############################# FUNCTIONAl STATEFUL TESTS ####################################### +############################################################################################### +# FunctionalStatefulTestRelease: +# needs: [BuilderDebRelease] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Functional test +# env: +# TEMP_PATH: ${{runner.temp}}/stateful_debug +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stateful tests (release, actions)' +# REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse +# KILL_TIMEOUT: 3600 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# FunctionalStatefulTestAsan: +# needs: [BuilderDebAsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Functional test +# env: +# TEMP_PATH: ${{runner.temp}}/stateful_debug +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stateful tests (address, actions)' +# REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse +# KILL_TIMEOUT: 3600 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# FunctionalStatefulTestTsan: +# needs: [BuilderDebTsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Functional test +# env: +# TEMP_PATH: ${{runner.temp}}/stateful_tsan +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stateful tests (thread, actions)' +# REPO_COPY: ${{runner.temp}}/stateful_tsan/ClickHouse +# KILL_TIMEOUT: 3600 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# FunctionalStatefulTestMsan: +# needs: [BuilderDebMsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Functional test +# env: +# TEMP_PATH: ${{runner.temp}}/stateful_msan +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stateful tests (memory, actions)' +# REPO_COPY: ${{runner.temp}}/stateful_msan/ClickHouse +# KILL_TIMEOUT: 3600 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# FunctionalStatefulTestUBsan: +# needs: [BuilderDebUBsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Functional test +# env: +# TEMP_PATH: ${{runner.temp}}/stateful_ubsan +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stateful tests (ubsan, actions)' +# REPO_COPY: ${{runner.temp}}/stateful_ubsan/ClickHouse +# KILL_TIMEOUT: 3600 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# FunctionalStatefulTestDebug: +# needs: [BuilderDebDebug] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Functional test +# env: +# TEMP_PATH: ${{runner.temp}}/stateful_debug +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stateful tests (debug, actions)' +# REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse +# KILL_TIMEOUT: 3600 +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +############################################################################################### +########################################## STRESS TESTS ####################################### +############################################################################################### +# StressTestAsan: # needs: [BuilderDebAsan] # runs-on: [self-hosted, stress-tester] # steps: @@ -1220,211 +937,496 @@ jobs: # path: ${{runner.temp}}/reports_dir # - name: Check out repository code # uses: actions/checkout@v2 -# - name: Integration test +# - name: Stress test # env: -# TEMP_PATH: ${{runner.temp}}/integration_tests_asan +# TEMP_PATH: ${{runner.temp}}/stress_thread # REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Integration tests (asan, actions)' -# REPO_COPY: ${{runner.temp}}/integration_tests_asan/ClickHouse +# CHECK_NAME: 'Stress test (address, actions)' +# REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse # run: | # sudo rm -fr $TEMP_PATH # mkdir -p $TEMP_PATH # cp -r $GITHUB_WORKSPACE $TEMP_PATH # cd $REPO_COPY/tests/ci -# python3 integration_test_check.py "$CHECK_NAME" +# python3 stress_check.py "$CHECK_NAME" # - name: Cleanup # if: always() # run: | # docker kill $(docker ps -q) ||: # docker rm -f $(docker ps -a -q) ||: # sudo rm -fr $TEMP_PATH - +# StressTestTsan: +# needs: [BuilderDebTsan] +# runs-on: [self-hosted, stress-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Stress test +# env: +# TEMP_PATH: ${{runner.temp}}/stress_thread +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stress test (thread, actions)' +# REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 stress_check.py "$CHECK_NAME" +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# StressTestMsan: +# needs: [BuilderDebMsan] +# runs-on: [self-hosted, stress-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Stress test +# env: +# TEMP_PATH: ${{runner.temp}}/stress_memory +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stress test (memory, actions)' +# REPO_COPY: ${{runner.temp}}/stress_memory/ClickHouse +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 stress_check.py "$CHECK_NAME" +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# StressTestUBsan: +# needs: [BuilderDebUBsan] +# runs-on: [self-hosted, stress-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Stress test +# env: +# TEMP_PATH: ${{runner.temp}}/stress_undefined +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stress test (undefined, actions)' +# REPO_COPY: ${{runner.temp}}/stress_undefined/ClickHouse +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 stress_check.py "$CHECK_NAME" +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# StressTestDebug: +# needs: [BuilderDebDebug] +# runs-on: [self-hosted, stress-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Stress test +# env: +# TEMP_PATH: ${{runner.temp}}/stress_debug +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Stress test (debug, actions)' +# REPO_COPY: ${{runner.temp}}/stress_debug/ClickHouse +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 stress_check.py "$CHECK_NAME" +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +############################################################################################### +###################################### AST FUZZERS ############################################ +############################################################################################### +# ASTFuzzerTestAsan: +# needs: [BuilderDebAsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Fuzzer +# env: +# TEMP_PATH: ${{runner.temp}}/ast_fuzzer_asan +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'AST fuzzer (ASan, actions)' +# REPO_COPY: ${{runner.temp}}/ast_fuzzer_asan/ClickHouse +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 ast_fuzzer_check.py "$CHECK_NAME" +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# ASTFuzzerTestTsan: +# needs: [BuilderDebTsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Fuzzer +# env: +# TEMP_PATH: ${{runner.temp}}/ast_fuzzer_tsan +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'AST fuzzer (TSan, actions)' +# REPO_COPY: ${{runner.temp}}/ast_fuzzer_tsan/ClickHouse +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 ast_fuzzer_check.py "$CHECK_NAME" +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# ASTFuzzerTestUBSan: +# needs: [BuilderDebUBsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Fuzzer +# env: +# TEMP_PATH: ${{runner.temp}}/ast_fuzzer_ubsan +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'AST fuzzer (UBSan, actions)' +# REPO_COPY: ${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 ast_fuzzer_check.py "$CHECK_NAME" +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# ASTFuzzerTestMSan: +# needs: [BuilderDebMsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Fuzzer +# env: +# TEMP_PATH: ${{runner.temp}}/ast_fuzzer_msan +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'AST fuzzer (MSan, actions)' +# REPO_COPY: ${{runner.temp}}/ast_fuzzer_msan/ClickHouse +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 ast_fuzzer_check.py "$CHECK_NAME" +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# ASTFuzzerTestDebug: +# needs: [BuilderDebDebug] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Fuzzer +# env: +# TEMP_PATH: ${{runner.temp}}/ast_fuzzer_debug +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'AST fuzzer (debug, actions)' +# REPO_COPY: ${{runner.temp}}/ast_fuzzer_debug/ClickHouse +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 ast_fuzzer_check.py "$CHECK_NAME" +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +############################################################################################## +############################## INTEGRATION TESTS ############################################# +############################################################################################## + IntegrationTestsAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + env: + TEMP_PATH: ${{runner.temp}}/integration_tests_asan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Integration tests (asan, actions)' + REPO_COPY: ${{runner.temp}}/integration_tests_asan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH ############################################################################################## ##################################### UNIT TESTS ############################################# ############################################################################################## - UnitTestsAsan: - needs: [BuilderDebAsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - env: - TEMP_PATH: ${{runner.temp}}/unit_tests_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (asan, actions)' - REPO_COPY: ${{runner.temp}}/unit_tests_asan/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - UnitTestsReleaseClang: - needs: [BuilderBinRelease] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - env: - TEMP_PATH: ${{runner.temp}}/unit_tests_asan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (release-clang, actions)' - REPO_COPY: ${{runner.temp}}/unit_tests_asan/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - UnitTestsTsan: - needs: [BuilderDebTsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - env: - TEMP_PATH: ${{runner.temp}}/unit_tests_tsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (tsan, actions)' - REPO_COPY: ${{runner.temp}}/unit_tests_tsan/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - UnitTestsMsan: - needs: [BuilderDebMsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - env: - TEMP_PATH: ${{runner.temp}}/unit_tests_msan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (msan, actions)' - REPO_COPY: ${{runner.temp}}/unit_tests_msan/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - UnitTestsUBsan: - needs: [BuilderDebUBsan] - runs-on: [self-hosted, func-tester] - steps: - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Check out repository code - uses: actions/checkout@v2 - - name: Unit test - env: - TEMP_PATH: ${{runner.temp}}/unit_tests_ubsan - REPORTS_PATH: ${{runner.temp}}/reports_dir - CHECK_NAME: 'Unit tests (msan, actions)' - REPO_COPY: ${{runner.temp}}/unit_tests_ubsan/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci - python3 unit_tests_check.py "$CHECK_NAME" - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - FinishCheck: - needs: - - StyleCheck - - DockerHubPush - - CheckLabels - - BuilderReport - - FastTest - - FunctionalStatelessTestDebug - - FunctionalStatelessTestRelease - - FunctionalStatelessTestAsan - - FunctionalStatelessTestTsan - - FunctionalStatelessTestMsan - - FunctionalStatelessTestUBsan - - FunctionalStatefulTestDebug - - FunctionalStatefulTestRelease - - FunctionalStatefulTestAsan - - FunctionalStatefulTestTsan - - FunctionalStatefulTestMsan - - FunctionalStatefulTestUBsan - - DocsCheck - - StressTestDebug - - StressTestAsan - - StressTestTsan - - StressTestMsan - - StressTestUBsan - - ASTFuzzerTestDebug - - ASTFuzzerTestAsan - - ASTFuzzerTestTsan - - ASTFuzzerTestMSan - - ASTFuzzerTestUBSan - - PVSCheck - - UnitTestsAsan - - UnitTestsTsan - - UnitTestsMsan - - UnitTestsUBsan - - UnitTestsReleaseClang - - SplitBuildSmokeTest - - CompatibilityCheck - runs-on: [self-hosted, style-checker] - steps: - - name: Check out repository code - uses: actions/checkout@v2 - - name: Finish label - run: | - cd $GITHUB_WORKSPACE/tests/ci - python3 finish_check.py +# UnitTestsAsan: +# needs: [BuilderDebAsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Unit test +# env: +# TEMP_PATH: ${{runner.temp}}/unit_tests_asan +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Unit tests (asan, actions)' +# REPO_COPY: ${{runner.temp}}/unit_tests_asan/ClickHouse +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 unit_tests_check.py "$CHECK_NAME" +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# UnitTestsReleaseClang: +# needs: [BuilderBinRelease] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Unit test +# env: +# TEMP_PATH: ${{runner.temp}}/unit_tests_asan +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Unit tests (release-clang, actions)' +# REPO_COPY: ${{runner.temp}}/unit_tests_asan/ClickHouse +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 unit_tests_check.py "$CHECK_NAME" +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# UnitTestsTsan: +# needs: [BuilderDebTsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Unit test +# env: +# TEMP_PATH: ${{runner.temp}}/unit_tests_tsan +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Unit tests (tsan, actions)' +# REPO_COPY: ${{runner.temp}}/unit_tests_tsan/ClickHouse +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 unit_tests_check.py "$CHECK_NAME" +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# UnitTestsMsan: +# needs: [BuilderDebMsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Unit test +# env: +# TEMP_PATH: ${{runner.temp}}/unit_tests_msan +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Unit tests (msan, actions)' +# REPO_COPY: ${{runner.temp}}/unit_tests_msan/ClickHouse +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 unit_tests_check.py "$CHECK_NAME" +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# UnitTestsUBsan: +# needs: [BuilderDebUBsan] +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Unit test +# env: +# TEMP_PATH: ${{runner.temp}}/unit_tests_ubsan +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# CHECK_NAME: 'Unit tests (msan, actions)' +# REPO_COPY: ${{runner.temp}}/unit_tests_ubsan/ClickHouse +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci +# python3 unit_tests_check.py "$CHECK_NAME" +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# FinishCheck: +# needs: +# - StyleCheck +# - DockerHubPush +# - CheckLabels +# - BuilderReport +# - FastTest +# - FunctionalStatelessTestDebug +# - FunctionalStatelessTestRelease +# - FunctionalStatelessTestAsan +# - FunctionalStatelessTestTsan +# - FunctionalStatelessTestMsan +# - FunctionalStatelessTestUBsan +# - FunctionalStatefulTestDebug +# - FunctionalStatefulTestRelease +# - FunctionalStatefulTestAsan +# - FunctionalStatefulTestTsan +# - FunctionalStatefulTestMsan +# - FunctionalStatefulTestUBsan +# - DocsCheck +# - StressTestDebug +# - StressTestAsan +# - StressTestTsan +# - StressTestMsan +# - StressTestUBsan +# - ASTFuzzerTestDebug +# - ASTFuzzerTestAsan +# - ASTFuzzerTestTsan +# - ASTFuzzerTestMSan +# - ASTFuzzerTestUBSan +# - PVSCheck +# - UnitTestsAsan +# - UnitTestsTsan +# - UnitTestsMsan +# - UnitTestsUBsan +# - UnitTestsReleaseClang +# - SplitBuildSmokeTest +# - CompatibilityCheck +# runs-on: [self-hosted, style-checker] +# steps: +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Finish label +# run: | +# cd $GITHUB_WORKSPACE/tests/ci +# python3 finish_check.py diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index d58f043e8ba..7f24fb44e79 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -39,6 +39,8 @@ def get_json_params_dict(check_name, commit_sha, pr_number, docker_images): 'pr_info': None, 'docker_images_with_versions': docker_images, 'shuffle_test_groups': False, + 'use_tmpfs': False, + 'disable_net_host': True, } def get_env_for_runner(build_path, repo_path, result_path, work_path): @@ -142,7 +144,6 @@ if __name__ == "__main__": else: logging.info("Some tests failed") - subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) state, description, test_results, additional_logs = process_results(result_path) diff --git a/tests/integration/ci-runner.py b/tests/integration/ci-runner.py index f54f172d8f4..25d09a8c4c5 100755 --- a/tests/integration/ci-runner.py +++ b/tests/integration/ci-runner.py @@ -177,6 +177,9 @@ class ClickhouseIntegrationTestsRunner: self.image_versions = self.params['docker_images_with_versions'] self.shuffle_groups = self.params['shuffle_test_groups'] self.flaky_check = 'flaky check' in self.params['context_name'] + # if use_tmpfs is not set we assume it to be true, otherwise check + self.use_tmpfs = 'use_tmpfs' not in self.params or self.params['use_tmpfs'] + self.disable_net_host = 'disable_net_host' in self.params and self.params['disable_net_host'] self.start_time = time.time() self.soft_deadline_time = self.start_time + (TASK_TIMEOUT - MAX_TIME_IN_SANDBOX) @@ -257,15 +260,23 @@ class ClickhouseIntegrationTestsRunner: def _compress_logs(self, dir, relpaths, result_path): subprocess.check_call("tar czf {} -C {} {}".format(result_path, dir, ' '.join(relpaths)), shell=True) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL + def _get_runner_opts(self): + result = [] + if self.use_tmpfs: + result.append("--tmpfs") + if self.disable_net_host: + result.append("--disable-net-host") + return " ".join(result) + def _get_all_tests(self, repo_path): image_cmd = self._get_runner_image_cmd(repo_path) out_file = "all_tests.txt" out_file_full = "all_tests_full.txt" cmd = "cd {repo_path}/tests/integration && " \ - "timeout -s 9 1h ./runner --tmpfs {image_cmd} ' --setup-plan' " \ + "timeout -s 9 1h ./runner {runner_opts} {image_cmd} ' --setup-plan' " \ "| tee {out_file_full} | grep '::' | sed 's/ (fixtures used:.*//g' | sed 's/^ *//g' | sed 's/ *$//g' " \ "| grep -v 'SKIPPED' | sort -u > {out_file}".format( - repo_path=repo_path, image_cmd=image_cmd, out_file=out_file, out_file_full=out_file_full) + repo_path=repo_path, runner_opts=self._get_runner_opts(), image_cmd=image_cmd, out_file=out_file, out_file_full=out_file_full) logging.info("Getting all tests with cmd '%s'", cmd) subprocess.check_call(cmd, shell=True) # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL @@ -435,8 +446,8 @@ class ClickhouseIntegrationTestsRunner: test_cmd = ' '.join([test for test in sorted(test_names)]) parallel_cmd = " --parallel {} ".format(num_workers) if num_workers > 0 else "" - cmd = "cd {}/tests/integration && timeout -s 9 1h ./runner --tmpfs {} -t {} {} '-rfEp --run-id={} --color=no --durations=0 {}' | tee {}".format( - repo_path, image_cmd, test_cmd, parallel_cmd, i, _get_deselect_option(self.should_skip_tests()), info_path) + cmd = "cd {}/tests/integration && timeout -s 9 1h ./runner {} {} -t {} {} '-rfEp --run-id={} --color=no --durations=0 {}' | tee {}".format( + repo_path, self._get_runner_opts(), image_cmd, test_cmd, parallel_cmd, i, _get_deselect_option(self.should_skip_tests()), info_path) log_basename = test_group_str + "_" + str(i) + ".log" log_path = os.path.join(repo_path, "tests/integration", log_basename) diff --git a/tests/integration/runner b/tests/integration/runner index 03ea091d750..abff7cefce4 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -251,7 +251,11 @@ if __name__ == "__main__": elif args.dockerd_volume: dockerd_internal_volume = "--mount type=bind,source={},target=/var/lib/docker".format(args.dockerd_volume) else: - subprocess.check_call('docker volume create {name}_volume'.format(name=CONTAINER_NAME), shell=True) + try: + subprocess.check_call('docker volume create {name}_volume'.format(name=CONTAINER_NAME), shell=True) + except Exception as ex: + print("Volume creationg failed, probably it already exists, exception", ex) + dockerd_internal_volume = "--volume={}_volume:/var/lib/docker".format(CONTAINER_NAME) # If enabled we kill and remove containers before pytest session run. From f5a81767e5987eda67444e15504f4883599daf5e Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Nov 2021 12:01:12 +0300 Subject: [PATCH 092/200] Fix main.yml --- .github/workflows/main.yml | 202 ++++++++++++++++++------------------- 1 file changed, 101 insertions(+), 101 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 00b6812787a..b7c0347e593 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -85,107 +85,107 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH - FastTest: - needs: DockerHubPush - if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} - runs-on: [self-hosted, builder] - steps: - - name: Check out repository code - uses: actions/checkout@v2 - - name: Fast Test - env: - TEMP_PATH: ${{runner.temp}}/fasttest - REPO_COPY: ${{runner.temp}}/fasttest/ClickHouse - CACHES_PATH: ${{runner.temp}}/../ccaches - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci && python3 fast_test_check.py - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - PVSCheck: - needs: DockerHubPush - if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} - runs-on: [self-hosted, func-tester] - steps: - - name: Check out repository code - uses: actions/checkout@v2 - with: - submodules: 'recursive' - - name: PVS Check - env: - TEMP_PATH: ${{runner.temp}}/pvs_check - REPO_COPY: ${{runner.temp}}/pvs_check/ClickHouse - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci && python3 pvs_check.py - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - CompatibilityCheck: - needs: [BuilderDebRelease] - if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} - runs-on: [self-hosted, style-checker] - steps: - - name: Check out repository code - uses: actions/checkout@v2 - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: CompatibilityCheck - env: - TEMP_PATH: ${{runner.temp}}/compatibility_check - REPO_COPY: ${{runner.temp}}/compatibility_check/ClickHouse - REPORTS_PATH: ${{runner.temp}}/reports_dir - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci && python3 compatibility_check.py 0 - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH - SplitBuildSmokeTest: - needs: [BuilderDebSplitted] - if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} - runs-on: [self-hosted, style-checker] - steps: - - name: Check out repository code - uses: actions/checkout@v2 - - name: Download json reports - uses: actions/download-artifact@v2 - with: - path: ${{runner.temp}}/reports_dir - - name: Split build check - env: - TEMP_PATH: ${{runner.temp}}/split_build_check - REPO_COPY: ${{runner.temp}}/split_build_check/ClickHouse - REPORTS_PATH: ${{runner.temp}}/reports_dir - run: | - sudo rm -fr $TEMP_PATH - mkdir -p $TEMP_PATH - cp -r $GITHUB_WORKSPACE $TEMP_PATH - cd $REPO_COPY/tests/ci && python3 split_build_smoke_check.py - - name: Cleanup - if: always() - run: | - docker kill $(docker ps -q) ||: - docker rm -f $(docker ps -a -q) ||: - sudo rm -fr $TEMP_PATH +# FastTest: +# needs: DockerHubPush +# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} +# runs-on: [self-hosted, builder] +# steps: +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Fast Test +# env: +# TEMP_PATH: ${{runner.temp}}/fasttest +# REPO_COPY: ${{runner.temp}}/fasttest/ClickHouse +# CACHES_PATH: ${{runner.temp}}/../ccaches +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci && python3 fast_test_check.py +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# PVSCheck: +# needs: DockerHubPush +# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} +# runs-on: [self-hosted, func-tester] +# steps: +# - name: Check out repository code +# uses: actions/checkout@v2 +# with: +# submodules: 'recursive' +# - name: PVS Check +# env: +# TEMP_PATH: ${{runner.temp}}/pvs_check +# REPO_COPY: ${{runner.temp}}/pvs_check/ClickHouse +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci && python3 pvs_check.py +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# CompatibilityCheck: +# needs: [BuilderDebRelease] +# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} +# runs-on: [self-hosted, style-checker] +# steps: +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: CompatibilityCheck +# env: +# TEMP_PATH: ${{runner.temp}}/compatibility_check +# REPO_COPY: ${{runner.temp}}/compatibility_check/ClickHouse +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci && python3 compatibility_check.py 0 +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH +# SplitBuildSmokeTest: +# needs: [BuilderDebSplitted] +# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} +# runs-on: [self-hosted, style-checker] +# steps: +# - name: Check out repository code +# uses: actions/checkout@v2 +# - name: Download json reports +# uses: actions/download-artifact@v2 +# with: +# path: ${{runner.temp}}/reports_dir +# - name: Split build check +# env: +# TEMP_PATH: ${{runner.temp}}/split_build_check +# REPO_COPY: ${{runner.temp}}/split_build_check/ClickHouse +# REPORTS_PATH: ${{runner.temp}}/reports_dir +# run: | +# sudo rm -fr $TEMP_PATH +# mkdir -p $TEMP_PATH +# cp -r $GITHUB_WORKSPACE $TEMP_PATH +# cd $REPO_COPY/tests/ci && python3 split_build_smoke_check.py +# - name: Cleanup +# if: always() +# run: | +# docker kill $(docker ps -q) ||: +# docker rm -f $(docker ps -a -q) ||: +# sudo rm -fr $TEMP_PATH ########################################################################################## ##################################### ORDINARY BUILDS #################################### ########################################################################################## From 1e79177ce250f2275e02ca0bd98002a1db2ac318 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Tue, 16 Nov 2021 12:31:51 +0300 Subject: [PATCH 093/200] Build fix. --- src/IO/S3Common.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index bca6b7e95f9..314576e6f0a 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -459,7 +459,7 @@ public: { return Aws::Auth::AWSCredentials(); } - RefreshIfExpired(); + refreshIfExpired(); Aws::Utils::Threading::ReaderLockGuard guard(m_reloadLock); return credentials; } @@ -488,7 +488,7 @@ protected: } private: - void RefreshIfExpired() + void refreshIfExpired() { Aws::Utils::Threading::ReaderLockGuard guard(m_reloadLock); if (!credentials.IsExpiredOrEmpty()) From 538787b7494d0a074d8769ce3f580e04237b7478 Mon Sep 17 00:00:00 2001 From: jus1096 <40656180+jus1096@users.noreply.github.com> Date: Tue, 16 Nov 2021 13:49:24 +0300 Subject: [PATCH 094/200] Fix link to external MySQL dict Corrected link to external dictionary MySQL --- docs/ru/sql-reference/table-functions/mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/table-functions/mysql.md b/docs/ru/sql-reference/table-functions/mysql.md index e881961e3d9..7a4cc9c234c 100644 --- a/docs/ru/sql-reference/table-functions/mysql.md +++ b/docs/ru/sql-reference/table-functions/mysql.md @@ -107,5 +107,5 @@ SELECT * FROM mysql('localhost:3306', 'test', 'test', 'bayonet', '123'); **Смотрите также** - [Движок таблиц ‘MySQL’](../../sql-reference/table-functions/mysql.md) -- [Использование MySQL как источника данных для внешнего словаря](../../sql-reference/table-functions/mysql.md#dicts-external_dicts_dict_sources-mysql) +- [Использование MySQL как источника данных для внешнего словаря](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md##dicts-external_dicts_dict_sources-mysql) From 73d191841003188f7f43d9cf32bc170df5155770 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 9 Nov 2021 16:14:07 +0300 Subject: [PATCH 095/200] tmp --- src/Core/Settings.h | 4 +- src/Core/SettingsEnums.cpp | 9 + src/Core/SettingsEnums.h | 2 + src/Formats/EscapingRuleUtils.cpp | 224 +++++++++++++++++ src/Formats/EscapingRuleUtils.h | 37 +++ src/Formats/FormatSettings.h | 15 +- src/Formats/ParsedTemplateFormatString.cpp | 85 +------ src/Formats/ParsedTemplateFormatString.h | 18 +- src/Formats/registerFormats.cpp | 4 + src/Formats/verbosePrintString.h | 1 + src/IO/ReadHelpers.cpp | 46 ++++ src/IO/ReadHelpers.h | 4 + .../Impl/CustomSeparatedRowInputFormat.cpp | 233 ++++++++++++++++++ .../Impl/CustomSeparatedRowInputFormat.h | 64 +++++ .../Impl/CustomSeparatedRowOutputFormat.cpp | 98 ++++++++ .../Impl/CustomSeparatedRowOutputFormat.h | 36 +++ .../Formats/Impl/RegexpRowInputFormat.cpp | 58 +---- .../Formats/Impl/RegexpRowInputFormat.h | 5 +- .../Impl/TabSeparatedRowOutputFormat.cpp | 2 +- .../Impl/TabSeparatedRowOutputFormat.h | 2 +- .../Impl/TemplateBlockOutputFormat.cpp | 64 +---- .../Formats/Impl/TemplateBlockOutputFormat.h | 5 +- .../Formats/Impl/TemplateRowInputFormat.cpp | 217 +++------------- .../Formats/Impl/TemplateRowInputFormat.h | 10 +- .../RowInputFormatWithNamesAndTypes.cpp | 33 ++- .../Formats/RowInputFormatWithNamesAndTypes.h | 7 + 26 files changed, 883 insertions(+), 400 deletions(-) create mode 100644 src/Formats/EscapingRuleUtils.cpp create mode 100644 src/Formats/EscapingRuleUtils.h create mode 100644 src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp create mode 100644 src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h create mode 100644 src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp create mode 100644 src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4988de44f9d..531a0f905af 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -617,7 +617,7 @@ class IColumn; M(String, format_template_row, "", "Path to file which contains format string for rows (for Template format)", 0) \ M(String, format_template_rows_between_delimiter, "\n", "Delimiter between rows (for Template format)", 0) \ \ - M(String, format_custom_escaping_rule, "Escaped", "Field escaping rule (for CustomSeparated format)", 0) \ + M(EscapingRule, format_custom_escaping_rule, "Escaped", "Field escaping rule (for CustomSeparated format)", 0) \ M(String, format_custom_field_delimiter, "\t", "Delimiter between fields (for CustomSeparated format)", 0) \ M(String, format_custom_row_before_delimiter, "", "Delimiter before field of the first column (for CustomSeparated format)", 0) \ M(String, format_custom_row_after_delimiter, "\n", "Delimiter after field of the last column (for CustomSeparated format)", 0) \ @@ -626,7 +626,7 @@ class IColumn; M(String, format_custom_result_after_delimiter, "", "Suffix after result set (for CustomSeparated format)", 0) \ \ M(String, format_regexp, "", "Regular expression (for Regexp format)", 0) \ - M(String, format_regexp_escaping_rule, "Raw", "Field escaping rule (for Regexp format)", 0) \ + M(EscapingRule, format_regexp_escaping_rule, "Raw", "Field escaping rule (for Regexp format)", 0) \ M(Bool, format_regexp_skip_unmatched, false, "Skip lines unmatched by regular expression (for Regexp format", 0) \ \ M(Bool, output_format_enable_streaming, false, "Enable streaming in output formats that support it.", 0) \ diff --git a/src/Core/SettingsEnums.cpp b/src/Core/SettingsEnums.cpp index f5497588891..b62575c9730 100644 --- a/src/Core/SettingsEnums.cpp +++ b/src/Core/SettingsEnums.cpp @@ -121,4 +121,13 @@ IMPLEMENT_SETTING_ENUM(EnumComparingMode, ErrorCodes::BAD_ARGUMENTS, {{"by_names", FormatSettings::EnumComparingMode::BY_NAMES}, {"by_values", FormatSettings::EnumComparingMode::BY_VALUES}, {"by_names_case_insensitive", FormatSettings::EnumComparingMode::BY_NAMES_CASE_INSENSITIVE}}) + +IMPLEMENT_SETTING_ENUM(EscapingRule, ErrorCodes::BAD_ARGUMENTS, + {{"None", FormatSettings::EscapingRule::None}, + {"Escaped", FormatSettings::EscapingRule::Escaped}, + {"Quoted", FormatSettings::EscapingRule::Quoted}, + {"CSV", FormatSettings::EscapingRule::CSV}, + {"JSON", FormatSettings::EscapingRule::JSON}, + {"XML", FormatSettings::EscapingRule::XML}, + {"Raw", FormatSettings::EscapingRule::Raw}}) } diff --git a/src/Core/SettingsEnums.h b/src/Core/SettingsEnums.h index 4bdb3c83ea5..106589f5d24 100644 --- a/src/Core/SettingsEnums.h +++ b/src/Core/SettingsEnums.h @@ -170,4 +170,6 @@ DECLARE_SETTING_ENUM(ShortCircuitFunctionEvaluation) DECLARE_SETTING_ENUM_WITH_RENAME(EnumComparingMode, FormatSettings::EnumComparingMode) +DECLARE_SETTING_ENUM_WITH_RENAME(EscapingRule, FormatSettings::EscapingRule) + } diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp new file mode 100644 index 00000000000..c98d64e755c --- /dev/null +++ b/src/Formats/EscapingRuleUtils.cpp @@ -0,0 +1,224 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int CANNOT_READ_ALL_DATA; + extern const int ATTEMPT_TO_READ_AFTER_EOF; +} + +FormatSettings::EscapingRule stringToEscapingRule(const String & escaping_rule) +{ + if (escaping_rule.empty()) + return FormatSettings::EscapingRule::None; + else if (escaping_rule == "None") + return FormatSettings::EscapingRule::None; + else if (escaping_rule == "Escaped") + return FormatSettings::EscapingRule::Escaped; + else if (escaping_rule == "Quoted") + return FormatSettings::EscapingRule::Quoted; + else if (escaping_rule == "CSV") + return FormatSettings::EscapingRule::CSV; + else if (escaping_rule == "JSON") + return FormatSettings::EscapingRule::JSON; + else if (escaping_rule == "XML") + return FormatSettings::EscapingRule::XML; + else if (escaping_rule == "Raw") + return FormatSettings::EscapingRule::Raw; + else + throw Exception("Unknown escaping rule \"" + escaping_rule + "\"", ErrorCodes::BAD_ARGUMENTS); +} + +String escapingRuleToString(FormatSettings::EscapingRule escaping_rule) +{ + switch (escaping_rule) + { + case FormatSettings::EscapingRule::None: + return "None"; + case FormatSettings::EscapingRule::Escaped: + return "Escaped"; + case FormatSettings::EscapingRule::Quoted: + return "Quoted"; + case FormatSettings::EscapingRule::CSV: + return "CSV"; + case FormatSettings::EscapingRule::JSON: + return "JSON"; + case FormatSettings::EscapingRule::XML: + return "XML"; + case FormatSettings::EscapingRule::Raw: + return "Raw"; + } + __builtin_unreachable(); +} + +void skipFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings) +{ + String tmp; + constexpr const char * field_name = ""; + constexpr size_t field_name_len = 16; + switch (escaping_rule) + { + case FormatSettings::EscapingRule::None: + /// Empty field, just skip spaces + break; + case FormatSettings::EscapingRule::Escaped: + readEscapedString(tmp, buf); + break; + case FormatSettings::EscapingRule::Quoted: + readQuotedString(tmp, buf); + break; + case FormatSettings::EscapingRule::CSV: + readCSVString(tmp, buf, format_settings.csv); + break; + case FormatSettings::EscapingRule::JSON: + skipJSONField(buf, StringRef(field_name, field_name_len)); + break; + case FormatSettings::EscapingRule::Raw: + readString(tmp, buf); + break; + default: + __builtin_unreachable(); + } +} + +bool deserializeFieldByEscapingRule( + const DataTypePtr & type, + const SerializationPtr & serialization, + IColumn & column, + ReadBuffer & buf, + FormatSettings::EscapingRule escaping_rule, + const FormatSettings & format_settings) +{ + bool read = true; + bool parse_as_nullable = format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable(); + switch (escaping_rule) + { + case FormatSettings::EscapingRule::Escaped: + if (parse_as_nullable) + read = SerializationNullable::deserializeTextEscapedImpl(column, buf, format_settings, serialization); + else + serialization->deserializeTextEscaped(column, buf, format_settings); + break; + case FormatSettings::EscapingRule::Quoted: + if (parse_as_nullable) + read = SerializationNullable::deserializeTextQuotedImpl(column, buf, format_settings, serialization); + else + serialization->deserializeTextQuoted(column, buf, format_settings); + break; + case FormatSettings::EscapingRule::CSV: + if (parse_as_nullable) + read = SerializationNullable::deserializeTextCSVImpl(column, buf, format_settings, serialization); + else + serialization->deserializeTextCSV(column, buf, format_settings); + break; + case FormatSettings::EscapingRule::JSON: + if (parse_as_nullable) + read = SerializationNullable::deserializeTextJSONImpl(column, buf, format_settings, serialization); + else + serialization->deserializeTextJSON(column, buf, format_settings); + break; + case FormatSettings::EscapingRule::Raw: + if (parse_as_nullable) + read = SerializationNullable::deserializeTextRawImpl(column, buf, format_settings, serialization); + else + serialization->deserializeTextRaw(column, buf, format_settings); + break; + default: + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Escaping rule {} is not suitable for deserialization", escapingRuleToString(escaping_rule)); + } + return read; +} + +void serializeFieldByEscapingRule( + const IColumn & column, + const ISerialization & serialization, + WriteBuffer & out, + size_t row_num, + FormatSettings::EscapingRule escaping_rule, + const FormatSettings & format_settings) +{ + switch (escaping_rule) + { + case FormatSettings::EscapingRule::Escaped: + serialization.serializeTextEscaped(column, row_num, out, format_settings); + break; + case FormatSettings::EscapingRule::Quoted: + serialization.serializeTextQuoted(column, row_num, out, format_settings); + break; + case FormatSettings::EscapingRule::CSV: + serialization.serializeTextCSV(column, row_num, out, format_settings); + break; + case FormatSettings::EscapingRule::JSON: + serialization.serializeTextJSON(column, row_num, out, format_settings); + break; + case FormatSettings::EscapingRule::XML: + serialization.serializeTextXML(column, row_num, out, format_settings); + break; + case FormatSettings::EscapingRule::Raw: + serialization.serializeTextRaw(column, row_num, out, format_settings); + break; + case FormatSettings::EscapingRule::None: + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot serialize field with None escaping rule"); + } +} + +void writeStringByEscapingRule(const String & value, WriteBuffer & out, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings) +{ + switch (escaping_rule) + { + case FormatSettings::EscapingRule::Quoted: + writeQuotedString(value, out); + break; + case FormatSettings::EscapingRule::JSON: + writeJSONString(value, out, format_settings); + break; + case FormatSettings::EscapingRule::Raw: + writeString(value, out); + break; + case FormatSettings::EscapingRule::CSV: + writeCSVString(value, out); + break; + case FormatSettings::EscapingRule::Escaped: + writeEscapedString(value, out); + break; + case FormatSettings::EscapingRule::XML: + writeXMLStringForTextElement(value, out); + break; + case FormatSettings::EscapingRule::None: + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot serialize string with None escaping rule"); + } +} + +String readStringByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings) +{ + String result; + switch (escaping_rule) + { + case FormatSettings::EscapingRule::Quoted: + readQuotedString(result, buf); + break; + case FormatSettings::EscapingRule::JSON: + readJSONString(result, buf); + break; + case FormatSettings::EscapingRule::Raw: + readString(result, buf); + break; + case FormatSettings::EscapingRule::CSV: + readCSVString(result, buf, format_settings.csv); + break; + case FormatSettings::EscapingRule::Escaped: + readEscapedString(result, buf); + break; + default: + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read string with {} escaping rule", escapingRuleToString(escaping_rule)); + } + return result; +} + +} diff --git a/src/Formats/EscapingRuleUtils.h b/src/Formats/EscapingRuleUtils.h new file mode 100644 index 00000000000..02f027db74d --- /dev/null +++ b/src/Formats/EscapingRuleUtils.h @@ -0,0 +1,37 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +FormatSettings::EscapingRule stringToEscapingRule(const String & escaping_rule); + +String escapingRuleToString(FormatSettings::EscapingRule escaping_rule); + +void skipFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings); + +bool deserializeFieldByEscapingRule( + const DataTypePtr & type, + const SerializationPtr & serialization, + IColumn & column, + ReadBuffer & buf, + FormatSettings::EscapingRule escaping_rule, + const FormatSettings & format_settings); + +void serializeFieldByEscapingRule( + const IColumn & column, + const ISerialization & serialization, + WriteBuffer & out, + size_t row_num, + FormatSettings::EscapingRule escaping_rule, + const FormatSettings & format_settings); + +void writeStringByEscapingRule(const String & value, WriteBuffer & out, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings); + +String readStringByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings); + +} diff --git a/src/Formats/FormatSettings.h b/src/Formats/FormatSettings.h index b4f1550f0bd..e5799668ff3 100644 --- a/src/Formats/FormatSettings.h +++ b/src/Formats/FormatSettings.h @@ -47,6 +47,17 @@ struct FormatSettings UnixTimestamp }; + enum class EscapingRule + { + None, + Escaped, + Quoted, + CSV, + JSON, + XML, + Raw + }; + DateTimeOutputFormat date_time_output_format = DateTimeOutputFormat::Simple; UInt64 input_allow_errors_num = 0; @@ -89,7 +100,7 @@ struct FormatSettings std::string row_after_delimiter; std::string row_between_delimiter; std::string field_delimiter; - std::string escaping_rule; + EscapingRule escaping_rule = EscapingRule::Escaped; } custom; struct @@ -148,7 +159,7 @@ struct FormatSettings struct { std::string regexp; - std::string escaping_rule; + EscapingRule escaping_rule = EscapingRule::Raw; bool skip_unmatched = false; } regexp; diff --git a/src/Formats/ParsedTemplateFormatString.cpp b/src/Formats/ParsedTemplateFormatString.cpp index a01a8adcf26..013e4176fed 100644 --- a/src/Formats/ParsedTemplateFormatString.cpp +++ b/src/Formats/ParsedTemplateFormatString.cpp @@ -1,9 +1,9 @@ #include #include +#include #include #include #include -#include #include namespace DB @@ -83,7 +83,7 @@ void ParsedTemplateFormatString::parse(const String & format_string, const Colum state = Format; else if (*pos == '}') { - formats.push_back(ColumnFormat::None); + escaping_rules.push_back(EscapingRule::None); delimiters.emplace_back(); state = Delimiter; } @@ -108,7 +108,7 @@ void ParsedTemplateFormatString::parse(const String & format_string, const Colum case Format: if (*pos == '}') { - formats.push_back(stringToFormat(String(token_begin, pos - token_begin))); + escaping_rules.push_back(stringToEscapingRule(String(token_begin, pos - token_begin))); token_begin = pos + 1; delimiters.emplace_back(); state = Delimiter; @@ -120,56 +120,11 @@ void ParsedTemplateFormatString::parse(const String & format_string, const Colum delimiters.back().append(token_begin, pos - token_begin); } - -ParsedTemplateFormatString::ColumnFormat ParsedTemplateFormatString::stringToFormat(const String & col_format) -{ - if (col_format.empty()) - return ColumnFormat::None; - else if (col_format == "None") - return ColumnFormat::None; - else if (col_format == "Escaped") - return ColumnFormat::Escaped; - else if (col_format == "Quoted") - return ColumnFormat::Quoted; - else if (col_format == "CSV") - return ColumnFormat::Csv; - else if (col_format == "JSON") - return ColumnFormat::Json; - else if (col_format == "XML") - return ColumnFormat::Xml; - else if (col_format == "Raw") - return ColumnFormat::Raw; - else - throw Exception("Unknown field format \"" + col_format + "\"", ErrorCodes::BAD_ARGUMENTS); -} - size_t ParsedTemplateFormatString::columnsCount() const { return format_idx_to_column_idx.size(); } -String ParsedTemplateFormatString::formatToString(ParsedTemplateFormatString::ColumnFormat format) -{ - switch (format) - { - case ColumnFormat::None: - return "None"; - case ColumnFormat::Escaped: - return "Escaped"; - case ColumnFormat::Quoted: - return "Quoted"; - case ColumnFormat::Csv: - return "CSV"; - case ColumnFormat::Json: - return "Json"; - case ColumnFormat::Xml: - return "Xml"; - case ColumnFormat::Raw: - return "Raw"; - } - __builtin_unreachable(); -} - const char * ParsedTemplateFormatString::readMayBeQuotedColumnNameInto(const char * pos, size_t size, String & s) { s.clear(); @@ -197,7 +152,7 @@ String ParsedTemplateFormatString::dump() const res << "\nDelimiter " << 0 << ": "; verbosePrintString(delimiters.front().c_str(), delimiters.front().c_str() + delimiters.front().size(), res); - size_t num_columns = std::max(formats.size(), format_idx_to_column_idx.size()); + size_t num_columns = std::max(escaping_rules.size(), format_idx_to_column_idx.size()); for (size_t i = 0; i < num_columns; ++i) { res << "\nColumn " << i << ": \""; @@ -216,7 +171,7 @@ String ParsedTemplateFormatString::dump() const else res << *format_idx_to_column_idx[i]; - res << "), Format " << (i < formats.size() ? formatToString(formats[i]) : ""); + res << "), Format " << (i < escaping_rules.size() ? escapingRuleToString(escaping_rules[i]) : ""); res << "\nDelimiter " << i + 1 << ": "; if (delimiters.size() <= i + 1) @@ -235,34 +190,4 @@ void ParsedTemplateFormatString::throwInvalidFormat(const String & message, size ErrorCodes::INVALID_TEMPLATE_FORMAT); } -ParsedTemplateFormatString ParsedTemplateFormatString::setupCustomSeparatedResultsetFormat(const FormatSettings::Custom & settings) -{ - /// Set resultset format to "result_before_delimiter ${data} result_after_delimiter" - ParsedTemplateFormatString resultset_format; - resultset_format.delimiters.emplace_back(settings.result_before_delimiter); - resultset_format.delimiters.emplace_back(settings.result_after_delimiter); - resultset_format.formats.emplace_back(ParsedTemplateFormatString::ColumnFormat::None); - resultset_format.format_idx_to_column_idx.emplace_back(0); - resultset_format.column_names.emplace_back("data"); - return resultset_format; -} - -ParsedTemplateFormatString ParsedTemplateFormatString::setupCustomSeparatedRowFormat(const FormatSettings::Custom & settings, const Block & sample) -{ - /// Set row format to - /// "row_before_delimiter ${Col0:escaping} field_delimiter ${Col1:escaping} field_delimiter ... ${ColN:escaping} row_after_delimiter" - ParsedTemplateFormatString::ColumnFormat escaping = ParsedTemplateFormatString::stringToFormat(settings.escaping_rule); - ParsedTemplateFormatString row_format; - row_format.delimiters.emplace_back(settings.row_before_delimiter); - for (size_t i = 0; i < sample.columns(); ++i) - { - row_format.formats.emplace_back(escaping); - row_format.format_idx_to_column_idx.emplace_back(i); - row_format.column_names.emplace_back(sample.getByPosition(i).name); - bool last_column = i == sample.columns() - 1; - row_format.delimiters.emplace_back(last_column ? settings.row_after_delimiter : settings.field_delimiter); - } - return row_format; -} - } diff --git a/src/Formats/ParsedTemplateFormatString.h b/src/Formats/ParsedTemplateFormatString.h index 4021b71656f..ba0ebdf5aa8 100644 --- a/src/Formats/ParsedTemplateFormatString.h +++ b/src/Formats/ParsedTemplateFormatString.h @@ -15,23 +15,14 @@ using Strings = std::vector; struct ParsedTemplateFormatString { - enum class ColumnFormat - { - None, - Escaped, - Quoted, - Csv, - Json, - Xml, - Raw - }; + using EscapingRule = FormatSettings::EscapingRule; /// Format string has syntax: "Delimiter0 ${ColumnName0:Format0} Delimiter1 ${ColumnName1:Format1} Delimiter2" /// The following vectors is filled with corresponding values, delimiters.size() - 1 = formats.size() = format_idx_to_column_idx.size() /// If format_idx_to_column_idx[i] has no value, then TemplateRowInputFormat will skip i-th column. std::vector delimiters; - std::vector formats; + std::vector escaping_rules; std::vector> format_idx_to_column_idx; /// For diagnostic info @@ -44,16 +35,11 @@ struct ParsedTemplateFormatString void parse(const String & format_string, const ColumnIdxGetter & idx_by_name); - static ColumnFormat stringToFormat(const String & format); - static String formatToString(ColumnFormat format); static const char * readMayBeQuotedColumnNameInto(const char * pos, size_t size, String & s); size_t columnsCount() const; String dump() const; [[noreturn]] void throwInvalidFormat(const String & message, size_t column) const; - - static ParsedTemplateFormatString setupCustomSeparatedResultsetFormat(const FormatSettings::Custom & settings); - static ParsedTemplateFormatString setupCustomSeparatedRowFormat(const FormatSettings::Custom & settings, const Block & sample); }; } diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index 1aedff5fceb..7425c6898de 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -50,6 +50,8 @@ void registerInputFormatAvro(FormatFactory & factory); void registerOutputFormatAvro(FormatFactory & factory); void registerInputFormatRawBLOB(FormatFactory & factory); void registerOutputFormatRawBLOB(FormatFactory & factory); +void registerInputFormatCustomSeparated(FormatFactory & factory); +void registerOutputFormatCustomSeparated(FormatFactory & factory); /// Output only (presentational) formats. @@ -115,6 +117,8 @@ void registerFormats() registerOutputFormatMsgPack(factory); registerInputFormatRawBLOB(factory); registerOutputFormatRawBLOB(factory); + registerInputFormatCustomSeparated(factory); + registerOutputFormatCustomSeparated(factory); registerInputFormatORC(factory); registerOutputFormatORC(factory); diff --git a/src/Formats/verbosePrintString.h b/src/Formats/verbosePrintString.h index 26bd663d559..02034cb8e97 100644 --- a/src/Formats/verbosePrintString.h +++ b/src/Formats/verbosePrintString.h @@ -5,6 +5,7 @@ namespace DB { class WriteBuffer; +class ReadBuffer; /** Print string in double quotes and with control characters in "" form - for output diagnostic info to user. diff --git a/src/IO/ReadHelpers.cpp b/src/IO/ReadHelpers.cpp index 41dcd9fde6c..675adc43ce6 100644 --- a/src/IO/ReadHelpers.cpp +++ b/src/IO/ReadHelpers.cpp @@ -1166,4 +1166,50 @@ bool loadAtPosition(ReadBuffer & in, Memory<> & memory, char * & current) return loaded_more; } +/// Searches for delimiter in input stream and sets buffer position after delimiter (if found) or EOF (if not) +static void findAndSkipNextDelimiter(PeekableReadBuffer & buf, const String & delimiter) +{ + if (delimiter.empty()) + return; + + while (!buf.eof()) + { + void * pos = memchr(buf.position(), delimiter[0], buf.available()); + if (!pos) + { + buf.position() += buf.available(); + continue; + } + + buf.position() = static_cast(pos); + + PeekableReadBufferCheckpoint checkpoint{buf}; + if (checkString(delimiter, buf)) + return; + + buf.rollbackToCheckpoint(); + ++buf.position(); + } +} + +void skipToNextRowOrEof(PeekableReadBuffer & buf, const String & row_after_delimiter, const String & row_between_delimiter, bool skip_spaces) +{ + if (row_after_delimiter.empty()) + { + findAndSkipNextDelimiter(buf, row_between_delimiter); + return; + } + + while (true) + { + findAndSkipNextDelimiter(buf, row_after_delimiter); + + if (skip_spaces) + skipWhitespaceIfAny(buf); + + if (checkString(row_between_delimiter, buf)) + break; + } +} + } diff --git a/src/IO/ReadHelpers.h b/src/IO/ReadHelpers.h index 81973bcd8a4..aa9f0fe6c07 100644 --- a/src/IO/ReadHelpers.h +++ b/src/IO/ReadHelpers.h @@ -30,6 +30,7 @@ #include #include #include +#include #include #include @@ -1324,6 +1325,9 @@ void saveUpToPosition(ReadBuffer & in, Memory> & memory, char * */ bool loadAtPosition(ReadBuffer & in, Memory> & memory, char * & current); +/// Skip data until start of the next row or eof (the end of row is determined by two delimiters: +/// row_after_delimiter and row_between_delimiter). +void skipToNextRowOrEof(PeekableReadBuffer & buf, const String & row_after_delimiter, const String & row_between_delimiter, bool skip_spaces); struct PcgDeserializer { diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp new file mode 100644 index 00000000000..8e2c5165531 --- /dev/null +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp @@ -0,0 +1,233 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat( + const Block & header_, + ReadBuffer & in_, + const Params & params_, + bool with_names_, + bool with_types_, + bool ignore_spaces_, + const FormatSettings & format_settings_) + : RowInputFormatWithNamesAndTypes(header_, in_, params_, with_names_, with_types_, format_settings_) + , buf(in_) + , ignore_spaces(ignore_spaces_) + , escaping_rule(format_settings_.custom.escaping_rule) +{ + /// In case of CustomSeparatedWithNames(AndTypes) formats and enabled setting input_format_with_names_use_header we don't know + /// the exact number of columns in data (because it can contain unknown columns). So, if field_delimiter and row_after_delimiter are + /// the same and row_between_delimiter is empty, we won't be able to determine the end of row while reading column names or types. + if ((with_types_ || with_names_) && format_settings_.with_names_use_header + && format_settings_.custom.field_delimiter == format_settings_.custom.row_after_delimiter + && format_settings_.custom.row_between_delimiter.empty()) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Input format CustomSeparatedWithNames(AndTypes) cannot work properly with enabled setting input_format_with_names_use_header, " + "when format_custom_field_delimiter and format_custom_row_after_delimiter are the same and format_custom_row_between_delimiter is empty."); + } +} + +void CustomSeparatedRowInputFormat::skipPrefixBeforeHeader() +{ + skipSpaces(); + assertString(format_settings.custom.result_before_delimiter, buf); +} + +void CustomSeparatedRowInputFormat::skipRowStartDelimiter() +{ + skipSpaces(); + assertString(format_settings.custom.row_before_delimiter, buf); +} + +void CustomSeparatedRowInputFormat::skipFieldDelimiter() +{ + skipSpaces(); + assertString(format_settings.custom.field_delimiter, buf); +} + +void CustomSeparatedRowInputFormat::skipRowEndDelimiter() +{ + skipSpaces(); + assertString(format_settings.custom.row_after_delimiter, buf); +} + +void CustomSeparatedRowInputFormat::skipRowBetweenDelimiter() +{ + skipSpaces(); + assertString(format_settings.custom.row_between_delimiter, buf); +} + +void CustomSeparatedRowInputFormat::skipField() +{ + skipSpaces(); + skipFieldByEscapingRule(buf, escaping_rule, format_settings); +} + +bool CustomSeparatedRowInputFormat::checkEndOfRow() +{ + PeekableReadBufferCheckpoint checkpoint{buf, true}; + + skipSpaces(); + if (!checkString(format_settings.custom.row_after_delimiter, buf)) + return false; + + skipSpaces(); + + /// At the end of row after row_after_delimiter we expect result_after_delimiter or row_between_delimiter. + + if (checkString(format_settings.custom.row_between_delimiter, buf)) + return true; + + buf.rollbackToCheckpoint(); + skipSpaces(); + buf.ignore(format_settings.custom.row_after_delimiter.size()); + return checkForSuffixImpl(true); +} + +std::vector CustomSeparatedRowInputFormat::readHeaderRow() +{ + std::vector values; + skipRowStartDelimiter(); + do + { + if (!values.empty()) + skipFieldDelimiter(); + skipSpaces(); + values.push_back(readStringByEscapingRule(buf, escaping_rule, format_settings)); + } + while (!checkEndOfRow()); + + skipRowEndDelimiter(); + return values; +} + +void CustomSeparatedRowInputFormat::skipHeaderRow() +{ + size_t columns = getPort().getHeader().columns(); + skipRowStartDelimiter(); + for (size_t i = 0; i != columns; ++i) + { + skipField(); + if (i + 1 != columns) + skipFieldDelimiter(); + } + skipRowEndDelimiter(); +} + +bool CustomSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool, const String &) +{ + skipSpaces(); + return deserializeFieldByEscapingRule(type, serialization, column, buf, escaping_rule, format_settings); +} + +bool CustomSeparatedRowInputFormat::checkForSuffixImpl(bool check_eof) +{ + skipSpaces(); + if (unlikely(checkString(format_settings.custom.result_after_delimiter, buf))) + { + skipSpaces(); + if (!check_eof) + return true; + + if (buf.eof()) + return true; + } + return false; +} + +bool CustomSeparatedRowInputFormat::tryParseSuffixWithDiagnosticInfo(WriteBuffer & out) +{ + PeekableReadBufferCheckpoint checkpoint{buf}; + if (checkForSuffixImpl(false)) + { + if (in->eof()) + out << "\n"; + else + out << " There is some data after suffix\n"; + return false; + } + buf.rollbackToCheckpoint(); + return true; +} + +bool CustomSeparatedRowInputFormat::checkForSuffix() +{ + PeekableReadBufferCheckpoint checkpoint{buf}; + if (checkForSuffixImpl(true)) + return true; + buf.rollbackToCheckpoint(); + return false; +} + + +bool CustomSeparatedRowInputFormat::allowSyncAfterError() const +{ + return !format_settings.custom.row_after_delimiter.empty() || !format_settings.custom.row_between_delimiter.empty(); +} + +void CustomSeparatedRowInputFormat::syncAfterError() +{ + skipToNextRowOrEof(buf, format_settings.custom.row_after_delimiter, format_settings.custom.row_between_delimiter, ignore_spaces); + end_of_stream = buf.eof(); + /// It can happen that buf.position() is not at the beginning of row + /// if some delimiters is similar to row_format.delimiters.back() and row_between_delimiter. + /// It will cause another parsing error. +} + +bool CustomSeparatedRowInputFormat::parseRowStartWithDiagnosticInfo(WriteBuffer & out) +{ + return parseDelimiterWithDiagnosticInfo(out, buf, format_settings.custom.row_before_delimiter, "delimiter before first firld", ignore_spaces); +} + +bool CustomSeparatedRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) +{ + return parseDelimiterWithDiagnosticInfo(out, buf, format_settings.custom.field_delimiter, "delimiter between fields", ignore_spaces); +} + +bool CustomSeparatedRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out) +{ + return parseDelimiterWithDiagnosticInfo(out, buf, format_settings.custom.row_after_delimiter, "delimiter after last field", ignore_spaces); +} + +bool CustomSeparatedRowInputFormat::parseRowBetweenDelimiterWithDiagnosticInfo(WriteBuffer & out) +{ + return parseDelimiterWithDiagnosticInfo(out, buf, format_settings.custom.row_between_delimiter, "delimiter between rows", ignore_spaces); +} + +void CustomSeparatedRowInputFormat::resetParser() +{ + RowInputFormatWithNamesAndTypes::resetParser(); + buf.reset(); +} + +void registerInputFormatCustomSeparated(FormatFactory & factory) +{ + for (bool ignore_spaces : {false, true}) + { + auto register_func = [&](const String & format_name, bool with_names, bool with_types) + { + factory.registerInputFormat(format_name, [=]( + ReadBuffer & buf, + const Block & sample, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(sample, buf, params, with_names, with_types, ignore_spaces, settings); + }); + }; + registerWithNamesAndTypes(ignore_spaces ? "CustomSeparatedIgnoreSpaces" : "CustomSeparated", register_func); + } +} + +} diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h new file mode 100644 index 00000000000..9dcda0810ac --- /dev/null +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h @@ -0,0 +1,64 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +class CustomSeparatedRowInputFormat : public RowInputFormatWithNamesAndTypes +{ +public: + CustomSeparatedRowInputFormat( + const Block & header_, + ReadBuffer & in_, + const Params & params_, + bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_settings_); + + void resetParser() override; + String getName() const override { return "CustomSeparatedRowInputFormat"; } + +private: + using EscapingRule = FormatSettings::EscapingRule; + + bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; + + void skipField(size_t /*file_column*/) override { skipField(); } + void skipField(); + void skipNames() override { skipHeaderRow(); } + void skipTypes() override { skipHeaderRow(); } + void skipHeaderRow(); + + void skipPrefixBeforeHeader() override; + void skipRowStartDelimiter() override; + void skipFieldDelimiter() override; + void skipRowEndDelimiter() override; + void skipRowBetweenDelimiter() override; + + bool checkForSuffix() override; + + bool allowSyncAfterError() const override; + void syncAfterError() override; + + bool parseRowStartWithDiagnosticInfo(WriteBuffer & out) override; + bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override; + bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override; + bool parseRowBetweenDelimiterWithDiagnosticInfo(WriteBuffer & out) override; + bool tryParseSuffixWithDiagnosticInfo(WriteBuffer & out) override; + + std::vector readNames() override { return readHeaderRow(); } + std::vector readTypes() override { return readHeaderRow(); } + std::vector readHeaderRow(); + + bool checkEndOfRow(); + bool checkForSuffixImpl(bool check_eof); + inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(buf); } + + PeekableReadBuffer buf; + bool ignore_spaces; + EscapingRule escaping_rule; +}; + +} diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp new file mode 100644 index 00000000000..a1ce4fc50d5 --- /dev/null +++ b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp @@ -0,0 +1,98 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +CustomSeparatedRowOutputFormat::CustomSeparatedRowOutputFormat( + const Block & header_, WriteBuffer & out_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_, bool with_names_, bool with_types_) + : IRowOutputFormat(header_, out_, params_) + , with_names(with_names_) + , with_types(with_types_) + , format_settings(format_settings_) + , escaping_rule(format_settings.custom.escaping_rule) +{ +} + +void CustomSeparatedRowOutputFormat::writeLine(const std::vector & values) +{ + for (const auto & value : values) + writeStringByEscapingRule(value, out, escaping_rule, format_settings); +} + +void CustomSeparatedRowOutputFormat::writePrefix() +{ + writeString(format_settings.custom.result_before_delimiter, out); + + const auto & header = getPort(PortKind::Main).getHeader(); + if (with_names) + { + writeLine(header.getNames()); + writeRowBetweenDelimiter(); + } + + if (with_types) + { + writeLine(header.getDataTypeNames()); + writeRowBetweenDelimiter(); + } +} + +void CustomSeparatedRowOutputFormat::writeSuffix() +{ + writeString(format_settings.custom.result_after_delimiter, out); +} + +void CustomSeparatedRowOutputFormat::writeRowStartDelimiter() +{ + writeString(format_settings.custom.row_before_delimiter, out); +} + +void CustomSeparatedRowOutputFormat::writeFieldDelimiter() +{ + writeString(format_settings.custom.field_delimiter, out); +} + +void CustomSeparatedRowOutputFormat::writeRowEndDelimiter() +{ + writeString(format_settings.custom.row_after_delimiter, out); +} + +void CustomSeparatedRowOutputFormat::writeRowBetweenDelimiter() +{ + writeString(format_settings.custom.row_between_delimiter, out); +} + +void CustomSeparatedRowOutputFormat::writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) +{ + serializeFieldByEscapingRule(column, serialization, out, row_num, escaping_rule, format_settings); +} + +void registerOutputFormatCustomSeparated(FormatFactory & factory) +{ + auto register_func = [&](const String & format_name, bool with_names, bool with_types) + { + factory.registerOutputFormat(format_name, [with_names, with_types]( + WriteBuffer & buf, + const Block & sample, + const RowOutputFormatParams & params, + const FormatSettings & settings) + { + return std::make_shared(sample, buf, params, settings, with_names, with_types); + }); + + factory.markOutputFormatSupportsParallelFormatting(format_name); + }; + + registerWithNamesAndTypes("CustomSeparated", register_func); +} + +} diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h new file mode 100644 index 00000000000..274df1af330 --- /dev/null +++ b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h @@ -0,0 +1,36 @@ +#pragma once + +#include +#include + +namespace DB +{ + +class WriteBuffer; + +class CustomSeparatedRowOutputFormat : public IRowOutputFormat +{ +public: + CustomSeparatedRowOutputFormat(const Block & header_, WriteBuffer & out_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_, bool with_names_, bool with_types_); + + String getName() const override { return "CustomSeparatedRowOutputFormat"; } + +private: + using EscapingRule = FormatSettings::EscapingRule; + + void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override; + void writeFieldDelimiter() override; + void writeRowStartDelimiter() override; + void writeRowEndDelimiter() override; + void writeRowBetweenDelimiter() override; + void writePrefix() override; + void writeSuffix() override; + + void writeLine(const std::vector & values); + bool with_names; + bool with_types; + const FormatSettings format_settings; + EscapingRule escaping_rule; +}; + +} diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index 62c0eaa457e..b0b979fab11 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include namespace DB @@ -19,7 +20,7 @@ RegexpRowInputFormat::RegexpRowInputFormat( : IRowInputFormat(header_, in_, std::move(params_)) , buf(in_) , format_settings(format_settings_) - , field_format(stringToFormat(format_settings_.regexp.escaping_rule)) + , escaping_rule(format_settings_.regexp.escaping_rule) , regexp(format_settings_.regexp.regexp) { size_t fields_count = regexp.NumberOfCapturingGroups(); @@ -42,72 +43,19 @@ void RegexpRowInputFormat::resetParser() buf.reset(); } -RegexpRowInputFormat::ColumnFormat RegexpRowInputFormat::stringToFormat(const String & format) -{ - if (format == "Escaped") - return ColumnFormat::Escaped; - if (format == "Quoted") - return ColumnFormat::Quoted; - if (format == "CSV") - return ColumnFormat::Csv; - if (format == "JSON") - return ColumnFormat::Json; - if (format == "Raw") - return ColumnFormat::Raw; - throw Exception("Unsupported column format \"" + format + "\".", ErrorCodes::BAD_ARGUMENTS); -} - bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns) { const auto & type = getPort().getHeader().getByPosition(index).type; - bool parse_as_nullable = format_settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable(); - bool read = true; ReadBuffer field_buf(const_cast(matched_fields[index].data()), matched_fields[index].size(), 0); try { - const auto & serialization = serializations[index]; - switch (field_format) - { - case ColumnFormat::Escaped: - if (parse_as_nullable) - read = SerializationNullable::deserializeTextEscapedImpl(*columns[index], field_buf, format_settings, serialization); - else - serialization->deserializeTextEscaped(*columns[index], field_buf, format_settings); - break; - case ColumnFormat::Quoted: - if (parse_as_nullable) - read = SerializationNullable::deserializeTextQuotedImpl(*columns[index], field_buf, format_settings, serialization); - else - serialization->deserializeTextQuoted(*columns[index], field_buf, format_settings); - break; - case ColumnFormat::Csv: - if (parse_as_nullable) - read = SerializationNullable::deserializeTextCSVImpl(*columns[index], field_buf, format_settings, serialization); - else - serialization->deserializeTextCSV(*columns[index], field_buf, format_settings); - break; - case ColumnFormat::Json: - if (parse_as_nullable) - read = SerializationNullable::deserializeTextJSONImpl(*columns[index], field_buf, format_settings, serialization); - else - serialization->deserializeTextJSON(*columns[index], field_buf, format_settings); - break; - case ColumnFormat::Raw: - if (parse_as_nullable) - read = SerializationNullable::deserializeTextRawImpl(*columns[index], field_buf, format_settings, serialization); - else - serialization->deserializeTextRaw(*columns[index], field_buf, format_settings); - break; - default: - break; - } + return deserializeFieldByEscapingRule(type, serializations[index], *columns[index], field_buf, escaping_rule, format_settings); } catch (Exception & e) { e.addMessage("(while reading the value of column " + getPort().getHeader().getByPosition(index).name + ")"); throw; } - return read; } void RegexpRowInputFormat::readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext) diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.h b/src/Processors/Formats/Impl/RegexpRowInputFormat.h index 0cd8778e499..70726c919a7 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.h +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.h @@ -25,7 +25,7 @@ class ReadBuffer; class RegexpRowInputFormat : public IRowInputFormat { - using ColumnFormat = ParsedTemplateFormatString::ColumnFormat; + using EscapingRule = FormatSettings::EscapingRule; public: RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_); @@ -37,11 +37,10 @@ public: private: bool readField(size_t index, MutableColumns & columns); void readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext); - static ColumnFormat stringToFormat(const String & format); PeekableReadBuffer buf; const FormatSettings format_settings; - const ColumnFormat field_format; + const EscapingRule escaping_rule; const RE2 regexp; // The vector of fields extracted from line using regexp. diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index df0c19ad409..5d87f5a0b14 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -30,7 +30,7 @@ void TabSeparatedRowOutputFormat::writeLine(const std::vector & values) } } -void TabSeparatedRowOutputFormat::doWritePrefix() +void TabSeparatedRowOutputFormat::writePrefix() { const auto & header = getPort(PortKind::Main).getHeader(); diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h index 7dcc6529f1c..c7e12716a23 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h @@ -35,7 +35,7 @@ public: void writeBeforeTotals() override; void writeBeforeExtremes() override; - void doWritePrefix() override; + void writePrefix() override; /// https://www.iana.org/assignments/media-types/text/tab-separated-values String getContentType() const override { return "text/tab-separated-values; charset=UTF-8"; } diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp index db5db4701a9..cce46cac183 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -39,7 +40,7 @@ TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, Writ case static_cast(ResultsetPart::Totals): case static_cast(ResultsetPart::ExtremesMin): case static_cast(ResultsetPart::ExtremesMax): - if (format.formats[i] != ColumnFormat::None) + if (format.escaping_rules[i] != EscapingRule::None) format.throwInvalidFormat("Serialization type for data, totals, min and max must be empty or None", i); break; case static_cast(ResultsetPart::Rows): @@ -47,7 +48,7 @@ TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, Writ case static_cast(ResultsetPart::TimeElapsed): case static_cast(ResultsetPart::RowsRead): case static_cast(ResultsetPart::BytesRead): - if (format.formats[i] == ColumnFormat::None) + if (format.escaping_rules[i] == EscapingRule::None) format.throwInvalidFormat("Serialization type for output part rows, rows_before_limit, time, " "rows_read or bytes_read is not specified", i); break; @@ -68,7 +69,7 @@ TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, Writ if (header_.columns() <= *row_format.format_idx_to_column_idx[i]) row_format.throwInvalidFormat("Column index " + std::to_string(*row_format.format_idx_to_column_idx[i]) + " must be less then number of columns (" + std::to_string(header_.columns()) + ")", i); - if (row_format.formats[i] == ColumnFormat::None) + if (row_format.escaping_rules[i] == EscapingRule::None) row_format.throwInvalidFormat("Serialization type for file column is not specified", i); } } @@ -105,44 +106,17 @@ void TemplateBlockOutputFormat::writeRow(const Chunk & chunk, size_t row_num) writeString(row_format.delimiters[j], out); size_t col_idx = *row_format.format_idx_to_column_idx[j]; - serializeField(*chunk.getColumns()[col_idx], *serializations[col_idx], row_num, row_format.formats[j]); + serializeFieldByEscapingRule(*chunk.getColumns()[col_idx], *serializations[col_idx], out, row_num, row_format.escaping_rules[j], settings); } writeString(row_format.delimiters[columns], out); } -void TemplateBlockOutputFormat::serializeField(const IColumn & column, const ISerialization & serialization, size_t row_num, ColumnFormat col_format) -{ - switch (col_format) - { - case ColumnFormat::Escaped: - serialization.serializeTextEscaped(column, row_num, out, settings); - break; - case ColumnFormat::Quoted: - serialization.serializeTextQuoted(column, row_num, out, settings); - break; - case ColumnFormat::Csv: - serialization.serializeTextCSV(column, row_num, out, settings); - break; - case ColumnFormat::Json: - serialization.serializeTextJSON(column, row_num, out, settings); - break; - case ColumnFormat::Xml: - serialization.serializeTextXML(column, row_num, out, settings); - break; - case ColumnFormat::Raw: - serialization.serializeTextRaw(column, row_num, out, settings); - break; - default: - __builtin_unreachable(); - } -} - -template void TemplateBlockOutputFormat::writeValue(U value, ColumnFormat col_format) +template void TemplateBlockOutputFormat::writeValue(U value, EscapingRule escaping_rule) { auto type = std::make_unique(); auto col = type->createColumn(); col->insert(value); - serializeField(*col, *type->getDefaultSerialization(), 0, col_format); + serializeFieldByEscapingRule(*col, *type->getDefaultSerialization(), out, 0, escaping_rule, settings); } void TemplateBlockOutputFormat::consume(Chunk chunk) @@ -201,21 +175,21 @@ void TemplateBlockOutputFormat::finalize() writeRow(extremes, 1); break; case ResultsetPart::Rows: - writeValue(row_count, format.formats[i]); + writeValue(row_count, format.escaping_rules[i]); break; case ResultsetPart::RowsBeforeLimit: if (!rows_before_limit_set) format.throwInvalidFormat("Cannot print rows_before_limit for this request", i); - writeValue(rows_before_limit, format.formats[i]); + writeValue(rows_before_limit, format.escaping_rules[i]); break; case ResultsetPart::TimeElapsed: - writeValue(watch.elapsedSeconds(), format.formats[i]); + writeValue(watch.elapsedSeconds(), format.escaping_rules[i]); break; case ResultsetPart::RowsRead: - writeValue(progress.read_rows.load(), format.formats[i]); + writeValue(progress.read_rows.load(), format.escaping_rules[i]); break; case ResultsetPart::BytesRead: - writeValue(progress.read_bytes.load(), format.formats[i]); + writeValue(progress.read_bytes.load(), format.escaping_rules[i]); break; default: break; @@ -240,7 +214,7 @@ void registerOutputFormatTemplate(FormatFactory & factory) { /// Default format string: "${data}" resultset_format.delimiters.resize(2); - resultset_format.formats.emplace_back(ParsedTemplateFormatString::ColumnFormat::None); + resultset_format.escaping_rules.emplace_back(ParsedTemplateFormatString::EscapingRule::None); resultset_format.format_idx_to_column_idx.emplace_back(0); resultset_format.column_names.emplace_back("data"); } @@ -266,17 +240,5 @@ void registerOutputFormatTemplate(FormatFactory & factory) return std::make_shared(sample, buf, settings, resultset_format, row_format, settings.template_settings.row_between_delimiter); }); - - factory.registerOutputFormat("CustomSeparated", []( - WriteBuffer & buf, - const Block & sample, - const RowOutputFormatParams &, - const FormatSettings & settings) - { - ParsedTemplateFormatString resultset_format = ParsedTemplateFormatString::setupCustomSeparatedResultsetFormat(settings.custom); - ParsedTemplateFormatString row_format = ParsedTemplateFormatString::setupCustomSeparatedRowFormat(settings.custom, sample); - - return std::make_shared(sample, buf, settings, resultset_format, row_format, settings.custom.row_between_delimiter); - }); } } diff --git a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h index 0d41b8888d4..5aff07619f3 100644 --- a/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h +++ b/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h @@ -12,7 +12,7 @@ namespace DB class TemplateBlockOutputFormat : public IOutputFormat { - using ColumnFormat = ParsedTemplateFormatString::ColumnFormat; + using EscapingRule = FormatSettings::EscapingRule; public: TemplateBlockOutputFormat(const Block & header_, WriteBuffer & out_, const FormatSettings & settings_, ParsedTemplateFormatString format_, ParsedTemplateFormatString row_format_, @@ -47,8 +47,7 @@ protected: void finalize() override; void writeRow(const Chunk & chunk, size_t row_num); - void serializeField(const IColumn & column, const ISerialization & serialization, size_t row_num, ColumnFormat format); - template void writeValue(U value, ColumnFormat col_format); + template void writeValue(U value, EscapingRule escaping_rule); protected: const FormatSettings settings; diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp index c096b62e967..f03b0f00e01 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -38,14 +39,14 @@ TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, ReadBuffer format.throwInvalidFormat("Invalid input part", i); if (has_data) format.throwInvalidFormat("${data} can occur only once", i); - if (format.formats[i] != ColumnFormat::None) + if (format.escaping_rules[i] != EscapingRule::None) format.throwInvalidFormat("${data} must have empty or None deserialization type", i); has_data = true; format_data_idx = i; } else { - if (format.formats[i] == ColumnFormat::Xml) + if (format.escaping_rules[i] == EscapingRule::XML) format.throwInvalidFormat("XML deserialization is not supported", i); } } @@ -54,7 +55,7 @@ TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, ReadBuffer std::vector column_in_format(header_.columns(), false); for (size_t i = 0; i < row_format.columnsCount(); ++i) { - if (row_format.formats[i] == ColumnFormat::Xml) + if (row_format.escaping_rules[i] == EscapingRule::XML) row_format.throwInvalidFormat("XML deserialization is not supported", i); if (row_format.format_idx_to_column_idx[i]) @@ -62,7 +63,7 @@ TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, ReadBuffer if (header_.columns() <= *row_format.format_idx_to_column_idx[i]) row_format.throwInvalidFormat("Column index " + std::to_string(*row_format.format_idx_to_column_idx[i]) + " must be less then number of columns (" + std::to_string(header_.columns()) + ")", i); - if (row_format.formats[i] == ColumnFormat::None) + if (row_format.escaping_rules[i] == EscapingRule::None) row_format.throwInvalidFormat("Column is not skipped, but deserialization type is None", i); size_t col_idx = *row_format.format_idx_to_column_idx[i]; @@ -111,12 +112,12 @@ ReturnType TemplateRowInputFormat::tryReadPrefixOrSuffix(size_t & input_part_beg { skipSpaces(); if constexpr (throw_exception) - skipField(format.formats[input_part_beg]); + skipField(format.escaping_rules[input_part_beg]); else { try { - skipField(format.formats[input_part_beg]); + skipField(format.escaping_rules[input_part_beg]); } catch (const Exception & e) { @@ -176,7 +177,7 @@ bool TemplateRowInputFormat::readRow(MutableColumns & columns, RowReadExtension extra.read_columns[col_idx] = deserializeField(data_types[col_idx], serializations[col_idx], *columns[col_idx], i); } else - skipField(row_format.formats[i]); + skipField(row_format.escaping_rules[i]); } @@ -192,49 +193,14 @@ bool TemplateRowInputFormat::readRow(MutableColumns & columns, RowReadExtension bool TemplateRowInputFormat::deserializeField(const DataTypePtr & type, const SerializationPtr & serialization, IColumn & column, size_t file_column) { - ColumnFormat col_format = row_format.formats[file_column]; - bool read = true; - bool parse_as_nullable = settings.null_as_default && !type->isNullable() && !type->isLowCardinalityNullable(); + EscapingRule escaping_rule = row_format.escaping_rules[file_column]; + if (escaping_rule == EscapingRule::CSV) + /// Will read unquoted string until settings.csv.delimiter + settings.csv.delimiter = row_format.delimiters[file_column + 1].empty() ? default_csv_delimiter : + row_format.delimiters[file_column + 1].front(); try { - switch (col_format) - { - case ColumnFormat::Escaped: - if (parse_as_nullable) - read = SerializationNullable::deserializeTextEscapedImpl(column, buf, settings, serialization); - else - serialization->deserializeTextEscaped(column, buf, settings); - break; - case ColumnFormat::Quoted: - if (parse_as_nullable) - read = SerializationNullable::deserializeTextQuotedImpl(column, buf, settings, serialization); - else - serialization->deserializeTextQuoted(column, buf, settings); - break; - case ColumnFormat::Csv: - /// Will read unquoted string until settings.csv.delimiter - settings.csv.delimiter = row_format.delimiters[file_column + 1].empty() ? default_csv_delimiter : - row_format.delimiters[file_column + 1].front(); - if (parse_as_nullable) - read = SerializationNullable::deserializeTextCSVImpl(column, buf, settings, serialization); - else - serialization->deserializeTextCSV(column, buf, settings); - break; - case ColumnFormat::Json: - if (parse_as_nullable) - read = SerializationNullable::deserializeTextJSONImpl(column, buf, settings, serialization); - else - serialization->deserializeTextJSON(column, buf, settings); - break; - case ColumnFormat::Raw: - if (parse_as_nullable) - read = SerializationNullable::deserializeTextRawImpl(column, buf, settings, serialization); - else - serialization->deserializeTextRaw(column, buf, settings); - break; - default: - __builtin_unreachable(); - } + return deserializeFieldByEscapingRule(type, serialization, column, buf, escaping_rule, settings); } catch (Exception & e) { @@ -242,36 +208,13 @@ bool TemplateRowInputFormat::deserializeField(const DataTypePtr & type, throwUnexpectedEof(); throw; } - return read; } -void TemplateRowInputFormat::skipField(TemplateRowInputFormat::ColumnFormat col_format) +void TemplateRowInputFormat::skipField(TemplateRowInputFormat::EscapingRule escaping_rule) { - String tmp; - constexpr const char * field_name = ""; - constexpr size_t field_name_len = 16; try { - switch (col_format) - { - case ColumnFormat::None: - /// Empty field, just skip spaces - break; - case ColumnFormat::Escaped: - readEscapedString(tmp, buf); - break; - case ColumnFormat::Quoted: - readQuotedString(tmp, buf); - break; - case ColumnFormat::Csv: - readCSVString(tmp, buf, settings.csv); - break; - case ColumnFormat::Json: - skipJSONField(buf, StringRef(field_name, field_name_len)); - break; - default: - __builtin_unreachable(); - } + skipFieldByEscapingRule(buf, escaping_rule, settings); } catch (Exception & e) { @@ -344,29 +287,13 @@ bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & col out << "\nUsing format string (from format_schema_rows): " << row_format.dump() << "\n"; out << "\nTrying to parse next row, because suffix does not match:\n"; - try - { - if (likely(row_num != 1)) - assertString(row_between_delimiter, buf); - } - catch (const DB::Exception &) - { - writeErrorStringForWrongDelimiter(out, "delimiter between rows", row_between_delimiter); - + if (likely(row_num != 1) && !parseDelimiterWithDiagnosticInfo(out, buf, row_between_delimiter, "delimiter between rows", ignore_spaces)) return false; - } + for (size_t i = 0; i < row_format.columnsCount(); ++i) { - skipSpaces(); - try - { - assertString(row_format.delimiters[i], buf); - } - catch (const DB::Exception &) - { - writeErrorStringForWrongDelimiter(out, "delimiter before field " + std::to_string(i), row_format.delimiters[i]); + if (!parseDelimiterWithDiagnosticInfo(out, buf, row_format.delimiters[i], "delimiter before field " + std::to_string(i), ignore_spaces)) return false; - } skipSpaces(); if (row_format.format_idx_to_column_idx[i]) @@ -377,7 +304,7 @@ bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & col *columns[col_idx], out, i)) { out << "Maybe it's not possible to deserialize field " + std::to_string(i) + - " as " + ParsedTemplateFormatString::formatToString(row_format.formats[i]); + " as " + escapingRuleToString(row_format.escaping_rules[i]); return false; } } @@ -391,39 +318,39 @@ bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & col } } - skipSpaces(); + return parseDelimiterWithDiagnosticInfo(out, buf, row_format.delimiters.back(), "delimiter after last field", ignore_spaces); +} + +bool parseDelimiterWithDiagnosticInfo(WriteBuffer & out, ReadBuffer & buf, const String & delimiter, const String & description, bool skip_spaces) +{ + if (skip_spaces) + skipWhitespaceIfAny(buf); try { - assertString(row_format.delimiters.back(), buf); + assertString(delimiter, buf); } catch (const DB::Exception &) { - writeErrorStringForWrongDelimiter(out, "delimiter after last field", row_format.delimiters.back()); + out << "ERROR: There is no " << description << ": expected "; + verbosePrintString(delimiter.data(), delimiter.data() + delimiter.size(), out); + out << ", got "; + if (buf.eof()) + out << ""; + else + verbosePrintString(buf.position(), std::min(buf.position() + delimiter.size() + 10, buf.buffer().end()), out); + out << '\n'; return false; } - return true; } -void TemplateRowInputFormat::writeErrorStringForWrongDelimiter(WriteBuffer & out, const String & description, const String & delim) -{ - out << "ERROR: There is no " << description << ": expected "; - verbosePrintString(delim.data(), delim.data() + delim.size(), out); - out << ", got "; - if (buf.eof()) - out << ""; - else - verbosePrintString(buf.position(), std::min(buf.position() + delim.size() + 10, buf.buffer().end()), out); - out << '\n'; -} - void TemplateRowInputFormat::tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) { const auto & index = row_format.format_idx_to_column_idx[file_column]; if (index) deserializeField(type, serializations[*index], column, file_column); else - skipField(row_format.formats[file_column]); + skipField(row_format.escaping_rules[file_column]); } bool TemplateRowInputFormat::isGarbageAfterField(size_t, ReadBuffer::Position) @@ -439,62 +366,13 @@ bool TemplateRowInputFormat::allowSyncAfterError() const void TemplateRowInputFormat::syncAfterError() { - bool at_beginning_of_row_or_eof = false; - while (!at_beginning_of_row_or_eof) - { - skipToNextDelimiterOrEof(row_format.delimiters.back()); - if (buf.eof()) - { - end_of_stream = true; - return; - } - buf.ignore(row_format.delimiters.back().size()); - - skipSpaces(); - if (checkForSuffix()) - return; - - bool last_delimiter_in_row_found = !row_format.delimiters.back().empty(); - - if (last_delimiter_in_row_found && checkString(row_between_delimiter, buf)) - at_beginning_of_row_or_eof = true; - else - skipToNextDelimiterOrEof(row_between_delimiter); - - if (buf.eof()) - at_beginning_of_row_or_eof = end_of_stream = true; - } + skipToNextRowOrEof(buf, row_format.delimiters.back(), row_between_delimiter, ignore_spaces); + end_of_stream = buf.eof(); /// It can happen that buf.position() is not at the beginning of row /// if some delimiters is similar to row_format.delimiters.back() and row_between_delimiter. /// It will cause another parsing error. } -/// Searches for delimiter in input stream and sets buffer position to the beginning of delimiter (if found) or EOF (if not) -void TemplateRowInputFormat::skipToNextDelimiterOrEof(const String & delimiter) -{ - if (delimiter.empty()) - return; - - while (!buf.eof()) - { - void * pos = memchr(buf.position(), delimiter[0], buf.available()); - if (!pos) - { - buf.position() += buf.available(); - continue; - } - - buf.position() = static_cast(pos); - - PeekableReadBufferCheckpoint checkpoint{buf}; - if (checkString(delimiter, buf)) - return; - - buf.rollbackToCheckpoint(); - ++buf.position(); - } -} - void TemplateRowInputFormat::throwUnexpectedEof() { throw ParsingException("Unexpected EOF while parsing row " + std::to_string(row_num) + ". " @@ -524,7 +402,7 @@ void registerInputFormatTemplate(FormatFactory & factory) { /// Default format string: "${data}" resultset_format.delimiters.resize(2); - resultset_format.formats.emplace_back(ParsedTemplateFormatString::ColumnFormat::None); + resultset_format.escaping_rules.emplace_back(ParsedTemplateFormatString::EscapingRule::None); resultset_format.format_idx_to_column_idx.emplace_back(0); resultset_format.column_names.emplace_back("data"); } @@ -554,21 +432,6 @@ void registerInputFormatTemplate(FormatFactory & factory) return std::make_shared(sample, buf, params, settings, ignore_spaces, resultset_format, row_format, settings.template_settings.row_between_delimiter); }); } - - for (bool ignore_spaces : {false, true}) - { - factory.registerInputFormat(ignore_spaces ? "CustomSeparatedIgnoreSpaces" : "CustomSeparated", [=]( - ReadBuffer & buf, - const Block & sample, - IRowInputFormat::Params params, - const FormatSettings & settings) - { - ParsedTemplateFormatString resultset_format = ParsedTemplateFormatString::setupCustomSeparatedResultsetFormat(settings.custom); - ParsedTemplateFormatString row_format = ParsedTemplateFormatString::setupCustomSeparatedRowFormat(settings.custom, sample); - - return std::make_shared(sample, buf, params, settings, ignore_spaces, resultset_format, row_format, settings.custom.row_between_delimiter); - }); - } } } diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.h b/src/Processors/Formats/Impl/TemplateRowInputFormat.h index 322f8570ab7..096382a2249 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.h +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.h @@ -13,7 +13,7 @@ namespace DB class TemplateRowInputFormat : public RowInputFormatWithDiagnosticInfo { - using ColumnFormat = ParsedTemplateFormatString::ColumnFormat; + using EscapingRule = FormatSettings::EscapingRule; public: TemplateRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, FormatSettings settings_, bool ignore_spaces_, @@ -35,7 +35,7 @@ private: bool deserializeField(const DataTypePtr & type, const SerializationPtr & serialization, IColumn & column, size_t file_column); - void skipField(ColumnFormat col_format); + void skipField(EscapingRule escaping_rule); inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(buf); } template @@ -47,11 +47,7 @@ private: void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override; bool isGarbageAfterField(size_t after_col_idx, ReadBuffer::Position pos) override; - void writeErrorStringForWrongDelimiter(WriteBuffer & out, const String & description, const String & delim); - void skipToNextDelimiterOrEof(const String & delimiter); - -private: PeekableReadBuffer buf; const DataTypes data_types; @@ -68,4 +64,6 @@ private: const std::string row_between_delimiter; }; +bool parseDelimiterWithDiagnosticInfo(WriteBuffer & out, ReadBuffer & buf, const String & delimiter, const String & description, bool skip_spaces); + } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index b5690d9dafb..5dd5b586313 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -75,6 +75,11 @@ void RowInputFormatWithNamesAndTypes::addInputColumn(const String & column_name, void RowInputFormatWithNamesAndTypes::readPrefix() { + /// This is a bit of abstraction leakage, but we need it in parallel parsing: + /// we check if this InputFormat is working with the "real" beginning of the data. + if (getCurrentUnitNumber() != 0) + return; + if (with_names || with_types || data_types.at(0)->textCanContainOnlyValidUTF8()) { /// We assume that column name or type cannot contain BOM, so, if format has header, @@ -82,9 +87,12 @@ void RowInputFormatWithNamesAndTypes::readPrefix() skipBOMIfExists(*in); } + /// Skip prefix before names and types. + skipPrefixBeforeHeader(); + /// This is a bit of abstraction leakage, but we need it in parallel parsing: /// we check if this InputFormat is working with the "real" beginning of the data. - if (with_names && getCurrentUnitNumber() == 0) + if (with_names) { if (format_settings.with_names_use_header) { @@ -108,8 +116,10 @@ void RowInputFormatWithNamesAndTypes::readPrefix() else if (!column_mapping->is_set) setupAllColumnsByTableSchema(); - if (with_types && getCurrentUnitNumber() == 0) + if (with_types) { + /// Skip delimiter between names and types. + skipRowBetweenDelimiter(); if (format_settings.with_types_use_header) { auto types = readTypes(); @@ -148,10 +158,20 @@ void RowInputFormatWithNamesAndTypes::insertDefaultsForNotSeenColumns(MutableCol bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadExtension & ext) { - if (in->eof()) + if (unlikely(end_of_stream)) return false; + if (unlikely(checkForSuffix())) + { + end_of_stream = true; + return false; + } + updateDiagnosticInfo(); + + if (likely(row_num != 1 || getCurrentUnitNumber() == 0 && (with_names || with_types))) + skipRowBetweenDelimiter(); + skipRowStartDelimiter(); ext.read_columns.resize(data_types.size()); @@ -190,6 +210,7 @@ void RowInputFormatWithNamesAndTypes::resetParser() column_mapping->column_indexes_for_input_fields.clear(); column_mapping->not_presented_columns.clear(); column_mapping->names_of_columns.clear(); + end_of_stream = false; } void RowInputFormatWithNamesAndTypes::tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) @@ -215,6 +236,12 @@ bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColu return false; } + if (!tryParseSuffixWithDiagnosticInfo(out)) + return false; + + if (likely(row_num != 1) && !parseRowBetweenDelimiterWithDiagnosticInfo(out)) + return false; + if (!parseRowStartWithDiagnosticInfo(out)) return false; diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h index 0fd83238f5f..52fdcc88468 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.h @@ -46,16 +46,22 @@ protected: virtual void skipTypes() = 0; /// Skip delimiters, if any. + virtual void skipPrefixBeforeHeader() {} virtual void skipRowStartDelimiter() {} virtual void skipFieldDelimiter() {} virtual void skipRowEndDelimiter() {} + virtual void skipRowBetweenDelimiter() {} + /// Check suffix. + virtual bool checkForSuffix() { return in->eof(); } /// Methods for parsing with diagnostic info. virtual void checkNullValueForNonNullable(DataTypePtr) {} virtual bool parseRowStartWithDiagnosticInfo(WriteBuffer &) { return true; } virtual bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer &) { return true; } virtual bool parseRowEndWithDiagnosticInfo(WriteBuffer &) { return true;} + virtual bool parseRowBetweenDelimiterWithDiagnosticInfo(WriteBuffer &) { return true;} + virtual bool tryParseSuffixWithDiagnosticInfo(WriteBuffer &) { return true; } bool isGarbageAfterField(size_t, ReadBuffer::Position) override {return false; } /// Read row with names and return the list of them. @@ -65,6 +71,7 @@ protected: const FormatSettings format_settings; DataTypes data_types; + bool end_of_stream = false; private: bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override; From 8e9783388bbbf10a4fad6af47da627df5d909c6a Mon Sep 17 00:00:00 2001 From: avogar Date: Mon, 15 Nov 2021 22:59:24 +0300 Subject: [PATCH 096/200] Add formats CustomSeparatedWithNames/WithNamesAndTypes --- src/Formats/EscapingRuleUtils.cpp | 3 + .../Impl/CustomSeparatedRowInputFormat.cpp | 22 ++++- .../Impl/CustomSeparatedRowInputFormat.h | 5 ++ .../Impl/CustomSeparatedRowOutputFormat.cpp | 16 ++-- .../Impl/CustomSeparatedRowOutputFormat.h | 2 +- .../Impl/JSONAsStringRowInputFormat.cpp | 2 +- .../Formats/Impl/MsgPackRowInputFormat.cpp | 2 +- .../Formats/Impl/MsgPackRowOutputFormat.cpp | 2 +- .../Formats/Impl/RegexpRowInputFormat.cpp | 2 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 2 - .../01195_formats_diagnostic_info.reference | 6 +- .../02098_with_types_use_header.reference | 4 + .../02098_with_types_use_header.sh | 4 + ...m_separated_with_names_and_types.reference | 71 ++++++++++++++++ ...7_custom_separated_with_names_and_types.sh | 84 +++++++++++++++++++ 15 files changed, 209 insertions(+), 18 deletions(-) create mode 100644 tests/queries/0_stateless/02117_custom_separated_with_names_and_types.reference create mode 100755 tests/queries/0_stateless/02117_custom_separated_with_names_and_types.sh diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index c98d64e755c..d0f7e0b14f2 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -71,6 +71,9 @@ void skipFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule esca readEscapedString(tmp, buf); break; case FormatSettings::EscapingRule::Quoted: + /// FIXME: it skips only strings, not numbers, arrays or tuples. + /// we should read until delimiter and skip all data between + /// single quotes. readQuotedString(tmp, buf); break; case FormatSettings::EscapingRule::CSV: diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp index 8e2c5165531..6ff9a8cca2c 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.cpp @@ -12,6 +12,16 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +static FormatSettings updateFormatSettings(const FormatSettings & settings) +{ + if (settings.custom.escaping_rule != FormatSettings::EscapingRule::CSV || settings.custom.field_delimiter.empty()) + return settings; + + auto updated = settings; + updated.csv.delimiter = settings.custom.field_delimiter.front(); + return updated; +} + CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat( const Block & header_, ReadBuffer & in_, @@ -20,7 +30,7 @@ CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat( bool with_types_, bool ignore_spaces_, const FormatSettings & format_settings_) - : RowInputFormatWithNamesAndTypes(header_, in_, params_, with_names_, with_types_, format_settings_) + : RowInputFormatWithNamesAndTypes(header_, buf, params_, with_names_, with_types_, updateFormatSettings(format_settings_)) , buf(in_) , ignore_spaces(ignore_spaces_) , escaping_rule(format_settings_.custom.escaping_rule) @@ -134,6 +144,14 @@ bool CustomSeparatedRowInputFormat::readField(IColumn & column, const DataTypePt bool CustomSeparatedRowInputFormat::checkForSuffixImpl(bool check_eof) { skipSpaces(); + if (format_settings.custom.result_after_delimiter.empty()) + { + if (!check_eof) + return false; + + return buf.eof(); + } + if (unlikely(checkString(format_settings.custom.result_after_delimiter, buf))) { skipSpaces(); @@ -151,7 +169,7 @@ bool CustomSeparatedRowInputFormat::tryParseSuffixWithDiagnosticInfo(WriteBuffer PeekableReadBufferCheckpoint checkpoint{buf}; if (checkForSuffixImpl(false)) { - if (in->eof()) + if (buf.eof()) out << "\n"; else out << " There is some data after suffix\n"; diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h index 9dcda0810ac..00ee28e50cc 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h +++ b/src/Processors/Formats/Impl/CustomSeparatedRowInputFormat.h @@ -21,6 +21,11 @@ public: String getName() const override { return "CustomSeparatedRowInputFormat"; } private: + CustomSeparatedRowInputFormat( + const Block & header_, + std::unique_ptr in_, + const Params & params_, + bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_settings_); using EscapingRule = FormatSettings::EscapingRule; bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override; diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp index a1ce4fc50d5..d2a60e9aecb 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp @@ -24,11 +24,17 @@ CustomSeparatedRowOutputFormat::CustomSeparatedRowOutputFormat( void CustomSeparatedRowOutputFormat::writeLine(const std::vector & values) { - for (const auto & value : values) - writeStringByEscapingRule(value, out, escaping_rule, format_settings); + writeRowStartDelimiter(); + for (size_t i = 0; i != values.size(); ++i) + { + writeStringByEscapingRule(values[i], out, escaping_rule, format_settings); + if (i + 1 != values.size()) + writeFieldDelimiter(); + } + writeRowEndDelimiter(); } -void CustomSeparatedRowOutputFormat::writePrefix() +void CustomSeparatedRowOutputFormat::doWritePrefix() { writeString(format_settings.custom.result_before_delimiter, out); @@ -88,9 +94,7 @@ void registerOutputFormatCustomSeparated(FormatFactory & factory) { return std::make_shared(sample, buf, params, settings, with_names, with_types); }); - - factory.markOutputFormatSupportsParallelFormatting(format_name); - }; + }; registerWithNamesAndTypes("CustomSeparated", register_func); } diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h index 274df1af330..3ee5389d48c 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h +++ b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h @@ -23,7 +23,7 @@ private: void writeRowStartDelimiter() override; void writeRowEndDelimiter() override; void writeRowBetweenDelimiter() override; - void writePrefix() override; + void doWritePrefix() override; void writeSuffix() override; void writeLine(const std::vector & values); diff --git a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp index a5e0ac6862c..1cca53b2f56 100644 --- a/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONAsStringRowInputFormat.cpp @@ -15,7 +15,7 @@ namespace ErrorCodes } JSONAsStringRowInputFormat::JSONAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) : - IRowInputFormat(header_, in_, std::move(params_)), buf(*in) + IRowInputFormat(header_, buf, std::move(params_)), buf(in_) { if (header_.columns() > 1) throw Exception(ErrorCodes::BAD_ARGUMENTS, diff --git a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp index 86540be17b9..e34729be928 100644 --- a/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowInputFormat.cpp @@ -29,7 +29,7 @@ namespace ErrorCodes } MsgPackRowInputFormat::MsgPackRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_) - : IRowInputFormat(header_, in_, std::move(params_)), buf(*in), parser(visitor), data_types(header_.getDataTypes()) {} + : IRowInputFormat(header_, buf, std::move(params_)), buf(in_), parser(visitor), data_types(header_.getDataTypes()) {} void MsgPackRowInputFormat::resetParser() { diff --git a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp index bbc7b005fd8..36a8a62b39e 100644 --- a/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MsgPackRowOutputFormat.cpp @@ -182,7 +182,6 @@ void MsgPackRowOutputFormat::write(const Columns & columns, size_t row_num) void registerOutputFormatMsgPack(FormatFactory & factory) { - factory.registerOutputFormat("MsgPack", []( WriteBuffer & buf, const Block & sample, @@ -191,6 +190,7 @@ void registerOutputFormatMsgPack(FormatFactory & factory) { return std::make_shared(buf, sample, params); }); + factory.markOutputFormatSupportsParallelFormatting("MsgPack"); } } diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index b0b979fab11..db39c7d2d4c 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -17,7 +17,7 @@ namespace ErrorCodes RegexpRowInputFormat::RegexpRowInputFormat( ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_) - : IRowInputFormat(header_, in_, std::move(params_)) + : IRowInputFormat(header_, buf, std::move(params_)) , buf(in_) , format_settings(format_settings_) , escaping_rule(format_settings_.regexp.escaping_rule) diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 5f471dc0151..adf6d2e8a25 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include @@ -12,7 +11,6 @@ #include #include #include -#include #include #include #include diff --git a/tests/queries/0_stateless/01195_formats_diagnostic_info.reference b/tests/queries/0_stateless/01195_formats_diagnostic_info.reference index eddbb80198d..3c21f8124d8 100644 --- a/tests/queries/0_stateless/01195_formats_diagnostic_info.reference +++ b/tests/queries/0_stateless/01195_formats_diagnostic_info.reference @@ -7,8 +7,8 @@ ERROR: garbage after Decimal(18, 10): "Hello" Column 0, name: t, type: DateTime, ERROR: text "" is not like DateTime CustomSeparatedIgnoreSpaces -Column 2, name: d, type: Decimal(18, 10), parsed text: "123456789"ERROR -ERROR: There is no delimiter before field 1: expected ",", got "7, Hello," +Column 2, name: d, type: Decimal(18, 10), parsed text: " 123456789"ERROR +ERROR: There is no delimiter between fields: expected ",", got "7, Hello," Column 0, name: t, type: DateTime, ERROR: text ",1" is not like DateTime Column 0, name: t, type: DateTime, ERROR: text "Hello" is not like DateTime OK @@ -24,7 +24,7 @@ Unexpected NULL value CustomSeparated Column 2, name: d, type: Decimal(18, 10), parsed text: "123456789"ERROR -ERROR: There is no delimiter before field 1: expected "", got "7Hello123" +ERROR: There is no delimiter between fields: expected "", got "7Hello123" ERROR: There is no delimiter after last field: expected "", got "1" ERROR: There is no delimiter after last field: expected "", got "Hello" Column 0, name: t, type: DateTime, ERROR: text "" is not like DateTime diff --git a/tests/queries/0_stateless/02098_with_types_use_header.reference b/tests/queries/0_stateless/02098_with_types_use_header.reference index c1d70452d1d..fb79be8dccb 100644 --- a/tests/queries/0_stateless/02098_with_types_use_header.reference +++ b/tests/queries/0_stateless/02098_with_types_use_header.reference @@ -2,6 +2,10 @@ TSVWithNamesAndTypes OK OK OK +CustomSeparatedWithNamesAndTypes +OK +OK +OK CSVWithNamesAndTypes OK OK diff --git a/tests/queries/0_stateless/02098_with_types_use_header.sh b/tests/queries/0_stateless/02098_with_types_use_header.sh index 846696d18c0..5d88a994052 100755 --- a/tests/queries/0_stateless/02098_with_types_use_header.sh +++ b/tests/queries/0_stateless/02098_with_types_use_header.sh @@ -13,6 +13,10 @@ echo -e "x\ty\tz\nString\tDate\tUInt32\ntext\t2020-01-01\t1" | $CLICKHOUSE_CLIEN echo -e "y\tz\tx\nString\tDate\tUInt32\ntext\t2020-01-01\t1" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT TSVWithNamesAndTypes" && echo 'OK' || echo 'FAIL' echo -e "x\tz\ty\nUInt32\tString\tDate\n1\ttext\t2020-01-01" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT TSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo "CustomSeparatedWithNamesAndTypes" +echo -e "x\ty\tz\nString\tDate\tUInt32\ntext\t2020-01-01\t1" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CustomSeparatedWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' +echo -e "y\tz\tx\nString\tDate\tUInt32\ntext\t2020-01-01\t1" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CustomSeparatedWithNamesAndTypes" && echo 'OK' || echo 'FAIL' +echo -e "x\tz\ty\nUInt32\tString\tDate\n1\ttext\t2020-01-01" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CustomSeparatedWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' echo "CSVWithNamesAndTypes" echo -e "'x','y','z'\n'String','Date','UInt32'\n'text','2020-01-01',1" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02098 FORMAT CSVWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' diff --git a/tests/queries/0_stateless/02117_custom_separated_with_names_and_types.reference b/tests/queries/0_stateless/02117_custom_separated_with_names_and_types.reference new file mode 100644 index 00000000000..bbe0c6d9fcc --- /dev/null +++ b/tests/queries/0_stateless/02117_custom_separated_with_names_and_types.reference @@ -0,0 +1,71 @@ +CustomSeparated + +01"hello" + +12"hello" + +23"hello" + +34"hello" + +45"hello" + +0 1 hello +1 2 hello +2 3 hello +3 4 hello +4 5 hello +CustomSeparatedWithNames + +"x""y""s" + +01"hello" + +12"hello" + +23"hello" + +34"hello" + +45"hello" + +0 1 hello +1 2 hello +2 3 hello +3 4 hello +4 5 hello +CustomSeparatedWithNamesAndTypes + +"x""y""s" + +"UInt64""UInt64""String" + +01"hello" + +12"hello" + +23"hello" + +34"hello" + +45"hello" + +0 1 hello +1 2 hello +2 3 hello +3 4 hello +4 5 hello +1 text 2020-01-01 +1 text 2020-01-01 +1 text 2020-01-01 +1 text 2020-01-01 +1 text 2020-01-01 +1 text 2020-01-01 +1 default 1970-01-01 +1 default 1970-01-01 +1 1970-01-01 +1 1970-01-01 +1 default 1970-01-01 +1 default 1970-01-01 +OK +OK diff --git a/tests/queries/0_stateless/02117_custom_separated_with_names_and_types.sh b/tests/queries/0_stateless/02117_custom_separated_with_names_and_types.sh new file mode 100755 index 00000000000..c2dbec4b3be --- /dev/null +++ b/tests/queries/0_stateless/02117_custom_separated_with_names_and_types.sh @@ -0,0 +1,84 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +CUSTOM_SETTINGS="SETTINGS format_custom_row_before_delimiter='', format_custom_row_after_delimiter='\n', format_custom_row_between_delimiter='\n', format_custom_result_before_delimiter='\n', format_custom_result_after_delimiter='\n', format_custom_field_delimiter='', format_custom_escaping_rule='CSV'" + +$CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02117" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02117 (x UInt64, y UInt64, s String) engine=Memory()" + +for format in CustomSeparated CustomSeparatedWithNames CustomSeparatedWithNamesAndTypes +do + echo $format + $CLICKHOUSE_CLIENT -q "SELECT number AS x, number + 1 AS y, 'hello' AS s FROM numbers(5) FORMAT $format $CUSTOM_SETTINGS" + $CLICKHOUSE_CLIENT -q "SELECT number AS x, number + 1 AS y, 'hello' AS s FROM numbers(5) FORMAT $format $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT -q "INSERT INTO test_02117 FORMAT $format $CUSTOM_SETTINGS" + $CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" + $CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" +done + +$CLICKHOUSE_CLIENT -q "DROP TABLE test_02117" +$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02117 (x UInt32, y String DEFAULT 'default', z Date) engine=Memory()" + + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" + + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 --input_format_with_types_use_header=0 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" + + +$CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" + +$CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" + + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" + + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" + + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNames $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" + + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" +$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02117" +$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02117" + + +$CLICKHOUSE_CLIENT -q "SELECT 'text' AS x, toDate('2020-01-01') AS y, toUInt32(1) AS z FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' + +$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' as z, toDate('2020-01-01') AS y FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02117 FORMAT CustomSeparatedWithNamesAndTypes $CUSTOM_SETTINGS" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL' + +$CLICKHOUSE_CLIENT -q "DROP TABLE test_02117" \ No newline at end of file From 4f5e731513561cdbc31745d6ca1d947d4642874c Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 16 Nov 2021 11:39:18 +0300 Subject: [PATCH 097/200] Try fix tests --- src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp index 5d87f5a0b14..df0c19ad409 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.cpp @@ -30,7 +30,7 @@ void TabSeparatedRowOutputFormat::writeLine(const std::vector & values) } } -void TabSeparatedRowOutputFormat::writePrefix() +void TabSeparatedRowOutputFormat::doWritePrefix() { const auto & header = getPort(PortKind::Main).getHeader(); From be7a358f175ee38767a0e3a0dda5b939e389704f Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 16 Nov 2021 11:39:48 +0300 Subject: [PATCH 098/200] Fix --- src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h index c7e12716a23..7dcc6529f1c 100644 --- a/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h +++ b/src/Processors/Formats/Impl/TabSeparatedRowOutputFormat.h @@ -35,7 +35,7 @@ public: void writeBeforeTotals() override; void writeBeforeExtremes() override; - void writePrefix() override; + void doWritePrefix() override; /// https://www.iana.org/assignments/media-types/text/tab-separated-values String getContentType() const override { return "text/tab-separated-values; charset=UTF-8"; } From 1ebcbf47486de2d911e513165617255a158800c5 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 16 Nov 2021 17:10:18 +0300 Subject: [PATCH 099/200] Fix style --- src/Formats/EscapingRuleUtils.cpp | 2 -- src/Formats/ParsedTemplateFormatString.cpp | 1 - .../Impl/CustomSeparatedRowOutputFormat.cpp | 27 ++++++++----------- .../Formats/Impl/RegexpRowInputFormat.cpp | 1 - .../Formats/Impl/TemplateRowInputFormat.cpp | 4 +-- .../Formats/Impl/TemplateRowInputFormat.h | 2 +- .../RowInputFormatWithNamesAndTypes.cpp | 2 +- 7 files changed, 15 insertions(+), 24 deletions(-) diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index d0f7e0b14f2..2c2662a6a67 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -9,8 +9,6 @@ namespace DB namespace ErrorCodes { extern const int BAD_ARGUMENTS; - extern const int CANNOT_READ_ALL_DATA; - extern const int ATTEMPT_TO_READ_AFTER_EOF; } FormatSettings::EscapingRule stringToEscapingRule(const String & escaping_rule) diff --git a/src/Formats/ParsedTemplateFormatString.cpp b/src/Formats/ParsedTemplateFormatString.cpp index 013e4176fed..4966420f05b 100644 --- a/src/Formats/ParsedTemplateFormatString.cpp +++ b/src/Formats/ParsedTemplateFormatString.cpp @@ -11,7 +11,6 @@ namespace DB namespace ErrorCodes { - extern const int BAD_ARGUMENTS; extern const int INVALID_TEMPLATE_FORMAT; } diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp index d2a60e9aecb..4c6b980a7c4 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp @@ -7,11 +7,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - CustomSeparatedRowOutputFormat::CustomSeparatedRowOutputFormat( const Block & header_, WriteBuffer & out_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_, bool with_names_, bool with_types_) : IRowOutputFormat(header_, out_, params_) @@ -84,19 +79,19 @@ void CustomSeparatedRowOutputFormat::writeField(const IColumn & column, const IS void registerOutputFormatCustomSeparated(FormatFactory & factory) { - auto register_func = [&](const String & format_name, bool with_names, bool with_types) - { - factory.registerOutputFormat(format_name, [with_names, with_types]( - WriteBuffer & buf, - const Block & sample, - const RowOutputFormatParams & params, - const FormatSettings & settings) - { - return std::make_shared(sample, buf, params, settings, with_names, with_types); - }); + auto register_func = [&](const String & format_name, bool with_names, bool with_types) + { + factory.registerOutputFormat(format_name, [with_names, with_types]( + WriteBuffer & buf, + const Block & sample, + const RowOutputFormatParams & params, + const FormatSettings & settings) + { + return std::make_shared(sample, buf, params, settings, with_names, with_types); + }); }; - registerWithNamesAndTypes("CustomSeparated", register_func); + registerWithNamesAndTypes("CustomSeparated", register_func); } } diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index db39c7d2d4c..7dd7e6df267 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -11,7 +11,6 @@ namespace DB namespace ErrorCodes { extern const int INCORRECT_DATA; - extern const int BAD_ARGUMENTS; extern const int LOGICAL_ERROR; } diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp index f03b0f00e01..25162e927ac 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp @@ -318,10 +318,10 @@ bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & col } } - return parseDelimiterWithDiagnosticInfo(out, buf, row_format.delimiters.back(), "delimiter after last field", ignore_spaces); + return parseDelimiterWithDiagnosticInfo(out, buf, row_format.delimiters.back(), "delimiter after last field", ignore_spaces); } -bool parseDelimiterWithDiagnosticInfo(WriteBuffer & out, ReadBuffer & buf, const String & delimiter, const String & description, bool skip_spaces) +bool parseDelimiterWithDiagnosticInfo(WriteBuffer & out, ReadBuffer & buf, const String & delimiter, const String & description, bool skip_spaces) { if (skip_spaces) skipWhitespaceIfAny(buf); diff --git a/src/Processors/Formats/Impl/TemplateRowInputFormat.h b/src/Processors/Formats/Impl/TemplateRowInputFormat.h index 096382a2249..ff00b5c46d2 100644 --- a/src/Processors/Formats/Impl/TemplateRowInputFormat.h +++ b/src/Processors/Formats/Impl/TemplateRowInputFormat.h @@ -64,6 +64,6 @@ private: const std::string row_between_delimiter; }; -bool parseDelimiterWithDiagnosticInfo(WriteBuffer & out, ReadBuffer & buf, const String & delimiter, const String & description, bool skip_spaces); +bool parseDelimiterWithDiagnosticInfo(WriteBuffer & out, ReadBuffer & buf, const String & delimiter, const String & description, bool skip_spaces); } diff --git a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp index 5dd5b586313..87fa5ec1c4a 100644 --- a/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp +++ b/src/Processors/Formats/RowInputFormatWithNamesAndTypes.cpp @@ -169,7 +169,7 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE updateDiagnosticInfo(); - if (likely(row_num != 1 || getCurrentUnitNumber() == 0 && (with_names || with_types))) + if (likely(row_num != 1 || (getCurrentUnitNumber() == 0 && (with_names || with_types)))) skipRowBetweenDelimiter(); skipRowStartDelimiter(); From 4581b37da5a076fb395c348468062df6f1ca5481 Mon Sep 17 00:00:00 2001 From: Alexey Date: Tue, 16 Nov 2021 19:15:02 +0000 Subject: [PATCH 100/200] ru --- docs/ru/sql-reference/operators/exists.md | 31 +++++++++++++++++++ docs/ru/sql-reference/operators/index.md | 2 +- .../sql-reference/statements/select/where.md | 17 ++++++++++ 3 files changed, 49 insertions(+), 1 deletion(-) create mode 100644 docs/ru/sql-reference/operators/exists.md diff --git a/docs/ru/sql-reference/operators/exists.md b/docs/ru/sql-reference/operators/exists.md new file mode 100644 index 00000000000..c43150d9616 --- /dev/null +++ b/docs/ru/sql-reference/operators/exists.md @@ -0,0 +1,31 @@ +# EXISTS {#exists-operator} + +Оператор `EXISTS` проверяет, сколько строк возвращает подзапрос. Если результат пустой, то оператор возвращает `0`. В остальных случаях оператор возвращает `1`. + +`EXISTS` может быть использован в секции [WHERE](../../sql-reference/statements/select/where.md). + +!!! warning "Предупреждение" + Ссылки на таблицы или столбцы основного запроса не поддерживаются в подзапросе. + +**Синтаксис** + +```sql +WHERE EXISTS(subquery) +``` + +**Пример** + +Запрос: + +``` sql +SELECT 'Exists' WHERE EXISTS (SELECT * FROM numbers(10) WHERE number < 2); +SELECT 'Empty subquery' WHERE EXISTS (SELECT * FROM numbers(10) WHERE number > 12); +``` + +Первый запрос возвращает одну строку, а второй запрос не возвращает строк, так как результат его подзапроса пустой: + +``` text +┌─'Exists'─┐ +│ Exists │ +└──────────┘ +``` diff --git a/docs/ru/sql-reference/operators/index.md b/docs/ru/sql-reference/operators/index.md index 0041be745fc..3a2d3d69119 100644 --- a/docs/ru/sql-reference/operators/index.md +++ b/docs/ru/sql-reference/operators/index.md @@ -72,7 +72,7 @@ toc_title: "Операторы" ## Операторы для работы с множествами {#operatory-dlia-raboty-s-mnozhestvami} -*Смотрите раздел [Операторы IN](../../sql-reference/operators/in.md#select-in-operators).* +Смотрите [операторы IN](../../sql-reference/operators/in.md#select-in-operators) и оператор [EXISTS](../../sql-reference/operators/exists.md). `a IN ...` - функция `in(a, b)` diff --git a/docs/ru/sql-reference/statements/select/where.md b/docs/ru/sql-reference/statements/select/where.md index 8cb8e634303..3cfcbb0a849 100644 --- a/docs/ru/sql-reference/statements/select/where.md +++ b/docs/ru/sql-reference/statements/select/where.md @@ -27,3 +27,20 @@ SELECT * FROM t_null WHERE y IS NULL !!! note "Примечание" Существует оптимизация фильтрации под названием [prewhere](prewhere.md). +**Пример** + +Чтобы найти числа, которые кратны 3 и больше 10, можно выполнить запрос к [таблице numbers](../../../sql-reference/table-functions/numbers.md): + +``` sql +SELECT number FROM numbers(20) WHERE (number > 10) AND (number % 3 == 0); +``` + +Результат: + +``` text +┌─number─┐ +│ 12 │ +│ 15 │ +│ 18 │ +└────────┘ +``` From d331f4becea9391263aec1640714aa8ae3649038 Mon Sep 17 00:00:00 2001 From: Alexey Date: Tue, 16 Nov 2021 19:20:33 +0000 Subject: [PATCH 101/200] update --- docs/ru/sql-reference/operators/exists.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/operators/exists.md b/docs/ru/sql-reference/operators/exists.md index c43150d9616..d56765286f7 100644 --- a/docs/ru/sql-reference/operators/exists.md +++ b/docs/ru/sql-reference/operators/exists.md @@ -1,6 +1,6 @@ # EXISTS {#exists-operator} -Оператор `EXISTS` проверяет, сколько строк возвращает подзапрос. Если результат пустой, то оператор возвращает `0`. В остальных случаях оператор возвращает `1`. +Оператор `EXISTS` проверяет, сколько строк содержит результат выполнения подзапроса. Если результат пустой, то оператор возвращает `0`. В остальных случаях оператор возвращает `1`. `EXISTS` может быть использован в секции [WHERE](../../sql-reference/statements/select/where.md). From 41cdab6633b2b92b0df138b2352f7b14a09c2c36 Mon Sep 17 00:00:00 2001 From: alesapin Date: Tue, 16 Nov 2021 22:45:40 +0300 Subject: [PATCH 102/200] Enable all integration tests --- .github/workflows/main.yml | 2611 ++++++++++++++++++------------------ 1 file changed, 1335 insertions(+), 1276 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index b7c0347e593..836a2dd07a3 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -85,188 +85,188 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH -# FastTest: -# needs: DockerHubPush -# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} -# runs-on: [self-hosted, builder] -# steps: -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Fast Test -# env: -# TEMP_PATH: ${{runner.temp}}/fasttest -# REPO_COPY: ${{runner.temp}}/fasttest/ClickHouse -# CACHES_PATH: ${{runner.temp}}/../ccaches -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci && python3 fast_test_check.py -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# PVSCheck: -# needs: DockerHubPush -# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Check out repository code -# uses: actions/checkout@v2 -# with: -# submodules: 'recursive' -# - name: PVS Check -# env: -# TEMP_PATH: ${{runner.temp}}/pvs_check -# REPO_COPY: ${{runner.temp}}/pvs_check/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci && python3 pvs_check.py -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# CompatibilityCheck: -# needs: [BuilderDebRelease] -# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} -# runs-on: [self-hosted, style-checker] -# steps: -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: CompatibilityCheck -# env: -# TEMP_PATH: ${{runner.temp}}/compatibility_check -# REPO_COPY: ${{runner.temp}}/compatibility_check/ClickHouse -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci && python3 compatibility_check.py 0 -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# SplitBuildSmokeTest: -# needs: [BuilderDebSplitted] -# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} -# runs-on: [self-hosted, style-checker] -# steps: -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Split build check -# env: -# TEMP_PATH: ${{runner.temp}}/split_build_check -# REPO_COPY: ${{runner.temp}}/split_build_check/ClickHouse -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci && python3 split_build_smoke_check.py -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -########################################################################################## -##################################### ORDINARY BUILDS #################################### -########################################################################################## -# BuilderDebRelease: -# needs: DockerHubPush -# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} -# runs-on: [self-hosted, builder] -# steps: -# - name: Download changed images -# uses: actions/download-artifact@v2 -# with: -# name: changed_images -# path: ${{ runner.temp }}/images_path -# - name: Check out repository code -# uses: actions/checkout@v2 -# with: -# submodules: 'recursive' -# fetch-depth: 0 # otherwise we will have no info about contributors -# - name: Build -# env: -# TEMP_PATH: ${{runner.temp}}/build_check -# IMAGES_PATH: ${{runner.temp}}/images_path -# REPO_COPY: ${{runner.temp}}/build_check/ClickHouse -# CACHES_PATH: ${{runner.temp}}/../ccaches -# CHECK_NAME: 'ClickHouse build check (actions)' -# BUILD_NUMBER: 0 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER -# - name: Upload build URLs to artifacts -# uses: actions/upload-artifact@v2 -# with: -# name: ${{ env.BUILD_NAME }} -# path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# BuilderBinRelease: -# needs: DockerHubPush -# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} -# runs-on: [self-hosted, builder] -# steps: -# - name: Download changed images -# uses: actions/download-artifact@v2 -# with: -# name: changed_images -# path: ${{ runner.temp }}/images_path -# - name: Check out repository code -# uses: actions/checkout@v2 -# with: -# submodules: 'recursive' -# fetch-depth: 0 # otherwise we will have no info about contributors -# - name: Build -# env: -# TEMP_PATH: ${{runner.temp}}/build_check -# IMAGES_PATH: ${{runner.temp}}/images_path -# REPO_COPY: ${{runner.temp}}/build_check/ClickHouse -# CACHES_PATH: ${{runner.temp}}/../ccaches -# CHECK_NAME: 'ClickHouse build check (actions)' -# BUILD_NUMBER: 9 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER -# - name: Upload build URLs to artifacts -# uses: actions/upload-artifact@v2 -# with: -# name: ${{ env.BUILD_NAME }} -# path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH + FastTest: + needs: DockerHubPush + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, builder] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fast Test + env: + TEMP_PATH: ${{runner.temp}}/fasttest + REPO_COPY: ${{runner.temp}}/fasttest/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 fast_test_check.py + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + PVSCheck: + needs: DockerHubPush + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, func-tester] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + - name: PVS Check + env: + TEMP_PATH: ${{runner.temp}}/pvs_check + REPO_COPY: ${{runner.temp}}/pvs_check/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 pvs_check.py + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + CompatibilityCheck: + needs: [BuilderDebRelease] + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, style-checker] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: CompatibilityCheck + env: + TEMP_PATH: ${{runner.temp}}/compatibility_check + REPO_COPY: ${{runner.temp}}/compatibility_check/ClickHouse + REPORTS_PATH: ${{runner.temp}}/reports_dir + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 compatibility_check.py 0 + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + SplitBuildSmokeTest: + needs: [BuilderDebSplitted] + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, style-checker] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Split build check + env: + TEMP_PATH: ${{runner.temp}}/split_build_check + REPO_COPY: ${{runner.temp}}/split_build_check/ClickHouse + REPORTS_PATH: ${{runner.temp}}/reports_dir + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 split_build_smoke_check.py + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +######################################################################################### +#################################### ORDINARY BUILDS #################################### +######################################################################################### + BuilderDebRelease: + needs: DockerHubPush + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 0 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + BuilderBinRelease: + needs: DockerHubPush + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 9 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH BuilderDebAsan: needs: DockerHubPush if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} @@ -306,913 +306,913 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH -# BuilderDebUBsan: -# needs: DockerHubPush -# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} -# runs-on: [self-hosted, builder] -# steps: -# - name: Download changed images -# uses: actions/download-artifact@v2 -# with: -# name: changed_images -# path: ${{ runner.temp }}/images_path -# - name: Check out repository code -# uses: actions/checkout@v2 -# with: -# submodules: 'recursive' -# fetch-depth: 0 # otherwise we will have no info about contributors -# - name: Build -# env: -# TEMP_PATH: ${{runner.temp}}/build_check -# IMAGES_PATH: ${{runner.temp}}/images_path -# REPO_COPY: ${{runner.temp}}/build_check/ClickHouse -# CACHES_PATH: ${{runner.temp}}/../ccaches -# CHECK_NAME: 'ClickHouse build check (actions)' -# BUILD_NUMBER: 4 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER -# - name: Upload build URLs to artifacts -# uses: actions/upload-artifact@v2 -# with: -# name: ${{ env.BUILD_NAME }} -# path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# BuilderDebTsan: -# needs: DockerHubPush -# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} -# runs-on: [self-hosted, builder] -# steps: -# - name: Download changed images -# uses: actions/download-artifact@v2 -# with: -# name: changed_images -# path: ${{ runner.temp }}/images_path -# - name: Check out repository code -# uses: actions/checkout@v2 -# with: -# submodules: 'recursive' -# fetch-depth: 0 # otherwise we will have no info about contributors -# - name: Build -# env: -# TEMP_PATH: ${{runner.temp}}/build_check -# IMAGES_PATH: ${{runner.temp}}/images_path -# REPO_COPY: ${{runner.temp}}/build_check/ClickHouse -# CACHES_PATH: ${{runner.temp}}/../ccaches -# CHECK_NAME: 'ClickHouse build check (actions)' -# BUILD_NUMBER: 5 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER -# - name: Upload build URLs to artifacts -# uses: actions/upload-artifact@v2 -# with: -# name: ${{ env.BUILD_NAME }} -# path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# BuilderDebMsan: -# needs: DockerHubPush -# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} -# runs-on: [self-hosted, builder] -# steps: -# - name: Download changed images -# uses: actions/download-artifact@v2 -# with: -# name: changed_images -# path: ${{ runner.temp }}/images_path -# - name: Check out repository code -# uses: actions/checkout@v2 -# with: -# submodules: 'recursive' -# fetch-depth: 0 # otherwise we will have no info about contributors -# - name: Build -# env: -# TEMP_PATH: ${{runner.temp}}/build_check -# IMAGES_PATH: ${{runner.temp}}/images_path -# REPO_COPY: ${{runner.temp}}/build_check/ClickHouse -# CACHES_PATH: ${{runner.temp}}/../ccaches -# CHECK_NAME: 'ClickHouse build check (actions)' -# BUILD_NUMBER: 6 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER -# - name: Upload build URLs to artifacts -# uses: actions/upload-artifact@v2 -# with: -# name: ${{ env.BUILD_NAME }} -# path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# BuilderDebDebug: -# needs: DockerHubPush -# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} -# runs-on: [self-hosted, builder] -# steps: -# - name: Download changed images -# uses: actions/download-artifact@v2 -# with: -# name: changed_images -# path: ${{ runner.temp }}/images_path -# - name: Check out repository code -# uses: actions/checkout@v2 -# with: -# submodules: 'recursive' -# fetch-depth: 0 # otherwise we will have no info about contributors -# - name: Build -# env: -# TEMP_PATH: ${{runner.temp}}/build_check -# IMAGES_PATH: ${{runner.temp}}/images_path -# REPO_COPY: ${{runner.temp}}/build_check/ClickHouse -# CACHES_PATH: ${{runner.temp}}/../ccaches -# CHECK_NAME: 'ClickHouse build check (actions)' -# BUILD_NUMBER: 7 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER -# - name: Upload build URLs to artifacts -# uses: actions/upload-artifact@v2 -# with: -# name: ${{ env.BUILD_NAME }} -# path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -########################################################################################### -###################################### SPECIAL BUILDS ##################################### -########################################################################################### -# BuilderDebSplitted: -# needs: DockerHubPush -# if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} -# runs-on: [self-hosted, builder] -# steps: -# - name: Download changed images -# uses: actions/download-artifact@v2 -# with: -# name: changed_images -# path: ${{ runner.temp }}/images_path -# - name: Check out repository code -# uses: actions/checkout@v2 -# with: -# submodules: 'recursive' -# fetch-depth: 0 # otherwise we will have no info about contributors -# - name: Build -# env: -# TEMP_PATH: ${{runner.temp}}/build_check -# IMAGES_PATH: ${{runner.temp}}/images_path -# REPO_COPY: ${{runner.temp}}/build_check/ClickHouse -# CACHES_PATH: ${{runner.temp}}/../ccaches -# CHECK_NAME: 'ClickHouse special build check (actions)' -# BUILD_NUMBER: 1 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER -# - name: Upload build URLs to artifacts -# uses: actions/upload-artifact@v2 -# with: -# name: ${{ env.BUILD_NAME }} -# path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -############################################################################################# -###################################### BUILD REPORTER ####################################### -############################################################################################# -# BuilderReport: -# needs: -# - BuilderDebRelease -# - BuilderBinRelease -# - BuilderDebAsan -# - BuilderDebTsan -# - BuilderDebUBsan -# - BuilderDebMsan -# - BuilderDebDebug -# - BuilderDebSplitted -# runs-on: [self-hosted, style-checker] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Report Builder -# env: -# TEMP_PATH: ${{runner.temp}}/report_check -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'ClickHouse build check (actions)' -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cd $GITHUB_WORKSPACE/tests/ci -# python3 build_report_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -############################################################################################### -############################ FUNCTIONAl STATELESS TESTS ####################################### -############################################################################################### -# FunctionalStatelessTestRelease: -# needs: [BuilderDebRelease] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Functional test -# env: -# TEMP_PATH: ${{runner.temp}}/stateless_debug -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stateless tests (release, actions)' -# REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse -# KILL_TIMEOUT: 10800 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# FunctionalStatelessTestAsan: -# needs: [BuilderDebAsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Functional test -# env: -# TEMP_PATH: ${{runner.temp}}/stateless_debug -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stateless tests (address, actions)' -# REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse -# KILL_TIMEOUT: 10800 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# FunctionalStatelessTestTsan: -# needs: [BuilderDebTsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Functional test -# env: -# TEMP_PATH: ${{runner.temp}}/stateless_tsan -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stateless tests (thread, actions)' -# REPO_COPY: ${{runner.temp}}/stateless_tsan/ClickHouse -# KILL_TIMEOUT: 10800 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# FunctionalStatelessTestUBsan: -# needs: [BuilderDebUBsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Functional test -# env: -# TEMP_PATH: ${{runner.temp}}/stateless_ubsan -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stateless tests (ubsan, actions)' -# REPO_COPY: ${{runner.temp}}/stateless_ubsan/ClickHouse -# KILL_TIMEOUT: 10800 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# FunctionalStatelessTestMsan: -# needs: [BuilderDebMsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Functional test -# env: -# TEMP_PATH: ${{runner.temp}}/stateless_memory -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stateless tests (memory, actions)' -# REPO_COPY: ${{runner.temp}}/stateless_memory/ClickHouse -# KILL_TIMEOUT: 10800 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# FunctionalStatelessTestDebug: -# needs: [BuilderDebDebug] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Functional test -# env: -# TEMP_PATH: ${{runner.temp}}/stateless_debug -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stateless tests (debug, actions)' -# REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse -# KILL_TIMEOUT: 10800 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# FunctionalStatelessTestFlakyCheck: -# needs: [BuilderDebAsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Functional test -# env: -# TEMP_PATH: ${{runner.temp}}/stateless_flaky_asan -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stateless tests flaky check (address, actions)' -# REPO_COPY: ${{runner.temp}}/stateless_flaky_asan/ClickHouse -# KILL_TIMEOUT: 3600 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -############################################################################################### -############################# FUNCTIONAl STATEFUL TESTS ####################################### -############################################################################################### -# FunctionalStatefulTestRelease: -# needs: [BuilderDebRelease] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Functional test -# env: -# TEMP_PATH: ${{runner.temp}}/stateful_debug -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stateful tests (release, actions)' -# REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse -# KILL_TIMEOUT: 3600 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# FunctionalStatefulTestAsan: -# needs: [BuilderDebAsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Functional test -# env: -# TEMP_PATH: ${{runner.temp}}/stateful_debug -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stateful tests (address, actions)' -# REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse -# KILL_TIMEOUT: 3600 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# FunctionalStatefulTestTsan: -# needs: [BuilderDebTsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Functional test -# env: -# TEMP_PATH: ${{runner.temp}}/stateful_tsan -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stateful tests (thread, actions)' -# REPO_COPY: ${{runner.temp}}/stateful_tsan/ClickHouse -# KILL_TIMEOUT: 3600 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# FunctionalStatefulTestMsan: -# needs: [BuilderDebMsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Functional test -# env: -# TEMP_PATH: ${{runner.temp}}/stateful_msan -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stateful tests (memory, actions)' -# REPO_COPY: ${{runner.temp}}/stateful_msan/ClickHouse -# KILL_TIMEOUT: 3600 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# FunctionalStatefulTestUBsan: -# needs: [BuilderDebUBsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Functional test -# env: -# TEMP_PATH: ${{runner.temp}}/stateful_ubsan -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stateful tests (ubsan, actions)' -# REPO_COPY: ${{runner.temp}}/stateful_ubsan/ClickHouse -# KILL_TIMEOUT: 3600 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# FunctionalStatefulTestDebug: -# needs: [BuilderDebDebug] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Functional test -# env: -# TEMP_PATH: ${{runner.temp}}/stateful_debug -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stateful tests (debug, actions)' -# REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse -# KILL_TIMEOUT: 3600 -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -############################################################################################### -########################################## STRESS TESTS ####################################### -############################################################################################### -# StressTestAsan: -# needs: [BuilderDebAsan] -# runs-on: [self-hosted, stress-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Stress test -# env: -# TEMP_PATH: ${{runner.temp}}/stress_thread -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stress test (address, actions)' -# REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 stress_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# StressTestTsan: -# needs: [BuilderDebTsan] -# runs-on: [self-hosted, stress-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Stress test -# env: -# TEMP_PATH: ${{runner.temp}}/stress_thread -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stress test (thread, actions)' -# REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 stress_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# StressTestMsan: -# needs: [BuilderDebMsan] -# runs-on: [self-hosted, stress-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Stress test -# env: -# TEMP_PATH: ${{runner.temp}}/stress_memory -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stress test (memory, actions)' -# REPO_COPY: ${{runner.temp}}/stress_memory/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 stress_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# StressTestUBsan: -# needs: [BuilderDebUBsan] -# runs-on: [self-hosted, stress-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Stress test -# env: -# TEMP_PATH: ${{runner.temp}}/stress_undefined -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stress test (undefined, actions)' -# REPO_COPY: ${{runner.temp}}/stress_undefined/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 stress_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# StressTestDebug: -# needs: [BuilderDebDebug] -# runs-on: [self-hosted, stress-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Stress test -# env: -# TEMP_PATH: ${{runner.temp}}/stress_debug -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Stress test (debug, actions)' -# REPO_COPY: ${{runner.temp}}/stress_debug/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 stress_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -############################################################################################### -###################################### AST FUZZERS ############################################ -############################################################################################### -# ASTFuzzerTestAsan: -# needs: [BuilderDebAsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Fuzzer -# env: -# TEMP_PATH: ${{runner.temp}}/ast_fuzzer_asan -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'AST fuzzer (ASan, actions)' -# REPO_COPY: ${{runner.temp}}/ast_fuzzer_asan/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 ast_fuzzer_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# ASTFuzzerTestTsan: -# needs: [BuilderDebTsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Fuzzer -# env: -# TEMP_PATH: ${{runner.temp}}/ast_fuzzer_tsan -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'AST fuzzer (TSan, actions)' -# REPO_COPY: ${{runner.temp}}/ast_fuzzer_tsan/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 ast_fuzzer_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# ASTFuzzerTestUBSan: -# needs: [BuilderDebUBsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Fuzzer -# env: -# TEMP_PATH: ${{runner.temp}}/ast_fuzzer_ubsan -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'AST fuzzer (UBSan, actions)' -# REPO_COPY: ${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 ast_fuzzer_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# ASTFuzzerTestMSan: -# needs: [BuilderDebMsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Fuzzer -# env: -# TEMP_PATH: ${{runner.temp}}/ast_fuzzer_msan -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'AST fuzzer (MSan, actions)' -# REPO_COPY: ${{runner.temp}}/ast_fuzzer_msan/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 ast_fuzzer_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# ASTFuzzerTestDebug: -# needs: [BuilderDebDebug] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Fuzzer -# env: -# TEMP_PATH: ${{runner.temp}}/ast_fuzzer_debug -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'AST fuzzer (debug, actions)' -# REPO_COPY: ${{runner.temp}}/ast_fuzzer_debug/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 ast_fuzzer_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH + BuilderDebUBsan: + needs: DockerHubPush + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 4 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + BuilderDebTsan: + needs: DockerHubPush + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 5 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + BuilderDebMsan: + needs: DockerHubPush + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 6 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + BuilderDebDebug: + needs: DockerHubPush + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 7 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +########################################################################################## +##################################### SPECIAL BUILDS ##################################### +########################################################################################## + BuilderDebSplitted: + needs: DockerHubPush + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse special build check (actions)' + BUILD_NUMBER: 1 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################ +##################################### BUILD REPORTER ####################################### +############################################################################################ + BuilderReport: + needs: + - BuilderDebRelease + - BuilderBinRelease + - BuilderDebAsan + - BuilderDebTsan + - BuilderDebUBsan + - BuilderDebMsan + - BuilderDebDebug + - BuilderDebSplitted + runs-on: [self-hosted, style-checker] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Report Builder + env: + TEMP_PATH: ${{runner.temp}}/report_check + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'ClickHouse build check (actions)' + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cd $GITHUB_WORKSPACE/tests/ci + python3 build_report_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH ############################################################################################## -############################## INTEGRATION TESTS ############################################# +########################### FUNCTIONAl STATELESS TESTS ####################################### ############################################################################################## + FunctionalStatelessTestRelease: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (release, actions)' + REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (address, actions)' + REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_tsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (thread, actions)' + REPO_COPY: ${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_ubsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (ubsan, actions)' + REPO_COPY: ${{runner.temp}}/stateless_ubsan/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_memory + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (memory, actions)' + REPO_COPY: ${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (debug, actions)' + REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestFlakyCheck: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_flaky_asan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests flaky check (address, actions)' + REPO_COPY: ${{runner.temp}}/stateless_flaky_asan/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################## +############################ FUNCTIONAl STATEFUL TESTS ####################################### +############################################################################################## + FunctionalStatefulTestRelease: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (release, actions)' + REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatefulTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (address, actions)' + REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatefulTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_tsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (thread, actions)' + REPO_COPY: ${{runner.temp}}/stateful_tsan/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatefulTestMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_msan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (memory, actions)' + REPO_COPY: ${{runner.temp}}/stateful_msan/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatefulTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_ubsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (ubsan, actions)' + REPO_COPY: ${{runner.temp}}/stateful_ubsan/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatefulTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (debug, actions)' + REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################## +######################################### STRESS TESTS ####################################### +############################################################################################## + StressTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + env: + TEMP_PATH: ${{runner.temp}}/stress_thread + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress test (address, actions)' + REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + StressTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + env: + TEMP_PATH: ${{runner.temp}}/stress_thread + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress test (thread, actions)' + REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + StressTestMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + env: + TEMP_PATH: ${{runner.temp}}/stress_memory + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress test (memory, actions)' + REPO_COPY: ${{runner.temp}}/stress_memory/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + StressTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + env: + TEMP_PATH: ${{runner.temp}}/stress_undefined + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress test (undefined, actions)' + REPO_COPY: ${{runner.temp}}/stress_undefined/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + StressTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + env: + TEMP_PATH: ${{runner.temp}}/stress_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress test (debug, actions)' + REPO_COPY: ${{runner.temp}}/stress_debug/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################## +##################################### AST FUZZERS ############################################ +############################################################################################## + ASTFuzzerTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + env: + TEMP_PATH: ${{runner.temp}}/ast_fuzzer_asan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'AST fuzzer (ASan, actions)' + REPO_COPY: ${{runner.temp}}/ast_fuzzer_asan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + ASTFuzzerTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + env: + TEMP_PATH: ${{runner.temp}}/ast_fuzzer_tsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'AST fuzzer (TSan, actions)' + REPO_COPY: ${{runner.temp}}/ast_fuzzer_tsan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + ASTFuzzerTestUBSan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + env: + TEMP_PATH: ${{runner.temp}}/ast_fuzzer_ubsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'AST fuzzer (UBSan, actions)' + REPO_COPY: ${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + ASTFuzzerTestMSan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + env: + TEMP_PATH: ${{runner.temp}}/ast_fuzzer_msan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'AST fuzzer (MSan, actions)' + REPO_COPY: ${{runner.temp}}/ast_fuzzer_msan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + ASTFuzzerTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + env: + TEMP_PATH: ${{runner.temp}}/ast_fuzzer_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'AST fuzzer (debug, actions)' + REPO_COPY: ${{runner.temp}}/ast_fuzzer_debug/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################# +############################# INTEGRATION TESTS ############################################# +############################################################################################# IntegrationTestsAsan: needs: [BuilderDebAsan] runs-on: [self-hosted, stress-tester] @@ -1241,192 +1241,251 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH -############################################################################################## -##################################### UNIT TESTS ############################################# -############################################################################################## -# UnitTestsAsan: -# needs: [BuilderDebAsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Unit test -# env: -# TEMP_PATH: ${{runner.temp}}/unit_tests_asan -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Unit tests (asan, actions)' -# REPO_COPY: ${{runner.temp}}/unit_tests_asan/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 unit_tests_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# UnitTestsReleaseClang: -# needs: [BuilderBinRelease] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Unit test -# env: -# TEMP_PATH: ${{runner.temp}}/unit_tests_asan -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Unit tests (release-clang, actions)' -# REPO_COPY: ${{runner.temp}}/unit_tests_asan/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 unit_tests_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# UnitTestsTsan: -# needs: [BuilderDebTsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Unit test -# env: -# TEMP_PATH: ${{runner.temp}}/unit_tests_tsan -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Unit tests (tsan, actions)' -# REPO_COPY: ${{runner.temp}}/unit_tests_tsan/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 unit_tests_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# UnitTestsMsan: -# needs: [BuilderDebMsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Unit test -# env: -# TEMP_PATH: ${{runner.temp}}/unit_tests_msan -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Unit tests (msan, actions)' -# REPO_COPY: ${{runner.temp}}/unit_tests_msan/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 unit_tests_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# UnitTestsUBsan: -# needs: [BuilderDebUBsan] -# runs-on: [self-hosted, func-tester] -# steps: -# - name: Download json reports -# uses: actions/download-artifact@v2 -# with: -# path: ${{runner.temp}}/reports_dir -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Unit test -# env: -# TEMP_PATH: ${{runner.temp}}/unit_tests_ubsan -# REPORTS_PATH: ${{runner.temp}}/reports_dir -# CHECK_NAME: 'Unit tests (msan, actions)' -# REPO_COPY: ${{runner.temp}}/unit_tests_ubsan/ClickHouse -# run: | -# sudo rm -fr $TEMP_PATH -# mkdir -p $TEMP_PATH -# cp -r $GITHUB_WORKSPACE $TEMP_PATH -# cd $REPO_COPY/tests/ci -# python3 unit_tests_check.py "$CHECK_NAME" -# - name: Cleanup -# if: always() -# run: | -# docker kill $(docker ps -q) ||: -# docker rm -f $(docker ps -a -q) ||: -# sudo rm -fr $TEMP_PATH -# FinishCheck: -# needs: -# - StyleCheck -# - DockerHubPush -# - CheckLabels -# - BuilderReport -# - FastTest -# - FunctionalStatelessTestDebug -# - FunctionalStatelessTestRelease -# - FunctionalStatelessTestAsan -# - FunctionalStatelessTestTsan -# - FunctionalStatelessTestMsan -# - FunctionalStatelessTestUBsan -# - FunctionalStatefulTestDebug -# - FunctionalStatefulTestRelease -# - FunctionalStatefulTestAsan -# - FunctionalStatefulTestTsan -# - FunctionalStatefulTestMsan -# - FunctionalStatefulTestUBsan -# - DocsCheck -# - StressTestDebug -# - StressTestAsan -# - StressTestTsan -# - StressTestMsan -# - StressTestUBsan -# - ASTFuzzerTestDebug -# - ASTFuzzerTestAsan -# - ASTFuzzerTestTsan -# - ASTFuzzerTestMSan -# - ASTFuzzerTestUBSan -# - PVSCheck -# - UnitTestsAsan -# - UnitTestsTsan -# - UnitTestsMsan -# - UnitTestsUBsan -# - UnitTestsReleaseClang -# - SplitBuildSmokeTest -# - CompatibilityCheck -# runs-on: [self-hosted, style-checker] -# steps: -# - name: Check out repository code -# uses: actions/checkout@v2 -# - name: Finish label -# run: | -# cd $GITHUB_WORKSPACE/tests/ci -# python3 finish_check.py + IntegrationTestsTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + env: + TEMP_PATH: ${{runner.temp}}/integration_tests_tsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Integration tests (thread, actions)' + REPO_COPY: ${{runner.temp}}/integration_tests_tsan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + IntegrationTestsRelease: + needs: [BuilderDebRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + env: + TEMP_PATH: ${{runner.temp}}/integration_tests_release + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Integration tests (release, actions)' + REPO_COPY: ${{runner.temp}}/integration_tests_release/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################# +#################################### UNIT TESTS ############################################# +############################################################################################# + UnitTestsAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + env: + TEMP_PATH: ${{runner.temp}}/unit_tests_asan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Unit tests (asan, actions)' + REPO_COPY: ${{runner.temp}}/unit_tests_asan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + UnitTestsReleaseClang: + needs: [BuilderBinRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + env: + TEMP_PATH: ${{runner.temp}}/unit_tests_asan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Unit tests (release-clang, actions)' + REPO_COPY: ${{runner.temp}}/unit_tests_asan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + UnitTestsTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + env: + TEMP_PATH: ${{runner.temp}}/unit_tests_tsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Unit tests (tsan, actions)' + REPO_COPY: ${{runner.temp}}/unit_tests_tsan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + UnitTestsMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + env: + TEMP_PATH: ${{runner.temp}}/unit_tests_msan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Unit tests (msan, actions)' + REPO_COPY: ${{runner.temp}}/unit_tests_msan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + UnitTestsUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + env: + TEMP_PATH: ${{runner.temp}}/unit_tests_ubsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Unit tests (msan, actions)' + REPO_COPY: ${{runner.temp}}/unit_tests_ubsan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FinishCheck: + needs: + - StyleCheck + - DockerHubPush + - CheckLabels + - BuilderReport + - FastTest + - FunctionalStatelessTestDebug + - FunctionalStatelessTestRelease + - FunctionalStatelessTestAsan + - FunctionalStatelessTestTsan + - FunctionalStatelessTestMsan + - FunctionalStatelessTestUBsan + - FunctionalStatefulTestDebug + - FunctionalStatefulTestRelease + - FunctionalStatefulTestAsan + - FunctionalStatefulTestTsan + - FunctionalStatefulTestMsan + - FunctionalStatefulTestUBsan + - DocsCheck + - StressTestDebug + - StressTestAsan + - StressTestTsan + - StressTestMsan + - StressTestUBsan + - ASTFuzzerTestDebug + - ASTFuzzerTestAsan + - ASTFuzzerTestTsan + - ASTFuzzerTestMSan + - ASTFuzzerTestUBSan + - IntegrationTestsAsan + - IntegrationTestsRelease + - IntegrationTestsTsan + - PVSCheck + - UnitTestsAsan + - UnitTestsTsan + - UnitTestsMsan + - UnitTestsUBsan + - UnitTestsReleaseClang + - SplitBuildSmokeTest + - CompatibilityCheck + runs-on: [self-hosted, style-checker] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Finish label + run: | + cd $GITHUB_WORKSPACE/tests/ci + python3 finish_check.py From eafd3ddfac1c6e6632c8070d498ccb74388819c1 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 28 Oct 2021 16:24:00 +0300 Subject: [PATCH 103/200] Add lz4 stream compression --- base/daemon/BaseDaemon.cpp | 3 ++ base/loggers/Loggers.cpp | 2 ++ contrib/poco-cmake/Foundation/CMakeLists.txt | 3 +- docker/test/integration/base/Dockerfile | 1 + .../test_log_lz4_streaming/__init__.py | 0 .../test_log_lz4_streaming/configs/logs.xml | 5 ++++ .../test_log_lz4_streaming/test.py | 30 +++++++++++++++++++ 7 files changed, 43 insertions(+), 1 deletion(-) create mode 100644 tests/integration/test_log_lz4_streaming/__init__.py create mode 100644 tests/integration/test_log_lz4_streaming/configs/logs.xml create mode 100644 tests/integration/test_log_lz4_streaming/test.py diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index b3f6f05143d..6d58b96fb2f 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -37,6 +37,7 @@ #include #include #include +#include #include #include @@ -1038,6 +1039,8 @@ void BaseDaemon::setupWatchdog() throw Poco::Exception("Cannot waitpid, errno: " + std::string(strerror(errno))); } while (true); + Poco::LogFile::INDEPENDENT_STREAM_COMPRESS = true; + if (errno == ECHILD) { logger().information("Child process no longer exists."); diff --git a/base/loggers/Loggers.cpp b/base/loggers/Loggers.cpp index 0f41296819e..ac44605ef1d 100644 --- a/base/loggers/Loggers.cpp +++ b/base/loggers/Loggers.cpp @@ -75,6 +75,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M")); log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true")); + log_file->setProperty(Poco::FileChannel::PROP_STREAMCOMPRESS, config.getRawString("logger.stream_compress", "false")); log_file->setProperty(Poco::FileChannel::PROP_PURGECOUNT, config.getRawString("logger.count", "1")); log_file->setProperty(Poco::FileChannel::PROP_FLUSH, config.getRawString("logger.flush", "true")); log_file->setProperty(Poco::FileChannel::PROP_ROTATEONOPEN, config.getRawString("logger.rotateOnOpen", "false")); @@ -107,6 +108,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log error_log_file->setProperty(Poco::FileChannel::PROP_ROTATION, config.getRawString("logger.size", "100M")); error_log_file->setProperty(Poco::FileChannel::PROP_ARCHIVE, "number"); error_log_file->setProperty(Poco::FileChannel::PROP_COMPRESS, config.getRawString("logger.compress", "true")); + error_log_file->setProperty(Poco::FileChannel::PROP_STREAMCOMPRESS, config.getRawString("logger.stream_compress", "false")); error_log_file->setProperty(Poco::FileChannel::PROP_PURGECOUNT, config.getRawString("logger.count", "1")); error_log_file->setProperty(Poco::FileChannel::PROP_FLUSH, config.getRawString("logger.flush", "true")); error_log_file->setProperty(Poco::FileChannel::PROP_ROTATEONOPEN, config.getRawString("logger.rotateOnOpen", "false")); diff --git a/contrib/poco-cmake/Foundation/CMakeLists.txt b/contrib/poco-cmake/Foundation/CMakeLists.txt index a9a4933873c..0c13d109344 100644 --- a/contrib/poco-cmake/Foundation/CMakeLists.txt +++ b/contrib/poco-cmake/Foundation/CMakeLists.txt @@ -51,6 +51,7 @@ if (USE_INTERNAL_POCO_LIBRARY) "${LIBRARY_DIR}/Foundation/src/Channel.cpp" "${LIBRARY_DIR}/Foundation/src/Checksum.cpp" "${LIBRARY_DIR}/Foundation/src/Clock.cpp" + "${LIBRARY_DIR}/Foundation/src/CompressedLogFile.cpp" "${LIBRARY_DIR}/Foundation/src/Condition.cpp" "${LIBRARY_DIR}/Foundation/src/Configurable.cpp" "${LIBRARY_DIR}/Foundation/src/ConsoleChannel.cpp" @@ -222,7 +223,7 @@ if (USE_INTERNAL_POCO_LIBRARY) POCO_OS_FAMILY_UNIX ) target_include_directories (_poco_foundation SYSTEM PUBLIC "${LIBRARY_DIR}/Foundation/include") - target_link_libraries (_poco_foundation PRIVATE Poco::Foundation::PCRE ${ZLIB_LIBRARIES}) + target_link_libraries (_poco_foundation PRIVATE Poco::Foundation::PCRE ${ZLIB_LIBRARIES} lz4) else () add_library (Poco::Foundation UNKNOWN IMPORTED GLOBAL) diff --git a/docker/test/integration/base/Dockerfile b/docker/test/integration/base/Dockerfile index 519c64297e5..add4dad0132 100644 --- a/docker/test/integration/base/Dockerfile +++ b/docker/test/integration/base/Dockerfile @@ -19,6 +19,7 @@ RUN apt-get update \ sqlite3 \ curl \ tar \ + lz4 \ krb5-user \ iproute2 \ lsof \ diff --git a/tests/integration/test_log_lz4_streaming/__init__.py b/tests/integration/test_log_lz4_streaming/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_log_lz4_streaming/configs/logs.xml b/tests/integration/test_log_lz4_streaming/configs/logs.xml new file mode 100644 index 00000000000..43a43cd231b --- /dev/null +++ b/tests/integration/test_log_lz4_streaming/configs/logs.xml @@ -0,0 +1,5 @@ + + + true + + diff --git a/tests/integration/test_log_lz4_streaming/test.py b/tests/integration/test_log_lz4_streaming/test.py new file mode 100644 index 00000000000..0d0e722914b --- /dev/null +++ b/tests/integration/test_log_lz4_streaming/test.py @@ -0,0 +1,30 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node = cluster.add_instance('node', main_configs=['configs/logs.xml'], stay_alive=True) + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + yield cluster + + finally: + cluster.shutdown() + + +def test_log_lz4_streaming(started_cluster): + + node.stop_clickhouse() + node.start_clickhouse() + node.stop_clickhouse() + + lz4_output = node.exec_in_container(["bash", "-c", "lz4 -t /var/log/clickhouse-server/clickhouse-server.log.lz4 2>&1"]) + assert lz4_output.find('Error') == -1, lz4_output + + compressed_size = int(node.exec_in_container(["bash", "-c", "du -b /var/log/clickhouse-server/clickhouse-server.log.lz4 | awk {' print $1 '}"])) + uncompressed_size = int(lz4_output.split()[3]) + assert 0 < compressed_size < uncompressed_size, lz4_output From b3b061eeb97aabe3a20dd914563b7ce055cafa4b Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 28 Oct 2021 16:25:56 +0300 Subject: [PATCH 104/200] Update Poco --- contrib/poco | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/poco b/contrib/poco index 173fb317178..1a78aa0772f 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 173fb31717837d366152c508619b09dcf11786da +Subproject commit 1a78aa0772f36b46e813965bf78704ac888133dc From 2ae79233e4727e1043cab7676fce25c8b031d7f5 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 10 Nov 2021 21:10:03 +0000 Subject: [PATCH 105/200] Better --- base/daemon/BaseDaemon.cpp | 15 +++++++-- contrib/poco | 2 +- .../test_log_lz4_streaming/test.py | 32 +++++++++++++++---- 3 files changed, 38 insertions(+), 11 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 6d58b96fb2f..61a65f947fc 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -37,7 +37,7 @@ #include #include #include -#include +#include #include #include @@ -64,6 +64,9 @@ #include #include +#include +#include + #include #if defined(OS_DARWIN) @@ -1002,6 +1005,14 @@ void BaseDaemon::setupWatchdog() memcpy(argv0, new_process_name, std::min(strlen(new_process_name), original_process_name.size())); } + /// If streaming compression of logs is used then we write watchdog logs to cerr + if (config().getRawString("logger.stream_compress", "false") == "true") + { + Poco::AutoPtr pf = new OwnPatternFormatter; + Poco::AutoPtr log = new DB::OwnFormattingChannel(pf, new Poco::ConsoleChannel(std::cerr)); + logger().setChannel(log); + } + logger().information(fmt::format("Will watch for the process with pid {}", pid)); /// Forward signals to the child process. @@ -1039,8 +1050,6 @@ void BaseDaemon::setupWatchdog() throw Poco::Exception("Cannot waitpid, errno: " + std::string(strerror(errno))); } while (true); - Poco::LogFile::INDEPENDENT_STREAM_COMPRESS = true; - if (errno == ECHILD) { logger().information("Child process no longer exists."); diff --git a/contrib/poco b/contrib/poco index 1a78aa0772f..eddd17c7271 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit 1a78aa0772f36b46e813965bf78704ac888133dc +Subproject commit eddd17c7271db0e286a46d8f9b27e9451f3d3dd6 diff --git a/tests/integration/test_log_lz4_streaming/test.py b/tests/integration/test_log_lz4_streaming/test.py index 0d0e722914b..7572c181623 100644 --- a/tests/integration/test_log_lz4_streaming/test.py +++ b/tests/integration/test_log_lz4_streaming/test.py @@ -1,4 +1,5 @@ import pytest +import time from helpers.cluster import ClickHouseCluster @@ -13,18 +14,35 @@ def started_cluster(): yield cluster finally: - cluster.shutdown() + # It will print Fatal message after pkill -SEGV, suppress it + try: + cluster.shutdown() + except: + pass -def test_log_lz4_streaming(started_cluster): +def check_log_file(): + assert node.file_exists("/var/log/clickhouse-server/clickhouse-server.log.lz4") + lz4_output = node.exec_in_container(["bash", "-c", "lz4 -t /var/log/clickhouse-server/clickhouse-server.log.lz4 2>&1"], user='root') + assert lz4_output.count('Error') == 0, lz4_output + + compressed_size = int(node.exec_in_container(["bash", "-c", "du -b /var/log/clickhouse-server/clickhouse-server.log.lz4 | awk {' print $1 '}"], user='root')) + uncompressed_size = int(lz4_output.split()[3]) + assert 0 < compressed_size < uncompressed_size, lz4_output + + +def test_concatenation(started_cluster): node.stop_clickhouse() node.start_clickhouse() node.stop_clickhouse() - lz4_output = node.exec_in_container(["bash", "-c", "lz4 -t /var/log/clickhouse-server/clickhouse-server.log.lz4 2>&1"]) - assert lz4_output.find('Error') == -1, lz4_output + check_log_file() - compressed_size = int(node.exec_in_container(["bash", "-c", "du -b /var/log/clickhouse-server/clickhouse-server.log.lz4 | awk {' print $1 '}"])) - uncompressed_size = int(lz4_output.split()[3]) - assert 0 < compressed_size < uncompressed_size, lz4_output + +def test_incomplete_rotation(started_cluster): + node.stop_clickhouse(kill=True) + node.start_clickhouse() + node.stop_clickhouse() + + check_log_file() From ddef6d86ca950319baf2beb6a9f3830ff7c52be9 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Mon, 15 Nov 2021 11:27:04 +0000 Subject: [PATCH 106/200] Minor fixes --- base/daemon/BaseDaemon.cpp | 1 - base/loggers/Loggers.cpp | 14 ++++++++++++-- contrib/poco | 2 +- tests/integration/test_log_lz4_streaming/test.py | 6 +----- 4 files changed, 14 insertions(+), 9 deletions(-) diff --git a/base/daemon/BaseDaemon.cpp b/base/daemon/BaseDaemon.cpp index 61a65f947fc..2b703f7fa3a 100644 --- a/base/daemon/BaseDaemon.cpp +++ b/base/daemon/BaseDaemon.cpp @@ -37,7 +37,6 @@ #include #include #include -#include #include #include diff --git a/base/loggers/Loggers.cpp b/base/loggers/Loggers.cpp index ac44605ef1d..5eb9ef95176 100644 --- a/base/loggers/Loggers.cpp +++ b/base/loggers/Loggers.cpp @@ -62,7 +62,13 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log if (!log_path.empty()) { createDirectory(log_path); - std::cerr << "Logging " << log_level_string << " to " << log_path << std::endl; + + std::string ext; + if (config.getRawString("logger.stream_compress", "false") == "true") + ext = ".lz4"; + + std::cerr << "Logging " << log_level_string << " to " << log_path << ext << std::endl; + auto log_level = Poco::Logger::parseLevel(log_level_string); if (log_level > max_log_level) { @@ -101,7 +107,11 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log max_log_level = errorlog_level; } - std::cerr << "Logging errors to " << errorlog_path << std::endl; + std::string ext; + if (config.getRawString("logger.stream_compress", "false") == "true") + ext = ".lz4"; + + std::cerr << "Logging errors to " << errorlog_path << ext << std::endl; error_log_file = new Poco::FileChannel; error_log_file->setProperty(Poco::FileChannel::PROP_PATH, fs::weakly_canonical(errorlog_path)); diff --git a/contrib/poco b/contrib/poco index eddd17c7271..258b9ba6cd2 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit eddd17c7271db0e286a46d8f9b27e9451f3d3dd6 +Subproject commit 258b9ba6cd245ff88e9346f75c43464c403f329d diff --git a/tests/integration/test_log_lz4_streaming/test.py b/tests/integration/test_log_lz4_streaming/test.py index 7572c181623..7f2f22f28c9 100644 --- a/tests/integration/test_log_lz4_streaming/test.py +++ b/tests/integration/test_log_lz4_streaming/test.py @@ -14,11 +14,7 @@ def started_cluster(): yield cluster finally: - # It will print Fatal message after pkill -SEGV, suppress it - try: - cluster.shutdown() - except: - pass + cluster.shutdown() def check_log_file(): From 2d2ffc18546b87b78ac574e816a9911c295e69d5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 17 Nov 2021 10:34:06 +0300 Subject: [PATCH 107/200] Introduce some dependencies --- .github/workflows/main.yml | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/.github/workflows/main.yml b/.github/workflows/main.yml index 836a2dd07a3..daa05e19798 100644 --- a/.github/workflows/main.yml +++ b/.github/workflows/main.yml @@ -190,7 +190,7 @@ jobs: #################################### ORDINARY BUILDS #################################### ######################################################################################### BuilderDebRelease: - needs: DockerHubPush + needs: [DockerHubPush, FastTest] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: @@ -229,7 +229,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH BuilderBinRelease: - needs: DockerHubPush + needs: [DockerHubPush, FastTest] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: @@ -268,7 +268,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH BuilderDebAsan: - needs: DockerHubPush + needs: [DockerHubPush, FastTest] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: @@ -307,7 +307,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH BuilderDebUBsan: - needs: DockerHubPush + needs: [DockerHubPush, FastTest] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: @@ -346,7 +346,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH BuilderDebTsan: - needs: DockerHubPush + needs: [DockerHubPush, FastTest] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: @@ -385,7 +385,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH BuilderDebMsan: - needs: DockerHubPush + needs: [DockerHubPush, FastTest] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: @@ -424,7 +424,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH BuilderDebDebug: - needs: DockerHubPush + needs: [DockerHubPush, FastTest] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: @@ -466,7 +466,7 @@ jobs: ##################################### SPECIAL BUILDS ##################################### ########################################################################################## BuilderDebSplitted: - needs: DockerHubPush + needs: [DockerHubPush, FastTest] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: @@ -1214,7 +1214,7 @@ jobs: ############################# INTEGRATION TESTS ############################################# ############################################################################################# IntegrationTestsAsan: - needs: [BuilderDebAsan] + needs: [BuilderDebAsan, FunctionalStatelessTestAsan] runs-on: [self-hosted, stress-tester] steps: - name: Download json reports @@ -1242,7 +1242,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH IntegrationTestsTsan: - needs: [BuilderDebTsan] + needs: [BuilderDebTsan, FunctionalStatelessTestTsan] runs-on: [self-hosted, stress-tester] steps: - name: Download json reports @@ -1270,7 +1270,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH IntegrationTestsRelease: - needs: [BuilderDebRelease] + needs: [BuilderDebRelease, FunctionalStatelessTestRelease] runs-on: [self-hosted, stress-tester] steps: - name: Download json reports From 3fa846b944eeed250a3f63ffc642fa3942b43d84 Mon Sep 17 00:00:00 2001 From: jus1096 <40656180+jus1096@users.noreply.github.com> Date: Wed, 17 Nov 2021 12:47:01 +0300 Subject: [PATCH 108/200] Fix link Manipulating Partitions and Parts Correct link Manipulating Partitions and Parts --- .../mergetree-family/custom-partitioning-key.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/ru/engines/table-engines/mergetree-family/custom-partitioning-key.md b/docs/ru/engines/table-engines/mergetree-family/custom-partitioning-key.md index 4f0206158f1..8508ba18d9e 100644 --- a/docs/ru/engines/table-engines/mergetree-family/custom-partitioning-key.md +++ b/docs/ru/engines/table-engines/mergetree-family/custom-partitioning-key.md @@ -64,9 +64,9 @@ WHERE table = 'visits' └───────────┴────────────────┴────────┘ ``` -Столбец `partition` содержит имена всех партиций таблицы. Таблица `visits` из нашего примера содержит две партиции: `201901` и `201902`. Используйте значения из этого столбца в запросах [ALTER … PARTITION](#alter_manipulations-with-partitions). +Столбец `partition` содержит имена всех партиций таблицы. Таблица `visits` из нашего примера содержит две партиции: `201901` и `201902`. Используйте значения из этого столбца в запросах [ALTER … PARTITION](../../../sql-reference/statements/alter/partition.md). -Столбец `name` содержит названия кусков партиций. Значения из этого столбца можно использовать в запросах [ALTER ATTACH PART](#alter_attach-partition). +Столбец `name` содержит названия кусков партиций. Значения из этого столбца можно использовать в запросах [ALTER ATTACH PART](../../../sql-reference/statements/alter/partition.md#alter_attach-partition). Столбец `active` отображает состояние куска. `1` означает, что кусок активен; `0` – неактивен. К неактивным можно отнести куски, оставшиеся после слияния данных. Поврежденные куски также отображаются как неактивные. Неактивные куски удаляются приблизительно через 10 минут после того, как было выполнено слияние. @@ -82,7 +82,7 @@ WHERE table = 'visits' Как видно из примера выше, таблица содержит несколько отдельных кусков для одной и той же партиции (например, куски `201901_1_3_1` и `201901_1_9_2` принадлежат партиции `201901`). Это означает, что эти куски еще не были объединены – в файловой системе они хранятся отдельно. После того как будет выполнено автоматическое слияние данных (выполняется примерно спустя 10 минут после вставки данных), исходные куски будут объединены в один более крупный кусок и помечены как неактивные. -Вы можете запустить внеочередное слияние данных с помощью запроса [OPTIMIZE](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md#misc_operations-optimize). Пример: +Вы можете запустить внеочередное слияние данных с помощью запроса [OPTIMIZE](../../../sql-reference/statements/optimize.md). Пример: ``` sql OPTIMIZE TABLE visits PARTITION 201902; @@ -123,11 +123,11 @@ drwxr-xr-x 2 clickhouse clickhouse 4096 Feb 1 16:48 detached Директория `detached` содержит куски, отсоединенные от таблицы с помощью запроса [DETACH](../../../sql-reference/statements/alter/partition.md#alter_detach-partition). Поврежденные куски также попадают в эту директорию – они не удаляются с сервера. -Сервер не использует куски из директории `detached`. Вы можете в любое время добавлять, удалять, модифицировать данные в директории detached - сервер не будет об этом знать, пока вы не сделаете запрос [ATTACH](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md#alter_attach-partition). +Сервер не использует куски из директории `detached`. Вы можете в любое время добавлять, удалять, модифицировать данные в директории detached - сервер не будет об этом знать, пока вы не сделаете запрос [ATTACH](../../../sql-reference/statements/alter/partition.md#alter_attach-partition). Следует иметь в виду, что при работающем сервере нельзя вручную изменять набор кусков на файловой системе, так как сервер не будет знать об этом. Для нереплицируемых таблиц, вы можете это делать при остановленном сервере, однако это не рекомендуется. Для реплицируемых таблиц, набор кусков нельзя менять в любом случае. -ClickHouse позволяет производить различные манипуляции с кусками: удалять, копировать из одной таблицы в другую или создавать их резервные копии. Подробнее см. в разделе [Манипуляции с партициями и кусками](../../../engines/table-engines/mergetree-family/custom-partitioning-key.md#alter_manipulations-with-partitions). +ClickHouse позволяет производить различные манипуляции с кусками: удалять, копировать из одной таблицы в другую или создавать их резервные копии. Подробнее см. в разделе [Манипуляции с партициями и кусками](../../../sql-reference/statements/alter/partition.md). From 09d189292c617d6a5fa87d66c26d5670d9cab654 Mon Sep 17 00:00:00 2001 From: jus1096 <40656180+jus1096@users.noreply.github.com> Date: Wed, 17 Nov 2021 13:00:04 +0300 Subject: [PATCH 109/200] Fix link to dictGet functions Correct link to dictGet functions --- docs/ru/engines/table-engines/special/dictionary.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/special/dictionary.md b/docs/ru/engines/table-engines/special/dictionary.md index 15d32419472..05a78a24e8f 100644 --- a/docs/ru/engines/table-engines/special/dictionary.md +++ b/docs/ru/engines/table-engines/special/dictionary.md @@ -62,7 +62,7 @@ WHERE name = 'products' └──────────┴──────┴────────┴─────────────────┴─────────────────┴─────────────────┴───────────────┴─────────────────┘ ``` -В таком виде данные из словаря можно получить при помощи функций [dictGet\*](../../../engines/table-engines/special/dictionary.md#ext_dict_functions). +В таком виде данные из словаря можно получить при помощи функций [dictGet\*](../../../sql-reference/functions/ext-dict-functions.md#dictget). Такое представление неудобно, когда нам необходимо получить данные в чистом виде, а также при выполнении операции `JOIN`. Для этих случаев можно использовать движок `Dictionary`, который отобразит данные словаря в таблицу. From 1de670254606a02ad2d2dffd8a9e881e37f244bd Mon Sep 17 00:00:00 2001 From: jus1096 <40656180+jus1096@users.noreply.github.com> Date: Wed, 17 Nov 2021 13:11:24 +0300 Subject: [PATCH 110/200] Fix link to conf and attach Correct link to conf and attach --- docs/ru/engines/table-engines/special/file.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/engines/table-engines/special/file.md b/docs/ru/engines/table-engines/special/file.md index 6f1c723d2a7..fe93e814335 100644 --- a/docs/ru/engines/table-engines/special/file.md +++ b/docs/ru/engines/table-engines/special/file.md @@ -21,11 +21,11 @@ File(Format) `Format` должен быть таким, который ClickHouse может использовать и в запросах `INSERT` и в запросах `SELECT`. Полный список поддерживаемых форматов смотрите в разделе [Форматы](../../../interfaces/formats.md#formats). -Сервер ClickHouse не позволяет указать путь к файлу, с которым будет работать `File`. Используется путь к хранилищу, определенный параметром [path](../../../operations/server-configuration-parameters/settings.md) в конфигурации сервера. +Сервер ClickHouse не позволяет указать путь к файлу, с которым будет работать `File`. Используется путь к хранилищу, определенный параметром [path](../../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-path) в конфигурации сервера. При создании таблицы с помощью `File(Format)` сервер ClickHouse создает в хранилище каталог с именем таблицы, а после добавления в таблицу данных помещает туда файл `data.Format`. -Можно вручную создать в хранилище каталог таблицы, поместить туда файл, затем на сервере ClickHouse добавить ([ATTACH](../../../engines/table-engines/special/file.md)) информацию о таблице, соответствующей имени каталога и прочитать из файла данные. +Можно вручную создать в хранилище каталог таблицы, поместить туда файл, затем на сервере ClickHouse добавить ([ATTACH](../../../sql-reference/statements/attach.md#attach)) информацию о таблице, соответствующей имени каталога и прочитать из файла данные. !!! warning "Warning" Будьте аккуратны с этой функциональностью, поскольку сервер ClickHouse не отслеживает внешние изменения данных. Если в файл будет производиться запись одновременно со стороны сервера ClickHouse и с внешней стороны, то результат непредсказуем. From 2d20cf3a7d6072245024094c39b6b4156a348e25 Mon Sep 17 00:00:00 2001 From: jus1096 <40656180+jus1096@users.noreply.github.com> Date: Wed, 17 Nov 2021 13:24:42 +0300 Subject: [PATCH 111/200] Fix link to join clause Correct link to join clause --- docs/ru/engines/table-engines/special/join.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/ru/engines/table-engines/special/join.md b/docs/ru/engines/table-engines/special/join.md index 28bbfe6dea4..cbb536ac321 100644 --- a/docs/ru/engines/table-engines/special/join.md +++ b/docs/ru/engines/table-engines/special/join.md @@ -5,7 +5,7 @@ toc_title: Join # Join {#join} -Подготовленная структура данных для использования в операциях [JOIN](../../../engines/table-engines/special/join.md#select-join). +Подготовленная структура данных для использования в операциях [JOIN](../../../sql-reference/statements/select/join.md#select-join). ## Создание таблицы {#creating-a-table} @@ -21,8 +21,8 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] **Параметры движка** -- `join_strictness` – [строгость JOIN](../../../engines/table-engines/special/join.md#select-join-types). -- `join_type` – [тип JOIN](../../../engines/table-engines/special/join.md#select-join-types). +- `join_strictness` – [строгость JOIN](../../../sql-reference/statements/select/join.md#select-join-types). +- `join_type` – [тип JOIN](../../../sql-reference/statements/select/join.md#select-join-types). - `k1[, k2, ...]` – ключевые столбцы секции `USING` с которыми выполняется операция `JOIN`. Вводите параметры `join_strictness` и `join_type` без кавычек, например, `Join(ANY, LEFT, col1)`. Они должны быть такими же как и в той операции `JOIN`, в которой таблица будет использоваться. Если параметры не совпадают, ClickHouse не генерирует исключение и может возвращать неверные данные. @@ -42,7 +42,7 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] Основные применения `Join` таблиц: - Использование в правой части секции `JOIN`. -- Извлечение данных из таблицы таким же образом как из словаря с помощью функции [joinGet](../../../engines/table-engines/special/join.md#joinget). +- Извлечение данных из таблицы таким же образом как из словаря с помощью функции [joinGet](../../../sql-reference/functions/other-functions.md#joinget). ### Удаление данных {#deleting-data} From 15a8100be2ab6294572ca7d1faf104e4ee57034a Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Wed, 17 Nov 2021 12:08:57 +0000 Subject: [PATCH 112/200] Try to fix test --- .../test_backward_compatibility/test_aggregate_fixed_key.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py index 98658ec81d0..fc8d27cfa16 100644 --- a/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py +++ b/tests/integration/test_backward_compatibility/test_aggregate_fixed_key.py @@ -4,8 +4,8 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__, name="aggregate_fixed_key") node1 = cluster.add_instance('node1', with_zookeeper=True, image='yandex/clickhouse-server', tag='21.3', with_installed_binary=True) -node2 = cluster.add_instance('node2', with_zookeeper=True, image='yandex/clickhouse-server') -node3 = cluster.add_instance('node3', with_zookeeper=True, image='yandex/clickhouse-server') +node2 = cluster.add_instance('node2', with_zookeeper=True) +node3 = cluster.add_instance('node3', with_zookeeper=True) @pytest.fixture(scope="module") From 81044e6eefa72c5f425401fda42d6abdca83e5b7 Mon Sep 17 00:00:00 2001 From: jus1096 <40656180+jus1096@users.noreply.github.com> Date: Wed, 17 Nov 2021 15:30:26 +0300 Subject: [PATCH 113/200] fix --- docs/ru/sql-reference/table-functions/mysql.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/table-functions/mysql.md b/docs/ru/sql-reference/table-functions/mysql.md index 7a4cc9c234c..5ba7e8f1f0f 100644 --- a/docs/ru/sql-reference/table-functions/mysql.md +++ b/docs/ru/sql-reference/table-functions/mysql.md @@ -107,5 +107,5 @@ SELECT * FROM mysql('localhost:3306', 'test', 'test', 'bayonet', '123'); **Смотрите также** - [Движок таблиц ‘MySQL’](../../sql-reference/table-functions/mysql.md) -- [Использование MySQL как источника данных для внешнего словаря](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md##dicts-external_dicts_dict_sources-mysql) +- [Использование MySQL как источника данных для внешнего словаря](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md#dicts-external_dicts_dict_sources-mysql) From 7b9a19f06692e14c84b2b872bf1dc5e47598586d Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 17 Nov 2021 22:05:57 +0800 Subject: [PATCH 114/200] enhance minmax_count projection --- src/Storages/MergeTree/MergeTreeData.cpp | 21 +++++++++------- .../MergeTree/registerStorageMergeTree.cpp | 8 +++--- src/Storages/ProjectionsDescription.cpp | 25 +++++++++++++++---- src/Storages/ProjectionsDescription.h | 6 ++++- .../01710_minmax_count_projection.reference | 1 + .../01710_minmax_count_projection.sql | 7 ++++++ 6 files changed, 49 insertions(+), 19 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c5c14cd24a8..f94ea2511c0 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4430,7 +4430,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( required_columns.begin(), required_columns.end(), [&](const auto & name) { return primary_key_max_column_name == name; }); } - auto minmax_count_columns = block.mutateColumns(); + auto partition_minmax_count_columns = block.mutateColumns(); auto insert = [](ColumnAggregateFunction & column, const Field & value) { auto func = column.getAggregateFunction(); @@ -4475,11 +4475,15 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( } size_t pos = 0; + size_t partition_columns_size = part->partition.value.size(); + for (size_t i = 0; i < partition_columns_size; ++i) + partition_minmax_count_columns[pos++]->insert(part->partition.value[i]); + size_t minmax_idx_size = part->minmax_idx->hyperrectangle.size(); for (size_t i = 0; i < minmax_idx_size; ++i) { - auto & min_column = assert_cast(*minmax_count_columns[pos++]); - auto & max_column = assert_cast(*minmax_count_columns[pos++]); + auto & min_column = assert_cast(*partition_minmax_count_columns[pos++]); + auto & max_column = assert_cast(*partition_minmax_count_columns[pos++]); const auto & range = part->minmax_idx->hyperrectangle[i]; insert(min_column, range.left); insert(max_column, range.right); @@ -4488,15 +4492,14 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( if (!primary_key_max_column_name.empty()) { const auto & primary_key_column = *part->index[0]; - auto primary_key_column_size = primary_key_column.size(); - auto & min_column = assert_cast(*minmax_count_columns[pos++]); - auto & max_column = assert_cast(*minmax_count_columns[pos++]); + auto & min_column = assert_cast(*partition_minmax_count_columns[pos++]); + auto & max_column = assert_cast(*partition_minmax_count_columns[pos++]); insert(min_column, primary_key_column[0]); - insert(max_column, primary_key_column[primary_key_column_size - 1]); + insert(max_column, primary_key_column[primary_key_column.size() - 1]); } { - auto & column = assert_cast(*minmax_count_columns.back()); + auto & column = assert_cast(*partition_minmax_count_columns.back()); auto func = column.getAggregateFunction(); Arena & arena = column.createOrGetArena(); size_t size_of_state = func->sizeOfData(); @@ -4508,7 +4511,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( column.insertFrom(place); } } - block.setColumns(std::move(minmax_count_columns)); + block.setColumns(std::move(partition_minmax_count_columns)); Block res; for (const auto & name : required_columns) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 982acfe62a4..fe4485ac222 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -684,8 +684,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) auto minmax_columns = metadata.getColumnsRequiredForPartitionKey(); auto primary_key_asts = metadata.primary_key.expression_list_ast->children; - metadata.minmax_count_projection.emplace( - ProjectionDescription::getMinMaxCountProjection(args.columns, minmax_columns, primary_key_asts, args.getContext())); + metadata.minmax_count_projection.emplace(ProjectionDescription::getMinMaxCountProjection( + args.columns, metadata.partition_key.expression_list_ast, minmax_columns, primary_key_asts, args.getContext())); if (args.storage_def->sample_by) metadata.sampling_key = KeyDescription::getKeyFromAST(args.storage_def->sample_by->ptr(), metadata.columns, args.getContext()); @@ -764,8 +764,8 @@ static StoragePtr create(const StorageFactory::Arguments & args) auto minmax_columns = metadata.getColumnsRequiredForPartitionKey(); auto primary_key_asts = metadata.primary_key.expression_list_ast->children; - metadata.minmax_count_projection.emplace( - ProjectionDescription::getMinMaxCountProjection(args.columns, minmax_columns, primary_key_asts, args.getContext())); + metadata.minmax_count_projection.emplace(ProjectionDescription::getMinMaxCountProjection( + args.columns, metadata.partition_key.expression_list_ast, minmax_columns, primary_key_asts, args.getContext())); const auto * ast = engine_args[arg_num]->as(); if (ast && ast->value.getType() == Field::Types::UInt64) diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index dc0598de72f..323b832dca3 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -175,6 +175,7 @@ ProjectionDescription::getProjectionFromAST(const ASTPtr & definition_ast, const ProjectionDescription ProjectionDescription::getMinMaxCountProjection( const ColumnsDescription & columns, + const ASTPtr & partition_columns, const Names & minmax_columns, const ASTs & primary_key_asts, ContextPtr query_context) @@ -197,6 +198,9 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( select_expression_list->children.push_back(makeASTFunction("count")); select_query->setExpression(ASTProjectionSelectQuery::Expression::SELECT, std::move(select_expression_list)); + if (partition_columns) + select_query->setExpression(ASTProjectionSelectQuery::Expression::GROUP_BY, partition_columns->clone()); + result.definition_ast = select_query; result.name = MINMAX_COUNT_PROJECTION_NAME; result.query_ast = select_query->cloneToASTSelect(); @@ -207,12 +211,23 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( result.query_ast, query_context, storage, {}, SelectQueryOptions{QueryProcessingStage::WithMergeableState}.modify().ignoreAlias()); result.required_columns = select.getRequiredColumns(); result.sample_block = select.getSampleBlock(); - /// If we have primary key and it's not in minmax_columns, it will be used as one additional minmax columns. - if (!primary_key_asts.empty() && result.sample_block.columns() == 2 * (minmax_columns.size() + 1) + 1) + + const auto & analysis_result = select.getAnalysisResult(); + if (analysis_result.need_aggregate) { - /// min(p1), max(p1), min(p2), max(p2), ..., min(k1), max(k1), count() - /// ^ - /// size - 2 + for (const auto & key : select.getQueryAnalyzer()->aggregationKeys()) + result.sample_block_for_keys.insert({nullptr, key.type, key.name}); + } + + /// If we have primary key and it's not in minmax_columns, it will be used as one additional minmax columns. + if (!primary_key_asts.empty() + && result.sample_block.columns() + == 2 * (minmax_columns.size() + 1) /* minmax columns */ + 1 /* count() */ + + result.sample_block_for_keys.columns() /* partition_columns */) + { + /// partition_expr1, partition_expr2, ..., min(p1), max(p1), min(p2), max(p2), ..., min(k1), max(k1), count() + /// ^ + /// size - 2 result.primary_key_max_column_name = *(result.sample_block.getNames().cend() - 2); } result.type = ProjectionDescription::Type::Aggregate; diff --git a/src/Storages/ProjectionsDescription.h b/src/Storages/ProjectionsDescription.h index 7c254182ba4..e2b2e58a244 100644 --- a/src/Storages/ProjectionsDescription.h +++ b/src/Storages/ProjectionsDescription.h @@ -66,7 +66,11 @@ struct ProjectionDescription getProjectionFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr query_context); static ProjectionDescription getMinMaxCountProjection( - const ColumnsDescription & columns, const Names & minmax_columns, const ASTs & primary_key_asts, ContextPtr query_context); + const ColumnsDescription & columns, + const ASTPtr & partition_columns, + const Names & minmax_columns, + const ASTs & primary_key_asts, + ContextPtr query_context); ProjectionDescription() = default; diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.reference b/tests/queries/0_stateless/01710_minmax_count_projection.reference index ed78b475842..36f9f68c0bc 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.reference +++ b/tests/queries/0_stateless/01710_minmax_count_projection.reference @@ -8,3 +8,4 @@ 0 9999 1 9999 3 +2021-10-25 10:00:00 2021-10-27 10:00:00 3 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index 0138ad8c57e..5764f956803 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -43,3 +43,10 @@ drop table if exists t; create table t (server_date Date, something String) engine MergeTree partition by (toYYYYMM(server_date), server_date) order by (server_date, something); insert into t values ('2019-01-01', 'test1'), ('2019-02-01', 'test2'), ('2019-03-01', 'test3'); select count() from t; +drop table t; + +drop table if exists d; +create table d (dt DateTime, j int) engine MergeTree partition by toDate(dt) order by tuple(); +insert into d values ('2021-10-24 10:00:00', 10), ('2021-10-25 10:00:00', 10), ('2021-10-26 10:00:00', 10), ('2021-10-27 10:00:00', 10); +select min(dt), max(dt), count() from d where toDate(dt) >= '2021-10-25'; +drop table d; From 4cfb105d5a448835f5a2a4370ffd1f84ab3af0ff Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 17 Nov 2021 22:07:56 +0800 Subject: [PATCH 115/200] better --- src/Storages/MergeTree/MergeTreeData.cpp | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index f94ea2511c0..ebf8de7d245 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4394,8 +4394,13 @@ static void selectBestProjection( if (normal_result_ptr->error()) return; - sum_marks += normal_result_ptr->marks(); - candidate.merge_tree_normal_select_result_ptr = normal_result_ptr; + if (normal_result_ptr->marks() == 0) + candidate.complete = true; + else + { + sum_marks += normal_result_ptr->marks(); + candidate.merge_tree_normal_select_result_ptr = normal_result_ptr; + } } candidate.merge_tree_projection_select_result_ptr = projection_result_ptr; From 4b332aea2156121d7954e2b20f6a2a3cc5867c28 Mon Sep 17 00:00:00 2001 From: avogar Date: Wed, 17 Nov 2021 17:54:15 +0300 Subject: [PATCH 116/200] Temporary test fix --- tests/queries/0_stateless/01059_storage_file_compression.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01059_storage_file_compression.sh b/tests/queries/0_stateless/01059_storage_file_compression.sh index fbee5070d8d..8942113ab12 100755 --- a/tests/queries/0_stateless/01059_storage_file_compression.sh +++ b/tests/queries/0_stateless/01059_storage_file_compression.sh @@ -12,6 +12,7 @@ do ${CLICKHOUSE_CLIENT} --query "CREATE TABLE file (x UInt64) ENGINE = File(TSV, '${CLICKHOUSE_DATABASE}/${m}.tsv.${m}')" ${CLICKHOUSE_CLIENT} --query "TRUNCATE TABLE file" ${CLICKHOUSE_CLIENT} --query "INSERT INTO file SELECT * FROM numbers(1000000)" + sleep 1 ${CLICKHOUSE_CLIENT} --query "SELECT count(), max(x) FROM file" ${CLICKHOUSE_CLIENT} --query "DROP TABLE file" done From feb3a12d342683ed8811154c654f8f271ea3ef29 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 17 Nov 2021 23:11:23 +0800 Subject: [PATCH 117/200] fix --- src/Storages/MergeTree/MergeTreeData.cpp | 7 +++++-- src/Storages/ProjectionsDescription.cpp | 19 +++++++++++++++++-- src/Storages/ProjectionsDescription.h | 6 ++++++ .../01710_minmax_count_projection.sql | 2 +- 4 files changed, 29 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index ebf8de7d245..247fa1f4dd2 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4480,9 +4480,12 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( } size_t pos = 0; - size_t partition_columns_size = part->partition.value.size(); - for (size_t i = 0; i < partition_columns_size; ++i) + for(size_t i : metadata_snapshot->minmax_count_projection->partition_value_indices) + { + if (i >= part->partition.value.size()) + throw Exception("Partition value index is out of boundary. It's a bug", ErrorCodes::LOGICAL_ERROR); partition_minmax_count_columns[pos++]->insert(part->partition.value[i]); + } size_t minmax_idx_size = part->minmax_idx->hyperrectangle.size(); for (size_t i = 0; i < minmax_idx_size; ++i) diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 323b832dca3..c7d242476a1 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -15,6 +15,7 @@ #include #include #include +#include namespace DB @@ -61,6 +62,7 @@ ProjectionDescription ProjectionDescription::clone() const other.key_size = key_size; other.is_minmax_count_projection = is_minmax_count_projection; other.primary_key_max_column_name = primary_key_max_column_name; + other.partition_value_indices = partition_value_indices; return other; } @@ -212,18 +214,31 @@ ProjectionDescription ProjectionDescription::getMinMaxCountProjection( result.required_columns = select.getRequiredColumns(); result.sample_block = select.getSampleBlock(); + std::map partition_column_name_to_value_index; + if (partition_columns) + { + for (auto i : collections::range(partition_columns->children.size())) + partition_column_name_to_value_index[partition_columns->children[i]->getColumnNameWithoutAlias()] = i; + } + const auto & analysis_result = select.getAnalysisResult(); if (analysis_result.need_aggregate) { for (const auto & key : select.getQueryAnalyzer()->aggregationKeys()) + { result.sample_block_for_keys.insert({nullptr, key.type, key.name}); + auto it = partition_column_name_to_value_index.find(key.name); + if (it == partition_column_name_to_value_index.end()) + throw Exception("minmax_count projection can only have keys about partition columns. It's a bug", ErrorCodes::LOGICAL_ERROR); + result.partition_value_indices.push_back(it->second); + } } /// If we have primary key and it's not in minmax_columns, it will be used as one additional minmax columns. if (!primary_key_asts.empty() && result.sample_block.columns() == 2 * (minmax_columns.size() + 1) /* minmax columns */ + 1 /* count() */ - + result.sample_block_for_keys.columns() /* partition_columns */) + + result.partition_value_indices.size() /* partition_columns */) { /// partition_expr1, partition_expr2, ..., min(p1), max(p1), min(p2), max(p2), ..., min(k1), max(k1), count() /// ^ @@ -265,7 +280,7 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) Block ret; executor.pull(ret); if (executor.pull(ret)) - throw Exception("Projection cannot increase the number of rows in a block", ErrorCodes::LOGICAL_ERROR); + throw Exception("Projection cannot increase the number of rows in a block. It's a bug", ErrorCodes::LOGICAL_ERROR); return ret; } diff --git a/src/Storages/ProjectionsDescription.h b/src/Storages/ProjectionsDescription.h index e2b2e58a244..960e94e22f4 100644 --- a/src/Storages/ProjectionsDescription.h +++ b/src/Storages/ProjectionsDescription.h @@ -61,6 +61,12 @@ struct ProjectionDescription /// If a primary key expression is used in the minmax_count projection, store the name of max expression. String primary_key_max_column_name; + /// Stores partition value indices of partition value row. It's needed because identical + /// partition columns will appear only once in projection block, but every column will have a + /// value in the partition value row. This vector holds the biggest value index of give + /// partition columns. + std::vector partition_value_indices; + /// Parse projection from definition AST static ProjectionDescription getProjectionFromAST(const ASTPtr & definition_ast, const ColumnsDescription & columns, ContextPtr query_context); diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index 5764f956803..2b53cdc0bee 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -46,7 +46,7 @@ select count() from t; drop table t; drop table if exists d; -create table d (dt DateTime, j int) engine MergeTree partition by toDate(dt) order by tuple(); +create table d (dt DateTime, j int) engine MergeTree partition by (toDate(dt), ceiling(j), toDate(dt), CEILING(j)) order by tuple(); insert into d values ('2021-10-24 10:00:00', 10), ('2021-10-25 10:00:00', 10), ('2021-10-26 10:00:00', 10), ('2021-10-27 10:00:00', 10); select min(dt), max(dt), count() from d where toDate(dt) >= '2021-10-25'; drop table d; From dfb9bc2a016edabf20b529ac18fa383351d03007 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 10 Nov 2021 20:57:59 +0300 Subject: [PATCH 118/200] minor fixes in constraints --- .../AddIndexConstraintsOptimizer.cpp | 37 +- .../AddIndexConstraintsOptimizer.h | 4 + src/Interpreters/ComparisonGraph.cpp | 388 ++++++++++-------- src/Interpreters/ComparisonGraph.h | 78 +++- src/Interpreters/InterpreterCreateQuery.cpp | 9 +- src/Interpreters/InterpreterSelectQuery.cpp | 2 - .../SubstituteColumnOptimizer.cpp | 94 ++--- .../SubstituteColumnOptimizer.h | 2 + src/Interpreters/TreeCNFConverter.cpp | 25 +- src/Interpreters/TreeCNFConverter.h | 2 +- src/Interpreters/TreeOptimizer.cpp | 4 +- .../WhereConstraintsOptimizer.cpp | 64 ++- src/Interpreters/WhereConstraintsOptimizer.h | 11 +- .../tests/gtest_comparison_graph.cpp | 183 +++++++++ src/Parsers/ASTConstraintDeclaration.h | 2 +- src/Storages/AlterCommands.cpp | 9 +- src/Storages/ConstraintsDescription.cpp | 34 +- src/Storages/ConstraintsDescription.h | 13 +- .../MergeTreeIndexMergedCondition.cpp | 21 +- .../MergeTree/MergeTreeWhereOptimizer.cpp | 2 - .../MergeTree/registerStorageMergeTree.cpp | 2 +- ..._constraints_simple_optimization.reference | 1 + .../01622_constraints_simple_optimization.sql | 3 +- ...2_constraints_where_optimization.reference | 14 + .../01622_constraints_where_optimization.sql | 23 ++ .../01623_constraints_column_swap.reference | 2 + .../01623_constraints_column_swap.sql | 10 + 27 files changed, 665 insertions(+), 374 deletions(-) rename src/{Storages/MergeTree => Interpreters}/SubstituteColumnOptimizer.cpp (80%) rename src/{Storages/MergeTree => Interpreters}/SubstituteColumnOptimizer.h (82%) create mode 100644 src/Interpreters/tests/gtest_comparison_graph.cpp create mode 100644 tests/queries/0_stateless/01622_constraints_where_optimization.reference create mode 100644 tests/queries/0_stateless/01622_constraints_where_optimization.sql diff --git a/src/Interpreters/AddIndexConstraintsOptimizer.cpp b/src/Interpreters/AddIndexConstraintsOptimizer.cpp index c3151ead34e..b28e31d22f6 100644 --- a/src/Interpreters/AddIndexConstraintsOptimizer.cpp +++ b/src/Interpreters/AddIndexConstraintsOptimizer.cpp @@ -1,14 +1,12 @@ #include #include -#include #include #include #include #include #include #include -#include namespace DB { @@ -21,18 +19,6 @@ AddIndexConstraintsOptimizer::AddIndexConstraintsOptimizer( namespace { - - bool hasIndexColumns(const ASTPtr & ast, const std::unordered_set & primary_key_set) - { - const auto * identifier = ast->as(); - if (identifier && primary_key_set.contains(identifier->name())) - return true; - for (auto & child : ast->children) - if (hasIndexColumns(child, primary_key_set)) - return true; - return false; - } - bool onlyIndexColumns(const ASTPtr & ast, const std::unordered_set & primary_key_set) { const auto * identifier = ast->as(); @@ -57,7 +43,8 @@ namespace const std::unordered_map & getRelationMap() { - const static std::unordered_map relations = { + const static std::unordered_map relations = + { {"equals", ComparisonGraph::CompareResult::EQUAL}, {"less", ComparisonGraph::CompareResult::LESS}, {"lessOrEquals", ComparisonGraph::CompareResult::LESS_OR_EQUAL}, @@ -69,7 +56,8 @@ namespace const std::unordered_map & getReverseRelationMap() { - const static std::unordered_map relations = { + const static std::unordered_map relations = + { {ComparisonGraph::CompareResult::EQUAL, "equals"}, {ComparisonGraph::CompareResult::LESS, "less"}, {ComparisonGraph::CompareResult::LESS_OR_EQUAL, "lessOrEquals"}, @@ -109,7 +97,11 @@ namespace return CR::UNKNOWN; } - /// Create OR-group for index_hint + /// Create OR-group for 'indexHint'. + /// Consider we have expression like A C, where C is constant. + /// Consider we have a constraint I A, where I depends only on columns from primary key. + /// Then if op1 and op2 forms a sequence of comparisons (e.g. A < C and I < A), + /// we can add to expression 'indexHint(I < A)' condition. CNFQuery::OrGroup createIndexHintGroup( const CNFQuery::OrGroup & group, const ComparisonGraph & graph, @@ -121,7 +113,7 @@ namespace const auto * func = atom.ast->as(); if (func && func->arguments->children.size() == 2 && getRelationMap().contains(func->name)) { - auto check_and_insert = [&](const size_t index, const ComparisonGraph::CompareResult need_result) -> bool + auto check_and_insert = [&](const size_t index, const ComparisonGraph::CompareResult need_result) { if (!onlyConstants(func->arguments->children[1 - index])) return false; @@ -144,10 +136,12 @@ namespace return true; } } + return false; }; - if (!check_and_insert(1, getRelationMap().at(func->name)) && !check_and_insert(0, getRelationMap().at(func->name))) + auto expected = getRelationMap().at(func->name); + if (!check_and_insert(0, expected) && !check_and_insert(1, expected)) return {}; } } @@ -165,16 +159,17 @@ void AddIndexConstraintsOptimizer::perform(CNFQuery & cnf_query) ASTs primary_key_only_asts; for (const auto & vertex : graph.getVertices()) for (const auto & ast : vertex) - if (hasIndexColumns(ast, primary_key_set) && onlyIndexColumns(ast, primary_key_set)) + if (onlyIndexColumns(ast, primary_key_set)) primary_key_only_asts.push_back(ast); CNFQuery::AndGroup and_group; - cnf_query.iterateGroups([&and_group, &graph, &primary_key_only_asts](const auto & or_group) + cnf_query.iterateGroups([&](const auto & or_group) { auto add_group = createIndexHintGroup(or_group, graph, primary_key_only_asts); if (!add_group.empty()) and_group.emplace(std::move(add_group)); }); + if (!and_group.empty()) { CNFQuery::OrGroup new_or_group; diff --git a/src/Interpreters/AddIndexConstraintsOptimizer.h b/src/Interpreters/AddIndexConstraintsOptimizer.h index 3ec7b2f3594..228d8d8ad1a 100644 --- a/src/Interpreters/AddIndexConstraintsOptimizer.h +++ b/src/Interpreters/AddIndexConstraintsOptimizer.h @@ -16,6 +16,10 @@ namespace DB struct StorageInMemoryMetadata; using StorageMetadataPtr = std::shared_ptr; +/// Optimizer that extracts constraints that +/// depends only on columns of primary key +/// and tries to add function 'indexHint' to +/// WHERE clause, which reduces amount of read data. class AddIndexConstraintsOptimizer final { public: diff --git a/src/Interpreters/ComparisonGraph.cpp b/src/Interpreters/ComparisonGraph.cpp index 7abb8f8f001..e236de67fdc 100644 --- a/src/Interpreters/ComparisonGraph.cpp +++ b/src/Interpreters/ComparisonGraph.cpp @@ -2,56 +2,70 @@ #include #include #include - -#include -#include -#include -#include +#include +#include namespace DB { -/// make function a < b or a <= b -ASTPtr ComparisonGraph::normalizeAtom(const ASTPtr & atom) +namespace ErrorCodes { - static const std::map inverse_relations = { - {"greaterOrEquals", "lessOrEquals"}, - {"greater", "less"}, + extern const int VIOLATED_CONSTRAINT; +} + +namespace +{ + +/// Make function a > b or a >= b +ASTPtr normalizeAtom(const ASTPtr & atom) +{ + static const std::map inverse_relations = + { + {"lessOrEquals", "greaterOrEquals"}, + {"less", "greater"}, }; ASTPtr res = atom->clone(); + if (const auto * func = res->as()) { - auto * func = res->as(); - if (func) + if (const auto it = inverse_relations.find(func->name); it != std::end(inverse_relations)) { - if (const auto it = inverse_relations.find(func->name); it != std::end(inverse_relations)) - { - res = makeASTFunction(it->second, func->arguments->children[1]->clone(), func->arguments->children[0]->clone()); - } + res = makeASTFunction(it->second, func->arguments->children[1]->clone(), func->arguments->children[0]->clone()); } } return res; } -ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) +bool less(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess{}, lhs, rhs); } +bool greater(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess{}, rhs, lhs); } +bool equals(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateEquals{}, lhs, rhs); } + +} + +ComparisonGraph::ComparisonGraph(const ASTs & atomic_formulas) { if (atomic_formulas.empty()) return; + static const std::unordered_map relation_to_enum = { - {"equals", Edge::Type::EQUAL}, - {"less", Edge::Type::LESS}, - {"lessOrEquals", Edge::Type::LESS_OR_EQUAL}, + {"equals", Edge::EQUAL}, + {"greater", Edge::GREATER}, + {"greaterOrEquals", Edge::GREATER_OR_EQUAL}, }; + /// Firstly build an intermediate graph, + /// in which each vertex corresponds to one expression. + /// That means that if we have edge (A, B) with type GREATER, then always A > B. + /// If we have EQUAL relation, then we add both edges (A, B) and (B, A). + Graph g; for (const auto & atom_raw : atomic_formulas) { - const auto atom = ComparisonGraph::normalizeAtom(atom_raw); + const auto atom = normalizeAtom(atom_raw); - const auto bad_term = std::numeric_limits::max(); - auto get_index = [](const ASTPtr & ast, Graph & asts_graph) -> std::size_t + auto get_index = [](const ASTPtr & ast, Graph & asts_graph) -> std::optional { const auto it = asts_graph.ast_hash_to_component.find(ast->getTreeHash()); if (it != std::end(asts_graph.ast_hash_to_component)) @@ -65,7 +79,7 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) && constraint_ast->getColumnName() == ast->getColumnName(); })) { - return bad_term; + return {}; } return it->second; @@ -73,7 +87,7 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) else { asts_graph.ast_hash_to_component[ast->getTreeHash()] = asts_graph.vertices.size(); - asts_graph.vertices.push_back(EqualComponent{{ast}}); + asts_graph.vertices.push_back(EqualComponent{{ast}, std::nullopt}); asts_graph.edges.emplace_back(); return asts_graph.vertices.size() - 1; } @@ -82,118 +96,135 @@ ComparisonGraph::ComparisonGraph(const std::vector & atomic_formulas) const auto * func = atom->as(); if (func && func->arguments->children.size() == 2) { - const size_t index_left = get_index(func->arguments->children[0], g); - const size_t index_right = get_index(func->arguments->children[1], g); + auto index_left = get_index(func->arguments->children[0], g); + auto index_right = get_index(func->arguments->children[1], g); - if (index_left != bad_term && index_right != bad_term) + if (index_left && index_right) { - if (const auto it = relation_to_enum.find(func->name); - it != std::end(relation_to_enum) && func->arguments->children.size() == 2) + if (const auto it = relation_to_enum.find(func->name); it != std::end(relation_to_enum)) { - g.edges[index_right].push_back(Edge{it->second, index_left}); - if (func->name == "equals") - { - g.edges[index_left].push_back(Edge{it->second, index_right}); - } - } - else if (func->name == "notEquals") - { - /// Do nothing. + g.edges[*index_left].push_back(Edge{it->second, *index_right}); + if (it->second == Edge::EQUAL) + g.edges[*index_right].push_back(Edge{it->second, *index_left}); } } } } - graph = ComparisonGraph::BuildGraphFromAstsGraph(g); - dists = ComparisonGraph::BuildDistsFromGraph(graph); + /// Now expressions A and B are equal, if and only if + /// we have both paths from A to B and from B to A in graph. + /// That means that equivalence classes of expressions + /// are the same as strongly connected components in graph. + /// So, we find such components and build graph on them. + /// All expressions from one equivalence class will be stored + /// in the corresponding vertex of new graph. + + graph = buildGraphFromAstsGraph(g); + dists = buildDistsFromGraph(graph); std::tie(ast_const_lower_bound, ast_const_upper_bound) = buildConstBounds(); + /// Find expressions that are known to be unequal. + static const std::unordered_set not_equals_functions = {"notEquals", "greater"}; + + /// Explicitly save unequal components. + /// TODO: Build a graph for unequal components. for (const auto & atom_raw : atomic_formulas) { - const auto atom = ComparisonGraph::normalizeAtom(atom_raw); - if (const auto * func = atom->as(); func) + const auto atom = normalizeAtom(atom_raw); + const auto * func = atom->as(); + + if (func && not_equals_functions.contains(func->name)) { auto index_left = graph.ast_hash_to_component.at(func->arguments->children[0]->getTreeHash()); auto index_right = graph.ast_hash_to_component.at(func->arguments->children[1]->getTreeHash()); + + if (index_left == index_right) + throw Exception(ErrorCodes::VIOLATED_CONSTRAINT, + "Found expression '{}', but its arguments considered equal according to constraints", + queryToString(atom)); + not_equal.emplace(index_left, index_right); not_equal.emplace(index_right, index_left); } } } -/// returns {is less, is strict} -/// {true, true} = < -/// {true, false} = =< -/// {false, ...} = ? -std::pair ComparisonGraph::findPath(const size_t start, const size_t finish) const + +ComparisonGraph::CompareResult ComparisonGraph::pathToCompareResult(Path path, bool inverse) +{ + switch (path) + { + case Path::GREATER: return inverse ? CompareResult::LESS : CompareResult::GREATER; + case Path::GREATER_OR_EQUAL: return inverse ? CompareResult::LESS_OR_EQUAL : CompareResult::GREATER_OR_EQUAL; + } + __builtin_unreachable(); +} + +std::optional ComparisonGraph::findPath(const size_t start, const size_t finish) const { const auto it = dists.find(std::make_pair(start, finish)); if (it == std::end(dists)) - return {false, false}; - else - { - return {true, it->second == Path::LESS || not_equal.contains({start, finish})}; - } + return {}; + + /// Since path can be only GREATER or GREATER_OR_EQUALS, + /// we can strengthen the condition. + return not_equal.contains({start, finish}) ? Path::GREATER : it->second; } ComparisonGraph::CompareResult ComparisonGraph::compare(const ASTPtr & left, const ASTPtr & right) const { size_t start = 0; size_t finish = 0; + + /// TODO: check full ast + const auto it_left = graph.ast_hash_to_component.find(left->getTreeHash()); + const auto it_right = graph.ast_hash_to_component.find(right->getTreeHash()); + + if (it_left == std::end(graph.ast_hash_to_component) || it_right == std::end(graph.ast_hash_to_component)) { - /// TODO: check full ast - const auto it_left = graph.ast_hash_to_component.find(left->getTreeHash()); - const auto it_right = graph.ast_hash_to_component.find(right->getTreeHash()); - if (it_left == std::end(graph.ast_hash_to_component) || it_right == std::end(graph.ast_hash_to_component)) + CompareResult result = CompareResult::UNKNOWN; { + const auto left_bound = getConstLowerBound(left); + const auto right_bound = getConstUpperBound(right); + + if (left_bound && right_bound) { - const auto left_bound = getConstLowerBound(left); - const auto right_bound = getConstUpperBound(right); - if (left_bound && right_bound) - { - if (left_bound->first < right_bound->first) - return CompareResult::UNKNOWN; - else if (left_bound->first > right_bound->first) - return CompareResult::GREATER; - else if (left_bound->second || right_bound->second) - return CompareResult::GREATER; - else - return CompareResult::GREATER_OR_EQUAL; - } + if (greater(left_bound->first, right_bound->first)) + result = CompareResult::GREATER; + else if (equals(left_bound->first, right_bound->first)) + result = left_bound->second || right_bound->second + ? CompareResult::GREATER : CompareResult::GREATER_OR_EQUAL; } - { - const auto left_bound = getConstUpperBound(left); - const auto right_bound = getConstLowerBound(right); - if (left_bound && right_bound) - { - if (left_bound->first > right_bound->first) - return CompareResult::UNKNOWN; - else if (left_bound->first < right_bound->first) - return CompareResult::LESS; - else if (left_bound->second || right_bound->second) - return CompareResult::LESS; - else - return CompareResult::LESS_OR_EQUAL; - } - } - return CompareResult::UNKNOWN; } - else { - start = it_left->second; - finish = it_right->second; + const auto left_bound = getConstUpperBound(left); + const auto right_bound = getConstLowerBound(right); + + if (left_bound && right_bound) + { + if (less(left_bound->first, right_bound->first)) + result = CompareResult::LESS; + else if (equals(left_bound->first, right_bound->first)) + result = left_bound->second || right_bound->second + ? CompareResult::LESS : CompareResult::LESS_OR_EQUAL; + } } + + return result; + } + else + { + start = it_left->second; + finish = it_right->second; } if (start == finish) return CompareResult::EQUAL; - const auto [has_path, is_strict] = findPath(start, finish); - if (has_path) - return is_strict ? CompareResult::GREATER : CompareResult::GREATER_OR_EQUAL; + if (auto path = findPath(start, finish)) + return pathToCompareResult(*path, /*inverse=*/ false); - const auto [has_path_reverse, is_strict_reverse] = findPath(finish, start); - if (has_path_reverse) - return is_strict_reverse ? CompareResult::LESS : CompareResult::LESS_OR_EQUAL; + if (auto path = findPath(finish, start)) + return pathToCompareResult(*path, /*inverse=*/ true); if (not_equal.contains({start, finish})) return CompareResult::NOT_EQUAL; @@ -206,13 +237,13 @@ bool ComparisonGraph::isPossibleCompare(const CompareResult expected, const ASTP const auto result = compare(left, right); if (expected == CompareResult::UNKNOWN || result == CompareResult::UNKNOWN) - { return true; - } + if (expected == result) return true; - static const std::set> possible_pairs = { + static const std::set> possible_pairs = + { {CompareResult::EQUAL, CompareResult::LESS_OR_EQUAL}, {CompareResult::EQUAL, CompareResult::GREATER_OR_EQUAL}, {CompareResult::LESS_OR_EQUAL, CompareResult::LESS}, @@ -242,10 +273,12 @@ bool ComparisonGraph::isAlwaysCompare(const CompareResult expected, const ASTPtr if (expected == CompareResult::UNKNOWN || result == CompareResult::UNKNOWN) return false; + if (expected == result) return true; - static const std::set> possible_pairs = { + static const std::set> possible_pairs = + { {CompareResult::LESS_OR_EQUAL, CompareResult::LESS}, {CompareResult::LESS_OR_EQUAL, CompareResult::EQUAL}, {CompareResult::GREATER_OR_EQUAL, CompareResult::GREATER}, @@ -258,7 +291,7 @@ bool ComparisonGraph::isAlwaysCompare(const CompareResult expected, const ASTPtr } -std::vector ComparisonGraph::getEqual(const ASTPtr & ast) const +ASTs ComparisonGraph::getEqual(const ASTPtr & ast) const { const auto res = getComponentId(ast); if (!res) @@ -267,11 +300,12 @@ std::vector ComparisonGraph::getEqual(const ASTPtr & ast) const return getComponent(res.value()); } -std::optional ComparisonGraph::getComponentId(const ASTPtr & ast) const +std::optional ComparisonGraph::getComponentId(const ASTPtr & ast) const { const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash()); if (hash_it == std::end(graph.ast_hash_to_component)) return {}; + const size_t index = hash_it->second; if (std::any_of( std::cbegin(graph.vertices[index].asts), @@ -292,27 +326,28 @@ std::optional ComparisonGraph::getComponentId(const ASTPtr & ast) c bool ComparisonGraph::hasPath(const size_t left, const size_t right) const { - return findPath(left, right).first || findPath(right, left).first; + return findPath(left, right) || findPath(right, left); } -std::vector ComparisonGraph::getComponent(const std::size_t id) const +ASTs ComparisonGraph::getComponent(const size_t id) const { return graph.vertices[id].asts; } bool ComparisonGraph::EqualComponent::hasConstant() const { - return constant_index != -1; + return constant_index.has_value(); } ASTPtr ComparisonGraph::EqualComponent::getConstant() const { - return asts[constant_index]; + assert(constant_index); + return asts[*constant_index]; } void ComparisonGraph::EqualComponent::buildConstants() { - constant_index = -1; + constant_index.reset(); for (size_t i = 0; i < asts.size(); ++i) { if (asts[i]->as()) @@ -323,7 +358,20 @@ void ComparisonGraph::EqualComponent::buildConstants() } } -ComparisonGraph::CompareResult ComparisonGraph::getCompareResult(const std::string & name) +ComparisonGraph::CompareResult ComparisonGraph::atomToCompareResult(const CNFQuery::AtomicFormula & atom) +{ + if (const auto * func = atom.ast->as()) + { + auto expected = functionNameToCompareResult(func->name); + if (atom.negative) + expected = inverseCompareResult(expected); + return expected; + } + + return ComparisonGraph::CompareResult::UNKNOWN; +} + +ComparisonGraph::CompareResult ComparisonGraph::functionNameToCompareResult(const std::string & name) { static const std::unordered_map relation_to_compare = { @@ -359,6 +407,7 @@ std::optional ComparisonGraph::getEqualConst(const ASTPtr & ast) const const auto hash_it = graph.ast_hash_to_component.find(ast->getTreeHash()); if (hash_it == std::end(graph.ast_hash_to_component)) return std::nullopt; + const size_t index = hash_it->second; return graph.vertices[index].hasConstant() ? std::optional{graph.vertices[index].getConstant()} @@ -367,48 +416,45 @@ std::optional ComparisonGraph::getEqualConst(const ASTPtr & ast) const std::optional> ComparisonGraph::getConstUpperBound(const ASTPtr & ast) const { - { - const auto * literal = ast->as(); - if (literal) - return std::make_pair(literal->value, false); - } + if (const auto * literal = ast->as()) + return std::make_pair(literal->value, false); + const auto it = graph.ast_hash_to_component.find(ast->getTreeHash()); if (it == std::end(graph.ast_hash_to_component)) return std::nullopt; + const size_t to = it->second; const ssize_t from = ast_const_upper_bound[to]; if (from == -1) return std::nullopt; - return std::make_pair(graph.vertices[from].getConstant()->as()->value, dists.at({from, to}) == Path::LESS); + + return std::make_pair(graph.vertices[from].getConstant()->as()->value, dists.at({from, to}) == Path::GREATER); } std::optional> ComparisonGraph::getConstLowerBound(const ASTPtr & ast) const { - { - const auto * literal = ast->as(); - if (literal) - return std::make_pair(literal->value, false); - } + if (const auto * literal = ast->as()) + return std::make_pair(literal->value, false); + const auto it = graph.ast_hash_to_component.find(ast->getTreeHash()); if (it == std::end(graph.ast_hash_to_component)) return std::nullopt; + const size_t from = it->second; const ssize_t to = ast_const_lower_bound[from]; if (to == -1) - return std::nullopt; - return std::make_pair(graph.vertices[to].getConstant()->as()->value, dists.at({from, to}) == Path::LESS); + return std::nullopt; + + return std::make_pair(graph.vertices[to].getConstant()->as()->value, dists.at({from, to}) == Path::GREATER); } void ComparisonGraph::dfsOrder(const Graph & asts_graph, size_t v, std::vector & visited, std::vector & order) { visited[v] = true; for (const auto & edge : asts_graph.edges[v]) - { if (!visited[edge.to]) - { dfsOrder(asts_graph, edge.to, visited, order); - } - } + order.push_back(v); } @@ -418,14 +464,12 @@ ComparisonGraph::Graph ComparisonGraph::reverseGraph(const Graph & asts_graph) g.ast_hash_to_component = asts_graph.ast_hash_to_component; g.vertices = asts_graph.vertices; g.edges.resize(g.vertices.size()); + for (size_t v = 0; v < asts_graph.vertices.size(); ++v) - { for (const auto & edge : asts_graph.edges[v]) - { g.edges[edge.to].push_back(Edge{edge.type, v}); - } - } - return asts_graph; + + return g; } std::vector ComparisonGraph::getVertices() const @@ -441,21 +485,19 @@ std::vector ComparisonGraph::getVertices() const } void ComparisonGraph::dfsComponents( - const Graph & reversed_graph, size_t v, std::vector & components, const size_t not_visited, const size_t component) + const Graph & reversed_graph, size_t v, + OptionalIndices & components, const size_t component) { components[v] = component; for (const auto & edge : reversed_graph.edges[v]) - { - if (components[edge.to] == not_visited) - { - dfsComponents(reversed_graph, edge.to, components, not_visited, component); - } - } + if (!components[edge.to]) + dfsComponents(reversed_graph, edge.to, components, component); } -ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & asts_graph) +ComparisonGraph::Graph ComparisonGraph::buildGraphFromAstsGraph(const Graph & asts_graph) { - /// Find strongly connected component + /// Find strongly connected component by using 2 dfs traversals. + /// https://en.wikipedia.org/wiki/Kosaraju%27s_algorithm const auto n = asts_graph.vertices.size(); std::vector order; @@ -464,20 +506,19 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as for (size_t v = 0; v < n; ++v) { if (!visited[v]) - ComparisonGraph::dfsOrder(asts_graph, v, visited, order); + dfsOrder(asts_graph, v, visited, order); } } - const auto not_visited = std::numeric_limits::max(); - std::vector components(n, not_visited); + OptionalIndices components(n); size_t component = 0; { const Graph reversed_graph = reverseGraph(asts_graph); - for (const size_t v : order) + for (auto it = order.rbegin(); it != order.rend(); ++it) { - if (components[v] == not_visited) + if (!components[*it]) { - ComparisonGraph::dfsComponents(reversed_graph, v, components, not_visited, component); + dfsComponents(reversed_graph, *it, components, component); ++component; } } @@ -488,46 +529,40 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as result.edges.resize(component); for (const auto & [hash, index] : asts_graph.ast_hash_to_component) { - result.ast_hash_to_component[hash] = components[index]; - result.vertices[components[index]].asts.insert( - std::end(result.vertices[components[index]].asts), + assert(components[index]); + result.ast_hash_to_component[hash] = *components[index]; + result.vertices[*components[index]].asts.insert( + std::end(result.vertices[*components[index]].asts), std::begin(asts_graph.vertices[index].asts), std::end(asts_graph.vertices[index].asts)); // asts_graph has only one ast per vertex } /// Calculate constants for (auto & vertex : result.vertices) - { vertex.buildConstants(); - } - - Poco::Logger::get("ComparisonGraph").information("components: " + std::to_string(component)); + /// For each edge in initial graph, we add an edge between components in condensation graph. for (size_t v = 0; v < n; ++v) { for (const auto & edge : asts_graph.edges[v]) - { - result.edges[components[v]].push_back(Edge{edge.type, components[edge.to]}); - } - // TODO: make edges unique (left most strict) + result.edges[*components[v]].push_back(Edge{edge.type, *components[edge.to]}); + + /// TODO: make edges unique (left most strict) } + /// If we have constansts in two components, we can compare them and add and extra edge. for (size_t v = 0; v < result.vertices.size(); ++v) { for (size_t u = 0; u < result.vertices.size(); ++u) { - if (v == u) - continue; - if (result.vertices[v].hasConstant() && result.vertices[u].hasConstant()) + if (v != u && result.vertices[v].hasConstant() && result.vertices[u].hasConstant()) { const auto * left = result.vertices[v].getConstant()->as(); const auto * right = result.vertices[u].getConstant()->as(); - /// Only less. Equal constant fields = equal literals so it was already considered above. - if (left->value > right->value) - { - result.edges[v].push_back(Edge{Edge::LESS, u}); - } + /// Only GREATER. Equal constant fields = equal literals so it was already considered above. + if (greater(left->value, right->value)) + result.edges[v].push_back(Edge{Edge::GREATER, u}); } } } @@ -535,17 +570,21 @@ ComparisonGraph::Graph ComparisonGraph::BuildGraphFromAstsGraph(const Graph & as return result; } -std::map, ComparisonGraph::Path> ComparisonGraph::BuildDistsFromGraph(const Graph & g) +std::map, ComparisonGraph::Path> ComparisonGraph::buildDistsFromGraph(const Graph & g) { - // min path : < = -1, =< = 0 - const auto inf = std::numeric_limits::max(); + /// Min path : -1 means GREATER, 0 means GREATER_OR_EQUALS. + /// We use Floyd–Warshall algorithm to find distances between all pairs of vertices. + /// https://en.wikipedia.org/wiki/Floyd–Warshall_algorithm + + constexpr auto inf = std::numeric_limits::max(); const size_t n = g.vertices.size(); - std::vector> results(n, std::vector(n, inf)); + std::vector> results(n, std::vector(n, inf)); + for (size_t v = 0; v < n; ++v) { results[v][v] = 0; for (const auto & edge : g.edges[v]) - results[v][edge.to] = std::min(results[v][edge.to], static_cast(edge.type == Edge::LESS ? -1 : 0)); + results[v][edge.to] = std::min(results[v][edge.to], static_cast(edge.type == Edge::GREATER ? -1 : 0)); } for (size_t k = 0; k < n; ++k) @@ -558,7 +597,7 @@ std::map, ComparisonGraph::Path> ComparisonGraph::Buil for (size_t v = 0; v < n; ++v) for (size_t u = 0; u < n; ++u) if (results[v][u] != inf) - path[std::make_pair(v, u)] = (results[v][u] == -1 ? Path::LESS : Path::LESS_OR_EQUAL); + path[std::make_pair(v, u)] = (results[v][u] == -1 ? Path::GREATER : Path::GREATER_OR_EQUAL); return path; } @@ -577,18 +616,20 @@ std::pair, std::vector> ComparisonGraph::buildCons for (const auto & [edge, path] : dists) { const auto [from, to] = edge; + if (graph.vertices[to].hasConstant()) { if (lower[from] == -1 - || get_value(lower[from]) > get_value(to) - || (get_value(lower[from]) >= get_value(to) && dists.at({from, to}) == Path::LESS)) + || greater(get_value(to), get_value(lower[from])) + || (equals(get_value(to), get_value(lower[from])) && path == Path::GREATER)) lower[from] = to; } + if (graph.vertices[from].hasConstant()) { if (upper[to] == -1 - || get_value(upper[to]) < get_value(from) - || (get_value(upper[to]) <= get_value(from) && dists.at({from, to}) == Path::LESS)) + || less(get_value(from), get_value(upper[to])) + || (equals(get_value(from), get_value(upper[to])) && path == Path::GREATER)) upper[to] = from; } } @@ -596,5 +637,4 @@ std::pair, std::vector> ComparisonGraph::buildCons return {lower, upper}; } - } diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index 0c832c7ac85..b3858081910 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -16,6 +17,7 @@ namespace DB class ComparisonGraph { public: + /// atomic_formulas are extracted from constraints. ComparisonGraph(const std::vector & atomic_formulas); enum class CompareResult @@ -29,7 +31,8 @@ public: UNKNOWN, }; - static CompareResult getCompareResult(const std::string & name); + static CompareResult atomToCompareResult(const CNFQuery::AtomicFormula & atom); + static CompareResult functionNameToCompareResult(const std::string & name); static CompareResult inverseCompareResult(const CompareResult result); CompareResult compare(const ASTPtr & left, const ASTPtr & right) const; @@ -40,11 +43,20 @@ public: /// It's always true that left right bool isAlwaysCompare(const CompareResult expected, const ASTPtr & left, const ASTPtr & right) const; + /// Returns all expressions from component to which @ast belongs if any. std::vector getEqual(const ASTPtr & ast) const; + + /// Returns constant expression from component to which @ast belongs if any. std::optional getEqualConst(const ASTPtr & ast) const; + /// Finds component id to which @ast belongs if any. std::optional getComponentId(const ASTPtr & ast) const; + + /// Returns all expressions from component. std::vector getComponent(const std::size_t id) const; + + size_t getNumOfComponents() const { return graph.vertices.size(); } + bool hasPath(const size_t left, const size_t right) const; /// Find constants lessOrEqual and greaterOrEqual. @@ -53,30 +65,35 @@ public: std::optional> getConstUpperBound(const ASTPtr & ast) const; std::optional> getConstLowerBound(const ASTPtr & ast) const; + /// Returns all expression in graph. std::vector getVertices() const; private: - /// strongly connected component + /// Strongly connected component struct EqualComponent { + /// All these expressions are considered as equal. std::vector asts; - ssize_t constant_index = -1; + std::optional constant_index; bool hasConstant() const; ASTPtr getConstant() const; void buildConstants(); }; + /// Edge (from, to, type) means that it's always true that @from @to, + /// where @op is the operation of type @type. + /// /// TODO: move to diff for int and double: - /// LESS and LESS_OR_EQUAL with +const or 0 --- ok + /// GREATER and GREATER_OR_EQUAL with +const or 0 --- ok /// with -const --- not ok /// EQUAL is ok only for 0 struct Edge { enum Type { - LESS, - LESS_OR_EQUAL, + GREATER, + GREATER_OR_EQUAL, EQUAL, }; @@ -99,29 +116,60 @@ private: std::vector> edges; }; - static ASTPtr normalizeAtom(const ASTPtr & atom); - static Graph BuildGraphFromAstsGraph(const Graph & asts_graph); + /// Recieves graph, in which each vertex corresponds to one expression. + /// Then finds strongly connected components and builds graph on them. + static Graph buildGraphFromAstsGraph(const Graph & asts_graph); static Graph reverseGraph(const Graph & asts_graph); - static void dfsOrder(const Graph & asts_graph, size_t v, std::vector & visited, std::vector & order); - static void dfsComponents( - const Graph & reversed_graph, size_t v, std::vector & components, const size_t not_visited, const size_t component); - std::pair findPath(const size_t start, const size_t finish) const; + /// The first part of finding strongly connected components. + /// Finds order of exit from vertices of dfs traversal of graph. + static void dfsOrder(const Graph & asts_graph, size_t v, std::vector & visited, std::vector & order); + + using OptionalIndices = std::vector>; + + /// The second part of finding strongly connected components. + /// Assigns index of component for each vertex. + static void dfsComponents( + const Graph & reversed_graph, size_t v, + OptionalIndices & components, const size_t component); enum class Path { - LESS, - LESS_OR_EQUAL, + GREATER, + GREATER_OR_EQUAL, }; - static std::map, Path> BuildDistsFromGraph(const Graph & g); + static CompareResult pathToCompareResult(Path path, bool inverse); + std::optional findPath(const size_t start, const size_t finish) const; + + /// Calculate @dists. + static std::map, Path> buildDistsFromGraph(const Graph & g); + + /// Calculate @ast_const_lower_bound and @ast_const_lower_bound. std::pair, std::vector> buildConstBounds() const; + /// Direct acyclic graph in which each vertex corresponds + /// to one equivalence class of expressions. + /// Each edge sets the relation between classes (GREATER or GREATER_OR_EQUAL). Graph graph; + + /// Precalculated distances between each pair of vertices. + /// Distance can be either 0 or -1. + /// 0 means GREATER_OR_EQUAL. + /// -1 means GREATER. std::map, Path> dists; + + /// Explicitly collected components, for which it's known + /// that expressions in them are unequal. std::set> not_equal; + + /// Maximal constant value for each component that + /// is lower bound for all expressions in component. std::vector ast_const_lower_bound; + + /// Minimal constant value for each component that + /// is upper bound for all expressions in component. std::vector ast_const_upper_bound; }; diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index ff826c0fde6..864b9c4853d 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -555,13 +555,12 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription( ConstraintsDescription InterpreterCreateQuery::getConstraintsDescription(const ASTExpressionList * constraints) { - ConstraintsDescription res; - auto constraints_data = res.getConstraints(); + ASTs constraints_data; if (constraints) for (const auto & constraint : constraints->children) - constraints_data.push_back(std::dynamic_pointer_cast(constraint->clone())); - res.updateConstraints(constraints_data); - return res; + constraints_data.push_back(constraint->clone()); + + return ConstraintsDescription{constraints_data}; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 57f8f389fbc..fc6e193fc6e 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -411,8 +411,6 @@ InterpreterSelectQuery::InterpreterSelectQuery( current_info.query = query_ptr; current_info.syntax_analyzer_result = syntax_analyzer_result; - log->information("TRY MOVE TO PREWHERE ---------------------"); - log->information(query.where()->dumpTree()); MergeTreeWhereOptimizer{ current_info, context, diff --git a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp b/src/Interpreters/SubstituteColumnOptimizer.cpp similarity index 80% rename from src/Storages/MergeTree/SubstituteColumnOptimizer.cpp rename to src/Interpreters/SubstituteColumnOptimizer.cpp index cd4f0560a0c..b7f5de4c558 100644 --- a/src/Storages/MergeTree/SubstituteColumnOptimizer.cpp +++ b/src/Interpreters/SubstituteColumnOptimizer.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include #include @@ -21,8 +21,7 @@ namespace ErrorCodes namespace { -const String COMPONENT = "__aorLwT30aH_comp"; -const String COMPONENT_SEPARATOR = "_"; +constexpr auto COMPONENT_PART = "__component_"; constexpr UInt64 COLUMN_PENALTY = 10 * 1024 * 1024; constexpr Int64 INDEX_PRICE = -1'000'000'000'000'000'000; @@ -55,14 +54,13 @@ public: static void visit(ASTPtr & ast, Data & data) { - const auto id = data.graph.getComponentId(ast); - if (id) + if (auto id = data.graph.getComponentId(ast)) { - const String name = COMPONENT + std::to_string(id.value()) + COMPONENT_SEPARATOR + std::to_string(++data.current_id); + const String name = COMPONENT_PART + std::to_string(*id) + "_" + std::to_string(++data.current_id); data.old_name[name] = ast->getAliasOrColumnName(); - data.component[name] = id.value(); + data.component[name] = *id; + data.components.insert(*id); ast = std::make_shared(name); - data.components.insert(id.value()); } } @@ -74,19 +72,6 @@ public: using ComponentVisitor = ComponentMatcher::Visitor; - -void collectIdentifiers(const ASTPtr & ast, std::unordered_set & identifiers) -{ - const auto * identifier = ast->as(); - if (identifier) - identifiers.insert(identifier->name()); - else - { - for (const auto & child : ast->children) - collectIdentifiers(child, identifiers); - } -} - struct ColumnPrice { Int64 compressed_size; @@ -95,11 +80,10 @@ struct ColumnPrice ColumnPrice(const Int64 compressed_size_, const Int64 uncompressed_size_) : compressed_size(compressed_size_) , uncompressed_size(uncompressed_size_) - {} + { + } - ColumnPrice() - : ColumnPrice(0, 0) - {} + ColumnPrice() : ColumnPrice(0, 0) {} bool operator<(const ColumnPrice & that) const { @@ -127,6 +111,8 @@ struct ColumnPrice } }; +using ColumnPriceByName = std::unordered_map; + class SubstituteColumnMatcher { public: @@ -148,10 +134,10 @@ public: const String & name = identifier->name(); const auto component_id = data.name_to_component_id.at(name); auto new_ast = data.id_to_expression_map.at(component_id)->clone(); + if (data.is_select) - { new_ast->setAlias(data.old_name.at(name)); - } + ast = new_ast; } } @@ -165,8 +151,8 @@ public: using SubstituteColumnVisitor = SubstituteColumnMatcher::Visitor; ColumnPrice calculatePrice( - const std::unordered_map & column_prices, - std::unordered_set identifiers) + const ColumnPriceByName & column_prices, + const IdentifierNameSet & identifiers) { ColumnPrice result(0, 0); for (const auto & ident : identifiers) @@ -174,12 +160,16 @@ ColumnPrice calculatePrice( return result; } -// TODO: branch-and-bound +/// We need to choose one expression in each component, +/// so that total price of all read columns will be minimal. +/// Bruteforce equal ASTs in each component and calculate +/// price of all columns on which ast depends. +/// TODO: branch-and-bound void bruteforce( const ComparisonGraph & graph, const std::vector & components, size_t current_component, - const std::unordered_map & column_prices, + const ColumnPriceByName & column_prices, ColumnPrice current_price, std::vector & expressions_stack, ColumnPrice & min_price, @@ -197,14 +187,15 @@ void bruteforce( { for (const auto & ast : graph.getComponent(components[current_component])) { - std::unordered_set identifiers; - collectIdentifiers(ast, identifiers); + IdentifierNameSet identifiers; + ast->collectIdentifierNames(identifiers); ColumnPrice expression_price = calculatePrice(column_prices, identifiers); expressions_stack.push_back(ast); current_price += expression_price; - std::unordered_map new_prices(column_prices); + ColumnPriceByName new_prices(column_prices); + /// Update prices of already counted columns. for (const auto & identifier : identifiers) new_prices[identifier] = ColumnPrice(0, 0); @@ -240,12 +231,10 @@ void SubstituteColumnOptimizer::perform() { if (!storage) return; + const auto column_sizes = storage->getColumnSizes(); if (column_sizes.empty()) - { - Poco::Logger::get("SubstituteColumnOptimizer").information("skip: column sizes not available"); return; - } const auto & compare_graph = metadata_snapshot->getConstraints().getGraph(); @@ -254,7 +243,8 @@ void SubstituteColumnOptimizer::perform() { auto * list = select_query->refSelect()->as(); if (!list) - throw Exception("Bad select list.", ErrorCodes::LOGICAL_ERROR); + throw Exception("List of selected columns must be ASTExpressionList", ErrorCodes::LOGICAL_ERROR); + for (ASTPtr & ast : list->children) ast->setAlias(ast->getAliasOrColumnName()); } @@ -274,40 +264,50 @@ void SubstituteColumnOptimizer::perform() std::set components; std::unordered_map old_name; std::unordered_map name_to_component; + UInt64 counter_id = 0; + ComponentVisitor::Data component_data( compare_graph, components, old_name, name_to_component, counter_id); - std::unordered_set identifiers; + + IdentifierNameSet identifiers; auto preprocess = [&](ASTPtr & ast, bool) { ComponentVisitor(component_data).visit(ast); - collectIdentifiers(ast, identifiers); + ast->collectIdentifierNames(identifiers); }; run_for_all(preprocess); const auto primary_key = metadata_snapshot->getColumnsRequiredForPrimaryKey(); const std::unordered_set primary_key_set(std::begin(primary_key), std::end(primary_key)); - std::unordered_map column_prices; + ColumnPriceByName column_prices; + for (const auto & [column_name, column_size] : column_sizes) - column_prices[column_name] = ColumnPrice( - column_size.data_compressed + COLUMN_PENALTY, column_size.data_uncompressed); + column_prices[column_name] = ColumnPrice(column_size.data_compressed + COLUMN_PENALTY, column_size.data_uncompressed); + for (const auto & column_name : primary_key) column_prices[column_name] = ColumnPrice(INDEX_PRICE, INDEX_PRICE); + for (const auto & column_name : identifiers) column_prices[column_name] = ColumnPrice(0, 0); std::unordered_map id_to_expression_map; std::vector components_list; - for (const UInt64 component : components) - if (compare_graph.getComponent(component).size() == 1) - id_to_expression_map[component] = compare_graph.getComponent(component).front(); + + for (const UInt64 component_id : components) + { + auto component = compare_graph.getComponent(component_id); + if (component.size() == 1) + id_to_expression_map[component_id] = component.front(); else - components_list.push_back(component); + components_list.push_back(component_id); + } std::vector expressions_stack; ColumnPrice min_price(std::numeric_limits::max(), std::numeric_limits::max()); std::vector min_expressions; + bruteforce(compare_graph, components_list, 0, diff --git a/src/Storages/MergeTree/SubstituteColumnOptimizer.h b/src/Interpreters/SubstituteColumnOptimizer.h similarity index 82% rename from src/Storages/MergeTree/SubstituteColumnOptimizer.h rename to src/Interpreters/SubstituteColumnOptimizer.h index 6fc7c215afd..63867e80386 100644 --- a/src/Storages/MergeTree/SubstituteColumnOptimizer.h +++ b/src/Interpreters/SubstituteColumnOptimizer.h @@ -14,6 +14,8 @@ namespace DB struct StorageInMemoryMetadata; using StorageMetadataPtr = std::shared_ptr; +/// Optimizer that tries to replace columns to equal columns (according to constraints) +/// with lower size (accorsing to compressed and uncomressed size). class SubstituteColumnOptimizer { public: diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index b60eaa28e0a..46002f5be99 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -5,12 +5,16 @@ namespace DB { + namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int INCORRECT_QUERY; } +namespace +{ + /// Splits AND(a, b, c) to AND(a, AND(b, c)) for AND/OR void splitMultiLogic(ASTPtr & node) { @@ -19,7 +23,7 @@ void splitMultiLogic(ASTPtr & node) if (func && (func->name == "and" || func->name == "or")) { if (func->arguments->children.size() < 2) - throw Exception("Bad logical function", ErrorCodes::INCORRECT_QUERY); + throw Exception("Bad AND or OR function. Expected at least 2 arguments", ErrorCodes::INCORRECT_QUERY); if (func->arguments->children.size() > 2) { @@ -52,7 +56,8 @@ void traversePushNot(ASTPtr & node, bool add_negation) if (add_negation) { if (func->arguments->children.size() != 2) - throw Exception("Bad AND or OR function.", ErrorCodes::LOGICAL_ERROR); + throw Exception("Bad AND or OR function. Expected at least 2 arguments", ErrorCodes::LOGICAL_ERROR); + /// apply De Morgan's Law node = makeASTFunction( (func->name == "and" ? "or" : "and"), @@ -67,7 +72,7 @@ void traversePushNot(ASTPtr & node, bool add_negation) else if (func && func->name == "not") { if (func->arguments->children.size() != 1) - throw Exception("Bad NOT function.", ErrorCodes::INCORRECT_QUERY); + throw Exception("Bad NOT function. Expected 1 argument", ErrorCodes::INCORRECT_QUERY); /// delete NOT node = func->arguments->children[0]->clone(); @@ -93,6 +98,7 @@ void traversePushOr(ASTPtr & node) if (func && func->name == "or") { + assert(func->arguments->children.size() == 2); size_t and_node_id = func->arguments->children.size(); for (size_t i = 0; i < func->arguments->children.size(); ++i) { @@ -146,7 +152,7 @@ void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & and_group, CNFQuery:: else if (func && func->name == "not") { if (func->arguments->children.size() != 1) - throw Exception("Bad NOT function", ErrorCodes::INCORRECT_QUERY); + throw Exception("Bad NOT function. Expected 1 argument", ErrorCodes::INCORRECT_QUERY); or_group.insert(CNFQuery::AtomicFormula{true, func->arguments->children.front()}); } else @@ -163,6 +169,8 @@ void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & result) result.insert(or_group); } +} + CNFQuery TreeCNFConverter::toCNF(const ASTPtr & query) { auto cnf = query->clone(); @@ -175,7 +183,6 @@ CNFQuery TreeCNFConverter::toCNF(const ASTPtr & query) CNFQuery result{std::move(and_group)}; - Poco::Logger::get("TreeCNFConverter").information("Converted to CNF: " + result.dump()); return result; } @@ -220,7 +227,7 @@ ASTPtr TreeCNFConverter::fromCNF(const CNFQuery & cnf) return res; } -void pushPullNotInAtom(CNFQuery::AtomicFormula & atom, const std::map & inverse_relations) +static void pushPullNotInAtom(CNFQuery::AtomicFormula & atom, const std::unordered_map & inverse_relations) { auto * func = atom.ast->as(); if (!func) @@ -236,9 +243,9 @@ void pushPullNotInAtom(CNFQuery::AtomicFormula & atom, const std::map inverse_relations = { + static const std::unordered_map inverse_relations = { {"notEquals", "equals"}, {"greaterOrEquals", "less"}, {"greater", "lessOrEquals"}, @@ -255,7 +262,7 @@ void pushNotIn(CNFQuery::AtomicFormula & atom) if (!atom.negative) return; - static const std::map inverse_relations = { + static const std::unordered_map inverse_relations = { {"equals", "notEquals"}, {"less", "greaterOrEquals"}, {"lessOrEquals", "greater"}, diff --git a/src/Interpreters/TreeCNFConverter.h b/src/Interpreters/TreeCNFConverter.h index 7dfb2e511ba..ba7fb299644 100644 --- a/src/Interpreters/TreeCNFConverter.h +++ b/src/Interpreters/TreeCNFConverter.h @@ -67,7 +67,7 @@ public: filtered.insert(filtered_group); else { - /// all atoms false -> group false -> CNF false + /// all atoms false -> group false -> CNF false filtered.clear(); filtered_group.clear(); filtered_group.insert(AtomicFormula{false, std::make_shared(static_cast(0))}); diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 6665b36357c..1b816984647 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -5,7 +5,7 @@ #include #include #include -#include +#include #include #include #include @@ -563,7 +563,7 @@ void optimizeSubstituteColumn(ASTSelectQuery * select_query, SubstituteColumnOptimizer(select_query, metadata_snapshot, storage).perform(); } -/// transform where to CNF for more convenient optimization +/// Transform WHERE to CNF for more convenient optimization. void convertQueryToCNF(ASTSelectQuery * select_query) { if (select_query->where()) diff --git a/src/Interpreters/WhereConstraintsOptimizer.cpp b/src/Interpreters/WhereConstraintsOptimizer.cpp index 595d39c406d..83bdcfeb2e1 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.cpp +++ b/src/Interpreters/WhereConstraintsOptimizer.cpp @@ -9,13 +9,15 @@ #include #include +#include + namespace DB { WhereConstraintsOptimizer::WhereConstraintsOptimizer( ASTSelectQuery * select_query_, const StorageMetadataPtr & metadata_snapshot_, - const bool optimize_append_index_) + bool optimize_append_index_) : select_query(select_query_) , metadata_snapshot(metadata_snapshot_) , optimize_append_index(optimize_append_index_) @@ -24,29 +26,32 @@ WhereConstraintsOptimizer::WhereConstraintsOptimizer( namespace { - enum class MatchState - { - FULL_MATCH, /// a = b - NOT_MATCH, /// a = not b - NONE, /// other - }; -} + +enum class MatchState +{ + FULL_MATCH, /// a = b + NOT_MATCH, /// a = not b + NONE, /// other +}; MatchState match(CNFQuery::AtomicFormula a, CNFQuery::AtomicFormula b) { - bool match_means_ok = true ^ a.negative ^ b.negative; - + bool match_means_ok = (a.negative == b.negative); if (a.ast->getTreeHash() == b.ast->getTreeHash()) - { return match_means_ok ? MatchState::FULL_MATCH : MatchState::NOT_MATCH; - } + return MatchState::NONE; } bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const ConstraintsDescription & constraints_description) { + /// We have constraints in CNF. + /// CNF is always true => Each OR group in CNF is always true. + /// So, we try to check whether we have al least one OR group from CNF as subset in our group. + /// If we've found one then our group is always true too. + const auto & constraints_data = constraints_description.getConstraintData(); - std::vector found(constraints_data.size(), 0); + std::vector found(constraints_data.size()); for (size_t i = 0; i < constraints_data.size(); ++i) found[i] = constraints_data[i].size(); @@ -55,12 +60,12 @@ bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const Cons const auto constraint_atom_ids = constraints_description.getAtomIds(atom.ast); if (constraint_atom_ids) { - const auto constraint_atoms = constraints_description.getAtomsById(constraint_atom_ids.value()); + const auto constraint_atoms = constraints_description.getAtomsById(*constraint_atom_ids); for (size_t i = 0; i < constraint_atoms.size(); ++i) { if (match(constraint_atoms[i], atom) == MatchState::FULL_MATCH) { - if ((--found[(*constraint_atom_ids)[i].and_group]) == 0) + if ((--found[(*constraint_atom_ids)[i].group_id]) == 0) return true; } } @@ -69,31 +74,20 @@ bool checkIfGroupAlwaysTrueFullMatch(const CNFQuery::OrGroup & group, const Cons return false; } -ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula & atom) -{ - const auto * func = atom.ast->as(); - if (func) - { - auto expected = ComparisonGraph::getCompareResult(func->name); - if (atom.negative) - expected = ComparisonGraph::inverseCompareResult(expected); - return expected; - } - return ComparisonGraph::CompareResult::UNKNOWN; -} - - bool checkIfGroupAlwaysTrueGraph(const CNFQuery::OrGroup & group, const ComparisonGraph & graph) { + /// We try to find at least one atom that is always true by using comparison graph. for (const auto & atom : group) { const auto * func = atom.ast->as(); if (func && func->arguments->children.size() == 2) { - const auto expected = getExpectedCompare(atom); - return graph.isAlwaysCompare(expected, func->arguments->children[0], func->arguments->children[1]); + const auto expected = ComparisonGraph::atomToCompareResult(atom); + if (graph.isAlwaysCompare(expected, func->arguments->children[0], func->arguments->children[1])) + return true; } } + return false; } @@ -103,7 +97,7 @@ bool checkIfAtomAlwaysFalseFullMatch(const CNFQuery::AtomicFormula & atom, const const auto constraint_atom_ids = constraints_description.getAtomIds(atom.ast); if (constraint_atom_ids) { - for (const auto & constraint_atom : constraints_description.getAtomsById(constraint_atom_ids.value())) + for (const auto & constraint_atom : constraints_description.getAtomsById(*constraint_atom_ids)) { const auto match_result = match(constraint_atom, atom); if (match_result == MatchState::NOT_MATCH) @@ -120,7 +114,7 @@ bool checkIfAtomAlwaysFalseGraph(const CNFQuery::AtomicFormula & atom, const Com if (func && func->arguments->children.size() == 2) { /// TODO: special support for != - const auto expected = getExpectedCompare(atom); + const auto expected = ComparisonGraph::atomToCompareResult(atom); return !graph.isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1]); } @@ -152,13 +146,14 @@ CNFQuery::AtomicFormula replaceTermsToConstants(const CNFQuery::AtomicFormula & return result; } +} + void WhereConstraintsOptimizer::perform() { if (select_query->where() && metadata_snapshot) { const auto & compare_graph = metadata_snapshot->getConstraints().getGraph(); auto cnf = TreeCNFConverter::toCNF(select_query->where()); - Poco::Logger::get("WhereConstraintsOptimizer").information("Before optimization: " + cnf.dump()); cnf.pullNotOutFunctions() .filterAlwaysTrueGroups([&compare_graph, this](const auto & group) { @@ -180,7 +175,6 @@ void WhereConstraintsOptimizer::perform() if (optimize_append_index) AddIndexConstraintsOptimizer(metadata_snapshot).perform(cnf); - Poco::Logger::get("WhereConstraintsOptimizer").information("After optimization: " + cnf.dump()); select_query->setExpression(ASTSelectQuery::Expression::WHERE, TreeCNFConverter::fromCNF(cnf)); } } diff --git a/src/Interpreters/WhereConstraintsOptimizer.h b/src/Interpreters/WhereConstraintsOptimizer.h index 75160983a76..01cee91fca5 100644 --- a/src/Interpreters/WhereConstraintsOptimizer.h +++ b/src/Interpreters/WhereConstraintsOptimizer.h @@ -1,12 +1,7 @@ #pragma once #include -#include -#include -#include -#include -#include -#include +#include namespace DB { @@ -14,13 +9,15 @@ namespace DB struct StorageInMemoryMetadata; using StorageMetadataPtr = std::shared_ptr; +/// Optimizer that can remove useless parts of conditions +/// in WHERE clause according to table constraints. class WhereConstraintsOptimizer final { public: WhereConstraintsOptimizer( ASTSelectQuery * select_query, const StorageMetadataPtr & metadata_snapshot, - const bool optimize_append_index_); + bool optimize_append_index_); void perform(); diff --git a/src/Interpreters/tests/gtest_comparison_graph.cpp b/src/Interpreters/tests/gtest_comparison_graph.cpp new file mode 100644 index 00000000000..6348bd13b8e --- /dev/null +++ b/src/Interpreters/tests/gtest_comparison_graph.cpp @@ -0,0 +1,183 @@ +#include +#include +#include +#include +#include +#include + +#include + +using namespace DB; + +static ComparisonGraph getGraph(const String & query) +{ + ParserExpressionList parser(false); + ASTPtr ast = parseQuery(parser, query, 0, 0); + return ComparisonGraph(ast->children); +} + +TEST(ComparisonGraph, Bounds) +{ + String query = "x <= 1, 1 < c, 3 < c, c < d, d < e, e < 7, e < 10, 10 <= y"; + auto graph = getGraph(query); + + auto d = std::make_shared("d"); + + { + auto res = graph.getConstLowerBound(d); + ASSERT_TRUE(res.has_value()); + + const auto & [lower, strict] = *res; + + ASSERT_EQ(get(lower), 3); + ASSERT_TRUE(strict); + } + + { + auto res = graph.getConstUpperBound(d); + ASSERT_TRUE(res.has_value()); + + const auto & [upper, strict] = *res; + + ASSERT_EQ(get(upper), 7); + ASSERT_TRUE(strict); + } + + { + auto x = std::make_shared("x"); + auto y = std::make_shared("y"); + + ASSERT_EQ(graph.compare(x, y), ComparisonGraph::CompareResult::LESS); + ASSERT_EQ(graph.compare(y, x), ComparisonGraph::CompareResult::GREATER); + } +} + +using Components = std::set>; + +static std::set componentToStrings(const ASTs & comp) +{ + std::set res; + for (const auto & ast : comp) + res.insert(ast->getColumnName()); + return res; +} + +static void checkComponents(const String & query, const Components & expected) +{ + auto graph = getGraph(query); + + size_t num_components = graph.getNumOfComponents(); + ASSERT_EQ(num_components, expected.size()); + + Components res; + for (size_t i = 0; i < num_components; ++i) + res.insert(componentToStrings(graph.getComponent(i))); + + ASSERT_EQ(res, expected); +} + +TEST(ComparisonGraph, Components) +{ + { + String query = "a >= b, b >= c, c >= d, d >= b, d >= e, a >= e"; + Components expected = {{"a"}, {"b", "c", "d"}, {"e"}}; + checkComponents(query, expected); + } + + { + String query = "a >= b, b >= a, b >= c, c >= d, d >= c"; + Components expected = {{"a", "b"}, {"c", "d"}}; + checkComponents(query, expected); + } +} + +TEST(ComparisonGraph, Compare) +{ + using CompareResult = ComparisonGraph::CompareResult; + + { + String query = "a >= b, c >= b"; + auto graph = getGraph(query); + + auto a = std::make_shared("a"); + auto c = std::make_shared("c"); + + ASSERT_EQ(graph.compare(a, c), CompareResult::UNKNOWN); + } + + { + String query = "a >= b, b > c"; + auto graph = getGraph(query); + + auto a = std::make_shared("a"); + auto b = std::make_shared("b"); + auto c = std::make_shared("c"); + + ASSERT_EQ(graph.compare(a, c), CompareResult::GREATER); + ASSERT_EQ(graph.compare(a, b), CompareResult::GREATER_OR_EQUAL); + ASSERT_EQ(graph.compare(b, c), CompareResult::GREATER); + } + + { + String query = "a != b, c < a"; + auto graph = getGraph(query); + + auto a = std::make_shared("a"); + auto b = std::make_shared("b"); + auto c = std::make_shared("c"); + + ASSERT_EQ(graph.compare(a, b), CompareResult::NOT_EQUAL); + ASSERT_EQ(graph.compare(a, c), CompareResult::GREATER); + ASSERT_EQ(graph.compare(b, c), CompareResult::UNKNOWN); + } + + { + /// These constraints are inconsistent. + String query = "a >= b, b >= a, a != b"; + ASSERT_THROW(getGraph(query), Exception); + } + + { + /// These constraints are inconsistent. + String query = "a > b, b > c, c > a"; + ASSERT_THROW(getGraph(query), Exception); + } + + { + String query = "a >= 3, b > a, c >= 3, d >= c"; + auto graph = getGraph(query); + + auto a = std::make_shared("a"); + auto b = std::make_shared("b"); + auto d = std::make_shared("d"); + auto lit_2 = std::make_shared(2u); + auto lit_3 = std::make_shared(3u); + auto lit_4 = std::make_shared(4u); + + ASSERT_EQ(graph.compare(lit_3, a), CompareResult::LESS_OR_EQUAL); + ASSERT_FALSE(graph.isAlwaysCompare(CompareResult::LESS, lit_3, a)); + ASSERT_TRUE(graph.isAlwaysCompare(CompareResult::LESS, lit_2, a)); + + ASSERT_EQ(graph.compare(b, lit_2), CompareResult::GREATER); + ASSERT_EQ(graph.compare(b, lit_3), CompareResult::GREATER); + ASSERT_EQ(graph.compare(b, lit_4), CompareResult::UNKNOWN); + + ASSERT_EQ(graph.compare(d, lit_2), CompareResult::GREATER); + ASSERT_EQ(graph.compare(d, lit_3), CompareResult::GREATER_OR_EQUAL); + ASSERT_EQ(graph.compare(d, lit_4), CompareResult::UNKNOWN); + } + + { + String query = "a >= 5, a <= 10"; + auto graph = getGraph(query); + + auto a = std::make_shared("a"); + auto lit_8 = std::make_shared(8); + auto lit_3 = std::make_shared(3); + auto lit_15 = std::make_shared(15); + + ASSERT_EQ(graph.compare(a, lit_8), CompareResult::UNKNOWN); + ASSERT_EQ(graph.compare(a, lit_3), CompareResult::GREATER); + ASSERT_EQ(graph.compare(a, lit_15), CompareResult::LESS); + } +} diff --git a/src/Parsers/ASTConstraintDeclaration.h b/src/Parsers/ASTConstraintDeclaration.h index 5db8df43a76..437aab1a82d 100644 --- a/src/Parsers/ASTConstraintDeclaration.h +++ b/src/Parsers/ASTConstraintDeclaration.h @@ -10,7 +10,7 @@ namespace DB class ASTConstraintDeclaration : public IAST { public: - enum class Type + enum class Type : UInt8 { CHECK, ASSUME, diff --git a/src/Storages/AlterCommands.cpp b/src/Storages/AlterCommands.cpp index 2a13dff9380..52825c4fb43 100644 --- a/src/Storages/AlterCommands.cpp +++ b/src/Storages/AlterCommands.cpp @@ -565,9 +565,8 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) } auto insert_it = constraints.end(); - - constraints.emplace(insert_it, std::dynamic_pointer_cast(constraint_decl)); - metadata.constraints.updateConstraints(constraints); + constraints.emplace(insert_it, constraint_decl); + metadata.constraints = ConstraintsDescription(constraints); } else if (type == DROP_CONSTRAINT) { @@ -588,7 +587,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) ErrorCodes::BAD_ARGUMENTS); } constraints.erase(erase_it); - metadata.constraints.updateConstraints(constraints); + metadata.constraints = ConstraintsDescription(constraints); } else if (type == ADD_PROJECTION) { @@ -661,7 +660,7 @@ void AlterCommand::apply(StorageInMemoryMetadata & metadata, ContextPtr context) auto constraints_data = metadata.constraints.getConstraints(); for (auto & constraint : constraints_data) rename_visitor.visit(constraint); - metadata.constraints.updateConstraints(constraints_data); + metadata.constraints = ConstraintsDescription(constraints_data); if (metadata.isSortingKeyDefined()) rename_visitor.visit(metadata.sorting_key.definition_ast); diff --git a/src/Storages/ConstraintsDescription.cpp b/src/Storages/ConstraintsDescription.cpp index 1bd7e01d5bb..60202e2055e 100644 --- a/src/Storages/ConstraintsDescription.cpp +++ b/src/Storages/ConstraintsDescription.cpp @@ -48,14 +48,14 @@ ConstraintsDescription ConstraintsDescription::parse(const String & str) ASTs ConstraintsDescription::filterConstraints(ConstraintType selection) const { - const auto ast_to_decr_constraint_type = [](ASTConstraintDeclaration::Type constraint_type) -> UInt32 + const auto ast_to_decr_constraint_type = [](ASTConstraintDeclaration::Type constraint_type) -> UInt8 { switch (constraint_type) { case ASTConstraintDeclaration::Type::CHECK: - return static_cast(ConstraintType::CHECK); + return static_cast(ConstraintType::CHECK); case ASTConstraintDeclaration::Type::ASSUME: - return static_cast(ConstraintType::ASSUME); + return static_cast(ConstraintType::ASSUME); } throw Exception("Unknown constraint type.", ErrorCodes::LOGICAL_ERROR); }; @@ -64,7 +64,7 @@ ASTs ConstraintsDescription::filterConstraints(ConstraintType selection) const res.reserve(constraints.size()); for (const auto & constraint : constraints) { - if ((ast_to_decr_constraint_type(constraint->as()->type) & static_cast(selection)) != 0) + if ((ast_to_decr_constraint_type(constraint->as()->type) & static_cast(selection)) != 0) { res.push_back(constraint); } @@ -91,7 +91,6 @@ std::vector ConstraintsDescription::getAtomicConstraint std::vector constraint_data; for (const auto & constraint : filterConstraints(ConstraintsDescription::ConstraintType::ALWAYS_TRUE)) { - Poco::Logger::get("atomic_formula: initial:").information(constraint->as()->expr->ptr()->dumpTree()); const auto cnf = TreeCNFConverter::toCNF(constraint->as()->expr->ptr()) .pullNotOutFunctions(); for (const auto & group : cnf.getStatements()) @@ -106,22 +105,18 @@ std::vector ConstraintsDescription::getAtomicConstraint std::unique_ptr ConstraintsDescription::buildGraph() const { - static const std::set relations = { - "equals", "less", "lessOrEquals", "greaterOrEquals", "greater"}; + static const NameSet relations = { "equals", "less", "lessOrEquals", "greaterOrEquals", "greater" }; std::vector constraints_for_graph; auto atomic_formulas = getAtomicConstraintData(); for (const auto & atomic_formula : atomic_formulas) { - Poco::Logger::get("atomic_formula: before:").information(atomic_formula.ast->dumpTree() + " " + std::to_string(atomic_formula.negative)); CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()}; pushNotIn(atom); auto * func = atom.ast->as(); if (func && relations.count(func->name)) { - if (atom.negative) - throw Exception(": ", ErrorCodes::LOGICAL_ERROR); - Poco::Logger::get("atomic_formula: after:").information(atom.ast->dumpTree() + " " + std::to_string(atom.negative)); + assert(!atom.negative); constraints_for_graph.push_back(atom.ast); } } @@ -166,8 +161,9 @@ const std::vector & ConstraintsDescription::getConstraints() const std::optional ConstraintsDescription::getAtomIds(const ASTPtr & ast) const { const auto hash = ast->getTreeHash(); - if (ast_to_atom_ids.contains(hash)) - return ast_to_atom_ids.at(hash); + auto it = ast_to_atom_ids.find(hash); + if (it != ast_to_atom_ids.end()) + return it->second; return std::nullopt; } @@ -175,13 +171,13 @@ std::vector ConstraintsDescription::getAtomsById(const { std::vector result; for (const auto & id : ids) - result.push_back(cnf_constraints[id.and_group][id.atom]); + result.push_back(cnf_constraints[id.group_id][id.atom_id]); return result; } -void ConstraintsDescription::updateConstraints(const std::vector & constraints_) +ConstraintsDescription::ConstraintsDescription(const ASTs & constraints_) + : constraints(constraints_) { - constraints = constraints_; update(); } @@ -211,15 +207,13 @@ void ConstraintsDescription::update() graph = std::make_unique(std::vector()); return; } + cnf_constraints = buildConstraintData(); ast_to_atom_ids.clear(); for (size_t i = 0; i < cnf_constraints.size(); ++i) - { for (size_t j = 0; j < cnf_constraints[i].size(); ++j) - { ast_to_atom_ids[cnf_constraints[i][j].ast->getTreeHash()].push_back({i, j}); - } - } + graph = buildGraph(); } diff --git a/src/Storages/ConstraintsDescription.h b/src/Storages/ConstraintsDescription.h index d08b26fc7c2..ad8bd371f38 100644 --- a/src/Storages/ConstraintsDescription.h +++ b/src/Storages/ConstraintsDescription.h @@ -14,7 +14,7 @@ struct ConstraintsDescription { public: ConstraintsDescription() { update(); } - + ConstraintsDescription(const ASTs & constraints_); ConstraintsDescription(const ConstraintsDescription & other); ConstraintsDescription & operator=(const ConstraintsDescription & other); @@ -23,7 +23,7 @@ public: static ConstraintsDescription parse(const String & str); - enum class ConstraintType + enum class ConstraintType : UInt8 { CHECK = 1, ASSUME = 2, @@ -33,8 +33,7 @@ public: ASTs filterConstraints(ConstraintType selection) const; - const std::vector & getConstraints() const; - void updateConstraints(const std::vector & constraints); + const ASTs & getConstraints() const; const std::vector> & getConstraintData() const; std::vector getAtomicConstraintData() const; @@ -45,8 +44,8 @@ public: struct AtomId { - size_t and_group; - size_t atom; + size_t group_id; + size_t atom_id; }; using AtomIds = std::vector; @@ -59,7 +58,7 @@ private: std::unique_ptr buildGraph() const; void update(); - std::vector constraints; + ASTs constraints; std::vector> cnf_constraints; std::map ast_to_atom_ids; std::unique_ptr graph; diff --git a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp index 6d960c2acf3..2de96c76933 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp @@ -86,24 +86,6 @@ void MergeTreeIndexMergedCondition::addConstraints(const ConstraintsDescription } } -namespace -{ - -ComparisonGraph::CompareResult getExpectedCompare(const CNFQuery::AtomicFormula & atom) -{ - const auto * func = atom.ast->as(); - if (func) - { - auto expected = ComparisonGraph::getCompareResult(func->name); - if (atom.negative) - expected = ComparisonGraph::inverseCompareResult(expected); - return expected; - } - return ComparisonGraph::CompareResult::UNKNOWN; -} - -} - /// Replaces < -> <=, > -> >= and assumes that all hypotheses are true then checks if path exists bool MergeTreeIndexMergedCondition::alwaysUnknownOrTrue() const { @@ -183,7 +165,7 @@ bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranu const auto * func = atom.ast->as(); if (func && func->arguments->children.size() == 2) { - const auto expected = getExpectedCompare(atom); + const auto expected = ComparisonGraph::atomToCompareResult(atom); if (graph.isPossibleCompare(expected, func->arguments->children[0], func->arguments->children[1])) { /// If graph failed use matching. @@ -194,6 +176,7 @@ bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranu } always_false = true; }); + answerCache[values] = !always_false; return !always_false; } diff --git a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp index 25d4deefdaa..7a85791d172 100644 --- a/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp +++ b/src/Storages/MergeTree/MergeTreeWhereOptimizer.cpp @@ -14,8 +14,6 @@ #include #include -#include - namespace DB { diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index a0c4fe5b7ca..14872b123c7 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -711,7 +711,7 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.query.columns_list && args.query.columns_list->constraints) for (auto & constraint : args.query.columns_list->constraints->children) constraints.push_back(constraint); - metadata.constraints.updateConstraints(constraints); + metadata.constraints = ConstraintsDescription(constraints); auto column_ttl_asts = args.columns.getColumnTTLs(); for (const auto & [name, ast] : column_ttl_asts) diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index 133771a5d47..800d77ea8c6 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -28,6 +28,7 @@ 0 1 1 +0 1 1 0 diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql index d4ef006e668..e1922975a2a 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.sql +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.sql @@ -91,7 +91,8 @@ SELECT count() FROM constraint_test_constants WHERE 9 < b; ---> assumption -> 1 SELECT count() FROM constraint_test_constants WHERE 11 < b; ---> assumption -> 0 SELECT count() FROM constraint_test_constants WHERE 10 <= b; ---> assumption -> 1 SELECT count() FROM constraint_test_constants WHERE 9 < a; ---> assumption -> 1 -SELECT count() FROM constraint_test_constants WHERE 10 < a; ---> assumption -> 1 +SELECT count() FROM constraint_test_constants WHERE 10 < a; ---> assumption -> 0 +SELECT count() FROM constraint_test_constants WHERE 10 <= a; ---> assumption -> 1 SELECT count() FROM constraint_test_constants WHERE 9 <= a; ---> assumption -> 1 SELECT count() FROM constraint_test_constants WHERE 11 <= a; ---> assumption -> 0 diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.reference b/tests/queries/0_stateless/01622_constraints_where_optimization.reference new file mode 100644 index 00000000000..c7c516025f2 --- /dev/null +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.reference @@ -0,0 +1,14 @@ +SELECT count() +FROM t_constraints_where +WHERE 0 +SELECT count() +FROM t_constraints_where +WHERE 0 +SELECT count() +FROM t_constraints_where +WHERE 0 +SELECT count() +FROM t_constraints_where +WHERE b < 8 +SELECT count() +FROM t_constraints_where diff --git a/tests/queries/0_stateless/01622_constraints_where_optimization.sql b/tests/queries/0_stateless/01622_constraints_where_optimization.sql new file mode 100644 index 00000000000..6a9d1ba9f6b --- /dev/null +++ b/tests/queries/0_stateless/01622_constraints_where_optimization.sql @@ -0,0 +1,23 @@ +SET convert_query_to_cnf = 1; +SET optimize_using_constraints = 1; + +DROP TABLE IF EXISTS t_constraints_where; + +CREATE TABLE t_constraints_where(a UInt32, b UInt32, CONSTRAINT c1 ASSUME b >= 5, CONSTRAINT c2 ASSUME b <= 10) ENGINE = Memory; + +INSERT INTO t_constraints_where VALUES (1, 7); + +EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b > 15; -- assumption -> 0 +EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b = 20; -- assumption -> 0 +EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b < 2; -- assumption -> 0 +EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b > 20 OR b < 8; -- assumption -> remove (b < 20) + +DROP TABLE t_constraints_where; + +CREATE TABLE t_constraints_where(a UInt32, b UInt32, CONSTRAINT c1 ASSUME b < 10) ENGINE = Memory; + +INSERT INTO t_constraints_where VALUES (1, 7); + +EXPLAIN SYNTAX SELECT count() FROM t_constraints_where WHERE b = 1 OR b < 18 OR b > 5; -- assumtion -> (b < 20) -> 0; + +DROP TABLE t_constraints_where; diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index f35246a59cd..c287ed073fc 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -49,3 +49,5 @@ WHERE a = \'c\' SELECT a AS `substring(reverse(b), 1, 1)` FROM column_swap_test_test WHERE a = \'c\' +SELECT toUInt32(s) AS a +FROM t_bad_constraint diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.sql b/tests/queries/0_stateless/01623_constraints_column_swap.sql index ee48cfbbd3e..0fb0d417a43 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.sql +++ b/tests/queries/0_stateless/01623_constraints_column_swap.sql @@ -30,3 +30,13 @@ EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) AS t1, a AS t2 FROM column_swa EXPLAIN SYNTAX SELECT substring(reverse(b), 1, 1) FROM column_swap_test_test WHERE substring(reverse(b), 1, 1) = 'c'; DROP TABLE column_swap_test_test; + +DROP TABLE IF EXISTS t_bad_constraint; + +CREATE TABLE t_bad_constraint(a UInt32, s String, CONSTRAINT c1 ASSUME a = toUInt32(s)) ENGINE = MergeTree ORDER BY tuple(); + +INSERT INTO t_bad_constraint SELECT number, randomPrintableASCII(100) FROM numbers(10000); + +EXPLAIN SYNTAX SELECT a FROM t_bad_constraint; + +DROP TABLE t_bad_constraint; From 3a814d5a2c592cbd5ee795bbd93a1564472ffa6f Mon Sep 17 00:00:00 2001 From: cgp Date: Thu, 18 Nov 2021 00:06:37 +0800 Subject: [PATCH 119/200] replace link of List of tasks --- docs/en/development/developer-instruction.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index 024ce27d60d..e4b51242be8 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -241,7 +241,7 @@ Adding third-party libraries: https://clickhouse.com/docs/en/development/contrib Writing tests: https://clickhouse.com/docs/en/development/tests/ -List of tasks: https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aopen+is%3Aissue+label%3A%22easy+task%22 +List of tasks: https://github.com/ClickHouse/ClickHouse/issues?q=is%3Aopen+is%3Aissue+label%3Ahacktoberfest ## Test Data {#test-data} From 90cddb0393ddb8c2d1e5247d24bb32bb970c6f3a Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Wed, 17 Nov 2021 21:25:39 +0300 Subject: [PATCH 120/200] [docs] probably fix list styles (#31482) * [docs] probably fix list styles * Update main.css --- website/css/main.css | 2 +- website/src/scss/components/_ul.scss | 1 - 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/website/css/main.css b/website/css/main.css index 73ff758625f..614422689a9 100644 --- a/website/css/main.css +++ b/website/css/main.css @@ -1 +1 @@ -@media screen and (max-width:978.98px){.btn{padding:8px 16px}}@media screen and (max-width:978.98px){.btn-lg{padding:12px 24px}}.btn-primary,.btn-primary:active,.btn-primary:hover{color:#212529}.btn-outline-primary{background:#fffaf0;border-color:#fc0;color:#212529}.btn-outline-primary:active,.btn-outline-primary:hover{background:#fc0;border-color:#fc0;color:#212529}.btn-secondary{border-color:#212529;color:#fff}.btn-outline-secondary,.btn-secondary:active,.btn-secondary:hover{background:#fff;border-color:#212529;color:#212529}.btn-outline-secondary:active,.btn-outline-secondary:hover{background:#212529;border-color:#212529;color:#fff}.btn-tertiary{border-color:#257af4;color:#fff}.btn-tertiary:active,.btn-tertiary:hover{background:#257af4;border-color:#257af4;color:#fff}.btn-outline-tertiary{background:#e3f1fe;color:#257af4}.btn-outline-tertiary:active,.btn-outline-tertiary:hover{background:#257af4;color:#fff}.btns{align-items:center;display:grid;-moz-column-gap:24px;column-gap:24px;row-gap:16px;grid-auto-flow:column;justify-content:center}@media screen and (max-width:767.98px){.btns{grid-auto-flow:row}}.btns.btns-lg{-moz-column-gap:40px;column-gap:40px}.btns.is-2{grid-template-columns:1fr 1fr}@media screen and (max-width:767.98px){.btns.is-2{grid-template-columns:1fr}}.btns.is-3{grid-template-columns:1fr 1fr 1fr}@media screen and (max-width:767.98px){.btns.is-3{grid-template-columns:1fr}}.card{box-shadow:0 8px 20px rgba(108,117,125,.2);overflow:hidden;transition:box-shadow .2s,transform .2s;width:100%}.card,.card-body{position:relative}.card-body{z-index:10}.card.is-large .card-body{padding:40px}.card.bg-primary-light{border-color:#fc0}.card.has-dark-border{border-color:#6c757d}.card.has-pattern:after,.card.has-pattern:before{background-repeat:no-repeat;background-size:auto 100%;bottom:0;content:"";display:block;position:absolute;top:0;width:72px}.card.has-pattern:before{background-image:url(../images/backgrounds/bg-card-pattern-blue-1.png);background-position:0 0;left:0}.card.has-pattern:after{background-image:url(../images/backgrounds/bg-card-pattern-blue-2.png);background-position:100% 0;right:0}.card.has-hover:active,.card.has-hover:hover,a.card:active,a.card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2);transform:translateY(-8px)}.card.has-highlight:after,.card.has-hover:after,a.card:after{content:"";display:block;height:8px;margin-top:auto;transition:background .2s;width:100%}.card.has-highlight:after,.card.has-hover:active:after,.card.has-hover:hover:after,a.card:active:after,a.card:hover:after{background:#e3f1fe}.case-study-cards{-moz-column-gap:40px;column-gap:40px;display:grid;grid-template-columns:1fr;row-gap:40px;padding-bottom:40px;position:relative}.case-study-cards>div{align-items:stretch;display:flex}.case-study-cards:before{background:#d6dbdf;bottom:0;content:"";display:block;left:20px;position:absolute;top:40px;width:100vw}@media screen and (min-width:980px){.case-study-cards{grid-template-columns:repeat(2,minmax(0,1fr));row-gap:80px;padding-bottom:120px}.case-study-cards:before{left:-40px;top:120px}}.case-study-card{align-items:stretch;flex-direction:row;flex-shrink:0;left:0;transition:box-shadow .2s,left .4s,width .4s,z-index 0s;transition-delay:0s,.6s,.6s,0s;width:100%;z-index:2}@media screen and (max-width:979.98px){.case-study-card .row{min-height:0!important}}@media screen and (min-width:980px){.case-study-card:active,.case-study-card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2)}.case-study-card:not(.is-open){cursor:pointer}.case-study-card.is-open{transform:none!important;transition-delay:0s,0s,0s,0s;width:calc(200% + 40px);z-index:10}.case-study-card.is-closing{z-index:10}.case-study-card.open-left.is-open{left:calc(-100% - 40px)}.case-study-card:before{background:no-repeat url(../images/backgrounds/bg-card-pattern-red.png);background-position:100%;background-size:contain;content:"";display:block;height:calc(100% - 80px);max-height:224px;max-width:234px;position:absolute;right:0;top:40px;transform:translateX(30%);transition:transform .4s;transition-delay:.6s;width:100%;z-index:1}}@media screen and (min-width:980px)and (min-width:1240px){.case-study-card:before{transform:translateX(50%)}}@media screen and (min-width:980px){.case-study-card.is-open:before{transform:translateX(70%);transition-delay:0s}}@media screen and (min-width:980px){.case-study-card-wrap{align-items:stretch;display:flex;flex-shrink:0;min-height:304px;position:relative;transition:width .4s;transition-delay:.6s;width:calc(200% + 42px);z-index:2}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-wrap{transition-delay:0s;width:100%}}@media screen and (min-width:980px){.case-study-card-body{display:flex;flex-direction:column;padding-right:80px!important}.case-study-card-body>.row{align-self:stretch}}@media screen and (min-width:980px){.case-study-card-toggle{background:#fff;box-shadow:0 8px 20px rgba(108,117,125,.2);border-radius:100%;cursor:pointer;height:56px;position:relative;width:56px}.case-study-card-toggle:after,.case-study-card-toggle:before{background:#257af4;content:"";display:block;height:4px;left:calc(50% - 15px);position:absolute;top:calc(50% - 2px);transition:opacity .2s,transform .2s;width:30px}.case-study-card-toggle:after{transform:rotate(90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:before{opacity:0;transform:rotate(-90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:after{transform:rotate(0)}}@media screen and (min-width:980px){.case-study-card .col-lg-3{left:-60%;position:relative;transition:left .4s;transition-delay:.6s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-3{flex:0 0 250px;max-width:250px;width:250px}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-3{left:0;transition-delay:0s}}@media screen and (min-width:980px){.case-study-card .col-lg-auto{opacity:0;transform:translateX(24px);transition:opacity .4s,transform .4s;transition-delay:.2s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-auto{max-width:605px;width:calc(100% - 319px)}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-auto{opacity:1;transform:none;transition-delay:.2s}}.footer-copy{white-space:nowrap}form .form-control{border:1px solid #6c757d;border-radius:6px;height:auto;line-height:20px;min-height:44px;padding:12px 16px;width:100%}form .form-control,form .form-control:focus{box-shadow:0 8px 20px rgba(108,117,125,.2);color:#212529}form .form-control:focus{border-color:#212529}form .form-control::-moz-placeholder{color:#6c757d}form .form-control:-ms-input-placeholder{color:#6c757d}form .form-control::placeholder{color:#6c757d}form select.form-control{-webkit-appearance:none;-moz-appearance:none;appearance:none}form select.form-control:not([data-chosen]){color:#6c757d}form .btn-secondary:active,form .btn-secondary:hover{color:#212529;background:#fc0;border-color:#fc0}.hero{overflow:visible;position:relative}.hero,.hero-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.hero-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.hero>.container{position:relative;z-index:2}.hero.has-offset{margin-bottom:-160px;padding-bottom:160px}.base-hero{height:22.5vw;max-height:324px;min-height:280px}.index-hero{background-image:url(/images/backgrounds/bg-hero-home.svg);height:68vw;max-height:980px}.index-hero,.other-hero{max-width:2448px;width:170vw}.other-hero{background-image:url(/images/backgrounds/bg-hero.svg)}.bg-footer-cta{background-image:url(/images/backgrounds/bg-footer-cta.svg);width:2448px}.quickstart-bg{background-image:url(/images/backgrounds/bg-quick-start.svg);height:40vw;top:220px;width:170vw}hr{background:#f1f6f9;border:0;display:block;height:4px;margin:0;width:100%}hr.is-small{height:2px}hr.is-large{height:8px}hr.is-medium{background:#d6dbdf}hr.is-dark{background:#495057}hr.is-yellow{background:linear-gradient(90deg,#ff8c00,#ff8c00 8px,#fc0 16px,rgba(255,204,0,0));-webkit-clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);height:8px}.icon{display:block;height:48px;margin-bottom:24px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center}@media screen and (min-width:576px){.icon{height:64px}}@media screen and (min-width:980px){.icon{height:80px}}img{max-width:100%}.kicker{color:#6c757d;font-family:Hind Siliguri,sans-serif;font-size:.875rem;font-weight:600;letter-spacing:1px;margin:0}@media screen and (max-width:978.98px){.lead{font-size:1.125rem}}.logo{display:block;height:36px;max-width:220px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;width:100%}.navbar-clickhouse{border-bottom:4px solid #f1f6f9;height:142px}.navbar-clickhouse>.container{flex-wrap:wrap}.navbar-super{flex-shrink:0;width:100%}.navbar-super ul{list-style:none}.navbar-super li:not(:last-child){margin-bottom:0;margin-right:24px}.navbar-super a{align-items:center;color:#212529;display:flex;font-size:.875rem}.navbar-super a:active,.navbar-super a:hover{color:#257af4;text-decoration:none}.navbar-super img{flex-shrink:0;margin-right:4px}.navbar-brand-clickhouse{background:no-repeat url(../images/logo-clickhouse.svg);background-size:contain;flex-shrink:0;height:28px;margin-right:48px;padding:0;width:180px}.navbar-nav{align-items:center;height:46px}.navbar .nav-item:not(:last-child){margin-bottom:0;margin-right:24px}.navbar .nav-link{color:#212529}.navbar .nav-link:active,.navbar .nav-link:hover{color:#257af4}.navbar .navbar-nav{flex-direction:row}@media screen and (max-width:978.98px){.navbar>.container{padding-left:20px;padding-right:20px}.navbar .navbar-toggler{height:24px;padding:0;width:24px}.navbar .navbar-toggler:focus{outline:none}.navbar .navbar-toggler-icon{background:no-repeat url(../images/icons/icon-menu.svg);background-position:50%;background-size:contain;height:24px;width:24px}.navbar .navbar-collapse{background:#fff;border-bottom:4px solid #f1f6f9;height:56px;left:0;padding:0 20px 16px;position:absolute;right:0;top:100%}.navbar .nav-link{font-size:.875rem;white-space:nowrap}}@media screen and (max-width:615.98px){.navbar .navbar-collapse{height:auto}.navbar .navbar-nav{flex-direction:column;height:auto}.navbar .nav-item:not(:last-child){margin-bottom:16px;margin-right:0}}@media screen and (max-width:399.98px){.navbar{height:80px}}.page,.photo-frame{overflow:hidden;width:100%}.photo-frame{background:hsla(0,0%,100%,.6);border-radius:100%;box-shadow:0 8px 20px rgba(108,117,125,.2);display:block;margin-bottom:24px;max-width:160px;position:relative}.photo-frame:before{content:"";display:block;padding-bottom:100%;width:100%}.photo-frame img{display:block;height:100%;left:0;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;position:absolute;top:0;width:100%}.pullquote{position:relative;width:70%}.pullquote:before{background:no-repeat url(../images/backgrounds/bg-quotes.svg);background-position:50%;background-size:100%;content:"";mix-blend-mode:multiply;right:56px;width:calc(100% - 16px);z-index:2}.pullquote-bg,.pullquote:before{bottom:0;display:block;position:absolute;top:0}.pullquote-bg{right:0;width:calc(50vw + 28.57143%);z-index:1}.pullquote-body{padding:64px 40px 64px 0;position:relative;z-index:3}.pullquote-quote{font-family:Hind Siliguri,sans-serif;font-size:32px;font-weight:700}.pullquote-citation{font-size:1.125rem}.section{overflow:visible;position:relative}.section,.section-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.section-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.section>.container{position:relative;z-index:2}.social-icons{align-items:center;display:flex}.social-icons>a{aspect-ratio:24/24;background:#6c757d;display:block;height:24px;width:24px;-webkit-mask-position:center;mask-position:center;-webkit-mask-repeat:no-repeat;mask-repeat:no-repeat;-webkit-mask-size:contain;mask-size:contain;transition:background .2s}.social-icons>a:active,.social-icons>a:hover{background:#212529}.social-icons>a+a{margin-left:32px}.social-icons-facebook{-webkit-mask-image:url(/images/icons/icon-facebook-gray.svg);mask-image:url(/images/icons/icon-facebook-gray.svg)}.social-icons-twitter{-webkit-mask-image:url(/images/icons/icon-twitter-gray.svg);mask-image:url(/images/icons/icon-twitter-gray.svg);width:31px}.social-icons-linkedin{-webkit-mask-image:url(/images/icons/icon-linkedin-gray.svg);mask-image:url(/images/icons/icon-linkedin-gray.svg)}.social-icons-linkedin-alt{-webkit-mask-image:url(/images/icons/icon-linkedin-alt-gray.svg);mask-image:url(/images/icons/icon-linkedin-alt-gray.svg)}.social-icons.size-small>a{height:20px;width:20px}.social-icons.size-small>a:active,.social-icons.size-small>a:hover{background:#212529}.social-icons.size-small>a+a{margin-left:16px}.tabs{position:relative}.tabs:before{background:#fff;border-radius:7px 7px 0 0;content:"";display:block;height:8px;left:1px;position:absolute;right:1px;top:68px;z-index:10}@media screen and (min-width:1240px){.tabs:before{top:76px}}.tabs-body{background:#fff;border-radius:8px;border:1px solid #6c757d;box-shadow:0 8px 20px rgba(108,117,125,.2);padding:24px}@media screen and (min-width:980px){.tabs-body{padding:32px}}@media screen and (min-width:1240px){.tabs-body{padding:40px}}.tabs .nav-tabs{border-bottom:0;flex-wrap:nowrap;height:76px;margin:-20px -20px -9px;-webkit-mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);overflow:scroll;overflow-x:scroll;overflow-y:visible;padding:20px 20px 0;position:relative}@media screen and (min-width:940px){.tabs .nav-tabs{overflow:visible}}@media screen and (min-width:1240px){.tabs .nav-tabs{height:84px}}.tabs .nav-link{align-items:center;border-bottom:0;color:#6c757d;display:flex;font-size:.875rem;flex-shrink:0;height:56px;justify-content:center;padding:0 12px 8px;text-align:center;white-space:nowrap}@media screen and (min-width:1240px){.tabs .nav-link{height:64px;padding:0 16px 8px}}.tabs .nav-link.active{background:#fff;box-shadow:0 -4px 8px rgba(108,117,125,.1);font-weight:700;padding:0 16px 8px}@media screen and (min-width:980px){.tabs .nav-link.active{padding:0 24px 8px}}@media screen and (min-width:1240px){.tabs .nav-link.active{padding:0 32px 8px}}.tab-pane pre{background:#212529;border-radius:16px;color:#fff;padding:24px 16px}@media screen and (min-width:1240px){.tab-pane pre{padding:32px 24px}}.trailing-link{align-items:center;color:#212529;display:flex;font-size:.875rem;font-weight:700}.trailing-link:after{background:no-repeat url(../images/icons/icon-arrow.svg);background-position:100%;background-size:contain;content:"";display:block;height:12px;transition:transform .2s;width:20px}.trailing-link:active,.trailing-link:hover{color:#212529;text-decoration:none}.trailing-link:active:after,.trailing-link:hover:after{transform:translateX(8px)}.trailing-link.span-full:after{margin-left:auto}ul{color:#495057;list-style-type:square;padding-left:1.25em}ul li:not(:last-child){margin-bottom:16px}ul li::marker{color:#ff3939}ul.has-separators{list-style:none;padding:0}ul.has-separators li:not(:last-child){border-bottom:4px solid #f1f6f9;margin-bottom:24px;padding-bottom:24px}.bg-gradient-secondary{background-image:linear-gradient(58deg,#ff6443 3%,#fe561d 24%,#e32f0d 93%)}.bg-gradient-light-orange{background-image:linear-gradient(90deg,rgba(255,203,128,0),#ffcb80)}.bg-offset-right{bottom:0;left:-24px;position:absolute;top:0;width:calc(100vw + 24px);z-index:-1}@media screen and (min-width:1240px){.bg-offset-right{left:-96px;width:calc(100vw + 96px)}}.bg-inset-right{bottom:0;left:40px;position:absolute;top:0;width:calc(100vw - 40px);z-index:-1}@media screen and (min-width:980px){.bg-inset-right{left:96px;width:calc(100vw - 96px)}}.has-border-left{border-left:8px solid #f1f6f9;padding-left:16px}.font-xl{font-size:1.25rem}.font-lg{font-size:1.125rem}.font-sm{font-size:.875rem}.font-xs{font-size:.625rem}.font-weight-semibold{font-weight:600}.display-5{color:#212529;font-size:20px;font-weight:500}.display-6{color:#212529;font-size:14px;font-weight:700}.overflow-auto{overflow:auto}.text-decoration-underline{text-decoration:underline}.text-upper{text-transform:uppercase} \ No newline at end of file +@media screen and (max-width:978.98px){.btn{padding:8px 16px}}@media screen and (max-width:978.98px){.btn-lg{padding:12px 24px}}.btn-primary,.btn-primary:active,.btn-primary:hover{color:#212529}.btn-outline-primary{background:#fffaf0;border-color:#fc0;color:#212529}.btn-outline-primary:active,.btn-outline-primary:hover{background:#fc0;border-color:#fc0;color:#212529}.btn-secondary{border-color:#212529;color:#fff}.btn-outline-secondary,.btn-secondary:active,.btn-secondary:hover{background:#fff;border-color:#212529;color:#212529}.btn-outline-secondary:active,.btn-outline-secondary:hover{background:#212529;border-color:#212529;color:#fff}.btn-tertiary{border-color:#257af4;color:#fff}.btn-tertiary:active,.btn-tertiary:hover{background:#257af4;border-color:#257af4;color:#fff}.btn-outline-tertiary{background:#e3f1fe;color:#257af4}.btn-outline-tertiary:active,.btn-outline-tertiary:hover{background:#257af4;color:#fff}.btns{align-items:center;display:grid;-moz-column-gap:24px;column-gap:24px;row-gap:16px;grid-auto-flow:column;justify-content:center}@media screen and (max-width:767.98px){.btns{grid-auto-flow:row}}.btns.btns-lg{-moz-column-gap:40px;column-gap:40px}.btns.is-2{grid-template-columns:1fr 1fr}@media screen and (max-width:767.98px){.btns.is-2{grid-template-columns:1fr}}.btns.is-3{grid-template-columns:1fr 1fr 1fr}@media screen and (max-width:767.98px){.btns.is-3{grid-template-columns:1fr}}.card{box-shadow:0 8px 20px rgba(108,117,125,.2);overflow:hidden;transition:box-shadow .2s,transform .2s;width:100%}.card,.card-body{position:relative}.card-body{z-index:10}.card.is-large .card-body{padding:40px}.card.bg-primary-light{border-color:#fc0}.card.has-dark-border{border-color:#6c757d}.card.has-pattern:after,.card.has-pattern:before{background-repeat:no-repeat;background-size:auto 100%;bottom:0;content:"";display:block;position:absolute;top:0;width:72px}.card.has-pattern:before{background-image:url(../images/backgrounds/bg-card-pattern-blue-1.png);background-position:0 0;left:0}.card.has-pattern:after{background-image:url(../images/backgrounds/bg-card-pattern-blue-2.png);background-position:100% 0;right:0}.card.has-hover:active,.card.has-hover:hover,a.card:active,a.card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2);transform:translateY(-8px)}.card.has-highlight:after,.card.has-hover:after,a.card:after{content:"";display:block;height:8px;margin-top:auto;transition:background .2s;width:100%}.card.has-highlight:after,.card.has-hover:active:after,.card.has-hover:hover:after,a.card:active:after,a.card:hover:after{background:#e3f1fe}.case-study-cards{-moz-column-gap:40px;column-gap:40px;display:grid;grid-template-columns:1fr;row-gap:40px;padding-bottom:40px;position:relative}.case-study-cards>div{align-items:stretch;display:flex}.case-study-cards:before{background:#d6dbdf;bottom:0;content:"";display:block;left:20px;position:absolute;top:40px;width:100vw}@media screen and (min-width:980px){.case-study-cards{grid-template-columns:repeat(2,minmax(0,1fr));row-gap:80px;padding-bottom:120px}.case-study-cards:before{left:-40px;top:120px}}.case-study-card{align-items:stretch;flex-direction:row;flex-shrink:0;left:0;transition:box-shadow .2s,left .4s,width .4s,z-index 0s;transition-delay:0s,.6s,.6s,0s;width:100%;z-index:2}@media screen and (max-width:979.98px){.case-study-card .row{min-height:0!important}}@media screen and (min-width:980px){.case-study-card:active,.case-study-card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2)}.case-study-card:not(.is-open){cursor:pointer}.case-study-card.is-open{transform:none!important;transition-delay:0s,0s,0s,0s;width:calc(200% + 40px);z-index:10}.case-study-card.is-closing{z-index:10}.case-study-card.open-left.is-open{left:calc(-100% - 40px)}.case-study-card:before{background:no-repeat url(../images/backgrounds/bg-card-pattern-red.png);background-position:100%;background-size:contain;content:"";display:block;height:calc(100% - 80px);max-height:224px;max-width:234px;position:absolute;right:0;top:40px;transform:translateX(30%);transition:transform .4s;transition-delay:.6s;width:100%;z-index:1}}@media screen and (min-width:980px)and (min-width:1240px){.case-study-card:before{transform:translateX(50%)}}@media screen and (min-width:980px){.case-study-card.is-open:before{transform:translateX(70%);transition-delay:0s}}@media screen and (min-width:980px){.case-study-card-wrap{align-items:stretch;display:flex;flex-shrink:0;min-height:304px;position:relative;transition:width .4s;transition-delay:.6s;width:calc(200% + 42px);z-index:2}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-wrap{transition-delay:0s;width:100%}}@media screen and (min-width:980px){.case-study-card-body{display:flex;flex-direction:column;padding-right:80px!important}.case-study-card-body>.row{align-self:stretch}}@media screen and (min-width:980px){.case-study-card-toggle{background:#fff;box-shadow:0 8px 20px rgba(108,117,125,.2);border-radius:100%;cursor:pointer;height:56px;position:relative;width:56px}.case-study-card-toggle:after,.case-study-card-toggle:before{background:#257af4;content:"";display:block;height:4px;left:calc(50% - 15px);position:absolute;top:calc(50% - 2px);transition:opacity .2s,transform .2s;width:30px}.case-study-card-toggle:after{transform:rotate(90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:before{opacity:0;transform:rotate(-90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:after{transform:rotate(0)}}@media screen and (min-width:980px){.case-study-card .col-lg-3{left:-60%;position:relative;transition:left .4s;transition-delay:.6s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-3{flex:0 0 250px;max-width:250px;width:250px}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-3{left:0;transition-delay:0s}}@media screen and (min-width:980px){.case-study-card .col-lg-auto{opacity:0;transform:translateX(24px);transition:opacity .4s,transform .4s;transition-delay:.2s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-auto{max-width:605px;width:calc(100% - 319px)}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-auto{opacity:1;transform:none;transition-delay:.2s}}.footer-copy{white-space:nowrap}form .form-control{border:1px solid #6c757d;border-radius:6px;height:auto;line-height:20px;min-height:44px;padding:12px 16px;width:100%}form .form-control,form .form-control:focus{box-shadow:0 8px 20px rgba(108,117,125,.2);color:#212529}form .form-control:focus{border-color:#212529}form .form-control::-moz-placeholder{color:#6c757d}form .form-control:-ms-input-placeholder{color:#6c757d}form .form-control::placeholder{color:#6c757d}form select.form-control{-webkit-appearance:none;-moz-appearance:none;appearance:none}form select.form-control:not([data-chosen]){color:#6c757d}form .btn-secondary:active,form .btn-secondary:hover{color:#212529;background:#fc0;border-color:#fc0}.hero{overflow:visible;position:relative}.hero,.hero-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.hero-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.hero>.container{position:relative;z-index:2}.hero.has-offset{margin-bottom:-160px;padding-bottom:160px}.base-hero{height:22.5vw;max-height:324px;min-height:280px}.index-hero{background-image:url(/images/backgrounds/bg-hero-home.svg);height:68vw;max-height:980px}.index-hero,.other-hero{max-width:2448px;width:170vw}.other-hero{background-image:url(/images/backgrounds/bg-hero.svg)}.bg-footer-cta{background-image:url(/images/backgrounds/bg-footer-cta.svg);width:2448px}.quickstart-bg{background-image:url(/images/backgrounds/bg-quick-start.svg);height:40vw;top:220px;width:170vw}hr{background:#f1f6f9;border:0;display:block;height:4px;margin:0;width:100%}hr.is-small{height:2px}hr.is-large{height:8px}hr.is-medium{background:#d6dbdf}hr.is-dark{background:#495057}hr.is-yellow{background:linear-gradient(90deg,#ff8c00,#ff8c00 8px,#fc0 16px,rgba(255,204,0,0));-webkit-clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);height:8px}.icon{display:block;height:48px;margin-bottom:24px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center}@media screen and (min-width:576px){.icon{height:64px}}@media screen and (min-width:980px){.icon{height:80px}}img{max-width:100%}.kicker{color:#6c757d;font-family:Hind Siliguri,sans-serif;font-size:.875rem;font-weight:600;letter-spacing:1px;margin:0}@media screen and (max-width:978.98px){.lead{font-size:1.125rem}}.logo{display:block;height:36px;max-width:220px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;width:100%}.navbar-clickhouse{border-bottom:4px solid #f1f6f9;height:142px}.navbar-clickhouse>.container{flex-wrap:wrap}.navbar-super{flex-shrink:0;width:100%}.navbar-super ul{list-style:none}.navbar-super li:not(:last-child){margin-bottom:0;margin-right:24px}.navbar-super a{align-items:center;color:#212529;display:flex;font-size:.875rem}.navbar-super a:active,.navbar-super a:hover{color:#257af4;text-decoration:none}.navbar-super img{flex-shrink:0;margin-right:4px}.navbar-brand-clickhouse{background:no-repeat url(../images/logo-clickhouse.svg);background-size:contain;flex-shrink:0;height:28px;margin-right:48px;padding:0;width:180px}.navbar-nav{align-items:center;height:46px}.navbar .nav-item:not(:last-child){margin-bottom:0;margin-right:24px}.navbar .nav-link{color:#212529}.navbar .nav-link:active,.navbar .nav-link:hover{color:#257af4}.navbar .navbar-nav{flex-direction:row}@media screen and (max-width:978.98px){.navbar>.container{padding-left:20px;padding-right:20px}.navbar .navbar-toggler{height:24px;padding:0;width:24px}.navbar .navbar-toggler:focus{outline:none}.navbar .navbar-toggler-icon{background:no-repeat url(../images/icons/icon-menu.svg);background-position:50%;background-size:contain;height:24px;width:24px}.navbar .navbar-collapse{background:#fff;border-bottom:4px solid #f1f6f9;height:56px;left:0;padding:0 20px 16px;position:absolute;right:0;top:100%}.navbar .nav-link{font-size:.875rem;white-space:nowrap}}@media screen and (max-width:615.98px){.navbar .navbar-collapse{height:auto}.navbar .navbar-nav{flex-direction:column;height:auto}.navbar .nav-item:not(:last-child){margin-bottom:16px;margin-right:0}}@media screen and (max-width:399.98px){.navbar{height:80px}}.page,.photo-frame{overflow:hidden;width:100%}.photo-frame{background:hsla(0,0%,100%,.6);border-radius:100%;box-shadow:0 8px 20px rgba(108,117,125,.2);display:block;margin-bottom:24px;max-width:160px;position:relative}.photo-frame:before{content:"";display:block;padding-bottom:100%;width:100%}.photo-frame img{display:block;height:100%;left:0;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;position:absolute;top:0;width:100%}.pullquote{position:relative;width:70%}.pullquote:before{background:no-repeat url(../images/backgrounds/bg-quotes.svg);background-position:50%;background-size:100%;content:"";mix-blend-mode:multiply;right:56px;width:calc(100% - 16px);z-index:2}.pullquote-bg,.pullquote:before{bottom:0;display:block;position:absolute;top:0}.pullquote-bg{right:0;width:calc(50vw + 28.57143%);z-index:1}.pullquote-body{padding:64px 40px 64px 0;position:relative;z-index:3}.pullquote-quote{font-family:Hind Siliguri,sans-serif;font-size:32px;font-weight:700}.pullquote-citation{font-size:1.125rem}.section{overflow:visible;position:relative}.section,.section-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.section-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.section>.container{position:relative;z-index:2}.social-icons{align-items:center;display:flex}.social-icons>a{aspect-ratio:24/24;background:#6c757d;display:block;height:24px;width:24px;-webkit-mask-position:center;mask-position:center;-webkit-mask-repeat:no-repeat;mask-repeat:no-repeat;-webkit-mask-size:contain;mask-size:contain;transition:background .2s}.social-icons>a:active,.social-icons>a:hover{background:#212529}.social-icons>a+a{margin-left:32px}.social-icons-facebook{-webkit-mask-image:url(/images/icons/icon-facebook-gray.svg);mask-image:url(/images/icons/icon-facebook-gray.svg)}.social-icons-twitter{-webkit-mask-image:url(/images/icons/icon-twitter-gray.svg);mask-image:url(/images/icons/icon-twitter-gray.svg);width:31px}.social-icons-linkedin{-webkit-mask-image:url(/images/icons/icon-linkedin-gray.svg);mask-image:url(/images/icons/icon-linkedin-gray.svg)}.social-icons-linkedin-alt{-webkit-mask-image:url(/images/icons/icon-linkedin-alt-gray.svg);mask-image:url(/images/icons/icon-linkedin-alt-gray.svg)}.social-icons.size-small>a{height:20px;width:20px}.social-icons.size-small>a:active,.social-icons.size-small>a:hover{background:#212529}.social-icons.size-small>a+a{margin-left:16px}.tabs{position:relative}.tabs:before{background:#fff;border-radius:7px 7px 0 0;content:"";display:block;height:8px;left:1px;position:absolute;right:1px;top:68px;z-index:10}@media screen and (min-width:1240px){.tabs:before{top:76px}}.tabs-body{background:#fff;border-radius:8px;border:1px solid #6c757d;box-shadow:0 8px 20px rgba(108,117,125,.2);padding:24px}@media screen and (min-width:980px){.tabs-body{padding:32px}}@media screen and (min-width:1240px){.tabs-body{padding:40px}}.tabs .nav-tabs{border-bottom:0;flex-wrap:nowrap;height:76px;margin:-20px -20px -9px;-webkit-mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);overflow:scroll;overflow-x:scroll;overflow-y:visible;padding:20px 20px 0;position:relative}@media screen and (min-width:940px){.tabs .nav-tabs{overflow:visible}}@media screen and (min-width:1240px){.tabs .nav-tabs{height:84px}}.tabs .nav-link{align-items:center;border-bottom:0;color:#6c757d;display:flex;font-size:.875rem;flex-shrink:0;height:56px;justify-content:center;padding:0 12px 8px;text-align:center;white-space:nowrap}@media screen and (min-width:1240px){.tabs .nav-link{height:64px;padding:0 16px 8px}}.tabs .nav-link.active{background:#fff;box-shadow:0 -4px 8px rgba(108,117,125,.1);font-weight:700;padding:0 16px 8px}@media screen and (min-width:980px){.tabs .nav-link.active{padding:0 24px 8px}}@media screen and (min-width:1240px){.tabs .nav-link.active{padding:0 32px 8px}}.tab-pane pre{background:#212529;border-radius:16px;color:#fff;padding:24px 16px}@media screen and (min-width:1240px){.tab-pane pre{padding:32px 24px}}.trailing-link{align-items:center;color:#212529;display:flex;font-size:.875rem;font-weight:700}.trailing-link:after{background:no-repeat url(../images/icons/icon-arrow.svg);background-position:100%;background-size:contain;content:"";display:block;height:12px;transition:transform .2s;width:20px}.trailing-link:active,.trailing-link:hover{color:#212529;text-decoration:none}.trailing-link:active:after,.trailing-link:hover:after{transform:translateX(8px)}.trailing-link.span-full:after{margin-left:auto}ul{list-style-type:square;padding-left:1.25em}ul li:not(:last-child){margin-bottom:16px}ul li::marker{color:#ff3939}ul.has-separators{list-style:none;padding:0}ul.has-separators li:not(:last-child){border-bottom:4px solid #f1f6f9;margin-bottom:24px;padding-bottom:24px}.bg-gradient-secondary{background-image:linear-gradient(58deg,#ff6443 3%,#fe561d 24%,#e32f0d 93%)}.bg-gradient-light-orange{background-image:linear-gradient(90deg,rgba(255,203,128,0),#ffcb80)}.bg-offset-right{bottom:0;left:-24px;position:absolute;top:0;width:calc(100vw + 24px);z-index:-1}@media screen and (min-width:1240px){.bg-offset-right{left:-96px;width:calc(100vw + 96px)}}.bg-inset-right{bottom:0;left:40px;position:absolute;top:0;width:calc(100vw - 40px);z-index:-1}@media screen and (min-width:980px){.bg-inset-right{left:96px;width:calc(100vw - 96px)}}.has-border-left{border-left:8px solid #f1f6f9;padding-left:16px}.font-xl{font-size:1.25rem}.font-lg{font-size:1.125rem}.font-sm{font-size:.875rem}.font-xs{font-size:.625rem}.font-weight-semibold{font-weight:600}.display-5{color:#212529;font-size:20px;font-weight:500}.display-6{color:#212529;font-size:14px;font-weight:700}.overflow-auto{overflow:auto}.text-decoration-underline{text-decoration:underline}.text-upper{text-transform:uppercase} diff --git a/website/src/scss/components/_ul.scss b/website/src/scss/components/_ul.scss index 9cf03aba169..2dc92a0d211 100644 --- a/website/src/scss/components/_ul.scss +++ b/website/src/scss/components/_ul.scss @@ -1,5 +1,4 @@ ul { - color: $gray-900; list-style-type: square; padding-left: 1.25em; From bba62d9c56b74f3dbbd3483ac04493038597e262 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Wed, 17 Nov 2021 16:02:56 -0700 Subject: [PATCH 121/200] Add new team members to company page (#31491) --- website/images/photos/christoph-wurm.jpg | Bin 0 -> 20591 bytes website/images/photos/mikhail-shiryaev.jpg | Bin 0 -> 12923 bytes website/templates/company/team.html | 32 +++++++++++++++++++-- 3 files changed, 29 insertions(+), 3 deletions(-) create mode 100644 website/images/photos/christoph-wurm.jpg create mode 100644 website/images/photos/mikhail-shiryaev.jpg diff --git a/website/images/photos/christoph-wurm.jpg b/website/images/photos/christoph-wurm.jpg new file mode 100644 index 0000000000000000000000000000000000000000..8a9adac2fb0ce2b45b3d3217bf5aa7594202b65d GIT binary patch literal 20591 zcmb4qV{|1=6zz>|TQ|0yiEZ1S=*FChZ95a&wrx&qCle==WP+FPX|4D3o$jjcsybab zt5@yneX2iKKX(DBvXU~A05C8B0PO1kd~N{50FV$6|D9g~1qBHOg8%~q4Gn_~4-bog zij0Paij0baj){+rj)9ATf{IOogG)d_L_~y!O-fEeNRCfPMEGANU{Fv{FwihaFfd4j z=&0y~|KIXC06>EU69yB807CASX&vgJI1Q-Au6$14uy(#=(CJV$@qzvGlyZO3|hWP{jO9KFKeP0nG;4Aj@gczJV zK_wC+&dnwE|H}POvcCc@nE!vb$|1rW*a-%xCT~qLb;i5@WdQ)jaKc|!4EhRZr)vI7 znjtLCBoSw#xk%u5zZn<$u!)ItzqB9#vj4S!`=w7z3>PUs5pt*_8rVd4#~~+P1T z4!D6~B|SxbfMFBTfR>}*mlQ-(4iL)|iL!q9(gNcFd^L=sBo>{u{D%Qi$6FrnY}Wk5s#G$6eOfkqYZ^+N%xZBvEV@oB>cf7Ay(jR zC>mQU_&QB!~ zFD^0cZn0lUK_&xN06X6u-ABo&N^IqF4hw*o8+8VnnXGO~ph_uPIgjAMH~EqvAUe{s zmpzMzkr%KVk8oro%w=TZ7Dbp?U@Q@F$f2stt5D%mbW~8EQ&@8rtBCM1(_T-At$E>%)VJTAw639+@GpHqu z+2ID8g1SO+i##YPo1|sRDM%^Kyx>RXwIK*$=hYJ$^WBnd5V`sw2fdImM5 zyg8!}Y!Gqfc4Jax@)SOJP=4o9Mn5aOEn`Hd>Ay^(+MdlRbd%% zptcn{k_5O?IBkx?mGyl__EfNSp*o6`G7LrzbYKH-tb{UGaz@$YQbm|b*DREw3w@(c z+Uktd2rC>u8d6d8!0|LE+0x7iJLTwn!m}*k1Y*7}649)DwN(D?d%-%5*@^#v8#ZBt z*CR^;B9J8}Vg!7eD17Z+J1VPg^+?|Lm5)RT&SvQ0;TS|PCLN!Mcx0|w5>7eTekZJ` zdxAd7!E1&>fu)j z6(EM`=h>D3PbBLr9ZL|T0nK&AKu6ix8V9!VhFui;Nxs>v}b|x_c9{C5qi2tfwQ4$ao}-%=TA3l+Q>4 zJ*S|mI2EbWpd};%J%G@DWK!f=^uJEP$jTjy za`>l5!b0sAme0vriFVQ~3Zr9SL{sNO9-KY0j2#vY!k7&%hzr6(9+tg&3P-=ubAW8n zKwy^s!cvN+2l-39B6#X0!J=wqYJ>4pezI(Pb+gG<7>nsQxU@^jab~>cnH=pC*2RG& zQo^|4BDmQsE@cb8!FZvdF7^7ifs1qyvFXAr^d; zcmPWNNO7qE@SNcjiaCUF)=fz|BBd{kra}dPL4rekO~c?{xcr|$f`LOo0iaRQ&@nJc zS%k5$m0`Y68}k0nIypuh$>wd%sg81q!&Jy2ES(*NEP?wSf+J$8LG9dks^z7cq#EjzaVY#VCEqTDxJi>fh;m6FlV&`P47+40@^DveoLy zXYskz5(u)(>6oF$-=e~E+lqx8=_Ve9WWL@DfS6)mNPRqQEyU}cAuC|R$Deah|nWxU@iA09;a7j zhKaOeCoe%^&moo;Ss|CY#XQq`oOi)~jofT8mZ0YI}Pwcau!ME#IPIbvAE_%AF?92%S*9sEa?l!Q0F;q}g8kHBwIye$wr- zue^>&zJ3yJCX*Q*3YBYe_-C4f*eDbkqkkL=UB9t;WoJ>XIZ<;A;7;VF?}HOE2}VZ_Xl`I|h&i88V)uvu zuTzG1TLaHP3#i8jx)?{{&wUW*k&wxcob?B{*!={-LNMaXdK-Ah@xt`E zS~xiH(psCf9MeA#pDx#ID=Np!@oT^Yxv7j=`H7P+`J-`&nk!0Y|L^khRg>w=8gRaQY1Z(GmFOUFaf_w8E9&Ed!irE_dxu8o66?#1kPWrs$97j|s)bWW zLcys=Kntc6IE%+JSfeSH<~B@xsz@68&S&c%{Zw3bUXIPm+4w3n~vDHiehuHskF9r&T7hVd|_--!J*3` z!Z_y(3U0SoQ$rboI;1?d=_=0EHO1Gm7|JKrwmIuZWBZ=s|1QP9!!6zX*$HtRy?wsj z?Ta>FByS8E_O3~i=-NJC>csjDU#*Bc!m72HvKiDA^$Ga3MDlV`ldv#x@jhNFA?_@w z1%Yql#y5TZujaOTuN;H$YLO`FS%2RD{nF!`K&b*9CrGzyCfK*ewVJw1vcS|zxwYBZ zOa%n;LbWb<*TvOoqj5=~NMv{n?@e}KrcG$}bnyHFBk3W?OF#7sb zgn&dtCu4=eAQw?V#bh%9ky41NLbIC&B;;cOS%k&v`^D7)=l3X;&72bpZaE~>8*ctL zTtR&Mf-65A3#az{kHd)*;Vh_D#BC(R_4>R(%Gqs=>l5l-LUyD|zpb z7F>~cYy{qDud>3=evdqt~?+C(38ZJ$cHUW?tDbS$pKG<`a{zKoP`gFNSj@~%9CZWd-#o9t#-4F zE*y24@9NeuTQ4&cmGI;x%7w(4qlK}Gk%U_P($gYhn%KL-?4VQNQ9h-u(9=bU<);sX z0Y*>`^-aOzw1W1{-N4-s3*f9-B|?#lP|XMI#~&<6Bt**ei0-Tmx;%&|vyjs1J_9K# zpClT>HB#kKqG9?evnAf1D8d0-?v~1o;8?Pwr%m;5yAg3$}me?QrJzLr8JvA2v(vSNOY16HHk&xb8>KG8fJO zMJ8ki=x>#TrRyljS*}m_@!T(udf%)@@J(@JV<@6O0g2gZLpDLAy&=_Oey5!EWB+9T zUf_;jegZQ1QssGINo+KgdpC`uG|~DF*IM2GqU)}bP{u_ z$#geO@4UFQ`S`C?{uGLYkh#Wa)MQ>oA~`}8(fA(xD*o+(Bf_ZBmNPx`YVhct%4CVm zB{J3VB;2h4r^&*OdzmaGJ7~n0=@^USav1Yu?oN{WD#F#7f52!j?fSeZzY=NoPh+utheG18sVq;4h*hCczxmNq2`gBVh9=YpL-cD|Ya<0ZY!W`ZBtq+v<( zRiI4_WE?GHJjOV!Ff9wtxs-f*NugRPwilx?3yv9qyFbn|XfTpua*^8YDV)64%Ph+s z*GqWNUWIR%UF|1`yCz&7=pGX`ckbv4eUf}3;IXd1MaH>*)H^r4x)1IHK`fCxiamtt z^Unuw+=@(2D-5;R+u#4{YB;m}WZ~|2Q1iy|hB?eV;EaHy_-)AqCflfhc!Eu>j~-ir5Mx%=Kd6p!SyzVv1G)2`{P{{PBd%lWW~u^o`bvS zZ{d#Lu%r!|iteS%K>PFD-yA_%9Aemyzfcv!DzaM!MDgxgDW?eMn*W@)9ynjXuyd_} zuHRIJ{{WVYVOyrSUd#Kfwj0FYgj4)Wnp-<|dST!FqZuSu8$?aatp-)JC&dNOa(|T9 zb6s)4XJ$;tbUAwLVp?55McOOcUp6q<)X$ukx(t#Hzhlq21QjM~G;2e1^{k&(7&mWp z5#G&}os)!I_MiABS?Ei+j^i}!A{1gBTDKiz-8*vHOa72aYwJ`>+>#v)+T#l*eCpjT zcR-P%P<-Vctt9!*;1oennB{tMQJMxk(T*Nf-(9+--~Q)CXN`9)37m)wnP(C-u3YlEy^Z9(W80Jf!lkSqAfSXqcV0m z$3mV;@=)XYbLBwFvO`%v`!qWoVizIx(Zf7#cS7Q!1NLvpVrZ1~p`)$7#UNcN@tT11 z(&Ogt69A<4729V<{`Zm}Fc%kMxxg^EbT3hNW?^~UNVni4QpTPWh^5qM;2yno8hzKo zGtv&g;1Wc&ueqqx1!6nA7-v5&b2 zR#oRA0#hBGjIQ93vLsh^2&4qT$lq!*dfEJEUxm5lYjYl&Roj*akUDsvlxaaGKX5E8 zEoBp{54t)E;6X_y4n~pHu^pwIJ^Ax3@{EA2jl}i6Xto2hOy@MDEr-Y!TV4{XQsO59c7f z17pCG^W~o5%~8S=Ll&lk%XHv%r>bn&EgM=|f~a?Ej%PgU+N4|K?N85(UoUuABNouy z5T1H-)YDCQXvBY_+nnP)8UL0}KE_dxkzud|SVSOLwmNebp`#4(GlX$%ffZ_@;81o= z(ZW$AW}cAdI*wIwftDHK}^GRx`SDg5XBaA3E;X=tBdJ=cyu7e&7${h zu5h~>aA|LWF0BhI2-=}5+dNjz`tk&VV+b;o%2MiDM-I%_`7Bv{$@^{pW-r?rsc=8* zGgZ*h=4dFd#2hO$+Q&b6Oi{>Hl!dcSMk7~xz7HKhegbA!Vc4pwd>HB__m&M!{Fs7n zVxM*n;@ zr7&o*t)r+kuwmd(=xXTEu%juLD@NIlysM=b^0)1oSS59%^)vbXLb5t1!Ot%eg_KX}@k`I~p9AB(>ClMVR`&+aBvtoAMJo~TUuhY=Cfs6f?t`yCe zMUQr;J2RxJO6zTwsS~enEG;399d;n(ng$C3w^7#V87XSSs^IxB6f>cza7BAboH&HV zEyLfU8(ln@^fO?OV)RrFf60fZ-;(vX_{FE(P{s?hA{{28ATD1AhlCGHDspWjvu9#@ zf#!Lgp_?cX+_2c=un%}uNmCiwDXUWTiQ|CKQxwjS;ZceHM9wTt*hicpKkxy#(;PKU zV-IrJ2OS2cpBZ{<9@V$O*I;80qTk!sUG;Kt*I9BaW1s4@+ngJ&lAwEOwq6)IJ-b%A zv0IM|bN&E5H7agLHeV|AHV5WD0TTQcTMMdYETd<4I$m#Pj-oqYqNO`n_rVHt7D6;~jND{FdS}S-1 z3qfi8-6f;r;Du|TRYd0;e9Yxpwka3t8+psI*z#!HYg6Cl%j5gE-yEx^z|$-uuYpxXB{@-oK}b|Mr` z+c2-vB_#&NvC;2&8oW#g^K$f#6qnxgeW0#u?+}`zCg;QCy`^D`!yO4A++;YT;r0{O0QYk2hKMbT0Xc< z2kLfE_STsG^r3i6rQ}jnxP!q>73^IeZA4NtyD!-%x{MI~EZ9*Q8f6VNX1H^M-xF+N z7wFvTena~B9#JgP;9h+KvQrzaZVzb$qA-=CnS2|X`1o$xh;var`HNB(n30@($#Oc{ zYoy0m_{yQC?>PFD7ddI$p*e;q-{_QTGQR!YhHvljel^N7`QA=ajz9Z^_ll9}7zb9T zhMBTrPv1kkFC|Fmdz4Z;+CClw=Yk%?l6scFv=%=uQs%QnfkTaLew&pSt^Fi;4g z&sMAcXe<`FWsSQ@e2@$i@-vaI(D%g~^M!)X?Pu{A`fMUTVjUJ(h= zG5mC{a1FCfKuusj;a%lTt5ehdLA;aO!WE=5O8HWfwq_Dx;hdoaHgx6fCRt%dHs*R+ z9c4X=1aYGvnC{I8b+72dfW`#kO>9N!ZW6iWfJlG3#9Eco==4ZLwnc(|OQ|(WWK*D& znQuoioI*w>OnErQY_RVqM5O*X?8HI10smC@~%?z#xgGWRmvvod~~edDeA(}>vXmj#=&L5 z{#5M&G3l^r1i@EPT`M$SX~r@)YB~C&0#6c;>9^}K2Jxo+Gyt6?dRzLY0F9XpBrkc1*I+$Z%7IyCx#eQOenKiF$;ActH>q`e-NcJ3*?5DbU?b zn~a-F0kjR4|EAUDb?B-P(X!mPeSu1~1Hajd!8=-In>u0hg#Iey&TRE8 z%Tm0yQmf_0*Qbxvv~Kmdr9!@+o@(XQ4J|+15}mS10B7grJt3E33{RZQw|=c}`7(N1 zjPl2K(hll+iVWP~^<5vJxl%bf3L!@JE&C*TLF(&4l6FX$ z<8R3_Sf_Tc=DaVu5|o`6vH47)&i0JGU!p66LEv2j)PS?-bU=TeW@2Fy@p&B ztl4EkqOF&5(AHx&B~ma@-mO{`+6_7gS=f1v=4gJz*o}xOLRc~mLI@dg(2)LETc=v& zF@yR9KzP3OA%>>m?dAjK9?&ZTS*^P=tK`C>eMR??QCxw=`fBWP0S`zwR z?6S6H6IJ{D6Tql+7@f>AOYekJC(v5JwzUI0&zYXP7g9Zy5{YVhfQ+ZZ8ct{15nYp0TEy6}Z+VRYb%-nKW2dUMGrQVVsZm3@xd zp4;J|(riE&4u1i6*I=7o>z#J>taPz^57*be${C)U*{}*w2li)OTzKV zdhp|aAClTeyX}1EcbQawmPAXZU1l^Za>uc8nU^Uv!uteZCCBG^4s1_!7pf_ergNn` z$q!DK>QO6BZ)&z0Ml{_7)S%`;vHwDvu&ei?dwH_sF$(26Ts$l;TTf8A_Ly1`^+KVH zxMUso6l+U&YY27!SL*qLI4W$N?5L~x63lGM9U7H!pv}WfWFDTTb#@OlF*XL0sTHT# zPgt>jTfE>AKt&05(HyhccxC%~gSa2t!@dPn7!_@3GG>VlLl{}U7fw$w7x;iBHIoi zpT6(>mUP%TuN#_x7sTD9Jgeifd3lEDc+_h_Z&mO7jpD_P_7S;f;?ZF42scQ2lTv8` zmFQxl*KhnrJ|QVK>f4@cQ~O*M8v7H&Al_zKL5re@HI2vR8GI{BdiQUvZ(uA#hE?w! z?AWC$L7rLf(%-X<*CrT$-)qTO=J^}`9a+JP9?b{&RXxGwW?w93OX{RIE?1=ueo5J& z#~-*G^F96Xs|*0neD2%|M%XsO1Eo=2(My-ae8g7B&=j?Y>>6VJkkLIp<5D8Cii=W#TF2_Qy-OUHW8FBwitd#=F8~e|H zvSk7Px(lUTo_GwN8B+|5CvrAs2btj`iwiwR#&|I|nYyl;dw{QIz=aKhH=&$X&8J?i z!n8f}7SNen%uQaEB_U7UP1*Fu3$4w#uj(#!!NJf;2Fb~#jsa4XNN=`DMY^fiHwbUq zp7cM*YHCKE>{MRdS29YBJJAi%!zTEQri5GKsT4E4Su2o~fAvW5Yt3!3gaLOgN z2mYg;Vi1|oV;)0$g^V5Dt}pemBdQ*?!)P_9vOsL_ahYX%vCrAg+%2ac`Rg~2g^3q3 zk}X29kCdj$1Vx&*I8H^L#vs4^3BTp|{^zCyHf}# zPA{lnuP4h!{aKSI1_z`#<(6uwAX|^<7s`DV!isXj!FzEBEaZ)KOKm(Cz_16~=o2GV z|G+y_VL)4PLKTs|U5lAQ=LYeQ5!>7v?F*aQ@MyN9XRLYDNWt!_3S7kKD7+T#)34dZ zKFl4(1GN!m=7?%zJ@Jq343+qs&f6FGNc6w5W&9n^o`Lz;UeF{l-1s%`Qqgeq5BaRV zb>?=NpXAx=$6(uFTkyZXAF>8lj2A(1{Q3OfbaU==73f_gVrha<6a(-pJY>()I7(Sp zoW@8cuJl!i(jHX$)yJuV6)+S*dEqoU^ndLcW8X;cVAF#wrP4PF;l^lni2pQ?aVD`t|fL1hVnoAyVehWz*qcn)gTw6HMDXEsOI1@u_|z`vf2&d;+2%vB^`=J0rgB(h?~b=WF@` zjjr=|H3%$>y61#SPF5qdB8d|OsZ^7kgI@@M?(==bayz2X6HOr`QMLI=CUzQPMPFW0 zv;0mnx76Ufai%#D4Xy?{cse@-4rvR)x-31rMKkOpYK$j8DCLspY$jRB0XnzGM+q3;WWs! zQ2n$wk*PD9QG9gHB8UlnjxxbCz7dnkvUH-qqGVJo)MD9j?_^M%M|f;nO+ zD6D$=IwOw)iw=h|elC%BTBx*q0)q71H71R(CX~C4kOXhWE7MMIo*gM}5#`2XB4+n= zJ^|Izdr08o&hlo=$hP6;XDA7EUZ@4%5yfm}N$b6bbBjDFtJI1cgjxrF(^kN8G@~!k zuAYw4oF|@XxR)S$<8T6R7P)&=yeWs$`sdDIKAM=px#`g1U{O^mQPEHzU{iI2fcv~y zZ4CLXTz9;Fm4|8)#g2Q44L`}R_%t@1|4ps#pKT^F`O>{3zssA!{X?L;BaXc7%b>Ns zehVzSY+I8#ce{sDL8sjlg^i-|=dj(+f|?ao=B&zc!%b;+?tzsNj{$PcC&HFRss$|` zYTcbakc;d=?yJ0F%`7>QRR!#!22#SC`@g7#q^PYQK-J7ZioU z7oY~Pcj-i_8^|VF1f~i=NuTtyj)~}~^E1ON=4W$>kHjEnWG0PCjyVTv{ZxUo3n_4} zX|rJIL8i2P8(|vm2`1y+byIZ(3xv>Po;koHZ6cCK=lu9(NQidn0ioI7*qqp5QFWBx z6hX={KB~blVs^UaxFpBckPB%$>*oBVMk4l;4eNVyH*)|%wUoM2V$msb2BI&z`u5#f zJp_I~B2pgQRM~xdeK9xz;>;DM%-+D|f2)RZ*pV?Lq_Y zAM}Uv$g(6fjnGY6O?mvD71G>?dqi5%;=Z-G7e|g_*x6LGB@KK#jK{{g(oN|>&Gf1j60Ox{GNpZeEqwXD zcmR-Q>fr6g4|9iL4TE-G@vBfnV;_`c5lntN9akE=1oIkgTEQK%f7_**559?vz+sR4 zN=Bi<7mGK3>?F9hJiNQ+8xz^!tpG3Vy61a+-}>?y1D1gI zmHBj<4{y;4c-c{g`*JE}dV{&WW8G=D4Y$N7)M;X`FzZc`R6d)Gyt%{kDdn+4NAEm( zL)$Bwh9@Rs7Glo&P_Iv)sU>oYjTbh)H-dN;-DtKlxV?T^47%){Yk#Db&9UrJcEG?+ zMOCFOPHjneaE=QWFFlN@Bw5RwBJ}ep{6}@GGgg&jyXU-VgbLe3y{37}6AOe^UGwOFyf0YFsGkY<#Q25LXSPyw zVw%mFbY#J>*ndUYF({wT^{xPAv_~dBxq;S=hSZF%6H$Xk1Dwvr9cUd$8j+cenyo5h z=QvhiI1!at>`Lxjv_7%gufDLqj*%wcf`Qc{7aW?AR$zP8A$ew z&6ik(fKu_84^f(_6n;}cWA?ENDV2|{6h2d>seKkqN*>HI4in0!zkpA)(Vzx2-+$k~ zw7ZNFSB{845#6H#UO7o9+R}LSbtq8X0welHcpNI55VdW@*IThzBeP98H_01{cs9rL1p2}9pP<@e53jV1eVVkjlpj(srgFcy|tlkkCVs5 zSws-QyKT}@li;%?AN^xu6spi$wp@hRQO{qIWACdHn!(jch4aGvn#9F1y8;P2ao4EQ zzz_;%j&Ks{vadt`$xE$dYO9L z>-vU_;HbH|a^N@?KDjr#Y?zHVy%IMk<&`|Ai0%ZLBsIVCP?{drM?nHi>x~4fo&O2^ z_(0o2EiJ7~W13r85|8W|x)~*0Iox65oJ_mt*dn>M#0C>zqL*xH^jegCtlUinzjeQQ zyo8%7L6ZjRW7x9$&GLY6s+Iyen-@>|LS@^sSc!1Mo_WbfL)=7^JrAkE;)ZKJ2Co){ zIG<2d%pLO!Ue@32B>v=G157`;Pk>wvd_Y8=?ruu3da^4$TD#LHAXuu)^wh~j&t@Jr zu^AZk1GPY~iAF?0)_gpB>H$lE*>Y+ZHtdj|4(B&QBPUttI3JzD+zs=$=YY#^ENEx7kJJ22J(=0~i?GP|aUpdEMfb#- z%!BYq6B&;YhNU`~T`uPgL3(}Rw%UgiyVX088XbU`8`(>y@M5N#f*lci%PhO`b2>C~ z<%;OEritHDB~kV0Z4`9`DpTnCbTSMCiJ<*ZJ+@c%`Y4GHNgz`pxW;^v+X_fMB5hB^ z$bTK|L;3m#dhB?E??k#fhSeGw15n(EPX{ioC%UeFY=fi<-c!yr+4KboZ@Zh309q{$ z65p9ii@bIk5v^^oZyb8bwOT2Zk}vi^k9KTjHDfN6L~Bt2sUIw|;%9nRDpvzv?S)_n zp-G$gVQwCc3GEcT%Ru6?mr^MWJ1y~2VUqom~C;&#kSM)lnQK%v1jHRfSEp9N4U>Ky1b|v;V*vBb7W8V}FaYYcG)x9sV&7;<^IT@s! zl0hOZs2ioCi(nMw3yLb)fyOG6{~q$y4-6%t)|=9Bjng!!2on@5$1nnhDd+?UGK0ZK z4n@W2%VGvLQq44P=smuCstW0m9v95aB0azNeDz^#q5n9wiNaINBAjMuPA}?niV@M4 zmr{t!B*HdWqVC0Wi3L_@P1VAvHiHqg$uMBvavIi+LyA{X?)`*uhuJGL5V+$MglIxl z1DBZxA(zoT(q&XEX$_RcxMD#@j)^)~H&Q9Gv8gQlbFO+|qTG$+?rYTwgm7^?Ymusjsdbs;Pn-290z@L4XOC(7THNt zM3Eh~`or<+!Kx<%xNYMQkgB= zt~x(D$&amNPQ?UaVfK$4_H_YN3F-`5`{XHkFXs((ZOkS$?(bl3Ayo3~bG;wIYP#NR z|1uDB#vJ}6ktS9Wh3pod2G_|aN0Mm#3T9tG&Btzl=dbWAJjdz(1Pq(e{k4|`Hc8We z0?@`N$I3YB7m2?6HIJ6n+A#qWL&W=v*yip?L-s15xD9R;4xiQ&)M5xjPW`-;DBW^^ z%jR=Yq8)~PO31T~=w-bjW6`}iBk_?RhE`XX|8#cxT1$pLvMj*!$zT18cj;DhQqqXL zqRVwr!Q3$8UOm3Z& zMK6sUPCZje^jq$U@WW*8qw&x(C$nA?M~L!KNA0*=Tjj(v0e_M)&;RBWdSPN!&u?j* zFANLBPZY@)As2-+?cQGX%aJHJGzQMjf+puN@s4JxR>rv8+AMZ+=U1Nf(Em7>+Td6h zOd9%WIf%P!$vG~k^3BBy_KhddA}{pJW0+byecj3 zF+}-wH*{9XE%5)M;aE@7wM`w$yF@JuoA6Pw(I?oYJsWeXLNo`nTjG6^w4@L?yh zgiT(;pbmlea3;g3?#^q?2S+rp;OxaiB#}M zkzd6g*tewDxhy?Dzj1v6yn47%`y`wIj2ivuCSC8i3OyV`I(VvB3}T2BLPeZNO%6N? zFh3=O5$h+L3YF9hJ6v{1S_Wa_gNa@DebXbOs(&`s4jjXd#MIep{`eie>n~_r9|gJP ze2lg(^h?}bfTO#*-vv8sAC$JitRu=3 zv0u<4;po6CoR3b;6rKFX6u*We|&rV06Fzp1M-0KCPWc zn1n+P;bom{T*hq&%nq$|j@E7VGEw~dogV+E;^en+qtndfXiMEzCrO8a-{)ZPu4mRj zX}8tiFj%Y#uy~`&MaDh9XXK49RIC@C(*mQBv*oj*$8(}#;@hYlz&xGpd$d=6eF8i? zQyY%4kM)47<+uL9E^bcneP=GZQr8D4geLi38<6OYuAvhX+>*)t19J?$7B(vcHc4PE8T5q82skW8-ir|kLJ1G{edMP4_M1vyiz52 zhV!&87T#jN6A3f@6=$GqzZZBw3p9#KZX=AS__UFRWBRep!qh&XOk#08kN9zB8&a0t z(;~wx>9$_k+X5}2exujOxv>lI$x~bg+)p28rL++XQKq;^<-s9p*4LIq5$gYPte*cH z$Zft?ax_>h0=XrLkiX~539CQvHdwp(>(@;i4XJX-FAfvPWdfpzI!}e4r6W{S9-D=T z0ffWg$FNDVf2S1Og9Uz7)apz1+0yh^4`aUdhowyK1s}s(pbb~X^hYj3khxCIi;93v zKm}TG$j*w!^FttGAOwvQT}ECq-k){Gczbo7tz?_|kap`h7LDajl$mtC%;0Ap%tPl7 z%ccNIrQRG3W&}iz;-JKjTERWAui>&rvCse&8=I9#?zYVAujw7+ujtfx&SK89K-`gf z=`ISG{f%BEyi^VFDwcU4iYb}&>E~E6>axtoMfE-V<-nVXah)Xp8WA32w@MM!XT}9d zfi#ifmH~5@0TJ;jbjP#f&U3rw&7M`Un={PNgx;IHP|r9NP%E}X&puZwQ_at3DkO0t zir;E;*JVJY-f|bZh0dUIc8I)oi_DD>9>skj_)*gK(205uQPa%T{$mwFmHD%!wbg+B zj??!q%Q~e}bfiHhE-Lem$FioS`DJ02jMOg|w+#t!Cg1d)F6i$BaFd3INWlkw*+P8sQpnL{I21uRcLnUhK?KRkVr zb!|ao`Py zVtKr$L8zF&Zrs!hu0B)Pe$j7n3L3l?O*4?=UM^yGJ()71{(gT_uFJxK@bJRROU3)t zWG$Akw#j`~a_zN*`9(^~(+ksTo9LSwPIDe+d3W_e->)sX+h&YxybY{IL0DVF3aJ!= zQRp)*uussQ)kHl++N8GvAUh@~3?+)k9cT%f%>9x5?N+JLo6pqL(x!WGX%vdPbZmat z9As$pHI6f2aa1@gnzk(*=f_8pAE8kO5EN1LaKe59%tfe)12zYgHBg8x9!qMqP@Q;K zcxETV=aAxWH~P-5P$IDWP^_L)&Ue?IB(%!3O#XzmSx;ZKq5#`_x;kw72b?`ss^=~9 zCt8g&8VsYYJS8@2isQl#%Q4am7!Hr#cfzBzWqJEXY$*jQJwAZ&bNV`R#hZGV~x$S?ICh2|Do2-S~pfy-J9woR!u5Vb&pEFFc2y(_$|qis0tr?yUY3j8G~jmN?Xy zFQs>1pri`<5C9f>EyrbtMn}c)v$TsG>`?eM5D?Sc&n)HVfm}qHwhw-?d+Ix2On^J< z)U@(PaZG1Kfd>9z0gek7idJ%Ig%u>0nMaMXv_}DsuP%~|fEV)5s`my;g^?;(o00CH z2HS62t_sX(Fk7LApazq{U%(skwds$h6*&my0p?tv?MM#7bDE0M&a|0@3?@j4+R$jy+BeCtZ&XIl>5p4)@m zS$j0ZKHi^sA4(dDqoKz91bfC2MN;~#M>Ry(^2#+5#z(*QXm<~r<~K6p@`Y*Jm(;g0 z!VA}_!BfmE2gZO*N!FBNzF3RES<2M{wd$%=h+!eT1f!sKiAXi2XN%JY4bWr}e(;jN zucioTOK@gmIKzN%=DRF(ZLD(Ti={iZ_XSV-=7ghbq~!RzNA%|27t2+s2&ey}gxOL2 zX9bJ9SJtV&4c|yc2C+eFObJ?%0*z>AN+oO!s3M(}b7CGoJNdXx z24}fQU6LuYcxK^m5bW=0QGNn#WheXp>J{S5I8yGvdxp8t6GLsVX%6!i?>_&J08tOF z@Pa65dsZkTPi46)Xl@!8mmCj!HzJlgEP*3%;eZFkCIh9b&rkqcBIaBfX2X}LjxcVOiG+(Uua?KLoa6>)=%Qm|gc~!^$%;8zHWo#Uofjyq?bGbbBVS+~jTjR?c ztMWng=X=)Tc&lzu@aW#P0qUi%h^f-<3YLIrR)Ogj zu_Q+2(R(AiZH^$LQ#4kuuXb53&Q!WPHqi!+X1!a3p)DfiyjhvMEfcu2bIyU6RW{QU z3^i*_a|Pmbx=1&DTh((b66h+8JHZaApO(8t;H4FS`W)&KckJ3JHwS_A4 zNL5|o?Krn{d*u%}_T;FxTO`Qhmy3YXoZAi{fo<>uJa8Yi9?ef}ih7px04N2f)4t)i z3wQI>;7mSkWa>obq~QedL{C6Gmc`ci?vZH_)jMoRF~!(bUjpd?l%7b!4Y7p zl-;Z~B}%%i{G~-toLpQ|ucXWtz1uOMx%D!DdFmdha-|>+cG6lJcmlp4oZ;Ee5FdQP zHoZ!xPVQS0C_%-*#l~Q~Cdud`xPSI14P9^60XFK6Y?a+lOBopA?&I{gejy5d#^04K zV7oB`MBpYI)ODmz!VKs;rdVEU^vYJ0Fg!{YjEv&yU+b9gFBO?b6eAN+V)IYogg~%X zP_(s-L*1Wf1{1lW<*yfo&i4f+V(&E5!4>%HS!!Om54H&WB&*_zJVo>wWzbJp!A@lk zhAIRAt)XZwH*K-4;#X58N-!yBx5M`eBeZMVFOj8GsLrefR4cejm9dioiFq=`4FY^W zH_1>-_kJVT8D|AW(ym3>iicLf3ed!9iV)TY5N?a|J^PG%R(6yGTAy=AoW}P!M2@;6 z9~55($Eii!7RM4nstusFonMKBFr~H3z5VtK$Q!#cSu=JQe(f5|a=h~i2= zNFn%-k}c~rs14-dEu<{=#-)FH*#?yr>iFK`+rQ9x5zWz<@$8MwG;%;K&hA`8%Y?u1 z=bAEzmAB855Vr6eCa_jIGvq&Q=&SIkLM}C>^2C3e+em)MzjD7&{CckAw`&gl$5o3$ z@$Tgv=V{t|g3v-7RcuA73x7Q`j-?dMuw5(J@XFS>hMI`v$oJ?R^NRpfL6~VecLdL} zpqvSA4>2gRG*3tU$4-a3O5CtufpQ)M$>)m8O}99CAkbd-Tq&kqIjE_pC2YisfL&0Fm~U>E z_4%jZVe>0Texz428T{%pK4N%Sy1HY~;Wbj?WCS%gux!5CdLr<)o(K-H{7s)E4(u&2 z2y^aS{{VA#?fgae8h-7$Wq`ek$4>A(>=AgY&*f+$^`P61zO|D3xkJ=gd6D`?wcfKg z@~Uotyh^=bZ>gmI=fSErp?pfz=9_$%$pW6J`xyOG7nVNLxnX-9n<#mCV(g_%9OJ9$;CN_9rT!-fvPL7MUA5d1q2QQO=&`<2mY3^zuJ!xD3m ziFt7dnk=EuiG3+v^N~-9hY`HV*TKKkV%G8OvORtKd>$5NVH08b3QpYAwSEv&wx-R$ z7OyLC6^?!_6NA3s3>Gw3P&9~86ryf}p963-NJj8ox@#mgXl2^I7uZZ3wDtn?vO1=N zb*^g`b#Q5cW!g4sD&c0Cf!doqVw?_Pd~iX)h0^icEilWDJ2w;3;<}psr#)^`cCEdT z9veo~-UvL>7}8s&$k*$i1e(|jwfdB}TA)jKo3JWo^vdf7WmTXi zSx3wAqb6+<#{OJm_=EIBqFu0ibE7KJ2)EOkF_RMmw;WZufC$P2z0z)y?^eo^ldXIP zpr+Ag z2YBcT2|?N(6J$ASI|+w3DRJ7p-BcaQTD=M$l4jd-#$`3<@Uc#TSJXAXjqv!BFbaQj zDlybjb}|_763VCxZd=n@4Ney0{{V8NEf8&L2n&|jx-R1SqOn0{&c{T&-9w|Z54e#)DLGwz zFj<&r(`vZ+j^a;SAb-G^a7eUDRrB?K*9rdsdb8iF_bX>-Jms=L2deqs-of=5{AsB< z5`1p^$ahsdT*NrOEnG3!U{e7FE%6;HC0C9=1%PTVII{8bnEn2g8kcgQm*!JNE^6&# z9HB5ZtKtrM0xYZ@<1-*%V_z^@vuS#>D#Kk!_T&#p$~hdD%%&W}w6$IbKE`HCOYhd= zJt7%!?j-|mrz_-{8)lQwUodNGY~&H=#Bc=1uHk8wi)cKjr|K&hT$bPM8(AMlHmf$d&ccj_$L zX|Os!RfiUYUmL5@>@G&=ihkGTN?#~|mjD&`6_Pvw8{7{+;TuB7w72kOosiq9lM{vK z7g2WSr*ms-o5#*xkX*z)USPbpWlrZC#62lBt2~H*DTFVRN10G|pvwk=QCA%kV@v-4 zfdn`Z9L`2#=xzmnXm0sDc1m)otDbm0)H0%U-5@zCT&tCPVZ=eQDbBdOMW>#*-c>a4 zV?07HvV~XcBw1;vNIEL63!)O|h@PeVGRrg;{D&T;OP3$+d+zvfUx-s|OTaT8Eo$=y z?D*X>qavs~jMiy>{{ZVH7E|DZ`LOA({zo9?n10~r(hV!JnB}t)&2Mf5triBX(wGe> zoIFY<<+7Hmj13;OIFuLPcOTt>>=;P0#y-;lFdN9)7msm{qd>^kQ*ViEqg)<>C=MdC z8g2?xqgH3?+qH@xQJE5Qf7F;`{F&L@C~!8;o7 zysMqtHeGjw{j>ADAsx|f2D@U0BbQKEDA`}!3fg;wbOrS}rPqiKDZ9@NM|d7TC=Qb$ zUhhuR6T+HTA%2^K*#%`IY- zX@Ef>l>@kk>x`zg>H$}P*c)a^+ZtNiwPAiQQ60tAp}qD+-KBlcYaEoPF_yGyI5UW? z!7tlY0&15CciF^8*(~d8`INBsjO+VmjPxr))$+rL-Cre&vogwM+U0wQD93frh&Yxu zI7p804Xm_lstURymEhY{J`*H+7PZymM0=Fmf6;>VNqqb@4lDhCMi{p}4u_e&?xyfe z$`#fD@Km^b$%j!^2j`v+I~O~Y}>18LGYM(hL)Z-Mk!er4|Pe`n3lmBI3;BI zL2-$b8HUT;SFm*X?_Mrn(T(co+Z6Abg1cji<`~=m0JAgYWs{dUI_4?^JOmt>a9HkB z2QsHA;s}KjmfmJd71v1MAB*^ksf^-2x$aUup+t|Nk{_k;ufoO_bMU>`;?}woB>w;< z6vQL*aCo3$?7M~D@9l-2^TegR&bH|Oto%sp)&QLy%3COOGa6}#y>8GooRBy*Uo0yo)}xN(Q-E@LV{Mh zLwQ|9z{sOQW|4&U2}&+lp0#`NG5rVgkJ{TSm`8rUs%X>G+`gKpx|QS%Tt#Sd5|p&7 zq9^n7{{SG;$5EL0ksIO&R^pV}Ga!d41h0d9$7gLoR?Kc;qXsRO<4+%g_;W7UlhP@6 zs9FF%fonWj^ngFiL*j6)jx6ZRe#?$#wEKmMHW;>tUZE~8O)uXFDJ>~@%pJN*9}Eq$>&Vs#_h1!_B(Fq@pQp<{CCd5>TgiH(;rZCYhDFKn$v#i02C1c0000000000 Z000000001>|Jncu0RsU6KLAJ{|Jj0_uO$Eg literal 0 HcmV?d00001 diff --git a/website/images/photos/mikhail-shiryaev.jpg b/website/images/photos/mikhail-shiryaev.jpg new file mode 100644 index 0000000000000000000000000000000000000000..3f2f2f2334a12f3736dd5b1fdc190736f066176b GIT binary patch literal 12923 zcmb8VWmFtZ6EM2C%i_T`3GNa+XmEEHf;$`BH6geLT^xcf?(Xiv-QC^o^1Szc-~U^4 z&YYR4uIldU>guYRdYgY+2Vl!e$w&cUU;qG^_Y3g01P})xz`?`A!y&xi5D*X$koQ7# z5D9q7{{-%&>>m|_%%&m}J6%ce5Py9+coqmY@?xKYUqMnNz0;1fb9J({GkxkCH{uHk z2?3xZ2T&ho!vHO@VXy)H%n4huhs{@ykC@~wv0h)M6!xKx`3GI$1wsaCUrt|r(6;Pi zuQ`(xdnCk&0RDisLQ!gdQFzf1;^Zq9-$)9P_KFK1riE3Guo|11Wv&7)Yirp@2+Q(A zyVlsjle^DO|Bn!VM5>S&>ov(Q;0cfJ zl`vmoXS-v@W((a?JGp;31Aoe<1;9kWMuR>|IR8R3U1PV;``Zpl=91N2@q zxn@+`uP>HrXc~ahZ2zTXL7*$sg>!9uKJAQ!M;q5IiTp8un4WJ33#ddm{vja&IMKWr z8TY<^^M?&jj_OzZd73^e5eF8fE{C=njgt&@E?+hmpts&*&Fy11q=~YBiTq=6EaZFt z!ov56!6M=JFDWh=X5RAi9cfH4a2AjyjR?2`g=Q{`R-f@N^BcQyD~!xqE{m=o_{+5h zZ!l8mHgy4r02hH+Y#6*>fKZ%2!#5rt2R_@61e^w585!bpJC}yV4;kINuSwY#FRbPs zg?HtFZX4nCkld+@5_cbxH-JdnFe zTlRDOaCI!9T!9=+-RAn}iaXx0D){6r{dc1=S_$*06#T9Tq*K=(tS{sg+DBI@)S>_Y zP0_L_Kpf2sIV3EaVrl)k-LS>um@F4$<nfXu#Q|%mXJ1k!!b`?W)y<1Qmk}Mb8;UHz7WYlHI+6%uf)0{UeVr z`svJWDuZ&~9iLv0 z+PVUrqULShai{bpCB@~^1Q!oR{Z_cC%ciI|0Q;mkQAbFRi-5loHVksq5Vge2#mfp7 zYu%Kmk@F?WiB_A}$-z*8V(Rh5nNr}az}nT>^5wPO>C9YQBYDE%ef>)dhJ9WffcjmU z9mpHVVnDKFWJ$vCQ_#a}V&YodRg_8IO{xOmkg=4Tv2Q@_UF<+2Ky#UHG-k5XhMT@{{eHtHr$AOYQ#%akq2W zsmFEG`&l5ow)9Yzc<;YJAC-qL{$a+R3IOZ^0*dySkxVMZv~WLP%_L8_JaT_5X?|X_ZHLqVR0;tAV$Y=>cyVMvC@l3tC|t_ri-CCm zb)(##_*zhq^M0eV+LB7k8$c1y=h@e7ZD6awV(<2;?s=^5d0$tQKcC9KhYmSHEH;cIA;clFfl$J)G4-cCN&02Mc* z5Bf++G&p_Z6@u)K1_Q(fVkARW!MV}Bw>7cbD;}3C9?EM>3V%=fppCh1nZ8H;*Y*w9 zx6$C;I%}G5mayuG>``%Zn!h9fUw=tRe2bwr+BoXpc$#6_JxW)I-~DaKJM}Jp-KD*0 zkJhySz6e;n#42y*H-Gda{S0Ap%Oi2xjqe)5p#wSRpO`XUBJJN*+4 z5d+Wa)=%V59~zclELI-Up+OSFh%hpkd2#$JJreM+J>O!fMI&OPJSjM@8$~sAnYZ#E zN|u>T%|anALU+~chAtOSm@I&^DO881bUDOd3`qL5l3Fr09$m22g z-wM7Kv06i=?twT%(Tcj-?bdoG`0DpwmiUxMv#pDKAA4L;%viG{EG#umNGzq4e&Si| zn%-IF&5Of;R?46M6JfdSxt8$%wli6`P>669VHPlG*bNg`u@;_zsBtk z!7*mDBQ-2MLI@NBt!eY`a^NBN!M<9>CX?QBnfcJOuczzVC$YGkmArTKiVTZfc3MS+1;GYWt zX(k_0!tXhfFswkQ<;}Rg`o|yJrnc@D%q^tQTV<7K8n${4t*j zStcN6|Caus%mfDAbSZ6^Yvd>UBn~oBg?}jQcN!uftO;!qAi^p%@(YQTlYfuscQ|`B zroZ{}-zljjoO_Xx{d**kaftdjWcy6yafrs>JtbLe01O-~JS-v-JOU~_(!00w?j6Cw z0}*hj*uSbE;!?92IbhRph^Yp|;!$Q#aEcmNb|T@^a&fCUZk%6;o5bZ15K1}){;YcU ztdQT`F&Gh;Ywx-7374CvH4*9Yi7P|yatA+r7eC3FnVE5JKm7M+7e8r{>DArpMXI73 z^^sNgS13QkLnst-25CQ}xgo?}xke*6;3gw2%l|nyx^WC$P1@vaoRBaH%I&TxW~Ybd zyXhF13);)fxYU~%EVh!=p51?j8p4ad6s?M$X7$;8r-AmEq4QeKt~1f(b5^O486D|L z|4G2G;_ITTKW9n)Wx-5q6N_>v@bVg9~pQaEFe1;DWvXVG4h=#taw3)JRTX7E@rqhHaw zon*T$4&oh9iM8Y?$Fqn5Jn}h_O*@v`=}O}@c+SUmqvbAKwq9pR8bY|F>-pWbm;oY~ zBX5A9Tr;}Qk-yjk6x5HMNPU#swgaBkqVHl!j#*9;vDkBqn`xa`ZPtQWlfpXLKWP}y zlPrO!i|ZLf=Iz}-HdB;y8Y2V!h^x!%V)7HsF@j~^@qG` z=FO?$8w2u;n6;wNG}4pC0x=H6x>sR_D7F}w9dYR${Hsn%c`oFPw?j%`9W1Y zLyge|alBqSer|LV%gSogzmqU?Y~cKy&))#fSk3~=n&X9$!Tv9Ynl&~?DQZ|yYfD9A zFJ(zS9a-GK1d$0t5)cuc#JIY1YrQu7iz{ir1*=^%?U_C3pT5_ifXgM{nnFwCr#HaL zsQ7mXuG*iHs3^PL97tJ9616VwQ!)FRIjsWVn5UOh)CC!e`i({v!k3r*IT>! zz7TC+Su}z*_0AkL1~0WcC0FR3Dq0bZ8k&RR3f|R%GWx1Cn0rDxs?Ge~@ZxD}QedbE zo@;W*XEq_YrXQ{x9D0~N7b9IBy`5!g13-apaD{&=BF2lDlVD2AXYdsc&e0%<<6j-U z-EQI$cL>eiifM2YclK&A*X$L0x_ARWVmQ-fHDDRzsj9MjwuN^EZaamhZS1-Iw89*#`e59dYex2XuLkB*MyGlk^0m`E z^Lnw({;}R{?JG(79zWljo!+LdsJe^uX z75M?u6lmc^(%(z$tMrhqn8;5!9aaAcTkHOwR$~0f^bsHwED+1kMu0AppOXK%>akCd z;AkCB{n(rWh4Ln-HG926*6z3JgbxxdK3rARFo|&KY*o3_>+~xDp|(DSP2mUIP3I%p zX{@-ivJAvfJ-)*g#$341&5juiflW0X=Db65It%XZOL2KcW?!i~kfQW}qmsu7reDGK6qUK(#b(0 z%v)lHU6nJZYld%&O2(&x{@NILr9$3w6v!&!Og7i7!l+f8bLJ$4+Pi}p3uGgfP__Xm z1DQ&(!3k&}!`G^&i`f3J;Lh@uV_1f{5`=0&eW7)ofOjDqbBB@FSfi4JJ9Dqr?>jCJFXn&m|5c= zqTmH8V&%#DdRvdkhS2f(GHcbS?{rZ2eojy+7!M(=FV$19WtmV)e?oJA;N*s%BGaY6 z1#0Q6qtAjFq$av(_35t-Xy)7GQM#>$NX7I_l7fapU@;VZMOBGTy<15WmvWA?R_yc2 zLTe*E(Q4%|HU@43{81;HlalGVx|&OI@q|z|(71S!Y?~W|JgdlS%TijFK6bWP+1+$O+Z0cfN0 zX4qE8ajf+=vCcI##CU~u&~ztype=?%GHw>kx4ca*@-6_x)umJ@p^GYry>mpZ47I+N z((J@#Vi&RFlG*9Ik~8m_{&N0PkfN=e;z7nxp{pi-xX=u~;;$zRLXo&YX~e{Ose02Z z!9?l@W-vlXubS_wZ`tEL1AoeXdP=#I$#(IMohff~bP~4;?&4l^;;VI(8+VpX0ltzh zC%AxI4z*o<$XZ0QM^tZBsPua?*_6&W;JwEHlTUvx|@P3`HG3 zOsFxgP_w7)W10m;-vA+$meeEivFM9lU%6xcSQ>Ppv;`T_z7$2;&()7Axz>OX{8c+u zR5S!9_6F8x4egJ?YsN`o+6>M^b7p>Vp&O$Q^s9DBq1wFy-)Jj#cL@+a*a;9ZUxM`X zSEt0-*lcZ~lvb{zGwQ1S;C~hO3mPhH4U5RVZvZn9jG)(L+c&^p_{GmU6h@_X4JDX} zH;uocWnj=BXN3TVJIOgIwcwqkfo3|=$K574 zeHyirgKwtqu#%?XqL#`yY@nD#%7TwbM{8Oy99)CXEt4pC%9yJ%Xc)!sxTdAJB8XyW zu=yI>?$P~ElW!_sE`OFm8fSjR3(IKz4Nwm0C810?iHBXK=~XpGOk^i1aY`%QaUNRW z;DLNfOgwc@cU3jEJL;x$sn*x|s{cT|zwErRomb=pN9K@QBF(L0XkYbEI^cexCQ=eP zNf6#?ME7#^M5n3VddYvvGJUjJ$f@0sNr&E68etbyQ;)mZME7lgEqRAnSZq(=Xcgn> zv0@SqWW&GNE3Wuqwr5x03QT7xL$ZN2p%_~{&8 z!f!9&1@y(WUV-9j#fV*&zv)p_KUuYTP)%Z2e%IiZKS1ZV6ZV42o##UpYZw9=-Jp{#t?H8I^1_$e^s+~1cQmvvP_~oF!WGM zoz>X{Wwgq(!vaSc`DMSRKG+?2mf^)96m!i-pCHZ>VPJ*~43PbYJ2gXkDSCV%Si5}p z25^y^QmUF3jwTK~Gbx>$!gyYIU_c$xmr7ImszN4VvA*mP*(t5?6#Kr_eQC&q^tssU z%SuvfsEJK22wTo6LgnekX4rZEZNI2TNYQHNGnha}1V zs=IQ`i5>AZZya&sY3lv{!rFUQ_6Akv&^m|Kq;U65?j@dP5%__C>E5j|=&e~V%G zq6ZA-?W8(fxi2ZJy=+>3JwLtq*tKKqL)m%0_ayXYkUo0=0bds!eogx){Cr_H(hmC-oc8VB$S>$|ANrs zR2{_IW%gQ$puv7l_Wk+YdIZnC0n|zZP0oW|+k(~YF(Ny^k_+{MB!y@}cBU6<)oD>f zmG__{Ao)-3>&MX}&|tHu(WzBoYnC2)@NxO}m}{b@ zn%5i_MI-0r^x0SIA8HBQL+o)WO1^%QQ{GlTg>faG(xJ2zuSgeP3*P|sdCd^Q0{HAoI1;UG4uU}FSpRiq{k|fv+jucqhTx~-6`}l)ZqFf=n%Wpio%8CE5-6f1`gD5@k~q@_UT?Z5Ad--K9u{uA{5heB>urxG;5q5qzw#T& zywtA$<{&S#Hd#!5`wS=bf}8g#RA=T&2Yx)x;R`ON7STP1bR2>+90g8=kqh{y^aVG% z<%=hcA6GwhnCr57Q~bbvi5N64Fkwbxst#G2(BX1dOP1eIB{X3Scj2$2aut@CZ!>fJ z{1tgYW!*vhK=HRAC+Q6k06sN*2sZ>DHz_?XLz}!JN4n-ykAT$gP+ zWX-GgDw2zq{xcK}yw_Omy%d%Vk0}f`A2TeH25E-VlyK6(chXrOvk%(Ce!x3iH4S;A zKIW*MhqXpCtBH&DXIoCk~sn4txFgRTqC*%W-wHIq{USJ=C&B zuhD*|IdFBW9sC9$O0-C9uRs$n$7h|MTo|x90T%B+C;DbRvvv)Kv*@>nC$Z=^R7v|O z3WqHP)q0-dp=|VGX*pN2ud75}4m+03!{$cK>IQ*!5}U(j$q=enOuhdOF2`N4 zi+@%Ufb+r-&Qf#MA2?5?e&@)fq8gR2M_rPeLzo*HNh*>d zWnC^xliKGUVY239Nq9^WC85$g#t~OD;Q{}KuGbe~ZgZ7|v+^;A+sqQHMd#8enWmYk z>uTyjp=d4Vlo{U|9ZU${`M-B_KOeQ0^9tU44TtM&bXh9B6Il{ty-WS8H`-Vqs=!N$ zz!SET!<~Ojj2}HtI~tNS2k$L3tZ+ig!m=K!CAh!gO);Ouq|DJg@scd=VxP&YyDOLRU_1UA$;SwgV4YDSM841E_8ezssvVH3=hQcl^O-bEie;uf7?Q zd5vt%^-I@Z#2YN|X#JNJj&O7}TqM~R`W(YsiWNC zVX~8a(LL>}&%~IEO+!&gWrB@xpH>mtrP`?v6tg(GcC6Xvn}rFJE6q0lH(BaIvi2=w zB$M2>kE(Viq8%4&wwRXk+;!h0fN5VpY}yAztz&I4dis83xj`+OhCVEF+Z2f}uegwV zIj=P=SVgV7wltDT*yXlkNP5(S_Bi+!^^p7;8w>R|r_fF_s&R&8_2#zCMK4p_=94H%=X{_4xxiW%aHIj!&`( zz3G3e@}EKt)6^gNQpAIdmt5-4(E@7AK}d#0qV?X%oN14VDbo}!okxL86`--dRpd`U z=^b*B<-a>#DbhXjcMll|ZmaP)GcrJIgO;4iWE$l3@~mqI=_`VGv} zv(8h-t-oloSf~1U`;cbwucH_LlbZ0DtEg2+yip$(+$2ZwJ2t|oL8mVay47VRil@I& z*y>qy)Nq0qmtN~})QfGe+1Zso?U_UEGf|z&_7B=XMJxd=);P9 zwr|3^FUpwvqnxS6RHw$PKGdNDN@!g)uB;_s-RKoDn$>`5AuDrV&%A^I@T;7;X!;<_ zg*H(bCUW$UW3VrI-(Hwr#Aa?^*F9$HL#QEZV95f?7OjVSA+PO+4dfK|*J zDqLeSX$~5lOqN~W0Fl$Cc{dzzk*yzeF^qQ+7s&KJljKzEDgB5pwzVa(X^3Uk1sV@) z^s6yg3O2?jR2_Ni_t%n74vPr}N$A`8?xRS_a$)I9kxeFMY(`jGIiuJhe?^BUr*jFQ z^;bbR7{^YZ)~M3Xm^58tr%%}7Sx zC|AP#tZiAFFO(#+M(8^;41u!lV2sN1AKenYNN98Y$WtO!=xGFM|5JMd7-SLes&uS! zD&Vb!m}>lJ;o4-!;KSkwl`xgpwePnMUd-ndOF2YMC|nu)>h|@IK-2?%BrU-%-yC-E z+`B1MJ3kb`>?@lp()E9ZHCmphWwuamSf%1*CA;{q7f%j}7<3{YDSp0y%-0S}FgH3{+sZshpqQC}Ux zt#q(xIs}wdn9rtnD4Had^k|r|5E*2DowYr*IWqMa+{?gv25 zs0ljS^(_@lDbm#&+;H-^mS`|#{;l>BcuL#++jk1+F^xZ>qmHh~z-8xt0l*`km`2R3 zbvkF^wNxKw%5y(qy8ae^SEca#hKoBMlN%A{ zK2C`IT($`bEib|EGR9K5`Z(tClqmu7kRbaLr*d*i`Ezda6Mf95q{H=kgSPV}4SIKx zV>~M@H>vv1U}JPBTdl`oUUgL~E|qA#dc>})tL={ORJYbI7{gCpukY5&Ve3@Dw@%=} zm!Nc(bXcYork4dkK(&pwmz*n#=Y?0;C$5@qb+9DwBe;+IN%{8(Bhl1&k-3f;2OTs_ zYVKBw^n~jQpCfWjdl^2+VMmDFFDU7|5vAZ2{ZYtSm&)#!vi)*gq&lahgAb0cgctAf zdIN;c4H!^`P?{f*67p3eE2|d}+y5GU)``ix%6$A9_3Ias{1GN=@?Lb*F&|&{oJxa2 zod(hH;h^U^+GKu7LmqYL(1nj!3EMHLk#wH}5Hd6XBSAJ1r~bi3?{XHahO|C-^?OuN z0j@l{^_GXxy)P^FyU`yjcUZc9ZR5NNw^U~G8R3?a78E7R&u+9tSDoN&T^=j}6d{x} z+FY<4@qsEPubrk18Z>q(k#w$@4eHT128@lIzyAnS`$T5;fqpI#g+-66NVD%8Ni7bV zrqP&mLWgE4uZLTXuiH*0OQx}Mr*hwDpVz$}{9mt^FY?Cy`TwF+UaUbBxl7_%FbFA{G@`a%uP?;7a)^PoM_V5`pIhN+>_tT$Z509Mab zv(~)J@+v8WttnYLd_|(#@@3*wAZeJPo=4=66>PnYKhA`~t=)LLbWh2ArB5@k|ML^y z2rHipy2h6T0&eH8pZ^(W#dA8In6Nr(Sajn6xX6pP61b*#T-l z?=ayID+`hBR442nOd{PAV-R$&Z2vJ*98&${mQcq38ncOoQa$H5vTK-7`B=Uk-1jaI z?RK>DJ|-R6lMV$ZV|!XIkwGJUQGJwU&ly9O4NY@EYBqcFXPFTCV~71tGCu=8mDl{w zjn-3n9S3v}VtJZIKcs)&)^|L<3E>3n;RH+t?dn;cSeM@e(wfk50f9T%#%O}wy2yy@ zL7A@DbT1pzE$0ep#sVy`tn-YS{?zqWM|GOdQ{Fm67!LIo2{$5LW-*^VU!NE`UpxBu zR5vaaq8*lme6WUk10%~CP`ej7)elzht4Qh@*M##UZjh4K+)fW z%vBUX%kIT<+;;~u`bS!15S;vVuO)%#e0=|(<)x(qH2$$1GE?@39xmZ!j(Ov6?GXuhtMvp zpNQl*@gaxT>x(Sw{ie>pQ9fWE=?*DOGT*!FCC?C^3bN!KS+23WSjn7+O9Ot zPmA*>D-P6`LY4G63~?|dy$igSSkTrsn0vskyA%4A&4^zZ2Eh}W#vrbF$w!>(MCj^~ zJ=k?CIn~%>s|_*8kqG&o3}_koLirc1M$Gpv_iXumpi5Wl4WQx+!(d=0+f)3DFFr!8 zne#4HGWu3^OD+|%a3t+4pXy<4p_cjYy#HexxqA7IHoj5<>tF!+430~Av$H8 zvJ!DjeOOv?1^uFC&ejTHlhAWo*6<8>yq*27#&LeZ1Q{x5ACY%Vby1{vf(QRvobREl z@;bHoPZ<^8ONdn= z>ImaH|I_o2Jr#$&5MJ%>{^SV{MwJnp){AM)twdu@>R(@u!!s<7F5LOoQ}FuK^#UZG z`Ym67!wv9F!`Xi_>BzM6GdpkXWtw;xDOx7g7yD^IT`N`|9sDU&+pxdmXAeoB<{=4X z^X%q^jpL~XXr^Oi5owP-bg)-w$nenO6>t1ijl(ZH3TxlrZ!nFbYeed-`tuza`#nj+ zHu#(eb7K_gy_zg%5srSNzmb78Nb!s}BGQu^_at5Kqx3pS%e~p-&705dSCnBYGSp4n zzMSMYnAq;Z_UT2w{Z$R=alTaUd}s_TUm@{2LQc}k3HO|m{%;RA++#*i>Ayh#y`1*F zCiXqjhlhKQ^8bVPcWfMZASD~MC@zN>9=qyy8cw7C1^TcsFd~50?)JuL`|a_@(9qCm zlsMCok&%Z=BOBD+-P`_2gQ7o)y4dLM@QeZX{N5>KnjCTdE>Tm-{v3(ldk^iqoaLWMveoHqsr^7 z&cKvOQf$?xM#6x{C0<i2Bp3KV{4ZR`W|b#tf0mk@)!~9 z2Q^EEOoPAec8llg60IxM5L6`+mN&xgf0b0y221 zu_gF%%93kZnEMUTQb&i5j_zrGq!1)`lso7?j?&f_E*JeklF`>U*Au_@TP(YtOj0B+ zJt=<4#6%(QId!r}o#$Z2>cTeZc8^EG+*Un1PuExB=}|WAc1KI*fxb>qBD5mG<@LdW48KR&^FNhqlxd2zk<82ql!tnd2#uE6gA5BRT+>HpPd9C&OX zCABC5E(Z-8Cmy?)D*jh-WtHy^|I_OC0&)?+8=&Os79SzN#9JQSyUi0HA;NWi%VSX{ znlqMCo6X2ZsCr=lf@RyWzGW57OhaWt-6WZ%E19QcCG!^|lLbM^4lRvO{wVl2Fib+; zRhs~l=g-fdutOT*mOU<)%B83@_1UJQO)DI%1;D8Rv5-^7rC5t&#ko#X+^uX0{4+Af zsz0@{i#l-k`zo;BGAkBlKgp_ArSB7?x9AZe{8a?58KJZ-ejkvzLYMwkusq{9)zgkG z7Jk~=Qq$cuT2{MQQPx-2R#mCf`G|>*i4s}MzNi#=FnDUJpFFQc)=cL&KhvB1!eM^g z9ErMeQYLX*6cUtTs%KG``xv=E3<5W__J6^#Zu>;m#+&CW^?u%5KL$Qa5;ok<>qMZ&SP~VXkh^X~XdWKbPS)4L$=Y$9)7Y$T>Ys6wwlypP$6%!VZ)4CLr+NnBSyS(Hg zFf*VI)28(^vQb-j&6(|)EfKO7vX(biqaFNM^@Lvu+Uona5Sbh{h5WG zePQZOHj9utQ^v6EhC2eim|MKC-5lS7q=WWhD<=8SL=}6SOECGSsWyFiOO>y5y`(AHU1_rZ-_FrfE!EuQ=L)rOv z*H(DRWTRXIuXZ#nVY0T2ab{fXgl?AE<`3raXyW~SIBa%Kkj9f-3uO4E@8l)Ivq9BH z4UVF&f&mTnTYi|Ac1D@2ifaltr2KmBRG!EMikhyg$6jsEpUAVlJ^k)|guR@4^0BdV zd^wt2J9Vu;rDv13uWV^5cOs5702M#EYu~V{E7B3qfMcp;Ix4H1$_J|T+k2||DyzUq p(G~5+&Bx`NzEzzrw-lSqFEVor&(f~ur{@L7<(VQUIDT&n{}1dd5yJog literal 0 HcmV?d00001 diff --git a/website/templates/company/team.html b/website/templates/company/team.html index 28efdcfde06..b4ed1c26a29 100644 --- a/website/templates/company/team.html +++ b/website/templates/company/team.html @@ -6,7 +6,7 @@ {{ _('ClickHouse Team') }} -
+
+
+ + + + +

+ {{ _('Mikhail Shiryaev') }} +

+

+ {{ _('Site Reliability Engineer') }} +

+
@@ -358,7 +371,7 @@

-
+ -
+
+ + + + +

+ {{ _('Christoph Wurm') }} +

+

+ {{ _('Solutions Architect') }} +

+ +
+
From 864761b1bec248841deecd2d229b26bfa0ef50e1 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 18 Nov 2021 02:05:41 +0300 Subject: [PATCH 122/200] [ci] whitelist codyrobert --- tests/ci/run_check.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 1a4717b88be..49cc809dd9e 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -34,6 +34,7 @@ TRUSTED_CONTRIBUTORS = { "bharatnc", # Newbie, but already with many contributions. "bobrik", # Seasoned contributor, CloundFlare "BohuTANG", + "codyrobert", # Flickerbox engineer "damozhaeva", # DOCSUP "den-crane", "gyuton", # DOCSUP From 2248f43a993c1d6bebc145815ef86d82e25a6b9f Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 18 Nov 2021 08:03:49 +0300 Subject: [PATCH 123/200] Update tuple-map-functions.md --- docs/en/sql-reference/functions/tuple-map-functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/tuple-map-functions.md b/docs/en/sql-reference/functions/tuple-map-functions.md index 47e33806930..8ead8c58c7a 100644 --- a/docs/en/sql-reference/functions/tuple-map-functions.md +++ b/docs/en/sql-reference/functions/tuple-map-functions.md @@ -320,7 +320,7 @@ Can be optimized by enabling the [optimize_functions_to_subcolumns](../../operat **Syntax** ```sql -mapKeys(map) +mapValues(map) ``` **Parameters** From 2636a1b569c51f6a01ee8e830ec6cf4e4e7858a5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Nov 2021 10:42:46 +0300 Subject: [PATCH 124/200] Add results from Panelbear --- .../hardware/results/hetzner_epyc.json | 106 ++++++++++++++++++ .../hardware/results/scaleway_epyc.json | 54 +++++++++ 2 files changed, 160 insertions(+) create mode 100644 website/benchmark/hardware/results/hetzner_epyc.json create mode 100644 website/benchmark/hardware/results/scaleway_epyc.json diff --git a/website/benchmark/hardware/results/hetzner_epyc.json b/website/benchmark/hardware/results/hetzner_epyc.json new file mode 100644 index 00000000000..4ced699a56d --- /dev/null +++ b/website/benchmark/hardware/results/hetzner_epyc.json @@ -0,0 +1,106 @@ +[ + { + "system": "Hetzner CCX22", + "system_full": "Hetzner CCX22 (AMD EPYC 7003, 4 cores, 16 GiB RAM, NVMe)", + "time": "2021-11-17 00:00:00", + "kind": "server", + "result": + [ +[0.001, 0.001, 0.001], +[0.036, 0.023, 0.039], +[0.130, 0.102, 0.092], +[0.304, 0.148, 0.141], +[0.431, 0.298, 0.291], +[1.492, 1.359, 1.357], +[0.088, 0.087, 0.091], +[0.058, 0.039, 0.042], +[1.612, 1.477, 1.473], +[2.017, 1.805, 1.809], +[1.044, 0.925, 0.926], +[1.167, 1.050, 1.048], +[2.621, 2.447, 2.447], +[3.426, 3.176, 3.193], +[3.545, 3.475, 3.431], +[2.958, 2.805, 2.816], +[8.547, 8.320, 8.321], +[6.395, 5.992, 6.081], +[16.542, 16.407, 16.057], +[0.404, 0.166, 0.156], +[4.338, 3.419, 3.373], +[5.042, 4.102, 4.052], +[10.231, 8.420, 8.304], +[6.121, 3.904, 3.804], +[1.582, 1.297, 1.279], +[1.316, 1.183, 1.171], +[1.565, 1.305, 1.296], +[4.098, 3.290, 3.246], +[5.999, 5.242, 5.205], +[2.247, 2.198, 2.183], +[2.581, 2.336, 2.242], +[3.269, 2.806, 2.744], +[14.252, 14.052, 13.956], +[11.730, 10.638, 10.632], +[11.418, 10.659, 10.572], +[4.170, 4.086, 4.092], +[0.208, 0.173, 0.159], +[0.082, 0.075, 0.069], +[0.082, 0.062, 0.065], +[0.413, 0.392, 0.375], +[0.046, 0.021, 0.029], +[0.032, 0.016, 0.017], +[0.005, 0.004, 0.007] + ] + }, + { + "system": "Hetzner CCX32", + "system_full": "Hetzner CCX32 (AMD EPYC 7003, 8 cores, 32 GiB RAM, NVMe)", + "time": "2021-11-17 00:00:00", + "kind": "server", + "result": + [ +[0.001, 0.001, 0.001], +[0.021, 0.018, 0.017], +[0.078, 0.057, 0.063], +[0.178, 0.083, 0.076], +[0.229, 0.191, 0.182], +[1.141, 1.063, 0.977], +[0.071, 0.051, 0.068], +[0.056, 0.022, 0.035], +[1.043, 1.288, 1.272], +[1.757, 1.003, 0.996], +[0.554, 0.492, 0.555], +[0.931, 0.698, 0.582], +[1.471, 1.364, 1.310], +[2.284, 2.040, 1.720], +[1.852, 1.749, 1.710], +[1.551, 1.496, 1.482], +[4.852, 4.310, 4.964], +[3.384, 3.353, 3.015], +[10.150, 9.422, 10.005], +[0.230, 0.091, 0.089], +[3.525, 1.731, 1.721], +[2.939, 2.325, 2.077], +[7.716, 5.046, 4.394], +[3.927, 2.023, 1.951], +[0.848, 0.732, 0.874], +[1.005, 0.627, 0.606], +[0.968, 0.725, 0.687], +[2.771, 2.453, 1.815], +[3.536, 3.283, 3.020], +[1.661, 1.690, 1.761], +[1.511, 1.213, 1.205], +[2.002, 1.715, 1.518], +[8.160, 8.943, 8.982], +[6.999, 5.827, 6.024], +[7.777, 6.634, 6.338], +[2.391, 2.285, 2.284], +[0.221, 0.182, 0.196], +[0.114, 0.072, 0.069], +[0.096, 0.063, 0.065], +[0.423, 0.382, 0.405], +[0.077, 0.022, 0.024], +[0.030, 0.022, 0.018], +[0.011, 0.004, 0.008] + ] + } +] diff --git a/website/benchmark/hardware/results/scaleway_epyc.json b/website/benchmark/hardware/results/scaleway_epyc.json new file mode 100644 index 00000000000..54cf6eaf459 --- /dev/null +++ b/website/benchmark/hardware/results/scaleway_epyc.json @@ -0,0 +1,54 @@ +[ + { + "system": "Scaleway GP1-XS", + "system_full": "Scaleway GP1-XS (AMD EPYC 7401P, 4 cores, 16 GiB RAM, NVMe)", + "time": "2021-11-17 00:00:00", + "kind": "server", + "result": + [ +[0.002, 0.002, 0.002], +[0.044, 0.026, 0.028], +[0.100, 0.076, 0.067], +[0.151, 0.101, 0.102], +[0.276, 0.218, 0.207], +[0.740, 0.693, 0.703], +[0.066, 0.054, 0.050], +[0.062, 0.035, 0.041], +[1.271, 1.124, 1.141], +[1.441, 1.279, 1.280], +[0.438, 0.382, 0.376], +[0.514, 0.485, 0.467], +[1.914, 1.664, 1.694], +[2.367, 2.277, 2.258], +[2.143, 2.066, 2.131], +[1.923, 1.826, 1.777], +[5.894, 5.653, 5.765], +[3.545, 3.464, 3.405], +[12.060, 12.893, 13.049], +[0.196, 0.121, 0.118], +[2.328, 1.841, 1.808], +[2.498, 2.100, 2.067], +[5.839, 5.094, 5.078], +[3.068, 2.255, 2.202], +[0.718, 0.611, 0.616], +[0.597, 0.531, 0.529], +[0.702, 0.615, 0.592], +[2.310, 1.991, 1.969], +[3.540, 3.222, 3.179], +[3.950, 3.977, 3.876], +[1.527, 1.319, 1.319], +[2.264, 1.950, 1.927], +[11.987, 11.644, 11.777], +[10.142, 9.150, 9.204], +[9.627, 9.298, 9.183], +[2.937, 2.812, 2.849], +[0.229, 0.226, 0.227], +[0.096, 0.097, 0.095], +[0.087, 0.074, 0.071], +[0.464, 0.447, 0.463], +[0.037, 0.027, 0.032], +[0.030, 0.046, 0.029], +[0.006, 0.017, 0.014] + ] + } +] From b8c3785af33e81e9ecadc8d808c016accb652f1d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Nov 2021 10:44:36 +0300 Subject: [PATCH 125/200] Credits --- website/benchmark/hardware/index.html | 1 + 1 file changed, 1 insertion(+) diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index 84764403766..76b526c4d47 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -79,6 +79,7 @@ Results for ASUS A15 (Ryzen laptop) are from Kimmo Linna.
Results for MacBook Air M1 are from Denis Glazachev.
Results for Xeon Gold 6140 are from Shiv Iyer (ChistaDATA Labs).
Comparison of EBS and EFS is from Ramazan Polat.
+Results for Hetzner and Scaleway are from Anthony Najjar Simon (Panelbear).

From 8b622bd4ffc4ebe5f4b9559f080bc6a1ccebf515 Mon Sep 17 00:00:00 2001 From: Vladimir Chebotarev Date: Thu, 18 Nov 2021 10:54:59 +0300 Subject: [PATCH 126/200] Fixed a typo. --- src/IO/S3Common.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/IO/S3Common.cpp b/src/IO/S3Common.cpp index 314576e6f0a..900c68f76ad 100644 --- a/src/IO/S3Common.cpp +++ b/src/IO/S3Common.cpp @@ -370,9 +370,8 @@ class AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider : public Aws::Auth::AWS /// See STSAssumeRoleWebIdentityCredentialsProvider. public: - AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider(DB::S3::PocoHTTPClientConfiguration & aws_client_configuration) + explicit AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider(DB::S3::PocoHTTPClientConfiguration & aws_client_configuration) : logger(&Poco::Logger::get("AwsAuthSTSAssumeRoleWebIdentityCredentialsProvider")) - { // check environment variables String tmp_region = Aws::Environment::GetEnv("AWS_DEFAULT_REGION"); From d66a74237197987fa3ef9b6c5e3824f3542e2314 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 18 Nov 2021 11:25:57 +0300 Subject: [PATCH 127/200] Update nyc-taxi.md --- docs/en/getting-started/example-datasets/nyc-taxi.md | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/docs/en/getting-started/example-datasets/nyc-taxi.md b/docs/en/getting-started/example-datasets/nyc-taxi.md index ff29fef7fe0..64810d3fa37 100644 --- a/docs/en/getting-started/example-datasets/nyc-taxi.md +++ b/docs/en/getting-started/example-datasets/nyc-taxi.md @@ -381,8 +381,11 @@ We ran queries using a client located in a Yandex datacenter in Finland on a clu | servers | Q1 | Q2 | Q3 | Q4 | |---------|-------|-------|-------|-------| -| 1 | 0.490 | 1.224 | 2.104 | 3.593 | -| 3 | 0.212 | 0.438 | 0.733 | 1.241 | -| 140 | 0.028 | 0.043 | 0.051 | 0.072 | +| 1, E5-2650v2 | 0.490 | 1.224 | 2.104 | 3.593 | +| 3, E5-2650v2 | 0.212 | 0.438 | 0.733 | 1.241 | +| 1, AWS c5n.4xlarge | 0.249 | 1.279 | 1.738 | 3.527 | +| 1, AWS c5n.9xlarge | 0.130 | 0.584 | 0.777 | 1.811 | +| 3, AWS c5n.9xlarge | 0.057 | 0.231 | 0.285 | 0.641 | +| 140, E5-2650v2 | 0.028 | 0.043 | 0.051 | 0.072 | [Original article](https://clickhouse.com/docs/en/getting_started/example_datasets/nyc_taxi/) From 11ab1f1ae5b324a1cdef5e6d1f6b0f7d5e0f37f6 Mon Sep 17 00:00:00 2001 From: kssenii Date: Thu, 18 Nov 2021 08:45:59 +0000 Subject: [PATCH 128/200] Fix --- src/Functions/FunctionSQLJSON.h | 8 +++++--- .../0_stateless/01889_sql_json_functions.reference | 4 ++++ tests/queries/0_stateless/01889_sql_json_functions.sql | 1 + 3 files changed, 10 insertions(+), 3 deletions(-) diff --git a/src/Functions/FunctionSQLJSON.h b/src/Functions/FunctionSQLJSON.h index db28663c64f..d860da62b9d 100644 --- a/src/Functions/FunctionSQLJSON.h +++ b/src/Functions/FunctionSQLJSON.h @@ -268,13 +268,15 @@ public: out << current_element.getElement(); auto output_str = out.str(); ColumnString & col_str = assert_cast(dest); + ColumnString::Chars & data = col_str.getChars(); + ColumnString::Offsets & offsets = col_str.getOffsets(); if (current_element.isString()) { ReadBufferFromString buf(output_str); - String unquoted_output_str; - readJSONString(unquoted_output_str, buf); - col_str.insertData(unquoted_output_str.data(), unquoted_output_str.size()); + readJSONStringInto(data, buf); + data.push_back(0); + offsets.push_back(data.size()); } else { diff --git a/tests/queries/0_stateless/01889_sql_json_functions.reference b/tests/queries/0_stateless/01889_sql_json_functions.reference index 4ed6a9ee611..c2c106e8632 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.reference +++ b/tests/queries/0_stateless/01889_sql_json_functions.reference @@ -98,4 +98,8 @@ SELECT id, JSON_QUERY(json, '$.friends[0 to 2]') FROM 01889_sql_json ORDER BY id 0 ["Vasily", "Kostya"] 1 ["Tihon", "Ernest"] 2 ["Katya", "Anatoliy"] +SELECT id, JSON_VALUE(json, '$.friends[0]') FROM 01889_sql_json ORDER BY id; +0 Vasily +1 Tihon +2 Katya DROP TABLE 01889_sql_json; diff --git a/tests/queries/0_stateless/01889_sql_json_functions.sql b/tests/queries/0_stateless/01889_sql_json_functions.sql index 967e21df787..e816443382c 100644 --- a/tests/queries/0_stateless/01889_sql_json_functions.sql +++ b/tests/queries/0_stateless/01889_sql_json_functions.sql @@ -55,4 +55,5 @@ INSERT INTO 01889_sql_json(id, json) VALUES(0, '{"name":"Ivan","surname":"Ivanov INSERT INTO 01889_sql_json(id, json) VALUES(1, '{"name":"Katya","surname":"Baltica","friends":["Tihon","Ernest","Innokentiy"]}'); INSERT INTO 01889_sql_json(id, json) VALUES(2, '{"name":"Vitali","surname":"Brown","friends":["Katya","Anatoliy","Ivan","Oleg"]}'); SELECT id, JSON_QUERY(json, '$.friends[0 to 2]') FROM 01889_sql_json ORDER BY id; +SELECT id, JSON_VALUE(json, '$.friends[0]') FROM 01889_sql_json ORDER BY id; DROP TABLE 01889_sql_json; From 8d2a083bf9c552f1d518344e0bbf96ea057a25ff Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Thu, 18 Nov 2021 12:50:41 +0300 Subject: [PATCH 129/200] Update CHANGELOG.md --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 686f0072005..f34725448f2 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -17,7 +17,7 @@ * Support `EXISTS (subquery)`. Closes [#6852](https://github.com/ClickHouse/ClickHouse/issues/6852). [#29731](https://github.com/ClickHouse/ClickHouse/pull/29731) ([Kseniia Sumarokova](https://github.com/kssenii)). * Session logging for audit. Logging all successful and failed login and logout events to a new `system.session_log` table. [#22415](https://github.com/ClickHouse/ClickHouse/pull/22415) ([Vasily Nemkov](https://github.com/Enmk)) ([Vitaly Baranov](https://github.com/vitlibar)). * Support multidimensional cosine distance and euclidean distance functions; L1, L2, Lp, Linf distances and norms. Scalar product on tuples and various arithmetic operators on tuples. This fully closes [#4509](https://github.com/ClickHouse/ClickHouse/issues/4509) and even more. [#27933](https://github.com/ClickHouse/ClickHouse/pull/27933) ([Alexey Boykov](https://github.com/mathalex)). -* Add support for compression and decompression for `INTO OUTPUT` and `FROM INFILE` (with autodetect or with additional optional parameter). [#27135](https://github.com/ClickHouse/ClickHouse/pull/27135) ([Filatenkov Artur](https://github.com/FArthur-cmd)). +* Add support for compression and decompression for `INTO OUTFILE` and `FROM INFILE` (with autodetect or with additional optional parameter). [#27135](https://github.com/ClickHouse/ClickHouse/pull/27135) ([Filatenkov Artur](https://github.com/FArthur-cmd)). * Add CORS (Cross Origin Resource Sharing) support with HTTP `OPTIONS` request. It means, now Grafana will work with serverless requests without a kludges. Closes [#18693](https://github.com/ClickHouse/ClickHouse/issues/18693). [#29155](https://github.com/ClickHouse/ClickHouse/pull/29155) ([Filatenkov Artur](https://github.com/FArthur-cmd)). * Queries with JOIN ON now supports disjunctions (OR). [#21320](https://github.com/ClickHouse/ClickHouse/pull/21320) ([Ilya Golshtein](https://github.com/ilejn)). * Added function `tokens`. That allow to split string into tokens using non-alpha numeric ASCII characters as separators. [#29981](https://github.com/ClickHouse/ClickHouse/pull/29981) ([Maksim Kita](https://github.com/kitaisreal)). Added function `ngrams` to extract ngrams from text. Closes [#29699](https://github.com/ClickHouse/ClickHouse/issues/29699). [#29738](https://github.com/ClickHouse/ClickHouse/pull/29738) ([Maksim Kita](https://github.com/kitaisreal)). From ff46e8bb51a6189cbe5a526309a60365e6a528cc Mon Sep 17 00:00:00 2001 From: Stig Bakken Date: Wed, 10 Nov 2021 14:13:27 +0100 Subject: [PATCH 130/200] Drop support for DatabaseOrdinary in MaterializedMySQL 1. Dropped support for DatabaseOrdinary for MaterializeMySQL. It is marked as experimental, and dropping support makes the code more maintaible, and speeds up integration tests by 50%. 2. Get rid of thread name logic for StorageMaterializeMySQL wrapping, use setInternalQuery instead (similar to MaterializedPostgreSQL). --- programs/local/LocalServer.cpp | 4 +- programs/server/Server.cpp | 4 +- src/Databases/DatabaseAtomic.cpp | 4 +- src/Databases/DatabaseAtomic.h | 4 +- src/Databases/DatabaseFactory.cpp | 11 +- src/Databases/DatabaseLazy.cpp | 8 +- src/Databases/DatabaseLazy.h | 4 +- src/Databases/DatabaseOnDisk.cpp | 20 +- src/Databases/DatabaseOrdinary.cpp | 2 +- src/Databases/DatabaseReplicated.h | 5 +- src/Databases/DatabasesCommon.cpp | 4 +- src/Databases/DatabasesCommon.h | 4 +- src/Databases/IDatabase.h | 4 +- src/Databases/IDatabaseReplicating.h | 13 ++ .../MySQL/DatabaseMaterializedMySQL.cpp | 177 +++++------------- .../MySQL/DatabaseMaterializedMySQL.h | 33 ++-- src/Databases/MySQL/DatabaseMySQL.cpp | 8 +- src/Databases/MySQL/DatabaseMySQL.h | 4 +- .../MySQL/MaterializedMySQLSyncThread.cpp | 10 +- .../MySQL/MaterializedMySQLSyncThread.h | 2 - .../DatabaseMaterializedPostgreSQL.cpp | 10 +- .../DatabaseMaterializedPostgreSQL.h | 9 +- .../PostgreSQL/DatabasePostgreSQL.cpp | 8 +- src/Databases/PostgreSQL/DatabasePostgreSQL.h | 4 +- src/Interpreters/DatabaseCatalog.cpp | 5 +- src/Interpreters/InterpreterDropQuery.cpp | 64 +++---- src/Interpreters/InterpreterDropQuery.h | 2 +- src/Interpreters/InterpreterSystemQuery.cpp | 4 +- src/Storages/StorageMaterializedMySQL.cpp | 9 +- src/Storages/System/attachSystemTables.cpp | 145 +++++++------- src/Storages/System/attachSystemTables.h | 9 +- src/Storages/System/attachSystemTablesImpl.h | 6 +- ..._transform_query_for_external_database.cpp | 1 + tests/integration/helpers/cluster.py | 2 +- .../configs/users.xml | 2 +- .../configs/users_db_atomic.xml | 19 -- .../materialize_with_ddl.py | 2 +- .../test_materialized_mysql_database/test.py | 85 ++++----- 38 files changed, 305 insertions(+), 406 deletions(-) create mode 100644 src/Databases/IDatabaseReplicating.h delete mode 100644 tests/integration/test_materialized_mysql_database/configs/users_db_atomic.xml diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 0aaf78e70a0..815f918a9c9 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -622,7 +622,7 @@ void LocalServer::processConfig() fs::create_directories(fs::path(path) / "metadata/"); loadMetadataSystem(global_context); - attachSystemTablesLocal(*createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE)); + attachSystemTablesLocal(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE)); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA)); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)); loadMetadata(global_context); @@ -633,7 +633,7 @@ void LocalServer::processConfig() } else if (!config().has("no-system-tables")) { - attachSystemTablesLocal(*createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE)); + attachSystemTablesLocal(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::SYSTEM_DATABASE)); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA)); attachInformationSchema(global_context, *createMemoryDatabaseIfNotExists(global_context, DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 5cd295070a6..d61a5162780 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -1146,7 +1146,7 @@ if (ThreadFuzzer::instance().isEffective()) global_context->initializeSystemLogs(); global_context->setSystemZooKeeperLogAfterInitializationIfNeeded(); /// After the system database is created, attach virtual system tables (in addition to query_log and part_log) - attachSystemTablesServer(*database_catalog.getSystemDatabase(), has_zookeeper); + attachSystemTablesServer(global_context, *database_catalog.getSystemDatabase(), has_zookeeper); attachInformationSchema(global_context, *database_catalog.getDatabase(DatabaseCatalog::INFORMATION_SCHEMA)); attachInformationSchema(global_context, *database_catalog.getDatabase(DatabaseCatalog::INFORMATION_SCHEMA_UPPERCASE)); /// Firstly remove partially dropped databases, to avoid race with MaterializedMySQLSyncThread, @@ -1256,7 +1256,7 @@ if (ThreadFuzzer::instance().isEffective()) /// This object will periodically calculate some metrics. AsynchronousMetrics async_metrics( global_context, config().getUInt("asynchronous_metrics_update_period_s", 1), servers_to_start_before_tables, servers); - attachSystemTablesAsync(*DatabaseCatalog::instance().getSystemDatabase(), async_metrics); + attachSystemTablesAsync(global_context, *DatabaseCatalog::instance().getSystemDatabase(), async_metrics); for (const auto & listen_host : listen_hosts) { diff --git a/src/Databases/DatabaseAtomic.cpp b/src/Databases/DatabaseAtomic.cpp index 76e836a9b71..91a83ebf35c 100644 --- a/src/Databases/DatabaseAtomic.cpp +++ b/src/Databases/DatabaseAtomic.cpp @@ -84,7 +84,7 @@ void DatabaseAtomic::drop(ContextPtr) fs::remove_all(getMetadataPath()); } -void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) +void DatabaseAtomic::attachTable(ContextPtr /* context_ */, const String & name, const StoragePtr & table, const String & relative_table_path) { assert(relative_table_path != data_path && !relative_table_path.empty()); DetachedTables not_in_use; @@ -96,7 +96,7 @@ void DatabaseAtomic::attachTable(const String & name, const StoragePtr & table, table_name_to_path.emplace(std::make_pair(name, relative_table_path)); } -StoragePtr DatabaseAtomic::detachTable(const String & name) +StoragePtr DatabaseAtomic::detachTable(ContextPtr /* context */, const String & name) { DetachedTables not_in_use; std::unique_lock lock(mutex); diff --git a/src/Databases/DatabaseAtomic.h b/src/Databases/DatabaseAtomic.h index 4eee2338a3f..eae700d28c5 100644 --- a/src/Databases/DatabaseAtomic.h +++ b/src/Databases/DatabaseAtomic.h @@ -37,8 +37,8 @@ public: void dropTable(ContextPtr context, const String & table_name, bool no_delay) override; - void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) override; - StoragePtr detachTable(const String & name) override; + void attachTable(ContextPtr context, const String & name, const StoragePtr & table, const String & relative_table_path) override; + StoragePtr detachTable(ContextPtr context, const String & name) override; String getTableDataPath(const String & table_name) const override; String getTableDataPath(const ASTCreateQuery & query) const override; diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 74b103afbc4..bf24b0b5343 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -211,14 +211,9 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (engine_define->settings) materialize_mode_settings->loadFromQuery(*engine_define); - if (create.uuid == UUIDHelpers::Nil) - return std::make_shared>( - context, database_name, metadata_path, uuid, configuration.database, std::move(mysql_pool), - std::move(client), std::move(materialize_mode_settings)); - else - return std::make_shared>( - context, database_name, metadata_path, uuid, configuration.database, std::move(mysql_pool), - std::move(client), std::move(materialize_mode_settings)); + return std::make_shared( + context, database_name, metadata_path, uuid, configuration.database, std::move(mysql_pool), + std::move(client), std::move(materialize_mode_settings)); } catch (...) { diff --git a/src/Databases/DatabaseLazy.cpp b/src/Databases/DatabaseLazy.cpp index f4336fc7ef9..4b3e06e318e 100644 --- a/src/Databases/DatabaseLazy.cpp +++ b/src/Databases/DatabaseLazy.cpp @@ -39,7 +39,7 @@ DatabaseLazy::DatabaseLazy(const String & name_, const String & metadata_path_, void DatabaseLazy::loadStoredObjects( ContextMutablePtr local_context, bool /* force_restore */, bool /*force_attach*/, bool /* skip_startup_tables */) { - iterateMetadataFiles(local_context, [this](const String & file_name) + iterateMetadataFiles(local_context, [this, &local_context](const String & file_name) { const std::string table_name = unescapeForFileName(file_name.substr(0, file_name.size() - 4)); @@ -50,7 +50,7 @@ void DatabaseLazy::loadStoredObjects( return; } - attachTable(table_name, nullptr, {}); + attachTable(local_context, table_name, nullptr, {}); }); } @@ -160,7 +160,7 @@ bool DatabaseLazy::empty() const return tables_cache.empty(); } -void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & table, const String &) +void DatabaseLazy::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & table, const String &) { LOG_DEBUG(log, "Attach table {}.", backQuote(table_name)); std::lock_guard lock(mutex); @@ -175,7 +175,7 @@ void DatabaseLazy::attachTable(const String & table_name, const StoragePtr & tab it->second.expiration_iterator = cache_expiration_queue.emplace(cache_expiration_queue.end(), current_time, table_name); } -StoragePtr DatabaseLazy::detachTable(const String & table_name) +StoragePtr DatabaseLazy::detachTable(ContextPtr /* context */, const String & table_name) { StoragePtr res; { diff --git a/src/Databases/DatabaseLazy.h b/src/Databases/DatabaseLazy.h index 45c816c2e76..3a7d7b14be1 100644 --- a/src/Databases/DatabaseLazy.h +++ b/src/Databases/DatabaseLazy.h @@ -64,9 +64,9 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; - void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path) override; + void attachTable(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) override; - StoragePtr detachTable(const String & table_name) override; + StoragePtr detachTable(ContextPtr context, const String & table_name) override; void shutdown() override; diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index a1e0a825736..3f27350657c 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -185,7 +185,7 @@ void DatabaseOnDisk::createTable( { /// Metadata already exists, table was detached removeDetachedPermanentlyFlag(local_context, table_name, table_metadata_path, true); - attachTable(table_name, table, getTableDataPath(create)); + attachTable(getContext(), table_name, table, getTableDataPath(create)); return; } @@ -246,12 +246,12 @@ void DatabaseOnDisk::removeDetachedPermanentlyFlag(ContextPtr, const String & ta void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, const String & table_metadata_tmp_path, const String & table_metadata_path, - ContextPtr /*query_context*/) + ContextPtr query_context) { try { /// Add a table to the map of known tables. - attachTable(query.getTable(), table, getTableDataPath(query)); + attachTable(query_context, query.getTable(), table, getTableDataPath(query)); /// If it was ATTACH query and file with table metadata already exist /// (so, ATTACH is done after DETACH), then rename atomically replaces old file with new one. @@ -264,9 +264,9 @@ void DatabaseOnDisk::commitCreateTable(const ASTCreateQuery & query, const Stora } } -void DatabaseOnDisk::detachTablePermanently(ContextPtr, const String & table_name) +void DatabaseOnDisk::detachTablePermanently(ContextPtr query_context, const String & table_name) { - auto table = detachTable(table_name); + auto table = detachTable(query_context, table_name); fs::path detached_permanently_flag(getObjectMetadataPath(table_name) + detached_suffix); try @@ -288,7 +288,7 @@ void DatabaseOnDisk::dropTable(ContextPtr local_context, const String & table_na if (table_data_path_relative.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Path is empty"); - StoragePtr table = detachTable(table_name); + StoragePtr table = detachTable(local_context, table_name); /// This is possible for Lazy database. if (!table) @@ -309,7 +309,7 @@ void DatabaseOnDisk::dropTable(ContextPtr local_context, const String & table_na catch (...) { LOG_WARNING(log, getCurrentExceptionMessage(__PRETTY_FUNCTION__)); - attachTable(table_name, table, table_data_path_relative); + attachTable(local_context, table_name, table, table_data_path_relative); if (renamed) fs::rename(table_metadata_path_drop, table_metadata_path); throw; @@ -372,7 +372,7 @@ void DatabaseOnDisk::renameTable( ASTPtr attach_query; /// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case. StoragePtr table = tryGetTable(table_name, getContext()); - detachTable(table_name); + detachTable(local_context, table_name); UUID prev_uuid = UUIDHelpers::Nil; try { @@ -397,12 +397,12 @@ void DatabaseOnDisk::renameTable( } catch (const Exception &) { - attachTable(table_name, table, table_data_relative_path); + attachTable(local_context, table_name, table, table_data_relative_path); throw; } catch (const Poco::Exception & e) { - attachTable(table_name, table, table_data_relative_path); + attachTable(local_context, table_name, table, table_data_relative_path); /// Better diagnostics. throw Exception{Exception::CreateFromPocoTag{}, e}; } diff --git a/src/Databases/DatabaseOrdinary.cpp b/src/Databases/DatabaseOrdinary.cpp index 2da17ed388f..b7a0aff24d6 100644 --- a/src/Databases/DatabaseOrdinary.cpp +++ b/src/Databases/DatabaseOrdinary.cpp @@ -50,7 +50,7 @@ namespace context, force_restore); - database.attachTable(table_name, table, database.getTableDataPath(query)); + database.attachTable(context, table_name, table, database.getTableDataPath(query)); } catch (Exception & e) { diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 34cfb7df151..5c0a97a3296 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -2,6 +2,7 @@ #include #include +#include #include #include #include @@ -17,7 +18,7 @@ using ZooKeeperPtr = std::shared_ptr; class Cluster; using ClusterPtr = std::shared_ptr; -class DatabaseReplicated : public DatabaseAtomic +class DatabaseReplicated : public DatabaseAtomic, IDatabaseReplicating { public: DatabaseReplicated(const String & name_, const String & metadata_path_, UUID uuid, @@ -46,7 +47,7 @@ public: /// then it will be executed on all replicas. BlockIO tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context); - void stopReplication(); + void stopReplication() override; String getFullReplicaName() const; static std::pair parseFullReplicaName(const String & name); diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index c92aac44083..9f28a5fd71a 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -125,7 +125,7 @@ bool DatabaseWithOwnTablesBase::empty() const return tables.empty(); } -StoragePtr DatabaseWithOwnTablesBase::detachTable(const String & table_name) +StoragePtr DatabaseWithOwnTablesBase::detachTable(ContextPtr /* context_ */, const String & table_name) { std::unique_lock lock(mutex); return detachTableUnlocked(table_name, lock); @@ -152,7 +152,7 @@ StoragePtr DatabaseWithOwnTablesBase::detachTableUnlocked(const String & table_n return res; } -void DatabaseWithOwnTablesBase::attachTable(const String & table_name, const StoragePtr & table, const String &) +void DatabaseWithOwnTablesBase::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & table, const String &) { std::unique_lock lock(mutex); attachTableUnlocked(table_name, table, lock); diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index 59a2ddc3c41..c8811aef5fd 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -27,9 +27,9 @@ public: bool empty() const override; - void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path) override; + void attachTable(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) override; - StoragePtr detachTable(const String & table_name) override; + StoragePtr detachTable(ContextPtr context, const String & table_name) override; DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context, const FilterByNameFunction & filter_by_table_name) const override; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index 5d0a10a7df3..da8915ee751 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -197,13 +197,13 @@ public: /// Add a table to the database, but do not add it to the metadata. The database may not support this method. /// /// Note: ATTACH TABLE statement actually uses createTable method. - virtual void attachTable(const String & /*name*/, const StoragePtr & /*table*/, [[maybe_unused]] const String & relative_table_path = {}) + virtual void attachTable(ContextPtr /* context */, const String & /*name*/, const StoragePtr & /*table*/, [[maybe_unused]] const String & relative_table_path = {}) { throw Exception("There is no ATTACH TABLE query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); } /// Forget about the table without deleting it, and return it. The database may not support this method. - virtual StoragePtr detachTable(const String & /*name*/) + virtual StoragePtr detachTable(ContextPtr /* context */, const String & /*name*/) { throw Exception("There is no DETACH TABLE query for Database" + getEngineName(), ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Databases/IDatabaseReplicating.h b/src/Databases/IDatabaseReplicating.h new file mode 100644 index 00000000000..7d7cd63fe43 --- /dev/null +++ b/src/Databases/IDatabaseReplicating.h @@ -0,0 +1,13 @@ +#pragma once + +namespace DB +{ + +class IDatabaseReplicating +{ +public: + virtual void stopReplication() = 0; + virtual ~IDatabaseReplicating() = default; +}; + +} diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp index a2e03b10d47..5da5c660fb3 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.cpp @@ -5,8 +5,6 @@ # include # include -# include -# include # include # include # include @@ -23,32 +21,9 @@ namespace DB namespace ErrorCodes { extern const int NOT_IMPLEMENTED; - extern const int LOGICAL_ERROR; } -template <> -DatabaseMaterializedMySQL::DatabaseMaterializedMySQL( - ContextPtr context_, - const String & database_name_, - const String & metadata_path_, - UUID /*uuid*/, - const String & mysql_database_name_, - mysqlxx::Pool && pool_, - MySQLClient && client_, - std::unique_ptr settings_) - : DatabaseOrdinary( - database_name_, - metadata_path_, - "data/" + escapeForFileName(database_name_) + "/", - "DatabaseMaterializedMySQL (" + database_name_ + ")", - context_) - , settings(std::move(settings_)) - , materialize_thread(context_, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), settings.get()) -{ -} - -template <> -DatabaseMaterializedMySQL::DatabaseMaterializedMySQL( +DatabaseMaterializedMySQL::DatabaseMaterializedMySQL( ContextPtr context_, const String & database_name_, const String & metadata_path_, @@ -57,16 +32,15 @@ DatabaseMaterializedMySQL::DatabaseMaterializedMySQL( mysqlxx::Pool && pool_, MySQLClient && client_, std::unique_ptr settings_) - : DatabaseAtomic(database_name_, metadata_path_, uuid, "DatabaseMaterializedMySQL (" + database_name_ + ")", context_) + : DatabaseAtomic(database_name_, metadata_path_, uuid, "DatabaseMaterializedMySQL(" + database_name_ + ")", context_) , settings(std::move(settings_)) , materialize_thread(context_, database_name_, mysql_database_name_, std::move(pool_), std::move(client_), settings.get()) { } -template -void DatabaseMaterializedMySQL::rethrowExceptionIfNeed() const +void DatabaseMaterializedMySQL::rethrowExceptionIfNeeded() const { - std::unique_lock lock(Base::mutex); + std::unique_lock lock(mutex); if (!settings->allows_query_when_mysql_lost && exception) { @@ -84,17 +58,15 @@ void DatabaseMaterializedMySQL::rethrowExceptionIfNeed() const } } -template -void DatabaseMaterializedMySQL::setException(const std::exception_ptr & exception_) +void DatabaseMaterializedMySQL::setException(const std::exception_ptr & exception_) { - std::unique_lock lock(Base::mutex); + std::unique_lock lock(mutex); exception = exception_; } -template -void DatabaseMaterializedMySQL::startupTables(ThreadPool & thread_pool, bool force_restore, bool force_attach) +void DatabaseMaterializedMySQL::startupTables(ThreadPool & thread_pool, bool force_restore, bool force_attach) { - Base::startupTables(thread_pool, force_restore, force_attach); + DatabaseAtomic::startupTables(thread_pool, force_restore, force_attach); if (!force_attach) materialize_thread.assertMySQLAvailable(); @@ -103,149 +75,92 @@ void DatabaseMaterializedMySQL::startupTables(ThreadPool & thread_pool, bo started_up = true; } -template -void DatabaseMaterializedMySQL::createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) +void DatabaseMaterializedMySQL::createTable(ContextPtr context_, const String & name, const StoragePtr & table, const ASTPtr & query) { - assertCalledFromSyncThreadOrDrop("create table"); - Base::createTable(context_, name, table, query); + checkIsInternalQuery(context_, "CREATE TABLE"); + DatabaseAtomic::createTable(context_, name, table, query); } -template -void DatabaseMaterializedMySQL::dropTable(ContextPtr context_, const String & name, bool no_delay) +void DatabaseMaterializedMySQL::dropTable(ContextPtr context_, const String & name, bool no_delay) { - assertCalledFromSyncThreadOrDrop("drop table"); - Base::dropTable(context_, name, no_delay); + checkIsInternalQuery(context_, "DROP TABLE"); + DatabaseAtomic::dropTable(context_, name, no_delay); } -template -void DatabaseMaterializedMySQL::attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) +void DatabaseMaterializedMySQL::attachTable(ContextPtr context_, const String & name, const StoragePtr & table, const String & relative_table_path) { - assertCalledFromSyncThreadOrDrop("attach table"); - Base::attachTable(name, table, relative_table_path); + checkIsInternalQuery(context_, "ATTACH TABLE"); + DatabaseAtomic::attachTable(context_, name, table, relative_table_path); } -template -StoragePtr DatabaseMaterializedMySQL::detachTable(const String & name) +StoragePtr DatabaseMaterializedMySQL::detachTable(ContextPtr context_, const String & name) { - assertCalledFromSyncThreadOrDrop("detach table"); - return Base::detachTable(name); + checkIsInternalQuery(context_, "DETACH TABLE"); + return DatabaseAtomic::detachTable(context_, name); } -template -void DatabaseMaterializedMySQL::renameTable(ContextPtr context_, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) +void DatabaseMaterializedMySQL::renameTable(ContextPtr context_, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) { - assertCalledFromSyncThreadOrDrop("rename table"); + checkIsInternalQuery(context_, "RENAME TABLE"); if (exchange) - throw Exception("MaterializedMySQL database not support exchange table.", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("MaterializedMySQL database does not support EXCHANGE TABLE.", ErrorCodes::NOT_IMPLEMENTED); if (dictionary) - throw Exception("MaterializedMySQL database not support rename dictionary.", ErrorCodes::NOT_IMPLEMENTED); + throw Exception("MaterializedMySQL database does not support RENAME DICTIONARY.", ErrorCodes::NOT_IMPLEMENTED); - if (to_database.getDatabaseName() != Base::getDatabaseName()) + if (to_database.getDatabaseName() != DatabaseAtomic::getDatabaseName()) throw Exception("Cannot rename with other database for MaterializedMySQL database.", ErrorCodes::NOT_IMPLEMENTED); - Base::renameTable(context_, name, *this, to_name, exchange, dictionary); + DatabaseAtomic::renameTable(context_, name, *this, to_name, exchange, dictionary); } -template -void DatabaseMaterializedMySQL::alterTable(ContextPtr context_, const StorageID & table_id, const StorageInMemoryMetadata & metadata) +void DatabaseMaterializedMySQL::alterTable(ContextPtr context_, const StorageID & table_id, const StorageInMemoryMetadata & metadata) { - assertCalledFromSyncThreadOrDrop("alter table"); - Base::alterTable(context_, table_id, metadata); + checkIsInternalQuery(context_, "ALTER TABLE"); + DatabaseAtomic::alterTable(context_, table_id, metadata); } -template -void DatabaseMaterializedMySQL::drop(ContextPtr context_) +void DatabaseMaterializedMySQL::drop(ContextPtr context_) { /// Remove metadata info - fs::path metadata(Base::getMetadataPath() + "/.metadata"); + fs::path metadata(getMetadataPath() + "/.metadata"); if (fs::exists(metadata)) fs::remove(metadata); - Base::drop(context_); + DatabaseAtomic::drop(context_); } -template -StoragePtr DatabaseMaterializedMySQL::tryGetTable(const String & name, ContextPtr context_) const +StoragePtr DatabaseMaterializedMySQL::tryGetTable(const String & name, ContextPtr context_) const { - if (!MaterializedMySQLSyncThread::isMySQLSyncThread()) - { - StoragePtr nested_storage = Base::tryGetTable(name, context_); - - if (!nested_storage) - return {}; - - return std::make_shared(std::move(nested_storage), this); - } - - return Base::tryGetTable(name, context_); + StoragePtr nested_storage = DatabaseAtomic::tryGetTable(name, context_); + if (context_->isInternalQuery()) + return nested_storage; + return std::make_shared(std::move(nested_storage), this); } -template DatabaseTablesIteratorPtr -DatabaseMaterializedMySQL::getTablesIterator(ContextPtr context_, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) const +DatabaseMaterializedMySQL::getTablesIterator(ContextPtr context_, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) const { - if (!MaterializedMySQLSyncThread::isMySQLSyncThread()) - { - DatabaseTablesIteratorPtr iterator = Base::getTablesIterator(context_, filter_by_table_name); - return std::make_unique(std::move(iterator), this); - } - - return Base::getTablesIterator(context_, filter_by_table_name); + DatabaseTablesIteratorPtr iterator = DatabaseAtomic::getTablesIterator(context_, filter_by_table_name); + if (context_->isInternalQuery()) + return iterator; + return std::make_unique(std::move(iterator), this); } -template -void DatabaseMaterializedMySQL::assertCalledFromSyncThreadOrDrop(const char * method) const +void DatabaseMaterializedMySQL::checkIsInternalQuery(ContextPtr context_, const char * method) const { - if (!MaterializedMySQLSyncThread::isMySQLSyncThread() && started_up) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MaterializedMySQL database not support {}", method); + if (started_up && context_ && !context_->isInternalQuery()) + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "MaterializedMySQL database does not support {}", method); } -template -void DatabaseMaterializedMySQL::shutdownSynchronizationThread() +void DatabaseMaterializedMySQL::stopReplication() { materialize_thread.stopSynchronization(); started_up = false; } -template class Helper, typename... Args> -auto castToMaterializedMySQLAndCallHelper(Database * database, Args && ... args) -{ - using Ordinary = DatabaseMaterializedMySQL; - using Atomic = DatabaseMaterializedMySQL; - using ToOrdinary = typename std::conditional_t, const Ordinary *, Ordinary *>; - using ToAtomic = typename std::conditional_t, const Atomic *, Atomic *>; - if (auto * database_materialize = typeid_cast(database)) - return (database_materialize->*Helper::v)(std::forward(args)...); - if (auto * database_materialize = typeid_cast(database)) - return (database_materialize->*Helper::v)(std::forward(args)...); - - throw Exception("LOGICAL_ERROR: cannot cast to DatabaseMaterializedMySQL, it is a bug.", ErrorCodes::LOGICAL_ERROR); -} - -template struct HelperSetException { static constexpr auto v = &T::setException; }; -void setSynchronizationThreadException(const DatabasePtr & materialized_mysql_db, const std::exception_ptr & exception) -{ - castToMaterializedMySQLAndCallHelper(materialized_mysql_db.get(), exception); -} - -template struct HelperStopSync { static constexpr auto v = &T::shutdownSynchronizationThread; }; -void stopDatabaseSynchronization(const DatabasePtr & materialized_mysql_db) -{ - castToMaterializedMySQLAndCallHelper(materialized_mysql_db.get()); -} - -template struct HelperRethrow { static constexpr auto v = &T::rethrowExceptionIfNeed; }; -void rethrowSyncExceptionIfNeed(const IDatabase * materialized_mysql_db) -{ - castToMaterializedMySQLAndCallHelper(materialized_mysql_db); -} - -template class DatabaseMaterializedMySQL; -template class DatabaseMaterializedMySQL; - } #endif diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.h b/src/Databases/MySQL/DatabaseMaterializedMySQL.h index ac32607a22c..fa2f78ecd73 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.h @@ -6,7 +6,10 @@ #include #include +#include #include +#include +#include #include #include @@ -17,17 +20,20 @@ namespace DB * * All table structure and data will be written to the local file system */ -template -class DatabaseMaterializedMySQL : public Base +class DatabaseMaterializedMySQL : public DatabaseAtomic, IDatabaseReplicating { public: - DatabaseMaterializedMySQL( - ContextPtr context, const String & database_name_, const String & metadata_path_, UUID uuid, - const String & mysql_database_name_, mysqlxx::Pool && pool_, - MySQLClient && client_, std::unique_ptr settings_); + ContextPtr context, + const String & database_name_, + const String & metadata_path_, + UUID uuid, + const String & mysql_database_name_, + mysqlxx::Pool && pool_, + MySQLClient && client_, + std::unique_ptr settings_); - void rethrowExceptionIfNeed() const; + void rethrowExceptionIfNeeded() const; void setException(const std::exception_ptr & exception); protected: @@ -49,9 +55,9 @@ public: void dropTable(ContextPtr context_, const String & name, bool no_delay) override; - void attachTable(const String & name, const StoragePtr & table, const String & relative_table_path) override; + void attachTable(ContextPtr context_, const String & name, const StoragePtr & table, const String & relative_table_path) override; - StoragePtr detachTable(const String & name) override; + StoragePtr detachTable(ContextPtr context_, const String & name) override; void renameTable(ContextPtr context_, const String & name, IDatabase & to_database, const String & to_name, bool exchange, bool dictionary) override; @@ -63,18 +69,13 @@ public: DatabaseTablesIteratorPtr getTablesIterator(ContextPtr context_, const DatabaseOnDisk::FilterByNameFunction & filter_by_table_name) const override; - void assertCalledFromSyncThreadOrDrop(const char * method) const; + void checkIsInternalQuery(ContextPtr context_, const char * method) const; - void shutdownSynchronizationThread(); + void stopReplication() override; friend class DatabaseMaterializedTablesIterator; }; - -void setSynchronizationThreadException(const DatabasePtr & materialized_mysql_db, const std::exception_ptr & exception); -void stopDatabaseSynchronization(const DatabasePtr & materialized_mysql_db); -void rethrowSyncExceptionIfNeed(const IDatabase * materialized_mysql_db); - } #endif diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 275a1015c7b..772dbc63fc4 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -362,7 +362,7 @@ void DatabaseMySQL::cleanOutdatedTables() } } -void DatabaseMySQL::attachTable(const String & table_name, const StoragePtr & storage, const String &) +void DatabaseMySQL::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & storage, const String &) { std::lock_guard lock{mutex}; @@ -385,7 +385,7 @@ void DatabaseMySQL::attachTable(const String & table_name, const StoragePtr & st fs::remove(remove_flag); } -StoragePtr DatabaseMySQL::detachTable(const String & table_name) +StoragePtr DatabaseMySQL::detachTable(ContextPtr /* context */, const String & table_name) { std::lock_guard lock{mutex}; @@ -482,7 +482,7 @@ DatabaseMySQL::~DatabaseMySQL() } } -void DatabaseMySQL::createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) +void DatabaseMySQL::createTable(ContextPtr local_context, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) { const auto & create = create_query->as(); @@ -500,7 +500,7 @@ void DatabaseMySQL::createTable(ContextPtr, const String & table_name, const Sto throw Exception("The MySQL database engine can only execute attach statements of type attach table database_name.table_name", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - attachTable(table_name, storage, {}); + attachTable(local_context, table_name, storage, {}); } } diff --git a/src/Databases/MySQL/DatabaseMySQL.h b/src/Databases/MySQL/DatabaseMySQL.h index 363557fbacb..e57ac442db1 100644 --- a/src/Databases/MySQL/DatabaseMySQL.h +++ b/src/Databases/MySQL/DatabaseMySQL.h @@ -77,13 +77,13 @@ public: void loadStoredObjects(ContextMutablePtr, bool, bool force_attach, bool skip_startup_tables) override; - StoragePtr detachTable(const String & table_name) override; + StoragePtr detachTable(ContextPtr context, const String & table_name) override; void detachTablePermanently(ContextPtr context, const String & table_name) override; void dropTable(ContextPtr context, const String & table_name, bool no_delay) override; - void attachTable(const String & table_name, const StoragePtr & storage, const String & relative_table_path) override; + void attachTable(ContextPtr context, const String & table_name, const StoragePtr & storage, const String & relative_table_path) override; protected: ASTPtr getCreateTableQueryImpl(const String & name, ContextPtr context, bool throw_on_error) const override; diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index 058708afb80..0fa45c2e282 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -53,6 +53,8 @@ static ContextMutablePtr createQueryContext(ContextPtr context) auto query_context = Context::createCopy(context); query_context->setSettings(new_query_settings); + query_context->setInternalQuery(true); + query_context->getClientInfo().query_kind = ClientInfo::QueryKind::SECONDARY_QUERY; query_context->setCurrentQueryId(""); // generate random query_id return query_context; @@ -764,15 +766,9 @@ void MaterializedMySQLSyncThread::executeDDLAtomic(const QueryEvent & query_even } } -bool MaterializedMySQLSyncThread::isMySQLSyncThread() -{ - return getThreadName() == std::string_view(MYSQL_BACKGROUND_THREAD_NAME); -} - void MaterializedMySQLSyncThread::setSynchronizationThreadException(const std::exception_ptr & exception) { - auto db = DatabaseCatalog::instance().getDatabase(database_name); - DB::setSynchronizationThreadException(db, exception); + assert_cast(DatabaseCatalog::instance().getDatabase(database_name).get())->setException(exception); } void MaterializedMySQLSyncThread::Buffers::add(size_t block_rows, size_t block_bytes, size_t written_rows, size_t written_bytes) diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.h b/src/Databases/MySQL/MaterializedMySQLSyncThread.h index 524426bada6..ba5022137bf 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.h +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.h @@ -53,8 +53,6 @@ public: void assertMySQLAvailable(); - static bool isMySQLSyncThread(); - private: Poco::Logger * log; diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp index 14488f08811..5ab0eb47ea5 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.cpp @@ -266,11 +266,11 @@ void DatabaseMaterializedPostgreSQL::createTable(ContextPtr local_context, const DatabaseAtomic::createTable(StorageMaterializedPostgreSQL::makeNestedTableContext(local_context), table_name, table, query_copy); /// Attach MaterializedPostgreSQL table. - attachTable(table_name, table, {}); + attachTable(local_context, table_name, table, {}); } -void DatabaseMaterializedPostgreSQL::attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path) +void DatabaseMaterializedPostgreSQL::attachTable(ContextPtr context_, const String & table_name, const StoragePtr & table, const String & relative_table_path) { /// If there is query context then we need to attach materialized storage. /// If there is no query context then we need to attach internal storage from atomic database. @@ -310,12 +310,12 @@ void DatabaseMaterializedPostgreSQL::attachTable(const String & table_name, cons } else { - DatabaseAtomic::attachTable(table_name, table, relative_table_path); + DatabaseAtomic::attachTable(context_, table_name, table, relative_table_path); } } -StoragePtr DatabaseMaterializedPostgreSQL::detachTable(const String & table_name) +StoragePtr DatabaseMaterializedPostgreSQL::detachTable(ContextPtr context_, const String & table_name) { /// If there is query context then we need to detach materialized storage. /// If there is no query context then we need to detach internal storage from atomic database. @@ -369,7 +369,7 @@ StoragePtr DatabaseMaterializedPostgreSQL::detachTable(const String & table_name } else { - return DatabaseAtomic::detachTable(table_name); + return DatabaseAtomic::detachTable(context_, table_name); } } diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h index 25e32fb0312..9469af20405 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h @@ -11,6 +11,7 @@ #include #include #include +#include #include #include @@ -22,7 +23,7 @@ class PostgreSQLConnection; using PostgreSQLConnectionPtr = std::shared_ptr; -class DatabaseMaterializedPostgreSQL : public DatabaseAtomic +class DatabaseMaterializedPostgreSQL : public DatabaseAtomic, IDatabaseReplicating { public: @@ -49,15 +50,15 @@ public: void createTable(ContextPtr context, const String & table_name, const StoragePtr & table, const ASTPtr & query) override; - void attachTable(const String & table_name, const StoragePtr & table, const String & relative_table_path) override; + void attachTable(ContextPtr context, const String & table_name, const StoragePtr & table, const String & relative_table_path) override; - StoragePtr detachTable(const String & table_name) override; + StoragePtr detachTable(ContextPtr context, const String & table_name) override; void dropTable(ContextPtr local_context, const String & name, bool no_delay) override; void drop(ContextPtr local_context) override; - void stopReplication(); + void stopReplication() override; void applySettingsChanges(const SettingsChanges & settings_changes, ContextPtr query_context) override; diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp index e995a6d2377..d333e476069 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.cpp @@ -206,7 +206,7 @@ StoragePtr DatabasePostgreSQL::fetchTable(const String & table_name, ContextPtr, } -void DatabasePostgreSQL::attachTable(const String & table_name, const StoragePtr & storage, const String &) +void DatabasePostgreSQL::attachTable(ContextPtr /* context_ */, const String & table_name, const StoragePtr & storage, const String &) { std::lock_guard lock{mutex}; @@ -231,7 +231,7 @@ void DatabasePostgreSQL::attachTable(const String & table_name, const StoragePtr } -StoragePtr DatabasePostgreSQL::detachTable(const String & table_name) +StoragePtr DatabasePostgreSQL::detachTable(ContextPtr /* context_ */, const String & table_name) { std::lock_guard lock{mutex}; @@ -251,14 +251,14 @@ StoragePtr DatabasePostgreSQL::detachTable(const String & table_name) } -void DatabasePostgreSQL::createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) +void DatabasePostgreSQL::createTable(ContextPtr local_context, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) { const auto & create = create_query->as(); if (!create->attach) throw Exception("PostgreSQL database engine does not support create table", ErrorCodes::NOT_IMPLEMENTED); - attachTable(table_name, storage, {}); + attachTable(local_context, table_name, storage, {}); } diff --git a/src/Databases/PostgreSQL/DatabasePostgreSQL.h b/src/Databases/PostgreSQL/DatabasePostgreSQL.h index f801f9585d6..d41dbff1f54 100644 --- a/src/Databases/PostgreSQL/DatabasePostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabasePostgreSQL.h @@ -55,8 +55,8 @@ public: void createTable(ContextPtr, const String & table_name, const StoragePtr & storage, const ASTPtr & create_query) override; void dropTable(ContextPtr, const String & table_name, bool no_delay) override; - void attachTable(const String & table_name, const StoragePtr & storage, const String & relative_table_path) override; - StoragePtr detachTable(const String & table_name) override; + void attachTable(ContextPtr context, const String & table_name, const StoragePtr & storage, const String & relative_table_path) override; + StoragePtr detachTable(ContextPtr context, const String & table_name) override; void drop(ContextPtr /*context*/) override; void shutdown() override; diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 898e169bc44..117119a3ee8 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -248,10 +248,9 @@ DatabaseAndTable DatabaseCatalog::getTableImpl( #if USE_MYSQL /// It's definitely not the best place for this logic, but behaviour must be consistent with DatabaseMaterializedMySQL::tryGetTable(...) - if (db_and_table.first->getEngineName() == "MaterializedMySQL") + if (!context_->isInternalQuery() && db_and_table.first->getEngineName() == "MaterializedMySQL") { - if (!MaterializedMySQLSyncThread::isMySQLSyncThread()) - db_and_table.second = std::make_shared(std::move(db_and_table.second), db_and_table.first.get()); + db_and_table.second = std::make_shared(std::move(db_and_table.second), db_and_table.first.get()); } #endif return db_and_table; diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index ce54ecc529a..4139653f010 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -80,22 +80,22 @@ BlockIO InterpreterDropQuery::executeToTable(ASTDropQuery & query) { DatabasePtr database; UUID table_to_wait_on = UUIDHelpers::Nil; - auto res = executeToTableImpl(query, database, table_to_wait_on); + auto res = executeToTableImpl(getContext(), query, database, table_to_wait_on); if (query.no_delay) waitForTableToBeActuallyDroppedOrDetached(query, database, table_to_wait_on); return res; } -BlockIO InterpreterDropQuery::executeToTableImpl(ASTDropQuery & query, DatabasePtr & db, UUID & uuid_to_wait) +BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQuery & query, DatabasePtr & db, UUID & uuid_to_wait) { /// NOTE: it does not contain UUID, we will resolve it with locked DDLGuard auto table_id = StorageID(query); if (query.temporary || table_id.database_name.empty()) { - if (getContext()->tryResolveStorageID(table_id, Context::ResolveExternal)) + if (context_->tryResolveStorageID(table_id, Context::ResolveExternal)) return executeToTemporaryTable(table_id.getTableName(), query.kind); else - query.setDatabase(table_id.database_name = getContext()->getCurrentDatabase()); + query.setDatabase(table_id.database_name = context_->getCurrentDatabase()); } if (query.temporary) @@ -109,8 +109,8 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ASTDropQuery & query, DatabaseP auto ddl_guard = (!query.no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(table_id.database_name, table_id.table_name) : nullptr); /// If table was already dropped by anyone, an exception will be thrown - auto [database, table] = query.if_exists ? DatabaseCatalog::instance().tryGetDatabaseAndTable(table_id, getContext()) - : DatabaseCatalog::instance().getDatabaseAndTable(table_id, getContext()); + auto [database, table] = query.if_exists ? DatabaseCatalog::instance().tryGetDatabaseAndTable(table_id, context_) + : DatabaseCatalog::instance().getDatabaseAndTable(table_id, context_); if (database && table) { @@ -132,7 +132,7 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ASTDropQuery & query, DatabaseP /// Prevents recursive drop from drop database query. The original query must specify a table. bool is_drop_or_detach_database = !query_ptr->as()->table; bool is_replicated_ddl_query = typeid_cast(database.get()) && - !getContext()->getClientInfo().is_replicated_database_internal && + !context_->getClientInfo().is_replicated_database_internal && !is_drop_or_detach_database; AccessFlags drop_storage; @@ -147,20 +147,20 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ASTDropQuery & query, DatabaseP if (is_replicated_ddl_query) { if (query.kind == ASTDropQuery::Kind::Detach) - getContext()->checkAccess(drop_storage, table_id); + context_->checkAccess(drop_storage, table_id); else if (query.kind == ASTDropQuery::Kind::Truncate) - getContext()->checkAccess(AccessType::TRUNCATE, table_id); + context_->checkAccess(AccessType::TRUNCATE, table_id); else if (query.kind == ASTDropQuery::Kind::Drop) - getContext()->checkAccess(drop_storage, table_id); + context_->checkAccess(drop_storage, table_id); ddl_guard->releaseTableLock(); table.reset(); - return typeid_cast(database.get())->tryEnqueueReplicatedDDL(query.clone(), getContext()); + return typeid_cast(database.get())->tryEnqueueReplicatedDDL(query.clone(), context_); } if (query.kind == ASTDropQuery::Kind::Detach) { - getContext()->checkAccess(drop_storage, table_id); + context_->checkAccess(drop_storage, table_id); if (table->isDictionary()) { @@ -175,7 +175,7 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ASTDropQuery & query, DatabaseP TableExclusiveLockHolder table_lock; if (database->getUUID() == UUIDHelpers::Nil) - table_lock = table->lockExclusively(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); + table_lock = table->lockExclusively(context_->getCurrentQueryId(), context_->getSettingsRef().lock_acquire_timeout); if (query.permanently) { @@ -183,12 +183,12 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ASTDropQuery & query, DatabaseP DatabaseCatalog::instance().tryRemoveLoadingDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies, is_drop_or_detach_database); /// Drop table from memory, don't touch data, metadata file renamed and will be skipped during server restart - database->detachTablePermanently(getContext(), table_id.table_name); + database->detachTablePermanently(context_, table_id.table_name); } else { /// Drop table from memory, don't touch data and metadata - database->detachTable(table_id.table_name); + database->detachTable(context_, table_id.table_name); } } else if (query.kind == ASTDropQuery::Kind::Truncate) @@ -196,20 +196,20 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ASTDropQuery & query, DatabaseP if (table->isDictionary()) throw Exception("Cannot TRUNCATE dictionary", ErrorCodes::SYNTAX_ERROR); - getContext()->checkAccess(AccessType::TRUNCATE, table_id); + context_->checkAccess(AccessType::TRUNCATE, table_id); if (table->isStaticStorage()) throw Exception(ErrorCodes::TABLE_IS_READ_ONLY, "Table is read-only"); table->checkTableCanBeDropped(); - auto table_lock = table->lockExclusively(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); + auto table_lock = table->lockExclusively(context_->getCurrentQueryId(), context_->getSettingsRef().lock_acquire_timeout); auto metadata_snapshot = table->getInMemoryMetadataPtr(); /// Drop table data, don't touch metadata - table->truncate(query_ptr, metadata_snapshot, getContext(), table_lock); + table->truncate(query_ptr, metadata_snapshot, context_, table_lock); } else if (query.kind == ASTDropQuery::Kind::Drop) { - getContext()->checkAccess(drop_storage, table_id); + context_->checkAccess(drop_storage, table_id); if (table->isDictionary()) { @@ -224,11 +224,13 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ASTDropQuery & query, DatabaseP TableExclusiveLockHolder table_lock; if (database->getUUID() == UUIDHelpers::Nil) - table_lock = table->lockExclusively(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); + table_lock = table->lockExclusively(context_->getCurrentQueryId(), context_->getSettingsRef().lock_acquire_timeout); DatabaseCatalog::instance().tryRemoveLoadingDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies, is_drop_or_detach_database); - database->dropTable(getContext(), table_id.table_name, query.no_delay); + auto inner_context = Context::createCopy(context_); + inner_context->setInternalQuery(true); + database->dropTable(inner_context, table_id.table_name, query.no_delay); } db = database; @@ -320,16 +322,8 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, if (query.kind == ASTDropQuery::Kind::Detach && query.permanently) throw Exception("DETACH PERMANENTLY is not implemented for databases", ErrorCodes::NOT_IMPLEMENTED); -#if USE_MYSQL - if (database->getEngineName() == "MaterializedMySQL") - stopDatabaseSynchronization(database); -#endif - if (auto * replicated = typeid_cast(database.get())) - replicated->stopReplication(); -#if USE_LIBPQXX - if (auto * materialize_postgresql = typeid_cast(database.get())) - materialize_postgresql->stopReplication(); -#endif + if (auto * replicating = dynamic_cast(database.get())) + replicating->stopReplication(); if (database->shouldBeEmptyOnDetach()) { @@ -341,19 +335,21 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, /// Flush should not be done if shouldBeEmptyOnDetach() == false, /// since in this case getTablesIterator() may do some additional work, - /// see DatabaseMaterializedMySQL<>::getTablesIterator() + /// see DatabaseMaterializedMySQL::getTablesIterator() for (auto iterator = database->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) { iterator->table()->flush(); } - for (auto iterator = database->getTablesIterator(getContext()); iterator->isValid(); iterator->next()) + auto table_context = Context::createCopy(getContext()); + table_context->setInternalQuery(true); + for (auto iterator = database->getTablesIterator(table_context); iterator->isValid(); iterator->next()) { DatabasePtr db; UUID table_to_wait = UUIDHelpers::Nil; query_for_table.setTable(iterator->name()); query_for_table.is_dictionary = iterator->table()->isDictionary(); - executeToTableImpl(query_for_table, db, table_to_wait); + executeToTableImpl(table_context, query_for_table, db, table_to_wait); uuids_to_wait.push_back(table_to_wait); } } diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index accadf690ca..1a38abcdff9 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -36,7 +36,7 @@ private: BlockIO executeToDatabaseImpl(const ASTDropQuery & query, DatabasePtr & database, std::vector & uuids_to_wait); BlockIO executeToTable(ASTDropQuery & query); - BlockIO executeToTableImpl(ASTDropQuery & query, DatabasePtr & db, UUID & uuid_to_wait); + BlockIO executeToTableImpl(ContextPtr context_, ASTDropQuery & query, DatabasePtr & db, UUID & uuid_to_wait); static void waitForTableToBeActuallyDroppedOrDetached(const ASTDropQuery & query, const DatabasePtr & db, const UUID & uuid_to_wait); diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 1610ae56280..9376b19b17e 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -523,7 +523,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, auto table_lock = table->lockExclusively(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); create_ast = database->getCreateTableQuery(replica.table_name, getContext()); - database->detachTable(replica.table_name); + database->detachTable(getContext(), replica.table_name); } table.reset(); @@ -544,7 +544,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, constraints, false); - database->attachTable(replica.table_name, table, data_path); + database->attachTable(system_context, replica.table_name, table, data_path); table->startup(); return table; diff --git a/src/Storages/StorageMaterializedMySQL.cpp b/src/Storages/StorageMaterializedMySQL.cpp index c214540151a..79f4497de9c 100644 --- a/src/Storages/StorageMaterializedMySQL.cpp +++ b/src/Storages/StorageMaterializedMySQL.cpp @@ -48,8 +48,8 @@ Pipe StorageMaterializedMySQL::read( size_t max_block_size, unsigned int num_streams) { - /// If the background synchronization thread has exception. - rethrowSyncExceptionIfNeed(database); + if (auto * db = typeid_cast(database)) + db->rethrowExceptionIfNeeded(); return readFinalFromNestedStorage(nested_storage, column_names, metadata_snapshot, query_info, context, processed_stage, max_block_size, num_streams); @@ -57,8 +57,9 @@ Pipe StorageMaterializedMySQL::read( NamesAndTypesList StorageMaterializedMySQL::getVirtuals() const { - /// If the background synchronization thread has exception. - rethrowSyncExceptionIfNeed(database); + if (auto * db = typeid_cast(database)) + db->rethrowExceptionIfNeeded(); + return nested_storage->getVirtuals(); } diff --git a/src/Storages/System/attachSystemTables.cpp b/src/Storages/System/attachSystemTables.cpp index 30fe38f4936..023ced35a6b 100644 --- a/src/Storages/System/attachSystemTables.cpp +++ b/src/Storages/System/attachSystemTables.cpp @@ -4,6 +4,7 @@ #include #include +#include #include #include #include @@ -80,92 +81,92 @@ namespace DB { -void attachSystemTablesLocal(IDatabase & system_database) +void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database) { - attach(system_database, "one"); - attach(system_database, "numbers", false); - attach(system_database, "numbers_mt", true); - attach(system_database, "zeros", false); - attach(system_database, "zeros_mt", true); - attach(system_database, "databases"); - attach(system_database, "tables"); - attach(system_database, "columns"); - attach(system_database, "functions"); - attach(system_database, "events"); - attach(system_database, "settings"); - attach>(system_database, "merge_tree_settings"); - attach>(system_database, "replicated_merge_tree_settings"); - attach(system_database, "build_options"); - attach(system_database, "formats"); - attach(system_database, "table_functions"); - attach(system_database, "aggregate_function_combinators"); - attach(system_database, "data_type_families"); - attach(system_database, "collations"); - attach(system_database, "table_engines"); - attach(system_database, "contributors"); - attach(system_database, "users"); - attach(system_database, "roles"); - attach(system_database, "grants"); - attach(system_database, "role_grants"); - attach(system_database, "current_roles"); - attach(system_database, "enabled_roles"); - attach(system_database, "settings_profiles"); - attach(system_database, "settings_profile_elements"); - attach(system_database, "row_policies"); - attach(system_database, "quotas"); - attach(system_database, "quota_limits"); - attach(system_database, "quota_usage"); - attach(system_database, "quotas_usage"); - attach(system_database, "user_directories"); - attach(system_database, "privileges"); - attach(system_database, "errors"); - attach(system_database, "warnings"); - attach(system_database, "data_skipping_indices"); - attach(system_database, "licenses"); - attach(system_database, "time_zones"); + attach(context, system_database, "one"); + attach(context, system_database, "numbers", false); + attach(context, system_database, "numbers_mt", true); + attach(context, system_database, "zeros", false); + attach(context, system_database, "zeros_mt", true); + attach(context, system_database, "databases"); + attach(context, system_database, "tables"); + attach(context, system_database, "columns"); + attach(context, system_database, "functions"); + attach(context, system_database, "events"); + attach(context, system_database, "settings"); + attach>(context, system_database, "merge_tree_settings"); + attach>(context, system_database, "replicated_merge_tree_settings"); + attach(context, system_database, "build_options"); + attach(context, system_database, "formats"); + attach(context, system_database, "table_functions"); + attach(context, system_database, "aggregate_function_combinators"); + attach(context, system_database, "data_type_families"); + attach(context, system_database, "collations"); + attach(context, system_database, "table_engines"); + attach(context, system_database, "contributors"); + attach(context, system_database, "users"); + attach(context, system_database, "roles"); + attach(context, system_database, "grants"); + attach(context, system_database, "role_grants"); + attach(context, system_database, "current_roles"); + attach(context, system_database, "enabled_roles"); + attach(context, system_database, "settings_profiles"); + attach(context, system_database, "settings_profile_elements"); + attach(context, system_database, "row_policies"); + attach(context, system_database, "quotas"); + attach(context, system_database, "quota_limits"); + attach(context, system_database, "quota_usage"); + attach(context, system_database, "quotas_usage"); + attach(context, system_database, "user_directories"); + attach(context, system_database, "privileges"); + attach(context, system_database, "errors"); + attach(context, system_database, "warnings"); + attach(context, system_database, "data_skipping_indices"); + attach(context, system_database, "licenses"); + attach(context, system_database, "time_zones"); #ifdef OS_LINUX - attach(system_database, "stack_trace"); + attach(context, system_database, "stack_trace"); #endif #if USE_ROCKSDB - attach(system_database, "rocksdb"); + attach(context, system_database, "rocksdb"); #endif } -void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper) +void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, bool has_zookeeper) { - attachSystemTablesLocal(system_database); + attachSystemTablesLocal(context, system_database); - attach(system_database, "parts"); - attach(system_database, "projection_parts"); - attach(system_database, "detached_parts"); - attach(system_database, "parts_columns"); - attach(system_database, "projection_parts_columns"); - attach(system_database, "disks"); - attach(system_database, "storage_policies"); - attach(system_database, "processes"); - attach(system_database, "metrics"); - attach(system_database, "merges"); - attach(system_database, "mutations"); - attach(system_database, "replicas"); - attach(system_database, "replication_queue"); - attach(system_database, "distributed_ddl_queue"); - attach(system_database, "distribution_queue"); - attach(system_database, "dictionaries"); - attach(system_database, "models"); - attach(system_database, "clusters"); - attach(system_database, "graphite_retentions"); - attach(system_database, "macros"); - attach(system_database, "replicated_fetches"); - attach(system_database, "part_moves_between_shards"); - attach(system_database, "asynchronous_inserts"); + attach(context, system_database, "parts"); + attach(context, system_database, "projection_parts"); + attach(context, system_database, "detached_parts"); + attach(context, system_database, "parts_columns"); + attach(context, system_database, "projection_parts_columns"); + attach(context, system_database, "disks"); + attach(context, system_database, "storage_policies"); + attach(context, system_database, "processes"); + attach(context, system_database, "metrics"); + attach(context, system_database, "merges"); + attach(context, system_database, "mutations"); + attach(context, system_database, "replicas"); + attach(context, system_database, "replication_queue"); + attach(context, system_database, "distributed_ddl_queue"); + attach(context, system_database, "distribution_queue"); + attach(context, system_database, "dictionaries"); + attach(context, system_database, "models"); + attach(context, system_database, "clusters"); + attach(context, system_database, "graphite_retentions"); + attach(context, system_database, "macros"); + attach(context, system_database, "replicated_fetches"); + attach(context, system_database, "part_moves_between_shards"); + attach(context, system_database, "asynchronous_inserts"); if (has_zookeeper) - attach(system_database, "zookeeper"); + attach(context, system_database, "zookeeper"); } -void attachSystemTablesAsync(IDatabase & system_database, AsynchronousMetrics & async_metrics) +void attachSystemTablesAsync(ContextPtr context, IDatabase & system_database, AsynchronousMetrics & async_metrics) { - attach(system_database, "asynchronous_metrics", async_metrics); + attach(context, system_database, "asynchronous_metrics", async_metrics); } } diff --git a/src/Storages/System/attachSystemTables.h b/src/Storages/System/attachSystemTables.h index 71570506a1a..4c1a79f84dd 100644 --- a/src/Storages/System/attachSystemTables.h +++ b/src/Storages/System/attachSystemTables.h @@ -1,17 +1,16 @@ #pragma once #include - +#include namespace DB { -class Context; class AsynchronousMetrics; class IDatabase; -void attachSystemTablesServer(IDatabase & system_database, bool has_zookeeper); -void attachSystemTablesLocal(IDatabase & system_database); -void attachSystemTablesAsync(IDatabase & system_database, AsynchronousMetrics & async_metrics); +void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, bool has_zookeeper); +void attachSystemTablesLocal(ContextPtr context, IDatabase & system_database); +void attachSystemTablesAsync(ContextPtr context, IDatabase & system_database, AsynchronousMetrics & async_metrics); } diff --git a/src/Storages/System/attachSystemTablesImpl.h b/src/Storages/System/attachSystemTablesImpl.h index 1fdf677699a..4f83a0a4fda 100644 --- a/src/Storages/System/attachSystemTablesImpl.h +++ b/src/Storages/System/attachSystemTablesImpl.h @@ -7,14 +7,14 @@ namespace DB { template -void attach(IDatabase & system_database, const String & table_name, StorageArgs && ... args) +void attach(ContextPtr context, IDatabase & system_database, const String & table_name, StorageArgs && ... args) { assert(system_database.getDatabaseName() == DatabaseCatalog::SYSTEM_DATABASE); if (system_database.getUUID() == UUIDHelpers::Nil) { /// Attach to Ordinary database auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name); - system_database.attachTable(table_name, StorageT::create(table_id, std::forward(args)...)); + system_database.attachTable(context, table_name, StorageT::create(table_id, std::forward(args)...)); } else { @@ -23,7 +23,7 @@ void attach(IDatabase & system_database, const String & table_name, StorageArgs /// and path is actually not used auto table_id = StorageID(DatabaseCatalog::SYSTEM_DATABASE, table_name, UUIDHelpers::generateV4()); String path = "store/" + DatabaseCatalog::getPathForUUID(table_id.uuid); - system_database.attachTable(table_name, StorageT::create(table_id, std::forward(args)...), path); + system_database.attachTable(context, table_name, StorageT::create(table_id, std::forward(args)...), path); } } diff --git a/src/Storages/tests/gtest_transform_query_for_external_database.cpp b/src/Storages/tests/gtest_transform_query_for_external_database.cpp index ae9ec8745fd..f161400630b 100644 --- a/src/Storages/tests/gtest_transform_query_for_external_database.cpp +++ b/src/Storages/tests/gtest_transform_query_for_external_database.cpp @@ -84,6 +84,7 @@ private: const auto & table_name = tab.table.table; const auto & db_name = tab.table.database; database->attachTable( + context, table_name, StorageMemory::create( StorageID(db_name, table_name), ColumnsDescription{getColumns()}, ConstraintsDescription{}, String{})); diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6ab8b29895b..0d31a98a130 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -767,7 +767,7 @@ class ClickHouseCluster: hostname=None, env_variables=None, image="clickhouse/integration-test", tag=None, stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, external_dirs=None, tmpfs=None, zookeeper_docker_compose_path=None, minio_certs_dir=None, use_keeper=True, - main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, config_root_name="clickhouse"): + main_config_name="config.xml", users_config_name="users.xml", copy_common_configs=True, config_root_name="clickhouse") -> 'ClickHouseInstance': """Add an instance to the cluster. diff --git a/tests/integration/test_materialized_mysql_database/configs/users.xml b/tests/integration/test_materialized_mysql_database/configs/users.xml index b5de2b300d3..4b7f5a1b109 100644 --- a/tests/integration/test_materialized_mysql_database/configs/users.xml +++ b/tests/integration/test_materialized_mysql_database/configs/users.xml @@ -3,9 +3,9 @@ 1 + Atomic 1 0 - Ordinary diff --git a/tests/integration/test_materialized_mysql_database/configs/users_db_atomic.xml b/tests/integration/test_materialized_mysql_database/configs/users_db_atomic.xml deleted file mode 100644 index 3cf7285de59..00000000000 --- a/tests/integration/test_materialized_mysql_database/configs/users_db_atomic.xml +++ /dev/null @@ -1,19 +0,0 @@ - - - - - 1 - Atomic - - - - - - - - ::/0 - - default - - - diff --git a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py index ab9bfa4eed9..5524f842c36 100644 --- a/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py +++ b/tests/integration/test_materialized_mysql_database/materialize_with_ddl.py @@ -21,7 +21,7 @@ def check_query(clickhouse_node, query, result_set, retry_count=10, interval_sec if result_set == lastest_result: return - logging.debug(f"latest_result{lastest_result}") + logging.debug(f"latest_result {lastest_result}") time.sleep(interval_seconds) except Exception as e: logging.debug(f"check_query retry {i+1} exception {e}") diff --git a/tests/integration/test_materialized_mysql_database/test.py b/tests/integration/test_materialized_mysql_database/test.py index 05aa525b5c3..29194f4ab25 100644 --- a/tests/integration/test_materialized_mysql_database/test.py +++ b/tests/integration/test_materialized_mysql_database/test.py @@ -5,7 +5,7 @@ import pwd import re import pymysql.cursors import pytest -from helpers.cluster import ClickHouseCluster, get_docker_compose_path, run_and_check +from helpers.cluster import ClickHouseCluster, ClickHouseInstance, get_docker_compose_path, run_and_check import docker import logging @@ -17,10 +17,10 @@ cluster = ClickHouseCluster(__file__) mysql_node = None mysql8_node = None -node_db_ordinary = cluster.add_instance('node1', user_configs=["configs/users.xml"], with_mysql=True, stay_alive=True) -node_db_atomic = cluster.add_instance('node2', user_configs=["configs/users_db_atomic.xml"], with_mysql8=True, stay_alive=True) -node_disable_bytes_settings = cluster.add_instance('node3', user_configs=["configs/users_disable_bytes_settings.xml"], with_mysql=False, stay_alive=True) -node_disable_rows_settings = cluster.add_instance('node4', user_configs=["configs/users_disable_rows_settings.xml"], with_mysql=False, stay_alive=True) +node_db = cluster.add_instance('node1', user_configs=["configs/users.xml"], with_mysql=True, with_mysql8=True, stay_alive=True) +node_disable_bytes_settings = cluster.add_instance('node2', user_configs=["configs/users_disable_bytes_settings.xml"], with_mysql=False, stay_alive=True) +node_disable_rows_settings = cluster.add_instance('node3', user_configs=["configs/users_disable_rows_settings.xml"], with_mysql=False, stay_alive=True) + @pytest.fixture(scope="module") def started_cluster(): @@ -82,32 +82,39 @@ class MySQLConnection: if self.mysql_connection is not None: self.mysql_connection.close() + @pytest.fixture(scope="module") def started_mysql_5_7(): mysql_node = MySQLConnection(cluster.mysql_port, 'root', 'clickhouse', cluster.mysql_ip) yield mysql_node + @pytest.fixture(scope="module") def started_mysql_8_0(): mysql8_node = MySQLConnection(cluster.mysql8_port, 'root', 'clickhouse', cluster.mysql8_ip) yield mysql8_node -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) -def test_materialize_database_dml_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node): + +@pytest.fixture(scope='module') +def clickhouse_node(): + yield node_db + + +def test_materialized_database_dml_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node: ClickHouseInstance): materialize_with_ddl.dml_with_materialized_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.materialized_mysql_database_with_views(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.materialized_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.move_to_prewhere_and_column_filtering(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) -def test_materialize_database_dml_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): + +def test_materialized_database_dml_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.dml_with_materialized_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") materialize_with_ddl.materialized_mysql_database_with_views(clickhouse_node, started_mysql_8_0, "mysql80") materialize_with_ddl.materialized_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_8_0, "mysql80") materialize_with_ddl.move_to_prewhere_and_column_filtering(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) -def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node): + +def test_materialized_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.drop_table_with_materialized_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.create_table_with_materialized_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.rename_table_with_materialized_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") @@ -119,8 +126,8 @@ def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_ materialize_with_ddl.alter_modify_column_with_materialized_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.create_table_like_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) -def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): + +def test_materialized_database_ddl_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.drop_table_with_materialized_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") materialize_with_ddl.create_table_with_materialized_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") materialize_with_ddl.rename_table_with_materialized_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") @@ -131,102 +138,96 @@ def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_ materialize_with_ddl.alter_modify_column_with_materialized_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") materialize_with_ddl.create_table_like_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) -def test_materialize_database_ddl_with_empty_transaction_5_7(started_cluster, started_mysql_5_7, clickhouse_node): + +def test_materialized_database_ddl_with_empty_transaction_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) -def test_materialize_database_ddl_with_empty_transaction_8_0(started_cluster, started_mysql_8_0, clickhouse_node): + +def test_materialized_database_ddl_with_empty_transaction_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_select_without_columns_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_select_without_columns_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.select_without_columns(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_insert_with_modify_binlog_checksum_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) def test_insert_with_modify_binlog_checksum_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.insert_with_modify_binlog_checksum(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) -def test_materialize_database_err_sync_user_privs_5_7(started_cluster, started_mysql_5_7, clickhouse_node): +def test_materialized_database_err_sync_user_privs_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.err_sync_user_privs_with_materialized_mysql_database(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) -def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_mysql_8_0, clickhouse_node): +def test_materialized_database_err_sync_user_privs_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.err_sync_user_privs_with_materialized_mysql_database(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) + def test_network_partition_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) + def test_network_partition_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) + def test_mysql_kill_sync_thread_restore_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) + def test_mysql_kill_sync_thread_restore_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) + def test_mysql_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) + def test_mysql_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) + def test_clickhouse_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql57") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) + def test_clickhouse_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) + def test_utf8mb4(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) + def test_system_parts_table(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.system_parts_test(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) + def test_multi_table_update(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) + def test_system_tables_table(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): materialize_with_ddl.system_tables_test(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.system_tables_test(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) -def test_materialize_with_column_comments(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): + +def test_materialized_with_column_comments(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): materialize_with_ddl.materialize_with_column_comments_test(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.materialize_with_column_comments_test(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) -def test_materialize_with_enum(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): + +def test_materialized_with_enum(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): materialize_with_ddl.materialize_with_enum8_test(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.materialize_with_enum16_test(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.alter_enum8_to_enum16_test(clickhouse_node, started_mysql_5_7, "mysql57") @@ -240,7 +241,7 @@ def test_mysql_settings(started_cluster, started_mysql_8_0, started_mysql_5_7, c materialize_with_ddl.mysql_settings_test(clickhouse_node, started_mysql_5_7, "mysql57") materialize_with_ddl.mysql_settings_test(clickhouse_node, started_mysql_8_0, "mysql80") -@pytest.mark.parametrize(('clickhouse_node'), [pytest.param(node_db_ordinary, id="ordinary"), pytest.param(node_db_atomic, id="atomic")]) + def test_large_transaction(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): materialize_with_ddl.materialized_mysql_large_transaction(clickhouse_node, started_mysql_8_0, "mysql80") materialize_with_ddl.materialized_mysql_large_transaction(clickhouse_node, started_mysql_5_7, "mysql57") From 18ce50fd7f7f05ced983f4458ad898c80b614f43 Mon Sep 17 00:00:00 2001 From: Stig Bakken Date: Thu, 11 Nov 2021 21:35:26 +0100 Subject: [PATCH 131/200] Add hasReplicationThread() and stopReplication() methods to IDatabase --- src/Databases/DatabaseReplicated.h | 5 +++-- src/Databases/IDatabase.h | 7 +++++++ src/Databases/IDatabaseReplicating.h | 13 ------------- src/Databases/MySQL/DatabaseMaterializedMySQL.h | 5 +++-- .../PostgreSQL/DatabaseMaterializedPostgreSQL.h | 5 +++-- src/Interpreters/InterpreterDropQuery.cpp | 4 ++-- 6 files changed, 18 insertions(+), 21 deletions(-) delete mode 100644 src/Databases/IDatabaseReplicating.h diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 5c0a97a3296..cc25b2128fb 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include @@ -18,7 +17,7 @@ using ZooKeeperPtr = std::shared_ptr; class Cluster; using ClusterPtr = std::shared_ptr; -class DatabaseReplicated : public DatabaseAtomic, IDatabaseReplicating +class DatabaseReplicated : public DatabaseAtomic { public: DatabaseReplicated(const String & name_, const String & metadata_path_, UUID uuid, @@ -47,6 +46,8 @@ public: /// then it will be executed on all replicas. BlockIO tryEnqueueReplicatedDDL(const ASTPtr & query, ContextPtr query_context); + bool hasReplicationThread() const override { return true; } + void stopReplication() override; String getFullReplicaName() const; diff --git a/src/Databases/IDatabase.h b/src/Databases/IDatabase.h index da8915ee751..b1aa4eb1aae 100644 --- a/src/Databases/IDatabase.h +++ b/src/Databases/IDatabase.h @@ -323,6 +323,13 @@ public: getEngineName()); } + virtual bool hasReplicationThread() const { return false; } + + virtual void stopReplication() + { + throw Exception(ErrorCodes::LOGICAL_ERROR, "Database engine {} does not run a replication thread!", getEngineName()); + } + virtual ~IDatabase() = default; protected: diff --git a/src/Databases/IDatabaseReplicating.h b/src/Databases/IDatabaseReplicating.h deleted file mode 100644 index 7d7cd63fe43..00000000000 --- a/src/Databases/IDatabaseReplicating.h +++ /dev/null @@ -1,13 +0,0 @@ -#pragma once - -namespace DB -{ - -class IDatabaseReplicating -{ -public: - virtual void stopReplication() = 0; - virtual ~IDatabaseReplicating() = default; -}; - -} diff --git a/src/Databases/MySQL/DatabaseMaterializedMySQL.h b/src/Databases/MySQL/DatabaseMaterializedMySQL.h index fa2f78ecd73..32686784f2a 100644 --- a/src/Databases/MySQL/DatabaseMaterializedMySQL.h +++ b/src/Databases/MySQL/DatabaseMaterializedMySQL.h @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include @@ -20,7 +19,7 @@ namespace DB * * All table structure and data will be written to the local file system */ -class DatabaseMaterializedMySQL : public DatabaseAtomic, IDatabaseReplicating +class DatabaseMaterializedMySQL : public DatabaseAtomic { public: DatabaseMaterializedMySQL( @@ -71,6 +70,8 @@ public: void checkIsInternalQuery(ContextPtr context_, const char * method) const; + bool hasReplicationThread() const override { return true; } + void stopReplication() override; friend class DatabaseMaterializedTablesIterator; diff --git a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h index 9469af20405..3b7f0f9d29d 100644 --- a/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h +++ b/src/Databases/PostgreSQL/DatabaseMaterializedPostgreSQL.h @@ -11,7 +11,6 @@ #include #include #include -#include #include #include @@ -23,7 +22,7 @@ class PostgreSQLConnection; using PostgreSQLConnectionPtr = std::shared_ptr; -class DatabaseMaterializedPostgreSQL : public DatabaseAtomic, IDatabaseReplicating +class DatabaseMaterializedPostgreSQL : public DatabaseAtomic { public: @@ -58,6 +57,8 @@ public: void drop(ContextPtr local_context) override; + bool hasReplicationThread() const override { return true; } + void stopReplication() override; void applySettingsChanges(const SettingsChanges & settings_changes, ContextPtr query_context) override; diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 4139653f010..25d9e4c4dd1 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -322,8 +322,8 @@ BlockIO InterpreterDropQuery::executeToDatabaseImpl(const ASTDropQuery & query, if (query.kind == ASTDropQuery::Kind::Detach && query.permanently) throw Exception("DETACH PERMANENTLY is not implemented for databases", ErrorCodes::NOT_IMPLEMENTED); - if (auto * replicating = dynamic_cast(database.get())) - replicating->stopReplication(); + if (database->hasReplicationThread()) + database->stopReplication(); if (database->shouldBeEmptyOnDetach()) { From eb5115adb824c449abdb500b470c07ca01374fdf Mon Sep 17 00:00:00 2001 From: Stig Bakken Date: Thu, 11 Nov 2021 21:35:55 +0100 Subject: [PATCH 132/200] Fail when loading MaterializeMySQL database without uuid and print instructions on how to remove and recreate. --- src/Databases/DatabaseFactory.cpp | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index bf24b0b5343..7f41010f8cf 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -1,5 +1,6 @@ #include +#include #include #include #include @@ -12,9 +13,9 @@ #include #include #include -#include +#include #include -#include +#include #include "config_core.h" @@ -55,6 +56,7 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; extern const int UNKNOWN_DATABASE_ENGINE; extern const int CANNOT_CREATE_DATABASE; + extern const int NOT_IMPLEMENTED; } DatabasePtr DatabaseFactory::get(const ASTCreateQuery & create, const String & metadata_path, ContextPtr context) @@ -211,6 +213,15 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String if (engine_define->settings) materialize_mode_settings->loadFromQuery(*engine_define); + if (uuid == UUIDHelpers::Nil) + throw Exception( + fmt::format( + "The MaterializedMySQL database engine no longer supports Ordinary databases. To re-create the database, delete " + "the old one by executing \"rm -rf {}{{,.sql}}\", then re-create the database with the following query: {}", + metadata_path, + queryToString(create)), + ErrorCodes::NOT_IMPLEMENTED); + return std::make_shared( context, database_name, metadata_path, uuid, configuration.database, std::move(mysql_pool), std::move(client), std::move(materialize_mode_settings)); From 9190a883d3f2a4152dc7e1f52520494bf814d88a Mon Sep 17 00:00:00 2001 From: Stig Bakken Date: Tue, 16 Nov 2021 14:59:11 +0100 Subject: [PATCH 133/200] Print CREATE, not ATTACH --- src/Databases/DatabaseFactory.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Databases/DatabaseFactory.cpp b/src/Databases/DatabaseFactory.cpp index 7f41010f8cf..264807534b3 100644 --- a/src/Databases/DatabaseFactory.cpp +++ b/src/Databases/DatabaseFactory.cpp @@ -214,13 +214,17 @@ DatabasePtr DatabaseFactory::getImpl(const ASTCreateQuery & create, const String materialize_mode_settings->loadFromQuery(*engine_define); if (uuid == UUIDHelpers::Nil) + { + auto print_create_ast = create.clone(); + print_create_ast->as()->attach = false; throw Exception( fmt::format( "The MaterializedMySQL database engine no longer supports Ordinary databases. To re-create the database, delete " "the old one by executing \"rm -rf {}{{,.sql}}\", then re-create the database with the following query: {}", metadata_path, - queryToString(create)), + queryToString(print_create_ast)), ErrorCodes::NOT_IMPLEMENTED); + } return std::make_shared( context, database_name, metadata_path, uuid, configuration.database, std::move(mysql_pool), From b91f21d6584fbd135d72563cefc4fb0f5d3cdd6c Mon Sep 17 00:00:00 2001 From: Stig Bakken Date: Thu, 18 Nov 2021 11:45:12 +0100 Subject: [PATCH 134/200] Don't use setInternalQuery() when forwarding drop calls to database --- src/Databases/DatabaseOnDisk.cpp | 4 ++-- src/Interpreters/InterpreterDropQuery.cpp | 4 +--- src/Interpreters/InterpreterSystemQuery.cpp | 2 +- 3 files changed, 4 insertions(+), 6 deletions(-) diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index 3f27350657c..a69317eb22d 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -185,7 +185,7 @@ void DatabaseOnDisk::createTable( { /// Metadata already exists, table was detached removeDetachedPermanentlyFlag(local_context, table_name, table_metadata_path, true); - attachTable(getContext(), table_name, table, getTableDataPath(create)); + attachTable(local_context, table_name, table, getTableDataPath(create)); return; } @@ -371,7 +371,7 @@ void DatabaseOnDisk::renameTable( String table_metadata_path; ASTPtr attach_query; /// DatabaseLazy::detachTable may return nullptr even if table exists, so we need tryGetTable for this case. - StoragePtr table = tryGetTable(table_name, getContext()); + StoragePtr table = tryGetTable(table_name, local_context); detachTable(local_context, table_name); UUID prev_uuid = UUIDHelpers::Nil; try diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index 25d9e4c4dd1..90c4311b032 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -228,9 +228,7 @@ BlockIO InterpreterDropQuery::executeToTableImpl(ContextPtr context_, ASTDropQue DatabaseCatalog::instance().tryRemoveLoadingDependencies(table_id, getContext()->getSettingsRef().check_table_dependencies, is_drop_or_detach_database); - auto inner_context = Context::createCopy(context_); - inner_context->setInternalQuery(true); - database->dropTable(inner_context, table_id.table_name, query.no_delay); + database->dropTable(context_, table_id.table_name, query.no_delay); } db = database; diff --git a/src/Interpreters/InterpreterSystemQuery.cpp b/src/Interpreters/InterpreterSystemQuery.cpp index 9376b19b17e..960fddccb8c 100644 --- a/src/Interpreters/InterpreterSystemQuery.cpp +++ b/src/Interpreters/InterpreterSystemQuery.cpp @@ -523,7 +523,7 @@ StoragePtr InterpreterSystemQuery::tryRestartReplica(const StorageID & replica, auto table_lock = table->lockExclusively(getContext()->getCurrentQueryId(), getContext()->getSettingsRef().lock_acquire_timeout); create_ast = database->getCreateTableQuery(replica.table_name, getContext()); - database->detachTable(getContext(), replica.table_name); + database->detachTable(system_context, replica.table_name); } table.reset(); From 69559a4fd9f62488532d5ba319f0ee5be38986b4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 18 Nov 2021 17:24:06 +0300 Subject: [PATCH 135/200] fix convertion to CNF --- src/Interpreters/TreeCNFConverter.cpp | 85 ++++++++++++++++--- src/Interpreters/TreeCNFConverter.h | 13 ++- src/Interpreters/TreeOptimizer.cpp | 18 ++-- ..._constraints_simple_optimization.reference | 8 +- .../01623_constraints_column_swap.reference | 2 +- .../0_stateless/01626_cnf_fuzz_long.python | 73 ++++++++++++++++ .../0_stateless/01626_cnf_fuzz_long.reference | 1 + .../0_stateless/01626_cnf_fuzz_long.sh | 10 +++ .../0_stateless/01626_cnf_test.reference | 8 +- tests/queries/0_stateless/01626_cnf_test.sql | 6 +- 10 files changed, 194 insertions(+), 30 deletions(-) create mode 100644 tests/queries/0_stateless/01626_cnf_fuzz_long.python create mode 100644 tests/queries/0_stateless/01626_cnf_fuzz_long.reference create mode 100755 tests/queries/0_stateless/01626_cnf_fuzz_long.sh diff --git a/src/Interpreters/TreeCNFConverter.cpp b/src/Interpreters/TreeCNFConverter.cpp index 46002f5be99..a6b46c46589 100644 --- a/src/Interpreters/TreeCNFConverter.cpp +++ b/src/Interpreters/TreeCNFConverter.cpp @@ -1,7 +1,9 @@ #include #include #include -#include +#include +#include + namespace DB { @@ -10,14 +12,37 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int INCORRECT_QUERY; + extern const int TOO_MANY_TEMPORARY_COLUMNS; } namespace { +bool isLogicalFunction(const ASTFunction & func) +{ + return func.name == "and" || func.name == "or" || func.name == "not"; +} + +size_t countAtoms(const ASTPtr & node) +{ + checkStackSize(); + if (node->as()) + return 1; + + const auto * func = node->as(); + if (func && !isLogicalFunction(*func)) + return 1; + + size_t num_atoms = 0; + for (const auto & child : node->children) + num_atoms += countAtoms(child); + return num_atoms; +} + /// Splits AND(a, b, c) to AND(a, AND(b, c)) for AND/OR void splitMultiLogic(ASTPtr & node) { + checkStackSize(); auto * func = node->as(); if (func && (func->name == "and" || func->name == "or")) @@ -29,9 +54,8 @@ void splitMultiLogic(ASTPtr & node) { ASTPtr res = func->arguments->children[0]->clone(); for (size_t i = 1; i < func->arguments->children.size(); ++i) - { res = makeASTFunction(func->name, res, func->arguments->children[i]->clone()); - } + node = res; } @@ -49,6 +73,7 @@ void splitMultiLogic(ASTPtr & node) /// Push NOT to leafs, remove NOT NOT ... void traversePushNot(ASTPtr & node, bool add_negation) { + checkStackSize(); auto * func = node->as(); if (func && (func->name == "and" || func->name == "or")) @@ -86,14 +111,19 @@ void traversePushNot(ASTPtr & node, bool add_negation) } /// Push Or inside And (actually pull AND to top) -void traversePushOr(ASTPtr & node) +bool traversePushOr(ASTPtr & node, size_t num_atoms, size_t max_atoms) { + if (max_atoms && num_atoms > max_atoms) + return false; + + checkStackSize(); auto * func = node->as(); if (func && (func->name == "or" || func->name == "and")) { for (auto & child : func->arguments->children) - traversePushOr(child); + if (!traversePushOr(child, num_atoms, max_atoms)) + return false; } if (func && func->name == "or") @@ -105,15 +135,15 @@ void traversePushOr(ASTPtr & node) auto & child = func->arguments->children[i]; auto * and_func = child->as(); if (and_func && and_func->name == "and") - { and_node_id = i; - } } - if (and_node_id == func->arguments->children.size()) - return; - const size_t other_node_id = 1 - and_node_id; + if (and_node_id == func->arguments->children.size()) + return true; + + const size_t other_node_id = 1 - and_node_id; const auto * and_func = func->arguments->children[and_node_id]->as(); + auto a = func->arguments->children[other_node_id]; auto b = and_func->arguments->children[0]; auto c = and_func->arguments->children[1]; @@ -124,13 +154,19 @@ void traversePushOr(ASTPtr & node) makeASTFunction("or", a->clone(), b), makeASTFunction("or", a, c)); - traversePushOr(node); + /// Count all atoms from 'a', because it was cloned. + num_atoms += countAtoms(a); + return traversePushOr(node, num_atoms, max_atoms); } + + return true; } /// transform ast into cnf groups void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & and_group, CNFQuery::OrGroup & or_group) { + checkStackSize(); + auto * func = node->as(); if (func && func->name == "and") { @@ -171,13 +207,22 @@ void traverseCNF(const ASTPtr & node, CNFQuery::AndGroup & result) } -CNFQuery TreeCNFConverter::toCNF(const ASTPtr & query) +std::optional TreeCNFConverter::tryConvertToCNF( + const ASTPtr & query, size_t max_growth_multipler) { auto cnf = query->clone(); + size_t num_atoms = countAtoms(cnf); splitMultiLogic(cnf); traversePushNot(cnf, false); - traversePushOr(cnf); + + size_t max_atoms = max_growth_multipler + ? std::max(MAX_ATOMS_WITHOUT_CHECK, num_atoms * max_growth_multipler) + : 0; + + if (!traversePushOr(cnf, num_atoms, max_atoms)) + return {}; + CNFQuery::AndGroup and_group; traverseCNF(cnf, and_group); @@ -186,6 +231,18 @@ CNFQuery TreeCNFConverter::toCNF(const ASTPtr & query) return result; } +CNFQuery TreeCNFConverter::toCNF( + const ASTPtr & query, size_t max_growth_multipler) +{ + auto cnf = tryConvertToCNF(query, max_growth_multipler); + if (!cnf) + throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS, + "Cannot expression '{}' to CNF, because it produces to many clauses." + "Size of formula inCNF can be exponential of size of source formula."); + + return *cnf; +} + ASTPtr TreeCNFConverter::fromCNF(const CNFQuery & cnf) { const auto & groups = cnf.getStatements(); @@ -208,7 +265,7 @@ ASTPtr TreeCNFConverter::fromCNF(const CNFQuery & cnf) auto * func = or_groups.back()->as(); for (const auto & atom : group) { - if ((*group.begin()).negative) + if (atom.negative) func->arguments->children.push_back(makeASTFunction("not", atom.ast->clone())); else func->arguments->children.push_back(atom.ast->clone()); diff --git a/src/Interpreters/TreeCNFConverter.h b/src/Interpreters/TreeCNFConverter.h index ba7fb299644..52f997d83c9 100644 --- a/src/Interpreters/TreeCNFConverter.h +++ b/src/Interpreters/TreeCNFConverter.h @@ -145,8 +145,19 @@ private: class TreeCNFConverter { public: + static constexpr size_t DEFAULT_MAX_GROWTH_MULTIPLIER = 20; + static constexpr size_t MAX_ATOMS_WITHOUT_CHECK = 200; - static CNFQuery toCNF(const ASTPtr & query); + /// @max_growth_multipler means that it's allowed to grow size of formula only + /// in that amount of times. It's needed to avoid exponential explosion of formula. + /// CNF of boolean formula with N clauses can have 2^N clauses. + /// If amout of atomic formulas will be exceded nullopt will be returned. + /// 0 - means unlimited. + static std::optional tryConvertToCNF( + const ASTPtr & query, size_t max_growth_multipler = DEFAULT_MAX_GROWTH_MULTIPLIER); + + static CNFQuery toCNF( + const ASTPtr & query, size_t max_growth_multipler = DEFAULT_MAX_GROWTH_MULTIPLIER); static ASTPtr fromCNF(const CNFQuery & cnf); }; diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 1b816984647..fd53e6a0b7f 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -564,13 +564,20 @@ void optimizeSubstituteColumn(ASTSelectQuery * select_query, } /// Transform WHERE to CNF for more convenient optimization. -void convertQueryToCNF(ASTSelectQuery * select_query) +bool convertQueryToCNF(ASTSelectQuery * select_query) { if (select_query->where()) { - auto cnf_form = TreeCNFConverter::toCNF(select_query->where()).pushNotInFuntions(); - select_query->refWhere() = TreeCNFConverter::fromCNF(cnf_form); + auto cnf_form = TreeCNFConverter::tryConvertToCNF(select_query->where()); + if (!cnf_form) + return false; + + cnf_form->pushNotInFuntions(); + select_query->refWhere() = TreeCNFConverter::fromCNF(*cnf_form); + return true; } + + return false; } /// Remove duplicated columns from USING(...). @@ -734,10 +741,11 @@ void TreeOptimizer::apply(ASTPtr & query, TreeRewriterResult & result, if (settings.optimize_arithmetic_operations_in_aggregate_functions) optimizeAggregationFunctions(query); + bool converted_to_cnf = false; if (settings.convert_query_to_cnf) - convertQueryToCNF(select_query); + converted_to_cnf = convertQueryToCNF(select_query); - if (settings.convert_query_to_cnf && settings.optimize_using_constraints) + if (converted_to_cnf && settings.optimize_using_constraints) { optimizeWithConstraints(select_query, result.aliases, result.source_columns_set, tables_with_columns, result.metadata_snapshot, settings.optimize_append_index); diff --git a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference index 800d77ea8c6..7e012e1a17b 100644 --- a/tests/queries/0_stateless/01622_constraints_simple_optimization.reference +++ b/tests/queries/0_stateless/01622_constraints_simple_optimization.reference @@ -32,14 +32,14 @@ 1 1 0 -SELECT count() +SELECT count() AS `count()` FROM constraint_test_constants WHERE (c > 100) OR (b > 100) -SELECT count() +SELECT count() AS `count()` FROM constraint_test_constants WHERE c > 100 -SELECT count() +SELECT count() AS `count()` FROM constraint_test_constants WHERE c > 100 -SELECT count() +SELECT count() AS `count()` FROM constraint_test_constants diff --git a/tests/queries/0_stateless/01623_constraints_column_swap.reference b/tests/queries/0_stateless/01623_constraints_column_swap.reference index c287ed073fc..7ae4516fe9e 100644 --- a/tests/queries/0_stateless/01623_constraints_column_swap.reference +++ b/tests/queries/0_stateless/01623_constraints_column_swap.reference @@ -49,5 +49,5 @@ WHERE a = \'c\' SELECT a AS `substring(reverse(b), 1, 1)` FROM column_swap_test_test WHERE a = \'c\' -SELECT toUInt32(s) AS a +SELECT a FROM t_bad_constraint diff --git a/tests/queries/0_stateless/01626_cnf_fuzz_long.python b/tests/queries/0_stateless/01626_cnf_fuzz_long.python new file mode 100644 index 00000000000..10c12d14182 --- /dev/null +++ b/tests/queries/0_stateless/01626_cnf_fuzz_long.python @@ -0,0 +1,73 @@ +#!/usr/bin/env python3 +import os +from random import randint, choices +import sys + +CURDIR = os.path.dirname(os.path.realpath(__file__)) +sys.path.insert(0, os.path.join(CURDIR, 'helpers')) + +from pure_http_client import ClickHouseClient + +client = ClickHouseClient() + +N = 10 +create_query = "CREATE TABLE t_cnf_fuzz(" + ", ".join([f"c{i} UInt8" for i in range(N)]) + ") ENGINE = Memory" + +client.query("DROP TABLE IF EXISTS t_cnf_fuzz") +client.query(create_query) + +# Insert all possible combinations of bool columns. +insert_query = "INSERT INTO t_cnf_fuzz VALUES " +for i in range(2**N): + values = [] + cur = i + for _ in range(N): + values.append(cur % 2) + cur //= 2 + + insert_query += "(" + ", ".join(map(lambda x: str(x), values)) + ")" + +client.query(insert_query) + +# Let's try to covert DNF to CNF, +# because it's a worst case in a sense. + +MAX_CLAUSES = 10 +MAX_ATOMS = 5 + +def generate_dnf(): + clauses = [] + num_clauses = randint(1, MAX_CLAUSES) + for _ in range(num_clauses): + num_atoms = randint(1, MAX_ATOMS) + atom_ids = choices(range(N), k=num_atoms) + negates = choices([0, 1], k=num_atoms) + atoms = [f"(NOT c{i})" if neg else f"c{i}" for (i, neg) in zip(atom_ids, negates)] + clauses.append("(" + " AND ".join(atoms) + ")") + + return " OR ".join(clauses) + +select_query = "SELECT count() FROM t_cnf_fuzz WHERE {} SETTINGS convert_query_to_cnf = {}" + +fail_report = """ +Failed query: '{}'. +Result without optimization: {}. +Result with optimization: {}. +""" + +T = 500 +for _ in range(T): + condition = generate_dnf() + + query = select_query.format(condition, 0) + res = client.query(query).strip() + + query_cnf = select_query.format(condition, 1) + res_cnf = client.query(query_cnf).strip() + + if res != res_cnf: + print(fail_report.format(query_cnf, res, res_cnf)) + exit(1) + +client.query("DROP TABLE t_cnf_fuzz") +print("OK") diff --git a/tests/queries/0_stateless/01626_cnf_fuzz_long.reference b/tests/queries/0_stateless/01626_cnf_fuzz_long.reference new file mode 100644 index 00000000000..d86bac9de59 --- /dev/null +++ b/tests/queries/0_stateless/01626_cnf_fuzz_long.reference @@ -0,0 +1 @@ +OK diff --git a/tests/queries/0_stateless/01626_cnf_fuzz_long.sh b/tests/queries/0_stateless/01626_cnf_fuzz_long.sh new file mode 100755 index 00000000000..bdf53cdb252 --- /dev/null +++ b/tests/queries/0_stateless/01626_cnf_fuzz_long.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash +# Tags: no-fasttest, long +# Tag no-fasttest: Require python libraries like scipy, pandas and numpy + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CURDIR"/../shell_config.sh + +# We should have correct env vars from shell_config.sh to run this test +python3 "$CURDIR"/01626_cnf_fuzz_long.python diff --git a/tests/queries/0_stateless/01626_cnf_test.reference b/tests/queries/0_stateless/01626_cnf_test.reference index b8de3d3a57c..081215c9fb2 100644 --- a/tests/queries/0_stateless/01626_cnf_test.reference +++ b/tests/queries/0_stateless/01626_cnf_test.reference @@ -6,13 +6,13 @@ FROM cnf_test WHERE (i <= 2) OR (i <= 1) SELECT i FROM cnf_test -WHERE ((i > 4) OR (i > 1) OR (i > 6)) AND ((i > 4) OR (i > 1) OR (i > 5)) AND ((i > 4) OR (i > 6) OR (i > 2)) AND ((i > 4) OR (i > 2) OR (i > 5)) AND ((i > 1) OR (i > 6) OR (i > 3)) AND ((i > 1) OR (i > 3) OR (i > 5)) AND ((i > 6) OR (i > 2) OR (i > 3)) AND ((i > 2) OR (i > 3) OR (i > 5)) +WHERE ((i > 2) OR (i > 5) OR (i > 3)) AND ((i > 2) OR (i > 5) OR (i > 4)) AND ((i > 2) OR (i > 6) OR (i > 3)) AND ((i > 2) OR (i > 6) OR (i > 4)) AND ((i > 1) OR (i > 5) OR (i > 3)) AND ((i > 1) OR (i > 5) OR (i > 4)) AND ((i > 1) OR (i > 6) OR (i > 3)) AND ((i > 1) OR (i > 6) OR (i > 4)) SELECT i FROM cnf_test -WHERE ((i <= 5) OR (i <= 2) OR (i <= 3)) AND ((i <= 5) OR (i <= 2) OR (i <= 4)) AND ((i <= 5) OR (i <= 3) OR (i <= 1)) AND ((i <= 5) OR (i <= 4) OR (i <= 1)) AND ((i <= 2) OR (i <= 3) OR (i <= 6)) AND ((i <= 2) OR (i <= 4) OR (i <= 6)) AND ((i <= 3) OR (i <= 1) OR (i <= 6)) AND ((i <= 4) OR (i <= 1) OR (i <= 6)) +WHERE ((i <= 3) OR (i <= 2) OR (i <= 5)) AND ((i <= 3) OR (i <= 2) OR (i <= 6)) AND ((i <= 3) OR (i <= 5) OR (i <= 1)) AND ((i <= 3) OR (i <= 6) OR (i <= 1)) AND ((i <= 2) OR (i <= 5) OR (i <= 4)) AND ((i <= 2) OR (i <= 6) OR (i <= 4)) AND ((i <= 5) OR (i <= 1) OR (i <= 4)) AND ((i <= 6) OR (i <= 1) OR (i <= 4)) SELECT i FROM cnf_test -WHERE ((i > 4) OR (i > 1) OR (i > 6)) AND ((i > 4) OR (i > 1) OR (i > 5)) AND ((i > 4) OR (i > 6) OR (i > 2)) AND ((i > 4) OR (i > 6) OR (i > 7)) AND ((i > 4) OR (i > 2) OR (i > 5)) AND ((i > 4) OR (i > 7) OR (i > 5)) AND ((i > 1) OR (i > 8) OR (i > 6)) AND ((i > 1) OR (i > 8) OR (i > 5)) AND ((i > 1) OR (i > 6) OR (i > 3)) AND ((i > 1) OR (i > 3) OR (i > 5)) AND ((i > 8) OR (i > 6) OR (i > 2)) AND ((i > 8) OR (i > 6) OR (i > 7)) AND ((i > 8) OR (i > 2) OR (i > 5)) AND ((i > 8) OR (i > 7) OR (i > 5)) AND ((i > 6) OR (i > 2) OR (i > 3)) AND ((i > 6) OR (i > 3) OR (i > 7)) AND ((i > 2) OR (i > 3) OR (i > 5)) AND ((i > 3) OR (i > 7) OR (i > 5)) +WHERE ((i > 2) OR (i > 5) OR (i > 3)) AND ((i > 2) OR (i > 5) OR (i > 4)) AND ((i > 2) OR (i > 5) OR (i > 8)) AND ((i > 2) OR (i > 6) OR (i > 3)) AND ((i > 2) OR (i > 6) OR (i > 4)) AND ((i > 2) OR (i > 6) OR (i > 8)) AND ((i > 1) OR (i > 5) OR (i > 3)) AND ((i > 1) OR (i > 5) OR (i > 4)) AND ((i > 1) OR (i > 5) OR (i > 8)) AND ((i > 1) OR (i > 6) OR (i > 3)) AND ((i > 1) OR (i > 6) OR (i > 4)) AND ((i > 1) OR (i > 6) OR (i > 8)) AND ((i > 5) OR (i > 3) OR (i > 7)) AND ((i > 5) OR (i > 4) OR (i > 7)) AND ((i > 5) OR (i > 8) OR (i > 7)) AND ((i > 6) OR (i > 3) OR (i > 7)) AND ((i > 6) OR (i > 4) OR (i > 7)) AND ((i > 6) OR (i > 8) OR (i > 7)) SELECT i FROM cnf_test -WHERE ((i > 4) OR (i > 8) OR (i > 3)) AND (i <= 5) AND ((i > 1) OR (i > 2) OR (i > 7)) AND (i <= 6) +WHERE ((i > 2) OR (i > 1) OR (i > 7)) AND (i <= 5) AND (i <= 6) AND ((i > 3) OR (i > 4) OR (i > 8)) diff --git a/tests/queries/0_stateless/01626_cnf_test.sql b/tests/queries/0_stateless/01626_cnf_test.sql index e014441cbb3..8db732bc227 100644 --- a/tests/queries/0_stateless/01626_cnf_test.sql +++ b/tests/queries/0_stateless/01626_cnf_test.sql @@ -1,6 +1,8 @@ SET convert_query_to_cnf = 1; -CREATE TABLE cnf_test (i Int64) ENGINE = MergeTree() ORDER BY i; +DROP TABLE IF EXISTS cnf_test; + +CREATE TABLE cnf_test (i Int64) ENGINE = MergeTree() ORDER BY i; EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE NOT ((i > 1) OR (i > 2)); EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE NOT ((i > 1) AND (i > 2)); @@ -12,3 +14,5 @@ EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE NOT (((i > 1) OR (i > 2)) AND ((i > EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE ((i > 1) AND (i > 2) AND (i > 7)) OR ((i > 3) AND (i > 4) AND (i > 8)) OR ((i > 5) AND (i > 6)); EXPLAIN SYNTAX SELECT i FROM cnf_test WHERE ((i > 1) OR (i > 2) OR (i > 7)) AND ((i > 3) OR (i > 4) OR (i > 8)) AND NOT ((i > 5) OR (i > 6)); + +DROP TABLE cnf_test; From c988a4e77090a75f08f617a21f3d629f1c31a54b Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 18 Nov 2021 23:09:17 +0800 Subject: [PATCH 136/200] fix again --- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp | 8 ++++++-- src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h | 2 ++ src/Storages/MergeTree/MergeTreeData.cpp | 6 +++++- .../0_stateless/01710_minmax_count_projection.reference | 6 ++++++ .../queries/0_stateless/01710_minmax_count_projection.sql | 6 ++++++ 5 files changed, 25 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 2f46543b03c..80dc3f149e9 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -415,8 +415,7 @@ void MergeTreeBaseSelectProcessor::injectVirtualColumns( chunk.setColumns(columns, num_rows); } -Block MergeTreeBaseSelectProcessor::transformHeader( - Block block, const PrewhereInfoPtr & prewhere_info, const DataTypePtr & partition_value_type, const Names & virtual_columns) +void MergeTreeBaseSelectProcessor::transformBlockViaPrewhereInfo(Block & block, const PrewhereInfoPtr & prewhere_info) { if (prewhere_info) { @@ -460,7 +459,12 @@ Block MergeTreeBaseSelectProcessor::transformHeader( ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER); } } +} +Block MergeTreeBaseSelectProcessor::transformHeader( + Block block, const PrewhereInfoPtr & prewhere_info, const DataTypePtr & partition_value_type, const Names & virtual_columns) +{ + transformBlockViaPrewhereInfo(block, prewhere_info); injectVirtualColumns(block, nullptr, partition_value_type, virtual_columns); return block; } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index d102e4f07a4..01e55486815 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -42,6 +42,8 @@ public: const MergeTreeReadTaskColumns & task_columns, const Block & sample_block); + static void transformBlockViaPrewhereInfo(Block & block, const PrewhereInfoPtr & prewhere_info); + protected: Chunk generate() final; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 247fa1f4dd2..44ea78d3cd5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -35,6 +35,7 @@ #include #include #include +#include #include #include #include @@ -4480,7 +4481,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( } size_t pos = 0; - for(size_t i : metadata_snapshot->minmax_count_projection->partition_value_indices) + for (size_t i : metadata_snapshot->minmax_count_projection->partition_value_indices) { if (i >= part->partition.value.size()) throw Exception("Partition value index is out of boundary. It's a bug", ErrorCodes::LOGICAL_ERROR); @@ -4788,6 +4789,9 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( query_info.minmax_count_projection_block = getMinMaxCountProjectionBlock( metadata_snapshot, minmax_conut_projection_candidate->required_columns, query_info, parts, normal_parts, query_context); + MergeTreeBaseSelectProcessor::transformBlockViaPrewhereInfo( + query_info.minmax_count_projection_block, minmax_conut_projection_candidate->prewhere_info); + if (normal_parts.empty()) { selected_candidate = &*minmax_conut_projection_candidate; diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.reference b/tests/queries/0_stateless/01710_minmax_count_projection.reference index 36f9f68c0bc..201156465a6 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.reference +++ b/tests/queries/0_stateless/01710_minmax_count_projection.reference @@ -9,3 +9,9 @@ 1 9999 3 2021-10-25 10:00:00 2021-10-27 10:00:00 3 +1 +1 +1 +1 +\N 2021-10-27 10:00:00 4 +2021-10-24 10:00:00 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index 2b53cdc0bee..597cd653ab3 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -49,4 +49,10 @@ drop table if exists d; create table d (dt DateTime, j int) engine MergeTree partition by (toDate(dt), ceiling(j), toDate(dt), CEILING(j)) order by tuple(); insert into d values ('2021-10-24 10:00:00', 10), ('2021-10-25 10:00:00', 10), ('2021-10-26 10:00:00', 10), ('2021-10-27 10:00:00', 10); select min(dt), max(dt), count() from d where toDate(dt) >= '2021-10-25'; +select count() from d group by toDate(dt); + +-- fuzz crash +SELECT pointInEllipses(min(j), NULL), max(dt), count('0.0000000007') FROM d WHERE toDate(dt) >= '2021-10-25'; +SELECT min(dt) FROM d PREWHERE ceil(j) <= 0; + drop table d; From f8c53e839afe0a3c8166f19d30b3be0684949b9c Mon Sep 17 00:00:00 2001 From: Stig Bakken Date: Thu, 18 Nov 2021 16:53:41 +0100 Subject: [PATCH 137/200] fix clang13 warning --- src/Storages/StorageMaterializedMySQL.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/StorageMaterializedMySQL.cpp b/src/Storages/StorageMaterializedMySQL.cpp index 79f4497de9c..922dff2faf7 100644 --- a/src/Storages/StorageMaterializedMySQL.cpp +++ b/src/Storages/StorageMaterializedMySQL.cpp @@ -48,7 +48,7 @@ Pipe StorageMaterializedMySQL::read( size_t max_block_size, unsigned int num_streams) { - if (auto * db = typeid_cast(database)) + if (const auto * db = typeid_cast(database)) db->rethrowExceptionIfNeeded(); return readFinalFromNestedStorage(nested_storage, column_names, metadata_snapshot, @@ -57,7 +57,7 @@ Pipe StorageMaterializedMySQL::read( NamesAndTypesList StorageMaterializedMySQL::getVirtuals() const { - if (auto * db = typeid_cast(database)) + if (const auto * db = typeid_cast(database)) db->rethrowExceptionIfNeeded(); return nested_storage->getVirtuals(); From b33ab897063ceea84ee2bf4305df0ac0ccc7e906 Mon Sep 17 00:00:00 2001 From: Cody Baker Date: Thu, 18 Nov 2021 10:44:37 -0700 Subject: [PATCH 138/200] Reapply style changes to hardware page (#31506) --- website/benchmark/benchmark.js | 12 ++++++------ website/benchmark/hardware/index.html | 10 +++++----- website/css/bootstrap.css | 2 +- website/css/main.css | 2 +- website/src/scss/_variables.scss | 10 +++++----- 5 files changed, 18 insertions(+), 18 deletions(-) diff --git a/website/benchmark/benchmark.js b/website/benchmark/benchmark.js index bd5ec3c083e..09607b27263 100644 --- a/website/benchmark/benchmark.js +++ b/website/benchmark/benchmark.js @@ -40,7 +40,7 @@ function update_hash() { function generate_selectors(elem) { - var html = '

Compare

'; + var html = '

Compare

'; var available_results = results; @@ -63,7 +63,7 @@ function generate_selectors(elem) { button_class = 'btn-outline-secondary'; }; - html += ''; + html += ''; } } - html += '

Run

'; + html += '

Run

'; for (var i = 0; i < runs.length; i++) { - html += ''; + html += ''; } html += '
'; diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index 76b526c4d47..a0b4001361a 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -13,11 +13,11 @@
-
+
ClickHouse -

{{ title }}

+

{{ title }}

@@ -27,21 +27,21 @@
-

Relative query processing time (lower is better)

+

Relative query processing time (lower is better)

-

Full results

+

Full results

-

Comments

+

Comments

Submit your own results: https://clickhouse.com/docs/en/operations/performance-test/

Results for Lenovo B580 Laptop are from Ragıp Ünal. 16GB RAM 1600 GHz, 240GB SSD, Intel(R) Core(TM) i5-3210M CPU @ 2.50GHz (2 Core / 4 HT)
diff --git a/website/css/bootstrap.css b/website/css/bootstrap.css index b65cbbfed01..eb26dce70d7 100644 --- a/website/css/bootstrap.css +++ b/website/css/bootstrap.css @@ -3,4 +3,4 @@ * Copyright 2011-2019 The Bootstrap Authors * Copyright 2011-2019 Twitter, Inc. * Licensed under MIT (https://github.com/twbs/bootstrap/blob/master/LICENSE) - */:root{--blue:#007bff;--indigo:#6610f2;--purple:#6f42c1;--pink:#e83e8c;--red:#dc3545;--orange:#fd7e14;--yellow:#ffc107;--green:#28a745;--teal:#20c997;--cyan:#17a2b8;--gray:#6c757d;--gray-dark:#343a40;--brand-primary:#fc0;--brand-secondary:#ff3939;--primary-accent-yellow:#fc0;--primary-accent-light-yellow:#fffaf0;--primary-accent-blue:#257af4;--primary-accent-light-blue:#e3f1fe;--secondary-accent-orange:#ff8c00;--secondary-accent-light-orange:#ffe4b5;--secondary-accent-red:#ff3939;--secondary-accent-light-red:#ffe4e1;--primary:#fc0;--secondary:#212529;--success:#28a745;--info:#17a2b8;--warning:#ffc107;--danger:#dc3545;--light:#f1f6f9;--dark:#495057;--primary-light:#fffaf0;--secondary-light:#fff;--tertiary:#257af4;--tertiary-light:#e3f1fe;--white:#fff;--black:#212529;--blue:#257af4;--light-blue:#e3f1fe;--yellow:#fc0;--light-yellow:#fffaf0;--orange:#ff8c00;--light-orange:#ffe4b5;--red:#ff3939;--light-red:#ffe4e1;--medium:#d6dbdf;--breakpoint-xxs:0;--breakpoint-xs:400px;--breakpoint-sm:616px;--breakpoint-md:768px;--breakpoint-lg:980px;--breakpoint-xl:1240px;--font-family-sans-serif:"Noto Sans",sans-serif;--font-family-monospace:SFMono-Regular,Menlo,Monaco,Consolas,"Liberation Mono","Courier New",monospace}*,:after,:before{box-sizing:border-box}html{font-family:sans-serif;line-height:1.15;-webkit-text-size-adjust:100%;-webkit-tap-highlight-color:rgba(33,37,41,0)}article,aside,figcaption,figure,footer,header,hgroup,main,nav,section{display:block}body{margin:0;font-family:Noto Sans,sans-serif;font-size:1rem;font-weight:400;line-height:1.5;color:#212529;text-align:left;background-color:#fff}[tabindex="-1"]:focus:not(:focus-visible){outline:0!important}hr{box-sizing:content-box;height:0;overflow:visible}h1,h2,h3,h4,h5,h6{margin-top:0;margin-bottom:16px}p{margin-top:0;margin-bottom:1rem}abbr[data-original-title],abbr[title]{text-decoration:underline;-webkit-text-decoration:underline dotted;text-decoration:underline dotted;cursor:help;border-bottom:0;-webkit-text-decoration-skip-ink:none;text-decoration-skip-ink:none}address{font-style:normal;line-height:inherit}address,dl,ol,ul{margin-bottom:1rem}dl,ol,ul{margin-top:0}ol ol,ol ul,ul ol,ul ul{margin-bottom:0}dt{font-weight:700}dd{margin-bottom:.5rem;margin-left:0}blockquote{margin:0 0 1rem}b,strong{font-weight:bolder}small{font-size:80%}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sub{bottom:-.25em}sup{top:-.5em}a{text-decoration:none;background-color:transparent}a,a:hover{color:#ff8c00}a:hover{text-decoration:underline}a:not([href]),a:not([href]):hover{color:inherit;text-decoration:none}code,kbd,pre,samp{font-family:SFMono-Regular,Menlo,Monaco,Consolas,Liberation Mono,Courier New,monospace;font-size:1em}pre{margin-top:0;margin-bottom:1rem;overflow:auto}figure{margin:0 0 1rem}img{border-style:none}img,svg{vertical-align:middle}svg{overflow:hidden}table{border-collapse:collapse}caption{padding-top:.75rem;padding-bottom:.75rem;color:#6c757d;text-align:left;caption-side:bottom}th{text-align:inherit}label{display:inline-block;margin-bottom:.5rem}button{border-radius:0}button:focus{outline:1px dotted;outline:5px auto -webkit-focus-ring-color}button,input,optgroup,select,textarea{margin:0;font-family:inherit;font-size:inherit;line-height:inherit}button,input{overflow:visible}button,select{text-transform:none}select{word-wrap:normal}[type=button],[type=reset],[type=submit],button{-webkit-appearance:button}[type=button]:not(:disabled),[type=reset]:not(:disabled),[type=submit]:not(:disabled),button:not(:disabled){cursor:pointer}[type=button]::-moz-focus-inner,[type=reset]::-moz-focus-inner,[type=submit]::-moz-focus-inner,button::-moz-focus-inner{padding:0;border-style:none}input[type=checkbox],input[type=radio]{box-sizing:border-box;padding:0}input[type=date],input[type=datetime-local],input[type=month],input[type=time]{-webkit-appearance:listbox}textarea{overflow:auto;resize:vertical}fieldset{min-width:0;padding:0;margin:0;border:0}legend{display:block;width:100%;max-width:100%;padding:0;margin-bottom:.5rem;font-size:1.5rem;line-height:inherit;color:inherit;white-space:normal}@media(max-width:1200px){legend{font-size:calc(1.275rem + .3vw)}}progress{vertical-align:baseline}[type=number]::-webkit-inner-spin-button,[type=number]::-webkit-outer-spin-button{height:auto}[type=search]{outline-offset:-2px;-webkit-appearance:none}[type=search]::-webkit-search-decoration{-webkit-appearance:none}::-webkit-file-upload-button{font:inherit;-webkit-appearance:button}output{display:inline-block}summary{display:list-item;cursor:pointer}template{display:none}[hidden]{display:none!important}.h1,.h2,.h3,.h4,.h5,.h6,h1,h2,h3,h4,h5,h6{margin-bottom:16px;font-family:Hind Siliguri,sans-serif;font-weight:500;line-height:1.125}.h1,h1{font-size:2.5rem}@media(max-width:1200px){.h1,h1{font-size:calc(1.375rem + 1.5vw)}}.h2,h2{font-size:2rem}@media(max-width:1200px){.h2,h2{font-size:calc(1.325rem + .9vw)}}.h3,h3{font-size:1.75rem}@media(max-width:1200px){.h3,h3{font-size:calc(1.3rem + .6vw)}}.h4,h4{font-size:1.5rem}@media(max-width:1200px){.h4,h4{font-size:calc(1.275rem + .3vw)}}.h5,h5{font-size:1.125rem}.h6,h6{font-size:.875rem}.lead{font-size:1.375rem;font-weight:400}@media(max-width:1200px){.lead{font-size:calc(1.2625rem + .15vw)}}.display-1{font-size:4rem;font-weight:600;line-height:1.125}@media(max-width:1200px){.display-1{font-size:calc(1.525rem + 3.3vw)}}.display-2{font-size:2.5rem;font-weight:600;line-height:1.125}@media(max-width:1200px){.display-2{font-size:calc(1.375rem + 1.5vw)}}.display-3{font-size:2rem;font-weight:500;line-height:1.125}@media(max-width:1200px){.display-3{font-size:calc(1.325rem + .9vw)}}.display-4{font-size:1.75rem;font-weight:500;line-height:1.125}@media(max-width:1200px){.display-4{font-size:calc(1.3rem + .6vw)}}hr{margin-top:8px;margin-bottom:8px;border:0;border-top:1px solid rgba(33,37,41,.1)}.small,small{font-size:80%;font-weight:400}.mark,mark{padding:.2em;background-color:#fcf8e3}.list-inline,.list-unstyled{padding-left:0;list-style:none}.list-inline-item{display:inline-block}.list-inline-item:not(:last-child){margin-right:.5rem}.initialism{font-size:90%;text-transform:uppercase}.blockquote{margin-bottom:8px;font-size:1.25rem}.blockquote-footer{display:block;font-size:80%;color:#6c757d}.blockquote-footer:before{content:"— "}.img-fluid,.img-thumbnail{max-width:100%;height:auto}.img-thumbnail{padding:.25rem;background-color:#fff;border:1px solid #dee2e6;border-radius:8px}.figure{display:inline-block}.figure-img{margin-bottom:4px;line-height:1}.figure-caption{font-size:90%;color:#6c757d}code{font-size:87.5%;color:#e83e8c;word-wrap:break-word}a>code{color:inherit}kbd{padding:.2rem .4rem;font-size:87.5%;color:#fff;background-color:#495057;border-radius:8px}kbd kbd{padding:0;font-size:100%;font-weight:700}pre{display:block;font-size:87.5%;color:#495057}pre code{font-size:inherit;color:inherit;word-break:normal}.pre-scrollable{max-height:340px;overflow-y:scroll}.container{width:100%;padding-right:20px;padding-left:20px;margin-right:auto;margin-left:auto}@media(min-width:400px){.container{max-width:576px}}@media(min-width:616px){.container{max-width:576px}}@media(min-width:768px){.container{max-width:958px}}@media(min-width:980px){.container{max-width:1008px}}@media(min-width:1240px){.container{max-width:1118px}}.container-fluid,.container-lg,.container-md,.container-sm,.container-xl,.container-xs{width:100%;padding-right:20px;padding-left:20px;margin-right:auto;margin-left:auto}@media(min-width:400px){.container,.container-xs{max-width:576px}}@media(min-width:616px){.container,.container-sm,.container-xs{max-width:576px}}@media(min-width:768px){.container,.container-md,.container-sm,.container-xs{max-width:958px}}@media(min-width:980px){.container,.container-lg,.container-md,.container-sm,.container-xs{max-width:1008px}}@media(min-width:1240px){.container,.container-lg,.container-md,.container-sm,.container-xl,.container-xs{max-width:1118px}}.row{display:flex;flex-wrap:wrap;margin-right:-20px;margin-left:-20px}.no-gutters{margin-right:0;margin-left:0}.no-gutters>.col,.no-gutters>[class*=col-]{padding-right:0;padding-left:0}.col,.col-1,.col-2,.col-3,.col-4,.col-5,.col-6,.col-7,.col-8,.col-9,.col-10,.col-11,.col-12,.col-auto,.col-lg,.col-lg-1,.col-lg-2,.col-lg-3,.col-lg-4,.col-lg-5,.col-lg-6,.col-lg-7,.col-lg-8,.col-lg-9,.col-lg-10,.col-lg-11,.col-lg-12,.col-lg-auto,.col-md,.col-md-1,.col-md-2,.col-md-3,.col-md-4,.col-md-5,.col-md-6,.col-md-7,.col-md-8,.col-md-9,.col-md-10,.col-md-11,.col-md-12,.col-md-auto,.col-sm,.col-sm-1,.col-sm-2,.col-sm-3,.col-sm-4,.col-sm-5,.col-sm-6,.col-sm-7,.col-sm-8,.col-sm-9,.col-sm-10,.col-sm-11,.col-sm-12,.col-sm-auto,.col-xl,.col-xl-1,.col-xl-2,.col-xl-3,.col-xl-4,.col-xl-5,.col-xl-6,.col-xl-7,.col-xl-8,.col-xl-9,.col-xl-10,.col-xl-11,.col-xl-12,.col-xl-auto,.col-xs,.col-xs-1,.col-xs-2,.col-xs-3,.col-xs-4,.col-xs-5,.col-xs-6,.col-xs-7,.col-xs-8,.col-xs-9,.col-xs-10,.col-xs-11,.col-xs-12,.col-xs-auto{position:relative;width:100%;padding-right:20px;padding-left:20px}.col{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-1>*{flex:0 0 100%;max-width:100%}.row-cols-2>*{flex:0 0 50%;max-width:50%}.row-cols-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-4>*{flex:0 0 25%;max-width:25%}.row-cols-5>*{flex:0 0 20%;max-width:20%}.row-cols-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-auto{flex:0 0 auto;width:auto;max-width:100%}.col-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-3{flex:0 0 25%;max-width:25%}.col-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-6{flex:0 0 50%;max-width:50%}.col-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-9{flex:0 0 75%;max-width:75%}.col-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-12{flex:0 0 100%;max-width:100%}.order-first{order:-1}.order-last{order:13}.order-0{order:0}.order-1{order:1}.order-2{order:2}.order-3{order:3}.order-4{order:4}.order-5{order:5}.order-6{order:6}.order-7{order:7}.order-8{order:8}.order-9{order:9}.order-10{order:10}.order-11{order:11}.order-12{order:12}.offset-1{margin-left:8.3333333333%}.offset-2{margin-left:16.6666666667%}.offset-3{margin-left:25%}.offset-4{margin-left:33.3333333333%}.offset-5{margin-left:41.6666666667%}.offset-6{margin-left:50%}.offset-7{margin-left:58.3333333333%}.offset-8{margin-left:66.6666666667%}.offset-9{margin-left:75%}.offset-10{margin-left:83.3333333333%}.offset-11{margin-left:91.6666666667%}@media(min-width:400px){.col-xs{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-xs-1>*{flex:0 0 100%;max-width:100%}.row-cols-xs-2>*{flex:0 0 50%;max-width:50%}.row-cols-xs-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-xs-4>*{flex:0 0 25%;max-width:25%}.row-cols-xs-5>*{flex:0 0 20%;max-width:20%}.row-cols-xs-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-xs-auto{flex:0 0 auto;width:auto;max-width:100%}.col-xs-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-xs-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-xs-3{flex:0 0 25%;max-width:25%}.col-xs-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-xs-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-xs-6{flex:0 0 50%;max-width:50%}.col-xs-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-xs-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-xs-9{flex:0 0 75%;max-width:75%}.col-xs-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-xs-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-xs-12{flex:0 0 100%;max-width:100%}.order-xs-first{order:-1}.order-xs-last{order:13}.order-xs-0{order:0}.order-xs-1{order:1}.order-xs-2{order:2}.order-xs-3{order:3}.order-xs-4{order:4}.order-xs-5{order:5}.order-xs-6{order:6}.order-xs-7{order:7}.order-xs-8{order:8}.order-xs-9{order:9}.order-xs-10{order:10}.order-xs-11{order:11}.order-xs-12{order:12}.offset-xs-0{margin-left:0}.offset-xs-1{margin-left:8.3333333333%}.offset-xs-2{margin-left:16.6666666667%}.offset-xs-3{margin-left:25%}.offset-xs-4{margin-left:33.3333333333%}.offset-xs-5{margin-left:41.6666666667%}.offset-xs-6{margin-left:50%}.offset-xs-7{margin-left:58.3333333333%}.offset-xs-8{margin-left:66.6666666667%}.offset-xs-9{margin-left:75%}.offset-xs-10{margin-left:83.3333333333%}.offset-xs-11{margin-left:91.6666666667%}}@media(min-width:616px){.col-sm{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-sm-1>*{flex:0 0 100%;max-width:100%}.row-cols-sm-2>*{flex:0 0 50%;max-width:50%}.row-cols-sm-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-sm-4>*{flex:0 0 25%;max-width:25%}.row-cols-sm-5>*{flex:0 0 20%;max-width:20%}.row-cols-sm-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-sm-auto{flex:0 0 auto;width:auto;max-width:100%}.col-sm-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-sm-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-sm-3{flex:0 0 25%;max-width:25%}.col-sm-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-sm-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-sm-6{flex:0 0 50%;max-width:50%}.col-sm-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-sm-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-sm-9{flex:0 0 75%;max-width:75%}.col-sm-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-sm-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-sm-12{flex:0 0 100%;max-width:100%}.order-sm-first{order:-1}.order-sm-last{order:13}.order-sm-0{order:0}.order-sm-1{order:1}.order-sm-2{order:2}.order-sm-3{order:3}.order-sm-4{order:4}.order-sm-5{order:5}.order-sm-6{order:6}.order-sm-7{order:7}.order-sm-8{order:8}.order-sm-9{order:9}.order-sm-10{order:10}.order-sm-11{order:11}.order-sm-12{order:12}.offset-sm-0{margin-left:0}.offset-sm-1{margin-left:8.3333333333%}.offset-sm-2{margin-left:16.6666666667%}.offset-sm-3{margin-left:25%}.offset-sm-4{margin-left:33.3333333333%}.offset-sm-5{margin-left:41.6666666667%}.offset-sm-6{margin-left:50%}.offset-sm-7{margin-left:58.3333333333%}.offset-sm-8{margin-left:66.6666666667%}.offset-sm-9{margin-left:75%}.offset-sm-10{margin-left:83.3333333333%}.offset-sm-11{margin-left:91.6666666667%}}@media(min-width:768px){.col-md{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-md-1>*{flex:0 0 100%;max-width:100%}.row-cols-md-2>*{flex:0 0 50%;max-width:50%}.row-cols-md-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-md-4>*{flex:0 0 25%;max-width:25%}.row-cols-md-5>*{flex:0 0 20%;max-width:20%}.row-cols-md-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-md-auto{flex:0 0 auto;width:auto;max-width:100%}.col-md-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-md-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-md-3{flex:0 0 25%;max-width:25%}.col-md-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-md-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-md-6{flex:0 0 50%;max-width:50%}.col-md-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-md-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-md-9{flex:0 0 75%;max-width:75%}.col-md-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-md-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-md-12{flex:0 0 100%;max-width:100%}.order-md-first{order:-1}.order-md-last{order:13}.order-md-0{order:0}.order-md-1{order:1}.order-md-2{order:2}.order-md-3{order:3}.order-md-4{order:4}.order-md-5{order:5}.order-md-6{order:6}.order-md-7{order:7}.order-md-8{order:8}.order-md-9{order:9}.order-md-10{order:10}.order-md-11{order:11}.order-md-12{order:12}.offset-md-0{margin-left:0}.offset-md-1{margin-left:8.3333333333%}.offset-md-2{margin-left:16.6666666667%}.offset-md-3{margin-left:25%}.offset-md-4{margin-left:33.3333333333%}.offset-md-5{margin-left:41.6666666667%}.offset-md-6{margin-left:50%}.offset-md-7{margin-left:58.3333333333%}.offset-md-8{margin-left:66.6666666667%}.offset-md-9{margin-left:75%}.offset-md-10{margin-left:83.3333333333%}.offset-md-11{margin-left:91.6666666667%}}@media(min-width:980px){.col-lg{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-lg-1>*{flex:0 0 100%;max-width:100%}.row-cols-lg-2>*{flex:0 0 50%;max-width:50%}.row-cols-lg-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-lg-4>*{flex:0 0 25%;max-width:25%}.row-cols-lg-5>*{flex:0 0 20%;max-width:20%}.row-cols-lg-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-lg-auto{flex:0 0 auto;width:auto;max-width:100%}.col-lg-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-lg-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-lg-3{flex:0 0 25%;max-width:25%}.col-lg-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-lg-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-lg-6{flex:0 0 50%;max-width:50%}.col-lg-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-lg-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-lg-9{flex:0 0 75%;max-width:75%}.col-lg-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-lg-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-lg-12{flex:0 0 100%;max-width:100%}.order-lg-first{order:-1}.order-lg-last{order:13}.order-lg-0{order:0}.order-lg-1{order:1}.order-lg-2{order:2}.order-lg-3{order:3}.order-lg-4{order:4}.order-lg-5{order:5}.order-lg-6{order:6}.order-lg-7{order:7}.order-lg-8{order:8}.order-lg-9{order:9}.order-lg-10{order:10}.order-lg-11{order:11}.order-lg-12{order:12}.offset-lg-0{margin-left:0}.offset-lg-1{margin-left:8.3333333333%}.offset-lg-2{margin-left:16.6666666667%}.offset-lg-3{margin-left:25%}.offset-lg-4{margin-left:33.3333333333%}.offset-lg-5{margin-left:41.6666666667%}.offset-lg-6{margin-left:50%}.offset-lg-7{margin-left:58.3333333333%}.offset-lg-8{margin-left:66.6666666667%}.offset-lg-9{margin-left:75%}.offset-lg-10{margin-left:83.3333333333%}.offset-lg-11{margin-left:91.6666666667%}}@media(min-width:1240px){.col-xl{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-xl-1>*{flex:0 0 100%;max-width:100%}.row-cols-xl-2>*{flex:0 0 50%;max-width:50%}.row-cols-xl-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-xl-4>*{flex:0 0 25%;max-width:25%}.row-cols-xl-5>*{flex:0 0 20%;max-width:20%}.row-cols-xl-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-xl-auto{flex:0 0 auto;width:auto;max-width:100%}.col-xl-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-xl-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-xl-3{flex:0 0 25%;max-width:25%}.col-xl-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-xl-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-xl-6{flex:0 0 50%;max-width:50%}.col-xl-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-xl-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-xl-9{flex:0 0 75%;max-width:75%}.col-xl-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-xl-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-xl-12{flex:0 0 100%;max-width:100%}.order-xl-first{order:-1}.order-xl-last{order:13}.order-xl-0{order:0}.order-xl-1{order:1}.order-xl-2{order:2}.order-xl-3{order:3}.order-xl-4{order:4}.order-xl-5{order:5}.order-xl-6{order:6}.order-xl-7{order:7}.order-xl-8{order:8}.order-xl-9{order:9}.order-xl-10{order:10}.order-xl-11{order:11}.order-xl-12{order:12}.offset-xl-0{margin-left:0}.offset-xl-1{margin-left:8.3333333333%}.offset-xl-2{margin-left:16.6666666667%}.offset-xl-3{margin-left:25%}.offset-xl-4{margin-left:33.3333333333%}.offset-xl-5{margin-left:41.6666666667%}.offset-xl-6{margin-left:50%}.offset-xl-7{margin-left:58.3333333333%}.offset-xl-8{margin-left:66.6666666667%}.offset-xl-9{margin-left:75%}.offset-xl-10{margin-left:83.3333333333%}.offset-xl-11{margin-left:91.6666666667%}}.table{width:100%;margin-bottom:8px;color:#212529}.table td,.table th{padding:.75rem;vertical-align:top;border-top:1px solid #d6dbdf}.table thead th{vertical-align:bottom;border-bottom:2px solid #d6dbdf}.table tbody+tbody{border-top:2px solid #d6dbdf}.table-sm td,.table-sm th{padding:.3rem}.table-bordered,.table-bordered td,.table-bordered th{border:1px solid #d6dbdf}.table-bordered thead td,.table-bordered thead th{border-bottom-width:2px}.table-borderless tbody+tbody,.table-borderless td,.table-borderless th,.table-borderless thead th{border:0}.table-striped tbody tr:nth-of-type(odd){background-color:rgba(33,37,41,.05)}.table-hover tbody tr:hover{color:#212529;background-color:rgba(33,37,41,.075)}.table-primary,.table-primary>td,.table-primary>th{background-color:#fff1b8}.table-primary tbody+tbody,.table-primary td,.table-primary th,.table-primary thead th{border-color:#ffe47a}.table-hover .table-primary:hover,.table-hover .table-primary:hover>td,.table-hover .table-primary:hover>th{background-color:#ffec9f}.table-secondary,.table-secondary>td,.table-secondary>th{background-color:#c1c2c3}.table-secondary tbody+tbody,.table-secondary td,.table-secondary th,.table-secondary thead th{border-color:#8c8e90}.table-hover .table-secondary:hover,.table-hover .table-secondary:hover>td,.table-hover .table-secondary:hover>th{background-color:#b4b5b6}.table-success,.table-success>td,.table-success>th{background-color:#c3e6cb}.table-success tbody+tbody,.table-success td,.table-success th,.table-success thead th{border-color:#8fd19e}.table-hover .table-success:hover,.table-hover .table-success:hover>td,.table-hover .table-success:hover>th{background-color:#b1dfbb}.table-info,.table-info>td,.table-info>th{background-color:#bee5eb}.table-info tbody+tbody,.table-info td,.table-info th,.table-info thead th{border-color:#86cfda}.table-hover .table-info:hover,.table-hover .table-info:hover>td,.table-hover .table-info:hover>th{background-color:#abdde5}.table-warning,.table-warning>td,.table-warning>th{background-color:#ffeeba}.table-warning tbody+tbody,.table-warning td,.table-warning th,.table-warning thead th{border-color:#ffdf7e}.table-hover .table-warning:hover,.table-hover .table-warning:hover>td,.table-hover .table-warning:hover>th{background-color:#ffe8a1}.table-danger,.table-danger>td,.table-danger>th{background-color:#f5c6cb}.table-danger tbody+tbody,.table-danger td,.table-danger th,.table-danger thead th{border-color:#ed969e}.table-hover .table-danger:hover,.table-hover .table-danger:hover>td,.table-hover .table-danger:hover>th{background-color:#f1b0b7}.table-light,.table-light>td,.table-light>th{background-color:#fbfcfd}.table-light tbody+tbody,.table-light td,.table-light th,.table-light thead th{border-color:#f8fafc}.table-hover .table-light:hover,.table-hover .table-light:hover>td,.table-hover .table-light:hover>th{background-color:#eaeff5}.table-dark,.table-dark>td,.table-dark>th{background-color:#ccced0}.table-dark tbody+tbody,.table-dark td,.table-dark th,.table-dark thead th{border-color:#a0a4a8}.table-hover .table-dark:hover,.table-hover .table-dark:hover>td,.table-hover .table-dark:hover>th{background-color:#bfc1c4}.table-primary-light,.table-primary-light>td,.table-primary-light>th{background-color:#fffefb}.table-primary-light tbody+tbody,.table-primary-light td,.table-primary-light th,.table-primary-light thead th{border-color:#fffcf7}.table-hover .table-primary-light:hover,.table-hover .table-primary-light:hover>td,.table-hover .table-primary-light:hover>th{background-color:#fff8e2}.table-secondary-light,.table-secondary-light>td,.table-secondary-light>th{background-color:#fff}.table-secondary-light tbody+tbody,.table-secondary-light td,.table-secondary-light th,.table-secondary-light thead th{border-color:#fff}.table-hover .table-secondary-light:hover,.table-hover .table-secondary-light:hover>td,.table-hover .table-secondary-light:hover>th{background-color:#f2f2f2}.table-tertiary,.table-tertiary>td,.table-tertiary>th{background-color:#c2dafc}.table-tertiary tbody+tbody,.table-tertiary td,.table-tertiary th,.table-tertiary thead th{border-color:#8ebaf9}.table-hover .table-tertiary:hover,.table-hover .table-tertiary:hover>td,.table-hover .table-tertiary:hover>th{background-color:#aacbfb}.table-tertiary-light,.table-tertiary-light>td,.table-tertiary-light>th{background-color:#f7fbff}.table-tertiary-light tbody+tbody,.table-tertiary-light td,.table-tertiary-light th,.table-tertiary-light thead th{border-color:#f0f8fe}.table-hover .table-tertiary-light:hover,.table-hover .table-tertiary-light:hover>td,.table-hover .table-tertiary-light:hover>th{background-color:#deeeff}.table-white,.table-white>td,.table-white>th{background-color:#fff}.table-white tbody+tbody,.table-white td,.table-white th,.table-white thead th{border-color:#fff}.table-hover .table-white:hover,.table-hover .table-white:hover>td,.table-hover .table-white:hover>th{background-color:#f2f2f2}.table-black,.table-black>td,.table-black>th{background-color:#c1c2c3}.table-black tbody+tbody,.table-black td,.table-black th,.table-black thead th{border-color:#8c8e90}.table-hover .table-black:hover,.table-hover .table-black:hover>td,.table-hover .table-black:hover>th{background-color:#b4b5b6}.table-blue,.table-blue>td,.table-blue>th{background-color:#c2dafc}.table-blue tbody+tbody,.table-blue td,.table-blue th,.table-blue thead th{border-color:#8ebaf9}.table-hover .table-blue:hover,.table-hover .table-blue:hover>td,.table-hover .table-blue:hover>th{background-color:#aacbfb}.table-light-blue,.table-light-blue>td,.table-light-blue>th{background-color:#f7fbff}.table-light-blue tbody+tbody,.table-light-blue td,.table-light-blue th,.table-light-blue thead th{border-color:#f0f8fe}.table-hover .table-light-blue:hover,.table-hover .table-light-blue:hover>td,.table-hover .table-light-blue:hover>th{background-color:#deeeff}.table-yellow,.table-yellow>td,.table-yellow>th{background-color:#fff1b8}.table-yellow tbody+tbody,.table-yellow td,.table-yellow th,.table-yellow thead th{border-color:#ffe47a}.table-hover .table-yellow:hover,.table-hover .table-yellow:hover>td,.table-hover .table-yellow:hover>th{background-color:#ffec9f}.table-light-yellow,.table-light-yellow>td,.table-light-yellow>th{background-color:#fffefb}.table-light-yellow tbody+tbody,.table-light-yellow td,.table-light-yellow th,.table-light-yellow thead th{border-color:#fffcf7}.table-hover .table-light-yellow:hover,.table-hover .table-light-yellow:hover>td,.table-hover .table-light-yellow:hover>th{background-color:#fff8e2}.table-orange,.table-orange>td,.table-orange>th{background-color:#ffdfb8}.table-orange tbody+tbody,.table-orange td,.table-orange th,.table-orange thead th{border-color:#ffc37a}.table-hover .table-orange:hover,.table-hover .table-orange:hover>td,.table-hover .table-orange:hover>th{background-color:#ffd49f}.table-light-orange,.table-light-orange>td,.table-light-orange>th{background-color:#fff7ea}.table-light-orange tbody+tbody,.table-light-orange td,.table-light-orange th,.table-light-orange thead th{border-color:#fff1d9}.table-hover .table-light-orange:hover,.table-hover .table-light-orange:hover>td,.table-hover .table-light-orange:hover>th{background-color:#ffedd1}.table-red,.table-red>td,.table-red>th{background-color:#ffc8c8}.table-red tbody+tbody,.table-red td,.table-red th,.table-red thead th{border-color:#ff9898}.table-hover .table-red:hover,.table-hover .table-red:hover>td,.table-hover .table-red:hover>th{background-color:#ffafaf}.table-light-red,.table-light-red>td,.table-light-red>th{background-color:#fff7f7}.table-light-red tbody+tbody,.table-light-red td,.table-light-red th,.table-light-red thead th{border-color:#fff1ef}.table-hover .table-light-red:hover,.table-hover .table-light-red:hover>td,.table-hover .table-light-red:hover>th{background-color:#ffdede}.table-medium,.table-medium>td,.table-medium>th{background-color:#f4f5f6}.table-medium tbody+tbody,.table-medium td,.table-medium th,.table-medium thead th{border-color:#eaecee}.table-hover .table-medium:hover,.table-hover .table-medium:hover>td,.table-hover .table-medium:hover>th{background-color:#e6e8eb}.table-active,.table-active>td,.table-active>th{background-color:rgba(33,37,41,.075)}.table-hover .table-active:hover,.table-hover .table-active:hover>td,.table-hover .table-active:hover>th{background-color:rgba(22,24,27,.075)}.table .thead-dark th{color:#fff;background-color:#343a40;border-color:#454d55}.table .thead-light th{color:#6c757d;background-color:#e9ecef;border-color:#d6dbdf}.table-dark{color:#fff;background-color:#343a40}.table-dark td,.table-dark th,.table-dark thead th{border-color:#454d55}.table-dark.table-bordered{border:0}.table-dark.table-striped tbody tr:nth-of-type(odd){background-color:hsla(0,0%,100%,.05)}.table-dark.table-hover tbody tr:hover{color:#fff;background-color:hsla(0,0%,100%,.075)}@media(max-width:399.98px){.table-responsive-xs{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive-xs>.table-bordered{border:0}}@media(max-width:615.98px){.table-responsive-sm{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive-sm>.table-bordered{border:0}}@media(max-width:767.98px){.table-responsive-md{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive-md>.table-bordered{border:0}}@media(max-width:979.98px){.table-responsive-lg{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive-lg>.table-bordered{border:0}}@media(max-width:1239.98px){.table-responsive-xl{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive-xl>.table-bordered{border:0}}.table-responsive{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive>.table-bordered{border:0}.form-control{display:block;width:100%;height:calc(1.5em + .75rem + 2px);padding:.375rem .75rem;font-size:1rem;font-weight:400;line-height:1.5;color:#6c757d;background-color:#fff;background-clip:padding-box;border:1px solid #ced4da;border-radius:8px;transition:border-color .15s ease-in-out,box-shadow .15s ease-in-out}@media(prefers-reduced-motion:reduce){.form-control{transition:none}}.form-control::-ms-expand{background-color:transparent;border:0}.form-control:-moz-focusring{color:transparent;text-shadow:0 0 0 #6c757d}.form-control:focus{color:#6c757d;background-color:#fff;border-color:#ffe680;outline:0;box-shadow:0 0 0 .2rem rgba(255,204,0,.25)}.form-control::-moz-placeholder{color:#6c757d;opacity:1}.form-control:-ms-input-placeholder{color:#6c757d;opacity:1}.form-control::placeholder{color:#6c757d;opacity:1}.form-control:disabled,.form-control[readonly]{background-color:#e9ecef;opacity:1}select.form-control:focus::-ms-value{color:#6c757d;background-color:#fff}.form-control-file,.form-control-range{display:block;width:100%}.col-form-label{padding-top:calc(.375rem + 1px);padding-bottom:calc(.375rem + 1px);margin-bottom:0;font-size:inherit;line-height:1.5}.col-form-label-lg{padding-top:calc(.5rem + 1px);padding-bottom:calc(.5rem + 1px);font-size:1.125rem;line-height:1.5}.col-form-label-sm{padding-top:calc(.25rem + 1px);padding-bottom:calc(.25rem + 1px);font-size:.875rem;line-height:1.5}.form-control-plaintext{display:block;width:100%;padding:.375rem 0;margin-bottom:0;font-size:1rem;line-height:1.5;color:#212529;background-color:transparent;border:solid transparent;border-width:1px 0}.form-control-plaintext.form-control-lg,.form-control-plaintext.form-control-sm{padding-right:0;padding-left:0}.form-control-sm{height:calc(1.5em + .5rem + 2px);padding:.25rem .5rem;font-size:.875rem;line-height:1.5;border-radius:8px}.form-control-lg{height:calc(1.5em + 1rem + 2px);padding:.5rem 1rem;font-size:1.125rem;line-height:1.5;border-radius:8px}select.form-control[multiple],select.form-control[size],textarea.form-control{height:auto}.form-group{margin-bottom:1rem}.form-text{display:block;margin-top:.25rem}.form-row{display:flex;flex-wrap:wrap;margin-right:-5px;margin-left:-5px}.form-row>.col,.form-row>[class*=col-]{padding-right:5px;padding-left:5px}.form-check{position:relative;display:block;padding-left:1.25rem}.form-check-input{position:absolute;margin-top:.3rem;margin-left:-1.25rem}.form-check-input:disabled~.form-check-label,.form-check-input[disabled]~.form-check-label{color:#6c757d}.form-check-label{margin-bottom:0}.form-check-inline{display:inline-flex;align-items:center;padding-left:0;margin-right:.75rem}.form-check-inline .form-check-input{position:static;margin-top:0;margin-right:.3125rem;margin-left:0}.valid-feedback{display:none;width:100%;margin-top:.25rem;font-size:80%;color:#28a745}.valid-tooltip{position:absolute;top:100%;z-index:5;display:none;max-width:100%;padding:.25rem .5rem;margin-top:.1rem;font-size:.875rem;line-height:1.5;color:#fff;background-color:rgba(40,167,69,.9);border-radius:8px}.is-valid~.valid-feedback,.is-valid~.valid-tooltip,.was-validated :valid~.valid-feedback,.was-validated :valid~.valid-tooltip{display:block}.form-control.is-valid,.was-validated .form-control:valid{border-color:#28a745;padding-right:calc(1.5em + .75rem);background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='8' height='8'%3E%3Cpath fill='%2328a745' d='M2.3 6.73L.6 4.53c-.4-1.04.46-1.4 1.1-.8l1.1 1.4 3.4-3.8c.6-.63 1.6-.27 1.2.7l-4 4.6c-.43.5-.8.4-1.1.1z'/%3E%3C/svg%3E");background-repeat:no-repeat;background-position:right calc(.375em + .1875rem) center;background-size:calc(.75em + .375rem) calc(.75em + .375rem)}.form-control.is-valid:focus,.was-validated .form-control:valid:focus{border-color:#28a745;box-shadow:0 0 0 .2rem rgba(40,167,69,.25)}.was-validated textarea.form-control:valid,textarea.form-control.is-valid{padding-right:calc(1.5em + .75rem);background-position:top calc(.375em + .1875rem) right calc(.375em + .1875rem)}.custom-select.is-valid,.was-validated .custom-select:valid{border-color:#28a745;padding-right:calc(.75em + 2.3125rem);background:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='4' height='5'%3E%3Cpath fill='%23343a40' d='M2 0L0 2h4zm0 5L0 3h4z'/%3E%3C/svg%3E") no-repeat right .75rem center/8px 10px,url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='8' height='8'%3E%3Cpath fill='%2328a745' d='M2.3 6.73L.6 4.53c-.4-1.04.46-1.4 1.1-.8l1.1 1.4 3.4-3.8c.6-.63 1.6-.27 1.2.7l-4 4.6c-.43.5-.8.4-1.1.1z'/%3E%3C/svg%3E") #fff no-repeat center right 1.75rem/calc(.75em + .375rem) calc(.75em + .375rem)}.custom-select.is-valid:focus,.was-validated .custom-select:valid:focus{border-color:#28a745;box-shadow:0 0 0 .2rem rgba(40,167,69,.25)}.form-check-input.is-valid~.form-check-label,.was-validated .form-check-input:valid~.form-check-label{color:#28a745}.form-check-input.is-valid~.valid-feedback,.form-check-input.is-valid~.valid-tooltip,.was-validated .form-check-input:valid~.valid-feedback,.was-validated .form-check-input:valid~.valid-tooltip{display:block}.custom-control-input.is-valid~.custom-control-label,.was-validated .custom-control-input:valid~.custom-control-label{color:#28a745}.custom-control-input.is-valid~.custom-control-label:before,.was-validated .custom-control-input:valid~.custom-control-label:before{border-color:#28a745}.custom-control-input.is-valid:checked~.custom-control-label:before,.was-validated .custom-control-input:valid:checked~.custom-control-label:before{border-color:#34ce57;background-color:#34ce57}.custom-control-input.is-valid:focus~.custom-control-label:before,.was-validated .custom-control-input:valid:focus~.custom-control-label:before{box-shadow:0 0 0 .2rem rgba(40,167,69,.25)}.custom-control-input.is-valid:focus:not(:checked)~.custom-control-label:before,.custom-file-input.is-valid~.custom-file-label,.was-validated .custom-control-input:valid:focus:not(:checked)~.custom-control-label:before,.was-validated .custom-file-input:valid~.custom-file-label{border-color:#28a745}.custom-file-input.is-valid:focus~.custom-file-label,.was-validated .custom-file-input:valid:focus~.custom-file-label{border-color:#28a745;box-shadow:0 0 0 .2rem rgba(40,167,69,.25)}.invalid-feedback{display:none;width:100%;margin-top:.25rem;font-size:80%;color:#dc3545}.invalid-tooltip{position:absolute;top:100%;z-index:5;display:none;max-width:100%;padding:.25rem .5rem;margin-top:.1rem;font-size:.875rem;line-height:1.5;color:#fff;background-color:rgba(220,53,69,.9);border-radius:8px}.is-invalid~.invalid-feedback,.is-invalid~.invalid-tooltip,.was-validated :invalid~.invalid-feedback,.was-validated :invalid~.invalid-tooltip{display:block}.form-control.is-invalid,.was-validated .form-control:invalid{border-color:#dc3545;padding-right:calc(1.5em + .75rem);background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='12' height='12' fill='none' stroke='%23dc3545'%3E%3Ccircle cx='6' cy='6' r='4.5'/%3E%3Cpath stroke-linejoin='round' d='M5.8 3.6h.4L6 6.5z'/%3E%3Ccircle cx='6' cy='8.2' r='.6' fill='%23dc3545' stroke='none'/%3E%3C/svg%3E");background-repeat:no-repeat;background-position:right calc(.375em + .1875rem) center;background-size:calc(.75em + .375rem) calc(.75em + .375rem)}.form-control.is-invalid:focus,.was-validated .form-control:invalid:focus{border-color:#dc3545;box-shadow:0 0 0 .2rem rgba(220,53,69,.25)}.was-validated textarea.form-control:invalid,textarea.form-control.is-invalid{padding-right:calc(1.5em + .75rem);background-position:top calc(.375em + .1875rem) right calc(.375em + .1875rem)}.custom-select.is-invalid,.was-validated .custom-select:invalid{border-color:#dc3545;padding-right:calc(.75em + 2.3125rem);background:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='4' height='5'%3E%3Cpath fill='%23343a40' d='M2 0L0 2h4zm0 5L0 3h4z'/%3E%3C/svg%3E") no-repeat right .75rem center/8px 10px,url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='12' height='12' fill='none' stroke='%23dc3545'%3E%3Ccircle cx='6' cy='6' r='4.5'/%3E%3Cpath stroke-linejoin='round' d='M5.8 3.6h.4L6 6.5z'/%3E%3Ccircle cx='6' cy='8.2' r='.6' fill='%23dc3545' stroke='none'/%3E%3C/svg%3E") #fff no-repeat center right 1.75rem/calc(.75em + .375rem) calc(.75em + .375rem)}.custom-select.is-invalid:focus,.was-validated .custom-select:invalid:focus{border-color:#dc3545;box-shadow:0 0 0 .2rem rgba(220,53,69,.25)}.form-check-input.is-invalid~.form-check-label,.was-validated .form-check-input:invalid~.form-check-label{color:#dc3545}.form-check-input.is-invalid~.invalid-feedback,.form-check-input.is-invalid~.invalid-tooltip,.was-validated .form-check-input:invalid~.invalid-feedback,.was-validated .form-check-input:invalid~.invalid-tooltip{display:block}.custom-control-input.is-invalid~.custom-control-label,.was-validated .custom-control-input:invalid~.custom-control-label{color:#dc3545}.custom-control-input.is-invalid~.custom-control-label:before,.was-validated .custom-control-input:invalid~.custom-control-label:before{border-color:#dc3545}.custom-control-input.is-invalid:checked~.custom-control-label:before,.was-validated .custom-control-input:invalid:checked~.custom-control-label:before{border-color:#e4606d;background-color:#e4606d}.custom-control-input.is-invalid:focus~.custom-control-label:before,.was-validated .custom-control-input:invalid:focus~.custom-control-label:before{box-shadow:0 0 0 .2rem rgba(220,53,69,.25)}.custom-control-input.is-invalid:focus:not(:checked)~.custom-control-label:before,.custom-file-input.is-invalid~.custom-file-label,.was-validated .custom-control-input:invalid:focus:not(:checked)~.custom-control-label:before,.was-validated .custom-file-input:invalid~.custom-file-label{border-color:#dc3545}.custom-file-input.is-invalid:focus~.custom-file-label,.was-validated .custom-file-input:invalid:focus~.custom-file-label{border-color:#dc3545;box-shadow:0 0 0 .2rem rgba(220,53,69,.25)}.form-inline{display:flex;flex-flow:row wrap;align-items:center}.form-inline .form-check{width:100%}@media(min-width:616px){.form-inline label{justify-content:center}.form-inline .form-group,.form-inline label{display:flex;align-items:center;margin-bottom:0}.form-inline .form-group{flex:0 0 auto;flex-flow:row wrap}.form-inline .form-control{display:inline-block;width:auto;vertical-align:middle}.form-inline .form-control-plaintext{display:inline-block}.form-inline .custom-select,.form-inline .input-group{width:auto}.form-inline .form-check{display:flex;align-items:center;justify-content:center;width:auto;padding-left:0}.form-inline .form-check-input{position:relative;flex-shrink:0;margin-top:0;margin-right:.25rem;margin-left:0}.form-inline .custom-control{align-items:center;justify-content:center}.form-inline .custom-control-label{margin-bottom:0}}.btn{display:inline-block;font-family:inherit;font-weight:700;color:#212529;text-align:center;vertical-align:middle;cursor:pointer;-webkit-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none;background-color:transparent;border:1px solid transparent;padding:12px 32px;font-size:.875rem;line-height:20px;border-radius:8px;transition:color .15s ease-in-out,background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out}@media(prefers-reduced-motion:reduce){.btn{transition:none}}.btn:hover{color:#212529;text-decoration:none}.btn.focus,.btn:focus{outline:0;box-shadow:none}.btn.disabled,.btn:disabled{opacity:.65}a.btn.disabled,fieldset:disabled a.btn{pointer-events:none}.btn-primary{color:#495057;background-color:#fc0;border-color:#fc0}.btn-primary.focus,.btn-primary:focus,.btn-primary:hover{color:#495057;background-color:#d9ad00;border-color:#cca300}.btn-primary.focus,.btn-primary:focus{box-shadow:0 0 0 0 rgba(228,185,13,.5)}.btn-primary.disabled,.btn-primary:disabled{color:#495057;background-color:#fc0;border-color:#fc0}.btn-primary:not(:disabled):not(.disabled).active,.btn-primary:not(:disabled):not(.disabled):active,.show>.btn-primary.dropdown-toggle{color:#495057;background-color:#cca300;border-color:#bf9900}.btn-primary:not(:disabled):not(.disabled).active:focus,.btn-primary:not(:disabled):not(.disabled):active:focus,.show>.btn-primary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,185,13,.5)}.btn-secondary{color:#fff;background-color:#212529;border-color:#212529}.btn-secondary.focus,.btn-secondary:focus,.btn-secondary:hover{color:#fff;background-color:#101214;border-color:#0a0c0d}.btn-secondary.focus,.btn-secondary:focus{box-shadow:0 0 0 0 rgba(66,70,73,.5)}.btn-secondary.disabled,.btn-secondary:disabled{color:#fff;background-color:#212529;border-color:#212529}.btn-secondary:not(:disabled):not(.disabled).active,.btn-secondary:not(:disabled):not(.disabled):active,.show>.btn-secondary.dropdown-toggle{color:#fff;background-color:#0a0c0d;border-color:#050506}.btn-secondary:not(:disabled):not(.disabled).active:focus,.btn-secondary:not(:disabled):not(.disabled):active:focus,.show>.btn-secondary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(66,70,73,.5)}.btn-success{color:#fff;background-color:#28a745;border-color:#28a745}.btn-success.focus,.btn-success:focus,.btn-success:hover{color:#fff;background-color:#218838;border-color:#1e7e34}.btn-success.focus,.btn-success:focus{box-shadow:0 0 0 0 rgba(72,180,97,.5)}.btn-success.disabled,.btn-success:disabled{color:#fff;background-color:#28a745;border-color:#28a745}.btn-success:not(:disabled):not(.disabled).active,.btn-success:not(:disabled):not(.disabled):active,.show>.btn-success.dropdown-toggle{color:#fff;background-color:#1e7e34;border-color:#1c7430}.btn-success:not(:disabled):not(.disabled).active:focus,.btn-success:not(:disabled):not(.disabled):active:focus,.show>.btn-success.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(72,180,97,.5)}.btn-info{color:#fff;background-color:#17a2b8;border-color:#17a2b8}.btn-info.focus,.btn-info:focus,.btn-info:hover{color:#fff;background-color:#138496;border-color:#117a8b}.btn-info.focus,.btn-info:focus{box-shadow:0 0 0 0 rgba(58,176,195,.5)}.btn-info.disabled,.btn-info:disabled{color:#fff;background-color:#17a2b8;border-color:#17a2b8}.btn-info:not(:disabled):not(.disabled).active,.btn-info:not(:disabled):not(.disabled):active,.show>.btn-info.dropdown-toggle{color:#fff;background-color:#117a8b;border-color:#10707f}.btn-info:not(:disabled):not(.disabled).active:focus,.btn-info:not(:disabled):not(.disabled):active:focus,.show>.btn-info.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(58,176,195,.5)}.btn-warning{color:#495057;background-color:#ffc107;border-color:#ffc107}.btn-warning.focus,.btn-warning:focus,.btn-warning:hover{color:#495057;background-color:#e0a800;border-color:#d39e00}.btn-warning.focus,.btn-warning:focus{box-shadow:0 0 0 0 rgba(228,176,19,.5)}.btn-warning.disabled,.btn-warning:disabled{color:#495057;background-color:#ffc107;border-color:#ffc107}.btn-warning:not(:disabled):not(.disabled).active,.btn-warning:not(:disabled):not(.disabled):active,.show>.btn-warning.dropdown-toggle{color:#495057;background-color:#d39e00;border-color:#c69500}.btn-warning:not(:disabled):not(.disabled).active:focus,.btn-warning:not(:disabled):not(.disabled):active:focus,.show>.btn-warning.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,176,19,.5)}.btn-danger{color:#fff;background-color:#dc3545;border-color:#dc3545}.btn-danger.focus,.btn-danger:focus,.btn-danger:hover{color:#fff;background-color:#c82333;border-color:#bd2130}.btn-danger.focus,.btn-danger:focus{box-shadow:0 0 0 0 rgba(225,83,97,.5)}.btn-danger.disabled,.btn-danger:disabled{color:#fff;background-color:#dc3545;border-color:#dc3545}.btn-danger:not(:disabled):not(.disabled).active,.btn-danger:not(:disabled):not(.disabled):active,.show>.btn-danger.dropdown-toggle{color:#fff;background-color:#bd2130;border-color:#b21f2d}.btn-danger:not(:disabled):not(.disabled).active:focus,.btn-danger:not(:disabled):not(.disabled):active:focus,.show>.btn-danger.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(225,83,97,.5)}.btn-light{color:#495057;background-color:#f1f6f9;border-color:#f1f6f9}.btn-light.focus,.btn-light:focus,.btn-light:hover{color:#495057;background-color:#d6e5ee;border-color:#cddfea}.btn-light.focus,.btn-light:focus{box-shadow:0 0 0 0 rgba(216,221,225,.5)}.btn-light.disabled,.btn-light:disabled{color:#495057;background-color:#f1f6f9;border-color:#f1f6f9}.btn-light:not(:disabled):not(.disabled).active,.btn-light:not(:disabled):not(.disabled):active,.show>.btn-light.dropdown-toggle{color:#495057;background-color:#cddfea;border-color:#c4d9e6}.btn-light:not(:disabled):not(.disabled).active:focus,.btn-light:not(:disabled):not(.disabled):active:focus,.show>.btn-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(216,221,225,.5)}.btn-dark{color:#fff;background-color:#495057;border-color:#495057}.btn-dark.focus,.btn-dark:focus,.btn-dark:hover{color:#fff;background-color:#383d42;border-color:#32373b}.btn-dark.focus,.btn-dark:focus{box-shadow:0 0 0 0 rgba(100,106,112,.5)}.btn-dark.disabled,.btn-dark:disabled{color:#fff;background-color:#495057;border-color:#495057}.btn-dark:not(:disabled):not(.disabled).active,.btn-dark:not(:disabled):not(.disabled):active,.show>.btn-dark.dropdown-toggle{color:#fff;background-color:#32373b;border-color:#2c3034}.btn-dark:not(:disabled):not(.disabled).active:focus,.btn-dark:not(:disabled):not(.disabled):active:focus,.show>.btn-dark.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(100,106,112,.5)}.btn-primary-light{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-primary-light.focus,.btn-primary-light:focus,.btn-primary-light:hover{color:#495057;background-color:#ffedca;border-color:#ffe9bd}.btn-primary-light.focus,.btn-primary-light:focus{box-shadow:0 0 0 0 rgba(228,225,217,.5)}.btn-primary-light.disabled,.btn-primary-light:disabled{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-primary-light:not(:disabled):not(.disabled).active,.btn-primary-light:not(:disabled):not(.disabled):active,.show>.btn-primary-light.dropdown-toggle{color:#495057;background-color:#ffe9bd;border-color:#ffe5b0}.btn-primary-light:not(:disabled):not(.disabled).active:focus,.btn-primary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-primary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,225,217,.5)}.btn-secondary-light{color:#495057;background-color:#fff;border-color:#fff}.btn-secondary-light.focus,.btn-secondary-light:focus,.btn-secondary-light:hover{color:#495057;background-color:#ececec;border-color:#e6e6e6}.btn-secondary-light.focus,.btn-secondary-light:focus{box-shadow:0 0 0 0 rgba(228,229,230,.5)}.btn-secondary-light.disabled,.btn-secondary-light:disabled{color:#495057;background-color:#fff;border-color:#fff}.btn-secondary-light:not(:disabled):not(.disabled).active,.btn-secondary-light:not(:disabled):not(.disabled):active,.show>.btn-secondary-light.dropdown-toggle{color:#495057;background-color:#e6e6e6;border-color:#dfdfdf}.btn-secondary-light:not(:disabled):not(.disabled).active:focus,.btn-secondary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-secondary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,229,230,.5)}.btn-tertiary{color:#fff;background-color:#257af4;border-color:#257af4}.btn-tertiary.focus,.btn-tertiary:focus,.btn-tertiary:hover{color:#fff;background-color:#0c66e7;border-color:#0b60db}.btn-tertiary.focus,.btn-tertiary:focus{box-shadow:0 0 0 0 rgba(70,142,246,.5)}.btn-tertiary.disabled,.btn-tertiary:disabled{color:#fff;background-color:#257af4;border-color:#257af4}.btn-tertiary:not(:disabled):not(.disabled).active,.btn-tertiary:not(:disabled):not(.disabled):active,.show>.btn-tertiary.dropdown-toggle{color:#fff;background-color:#0b60db;border-color:#0a5bcf}.btn-tertiary:not(:disabled):not(.disabled).active:focus,.btn-tertiary:not(:disabled):not(.disabled):active:focus,.show>.btn-tertiary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(70,142,246,.5)}.btn-tertiary-light{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-tertiary-light.focus,.btn-tertiary-light:focus,.btn-tertiary-light:hover{color:#495057;background-color:#bedffd;border-color:#b2d8fc}.btn-tertiary-light.focus,.btn-tertiary-light:focus{box-shadow:0 0 0 0 rgba(204,217,229,.5)}.btn-tertiary-light.disabled,.btn-tertiary-light:disabled{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-tertiary-light:not(:disabled):not(.disabled).active,.btn-tertiary-light:not(:disabled):not(.disabled):active,.show>.btn-tertiary-light.dropdown-toggle{color:#495057;background-color:#b2d8fc;border-color:#a5d2fc}.btn-tertiary-light:not(:disabled):not(.disabled).active:focus,.btn-tertiary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-tertiary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(204,217,229,.5)}.btn-white{color:#495057;background-color:#fff;border-color:#fff}.btn-white.focus,.btn-white:focus,.btn-white:hover{color:#495057;background-color:#ececec;border-color:#e6e6e6}.btn-white.focus,.btn-white:focus{box-shadow:0 0 0 0 rgba(228,229,230,.5)}.btn-white.disabled,.btn-white:disabled{color:#495057;background-color:#fff;border-color:#fff}.btn-white:not(:disabled):not(.disabled).active,.btn-white:not(:disabled):not(.disabled):active,.show>.btn-white.dropdown-toggle{color:#495057;background-color:#e6e6e6;border-color:#dfdfdf}.btn-white:not(:disabled):not(.disabled).active:focus,.btn-white:not(:disabled):not(.disabled):active:focus,.show>.btn-white.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,229,230,.5)}.btn-black{color:#fff;background-color:#212529;border-color:#212529}.btn-black.focus,.btn-black:focus,.btn-black:hover{color:#fff;background-color:#101214;border-color:#0a0c0d}.btn-black.focus,.btn-black:focus{box-shadow:0 0 0 0 rgba(66,70,73,.5)}.btn-black.disabled,.btn-black:disabled{color:#fff;background-color:#212529;border-color:#212529}.btn-black:not(:disabled):not(.disabled).active,.btn-black:not(:disabled):not(.disabled):active,.show>.btn-black.dropdown-toggle{color:#fff;background-color:#0a0c0d;border-color:#050506}.btn-black:not(:disabled):not(.disabled).active:focus,.btn-black:not(:disabled):not(.disabled):active:focus,.show>.btn-black.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(66,70,73,.5)}.btn-blue{color:#fff;background-color:#257af4;border-color:#257af4}.btn-blue.focus,.btn-blue:focus,.btn-blue:hover{color:#fff;background-color:#0c66e7;border-color:#0b60db}.btn-blue.focus,.btn-blue:focus{box-shadow:0 0 0 0 rgba(70,142,246,.5)}.btn-blue.disabled,.btn-blue:disabled{color:#fff;background-color:#257af4;border-color:#257af4}.btn-blue:not(:disabled):not(.disabled).active,.btn-blue:not(:disabled):not(.disabled):active,.show>.btn-blue.dropdown-toggle{color:#fff;background-color:#0b60db;border-color:#0a5bcf}.btn-blue:not(:disabled):not(.disabled).active:focus,.btn-blue:not(:disabled):not(.disabled):active:focus,.show>.btn-blue.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(70,142,246,.5)}.btn-light-blue{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-light-blue.focus,.btn-light-blue:focus,.btn-light-blue:hover{color:#495057;background-color:#bedffd;border-color:#b2d8fc}.btn-light-blue.focus,.btn-light-blue:focus{box-shadow:0 0 0 0 rgba(204,217,229,.5)}.btn-light-blue.disabled,.btn-light-blue:disabled{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-light-blue:not(:disabled):not(.disabled).active,.btn-light-blue:not(:disabled):not(.disabled):active,.show>.btn-light-blue.dropdown-toggle{color:#495057;background-color:#b2d8fc;border-color:#a5d2fc}.btn-light-blue:not(:disabled):not(.disabled).active:focus,.btn-light-blue:not(:disabled):not(.disabled):active:focus,.show>.btn-light-blue.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(204,217,229,.5)}.btn-yellow{color:#495057;background-color:#fc0;border-color:#fc0}.btn-yellow.focus,.btn-yellow:focus,.btn-yellow:hover{color:#495057;background-color:#d9ad00;border-color:#cca300}.btn-yellow.focus,.btn-yellow:focus{box-shadow:0 0 0 0 rgba(228,185,13,.5)}.btn-yellow.disabled,.btn-yellow:disabled{color:#495057;background-color:#fc0;border-color:#fc0}.btn-yellow:not(:disabled):not(.disabled).active,.btn-yellow:not(:disabled):not(.disabled):active,.show>.btn-yellow.dropdown-toggle{color:#495057;background-color:#cca300;border-color:#bf9900}.btn-yellow:not(:disabled):not(.disabled).active:focus,.btn-yellow:not(:disabled):not(.disabled):active:focus,.show>.btn-yellow.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,185,13,.5)}.btn-light-yellow{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-light-yellow.focus,.btn-light-yellow:focus,.btn-light-yellow:hover{color:#495057;background-color:#ffedca;border-color:#ffe9bd}.btn-light-yellow.focus,.btn-light-yellow:focus{box-shadow:0 0 0 0 rgba(228,225,217,.5)}.btn-light-yellow.disabled,.btn-light-yellow:disabled{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-light-yellow:not(:disabled):not(.disabled).active,.btn-light-yellow:not(:disabled):not(.disabled):active,.show>.btn-light-yellow.dropdown-toggle{color:#495057;background-color:#ffe9bd;border-color:#ffe5b0}.btn-light-yellow:not(:disabled):not(.disabled).active:focus,.btn-light-yellow:not(:disabled):not(.disabled):active:focus,.show>.btn-light-yellow.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,225,217,.5)}.btn-orange{color:#495057;background-color:#ff8c00;border-color:#ff8c00}.btn-orange.focus,.btn-orange:focus,.btn-orange:hover{color:#fff;background-color:#d97700;border-color:#cc7000}.btn-orange.focus,.btn-orange:focus{box-shadow:0 0 0 0 rgba(228,131,13,.5)}.btn-orange.disabled,.btn-orange:disabled{color:#495057;background-color:#ff8c00;border-color:#ff8c00}.btn-orange:not(:disabled):not(.disabled).active,.btn-orange:not(:disabled):not(.disabled):active,.show>.btn-orange.dropdown-toggle{color:#fff;background-color:#cc7000;border-color:#bf6900}.btn-orange:not(:disabled):not(.disabled).active:focus,.btn-orange:not(:disabled):not(.disabled):active:focus,.show>.btn-orange.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,131,13,.5)}.btn-light-orange{color:#495057;background-color:#ffe4b5;border-color:#ffe4b5}.btn-light-orange.focus,.btn-light-orange:focus,.btn-light-orange:hover{color:#495057;background-color:#ffd68f;border-color:#ffd182}.btn-light-orange.focus,.btn-light-orange:focus{box-shadow:0 0 0 0 rgba(228,206,167,.5)}.btn-light-orange.disabled,.btn-light-orange:disabled{color:#495057;background-color:#ffe4b5;border-color:#ffe4b5}.btn-light-orange:not(:disabled):not(.disabled).active,.btn-light-orange:not(:disabled):not(.disabled):active,.show>.btn-light-orange.dropdown-toggle{color:#495057;background-color:#ffd182;border-color:#ffcd75}.btn-light-orange:not(:disabled):not(.disabled).active:focus,.btn-light-orange:not(:disabled):not(.disabled):active:focus,.show>.btn-light-orange.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,206,167,.5)}.btn-red{color:#fff;background-color:#ff3939;border-color:#ff3939}.btn-red.focus,.btn-red:focus,.btn-red:hover{color:#fff;background-color:#ff1313;border-color:#ff0606}.btn-red.focus,.btn-red:focus{box-shadow:0 0 0 0 rgba(255,87,87,.5)}.btn-red.disabled,.btn-red:disabled{color:#fff;background-color:#ff3939;border-color:#ff3939}.btn-red:not(:disabled):not(.disabled).active,.btn-red:not(:disabled):not(.disabled):active,.show>.btn-red.dropdown-toggle{color:#fff;background-color:#ff0606;border-color:#f80000}.btn-red:not(:disabled):not(.disabled).active:focus,.btn-red:not(:disabled):not(.disabled):active:focus,.show>.btn-red.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,87,87,.5)}.btn-light-red{color:#495057;background-color:#ffe4e1;border-color:#ffe4e1}.btn-light-red.focus,.btn-light-red:focus,.btn-light-red:hover{color:#495057;background-color:#ffc2bb;border-color:#ffb6ae}.btn-light-red.focus,.btn-light-red:focus{box-shadow:0 0 0 0 rgba(228,206,204,.5)}.btn-light-red.disabled,.btn-light-red:disabled{color:#495057;background-color:#ffe4e1;border-color:#ffe4e1}.btn-light-red:not(:disabled):not(.disabled).active,.btn-light-red:not(:disabled):not(.disabled):active,.show>.btn-light-red.dropdown-toggle{color:#495057;background-color:#ffb6ae;border-color:#ffaba1}.btn-light-red:not(:disabled):not(.disabled).active:focus,.btn-light-red:not(:disabled):not(.disabled):active:focus,.show>.btn-light-red.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,206,204,.5)}.btn-medium{color:#495057;background-color:#d6dbdf;border-color:#d6dbdf}.btn-medium.focus,.btn-medium:focus,.btn-medium:hover{color:#495057;background-color:#c1c8ce;border-color:#b9c2c9}.btn-medium.focus,.btn-medium:focus{box-shadow:0 0 0 0 rgba(193,198,203,.5)}.btn-medium.disabled,.btn-medium:disabled{color:#495057;background-color:#d6dbdf;border-color:#d6dbdf}.btn-medium:not(:disabled):not(.disabled).active,.btn-medium:not(:disabled):not(.disabled):active,.show>.btn-medium.dropdown-toggle{color:#495057;background-color:#b9c2c9;border-color:#b2bcc3}.btn-medium:not(:disabled):not(.disabled).active:focus,.btn-medium:not(:disabled):not(.disabled):active:focus,.show>.btn-medium.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(193,198,203,.5)}.btn-outline-primary{color:#fc0;border-color:#fc0}.btn-outline-primary:hover{color:#495057;background-color:#fc0;border-color:#fc0}.btn-outline-primary.focus,.btn-outline-primary:focus{box-shadow:0 0 0 0 rgba(255,204,0,.5)}.btn-outline-primary.disabled,.btn-outline-primary:disabled{color:#fc0;background-color:transparent}.btn-outline-primary:not(:disabled):not(.disabled).active,.btn-outline-primary:not(:disabled):not(.disabled):active,.show>.btn-outline-primary.dropdown-toggle{color:#495057;background-color:#fc0;border-color:#fc0}.btn-outline-primary:not(:disabled):not(.disabled).active:focus,.btn-outline-primary:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-primary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,204,0,.5)}.btn-outline-secondary{color:#212529;border-color:#212529}.btn-outline-secondary:hover{color:#fff;background-color:#212529;border-color:#212529}.btn-outline-secondary.focus,.btn-outline-secondary:focus{box-shadow:0 0 0 0 rgba(33,37,41,.5)}.btn-outline-secondary.disabled,.btn-outline-secondary:disabled{color:#212529;background-color:transparent}.btn-outline-secondary:not(:disabled):not(.disabled).active,.btn-outline-secondary:not(:disabled):not(.disabled):active,.show>.btn-outline-secondary.dropdown-toggle{color:#fff;background-color:#212529;border-color:#212529}.btn-outline-secondary:not(:disabled):not(.disabled).active:focus,.btn-outline-secondary:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-secondary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(33,37,41,.5)}.btn-outline-success{color:#28a745;border-color:#28a745}.btn-outline-success:hover{color:#fff;background-color:#28a745;border-color:#28a745}.btn-outline-success.focus,.btn-outline-success:focus{box-shadow:0 0 0 0 rgba(40,167,69,.5)}.btn-outline-success.disabled,.btn-outline-success:disabled{color:#28a745;background-color:transparent}.btn-outline-success:not(:disabled):not(.disabled).active,.btn-outline-success:not(:disabled):not(.disabled):active,.show>.btn-outline-success.dropdown-toggle{color:#fff;background-color:#28a745;border-color:#28a745}.btn-outline-success:not(:disabled):not(.disabled).active:focus,.btn-outline-success:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-success.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(40,167,69,.5)}.btn-outline-info{color:#17a2b8;border-color:#17a2b8}.btn-outline-info:hover{color:#fff;background-color:#17a2b8;border-color:#17a2b8}.btn-outline-info.focus,.btn-outline-info:focus{box-shadow:0 0 0 0 rgba(23,162,184,.5)}.btn-outline-info.disabled,.btn-outline-info:disabled{color:#17a2b8;background-color:transparent}.btn-outline-info:not(:disabled):not(.disabled).active,.btn-outline-info:not(:disabled):not(.disabled):active,.show>.btn-outline-info.dropdown-toggle{color:#fff;background-color:#17a2b8;border-color:#17a2b8}.btn-outline-info:not(:disabled):not(.disabled).active:focus,.btn-outline-info:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-info.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(23,162,184,.5)}.btn-outline-warning{color:#ffc107;border-color:#ffc107}.btn-outline-warning:hover{color:#495057;background-color:#ffc107;border-color:#ffc107}.btn-outline-warning.focus,.btn-outline-warning:focus{box-shadow:0 0 0 0 rgba(255,193,7,.5)}.btn-outline-warning.disabled,.btn-outline-warning:disabled{color:#ffc107;background-color:transparent}.btn-outline-warning:not(:disabled):not(.disabled).active,.btn-outline-warning:not(:disabled):not(.disabled):active,.show>.btn-outline-warning.dropdown-toggle{color:#495057;background-color:#ffc107;border-color:#ffc107}.btn-outline-warning:not(:disabled):not(.disabled).active:focus,.btn-outline-warning:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-warning.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,193,7,.5)}.btn-outline-danger{color:#dc3545;border-color:#dc3545}.btn-outline-danger:hover{color:#fff;background-color:#dc3545;border-color:#dc3545}.btn-outline-danger.focus,.btn-outline-danger:focus{box-shadow:0 0 0 0 rgba(220,53,69,.5)}.btn-outline-danger.disabled,.btn-outline-danger:disabled{color:#dc3545;background-color:transparent}.btn-outline-danger:not(:disabled):not(.disabled).active,.btn-outline-danger:not(:disabled):not(.disabled):active,.show>.btn-outline-danger.dropdown-toggle{color:#fff;background-color:#dc3545;border-color:#dc3545}.btn-outline-danger:not(:disabled):not(.disabled).active:focus,.btn-outline-danger:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-danger.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(220,53,69,.5)}.btn-outline-light{color:#f1f6f9;border-color:#f1f6f9}.btn-outline-light:hover{color:#495057;background-color:#f1f6f9;border-color:#f1f6f9}.btn-outline-light.focus,.btn-outline-light:focus{box-shadow:0 0 0 0 rgba(241,246,249,.5)}.btn-outline-light.disabled,.btn-outline-light:disabled{color:#f1f6f9;background-color:transparent}.btn-outline-light:not(:disabled):not(.disabled).active,.btn-outline-light:not(:disabled):not(.disabled):active,.show>.btn-outline-light.dropdown-toggle{color:#495057;background-color:#f1f6f9;border-color:#f1f6f9}.btn-outline-light:not(:disabled):not(.disabled).active:focus,.btn-outline-light:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(241,246,249,.5)}.btn-outline-dark{color:#495057;border-color:#495057}.btn-outline-dark:hover{color:#fff;background-color:#495057;border-color:#495057}.btn-outline-dark.focus,.btn-outline-dark:focus{box-shadow:0 0 0 0 rgba(73,80,87,.5)}.btn-outline-dark.disabled,.btn-outline-dark:disabled{color:#495057;background-color:transparent}.btn-outline-dark:not(:disabled):not(.disabled).active,.btn-outline-dark:not(:disabled):not(.disabled):active,.show>.btn-outline-dark.dropdown-toggle{color:#fff;background-color:#495057;border-color:#495057}.btn-outline-dark:not(:disabled):not(.disabled).active:focus,.btn-outline-dark:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-dark.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(73,80,87,.5)}.btn-outline-primary-light{color:#fffaf0;border-color:#fffaf0}.btn-outline-primary-light:hover{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-outline-primary-light.focus,.btn-outline-primary-light:focus{box-shadow:0 0 0 0 rgba(255,250,240,.5)}.btn-outline-primary-light.disabled,.btn-outline-primary-light:disabled{color:#fffaf0;background-color:transparent}.btn-outline-primary-light:not(:disabled):not(.disabled).active,.btn-outline-primary-light:not(:disabled):not(.disabled):active,.show>.btn-outline-primary-light.dropdown-toggle{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-outline-primary-light:not(:disabled):not(.disabled).active:focus,.btn-outline-primary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-primary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,250,240,.5)}.btn-outline-secondary-light{color:#fff;border-color:#fff}.btn-outline-secondary-light:hover{color:#495057;background-color:#fff;border-color:#fff}.btn-outline-secondary-light.focus,.btn-outline-secondary-light:focus{box-shadow:0 0 0 0 hsla(0,0%,100%,.5)}.btn-outline-secondary-light.disabled,.btn-outline-secondary-light:disabled{color:#fff;background-color:transparent}.btn-outline-secondary-light:not(:disabled):not(.disabled).active,.btn-outline-secondary-light:not(:disabled):not(.disabled):active,.show>.btn-outline-secondary-light.dropdown-toggle{color:#495057;background-color:#fff;border-color:#fff}.btn-outline-secondary-light:not(:disabled):not(.disabled).active:focus,.btn-outline-secondary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-secondary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 hsla(0,0%,100%,.5)}.btn-outline-tertiary{color:#257af4;border-color:#257af4}.btn-outline-tertiary:hover{color:#fff;background-color:#257af4;border-color:#257af4}.btn-outline-tertiary.focus,.btn-outline-tertiary:focus{box-shadow:0 0 0 0 rgba(37,122,244,.5)}.btn-outline-tertiary.disabled,.btn-outline-tertiary:disabled{color:#257af4;background-color:transparent}.btn-outline-tertiary:not(:disabled):not(.disabled).active,.btn-outline-tertiary:not(:disabled):not(.disabled):active,.show>.btn-outline-tertiary.dropdown-toggle{color:#fff;background-color:#257af4;border-color:#257af4}.btn-outline-tertiary:not(:disabled):not(.disabled).active:focus,.btn-outline-tertiary:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-tertiary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(37,122,244,.5)}.btn-outline-tertiary-light{color:#e3f1fe;border-color:#e3f1fe}.btn-outline-tertiary-light:hover{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-outline-tertiary-light.focus,.btn-outline-tertiary-light:focus{box-shadow:0 0 0 0 rgba(227,241,254,.5)}.btn-outline-tertiary-light.disabled,.btn-outline-tertiary-light:disabled{color:#e3f1fe;background-color:transparent}.btn-outline-tertiary-light:not(:disabled):not(.disabled).active,.btn-outline-tertiary-light:not(:disabled):not(.disabled):active,.show>.btn-outline-tertiary-light.dropdown-toggle{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-outline-tertiary-light:not(:disabled):not(.disabled).active:focus,.btn-outline-tertiary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-tertiary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(227,241,254,.5)}.btn-outline-white{color:#fff;border-color:#fff}.btn-outline-white:hover{color:#495057;background-color:#fff;border-color:#fff}.btn-outline-white.focus,.btn-outline-white:focus{box-shadow:0 0 0 0 hsla(0,0%,100%,.5)}.btn-outline-white.disabled,.btn-outline-white:disabled{color:#fff;background-color:transparent}.btn-outline-white:not(:disabled):not(.disabled).active,.btn-outline-white:not(:disabled):not(.disabled):active,.show>.btn-outline-white.dropdown-toggle{color:#495057;background-color:#fff;border-color:#fff}.btn-outline-white:not(:disabled):not(.disabled).active:focus,.btn-outline-white:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-white.dropdown-toggle:focus{box-shadow:0 0 0 0 hsla(0,0%,100%,.5)}.btn-outline-black{color:#212529;border-color:#212529}.btn-outline-black:hover{color:#fff;background-color:#212529;border-color:#212529}.btn-outline-black.focus,.btn-outline-black:focus{box-shadow:0 0 0 0 rgba(33,37,41,.5)}.btn-outline-black.disabled,.btn-outline-black:disabled{color:#212529;background-color:transparent}.btn-outline-black:not(:disabled):not(.disabled).active,.btn-outline-black:not(:disabled):not(.disabled):active,.show>.btn-outline-black.dropdown-toggle{color:#fff;background-color:#212529;border-color:#212529}.btn-outline-black:not(:disabled):not(.disabled).active:focus,.btn-outline-black:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-black.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(33,37,41,.5)}.btn-outline-blue{color:#257af4;border-color:#257af4}.btn-outline-blue:hover{color:#fff;background-color:#257af4;border-color:#257af4}.btn-outline-blue.focus,.btn-outline-blue:focus{box-shadow:0 0 0 0 rgba(37,122,244,.5)}.btn-outline-blue.disabled,.btn-outline-blue:disabled{color:#257af4;background-color:transparent}.btn-outline-blue:not(:disabled):not(.disabled).active,.btn-outline-blue:not(:disabled):not(.disabled):active,.show>.btn-outline-blue.dropdown-toggle{color:#fff;background-color:#257af4;border-color:#257af4}.btn-outline-blue:not(:disabled):not(.disabled).active:focus,.btn-outline-blue:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-blue.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(37,122,244,.5)}.btn-outline-light-blue{color:#e3f1fe;border-color:#e3f1fe}.btn-outline-light-blue:hover{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-outline-light-blue.focus,.btn-outline-light-blue:focus{box-shadow:0 0 0 0 rgba(227,241,254,.5)}.btn-outline-light-blue.disabled,.btn-outline-light-blue:disabled{color:#e3f1fe;background-color:transparent}.btn-outline-light-blue:not(:disabled):not(.disabled).active,.btn-outline-light-blue:not(:disabled):not(.disabled):active,.show>.btn-outline-light-blue.dropdown-toggle{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-outline-light-blue:not(:disabled):not(.disabled).active:focus,.btn-outline-light-blue:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-light-blue.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(227,241,254,.5)}.btn-outline-yellow{color:#fc0;border-color:#fc0}.btn-outline-yellow:hover{color:#495057;background-color:#fc0;border-color:#fc0}.btn-outline-yellow.focus,.btn-outline-yellow:focus{box-shadow:0 0 0 0 rgba(255,204,0,.5)}.btn-outline-yellow.disabled,.btn-outline-yellow:disabled{color:#fc0;background-color:transparent}.btn-outline-yellow:not(:disabled):not(.disabled).active,.btn-outline-yellow:not(:disabled):not(.disabled):active,.show>.btn-outline-yellow.dropdown-toggle{color:#495057;background-color:#fc0;border-color:#fc0}.btn-outline-yellow:not(:disabled):not(.disabled).active:focus,.btn-outline-yellow:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-yellow.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,204,0,.5)}.btn-outline-light-yellow{color:#fffaf0;border-color:#fffaf0}.btn-outline-light-yellow:hover{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-outline-light-yellow.focus,.btn-outline-light-yellow:focus{box-shadow:0 0 0 0 rgba(255,250,240,.5)}.btn-outline-light-yellow.disabled,.btn-outline-light-yellow:disabled{color:#fffaf0;background-color:transparent}.btn-outline-light-yellow:not(:disabled):not(.disabled).active,.btn-outline-light-yellow:not(:disabled):not(.disabled):active,.show>.btn-outline-light-yellow.dropdown-toggle{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-outline-light-yellow:not(:disabled):not(.disabled).active:focus,.btn-outline-light-yellow:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-light-yellow.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,250,240,.5)}.btn-outline-orange{color:#ff8c00;border-color:#ff8c00}.btn-outline-orange:hover{color:#495057;background-color:#ff8c00;border-color:#ff8c00}.btn-outline-orange.focus,.btn-outline-orange:focus{box-shadow:0 0 0 0 rgba(255,140,0,.5)}.btn-outline-orange.disabled,.btn-outline-orange:disabled{color:#ff8c00;background-color:transparent}.btn-outline-orange:not(:disabled):not(.disabled).active,.btn-outline-orange:not(:disabled):not(.disabled):active,.show>.btn-outline-orange.dropdown-toggle{color:#495057;background-color:#ff8c00;border-color:#ff8c00}.btn-outline-orange:not(:disabled):not(.disabled).active:focus,.btn-outline-orange:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-orange.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,140,0,.5)}.btn-outline-light-orange{color:#ffe4b5;border-color:#ffe4b5}.btn-outline-light-orange:hover{color:#495057;background-color:#ffe4b5;border-color:#ffe4b5}.btn-outline-light-orange.focus,.btn-outline-light-orange:focus{box-shadow:0 0 0 0 rgba(255,228,181,.5)}.btn-outline-light-orange.disabled,.btn-outline-light-orange:disabled{color:#ffe4b5;background-color:transparent}.btn-outline-light-orange:not(:disabled):not(.disabled).active,.btn-outline-light-orange:not(:disabled):not(.disabled):active,.show>.btn-outline-light-orange.dropdown-toggle{color:#495057;background-color:#ffe4b5;border-color:#ffe4b5}.btn-outline-light-orange:not(:disabled):not(.disabled).active:focus,.btn-outline-light-orange:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-light-orange.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,228,181,.5)}.btn-outline-red{color:#ff3939;border-color:#ff3939}.btn-outline-red:hover{color:#fff;background-color:#ff3939;border-color:#ff3939}.btn-outline-red.focus,.btn-outline-red:focus{box-shadow:0 0 0 0 rgba(255,57,57,.5)}.btn-outline-red.disabled,.btn-outline-red:disabled{color:#ff3939;background-color:transparent}.btn-outline-red:not(:disabled):not(.disabled).active,.btn-outline-red:not(:disabled):not(.disabled):active,.show>.btn-outline-red.dropdown-toggle{color:#fff;background-color:#ff3939;border-color:#ff3939}.btn-outline-red:not(:disabled):not(.disabled).active:focus,.btn-outline-red:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-red.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,57,57,.5)}.btn-outline-light-red{color:#ffe4e1;border-color:#ffe4e1}.btn-outline-light-red:hover{color:#495057;background-color:#ffe4e1;border-color:#ffe4e1}.btn-outline-light-red.focus,.btn-outline-light-red:focus{box-shadow:0 0 0 0 rgba(255,228,225,.5)}.btn-outline-light-red.disabled,.btn-outline-light-red:disabled{color:#ffe4e1;background-color:transparent}.btn-outline-light-red:not(:disabled):not(.disabled).active,.btn-outline-light-red:not(:disabled):not(.disabled):active,.show>.btn-outline-light-red.dropdown-toggle{color:#495057;background-color:#ffe4e1;border-color:#ffe4e1}.btn-outline-light-red:not(:disabled):not(.disabled).active:focus,.btn-outline-light-red:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-light-red.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,228,225,.5)}.btn-outline-medium{color:#d6dbdf;border-color:#d6dbdf}.btn-outline-medium:hover{color:#495057;background-color:#d6dbdf;border-color:#d6dbdf}.btn-outline-medium.focus,.btn-outline-medium:focus{box-shadow:0 0 0 0 rgba(214,219,223,.5)}.btn-outline-medium.disabled,.btn-outline-medium:disabled{color:#d6dbdf;background-color:transparent}.btn-outline-medium:not(:disabled):not(.disabled).active,.btn-outline-medium:not(:disabled):not(.disabled):active,.show>.btn-outline-medium.dropdown-toggle{color:#495057;background-color:#d6dbdf;border-color:#d6dbdf}.btn-outline-medium:not(:disabled):not(.disabled).active:focus,.btn-outline-medium:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-medium.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(214,219,223,.5)}.btn-link{font-weight:400;color:#ff8c00;text-decoration:none}.btn-link:hover{color:#ff8c00;text-decoration:underline}.btn-link.focus,.btn-link:focus{text-decoration:underline;box-shadow:none}.btn-link.disabled,.btn-link:disabled{color:#d6dbdf;pointer-events:none}.btn-group-lg>.btn,.btn-lg{padding:16px 32px;font-size:1.125rem;line-height:26px;border-radius:8px}.btn-group-sm>.btn,.btn-sm{padding:12px 32px;font-size:.875rem;line-height:20px;border-radius:8px}.btn-block{display:block;width:100%}.btn-block+.btn-block{margin-top:24px}input[type=button].btn-block,input[type=reset].btn-block,input[type=submit].btn-block{width:100%}.fade{transition:opacity .15s linear}@media(prefers-reduced-motion:reduce){.fade{transition:none}}.fade:not(.show){opacity:0}.collapse:not(.show){display:none}.collapsing{position:relative;height:0;overflow:hidden;transition:height .35s ease}@media(prefers-reduced-motion:reduce){.collapsing{transition:none}}.dropdown,.dropleft,.dropright,.dropup{position:relative}.dropdown-toggle{white-space:nowrap}.dropdown-toggle:after{display:inline-block;margin-left:.255em;vertical-align:.255em;content:"";border-top:.3em solid;border-right:.3em solid transparent;border-bottom:0;border-left:.3em solid transparent}.dropdown-toggle:empty:after{margin-left:0}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:10rem;padding:.5rem 0;margin:.125rem 0 0;font-size:1rem;color:#212529;text-align:left;list-style:none;background-color:#fff;background-clip:padding-box;border:1px solid rgba(33,37,41,.15);border-radius:8px}.dropdown-menu-left{right:auto;left:0}.dropdown-menu-right{right:0;left:auto}@media(min-width:400px){.dropdown-menu-xs-left{right:auto;left:0}.dropdown-menu-xs-right{right:0;left:auto}}@media(min-width:616px){.dropdown-menu-sm-left{right:auto;left:0}.dropdown-menu-sm-right{right:0;left:auto}}@media(min-width:768px){.dropdown-menu-md-left{right:auto;left:0}.dropdown-menu-md-right{right:0;left:auto}}@media(min-width:980px){.dropdown-menu-lg-left{right:auto;left:0}.dropdown-menu-lg-right{right:0;left:auto}}@media(min-width:1240px){.dropdown-menu-xl-left{right:auto;left:0}.dropdown-menu-xl-right{right:0;left:auto}}.dropup .dropdown-menu{top:auto;bottom:100%;margin-top:0;margin-bottom:.125rem}.dropup .dropdown-toggle:after{display:inline-block;margin-left:.255em;vertical-align:.255em;content:"";border-top:0;border-right:.3em solid transparent;border-bottom:.3em solid;border-left:.3em solid transparent}.dropup .dropdown-toggle:empty:after{margin-left:0}.dropright .dropdown-menu{top:0;right:auto;left:100%;margin-top:0;margin-left:.125rem}.dropright .dropdown-toggle:after{display:inline-block;margin-left:.255em;vertical-align:.255em;content:"";border-top:.3em solid transparent;border-right:0;border-bottom:.3em solid transparent;border-left:.3em solid}.dropright .dropdown-toggle:empty:after{margin-left:0}.dropright .dropdown-toggle:after{vertical-align:0}.dropleft .dropdown-menu{top:0;right:100%;left:auto;margin-top:0;margin-right:.125rem}.dropleft .dropdown-toggle:after{display:inline-block;margin-left:.255em;vertical-align:.255em;content:"";display:none}.dropleft .dropdown-toggle:before{display:inline-block;margin-right:.255em;vertical-align:.255em;content:"";border-top:.3em solid transparent;border-right:.3em solid;border-bottom:.3em solid transparent}.dropleft .dropdown-toggle:empty:after{margin-left:0}.dropleft .dropdown-toggle:before{vertical-align:0}.dropdown-menu[x-placement^=bottom],.dropdown-menu[x-placement^=left],.dropdown-menu[x-placement^=right],.dropdown-menu[x-placement^=top]{right:auto;bottom:auto}.dropdown-divider{height:0;margin:4px 0;overflow:hidden;border-top:1px solid #e9ecef}.dropdown-item{display:block;width:100%;padding:.25rem 1.5rem;clear:both;font-weight:400;color:#495057;text-align:inherit;white-space:nowrap;background-color:transparent;border:0}.dropdown-item:focus,.dropdown-item:hover{color:#3d4349;text-decoration:none;background-color:#f1f6f9}.dropdown-item.active,.dropdown-item:active{color:#fff;text-decoration:none;background-color:#fc0}.dropdown-item.disabled,.dropdown-item:disabled{color:#6c757d;pointer-events:none;background-color:transparent}.dropdown-menu.show{display:block}.dropdown-header{display:block;padding:.5rem 1.5rem;margin-bottom:0;font-size:.875rem;color:#6c757d;white-space:nowrap}.dropdown-item-text{display:block;padding:.25rem 1.5rem;color:#495057}.btn-group,.btn-group-vertical{position:relative;display:inline-flex;vertical-align:middle}.btn-group-vertical>.btn,.btn-group>.btn{position:relative;flex:1 1 auto}.btn-group-vertical>.btn.active,.btn-group-vertical>.btn:active,.btn-group-vertical>.btn:focus,.btn-group-vertical>.btn:hover,.btn-group>.btn.active,.btn-group>.btn:active,.btn-group>.btn:focus,.btn-group>.btn:hover{z-index:1}.btn-toolbar{display:flex;flex-wrap:wrap;justify-content:flex-start}.btn-toolbar .input-group{width:auto}.btn-group>.btn-group:not(:first-child),.btn-group>.btn:not(:first-child){margin-left:-1px}.btn-group>.btn-group:not(:last-child)>.btn,.btn-group>.btn:not(:last-child):not(.dropdown-toggle){border-top-right-radius:0;border-bottom-right-radius:0}.btn-group>.btn-group:not(:first-child)>.btn,.btn-group>.btn:not(:first-child){border-top-left-radius:0;border-bottom-left-radius:0}.dropdown-toggle-split{padding-right:24px;padding-left:24px}.dropdown-toggle-split:after,.dropright .dropdown-toggle-split:after,.dropup .dropdown-toggle-split:after{margin-left:0}.dropleft .dropdown-toggle-split:before{margin-right:0}.btn-group-lg>.btn+.dropdown-toggle-split,.btn-group-sm>.btn+.dropdown-toggle-split,.btn-lg+.dropdown-toggle-split,.btn-sm+.dropdown-toggle-split{padding-right:24px;padding-left:24px}.btn-group-vertical{flex-direction:column;align-items:flex-start;justify-content:center}.btn-group-vertical>.btn,.btn-group-vertical>.btn-group{width:100%}.btn-group-vertical>.btn-group:not(:first-child),.btn-group-vertical>.btn:not(:first-child){margin-top:-1px}.btn-group-vertical>.btn-group:not(:last-child)>.btn,.btn-group-vertical>.btn:not(:last-child):not(.dropdown-toggle){border-bottom-right-radius:0;border-bottom-left-radius:0}.btn-group-vertical>.btn-group:not(:first-child)>.btn,.btn-group-vertical>.btn:not(:first-child){border-top-left-radius:0;border-top-right-radius:0}.btn-group-toggle>.btn,.btn-group-toggle>.btn-group>.btn{margin-bottom:0}.btn-group-toggle>.btn-group>.btn input[type=checkbox],.btn-group-toggle>.btn-group>.btn input[type=radio],.btn-group-toggle>.btn input[type=checkbox],.btn-group-toggle>.btn input[type=radio]{position:absolute;clip:rect(0,0,0,0);pointer-events:none}.input-group{position:relative;display:flex;flex-wrap:wrap;align-items:stretch;width:100%}.input-group>.custom-file,.input-group>.custom-select,.input-group>.form-control,.input-group>.form-control-plaintext{position:relative;flex:1 1 0%;min-width:0;margin-bottom:0}.input-group>.custom-file+.custom-file,.input-group>.custom-file+.custom-select,.input-group>.custom-file+.form-control,.input-group>.custom-select+.custom-file,.input-group>.custom-select+.custom-select,.input-group>.custom-select+.form-control,.input-group>.form-control+.custom-file,.input-group>.form-control+.custom-select,.input-group>.form-control+.form-control,.input-group>.form-control-plaintext+.custom-file,.input-group>.form-control-plaintext+.custom-select,.input-group>.form-control-plaintext+.form-control{margin-left:-1px}.input-group>.custom-file .custom-file-input:focus~.custom-file-label,.input-group>.custom-select:focus,.input-group>.form-control:focus{z-index:3}.input-group>.custom-file .custom-file-input:focus{z-index:4}.input-group>.custom-select:not(:last-child),.input-group>.form-control:not(:last-child){border-top-right-radius:0;border-bottom-right-radius:0}.input-group>.custom-select:not(:first-child),.input-group>.form-control:not(:first-child){border-top-left-radius:0;border-bottom-left-radius:0}.input-group>.custom-file{display:flex;align-items:center}.input-group>.custom-file:not(:last-child) .custom-file-label,.input-group>.custom-file:not(:last-child) .custom-file-label:after{border-top-right-radius:0;border-bottom-right-radius:0}.input-group>.custom-file:not(:first-child) .custom-file-label{border-top-left-radius:0;border-bottom-left-radius:0}.input-group-append,.input-group-prepend{display:flex}.input-group-append .btn,.input-group-prepend .btn{position:relative;z-index:2}.input-group-append .btn:focus,.input-group-prepend .btn:focus{z-index:3}.input-group-append .btn+.btn,.input-group-append .btn+.input-group-text,.input-group-append .input-group-text+.btn,.input-group-append .input-group-text+.input-group-text,.input-group-prepend .btn+.btn,.input-group-prepend .btn+.input-group-text,.input-group-prepend .input-group-text+.btn,.input-group-prepend .input-group-text+.input-group-text{margin-left:-1px}.input-group-prepend{margin-right:-1px}.input-group-append{margin-left:-1px}.input-group-text{display:flex;align-items:center;padding:.375rem .75rem;margin-bottom:0;font-size:1rem;font-weight:400;line-height:1.5;color:#6c757d;text-align:center;white-space:nowrap;background-color:#e9ecef;border:1px solid #ced4da;border-radius:8px}.input-group-text input[type=checkbox],.input-group-text input[type=radio]{margin-top:0}.input-group-lg>.custom-select,.input-group-lg>.form-control:not(textarea){height:calc(1.5em + 1rem + 2px)}.input-group-lg>.custom-select,.input-group-lg>.form-control,.input-group-lg>.input-group-append>.btn,.input-group-lg>.input-group-append>.input-group-text,.input-group-lg>.input-group-prepend>.btn,.input-group-lg>.input-group-prepend>.input-group-text{padding:.5rem 1rem;font-size:1.125rem;line-height:1.5;border-radius:8px}.input-group-sm>.custom-select,.input-group-sm>.form-control:not(textarea){height:calc(1.5em + .5rem + 2px)}.input-group-sm>.custom-select,.input-group-sm>.form-control,.input-group-sm>.input-group-append>.btn,.input-group-sm>.input-group-append>.input-group-text,.input-group-sm>.input-group-prepend>.btn,.input-group-sm>.input-group-prepend>.input-group-text{padding:.25rem .5rem;font-size:.875rem;line-height:1.5;border-radius:8px}.input-group-lg>.custom-select,.input-group-sm>.custom-select{padding-right:1.75rem}.input-group>.input-group-append:last-child>.btn:not(:last-child):not(.dropdown-toggle),.input-group>.input-group-append:last-child>.input-group-text:not(:last-child),.input-group>.input-group-append:not(:last-child)>.btn,.input-group>.input-group-append:not(:last-child)>.input-group-text,.input-group>.input-group-prepend>.btn,.input-group>.input-group-prepend>.input-group-text{border-top-right-radius:0;border-bottom-right-radius:0}.input-group>.input-group-append>.btn,.input-group>.input-group-append>.input-group-text,.input-group>.input-group-prepend:first-child>.btn:not(:first-child),.input-group>.input-group-prepend:first-child>.input-group-text:not(:first-child),.input-group>.input-group-prepend:not(:first-child)>.btn,.input-group>.input-group-prepend:not(:first-child)>.input-group-text{border-top-left-radius:0;border-bottom-left-radius:0}.custom-control{position:relative;display:block;min-height:1.5rem;padding-left:1.5rem}.custom-control-inline{display:inline-flex;margin-right:1rem}.custom-control-input{position:absolute;left:0;z-index:-1;width:1rem;height:1.25rem;opacity:0}.custom-control-input:checked~.custom-control-label:before{color:#fff;border-color:#fc0;background-color:#fc0}.custom-control-input:focus~.custom-control-label:before{box-shadow:0 0 0 .2rem rgba(255,204,0,.25)}.custom-control-input:focus:not(:checked)~.custom-control-label:before{border-color:#ffe680}.custom-control-input:not(:disabled):active~.custom-control-label:before{color:#fff;background-color:#fff0b3;border-color:#fff0b3}.custom-control-input:disabled~.custom-control-label,.custom-control-input[disabled]~.custom-control-label{color:#6c757d}.custom-control-input:disabled~.custom-control-label:before,.custom-control-input[disabled]~.custom-control-label:before{background-color:#e9ecef}.custom-control-label{position:relative;margin-bottom:0;vertical-align:top}.custom-control-label:before{pointer-events:none;background-color:#fff;border:1px solid #d6dbdf}.custom-control-label:after,.custom-control-label:before{position:absolute;top:.25rem;left:-1.5rem;display:block;width:1rem;height:1rem;content:""}.custom-control-label:after{background:no-repeat 50%/50% 50%}.custom-checkbox .custom-control-label:before{border-radius:8px}.custom-checkbox .custom-control-input:checked~.custom-control-label:after{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='8' height='8'%3E%3Cpath fill='%23fff' d='M6.564.75l-3.59 3.612-1.538-1.55L0 4.26l2.974 2.99L8 2.193z'/%3E%3C/svg%3E")}.custom-checkbox .custom-control-input:indeterminate~.custom-control-label:before{border-color:#fc0;background-color:#fc0}.custom-checkbox .custom-control-input:indeterminate~.custom-control-label:after{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='4' height='4'%3E%3Cpath stroke='%23fff' d='M0 2h4'/%3E%3C/svg%3E")}.custom-checkbox .custom-control-input:disabled:checked~.custom-control-label:before{background-color:rgba(255,204,0,.5)}.custom-checkbox .custom-control-input:disabled:indeterminate~.custom-control-label:before{background-color:rgba(255,204,0,.5)}.custom-radio .custom-control-label:before{border-radius:50%}.custom-radio .custom-control-input:checked~.custom-control-label:after{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='12' height='12' viewBox='-4 -4 8 8'%3E%3Ccircle r='3' fill='%23fff'/%3E%3C/svg%3E")}.custom-radio .custom-control-input:disabled:checked~.custom-control-label:before{background-color:rgba(255,204,0,.5)}.custom-switch{padding-left:2.25rem}.custom-switch .custom-control-label:before{left:-2.25rem;width:1.75rem;pointer-events:all;border-radius:.5rem}.custom-switch .custom-control-label:after{top:calc(.25rem + 2px);left:calc(-2.25rem + 2px);width:calc(1rem - 4px);height:calc(1rem - 4px);background-color:#d6dbdf;border-radius:.5rem;transition:transform .15s ease-in-out,background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out}@media(prefers-reduced-motion:reduce){.custom-switch .custom-control-label:after{transition:none}}.custom-switch .custom-control-input:checked~.custom-control-label:after{background-color:#fff;transform:translateX(.75rem)}.custom-switch .custom-control-input:disabled:checked~.custom-control-label:before{background-color:rgba(255,204,0,.5)}.custom-select{display:inline-block;width:100%;height:calc(1.5em + .75rem + 2px);padding:.375rem 1.75rem .375rem .75rem;font-size:1rem;font-weight:400;line-height:1.5;color:#6c757d;vertical-align:middle;background:#fff url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='4' height='5'%3E%3Cpath fill='%23343a40' d='M2 0L0 2h4zm0 5L0 3h4z'/%3E%3C/svg%3E") no-repeat right .75rem center/8px 10px;border:1px solid #ced4da;border-radius:8px;-webkit-appearance:none;-moz-appearance:none;appearance:none}.custom-select:focus{border-color:#ffe680;outline:0;box-shadow:0 0 0 .2rem rgba(255,204,0,.25)}.custom-select:focus::-ms-value{color:#6c757d;background-color:#fff}.custom-select[multiple],.custom-select[size]:not([size="1"]){height:auto;padding-right:.75rem;background-image:none}.custom-select:disabled{color:#6c757d;background-color:#e9ecef}.custom-select::-ms-expand{display:none}.custom-select:-moz-focusring{color:transparent;text-shadow:0 0 0 #6c757d}.custom-select-sm{height:calc(1.5em + .5rem + 2px);padding-top:.25rem;padding-bottom:.25rem;padding-left:.5rem;font-size:.875rem}.custom-select-lg{height:calc(1.5em + 1rem + 2px);padding-top:.5rem;padding-bottom:.5rem;padding-left:1rem;font-size:1.125rem}.custom-file{display:inline-block;margin-bottom:0}.custom-file,.custom-file-input{position:relative;width:100%;height:calc(1.5em + .75rem + 2px)}.custom-file-input{z-index:2;margin:0;opacity:0}.custom-file-input:focus~.custom-file-label{border-color:#ffe680;box-shadow:0 0 0 .2rem rgba(255,204,0,.25)}.custom-file-input:disabled~.custom-file-label,.custom-file-input[disabled]~.custom-file-label{background-color:#e9ecef}.custom-file-input:lang(en)~.custom-file-label:after{content:"Browse"}.custom-file-input~.custom-file-label[data-browse]:after{content:attr(data-browse)}.custom-file-label{left:0;z-index:1;height:calc(1.5em + .75rem + 2px);font-weight:400;background-color:#fff;border:1px solid #ced4da;border-radius:8px}.custom-file-label,.custom-file-label:after{position:absolute;top:0;right:0;padding:.375rem .75rem;line-height:1.5;color:#6c757d}.custom-file-label:after{bottom:0;z-index:3;display:block;height:calc(1.5em + .75rem);content:"Browse";background-color:#e9ecef;border-left:inherit;border-radius:0 8px 8px 0}.custom-range{width:100%;height:1.4rem;padding:0;background-color:transparent;-webkit-appearance:none;-moz-appearance:none;appearance:none}.custom-range:focus{outline:none}.custom-range:focus::-webkit-slider-thumb{box-shadow:0 0 0 1px #fff,0 0 0 .2rem rgba(255,204,0,.25)}.custom-range:focus::-moz-range-thumb{box-shadow:0 0 0 1px #fff,0 0 0 .2rem rgba(255,204,0,.25)}.custom-range:focus::-ms-thumb{box-shadow:0 0 0 1px #fff,0 0 0 .2rem rgba(255,204,0,.25)}.custom-range::-moz-focus-outer{border:0}.custom-range::-webkit-slider-thumb{width:1rem;height:1rem;margin-top:-.25rem;background-color:#fc0;border:0;border-radius:1rem;-webkit-transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;-webkit-appearance:none;appearance:none}@media(prefers-reduced-motion:reduce){.custom-range::-webkit-slider-thumb{-webkit-transition:none;transition:none}}.custom-range::-webkit-slider-thumb:active{background-color:#fff0b3}.custom-range::-webkit-slider-runnable-track{width:100%;height:.5rem;color:transparent;cursor:pointer;background-color:#dee2e6;border-color:transparent;border-radius:1rem}.custom-range::-moz-range-thumb{width:1rem;height:1rem;background-color:#fc0;border:0;border-radius:1rem;-moz-transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;-moz-appearance:none;appearance:none}@media(prefers-reduced-motion:reduce){.custom-range::-moz-range-thumb{-moz-transition:none;transition:none}}.custom-range::-moz-range-thumb:active{background-color:#fff0b3}.custom-range::-moz-range-track{width:100%;height:.5rem;color:transparent;cursor:pointer;background-color:#dee2e6;border-color:transparent;border-radius:1rem}.custom-range::-ms-thumb{width:1rem;height:1rem;margin-top:0;margin-right:.2rem;margin-left:.2rem;background-color:#fc0;border:0;border-radius:1rem;-ms-transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;appearance:none}@media(prefers-reduced-motion:reduce){.custom-range::-ms-thumb{-ms-transition:none;transition:none}}.custom-range::-ms-thumb:active{background-color:#fff0b3}.custom-range::-ms-track{width:100%;height:.5rem;color:transparent;cursor:pointer;background-color:transparent;border-color:transparent;border-width:.5rem}.custom-range::-ms-fill-lower,.custom-range::-ms-fill-upper{background-color:#dee2e6;border-radius:1rem}.custom-range::-ms-fill-upper{margin-right:15px}.custom-range:disabled::-webkit-slider-thumb{background-color:#d6dbdf}.custom-range:disabled::-webkit-slider-runnable-track{cursor:default}.custom-range:disabled::-moz-range-thumb{background-color:#d6dbdf}.custom-range:disabled::-moz-range-track{cursor:default}.custom-range:disabled::-ms-thumb{background-color:#d6dbdf}.custom-control-label:before,.custom-file-label,.custom-select{transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out}@media(prefers-reduced-motion:reduce){.custom-control-label:before,.custom-file-label,.custom-select{transition:none}}.nav{display:flex;flex-wrap:wrap;padding-left:0;margin-bottom:0;list-style:none}.nav-link{display:block;padding:0}.nav-link:focus,.nav-link:hover{text-decoration:none}.nav-link.disabled{color:#d6dbdf;pointer-events:none;cursor:default}.nav-tabs{border-bottom:1px solid #6c757d}.nav-tabs .nav-item{margin-bottom:-1px}.nav-tabs .nav-link{border:1px solid transparent;border-top-left-radius:8px;border-top-right-radius:8px}.nav-tabs .nav-link:focus,.nav-tabs .nav-link:hover{border-color:transparent}.nav-tabs .nav-link.disabled{color:#d6dbdf;background-color:transparent;border-color:transparent}.nav-tabs .nav-item.show .nav-link,.nav-tabs .nav-link.active{color:#257af4;background-color:#fff;border-color:#6c757d}.nav-tabs .dropdown-menu{margin-top:-1px;border-top-left-radius:0;border-top-right-radius:0}.nav-pills .nav-link{border-radius:8px}.nav-pills .nav-link.active,.nav-pills .show>.nav-link{color:#fff;background-color:#fc0}.nav-fill .nav-item{flex:1 1 auto;text-align:center}.nav-justified .nav-item{flex-basis:0;flex-grow:1;text-align:center}.tab-content>.tab-pane{display:none}.tab-content>.active{display:block}.navbar{position:relative;padding:24px 0}.navbar,.navbar .container,.navbar .container-fluid,.navbar .container-lg,.navbar .container-md,.navbar .container-sm,.navbar .container-xl,.navbar .container-xs{display:flex;flex-wrap:wrap;align-items:center;justify-content:space-between}.navbar-brand{display:inline-block;padding-top:-.09375rem;padding-bottom:-.09375rem;margin-right:0;font-size:1.125rem;line-height:inherit;white-space:nowrap}.navbar-brand:focus,.navbar-brand:hover{text-decoration:none}.navbar-nav{display:flex;flex-direction:column;padding-left:0;margin-bottom:0;list-style:none}.navbar-nav .nav-link{padding-right:0;padding-left:0}.navbar-nav .dropdown-menu{position:static;float:none}.navbar-text{display:inline-block;padding-top:0;padding-bottom:0}.navbar-collapse{flex-basis:100%;flex-grow:1;align-items:center}.navbar-toggler{padding:.25rem .75rem;font-size:1.125rem;line-height:1;background-color:transparent;border:1px solid transparent;border-radius:8px}.navbar-toggler:focus,.navbar-toggler:hover{text-decoration:none}.navbar-toggler-icon{display:inline-block;width:1.5em;height:1.5em;vertical-align:middle;content:"";background:no-repeat 50%;background-size:100% 100%}@media(max-width:399.98px){.navbar-expand-xs>.container,.navbar-expand-xs>.container-fluid,.navbar-expand-xs>.container-lg,.navbar-expand-xs>.container-md,.navbar-expand-xs>.container-sm,.navbar-expand-xs>.container-xl,.navbar-expand-xs>.container-xs{padding-right:0;padding-left:0}}@media(min-width:400px){.navbar-expand-xs{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand-xs .navbar-nav{flex-direction:row}.navbar-expand-xs .navbar-nav .dropdown-menu{position:absolute}.navbar-expand-xs .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand-xs>.container,.navbar-expand-xs>.container-fluid,.navbar-expand-xs>.container-lg,.navbar-expand-xs>.container-md,.navbar-expand-xs>.container-sm,.navbar-expand-xs>.container-xl,.navbar-expand-xs>.container-xs{flex-wrap:nowrap}.navbar-expand-xs .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand-xs .navbar-toggler{display:none}}@media(max-width:615.98px){.navbar-expand-sm>.container,.navbar-expand-sm>.container-fluid,.navbar-expand-sm>.container-lg,.navbar-expand-sm>.container-md,.navbar-expand-sm>.container-sm,.navbar-expand-sm>.container-xl,.navbar-expand-sm>.container-xs{padding-right:0;padding-left:0}}@media(min-width:616px){.navbar-expand-sm{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand-sm .navbar-nav{flex-direction:row}.navbar-expand-sm .navbar-nav .dropdown-menu{position:absolute}.navbar-expand-sm .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand-sm>.container,.navbar-expand-sm>.container-fluid,.navbar-expand-sm>.container-lg,.navbar-expand-sm>.container-md,.navbar-expand-sm>.container-sm,.navbar-expand-sm>.container-xl,.navbar-expand-sm>.container-xs{flex-wrap:nowrap}.navbar-expand-sm .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand-sm .navbar-toggler{display:none}}@media(max-width:767.98px){.navbar-expand-md>.container,.navbar-expand-md>.container-fluid,.navbar-expand-md>.container-lg,.navbar-expand-md>.container-md,.navbar-expand-md>.container-sm,.navbar-expand-md>.container-xl,.navbar-expand-md>.container-xs{padding-right:0;padding-left:0}}@media(min-width:768px){.navbar-expand-md{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand-md .navbar-nav{flex-direction:row}.navbar-expand-md .navbar-nav .dropdown-menu{position:absolute}.navbar-expand-md .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand-md>.container,.navbar-expand-md>.container-fluid,.navbar-expand-md>.container-lg,.navbar-expand-md>.container-md,.navbar-expand-md>.container-sm,.navbar-expand-md>.container-xl,.navbar-expand-md>.container-xs{flex-wrap:nowrap}.navbar-expand-md .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand-md .navbar-toggler{display:none}}@media(max-width:979.98px){.navbar-expand-lg>.container,.navbar-expand-lg>.container-fluid,.navbar-expand-lg>.container-lg,.navbar-expand-lg>.container-md,.navbar-expand-lg>.container-sm,.navbar-expand-lg>.container-xl,.navbar-expand-lg>.container-xs{padding-right:0;padding-left:0}}@media(min-width:980px){.navbar-expand-lg{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand-lg .navbar-nav{flex-direction:row}.navbar-expand-lg .navbar-nav .dropdown-menu{position:absolute}.navbar-expand-lg .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand-lg>.container,.navbar-expand-lg>.container-fluid,.navbar-expand-lg>.container-lg,.navbar-expand-lg>.container-md,.navbar-expand-lg>.container-sm,.navbar-expand-lg>.container-xl,.navbar-expand-lg>.container-xs{flex-wrap:nowrap}.navbar-expand-lg .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand-lg .navbar-toggler{display:none}}@media(max-width:1239.98px){.navbar-expand-xl>.container,.navbar-expand-xl>.container-fluid,.navbar-expand-xl>.container-lg,.navbar-expand-xl>.container-md,.navbar-expand-xl>.container-sm,.navbar-expand-xl>.container-xl,.navbar-expand-xl>.container-xs{padding-right:0;padding-left:0}}@media(min-width:1240px){.navbar-expand-xl{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand-xl .navbar-nav{flex-direction:row}.navbar-expand-xl .navbar-nav .dropdown-menu{position:absolute}.navbar-expand-xl .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand-xl>.container,.navbar-expand-xl>.container-fluid,.navbar-expand-xl>.container-lg,.navbar-expand-xl>.container-md,.navbar-expand-xl>.container-sm,.navbar-expand-xl>.container-xl,.navbar-expand-xl>.container-xs{flex-wrap:nowrap}.navbar-expand-xl .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand-xl .navbar-toggler{display:none}}.navbar-expand{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand>.container,.navbar-expand>.container-fluid,.navbar-expand>.container-lg,.navbar-expand>.container-md,.navbar-expand>.container-sm,.navbar-expand>.container-xl,.navbar-expand>.container-xs{padding-right:0;padding-left:0}.navbar-expand .navbar-nav{flex-direction:row}.navbar-expand .navbar-nav .dropdown-menu{position:absolute}.navbar-expand .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand>.container,.navbar-expand>.container-fluid,.navbar-expand>.container-lg,.navbar-expand>.container-md,.navbar-expand>.container-sm,.navbar-expand>.container-xl,.navbar-expand>.container-xs{flex-wrap:nowrap}.navbar-expand .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand .navbar-toggler{display:none}.navbar-light .navbar-brand,.navbar-light .navbar-brand:focus,.navbar-light .navbar-brand:hover{color:rgba(33,37,41,.9)}.navbar-light .navbar-nav .nav-link{color:rgba(33,37,41,.5)}.navbar-light .navbar-nav .nav-link:focus,.navbar-light .navbar-nav .nav-link:hover{color:rgba(33,37,41,.7)}.navbar-light .navbar-nav .nav-link.disabled{color:rgba(33,37,41,.3)}.navbar-light .navbar-nav .active>.nav-link,.navbar-light .navbar-nav .nav-link.active,.navbar-light .navbar-nav .nav-link.show,.navbar-light .navbar-nav .show>.nav-link{color:rgba(33,37,41,.9)}.navbar-light .navbar-toggler{color:rgba(33,37,41,.5);border-color:rgba(33,37,41,.1)}.navbar-light .navbar-toggler-icon{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='30' height='30'%3E%3Cpath stroke='rgba(33, 37, 41, 0.5)' stroke-linecap='round' stroke-miterlimit='10' stroke-width='2' d='M4 7h22M4 15h22M4 23h22'/%3E%3C/svg%3E")}.navbar-light .navbar-text{color:rgba(33,37,41,.5)}.navbar-light .navbar-text a,.navbar-light .navbar-text a:focus,.navbar-light .navbar-text a:hover{color:rgba(33,37,41,.9)}.navbar-dark .navbar-brand,.navbar-dark .navbar-brand:focus,.navbar-dark .navbar-brand:hover{color:#fff}.navbar-dark .navbar-nav .nav-link{color:hsla(0,0%,100%,.5)}.navbar-dark .navbar-nav .nav-link:focus,.navbar-dark .navbar-nav .nav-link:hover{color:hsla(0,0%,100%,.75)}.navbar-dark .navbar-nav .nav-link.disabled{color:hsla(0,0%,100%,.25)}.navbar-dark .navbar-nav .active>.nav-link,.navbar-dark .navbar-nav .nav-link.active,.navbar-dark .navbar-nav .nav-link.show,.navbar-dark .navbar-nav .show>.nav-link{color:#fff}.navbar-dark .navbar-toggler{color:hsla(0,0%,100%,.5);border-color:hsla(0,0%,100%,.1)}.navbar-dark .navbar-toggler-icon{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='30' height='30'%3E%3Cpath stroke='rgba(255, 255, 255, 0.5)' stroke-linecap='round' stroke-miterlimit='10' stroke-width='2' d='M4 7h22M4 15h22M4 23h22'/%3E%3C/svg%3E")}.navbar-dark .navbar-text{color:hsla(0,0%,100%,.5)}.navbar-dark .navbar-text a,.navbar-dark .navbar-text a:focus,.navbar-dark .navbar-text a:hover{color:#fff}.card{position:relative;display:flex;flex-direction:column;min-width:0;word-wrap:break-word;background-color:#fff;background-clip:border-box;border:1px solid #d6dbdf;border-radius:8px}.card>hr{margin-right:0;margin-left:0}.card>.list-group:first-child .list-group-item:first-child{border-top-left-radius:8px;border-top-right-radius:8px}.card>.list-group:last-child .list-group-item:last-child{border-bottom-right-radius:8px;border-bottom-left-radius:8px}.card-body{flex:1 1 auto;min-height:1px;padding:24px}.card-title{margin-bottom:24px}.card-subtitle{margin-top:-12px}.card-subtitle,.card-text:last-child{margin-bottom:0}.card-link:hover{text-decoration:none}.card-link+.card-link{margin-left:24px}.card-header{padding:24px;margin-bottom:0;background-color:#f1f6f9;border-bottom:1px solid #d6dbdf}.card-header:first-child{border-radius:subtract(8px,1px) subtract(8px,1px) 0 0}.card-header+.list-group .list-group-item:first-child{border-top:0}.card-footer{padding:24px;background-color:#f1f6f9;border-top:1px solid #d6dbdf}.card-footer:last-child{border-radius:0 0 subtract(8px,1px) subtract(8px,1px)}.card-header-tabs{margin-bottom:-24px;border-bottom:0}.card-header-pills,.card-header-tabs{margin-right:-12px;margin-left:-12px}.card-img-overlay{position:absolute;top:0;right:0;bottom:0;left:0;padding:24px}.card-img,.card-img-bottom,.card-img-top{flex-shrink:0;width:100%}.card-img,.card-img-top{border-top-left-radius:subtract(8px,1px);border-top-right-radius:subtract(8px,1px)}.card-img,.card-img-bottom{border-bottom-right-radius:subtract(8px,1px);border-bottom-left-radius:subtract(8px,1px)}.card-deck .card{margin-bottom:20px}@media(min-width:616px){.card-deck{display:flex;flex-flow:row wrap;margin-right:-20px;margin-left:-20px}.card-deck .card{flex:1 0 0%;margin-right:20px;margin-bottom:0;margin-left:20px}}.card-group>.card{margin-bottom:20px}@media(min-width:616px){.card-group{display:flex;flex-flow:row wrap}.card-group>.card{flex:1 0 0%;margin-bottom:0}.card-group>.card+.card{margin-left:0;border-left:0}.card-group>.card:not(:last-child){border-top-right-radius:0;border-bottom-right-radius:0}.card-group>.card:not(:last-child) .card-header,.card-group>.card:not(:last-child) .card-img-top{border-top-right-radius:0}.card-group>.card:not(:last-child) .card-footer,.card-group>.card:not(:last-child) .card-img-bottom{border-bottom-right-radius:0}.card-group>.card:not(:first-child){border-top-left-radius:0;border-bottom-left-radius:0}.card-group>.card:not(:first-child) .card-header,.card-group>.card:not(:first-child) .card-img-top{border-top-left-radius:0}.card-group>.card:not(:first-child) .card-footer,.card-group>.card:not(:first-child) .card-img-bottom{border-bottom-left-radius:0}}.card-columns .card{margin-bottom:40px}@media(min-width:616px){.card-columns{-moz-column-count:3;column-count:3;-moz-column-gap:40px;column-gap:40px;orphans:1;widows:1}.card-columns .card{display:inline-block;width:100%}}.accordion>.card{overflow:hidden}.accordion>.card:not(:last-of-type){border-bottom:0;border-bottom-right-radius:0;border-bottom-left-radius:0}.accordion>.card:not(:first-of-type){border-top-left-radius:0;border-top-right-radius:0}.accordion>.card>.card-header{border-radius:0;margin-bottom:-1px}.breadcrumb{display:flex;flex-wrap:wrap;padding:.75rem 1rem;margin-bottom:1rem;list-style:none;background-color:#e9ecef;border-radius:8px}.breadcrumb-item+.breadcrumb-item{padding-left:.5rem}.breadcrumb-item+.breadcrumb-item:before{display:inline-block;padding-right:.5rem;color:#6c757d;content:"/"}.breadcrumb-item+.breadcrumb-item:hover:before{text-decoration:underline;text-decoration:none}.breadcrumb-item.active{color:#6c757d}.pagination{display:flex;padding-left:0;list-style:none;border-radius:8px}.page-link{position:relative;display:block;padding:.5rem .75rem;margin-left:-1px;line-height:1.25;color:#ff8c00;background-color:#fff;border:1px solid #dee2e6}.page-link:hover{z-index:2;color:#ff8c00;text-decoration:none;background-color:#e9ecef;border-color:#dee2e6}.page-link:focus{z-index:3;outline:0;box-shadow:0 0 0 .2rem rgba(255,204,0,.25)}.page-item:first-child .page-link{margin-left:0;border-top-left-radius:8px;border-bottom-left-radius:8px}.page-item:last-child .page-link{border-top-right-radius:8px;border-bottom-right-radius:8px}.page-item.active .page-link{z-index:3;color:#fff;background-color:#fc0;border-color:#fc0}.page-item.disabled .page-link{color:#6c757d;pointer-events:none;cursor:auto;background-color:#fff;border-color:#dee2e6}.pagination-lg .page-link{padding:.75rem 1.5rem;font-size:1.125rem;line-height:1.5}.pagination-lg .page-item:first-child .page-link{border-top-left-radius:8px;border-bottom-left-radius:8px}.pagination-lg .page-item:last-child .page-link{border-top-right-radius:8px;border-bottom-right-radius:8px}.pagination-sm .page-link{padding:.25rem .5rem;font-size:.875rem;line-height:1.5}.pagination-sm .page-item:first-child .page-link{border-top-left-radius:8px;border-bottom-left-radius:8px}.pagination-sm .page-item:last-child .page-link{border-top-right-radius:8px;border-bottom-right-radius:8px}.badge{display:inline-block;padding:.25em .4em;font-size:75%;font-weight:700;line-height:1;text-align:center;white-space:nowrap;vertical-align:baseline;border-radius:8px;transition:color .15s ease-in-out,background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out}@media(prefers-reduced-motion:reduce){.badge{transition:none}}a.badge:focus,a.badge:hover{text-decoration:none}.badge:empty{display:none}.btn .badge{position:relative;top:-1px}.badge-pill{padding-right:.6em;padding-left:.6em;border-radius:10rem}.badge-primary{color:#495057;background-color:#fc0}a.badge-primary:focus,a.badge-primary:hover{color:#495057;background-color:#cca300}a.badge-primary.focus,a.badge-primary:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,204,0,.5)}.badge-secondary{color:#fff;background-color:#212529}a.badge-secondary:focus,a.badge-secondary:hover{color:#fff;background-color:#0a0c0d}a.badge-secondary.focus,a.badge-secondary:focus{outline:0;box-shadow:0 0 0 .2rem rgba(33,37,41,.5)}.badge-success{color:#fff;background-color:#28a745}a.badge-success:focus,a.badge-success:hover{color:#fff;background-color:#1e7e34}a.badge-success.focus,a.badge-success:focus{outline:0;box-shadow:0 0 0 .2rem rgba(40,167,69,.5)}.badge-info{color:#fff;background-color:#17a2b8}a.badge-info:focus,a.badge-info:hover{color:#fff;background-color:#117a8b}a.badge-info.focus,a.badge-info:focus{outline:0;box-shadow:0 0 0 .2rem rgba(23,162,184,.5)}.badge-warning{color:#495057;background-color:#ffc107}a.badge-warning:focus,a.badge-warning:hover{color:#495057;background-color:#d39e00}a.badge-warning.focus,a.badge-warning:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,193,7,.5)}.badge-danger{color:#fff;background-color:#dc3545}a.badge-danger:focus,a.badge-danger:hover{color:#fff;background-color:#bd2130}a.badge-danger.focus,a.badge-danger:focus{outline:0;box-shadow:0 0 0 .2rem rgba(220,53,69,.5)}.badge-light{color:#495057;background-color:#f1f6f9}a.badge-light:focus,a.badge-light:hover{color:#495057;background-color:#cddfea}a.badge-light.focus,a.badge-light:focus{outline:0;box-shadow:0 0 0 .2rem rgba(241,246,249,.5)}.badge-dark{color:#fff;background-color:#495057}a.badge-dark:focus,a.badge-dark:hover{color:#fff;background-color:#32373b}a.badge-dark.focus,a.badge-dark:focus{outline:0;box-shadow:0 0 0 .2rem rgba(73,80,87,.5)}.badge-primary-light{color:#495057;background-color:#fffaf0}a.badge-primary-light:focus,a.badge-primary-light:hover{color:#495057;background-color:#ffe9bd}a.badge-primary-light.focus,a.badge-primary-light:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,250,240,.5)}.badge-secondary-light{color:#495057;background-color:#fff}a.badge-secondary-light:focus,a.badge-secondary-light:hover{color:#495057;background-color:#e6e6e6}a.badge-secondary-light.focus,a.badge-secondary-light:focus{outline:0;box-shadow:0 0 0 .2rem hsla(0,0%,100%,.5)}.badge-tertiary{color:#fff;background-color:#257af4}a.badge-tertiary:focus,a.badge-tertiary:hover{color:#fff;background-color:#0b60db}a.badge-tertiary.focus,a.badge-tertiary:focus{outline:0;box-shadow:0 0 0 .2rem rgba(37,122,244,.5)}.badge-tertiary-light{color:#495057;background-color:#e3f1fe}a.badge-tertiary-light:focus,a.badge-tertiary-light:hover{color:#495057;background-color:#b2d8fc}a.badge-tertiary-light.focus,a.badge-tertiary-light:focus{outline:0;box-shadow:0 0 0 .2rem rgba(227,241,254,.5)}.badge-white{color:#495057;background-color:#fff}a.badge-white:focus,a.badge-white:hover{color:#495057;background-color:#e6e6e6}a.badge-white.focus,a.badge-white:focus{outline:0;box-shadow:0 0 0 .2rem hsla(0,0%,100%,.5)}.badge-black{color:#fff;background-color:#212529}a.badge-black:focus,a.badge-black:hover{color:#fff;background-color:#0a0c0d}a.badge-black.focus,a.badge-black:focus{outline:0;box-shadow:0 0 0 .2rem rgba(33,37,41,.5)}.badge-blue{color:#fff;background-color:#257af4}a.badge-blue:focus,a.badge-blue:hover{color:#fff;background-color:#0b60db}a.badge-blue.focus,a.badge-blue:focus{outline:0;box-shadow:0 0 0 .2rem rgba(37,122,244,.5)}.badge-light-blue{color:#495057;background-color:#e3f1fe}a.badge-light-blue:focus,a.badge-light-blue:hover{color:#495057;background-color:#b2d8fc}a.badge-light-blue.focus,a.badge-light-blue:focus{outline:0;box-shadow:0 0 0 .2rem rgba(227,241,254,.5)}.badge-yellow{color:#495057;background-color:#fc0}a.badge-yellow:focus,a.badge-yellow:hover{color:#495057;background-color:#cca300}a.badge-yellow.focus,a.badge-yellow:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,204,0,.5)}.badge-light-yellow{color:#495057;background-color:#fffaf0}a.badge-light-yellow:focus,a.badge-light-yellow:hover{color:#495057;background-color:#ffe9bd}a.badge-light-yellow.focus,a.badge-light-yellow:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,250,240,.5)}.badge-orange{color:#495057;background-color:#ff8c00}a.badge-orange:focus,a.badge-orange:hover{color:#495057;background-color:#cc7000}a.badge-orange.focus,a.badge-orange:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,140,0,.5)}.badge-light-orange{color:#495057;background-color:#ffe4b5}a.badge-light-orange:focus,a.badge-light-orange:hover{color:#495057;background-color:#ffd182}a.badge-light-orange.focus,a.badge-light-orange:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,228,181,.5)}.badge-red{color:#fff;background-color:#ff3939}a.badge-red:focus,a.badge-red:hover{color:#fff;background-color:#ff0606}a.badge-red.focus,a.badge-red:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,57,57,.5)}.badge-light-red{color:#495057;background-color:#ffe4e1}a.badge-light-red:focus,a.badge-light-red:hover{color:#495057;background-color:#ffb6ae}a.badge-light-red.focus,a.badge-light-red:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,228,225,.5)}.badge-medium{color:#495057;background-color:#d6dbdf}a.badge-medium:focus,a.badge-medium:hover{color:#495057;background-color:#b9c2c9}a.badge-medium.focus,a.badge-medium:focus{outline:0;box-shadow:0 0 0 .2rem rgba(214,219,223,.5)}.jumbotron{padding:2rem 1rem;margin-bottom:2rem;background-color:#e9ecef;border-radius:8px}@media(min-width:616px){.jumbotron{padding:4rem 2rem}}.jumbotron-fluid{padding-right:0;padding-left:0;border-radius:0}.alert{position:relative;padding:.75rem 1.25rem;margin-bottom:1rem;border:1px solid transparent;border-radius:8px}.alert-heading{color:inherit}.alert-link{font-weight:700}.alert-dismissible{padding-right:4rem}.alert-dismissible .close{position:absolute;top:0;right:0;padding:.75rem 1.25rem;color:inherit}.alert-primary{color:#947c14;background-color:#fff5cc;border-color:#fff1b8}.alert-primary hr{border-top-color:#ffec9f}.alert-primary .alert-link{color:#67560e}.alert-secondary{color:#212529;background-color:#d3d3d4;border-color:#c1c2c3}.alert-secondary hr{border-top-color:#b4b5b6}.alert-secondary .alert-link{color:#0a0c0d}.alert-success{color:#256938;background-color:#d4edda;border-color:#c3e6cb}.alert-success hr{border-top-color:#b1dfbb}.alert-success .alert-link{color:#184324}.alert-info{color:#1c6673;background-color:#d1ecf1;border-color:#bee5eb}.alert-info hr{border-top-color:#abdde5}.alert-info .alert-link{color:#12424a}.alert-warning{color:#947617;background-color:#fff3cd;border-color:#ffeeba}.alert-warning hr{border-top-color:#ffe8a1}.alert-warning .alert-link{color:#685310}.alert-danger{color:#822d38;background-color:#f8d7da;border-color:#f5c6cb}.alert-danger hr{border-top-color:#f1b0b7}.alert-danger .alert-link{color:#5c2028}.alert-light{color:#8d9295;background-color:#fcfdfe;border-color:#fbfcfd}.alert-light hr{border-top-color:#eaeff5}.alert-light .alert-link{color:#73797c}.alert-dark{color:#363b41;background-color:#dbdcdd;border-color:#ccced0}.alert-dark hr{border-top-color:#bfc1c4}.alert-dark .alert-link{color:#1f2225}.alert-primary-light{color:#949490;background-color:#fffefc;border-color:#fffefb}.alert-primary-light hr{border-top-color:#fff8e2}.alert-primary-light .alert-link{color:#7b7b76}.alert-secondary-light{color:#949698;background-color:#fff;border-color:#fff}.alert-secondary-light hr{border-top-color:#f2f2f2}.alert-secondary-light .alert-link{color:#7a7d7f}.alert-tertiary{color:#235193;background-color:#d3e4fd;border-color:#c2dafc}.alert-tertiary hr{border-top-color:#aacbfb}.alert-tertiary .alert-link{color:#193a6a}.alert-tertiary-light{color:#868f98;background-color:#f9fcff;border-color:#f7fbff}.alert-tertiary-light hr{border-top-color:#deeeff}.alert-tertiary-light .alert-link{color:#6c767f}.alert-white{color:#949698;background-color:#fff;border-color:#fff}.alert-white hr{border-top-color:#f2f2f2}.alert-white .alert-link{color:#7a7d7f}.alert-black{color:#212529;background-color:#d3d3d4;border-color:#c1c2c3}.alert-black hr{border-top-color:#b4b5b6}.alert-black .alert-link{color:#0a0c0d}.alert-blue{color:#235193;background-color:#d3e4fd;border-color:#c2dafc}.alert-blue hr{border-top-color:#aacbfb}.alert-blue .alert-link{color:#193a6a}.alert-light-blue{color:#868f98;background-color:#f9fcff;border-color:#f7fbff}.alert-light-blue hr{border-top-color:#deeeff}.alert-light-blue .alert-link{color:#6c767f}.alert-yellow{color:#947c14;background-color:#fff5cc;border-color:#fff1b8}.alert-yellow hr{border-top-color:#ffec9f}.alert-yellow .alert-link{color:#67560e}.alert-light-yellow{color:#949490;background-color:#fffefc;border-color:#fffefb}.alert-light-yellow hr{border-top-color:#fff8e2}.alert-light-yellow .alert-link{color:#7b7b76}.alert-orange{color:#945b14;background-color:#ffe8cc;border-color:#ffdfb8}.alert-orange hr{border-top-color:#ffd49f}.alert-orange .alert-link{color:#673f0e}.alert-light-orange{color:#948872;background-color:#fffaf0;border-color:#fff7ea}.alert-light-orange hr{border-top-color:#ffedd1}.alert-light-orange .alert-link{color:#786e5b}.alert-red{color:#942f31;background-color:#ffd7d7;border-color:#ffc8c8}.alert-red hr{border-top-color:#ffafaf}.alert-red .alert-link{color:#6d2324}.alert-light-red{color:#948889;background-color:#fffaf9;border-color:#fff7f7}.alert-light-red hr{border-top-color:#ffdede}.alert-light-red .alert-link{color:#7b6e6f}.alert-medium{color:#7f8488;background-color:#f7f8f9;border-color:#f4f5f6}.alert-medium hr{border-top-color:#e6e8eb}.alert-medium .alert-link{color:#666a6e}@-webkit-keyframes progress-bar-stripes{0%{background-position:1rem 0}to{background-position:0 0}}@keyframes progress-bar-stripes{0%{background-position:1rem 0}to{background-position:0 0}}.progress{height:1rem;font-size:.75rem;background-color:#e9ecef;border-radius:8px}.progress,.progress-bar{display:flex;overflow:hidden}.progress-bar{flex-direction:column;justify-content:center;color:#fff;text-align:center;white-space:nowrap;background-color:#fc0;transition:width .6s ease}@media(prefers-reduced-motion:reduce){.progress-bar{transition:none}}.progress-bar-striped{background-image:linear-gradient(45deg,hsla(0,0%,100%,.15) 25%,transparent 0,transparent 50%,hsla(0,0%,100%,.15) 0,hsla(0,0%,100%,.15) 75%,transparent 0,transparent);background-size:1rem 1rem}.progress-bar-animated{-webkit-animation:progress-bar-stripes 1s linear infinite;animation:progress-bar-stripes 1s linear infinite}@media(prefers-reduced-motion:reduce){.progress-bar-animated{-webkit-animation:none;animation:none}}.media{display:flex;align-items:flex-start}.media-body{flex:1}.list-group{display:flex;flex-direction:column;padding-left:0;margin-bottom:0}.list-group-item-action{width:100%;color:#6c757d;text-align:inherit}.list-group-item-action:focus,.list-group-item-action:hover{z-index:1;color:#6c757d;text-decoration:none;background-color:#f1f6f9}.list-group-item-action:active{color:#212529;background-color:#e9ecef}.list-group-item{position:relative;display:block;padding:.75rem 1.25rem;background-color:#fff;border:1px solid rgba(33,37,41,.125)}.list-group-item:first-child{border-top-left-radius:8px;border-top-right-radius:8px}.list-group-item:last-child{border-bottom-right-radius:8px;border-bottom-left-radius:8px}.list-group-item.disabled,.list-group-item:disabled{color:#6c757d;pointer-events:none;background-color:#fff}.list-group-item.active{z-index:2;color:#fff;background-color:#fc0;border-color:#fc0}.list-group-item+.list-group-item{border-top-width:0}.list-group-item+.list-group-item.active{margin-top:-1px;border-top-width:1px}.list-group-horizontal{flex-direction:row}.list-group-horizontal .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal .list-group-item.active{margin-top:0}.list-group-horizontal .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}@media(min-width:400px){.list-group-horizontal-xs{flex-direction:row}.list-group-horizontal-xs .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal-xs .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal-xs .list-group-item.active{margin-top:0}.list-group-horizontal-xs .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal-xs .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}}@media(min-width:616px){.list-group-horizontal-sm{flex-direction:row}.list-group-horizontal-sm .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal-sm .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal-sm .list-group-item.active{margin-top:0}.list-group-horizontal-sm .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal-sm .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}}@media(min-width:768px){.list-group-horizontal-md{flex-direction:row}.list-group-horizontal-md .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal-md .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal-md .list-group-item.active{margin-top:0}.list-group-horizontal-md .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal-md .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}}@media(min-width:980px){.list-group-horizontal-lg{flex-direction:row}.list-group-horizontal-lg .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal-lg .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal-lg .list-group-item.active{margin-top:0}.list-group-horizontal-lg .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal-lg .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}}@media(min-width:1240px){.list-group-horizontal-xl{flex-direction:row}.list-group-horizontal-xl .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal-xl .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal-xl .list-group-item.active{margin-top:0}.list-group-horizontal-xl .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal-xl .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}}.list-group-flush .list-group-item{border-right-width:0;border-left-width:0;border-radius:0}.list-group-flush .list-group-item:first-child{border-top-width:0}.list-group-flush:last-child .list-group-item:last-child{border-bottom-width:0}.list-group-item-primary{color:#947c14;background-color:#fff1b8}.list-group-item-primary.list-group-item-action:focus,.list-group-item-primary.list-group-item-action:hover{color:#947c14;background-color:#ffec9f}.list-group-item-primary.list-group-item-action.active{color:#fff;background-color:#947c14;border-color:#947c14}.list-group-item-secondary{color:#212529;background-color:#c1c2c3}.list-group-item-secondary.list-group-item-action:focus,.list-group-item-secondary.list-group-item-action:hover{color:#212529;background-color:#b4b5b6}.list-group-item-secondary.list-group-item-action.active{color:#fff;background-color:#212529;border-color:#212529}.list-group-item-success{color:#256938;background-color:#c3e6cb}.list-group-item-success.list-group-item-action:focus,.list-group-item-success.list-group-item-action:hover{color:#256938;background-color:#b1dfbb}.list-group-item-success.list-group-item-action.active{color:#fff;background-color:#256938;border-color:#256938}.list-group-item-info{color:#1c6673;background-color:#bee5eb}.list-group-item-info.list-group-item-action:focus,.list-group-item-info.list-group-item-action:hover{color:#1c6673;background-color:#abdde5}.list-group-item-info.list-group-item-action.active{color:#fff;background-color:#1c6673;border-color:#1c6673}.list-group-item-warning{color:#947617;background-color:#ffeeba}.list-group-item-warning.list-group-item-action:focus,.list-group-item-warning.list-group-item-action:hover{color:#947617;background-color:#ffe8a1}.list-group-item-warning.list-group-item-action.active{color:#fff;background-color:#947617;border-color:#947617}.list-group-item-danger{color:#822d38;background-color:#f5c6cb}.list-group-item-danger.list-group-item-action:focus,.list-group-item-danger.list-group-item-action:hover{color:#822d38;background-color:#f1b0b7}.list-group-item-danger.list-group-item-action.active{color:#fff;background-color:#822d38;border-color:#822d38}.list-group-item-light{color:#8d9295;background-color:#fbfcfd}.list-group-item-light.list-group-item-action:focus,.list-group-item-light.list-group-item-action:hover{color:#8d9295;background-color:#eaeff5}.list-group-item-light.list-group-item-action.active{color:#fff;background-color:#8d9295;border-color:#8d9295}.list-group-item-dark{color:#363b41;background-color:#ccced0}.list-group-item-dark.list-group-item-action:focus,.list-group-item-dark.list-group-item-action:hover{color:#363b41;background-color:#bfc1c4}.list-group-item-dark.list-group-item-action.active{color:#fff;background-color:#363b41;border-color:#363b41}.list-group-item-primary-light{color:#949490;background-color:#fffefb}.list-group-item-primary-light.list-group-item-action:focus,.list-group-item-primary-light.list-group-item-action:hover{color:#949490;background-color:#fff8e2}.list-group-item-primary-light.list-group-item-action.active{color:#fff;background-color:#949490;border-color:#949490}.list-group-item-secondary-light{color:#949698;background-color:#fff}.list-group-item-secondary-light.list-group-item-action:focus,.list-group-item-secondary-light.list-group-item-action:hover{color:#949698;background-color:#f2f2f2}.list-group-item-secondary-light.list-group-item-action.active{color:#fff;background-color:#949698;border-color:#949698}.list-group-item-tertiary{color:#235193;background-color:#c2dafc}.list-group-item-tertiary.list-group-item-action:focus,.list-group-item-tertiary.list-group-item-action:hover{color:#235193;background-color:#aacbfb}.list-group-item-tertiary.list-group-item-action.active{color:#fff;background-color:#235193;border-color:#235193}.list-group-item-tertiary-light{color:#868f98;background-color:#f7fbff}.list-group-item-tertiary-light.list-group-item-action:focus,.list-group-item-tertiary-light.list-group-item-action:hover{color:#868f98;background-color:#deeeff}.list-group-item-tertiary-light.list-group-item-action.active{color:#fff;background-color:#868f98;border-color:#868f98}.list-group-item-white{color:#949698;background-color:#fff}.list-group-item-white.list-group-item-action:focus,.list-group-item-white.list-group-item-action:hover{color:#949698;background-color:#f2f2f2}.list-group-item-white.list-group-item-action.active{color:#fff;background-color:#949698;border-color:#949698}.list-group-item-black{color:#212529;background-color:#c1c2c3}.list-group-item-black.list-group-item-action:focus,.list-group-item-black.list-group-item-action:hover{color:#212529;background-color:#b4b5b6}.list-group-item-black.list-group-item-action.active{color:#fff;background-color:#212529;border-color:#212529}.list-group-item-blue{color:#235193;background-color:#c2dafc}.list-group-item-blue.list-group-item-action:focus,.list-group-item-blue.list-group-item-action:hover{color:#235193;background-color:#aacbfb}.list-group-item-blue.list-group-item-action.active{color:#fff;background-color:#235193;border-color:#235193}.list-group-item-light-blue{color:#868f98;background-color:#f7fbff}.list-group-item-light-blue.list-group-item-action:focus,.list-group-item-light-blue.list-group-item-action:hover{color:#868f98;background-color:#deeeff}.list-group-item-light-blue.list-group-item-action.active{color:#fff;background-color:#868f98;border-color:#868f98}.list-group-item-yellow{color:#947c14;background-color:#fff1b8}.list-group-item-yellow.list-group-item-action:focus,.list-group-item-yellow.list-group-item-action:hover{color:#947c14;background-color:#ffec9f}.list-group-item-yellow.list-group-item-action.active{color:#fff;background-color:#947c14;border-color:#947c14}.list-group-item-light-yellow{color:#949490;background-color:#fffefb}.list-group-item-light-yellow.list-group-item-action:focus,.list-group-item-light-yellow.list-group-item-action:hover{color:#949490;background-color:#fff8e2}.list-group-item-light-yellow.list-group-item-action.active{color:#fff;background-color:#949490;border-color:#949490}.list-group-item-orange{color:#945b14;background-color:#ffdfb8}.list-group-item-orange.list-group-item-action:focus,.list-group-item-orange.list-group-item-action:hover{color:#945b14;background-color:#ffd49f}.list-group-item-orange.list-group-item-action.active{color:#fff;background-color:#945b14;border-color:#945b14}.list-group-item-light-orange{color:#948872;background-color:#fff7ea}.list-group-item-light-orange.list-group-item-action:focus,.list-group-item-light-orange.list-group-item-action:hover{color:#948872;background-color:#ffedd1}.list-group-item-light-orange.list-group-item-action.active{color:#fff;background-color:#948872;border-color:#948872}.list-group-item-red{color:#942f31;background-color:#ffc8c8}.list-group-item-red.list-group-item-action:focus,.list-group-item-red.list-group-item-action:hover{color:#942f31;background-color:#ffafaf}.list-group-item-red.list-group-item-action.active{color:#fff;background-color:#942f31;border-color:#942f31}.list-group-item-light-red{color:#948889;background-color:#fff7f7}.list-group-item-light-red.list-group-item-action:focus,.list-group-item-light-red.list-group-item-action:hover{color:#948889;background-color:#ffdede}.list-group-item-light-red.list-group-item-action.active{color:#fff;background-color:#948889;border-color:#948889}.list-group-item-medium{color:#7f8488;background-color:#f4f5f6}.list-group-item-medium.list-group-item-action:focus,.list-group-item-medium.list-group-item-action:hover{color:#7f8488;background-color:#e6e8eb}.list-group-item-medium.list-group-item-action.active{color:#fff;background-color:#7f8488;border-color:#7f8488}.close{float:right;font-size:1.5rem;font-weight:700;line-height:1;color:#212529;text-shadow:0 1px 0 #fff;opacity:.5}@media(max-width:1200px){.close{font-size:calc(1.275rem + .3vw)}}.close:hover{color:#212529;text-decoration:none}.close:not(:disabled):not(.disabled):focus,.close:not(:disabled):not(.disabled):hover{opacity:.75}button.close{padding:0;background-color:transparent;border:0;-webkit-appearance:none;-moz-appearance:none;appearance:none}a.close.disabled{pointer-events:none}.toast{max-width:350px;overflow:hidden;font-size:.875rem;background-color:hsla(0,0%,100%,.85);background-clip:padding-box;border:1px solid rgba(0,0,0,.1);box-shadow:0 .25rem .75rem rgba(33,37,41,.1);-webkit-backdrop-filter:blur(10px);backdrop-filter:blur(10px);opacity:0;border-radius:.25rem}.toast:not(:last-child){margin-bottom:.75rem}.toast.showing{opacity:1}.toast.show{display:block;opacity:1}.toast.hide{display:none}.toast-header{display:flex;align-items:center;padding:.25rem .75rem;color:#6c757d;background-color:hsla(0,0%,100%,.85);background-clip:padding-box;border-bottom:1px solid rgba(0,0,0,.05)}.toast-body{padding:.75rem}.modal-open{overflow:hidden}.modal-open .modal{overflow-x:hidden;overflow-y:auto}.modal{position:fixed;top:0;left:0;z-index:1050;display:none;width:100%;height:100%;overflow:hidden;outline:0}.modal-dialog{position:relative;width:auto;margin:.5rem;pointer-events:none}.modal.fade .modal-dialog{transition:transform .3s ease-out;transform:translateY(-50px)}@media(prefers-reduced-motion:reduce){.modal.fade .modal-dialog{transition:none}}.modal.show .modal-dialog{transform:none}.modal.modal-static .modal-dialog{transform:scale(1.02)}.modal-dialog-scrollable{display:flex;max-height:calc(100% - 1rem)}.modal-dialog-scrollable .modal-content{max-height:calc(100vh - 1rem);overflow:hidden}.modal-dialog-scrollable .modal-footer,.modal-dialog-scrollable .modal-header{flex-shrink:0}.modal-dialog-scrollable .modal-body{overflow-y:auto}.modal-dialog-centered{display:flex;align-items:center;min-height:calc(100% - 1rem)}.modal-dialog-centered:before{display:block;height:calc(100vh - 1rem);content:""}.modal-dialog-centered.modal-dialog-scrollable{flex-direction:column;justify-content:center;height:100%}.modal-dialog-centered.modal-dialog-scrollable .modal-content{max-height:none}.modal-dialog-centered.modal-dialog-scrollable:before{content:none}.modal-content{position:relative;display:flex;flex-direction:column;width:100%;pointer-events:auto;background-color:#fff;background-clip:padding-box;border:1px solid rgba(33,37,41,.2);border-radius:8px;outline:0}.modal-backdrop{position:fixed;top:0;left:0;z-index:1040;width:100vw;height:100vh;background-color:#212529}.modal-backdrop.fade{opacity:0}.modal-backdrop.show{opacity:.5}.modal-header{display:flex;align-items:flex-start;justify-content:space-between;padding:1rem;border-bottom:1px solid #d6dbdf;border-top-left-radius:7px;border-top-right-radius:7px}.modal-header .close{padding:1rem;margin:-1rem -1rem -1rem auto}.modal-title{margin-bottom:0;line-height:1.5}.modal-body{position:relative;flex:1 1 auto;padding:1rem}.modal-footer{display:flex;flex-wrap:wrap;align-items:center;justify-content:flex-end;padding:.75rem;border-top:1px solid #d6dbdf;border-bottom-right-radius:7px;border-bottom-left-radius:7px}.modal-footer>*{margin:.25rem}.modal-scrollbar-measure{position:absolute;top:-9999px;width:50px;height:50px;overflow:scroll}@media(min-width:616px){.modal-dialog{max-width:500px;margin:1.75rem auto}.modal-dialog-scrollable{max-height:calc(100% - 3.5rem)}.modal-dialog-scrollable .modal-content{max-height:calc(100vh - 3.5rem)}.modal-dialog-centered{min-height:calc(100% - 3.5rem)}.modal-dialog-centered:before{height:calc(100vh - 3.5rem)}.modal-sm{max-width:300px}}@media(min-width:980px){.modal-lg,.modal-xl{max-width:800px}}@media(min-width:1240px){.modal-xl{max-width:1140px}}.tooltip{position:absolute;z-index:1070;display:block;margin:0;font-family:Noto Sans,sans-serif;font-style:normal;font-weight:400;line-height:1.5;text-align:left;text-align:start;text-decoration:none;text-shadow:none;text-transform:none;letter-spacing:normal;word-break:normal;word-spacing:normal;white-space:normal;line-break:auto;font-size:.875rem;word-wrap:break-word;opacity:0}.tooltip.show{opacity:.9}.tooltip .arrow{position:absolute;display:block;width:.8rem;height:.4rem}.tooltip .arrow:before{position:absolute;content:"";border-color:transparent;border-style:solid}.bs-tooltip-auto[x-placement^=top],.bs-tooltip-top{padding:.4rem 0}.bs-tooltip-auto[x-placement^=top] .arrow,.bs-tooltip-top .arrow{bottom:0}.bs-tooltip-auto[x-placement^=top] .arrow:before,.bs-tooltip-top .arrow:before{top:0;border-width:.4rem .4rem 0;border-top-color:#212529}.bs-tooltip-auto[x-placement^=right],.bs-tooltip-right{padding:0 .4rem}.bs-tooltip-auto[x-placement^=right] .arrow,.bs-tooltip-right .arrow{left:0;width:.4rem;height:.8rem}.bs-tooltip-auto[x-placement^=right] .arrow:before,.bs-tooltip-right .arrow:before{right:0;border-width:.4rem .4rem .4rem 0;border-right-color:#212529}.bs-tooltip-auto[x-placement^=bottom],.bs-tooltip-bottom{padding:.4rem 0}.bs-tooltip-auto[x-placement^=bottom] .arrow,.bs-tooltip-bottom .arrow{top:0}.bs-tooltip-auto[x-placement^=bottom] .arrow:before,.bs-tooltip-bottom .arrow:before{bottom:0;border-width:0 .4rem .4rem;border-bottom-color:#212529}.bs-tooltip-auto[x-placement^=left],.bs-tooltip-left{padding:0 .4rem}.bs-tooltip-auto[x-placement^=left] .arrow,.bs-tooltip-left .arrow{right:0;width:.4rem;height:.8rem}.bs-tooltip-auto[x-placement^=left] .arrow:before,.bs-tooltip-left .arrow:before{left:0;border-width:.4rem 0 .4rem .4rem;border-left-color:#212529}.tooltip-inner{max-width:200px;padding:.25rem .5rem;color:#fff;text-align:center;background-color:#212529;border-radius:8px}.popover{top:0;left:0;z-index:1060;max-width:276px;font-family:Noto Sans,sans-serif;font-style:normal;font-weight:400;line-height:1.5;text-align:left;text-align:start;text-decoration:none;text-shadow:none;text-transform:none;letter-spacing:normal;word-break:normal;word-spacing:normal;white-space:normal;line-break:auto;font-size:.875rem;word-wrap:break-word;background-color:#fff;background-clip:padding-box;border:1px solid rgba(33,37,41,.2);border-radius:8px}.popover,.popover .arrow{position:absolute;display:block}.popover .arrow{width:1rem;height:.5rem;margin:0 8px}.popover .arrow:after,.popover .arrow:before{position:absolute;display:block;content:"";border-color:transparent;border-style:solid}.bs-popover-auto[x-placement^=top],.bs-popover-top{margin-bottom:.5rem}.bs-popover-auto[x-placement^=top]>.arrow,.bs-popover-top>.arrow{bottom:calc(-.5rem - 1px)}.bs-popover-auto[x-placement^=top]>.arrow:before,.bs-popover-top>.arrow:before{bottom:0;border-width:.5rem .5rem 0;border-top-color:rgba(33,37,41,.25)}.bs-popover-auto[x-placement^=top]>.arrow:after,.bs-popover-top>.arrow:after{bottom:1px;border-width:.5rem .5rem 0;border-top-color:#fff}.bs-popover-auto[x-placement^=right],.bs-popover-right{margin-left:.5rem}.bs-popover-auto[x-placement^=right]>.arrow,.bs-popover-right>.arrow{left:calc(-.5rem - 1px);width:.5rem;height:1rem;margin:8px 0}.bs-popover-auto[x-placement^=right]>.arrow:before,.bs-popover-right>.arrow:before{left:0;border-width:.5rem .5rem .5rem 0;border-right-color:rgba(33,37,41,.25)}.bs-popover-auto[x-placement^=right]>.arrow:after,.bs-popover-right>.arrow:after{left:1px;border-width:.5rem .5rem .5rem 0;border-right-color:#fff}.bs-popover-auto[x-placement^=bottom],.bs-popover-bottom{margin-top:.5rem}.bs-popover-auto[x-placement^=bottom]>.arrow,.bs-popover-bottom>.arrow{top:calc(-.5rem - 1px)}.bs-popover-auto[x-placement^=bottom]>.arrow:before,.bs-popover-bottom>.arrow:before{top:0;border-width:0 .5rem .5rem;border-bottom-color:rgba(33,37,41,.25)}.bs-popover-auto[x-placement^=bottom]>.arrow:after,.bs-popover-bottom>.arrow:after{top:1px;border-width:0 .5rem .5rem;border-bottom-color:#fff}.bs-popover-auto[x-placement^=bottom] .popover-header:before,.bs-popover-bottom .popover-header:before{position:absolute;top:0;left:50%;display:block;width:1rem;margin-left:-.5rem;content:"";border-bottom:1px solid #f7f7f7}.bs-popover-auto[x-placement^=left],.bs-popover-left{margin-right:.5rem}.bs-popover-auto[x-placement^=left]>.arrow,.bs-popover-left>.arrow{right:calc(-.5rem - 1px);width:.5rem;height:1rem;margin:8px 0}.bs-popover-auto[x-placement^=left]>.arrow:before,.bs-popover-left>.arrow:before{right:0;border-width:.5rem 0 .5rem .5rem;border-left-color:rgba(33,37,41,.25)}.bs-popover-auto[x-placement^=left]>.arrow:after,.bs-popover-left>.arrow:after{right:1px;border-width:.5rem 0 .5rem .5rem;border-left-color:#fff}.popover-header{padding:.5rem .75rem;margin-bottom:0;font-size:1rem;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;border-top-left-radius:7px;border-top-right-radius:7px}.popover-header:empty{display:none}.popover-body{padding:.5rem .75rem;color:#212529}.carousel{position:relative}.carousel.pointer-event{touch-action:pan-y}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel-inner:after{display:block;clear:both;content:""}.carousel-item{position:relative;display:none;float:left;width:100%;margin-right:-100%;-webkit-backface-visibility:hidden;backface-visibility:hidden;transition:transform .6s ease-in-out}@media(prefers-reduced-motion:reduce){.carousel-item{transition:none}}.carousel-item-next,.carousel-item-prev,.carousel-item.active{display:block}.active.carousel-item-right,.carousel-item-next:not(.carousel-item-left){transform:translateX(100%)}.active.carousel-item-left,.carousel-item-prev:not(.carousel-item-right){transform:translateX(-100%)}.carousel-fade .carousel-item{opacity:0;transition-property:opacity;transform:none}.carousel-fade .carousel-item-next.carousel-item-left,.carousel-fade .carousel-item-prev.carousel-item-right,.carousel-fade .carousel-item.active{z-index:1;opacity:1}.carousel-fade .active.carousel-item-left,.carousel-fade .active.carousel-item-right{z-index:0;opacity:0;transition:opacity 0s .6s}@media(prefers-reduced-motion:reduce){.carousel-fade .active.carousel-item-left,.carousel-fade .active.carousel-item-right{transition:none}}.carousel-control-next,.carousel-control-prev{position:absolute;top:0;bottom:0;z-index:1;display:flex;align-items:center;justify-content:center;width:15%;color:#fff;text-align:center;opacity:.5;transition:opacity .15s ease}@media(prefers-reduced-motion:reduce){.carousel-control-next,.carousel-control-prev{transition:none}}.carousel-control-next:focus,.carousel-control-next:hover,.carousel-control-prev:focus,.carousel-control-prev:hover{color:#fff;text-decoration:none;outline:0;opacity:.9}.carousel-control-prev{left:0}.carousel-control-next{right:0}.carousel-control-next-icon,.carousel-control-prev-icon{display:inline-block;width:20px;height:20px;background:no-repeat 50%/100% 100%}.carousel-control-prev-icon{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' fill='%23fff' width='8' height='8'%3E%3Cpath d='M5.25 0l-4 4 4 4 1.5-1.5L4.25 4l2.5-2.5L5.25 0z'/%3E%3C/svg%3E")}.carousel-control-next-icon{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' fill='%23fff' width='8' height='8'%3E%3Cpath d='M2.75 0l-1.5 1.5L3.75 4l-2.5 2.5L2.75 8l4-4-4-4z'/%3E%3C/svg%3E")}.carousel-indicators{position:absolute;right:0;bottom:0;left:0;z-index:15;display:flex;justify-content:center;padding-left:0;margin-right:15%;margin-left:15%;list-style:none}.carousel-indicators li{box-sizing:content-box;flex:0 1 auto;width:30px;height:3px;margin-right:3px;margin-left:3px;text-indent:-999px;cursor:pointer;background-color:#fff;background-clip:padding-box;border-top:10px solid transparent;border-bottom:10px solid transparent;opacity:.5;transition:opacity .6s ease}@media(prefers-reduced-motion:reduce){.carousel-indicators li{transition:none}}.carousel-indicators .active{opacity:1}.carousel-caption{position:absolute;right:15%;bottom:20px;left:15%;z-index:10;padding-top:20px;padding-bottom:20px;color:#fff;text-align:center}@-webkit-keyframes spinner-border{to{transform:rotate(1turn)}}@keyframes spinner-border{to{transform:rotate(1turn)}}.spinner-border{display:inline-block;width:2rem;height:2rem;vertical-align:text-bottom;border:.25em solid;border-right:.25em solid transparent;border-radius:50%;-webkit-animation:spinner-border .75s linear infinite;animation:spinner-border .75s linear infinite}.spinner-border-sm{width:1rem;height:1rem;border-width:.2em}@-webkit-keyframes spinner-grow{0%{transform:scale(0)}50%{opacity:1}}@keyframes spinner-grow{0%{transform:scale(0)}50%{opacity:1}}.spinner-grow{display:inline-block;width:2rem;height:2rem;vertical-align:text-bottom;background-color:currentColor;border-radius:50%;opacity:0;-webkit-animation:spinner-grow .75s linear infinite;animation:spinner-grow .75s linear infinite}.spinner-grow-sm{width:1rem;height:1rem}.align-baseline{vertical-align:baseline!important}.align-top{vertical-align:top!important}.align-middle{vertical-align:middle!important}.align-bottom{vertical-align:bottom!important}.align-text-bottom{vertical-align:text-bottom!important}.align-text-top{vertical-align:text-top!important}.bg-primary{background-color:#fc0!important}a.bg-primary:focus,a.bg-primary:hover,button.bg-primary:focus,button.bg-primary:hover{background-color:#cca300!important}.bg-secondary{background-color:#212529!important}a.bg-secondary:focus,a.bg-secondary:hover,button.bg-secondary:focus,button.bg-secondary:hover{background-color:#0a0c0d!important}.bg-success{background-color:#28a745!important}a.bg-success:focus,a.bg-success:hover,button.bg-success:focus,button.bg-success:hover{background-color:#1e7e34!important}.bg-info{background-color:#17a2b8!important}a.bg-info:focus,a.bg-info:hover,button.bg-info:focus,button.bg-info:hover{background-color:#117a8b!important}.bg-warning{background-color:#ffc107!important}a.bg-warning:focus,a.bg-warning:hover,button.bg-warning:focus,button.bg-warning:hover{background-color:#d39e00!important}.bg-danger{background-color:#dc3545!important}a.bg-danger:focus,a.bg-danger:hover,button.bg-danger:focus,button.bg-danger:hover{background-color:#bd2130!important}.bg-light{background-color:#f1f6f9!important}a.bg-light:focus,a.bg-light:hover,button.bg-light:focus,button.bg-light:hover{background-color:#cddfea!important}.bg-dark{background-color:#495057!important}a.bg-dark:focus,a.bg-dark:hover,button.bg-dark:focus,button.bg-dark:hover{background-color:#32373b!important}.bg-primary-light{background-color:#fffaf0!important}a.bg-primary-light:focus,a.bg-primary-light:hover,button.bg-primary-light:focus,button.bg-primary-light:hover{background-color:#ffe9bd!important}.bg-secondary-light{background-color:#fff!important}a.bg-secondary-light:focus,a.bg-secondary-light:hover,button.bg-secondary-light:focus,button.bg-secondary-light:hover{background-color:#e6e6e6!important}.bg-tertiary{background-color:#257af4!important}a.bg-tertiary:focus,a.bg-tertiary:hover,button.bg-tertiary:focus,button.bg-tertiary:hover{background-color:#0b60db!important}.bg-tertiary-light{background-color:#e3f1fe!important}a.bg-tertiary-light:focus,a.bg-tertiary-light:hover,button.bg-tertiary-light:focus,button.bg-tertiary-light:hover{background-color:#b2d8fc!important}a.bg-white:focus,a.bg-white:hover,button.bg-white:focus,button.bg-white:hover{background-color:#e6e6e6!important}.bg-black{background-color:#212529!important}a.bg-black:focus,a.bg-black:hover,button.bg-black:focus,button.bg-black:hover{background-color:#0a0c0d!important}.bg-blue{background-color:#257af4!important}a.bg-blue:focus,a.bg-blue:hover,button.bg-blue:focus,button.bg-blue:hover{background-color:#0b60db!important}.bg-light-blue{background-color:#e3f1fe!important}a.bg-light-blue:focus,a.bg-light-blue:hover,button.bg-light-blue:focus,button.bg-light-blue:hover{background-color:#b2d8fc!important}.bg-yellow{background-color:#fc0!important}a.bg-yellow:focus,a.bg-yellow:hover,button.bg-yellow:focus,button.bg-yellow:hover{background-color:#cca300!important}.bg-light-yellow{background-color:#fffaf0!important}a.bg-light-yellow:focus,a.bg-light-yellow:hover,button.bg-light-yellow:focus,button.bg-light-yellow:hover{background-color:#ffe9bd!important}.bg-orange{background-color:#ff8c00!important}a.bg-orange:focus,a.bg-orange:hover,button.bg-orange:focus,button.bg-orange:hover{background-color:#cc7000!important}.bg-light-orange{background-color:#ffe4b5!important}a.bg-light-orange:focus,a.bg-light-orange:hover,button.bg-light-orange:focus,button.bg-light-orange:hover{background-color:#ffd182!important}.bg-red{background-color:#ff3939!important}a.bg-red:focus,a.bg-red:hover,button.bg-red:focus,button.bg-red:hover{background-color:#ff0606!important}.bg-light-red{background-color:#ffe4e1!important}a.bg-light-red:focus,a.bg-light-red:hover,button.bg-light-red:focus,button.bg-light-red:hover{background-color:#ffb6ae!important}.bg-medium{background-color:#d6dbdf!important}a.bg-medium:focus,a.bg-medium:hover,button.bg-medium:focus,button.bg-medium:hover{background-color:#b9c2c9!important}.bg-white{background-color:#fff!important}.bg-transparent{background-color:transparent!important}.border{border:1px solid #d6dbdf!important}.border-top{border-top:1px solid #d6dbdf!important}.border-right{border-right:1px solid #d6dbdf!important}.border-bottom{border-bottom:1px solid #d6dbdf!important}.border-left{border-left:1px solid #d6dbdf!important}.border-0{border:0!important}.border-top-0{border-top:0!important}.border-right-0{border-right:0!important}.border-bottom-0{border-bottom:0!important}.border-left-0{border-left:0!important}.border-primary{border-color:#fc0!important}.border-secondary{border-color:#212529!important}.border-success{border-color:#28a745!important}.border-info{border-color:#17a2b8!important}.border-warning{border-color:#ffc107!important}.border-danger{border-color:#dc3545!important}.border-light{border-color:#f1f6f9!important}.border-dark{border-color:#495057!important}.border-primary-light{border-color:#fffaf0!important}.border-secondary-light{border-color:#fff!important}.border-tertiary{border-color:#257af4!important}.border-tertiary-light{border-color:#e3f1fe!important}.border-black{border-color:#212529!important}.border-blue{border-color:#257af4!important}.border-light-blue{border-color:#e3f1fe!important}.border-yellow{border-color:#fc0!important}.border-light-yellow{border-color:#fffaf0!important}.border-orange{border-color:#ff8c00!important}.border-light-orange{border-color:#ffe4b5!important}.border-red{border-color:#ff3939!important}.border-light-red{border-color:#ffe4e1!important}.border-medium{border-color:#d6dbdf!important}.border-white{border-color:#fff!important}.rounded,.rounded-sm{border-radius:8px!important}.rounded-top{border-top-left-radius:8px!important}.rounded-right,.rounded-top{border-top-right-radius:8px!important}.rounded-bottom,.rounded-right{border-bottom-right-radius:8px!important}.rounded-bottom,.rounded-left{border-bottom-left-radius:8px!important}.rounded-left{border-top-left-radius:8px!important}.rounded-lg{border-radius:8px!important}.rounded-circle{border-radius:50%!important}.rounded-pill{border-radius:50rem!important}.rounded-0{border-radius:0!important}.clearfix:after{display:block;clear:both;content:""}.d-none{display:none!important}.d-inline{display:inline!important}.d-inline-block{display:inline-block!important}.d-block{display:block!important}.d-table{display:table!important}.d-table-row{display:table-row!important}.d-table-cell{display:table-cell!important}.d-flex{display:flex!important}.d-inline-flex{display:inline-flex!important}@media(min-width:400px){.d-xs-none{display:none!important}.d-xs-inline{display:inline!important}.d-xs-inline-block{display:inline-block!important}.d-xs-block{display:block!important}.d-xs-table{display:table!important}.d-xs-table-row{display:table-row!important}.d-xs-table-cell{display:table-cell!important}.d-xs-flex{display:flex!important}.d-xs-inline-flex{display:inline-flex!important}}@media(min-width:616px){.d-sm-none{display:none!important}.d-sm-inline{display:inline!important}.d-sm-inline-block{display:inline-block!important}.d-sm-block{display:block!important}.d-sm-table{display:table!important}.d-sm-table-row{display:table-row!important}.d-sm-table-cell{display:table-cell!important}.d-sm-flex{display:flex!important}.d-sm-inline-flex{display:inline-flex!important}}@media(min-width:768px){.d-md-none{display:none!important}.d-md-inline{display:inline!important}.d-md-inline-block{display:inline-block!important}.d-md-block{display:block!important}.d-md-table{display:table!important}.d-md-table-row{display:table-row!important}.d-md-table-cell{display:table-cell!important}.d-md-flex{display:flex!important}.d-md-inline-flex{display:inline-flex!important}}@media(min-width:980px){.d-lg-none{display:none!important}.d-lg-inline{display:inline!important}.d-lg-inline-block{display:inline-block!important}.d-lg-block{display:block!important}.d-lg-table{display:table!important}.d-lg-table-row{display:table-row!important}.d-lg-table-cell{display:table-cell!important}.d-lg-flex{display:flex!important}.d-lg-inline-flex{display:inline-flex!important}}@media(min-width:1240px){.d-xl-none{display:none!important}.d-xl-inline{display:inline!important}.d-xl-inline-block{display:inline-block!important}.d-xl-block{display:block!important}.d-xl-table{display:table!important}.d-xl-table-row{display:table-row!important}.d-xl-table-cell{display:table-cell!important}.d-xl-flex{display:flex!important}.d-xl-inline-flex{display:inline-flex!important}}@media print{.d-print-none{display:none!important}.d-print-inline{display:inline!important}.d-print-inline-block{display:inline-block!important}.d-print-block{display:block!important}.d-print-table{display:table!important}.d-print-table-row{display:table-row!important}.d-print-table-cell{display:table-cell!important}.d-print-flex{display:flex!important}.d-print-inline-flex{display:inline-flex!important}}.embed-responsive{position:relative;display:block;width:100%;padding:0;overflow:hidden}.embed-responsive:before{display:block;content:""}.embed-responsive .embed-responsive-item,.embed-responsive embed,.embed-responsive iframe,.embed-responsive object,.embed-responsive video{position:absolute;top:0;bottom:0;left:0;width:100%;height:100%;border:0}.embed-responsive-21by9:before{padding-top:42.8571428571%}.embed-responsive-16by9:before{padding-top:56.25%}.embed-responsive-4by3:before{padding-top:75%}.embed-responsive-1by1:before{padding-top:100%}.flex-row{flex-direction:row!important}.flex-column{flex-direction:column!important}.flex-row-reverse{flex-direction:row-reverse!important}.flex-column-reverse{flex-direction:column-reverse!important}.flex-wrap{flex-wrap:wrap!important}.flex-nowrap{flex-wrap:nowrap!important}.flex-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-fill{flex:1 1 auto!important}.flex-grow-0{flex-grow:0!important}.flex-grow-1{flex-grow:1!important}.flex-shrink-0{flex-shrink:0!important}.flex-shrink-1{flex-shrink:1!important}.justify-content-start{justify-content:flex-start!important}.justify-content-end{justify-content:flex-end!important}.justify-content-center{justify-content:center!important}.justify-content-between{justify-content:space-between!important}.justify-content-around{justify-content:space-around!important}.align-items-start{align-items:flex-start!important}.align-items-end{align-items:flex-end!important}.align-items-center{align-items:center!important}.align-items-baseline{align-items:baseline!important}.align-items-stretch{align-items:stretch!important}.align-content-start{align-content:flex-start!important}.align-content-end{align-content:flex-end!important}.align-content-center{align-content:center!important}.align-content-between{align-content:space-between!important}.align-content-around{align-content:space-around!important}.align-content-stretch{align-content:stretch!important}.align-self-auto{align-self:auto!important}.align-self-start{align-self:flex-start!important}.align-self-end{align-self:flex-end!important}.align-self-center{align-self:center!important}.align-self-baseline{align-self:baseline!important}.align-self-stretch{align-self:stretch!important}@media(min-width:400px){.flex-xs-row{flex-direction:row!important}.flex-xs-column{flex-direction:column!important}.flex-xs-row-reverse{flex-direction:row-reverse!important}.flex-xs-column-reverse{flex-direction:column-reverse!important}.flex-xs-wrap{flex-wrap:wrap!important}.flex-xs-nowrap{flex-wrap:nowrap!important}.flex-xs-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-xs-fill{flex:1 1 auto!important}.flex-xs-grow-0{flex-grow:0!important}.flex-xs-grow-1{flex-grow:1!important}.flex-xs-shrink-0{flex-shrink:0!important}.flex-xs-shrink-1{flex-shrink:1!important}.justify-content-xs-start{justify-content:flex-start!important}.justify-content-xs-end{justify-content:flex-end!important}.justify-content-xs-center{justify-content:center!important}.justify-content-xs-between{justify-content:space-between!important}.justify-content-xs-around{justify-content:space-around!important}.align-items-xs-start{align-items:flex-start!important}.align-items-xs-end{align-items:flex-end!important}.align-items-xs-center{align-items:center!important}.align-items-xs-baseline{align-items:baseline!important}.align-items-xs-stretch{align-items:stretch!important}.align-content-xs-start{align-content:flex-start!important}.align-content-xs-end{align-content:flex-end!important}.align-content-xs-center{align-content:center!important}.align-content-xs-between{align-content:space-between!important}.align-content-xs-around{align-content:space-around!important}.align-content-xs-stretch{align-content:stretch!important}.align-self-xs-auto{align-self:auto!important}.align-self-xs-start{align-self:flex-start!important}.align-self-xs-end{align-self:flex-end!important}.align-self-xs-center{align-self:center!important}.align-self-xs-baseline{align-self:baseline!important}.align-self-xs-stretch{align-self:stretch!important}}@media(min-width:616px){.flex-sm-row{flex-direction:row!important}.flex-sm-column{flex-direction:column!important}.flex-sm-row-reverse{flex-direction:row-reverse!important}.flex-sm-column-reverse{flex-direction:column-reverse!important}.flex-sm-wrap{flex-wrap:wrap!important}.flex-sm-nowrap{flex-wrap:nowrap!important}.flex-sm-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-sm-fill{flex:1 1 auto!important}.flex-sm-grow-0{flex-grow:0!important}.flex-sm-grow-1{flex-grow:1!important}.flex-sm-shrink-0{flex-shrink:0!important}.flex-sm-shrink-1{flex-shrink:1!important}.justify-content-sm-start{justify-content:flex-start!important}.justify-content-sm-end{justify-content:flex-end!important}.justify-content-sm-center{justify-content:center!important}.justify-content-sm-between{justify-content:space-between!important}.justify-content-sm-around{justify-content:space-around!important}.align-items-sm-start{align-items:flex-start!important}.align-items-sm-end{align-items:flex-end!important}.align-items-sm-center{align-items:center!important}.align-items-sm-baseline{align-items:baseline!important}.align-items-sm-stretch{align-items:stretch!important}.align-content-sm-start{align-content:flex-start!important}.align-content-sm-end{align-content:flex-end!important}.align-content-sm-center{align-content:center!important}.align-content-sm-between{align-content:space-between!important}.align-content-sm-around{align-content:space-around!important}.align-content-sm-stretch{align-content:stretch!important}.align-self-sm-auto{align-self:auto!important}.align-self-sm-start{align-self:flex-start!important}.align-self-sm-end{align-self:flex-end!important}.align-self-sm-center{align-self:center!important}.align-self-sm-baseline{align-self:baseline!important}.align-self-sm-stretch{align-self:stretch!important}}@media(min-width:768px){.flex-md-row{flex-direction:row!important}.flex-md-column{flex-direction:column!important}.flex-md-row-reverse{flex-direction:row-reverse!important}.flex-md-column-reverse{flex-direction:column-reverse!important}.flex-md-wrap{flex-wrap:wrap!important}.flex-md-nowrap{flex-wrap:nowrap!important}.flex-md-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-md-fill{flex:1 1 auto!important}.flex-md-grow-0{flex-grow:0!important}.flex-md-grow-1{flex-grow:1!important}.flex-md-shrink-0{flex-shrink:0!important}.flex-md-shrink-1{flex-shrink:1!important}.justify-content-md-start{justify-content:flex-start!important}.justify-content-md-end{justify-content:flex-end!important}.justify-content-md-center{justify-content:center!important}.justify-content-md-between{justify-content:space-between!important}.justify-content-md-around{justify-content:space-around!important}.align-items-md-start{align-items:flex-start!important}.align-items-md-end{align-items:flex-end!important}.align-items-md-center{align-items:center!important}.align-items-md-baseline{align-items:baseline!important}.align-items-md-stretch{align-items:stretch!important}.align-content-md-start{align-content:flex-start!important}.align-content-md-end{align-content:flex-end!important}.align-content-md-center{align-content:center!important}.align-content-md-between{align-content:space-between!important}.align-content-md-around{align-content:space-around!important}.align-content-md-stretch{align-content:stretch!important}.align-self-md-auto{align-self:auto!important}.align-self-md-start{align-self:flex-start!important}.align-self-md-end{align-self:flex-end!important}.align-self-md-center{align-self:center!important}.align-self-md-baseline{align-self:baseline!important}.align-self-md-stretch{align-self:stretch!important}}@media(min-width:980px){.flex-lg-row{flex-direction:row!important}.flex-lg-column{flex-direction:column!important}.flex-lg-row-reverse{flex-direction:row-reverse!important}.flex-lg-column-reverse{flex-direction:column-reverse!important}.flex-lg-wrap{flex-wrap:wrap!important}.flex-lg-nowrap{flex-wrap:nowrap!important}.flex-lg-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-lg-fill{flex:1 1 auto!important}.flex-lg-grow-0{flex-grow:0!important}.flex-lg-grow-1{flex-grow:1!important}.flex-lg-shrink-0{flex-shrink:0!important}.flex-lg-shrink-1{flex-shrink:1!important}.justify-content-lg-start{justify-content:flex-start!important}.justify-content-lg-end{justify-content:flex-end!important}.justify-content-lg-center{justify-content:center!important}.justify-content-lg-between{justify-content:space-between!important}.justify-content-lg-around{justify-content:space-around!important}.align-items-lg-start{align-items:flex-start!important}.align-items-lg-end{align-items:flex-end!important}.align-items-lg-center{align-items:center!important}.align-items-lg-baseline{align-items:baseline!important}.align-items-lg-stretch{align-items:stretch!important}.align-content-lg-start{align-content:flex-start!important}.align-content-lg-end{align-content:flex-end!important}.align-content-lg-center{align-content:center!important}.align-content-lg-between{align-content:space-between!important}.align-content-lg-around{align-content:space-around!important}.align-content-lg-stretch{align-content:stretch!important}.align-self-lg-auto{align-self:auto!important}.align-self-lg-start{align-self:flex-start!important}.align-self-lg-end{align-self:flex-end!important}.align-self-lg-center{align-self:center!important}.align-self-lg-baseline{align-self:baseline!important}.align-self-lg-stretch{align-self:stretch!important}}@media(min-width:1240px){.flex-xl-row{flex-direction:row!important}.flex-xl-column{flex-direction:column!important}.flex-xl-row-reverse{flex-direction:row-reverse!important}.flex-xl-column-reverse{flex-direction:column-reverse!important}.flex-xl-wrap{flex-wrap:wrap!important}.flex-xl-nowrap{flex-wrap:nowrap!important}.flex-xl-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-xl-fill{flex:1 1 auto!important}.flex-xl-grow-0{flex-grow:0!important}.flex-xl-grow-1{flex-grow:1!important}.flex-xl-shrink-0{flex-shrink:0!important}.flex-xl-shrink-1{flex-shrink:1!important}.justify-content-xl-start{justify-content:flex-start!important}.justify-content-xl-end{justify-content:flex-end!important}.justify-content-xl-center{justify-content:center!important}.justify-content-xl-between{justify-content:space-between!important}.justify-content-xl-around{justify-content:space-around!important}.align-items-xl-start{align-items:flex-start!important}.align-items-xl-end{align-items:flex-end!important}.align-items-xl-center{align-items:center!important}.align-items-xl-baseline{align-items:baseline!important}.align-items-xl-stretch{align-items:stretch!important}.align-content-xl-start{align-content:flex-start!important}.align-content-xl-end{align-content:flex-end!important}.align-content-xl-center{align-content:center!important}.align-content-xl-between{align-content:space-between!important}.align-content-xl-around{align-content:space-around!important}.align-content-xl-stretch{align-content:stretch!important}.align-self-xl-auto{align-self:auto!important}.align-self-xl-start{align-self:flex-start!important}.align-self-xl-end{align-self:flex-end!important}.align-self-xl-center{align-self:center!important}.align-self-xl-baseline{align-self:baseline!important}.align-self-xl-stretch{align-self:stretch!important}}.float-left{float:left!important}.float-right{float:right!important}.float-none{float:none!important}@media(min-width:400px){.float-xs-left{float:left!important}.float-xs-right{float:right!important}.float-xs-none{float:none!important}}@media(min-width:616px){.float-sm-left{float:left!important}.float-sm-right{float:right!important}.float-sm-none{float:none!important}}@media(min-width:768px){.float-md-left{float:left!important}.float-md-right{float:right!important}.float-md-none{float:none!important}}@media(min-width:980px){.float-lg-left{float:left!important}.float-lg-right{float:right!important}.float-lg-none{float:none!important}}@media(min-width:1240px){.float-xl-left{float:left!important}.float-xl-right{float:right!important}.float-xl-none{float:none!important}}.overflow-auto{overflow:auto!important}.overflow-hidden{overflow:hidden!important}.position-static{position:static!important}.position-relative{position:relative!important}.position-absolute{position:absolute!important}.position-fixed{position:fixed!important}.position-sticky{position:sticky!important}.fixed-top{top:0}.fixed-bottom,.fixed-top{position:fixed;right:0;left:0;z-index:1030}.fixed-bottom{bottom:0}@supports(position:sticky){.sticky-top{position:sticky;top:0;z-index:1020}}.sr-only{position:absolute;width:1px;height:1px;padding:0;margin:-1px;overflow:hidden;clip:rect(0,0,0,0);white-space:nowrap;border:0}.sr-only-focusable:active,.sr-only-focusable:focus{position:static;width:auto;height:auto;overflow:visible;clip:auto;white-space:normal}.shadow-sm{box-shadow:0 2px 14px rgba(108,117,125,.2)!important}.shadow{box-shadow:0 8px 20px rgba(108,117,125,.2)!important}.shadow-lg{box-shadow:0 12px 32px rgba(108,117,125,.2)!important}.shadow-none{box-shadow:none!important}.w-25{width:25%!important}.w-50{width:50%!important}.w-75{width:75%!important}.w-100{width:100%!important}.w-auto{width:auto!important}.h-25{height:25%!important}.h-50{height:50%!important}.h-75{height:75%!important}.h-100{height:100%!important}.h-auto{height:auto!important}.mw-100{max-width:100%!important}.mh-100{max-height:100%!important}.min-vw-100{min-width:100vw!important}.min-vh-100{min-height:100vh!important}.vw-100{width:100vw!important}.vh-100{height:100vh!important}.stretched-link:after{position:absolute;top:0;right:0;bottom:0;left:0;z-index:1;pointer-events:auto;content:"";background-color:transparent}.m-0{margin:0!important}.mt-0,.my-0{margin-top:0!important}.mr-0,.mx-0{margin-right:0!important}.mb-0,.my-0{margin-bottom:0!important}.ml-0,.mx-0{margin-left:0!important}.m-1{margin:8px!important}.mt-1,.my-1{margin-top:8px!important}.mr-1,.mx-1{margin-right:8px!important}.mb-1,.my-1{margin-bottom:8px!important}.ml-1,.mx-1{margin-left:8px!important}.m-2{margin:16px!important}.mt-2,.my-2{margin-top:16px!important}.mr-2,.mx-2{margin-right:16px!important}.mb-2,.my-2{margin-bottom:16px!important}.ml-2,.mx-2{margin-left:16px!important}.m-3{margin:24px!important}.mt-3,.my-3{margin-top:24px!important}.mr-3,.mx-3{margin-right:24px!important}.mb-3,.my-3{margin-bottom:24px!important}.ml-3,.mx-3{margin-left:24px!important}.m-4{margin:32px!important}.mt-4,.my-4{margin-top:32px!important}.mr-4,.mx-4{margin-right:32px!important}.mb-4,.my-4{margin-bottom:32px!important}.ml-4,.mx-4{margin-left:32px!important}.m-5{margin:40px!important}.mt-5,.my-5{margin-top:40px!important}.mr-5,.mx-5{margin-right:40px!important}.mb-5,.my-5{margin-bottom:40px!important}.ml-5,.mx-5{margin-left:40px!important}.m-6{margin:48px!important}.mt-6,.my-6{margin-top:48px!important}.mr-6,.mx-6{margin-right:48px!important}.mb-6,.my-6{margin-bottom:48px!important}.ml-6,.mx-6{margin-left:48px!important}.m-7{margin:56px!important}.mt-7,.my-7{margin-top:56px!important}.mr-7,.mx-7{margin-right:56px!important}.mb-7,.my-7{margin-bottom:56px!important}.ml-7,.mx-7{margin-left:56px!important}.m-8{margin:64px!important}.mt-8,.my-8{margin-top:64px!important}.mr-8,.mx-8{margin-right:64px!important}.mb-8,.my-8{margin-bottom:64px!important}.ml-8,.mx-8{margin-left:64px!important}.m-9{margin:72px!important}.mt-9,.my-9{margin-top:72px!important}.mr-9,.mx-9{margin-right:72px!important}.mb-9,.my-9{margin-bottom:72px!important}.ml-9,.mx-9{margin-left:72px!important}.m-10{margin:80px!important}.mt-10,.my-10{margin-top:80px!important}.mr-10,.mx-10{margin-right:80px!important}.mb-10,.my-10{margin-bottom:80px!important}.ml-10,.mx-10{margin-left:80px!important}.m-12{margin:96px!important}.mt-12,.my-12{margin-top:96px!important}.mr-12,.mx-12{margin-right:96px!important}.mb-12,.my-12{margin-bottom:96px!important}.ml-12,.mx-12{margin-left:96px!important}.m-15{margin:120px!important}.mt-15,.my-15{margin-top:120px!important}.mr-15,.mx-15{margin-right:120px!important}.mb-15,.my-15{margin-bottom:120px!important}.ml-15,.mx-15{margin-left:120px!important}.p-0{padding:0!important}.pt-0,.py-0{padding-top:0!important}.pr-0,.px-0{padding-right:0!important}.pb-0,.py-0{padding-bottom:0!important}.pl-0,.px-0{padding-left:0!important}.p-1{padding:8px!important}.pt-1,.py-1{padding-top:8px!important}.pr-1,.px-1{padding-right:8px!important}.pb-1,.py-1{padding-bottom:8px!important}.pl-1,.px-1{padding-left:8px!important}.p-2{padding:16px!important}.pt-2,.py-2{padding-top:16px!important}.pr-2,.px-2{padding-right:16px!important}.pb-2,.py-2{padding-bottom:16px!important}.pl-2,.px-2{padding-left:16px!important}.p-3{padding:24px!important}.pt-3,.py-3{padding-top:24px!important}.pr-3,.px-3{padding-right:24px!important}.pb-3,.py-3{padding-bottom:24px!important}.pl-3,.px-3{padding-left:24px!important}.p-4{padding:32px!important}.pt-4,.py-4{padding-top:32px!important}.pr-4,.px-4{padding-right:32px!important}.pb-4,.py-4{padding-bottom:32px!important}.pl-4,.px-4{padding-left:32px!important}.p-5{padding:40px!important}.pt-5,.py-5{padding-top:40px!important}.pr-5,.px-5{padding-right:40px!important}.pb-5,.py-5{padding-bottom:40px!important}.pl-5,.px-5{padding-left:40px!important}.p-6{padding:48px!important}.pt-6,.py-6{padding-top:48px!important}.pr-6,.px-6{padding-right:48px!important}.pb-6,.py-6{padding-bottom:48px!important}.pl-6,.px-6{padding-left:48px!important}.p-7{padding:56px!important}.pt-7,.py-7{padding-top:56px!important}.pr-7,.px-7{padding-right:56px!important}.pb-7,.py-7{padding-bottom:56px!important}.pl-7,.px-7{padding-left:56px!important}.p-8{padding:64px!important}.pt-8,.py-8{padding-top:64px!important}.pr-8,.px-8{padding-right:64px!important}.pb-8,.py-8{padding-bottom:64px!important}.pl-8,.px-8{padding-left:64px!important}.p-9{padding:72px!important}.pt-9,.py-9{padding-top:72px!important}.pr-9,.px-9{padding-right:72px!important}.pb-9,.py-9{padding-bottom:72px!important}.pl-9,.px-9{padding-left:72px!important}.p-10{padding:80px!important}.pt-10,.py-10{padding-top:80px!important}.pr-10,.px-10{padding-right:80px!important}.pb-10,.py-10{padding-bottom:80px!important}.pl-10,.px-10{padding-left:80px!important}.p-12{padding:96px!important}.pt-12,.py-12{padding-top:96px!important}.pr-12,.px-12{padding-right:96px!important}.pb-12,.py-12{padding-bottom:96px!important}.pl-12,.px-12{padding-left:96px!important}.p-15{padding:120px!important}.pt-15,.py-15{padding-top:120px!important}.pr-15,.px-15{padding-right:120px!important}.pb-15,.py-15{padding-bottom:120px!important}.pl-15,.px-15{padding-left:120px!important}.m-n1{margin:-8px!important}.mt-n1,.my-n1{margin-top:-8px!important}.mr-n1,.mx-n1{margin-right:-8px!important}.mb-n1,.my-n1{margin-bottom:-8px!important}.ml-n1,.mx-n1{margin-left:-8px!important}.m-n2{margin:-16px!important}.mt-n2,.my-n2{margin-top:-16px!important}.mr-n2,.mx-n2{margin-right:-16px!important}.mb-n2,.my-n2{margin-bottom:-16px!important}.ml-n2,.mx-n2{margin-left:-16px!important}.m-n3{margin:-24px!important}.mt-n3,.my-n3{margin-top:-24px!important}.mr-n3,.mx-n3{margin-right:-24px!important}.mb-n3,.my-n3{margin-bottom:-24px!important}.ml-n3,.mx-n3{margin-left:-24px!important}.m-n4{margin:-32px!important}.mt-n4,.my-n4{margin-top:-32px!important}.mr-n4,.mx-n4{margin-right:-32px!important}.mb-n4,.my-n4{margin-bottom:-32px!important}.ml-n4,.mx-n4{margin-left:-32px!important}.m-n5{margin:-40px!important}.mt-n5,.my-n5{margin-top:-40px!important}.mr-n5,.mx-n5{margin-right:-40px!important}.mb-n5,.my-n5{margin-bottom:-40px!important}.ml-n5,.mx-n5{margin-left:-40px!important}.m-n6{margin:-48px!important}.mt-n6,.my-n6{margin-top:-48px!important}.mr-n6,.mx-n6{margin-right:-48px!important}.mb-n6,.my-n6{margin-bottom:-48px!important}.ml-n6,.mx-n6{margin-left:-48px!important}.m-n7{margin:-56px!important}.mt-n7,.my-n7{margin-top:-56px!important}.mr-n7,.mx-n7{margin-right:-56px!important}.mb-n7,.my-n7{margin-bottom:-56px!important}.ml-n7,.mx-n7{margin-left:-56px!important}.m-n8{margin:-64px!important}.mt-n8,.my-n8{margin-top:-64px!important}.mr-n8,.mx-n8{margin-right:-64px!important}.mb-n8,.my-n8{margin-bottom:-64px!important}.ml-n8,.mx-n8{margin-left:-64px!important}.m-n9{margin:-72px!important}.mt-n9,.my-n9{margin-top:-72px!important}.mr-n9,.mx-n9{margin-right:-72px!important}.mb-n9,.my-n9{margin-bottom:-72px!important}.ml-n9,.mx-n9{margin-left:-72px!important}.m-n10{margin:-80px!important}.mt-n10,.my-n10{margin-top:-80px!important}.mr-n10,.mx-n10{margin-right:-80px!important}.mb-n10,.my-n10{margin-bottom:-80px!important}.ml-n10,.mx-n10{margin-left:-80px!important}.m-n12{margin:-96px!important}.mt-n12,.my-n12{margin-top:-96px!important}.mr-n12,.mx-n12{margin-right:-96px!important}.mb-n12,.my-n12{margin-bottom:-96px!important}.ml-n12,.mx-n12{margin-left:-96px!important}.m-n15{margin:-120px!important}.mt-n15,.my-n15{margin-top:-120px!important}.mr-n15,.mx-n15{margin-right:-120px!important}.mb-n15,.my-n15{margin-bottom:-120px!important}.ml-n15,.mx-n15{margin-left:-120px!important}.m-auto{margin:auto!important}.mt-auto,.my-auto{margin-top:auto!important}.mr-auto,.mx-auto{margin-right:auto!important}.mb-auto,.my-auto{margin-bottom:auto!important}.ml-auto,.mx-auto{margin-left:auto!important}@media(min-width:400px){.m-xs-0{margin:0!important}.mt-xs-0,.my-xs-0{margin-top:0!important}.mr-xs-0,.mx-xs-0{margin-right:0!important}.mb-xs-0,.my-xs-0{margin-bottom:0!important}.ml-xs-0,.mx-xs-0{margin-left:0!important}.m-xs-1{margin:8px!important}.mt-xs-1,.my-xs-1{margin-top:8px!important}.mr-xs-1,.mx-xs-1{margin-right:8px!important}.mb-xs-1,.my-xs-1{margin-bottom:8px!important}.ml-xs-1,.mx-xs-1{margin-left:8px!important}.m-xs-2{margin:16px!important}.mt-xs-2,.my-xs-2{margin-top:16px!important}.mr-xs-2,.mx-xs-2{margin-right:16px!important}.mb-xs-2,.my-xs-2{margin-bottom:16px!important}.ml-xs-2,.mx-xs-2{margin-left:16px!important}.m-xs-3{margin:24px!important}.mt-xs-3,.my-xs-3{margin-top:24px!important}.mr-xs-3,.mx-xs-3{margin-right:24px!important}.mb-xs-3,.my-xs-3{margin-bottom:24px!important}.ml-xs-3,.mx-xs-3{margin-left:24px!important}.m-xs-4{margin:32px!important}.mt-xs-4,.my-xs-4{margin-top:32px!important}.mr-xs-4,.mx-xs-4{margin-right:32px!important}.mb-xs-4,.my-xs-4{margin-bottom:32px!important}.ml-xs-4,.mx-xs-4{margin-left:32px!important}.m-xs-5{margin:40px!important}.mt-xs-5,.my-xs-5{margin-top:40px!important}.mr-xs-5,.mx-xs-5{margin-right:40px!important}.mb-xs-5,.my-xs-5{margin-bottom:40px!important}.ml-xs-5,.mx-xs-5{margin-left:40px!important}.m-xs-6{margin:48px!important}.mt-xs-6,.my-xs-6{margin-top:48px!important}.mr-xs-6,.mx-xs-6{margin-right:48px!important}.mb-xs-6,.my-xs-6{margin-bottom:48px!important}.ml-xs-6,.mx-xs-6{margin-left:48px!important}.m-xs-7{margin:56px!important}.mt-xs-7,.my-xs-7{margin-top:56px!important}.mr-xs-7,.mx-xs-7{margin-right:56px!important}.mb-xs-7,.my-xs-7{margin-bottom:56px!important}.ml-xs-7,.mx-xs-7{margin-left:56px!important}.m-xs-8{margin:64px!important}.mt-xs-8,.my-xs-8{margin-top:64px!important}.mr-xs-8,.mx-xs-8{margin-right:64px!important}.mb-xs-8,.my-xs-8{margin-bottom:64px!important}.ml-xs-8,.mx-xs-8{margin-left:64px!important}.m-xs-9{margin:72px!important}.mt-xs-9,.my-xs-9{margin-top:72px!important}.mr-xs-9,.mx-xs-9{margin-right:72px!important}.mb-xs-9,.my-xs-9{margin-bottom:72px!important}.ml-xs-9,.mx-xs-9{margin-left:72px!important}.m-xs-10{margin:80px!important}.mt-xs-10,.my-xs-10{margin-top:80px!important}.mr-xs-10,.mx-xs-10{margin-right:80px!important}.mb-xs-10,.my-xs-10{margin-bottom:80px!important}.ml-xs-10,.mx-xs-10{margin-left:80px!important}.m-xs-12{margin:96px!important}.mt-xs-12,.my-xs-12{margin-top:96px!important}.mr-xs-12,.mx-xs-12{margin-right:96px!important}.mb-xs-12,.my-xs-12{margin-bottom:96px!important}.ml-xs-12,.mx-xs-12{margin-left:96px!important}.m-xs-15{margin:120px!important}.mt-xs-15,.my-xs-15{margin-top:120px!important}.mr-xs-15,.mx-xs-15{margin-right:120px!important}.mb-xs-15,.my-xs-15{margin-bottom:120px!important}.ml-xs-15,.mx-xs-15{margin-left:120px!important}.p-xs-0{padding:0!important}.pt-xs-0,.py-xs-0{padding-top:0!important}.pr-xs-0,.px-xs-0{padding-right:0!important}.pb-xs-0,.py-xs-0{padding-bottom:0!important}.pl-xs-0,.px-xs-0{padding-left:0!important}.p-xs-1{padding:8px!important}.pt-xs-1,.py-xs-1{padding-top:8px!important}.pr-xs-1,.px-xs-1{padding-right:8px!important}.pb-xs-1,.py-xs-1{padding-bottom:8px!important}.pl-xs-1,.px-xs-1{padding-left:8px!important}.p-xs-2{padding:16px!important}.pt-xs-2,.py-xs-2{padding-top:16px!important}.pr-xs-2,.px-xs-2{padding-right:16px!important}.pb-xs-2,.py-xs-2{padding-bottom:16px!important}.pl-xs-2,.px-xs-2{padding-left:16px!important}.p-xs-3{padding:24px!important}.pt-xs-3,.py-xs-3{padding-top:24px!important}.pr-xs-3,.px-xs-3{padding-right:24px!important}.pb-xs-3,.py-xs-3{padding-bottom:24px!important}.pl-xs-3,.px-xs-3{padding-left:24px!important}.p-xs-4{padding:32px!important}.pt-xs-4,.py-xs-4{padding-top:32px!important}.pr-xs-4,.px-xs-4{padding-right:32px!important}.pb-xs-4,.py-xs-4{padding-bottom:32px!important}.pl-xs-4,.px-xs-4{padding-left:32px!important}.p-xs-5{padding:40px!important}.pt-xs-5,.py-xs-5{padding-top:40px!important}.pr-xs-5,.px-xs-5{padding-right:40px!important}.pb-xs-5,.py-xs-5{padding-bottom:40px!important}.pl-xs-5,.px-xs-5{padding-left:40px!important}.p-xs-6{padding:48px!important}.pt-xs-6,.py-xs-6{padding-top:48px!important}.pr-xs-6,.px-xs-6{padding-right:48px!important}.pb-xs-6,.py-xs-6{padding-bottom:48px!important}.pl-xs-6,.px-xs-6{padding-left:48px!important}.p-xs-7{padding:56px!important}.pt-xs-7,.py-xs-7{padding-top:56px!important}.pr-xs-7,.px-xs-7{padding-right:56px!important}.pb-xs-7,.py-xs-7{padding-bottom:56px!important}.pl-xs-7,.px-xs-7{padding-left:56px!important}.p-xs-8{padding:64px!important}.pt-xs-8,.py-xs-8{padding-top:64px!important}.pr-xs-8,.px-xs-8{padding-right:64px!important}.pb-xs-8,.py-xs-8{padding-bottom:64px!important}.pl-xs-8,.px-xs-8{padding-left:64px!important}.p-xs-9{padding:72px!important}.pt-xs-9,.py-xs-9{padding-top:72px!important}.pr-xs-9,.px-xs-9{padding-right:72px!important}.pb-xs-9,.py-xs-9{padding-bottom:72px!important}.pl-xs-9,.px-xs-9{padding-left:72px!important}.p-xs-10{padding:80px!important}.pt-xs-10,.py-xs-10{padding-top:80px!important}.pr-xs-10,.px-xs-10{padding-right:80px!important}.pb-xs-10,.py-xs-10{padding-bottom:80px!important}.pl-xs-10,.px-xs-10{padding-left:80px!important}.p-xs-12{padding:96px!important}.pt-xs-12,.py-xs-12{padding-top:96px!important}.pr-xs-12,.px-xs-12{padding-right:96px!important}.pb-xs-12,.py-xs-12{padding-bottom:96px!important}.pl-xs-12,.px-xs-12{padding-left:96px!important}.p-xs-15{padding:120px!important}.pt-xs-15,.py-xs-15{padding-top:120px!important}.pr-xs-15,.px-xs-15{padding-right:120px!important}.pb-xs-15,.py-xs-15{padding-bottom:120px!important}.pl-xs-15,.px-xs-15{padding-left:120px!important}.m-xs-n1{margin:-8px!important}.mt-xs-n1,.my-xs-n1{margin-top:-8px!important}.mr-xs-n1,.mx-xs-n1{margin-right:-8px!important}.mb-xs-n1,.my-xs-n1{margin-bottom:-8px!important}.ml-xs-n1,.mx-xs-n1{margin-left:-8px!important}.m-xs-n2{margin:-16px!important}.mt-xs-n2,.my-xs-n2{margin-top:-16px!important}.mr-xs-n2,.mx-xs-n2{margin-right:-16px!important}.mb-xs-n2,.my-xs-n2{margin-bottom:-16px!important}.ml-xs-n2,.mx-xs-n2{margin-left:-16px!important}.m-xs-n3{margin:-24px!important}.mt-xs-n3,.my-xs-n3{margin-top:-24px!important}.mr-xs-n3,.mx-xs-n3{margin-right:-24px!important}.mb-xs-n3,.my-xs-n3{margin-bottom:-24px!important}.ml-xs-n3,.mx-xs-n3{margin-left:-24px!important}.m-xs-n4{margin:-32px!important}.mt-xs-n4,.my-xs-n4{margin-top:-32px!important}.mr-xs-n4,.mx-xs-n4{margin-right:-32px!important}.mb-xs-n4,.my-xs-n4{margin-bottom:-32px!important}.ml-xs-n4,.mx-xs-n4{margin-left:-32px!important}.m-xs-n5{margin:-40px!important}.mt-xs-n5,.my-xs-n5{margin-top:-40px!important}.mr-xs-n5,.mx-xs-n5{margin-right:-40px!important}.mb-xs-n5,.my-xs-n5{margin-bottom:-40px!important}.ml-xs-n5,.mx-xs-n5{margin-left:-40px!important}.m-xs-n6{margin:-48px!important}.mt-xs-n6,.my-xs-n6{margin-top:-48px!important}.mr-xs-n6,.mx-xs-n6{margin-right:-48px!important}.mb-xs-n6,.my-xs-n6{margin-bottom:-48px!important}.ml-xs-n6,.mx-xs-n6{margin-left:-48px!important}.m-xs-n7{margin:-56px!important}.mt-xs-n7,.my-xs-n7{margin-top:-56px!important}.mr-xs-n7,.mx-xs-n7{margin-right:-56px!important}.mb-xs-n7,.my-xs-n7{margin-bottom:-56px!important}.ml-xs-n7,.mx-xs-n7{margin-left:-56px!important}.m-xs-n8{margin:-64px!important}.mt-xs-n8,.my-xs-n8{margin-top:-64px!important}.mr-xs-n8,.mx-xs-n8{margin-right:-64px!important}.mb-xs-n8,.my-xs-n8{margin-bottom:-64px!important}.ml-xs-n8,.mx-xs-n8{margin-left:-64px!important}.m-xs-n9{margin:-72px!important}.mt-xs-n9,.my-xs-n9{margin-top:-72px!important}.mr-xs-n9,.mx-xs-n9{margin-right:-72px!important}.mb-xs-n9,.my-xs-n9{margin-bottom:-72px!important}.ml-xs-n9,.mx-xs-n9{margin-left:-72px!important}.m-xs-n10{margin:-80px!important}.mt-xs-n10,.my-xs-n10{margin-top:-80px!important}.mr-xs-n10,.mx-xs-n10{margin-right:-80px!important}.mb-xs-n10,.my-xs-n10{margin-bottom:-80px!important}.ml-xs-n10,.mx-xs-n10{margin-left:-80px!important}.m-xs-n12{margin:-96px!important}.mt-xs-n12,.my-xs-n12{margin-top:-96px!important}.mr-xs-n12,.mx-xs-n12{margin-right:-96px!important}.mb-xs-n12,.my-xs-n12{margin-bottom:-96px!important}.ml-xs-n12,.mx-xs-n12{margin-left:-96px!important}.m-xs-n15{margin:-120px!important}.mt-xs-n15,.my-xs-n15{margin-top:-120px!important}.mr-xs-n15,.mx-xs-n15{margin-right:-120px!important}.mb-xs-n15,.my-xs-n15{margin-bottom:-120px!important}.ml-xs-n15,.mx-xs-n15{margin-left:-120px!important}.m-xs-auto{margin:auto!important}.mt-xs-auto,.my-xs-auto{margin-top:auto!important}.mr-xs-auto,.mx-xs-auto{margin-right:auto!important}.mb-xs-auto,.my-xs-auto{margin-bottom:auto!important}.ml-xs-auto,.mx-xs-auto{margin-left:auto!important}}@media(min-width:616px){.m-sm-0{margin:0!important}.mt-sm-0,.my-sm-0{margin-top:0!important}.mr-sm-0,.mx-sm-0{margin-right:0!important}.mb-sm-0,.my-sm-0{margin-bottom:0!important}.ml-sm-0,.mx-sm-0{margin-left:0!important}.m-sm-1{margin:8px!important}.mt-sm-1,.my-sm-1{margin-top:8px!important}.mr-sm-1,.mx-sm-1{margin-right:8px!important}.mb-sm-1,.my-sm-1{margin-bottom:8px!important}.ml-sm-1,.mx-sm-1{margin-left:8px!important}.m-sm-2{margin:16px!important}.mt-sm-2,.my-sm-2{margin-top:16px!important}.mr-sm-2,.mx-sm-2{margin-right:16px!important}.mb-sm-2,.my-sm-2{margin-bottom:16px!important}.ml-sm-2,.mx-sm-2{margin-left:16px!important}.m-sm-3{margin:24px!important}.mt-sm-3,.my-sm-3{margin-top:24px!important}.mr-sm-3,.mx-sm-3{margin-right:24px!important}.mb-sm-3,.my-sm-3{margin-bottom:24px!important}.ml-sm-3,.mx-sm-3{margin-left:24px!important}.m-sm-4{margin:32px!important}.mt-sm-4,.my-sm-4{margin-top:32px!important}.mr-sm-4,.mx-sm-4{margin-right:32px!important}.mb-sm-4,.my-sm-4{margin-bottom:32px!important}.ml-sm-4,.mx-sm-4{margin-left:32px!important}.m-sm-5{margin:40px!important}.mt-sm-5,.my-sm-5{margin-top:40px!important}.mr-sm-5,.mx-sm-5{margin-right:40px!important}.mb-sm-5,.my-sm-5{margin-bottom:40px!important}.ml-sm-5,.mx-sm-5{margin-left:40px!important}.m-sm-6{margin:48px!important}.mt-sm-6,.my-sm-6{margin-top:48px!important}.mr-sm-6,.mx-sm-6{margin-right:48px!important}.mb-sm-6,.my-sm-6{margin-bottom:48px!important}.ml-sm-6,.mx-sm-6{margin-left:48px!important}.m-sm-7{margin:56px!important}.mt-sm-7,.my-sm-7{margin-top:56px!important}.mr-sm-7,.mx-sm-7{margin-right:56px!important}.mb-sm-7,.my-sm-7{margin-bottom:56px!important}.ml-sm-7,.mx-sm-7{margin-left:56px!important}.m-sm-8{margin:64px!important}.mt-sm-8,.my-sm-8{margin-top:64px!important}.mr-sm-8,.mx-sm-8{margin-right:64px!important}.mb-sm-8,.my-sm-8{margin-bottom:64px!important}.ml-sm-8,.mx-sm-8{margin-left:64px!important}.m-sm-9{margin:72px!important}.mt-sm-9,.my-sm-9{margin-top:72px!important}.mr-sm-9,.mx-sm-9{margin-right:72px!important}.mb-sm-9,.my-sm-9{margin-bottom:72px!important}.ml-sm-9,.mx-sm-9{margin-left:72px!important}.m-sm-10{margin:80px!important}.mt-sm-10,.my-sm-10{margin-top:80px!important}.mr-sm-10,.mx-sm-10{margin-right:80px!important}.mb-sm-10,.my-sm-10{margin-bottom:80px!important}.ml-sm-10,.mx-sm-10{margin-left:80px!important}.m-sm-12{margin:96px!important}.mt-sm-12,.my-sm-12{margin-top:96px!important}.mr-sm-12,.mx-sm-12{margin-right:96px!important}.mb-sm-12,.my-sm-12{margin-bottom:96px!important}.ml-sm-12,.mx-sm-12{margin-left:96px!important}.m-sm-15{margin:120px!important}.mt-sm-15,.my-sm-15{margin-top:120px!important}.mr-sm-15,.mx-sm-15{margin-right:120px!important}.mb-sm-15,.my-sm-15{margin-bottom:120px!important}.ml-sm-15,.mx-sm-15{margin-left:120px!important}.p-sm-0{padding:0!important}.pt-sm-0,.py-sm-0{padding-top:0!important}.pr-sm-0,.px-sm-0{padding-right:0!important}.pb-sm-0,.py-sm-0{padding-bottom:0!important}.pl-sm-0,.px-sm-0{padding-left:0!important}.p-sm-1{padding:8px!important}.pt-sm-1,.py-sm-1{padding-top:8px!important}.pr-sm-1,.px-sm-1{padding-right:8px!important}.pb-sm-1,.py-sm-1{padding-bottom:8px!important}.pl-sm-1,.px-sm-1{padding-left:8px!important}.p-sm-2{padding:16px!important}.pt-sm-2,.py-sm-2{padding-top:16px!important}.pr-sm-2,.px-sm-2{padding-right:16px!important}.pb-sm-2,.py-sm-2{padding-bottom:16px!important}.pl-sm-2,.px-sm-2{padding-left:16px!important}.p-sm-3{padding:24px!important}.pt-sm-3,.py-sm-3{padding-top:24px!important}.pr-sm-3,.px-sm-3{padding-right:24px!important}.pb-sm-3,.py-sm-3{padding-bottom:24px!important}.pl-sm-3,.px-sm-3{padding-left:24px!important}.p-sm-4{padding:32px!important}.pt-sm-4,.py-sm-4{padding-top:32px!important}.pr-sm-4,.px-sm-4{padding-right:32px!important}.pb-sm-4,.py-sm-4{padding-bottom:32px!important}.pl-sm-4,.px-sm-4{padding-left:32px!important}.p-sm-5{padding:40px!important}.pt-sm-5,.py-sm-5{padding-top:40px!important}.pr-sm-5,.px-sm-5{padding-right:40px!important}.pb-sm-5,.py-sm-5{padding-bottom:40px!important}.pl-sm-5,.px-sm-5{padding-left:40px!important}.p-sm-6{padding:48px!important}.pt-sm-6,.py-sm-6{padding-top:48px!important}.pr-sm-6,.px-sm-6{padding-right:48px!important}.pb-sm-6,.py-sm-6{padding-bottom:48px!important}.pl-sm-6,.px-sm-6{padding-left:48px!important}.p-sm-7{padding:56px!important}.pt-sm-7,.py-sm-7{padding-top:56px!important}.pr-sm-7,.px-sm-7{padding-right:56px!important}.pb-sm-7,.py-sm-7{padding-bottom:56px!important}.pl-sm-7,.px-sm-7{padding-left:56px!important}.p-sm-8{padding:64px!important}.pt-sm-8,.py-sm-8{padding-top:64px!important}.pr-sm-8,.px-sm-8{padding-right:64px!important}.pb-sm-8,.py-sm-8{padding-bottom:64px!important}.pl-sm-8,.px-sm-8{padding-left:64px!important}.p-sm-9{padding:72px!important}.pt-sm-9,.py-sm-9{padding-top:72px!important}.pr-sm-9,.px-sm-9{padding-right:72px!important}.pb-sm-9,.py-sm-9{padding-bottom:72px!important}.pl-sm-9,.px-sm-9{padding-left:72px!important}.p-sm-10{padding:80px!important}.pt-sm-10,.py-sm-10{padding-top:80px!important}.pr-sm-10,.px-sm-10{padding-right:80px!important}.pb-sm-10,.py-sm-10{padding-bottom:80px!important}.pl-sm-10,.px-sm-10{padding-left:80px!important}.p-sm-12{padding:96px!important}.pt-sm-12,.py-sm-12{padding-top:96px!important}.pr-sm-12,.px-sm-12{padding-right:96px!important}.pb-sm-12,.py-sm-12{padding-bottom:96px!important}.pl-sm-12,.px-sm-12{padding-left:96px!important}.p-sm-15{padding:120px!important}.pt-sm-15,.py-sm-15{padding-top:120px!important}.pr-sm-15,.px-sm-15{padding-right:120px!important}.pb-sm-15,.py-sm-15{padding-bottom:120px!important}.pl-sm-15,.px-sm-15{padding-left:120px!important}.m-sm-n1{margin:-8px!important}.mt-sm-n1,.my-sm-n1{margin-top:-8px!important}.mr-sm-n1,.mx-sm-n1{margin-right:-8px!important}.mb-sm-n1,.my-sm-n1{margin-bottom:-8px!important}.ml-sm-n1,.mx-sm-n1{margin-left:-8px!important}.m-sm-n2{margin:-16px!important}.mt-sm-n2,.my-sm-n2{margin-top:-16px!important}.mr-sm-n2,.mx-sm-n2{margin-right:-16px!important}.mb-sm-n2,.my-sm-n2{margin-bottom:-16px!important}.ml-sm-n2,.mx-sm-n2{margin-left:-16px!important}.m-sm-n3{margin:-24px!important}.mt-sm-n3,.my-sm-n3{margin-top:-24px!important}.mr-sm-n3,.mx-sm-n3{margin-right:-24px!important}.mb-sm-n3,.my-sm-n3{margin-bottom:-24px!important}.ml-sm-n3,.mx-sm-n3{margin-left:-24px!important}.m-sm-n4{margin:-32px!important}.mt-sm-n4,.my-sm-n4{margin-top:-32px!important}.mr-sm-n4,.mx-sm-n4{margin-right:-32px!important}.mb-sm-n4,.my-sm-n4{margin-bottom:-32px!important}.ml-sm-n4,.mx-sm-n4{margin-left:-32px!important}.m-sm-n5{margin:-40px!important}.mt-sm-n5,.my-sm-n5{margin-top:-40px!important}.mr-sm-n5,.mx-sm-n5{margin-right:-40px!important}.mb-sm-n5,.my-sm-n5{margin-bottom:-40px!important}.ml-sm-n5,.mx-sm-n5{margin-left:-40px!important}.m-sm-n6{margin:-48px!important}.mt-sm-n6,.my-sm-n6{margin-top:-48px!important}.mr-sm-n6,.mx-sm-n6{margin-right:-48px!important}.mb-sm-n6,.my-sm-n6{margin-bottom:-48px!important}.ml-sm-n6,.mx-sm-n6{margin-left:-48px!important}.m-sm-n7{margin:-56px!important}.mt-sm-n7,.my-sm-n7{margin-top:-56px!important}.mr-sm-n7,.mx-sm-n7{margin-right:-56px!important}.mb-sm-n7,.my-sm-n7{margin-bottom:-56px!important}.ml-sm-n7,.mx-sm-n7{margin-left:-56px!important}.m-sm-n8{margin:-64px!important}.mt-sm-n8,.my-sm-n8{margin-top:-64px!important}.mr-sm-n8,.mx-sm-n8{margin-right:-64px!important}.mb-sm-n8,.my-sm-n8{margin-bottom:-64px!important}.ml-sm-n8,.mx-sm-n8{margin-left:-64px!important}.m-sm-n9{margin:-72px!important}.mt-sm-n9,.my-sm-n9{margin-top:-72px!important}.mr-sm-n9,.mx-sm-n9{margin-right:-72px!important}.mb-sm-n9,.my-sm-n9{margin-bottom:-72px!important}.ml-sm-n9,.mx-sm-n9{margin-left:-72px!important}.m-sm-n10{margin:-80px!important}.mt-sm-n10,.my-sm-n10{margin-top:-80px!important}.mr-sm-n10,.mx-sm-n10{margin-right:-80px!important}.mb-sm-n10,.my-sm-n10{margin-bottom:-80px!important}.ml-sm-n10,.mx-sm-n10{margin-left:-80px!important}.m-sm-n12{margin:-96px!important}.mt-sm-n12,.my-sm-n12{margin-top:-96px!important}.mr-sm-n12,.mx-sm-n12{margin-right:-96px!important}.mb-sm-n12,.my-sm-n12{margin-bottom:-96px!important}.ml-sm-n12,.mx-sm-n12{margin-left:-96px!important}.m-sm-n15{margin:-120px!important}.mt-sm-n15,.my-sm-n15{margin-top:-120px!important}.mr-sm-n15,.mx-sm-n15{margin-right:-120px!important}.mb-sm-n15,.my-sm-n15{margin-bottom:-120px!important}.ml-sm-n15,.mx-sm-n15{margin-left:-120px!important}.m-sm-auto{margin:auto!important}.mt-sm-auto,.my-sm-auto{margin-top:auto!important}.mr-sm-auto,.mx-sm-auto{margin-right:auto!important}.mb-sm-auto,.my-sm-auto{margin-bottom:auto!important}.ml-sm-auto,.mx-sm-auto{margin-left:auto!important}}@media(min-width:768px){.m-md-0{margin:0!important}.mt-md-0,.my-md-0{margin-top:0!important}.mr-md-0,.mx-md-0{margin-right:0!important}.mb-md-0,.my-md-0{margin-bottom:0!important}.ml-md-0,.mx-md-0{margin-left:0!important}.m-md-1{margin:8px!important}.mt-md-1,.my-md-1{margin-top:8px!important}.mr-md-1,.mx-md-1{margin-right:8px!important}.mb-md-1,.my-md-1{margin-bottom:8px!important}.ml-md-1,.mx-md-1{margin-left:8px!important}.m-md-2{margin:16px!important}.mt-md-2,.my-md-2{margin-top:16px!important}.mr-md-2,.mx-md-2{margin-right:16px!important}.mb-md-2,.my-md-2{margin-bottom:16px!important}.ml-md-2,.mx-md-2{margin-left:16px!important}.m-md-3{margin:24px!important}.mt-md-3,.my-md-3{margin-top:24px!important}.mr-md-3,.mx-md-3{margin-right:24px!important}.mb-md-3,.my-md-3{margin-bottom:24px!important}.ml-md-3,.mx-md-3{margin-left:24px!important}.m-md-4{margin:32px!important}.mt-md-4,.my-md-4{margin-top:32px!important}.mr-md-4,.mx-md-4{margin-right:32px!important}.mb-md-4,.my-md-4{margin-bottom:32px!important}.ml-md-4,.mx-md-4{margin-left:32px!important}.m-md-5{margin:40px!important}.mt-md-5,.my-md-5{margin-top:40px!important}.mr-md-5,.mx-md-5{margin-right:40px!important}.mb-md-5,.my-md-5{margin-bottom:40px!important}.ml-md-5,.mx-md-5{margin-left:40px!important}.m-md-6{margin:48px!important}.mt-md-6,.my-md-6{margin-top:48px!important}.mr-md-6,.mx-md-6{margin-right:48px!important}.mb-md-6,.my-md-6{margin-bottom:48px!important}.ml-md-6,.mx-md-6{margin-left:48px!important}.m-md-7{margin:56px!important}.mt-md-7,.my-md-7{margin-top:56px!important}.mr-md-7,.mx-md-7{margin-right:56px!important}.mb-md-7,.my-md-7{margin-bottom:56px!important}.ml-md-7,.mx-md-7{margin-left:56px!important}.m-md-8{margin:64px!important}.mt-md-8,.my-md-8{margin-top:64px!important}.mr-md-8,.mx-md-8{margin-right:64px!important}.mb-md-8,.my-md-8{margin-bottom:64px!important}.ml-md-8,.mx-md-8{margin-left:64px!important}.m-md-9{margin:72px!important}.mt-md-9,.my-md-9{margin-top:72px!important}.mr-md-9,.mx-md-9{margin-right:72px!important}.mb-md-9,.my-md-9{margin-bottom:72px!important}.ml-md-9,.mx-md-9{margin-left:72px!important}.m-md-10{margin:80px!important}.mt-md-10,.my-md-10{margin-top:80px!important}.mr-md-10,.mx-md-10{margin-right:80px!important}.mb-md-10,.my-md-10{margin-bottom:80px!important}.ml-md-10,.mx-md-10{margin-left:80px!important}.m-md-12{margin:96px!important}.mt-md-12,.my-md-12{margin-top:96px!important}.mr-md-12,.mx-md-12{margin-right:96px!important}.mb-md-12,.my-md-12{margin-bottom:96px!important}.ml-md-12,.mx-md-12{margin-left:96px!important}.m-md-15{margin:120px!important}.mt-md-15,.my-md-15{margin-top:120px!important}.mr-md-15,.mx-md-15{margin-right:120px!important}.mb-md-15,.my-md-15{margin-bottom:120px!important}.ml-md-15,.mx-md-15{margin-left:120px!important}.p-md-0{padding:0!important}.pt-md-0,.py-md-0{padding-top:0!important}.pr-md-0,.px-md-0{padding-right:0!important}.pb-md-0,.py-md-0{padding-bottom:0!important}.pl-md-0,.px-md-0{padding-left:0!important}.p-md-1{padding:8px!important}.pt-md-1,.py-md-1{padding-top:8px!important}.pr-md-1,.px-md-1{padding-right:8px!important}.pb-md-1,.py-md-1{padding-bottom:8px!important}.pl-md-1,.px-md-1{padding-left:8px!important}.p-md-2{padding:16px!important}.pt-md-2,.py-md-2{padding-top:16px!important}.pr-md-2,.px-md-2{padding-right:16px!important}.pb-md-2,.py-md-2{padding-bottom:16px!important}.pl-md-2,.px-md-2{padding-left:16px!important}.p-md-3{padding:24px!important}.pt-md-3,.py-md-3{padding-top:24px!important}.pr-md-3,.px-md-3{padding-right:24px!important}.pb-md-3,.py-md-3{padding-bottom:24px!important}.pl-md-3,.px-md-3{padding-left:24px!important}.p-md-4{padding:32px!important}.pt-md-4,.py-md-4{padding-top:32px!important}.pr-md-4,.px-md-4{padding-right:32px!important}.pb-md-4,.py-md-4{padding-bottom:32px!important}.pl-md-4,.px-md-4{padding-left:32px!important}.p-md-5{padding:40px!important}.pt-md-5,.py-md-5{padding-top:40px!important}.pr-md-5,.px-md-5{padding-right:40px!important}.pb-md-5,.py-md-5{padding-bottom:40px!important}.pl-md-5,.px-md-5{padding-left:40px!important}.p-md-6{padding:48px!important}.pt-md-6,.py-md-6{padding-top:48px!important}.pr-md-6,.px-md-6{padding-right:48px!important}.pb-md-6,.py-md-6{padding-bottom:48px!important}.pl-md-6,.px-md-6{padding-left:48px!important}.p-md-7{padding:56px!important}.pt-md-7,.py-md-7{padding-top:56px!important}.pr-md-7,.px-md-7{padding-right:56px!important}.pb-md-7,.py-md-7{padding-bottom:56px!important}.pl-md-7,.px-md-7{padding-left:56px!important}.p-md-8{padding:64px!important}.pt-md-8,.py-md-8{padding-top:64px!important}.pr-md-8,.px-md-8{padding-right:64px!important}.pb-md-8,.py-md-8{padding-bottom:64px!important}.pl-md-8,.px-md-8{padding-left:64px!important}.p-md-9{padding:72px!important}.pt-md-9,.py-md-9{padding-top:72px!important}.pr-md-9,.px-md-9{padding-right:72px!important}.pb-md-9,.py-md-9{padding-bottom:72px!important}.pl-md-9,.px-md-9{padding-left:72px!important}.p-md-10{padding:80px!important}.pt-md-10,.py-md-10{padding-top:80px!important}.pr-md-10,.px-md-10{padding-right:80px!important}.pb-md-10,.py-md-10{padding-bottom:80px!important}.pl-md-10,.px-md-10{padding-left:80px!important}.p-md-12{padding:96px!important}.pt-md-12,.py-md-12{padding-top:96px!important}.pr-md-12,.px-md-12{padding-right:96px!important}.pb-md-12,.py-md-12{padding-bottom:96px!important}.pl-md-12,.px-md-12{padding-left:96px!important}.p-md-15{padding:120px!important}.pt-md-15,.py-md-15{padding-top:120px!important}.pr-md-15,.px-md-15{padding-right:120px!important}.pb-md-15,.py-md-15{padding-bottom:120px!important}.pl-md-15,.px-md-15{padding-left:120px!important}.m-md-n1{margin:-8px!important}.mt-md-n1,.my-md-n1{margin-top:-8px!important}.mr-md-n1,.mx-md-n1{margin-right:-8px!important}.mb-md-n1,.my-md-n1{margin-bottom:-8px!important}.ml-md-n1,.mx-md-n1{margin-left:-8px!important}.m-md-n2{margin:-16px!important}.mt-md-n2,.my-md-n2{margin-top:-16px!important}.mr-md-n2,.mx-md-n2{margin-right:-16px!important}.mb-md-n2,.my-md-n2{margin-bottom:-16px!important}.ml-md-n2,.mx-md-n2{margin-left:-16px!important}.m-md-n3{margin:-24px!important}.mt-md-n3,.my-md-n3{margin-top:-24px!important}.mr-md-n3,.mx-md-n3{margin-right:-24px!important}.mb-md-n3,.my-md-n3{margin-bottom:-24px!important}.ml-md-n3,.mx-md-n3{margin-left:-24px!important}.m-md-n4{margin:-32px!important}.mt-md-n4,.my-md-n4{margin-top:-32px!important}.mr-md-n4,.mx-md-n4{margin-right:-32px!important}.mb-md-n4,.my-md-n4{margin-bottom:-32px!important}.ml-md-n4,.mx-md-n4{margin-left:-32px!important}.m-md-n5{margin:-40px!important}.mt-md-n5,.my-md-n5{margin-top:-40px!important}.mr-md-n5,.mx-md-n5{margin-right:-40px!important}.mb-md-n5,.my-md-n5{margin-bottom:-40px!important}.ml-md-n5,.mx-md-n5{margin-left:-40px!important}.m-md-n6{margin:-48px!important}.mt-md-n6,.my-md-n6{margin-top:-48px!important}.mr-md-n6,.mx-md-n6{margin-right:-48px!important}.mb-md-n6,.my-md-n6{margin-bottom:-48px!important}.ml-md-n6,.mx-md-n6{margin-left:-48px!important}.m-md-n7{margin:-56px!important}.mt-md-n7,.my-md-n7{margin-top:-56px!important}.mr-md-n7,.mx-md-n7{margin-right:-56px!important}.mb-md-n7,.my-md-n7{margin-bottom:-56px!important}.ml-md-n7,.mx-md-n7{margin-left:-56px!important}.m-md-n8{margin:-64px!important}.mt-md-n8,.my-md-n8{margin-top:-64px!important}.mr-md-n8,.mx-md-n8{margin-right:-64px!important}.mb-md-n8,.my-md-n8{margin-bottom:-64px!important}.ml-md-n8,.mx-md-n8{margin-left:-64px!important}.m-md-n9{margin:-72px!important}.mt-md-n9,.my-md-n9{margin-top:-72px!important}.mr-md-n9,.mx-md-n9{margin-right:-72px!important}.mb-md-n9,.my-md-n9{margin-bottom:-72px!important}.ml-md-n9,.mx-md-n9{margin-left:-72px!important}.m-md-n10{margin:-80px!important}.mt-md-n10,.my-md-n10{margin-top:-80px!important}.mr-md-n10,.mx-md-n10{margin-right:-80px!important}.mb-md-n10,.my-md-n10{margin-bottom:-80px!important}.ml-md-n10,.mx-md-n10{margin-left:-80px!important}.m-md-n12{margin:-96px!important}.mt-md-n12,.my-md-n12{margin-top:-96px!important}.mr-md-n12,.mx-md-n12{margin-right:-96px!important}.mb-md-n12,.my-md-n12{margin-bottom:-96px!important}.ml-md-n12,.mx-md-n12{margin-left:-96px!important}.m-md-n15{margin:-120px!important}.mt-md-n15,.my-md-n15{margin-top:-120px!important}.mr-md-n15,.mx-md-n15{margin-right:-120px!important}.mb-md-n15,.my-md-n15{margin-bottom:-120px!important}.ml-md-n15,.mx-md-n15{margin-left:-120px!important}.m-md-auto{margin:auto!important}.mt-md-auto,.my-md-auto{margin-top:auto!important}.mr-md-auto,.mx-md-auto{margin-right:auto!important}.mb-md-auto,.my-md-auto{margin-bottom:auto!important}.ml-md-auto,.mx-md-auto{margin-left:auto!important}}@media(min-width:980px){.m-lg-0{margin:0!important}.mt-lg-0,.my-lg-0{margin-top:0!important}.mr-lg-0,.mx-lg-0{margin-right:0!important}.mb-lg-0,.my-lg-0{margin-bottom:0!important}.ml-lg-0,.mx-lg-0{margin-left:0!important}.m-lg-1{margin:8px!important}.mt-lg-1,.my-lg-1{margin-top:8px!important}.mr-lg-1,.mx-lg-1{margin-right:8px!important}.mb-lg-1,.my-lg-1{margin-bottom:8px!important}.ml-lg-1,.mx-lg-1{margin-left:8px!important}.m-lg-2{margin:16px!important}.mt-lg-2,.my-lg-2{margin-top:16px!important}.mr-lg-2,.mx-lg-2{margin-right:16px!important}.mb-lg-2,.my-lg-2{margin-bottom:16px!important}.ml-lg-2,.mx-lg-2{margin-left:16px!important}.m-lg-3{margin:24px!important}.mt-lg-3,.my-lg-3{margin-top:24px!important}.mr-lg-3,.mx-lg-3{margin-right:24px!important}.mb-lg-3,.my-lg-3{margin-bottom:24px!important}.ml-lg-3,.mx-lg-3{margin-left:24px!important}.m-lg-4{margin:32px!important}.mt-lg-4,.my-lg-4{margin-top:32px!important}.mr-lg-4,.mx-lg-4{margin-right:32px!important}.mb-lg-4,.my-lg-4{margin-bottom:32px!important}.ml-lg-4,.mx-lg-4{margin-left:32px!important}.m-lg-5{margin:40px!important}.mt-lg-5,.my-lg-5{margin-top:40px!important}.mr-lg-5,.mx-lg-5{margin-right:40px!important}.mb-lg-5,.my-lg-5{margin-bottom:40px!important}.ml-lg-5,.mx-lg-5{margin-left:40px!important}.m-lg-6{margin:48px!important}.mt-lg-6,.my-lg-6{margin-top:48px!important}.mr-lg-6,.mx-lg-6{margin-right:48px!important}.mb-lg-6,.my-lg-6{margin-bottom:48px!important}.ml-lg-6,.mx-lg-6{margin-left:48px!important}.m-lg-7{margin:56px!important}.mt-lg-7,.my-lg-7{margin-top:56px!important}.mr-lg-7,.mx-lg-7{margin-right:56px!important}.mb-lg-7,.my-lg-7{margin-bottom:56px!important}.ml-lg-7,.mx-lg-7{margin-left:56px!important}.m-lg-8{margin:64px!important}.mt-lg-8,.my-lg-8{margin-top:64px!important}.mr-lg-8,.mx-lg-8{margin-right:64px!important}.mb-lg-8,.my-lg-8{margin-bottom:64px!important}.ml-lg-8,.mx-lg-8{margin-left:64px!important}.m-lg-9{margin:72px!important}.mt-lg-9,.my-lg-9{margin-top:72px!important}.mr-lg-9,.mx-lg-9{margin-right:72px!important}.mb-lg-9,.my-lg-9{margin-bottom:72px!important}.ml-lg-9,.mx-lg-9{margin-left:72px!important}.m-lg-10{margin:80px!important}.mt-lg-10,.my-lg-10{margin-top:80px!important}.mr-lg-10,.mx-lg-10{margin-right:80px!important}.mb-lg-10,.my-lg-10{margin-bottom:80px!important}.ml-lg-10,.mx-lg-10{margin-left:80px!important}.m-lg-12{margin:96px!important}.mt-lg-12,.my-lg-12{margin-top:96px!important}.mr-lg-12,.mx-lg-12{margin-right:96px!important}.mb-lg-12,.my-lg-12{margin-bottom:96px!important}.ml-lg-12,.mx-lg-12{margin-left:96px!important}.m-lg-15{margin:120px!important}.mt-lg-15,.my-lg-15{margin-top:120px!important}.mr-lg-15,.mx-lg-15{margin-right:120px!important}.mb-lg-15,.my-lg-15{margin-bottom:120px!important}.ml-lg-15,.mx-lg-15{margin-left:120px!important}.p-lg-0{padding:0!important}.pt-lg-0,.py-lg-0{padding-top:0!important}.pr-lg-0,.px-lg-0{padding-right:0!important}.pb-lg-0,.py-lg-0{padding-bottom:0!important}.pl-lg-0,.px-lg-0{padding-left:0!important}.p-lg-1{padding:8px!important}.pt-lg-1,.py-lg-1{padding-top:8px!important}.pr-lg-1,.px-lg-1{padding-right:8px!important}.pb-lg-1,.py-lg-1{padding-bottom:8px!important}.pl-lg-1,.px-lg-1{padding-left:8px!important}.p-lg-2{padding:16px!important}.pt-lg-2,.py-lg-2{padding-top:16px!important}.pr-lg-2,.px-lg-2{padding-right:16px!important}.pb-lg-2,.py-lg-2{padding-bottom:16px!important}.pl-lg-2,.px-lg-2{padding-left:16px!important}.p-lg-3{padding:24px!important}.pt-lg-3,.py-lg-3{padding-top:24px!important}.pr-lg-3,.px-lg-3{padding-right:24px!important}.pb-lg-3,.py-lg-3{padding-bottom:24px!important}.pl-lg-3,.px-lg-3{padding-left:24px!important}.p-lg-4{padding:32px!important}.pt-lg-4,.py-lg-4{padding-top:32px!important}.pr-lg-4,.px-lg-4{padding-right:32px!important}.pb-lg-4,.py-lg-4{padding-bottom:32px!important}.pl-lg-4,.px-lg-4{padding-left:32px!important}.p-lg-5{padding:40px!important}.pt-lg-5,.py-lg-5{padding-top:40px!important}.pr-lg-5,.px-lg-5{padding-right:40px!important}.pb-lg-5,.py-lg-5{padding-bottom:40px!important}.pl-lg-5,.px-lg-5{padding-left:40px!important}.p-lg-6{padding:48px!important}.pt-lg-6,.py-lg-6{padding-top:48px!important}.pr-lg-6,.px-lg-6{padding-right:48px!important}.pb-lg-6,.py-lg-6{padding-bottom:48px!important}.pl-lg-6,.px-lg-6{padding-left:48px!important}.p-lg-7{padding:56px!important}.pt-lg-7,.py-lg-7{padding-top:56px!important}.pr-lg-7,.px-lg-7{padding-right:56px!important}.pb-lg-7,.py-lg-7{padding-bottom:56px!important}.pl-lg-7,.px-lg-7{padding-left:56px!important}.p-lg-8{padding:64px!important}.pt-lg-8,.py-lg-8{padding-top:64px!important}.pr-lg-8,.px-lg-8{padding-right:64px!important}.pb-lg-8,.py-lg-8{padding-bottom:64px!important}.pl-lg-8,.px-lg-8{padding-left:64px!important}.p-lg-9{padding:72px!important}.pt-lg-9,.py-lg-9{padding-top:72px!important}.pr-lg-9,.px-lg-9{padding-right:72px!important}.pb-lg-9,.py-lg-9{padding-bottom:72px!important}.pl-lg-9,.px-lg-9{padding-left:72px!important}.p-lg-10{padding:80px!important}.pt-lg-10,.py-lg-10{padding-top:80px!important}.pr-lg-10,.px-lg-10{padding-right:80px!important}.pb-lg-10,.py-lg-10{padding-bottom:80px!important}.pl-lg-10,.px-lg-10{padding-left:80px!important}.p-lg-12{padding:96px!important}.pt-lg-12,.py-lg-12{padding-top:96px!important}.pr-lg-12,.px-lg-12{padding-right:96px!important}.pb-lg-12,.py-lg-12{padding-bottom:96px!important}.pl-lg-12,.px-lg-12{padding-left:96px!important}.p-lg-15{padding:120px!important}.pt-lg-15,.py-lg-15{padding-top:120px!important}.pr-lg-15,.px-lg-15{padding-right:120px!important}.pb-lg-15,.py-lg-15{padding-bottom:120px!important}.pl-lg-15,.px-lg-15{padding-left:120px!important}.m-lg-n1{margin:-8px!important}.mt-lg-n1,.my-lg-n1{margin-top:-8px!important}.mr-lg-n1,.mx-lg-n1{margin-right:-8px!important}.mb-lg-n1,.my-lg-n1{margin-bottom:-8px!important}.ml-lg-n1,.mx-lg-n1{margin-left:-8px!important}.m-lg-n2{margin:-16px!important}.mt-lg-n2,.my-lg-n2{margin-top:-16px!important}.mr-lg-n2,.mx-lg-n2{margin-right:-16px!important}.mb-lg-n2,.my-lg-n2{margin-bottom:-16px!important}.ml-lg-n2,.mx-lg-n2{margin-left:-16px!important}.m-lg-n3{margin:-24px!important}.mt-lg-n3,.my-lg-n3{margin-top:-24px!important}.mr-lg-n3,.mx-lg-n3{margin-right:-24px!important}.mb-lg-n3,.my-lg-n3{margin-bottom:-24px!important}.ml-lg-n3,.mx-lg-n3{margin-left:-24px!important}.m-lg-n4{margin:-32px!important}.mt-lg-n4,.my-lg-n4{margin-top:-32px!important}.mr-lg-n4,.mx-lg-n4{margin-right:-32px!important}.mb-lg-n4,.my-lg-n4{margin-bottom:-32px!important}.ml-lg-n4,.mx-lg-n4{margin-left:-32px!important}.m-lg-n5{margin:-40px!important}.mt-lg-n5,.my-lg-n5{margin-top:-40px!important}.mr-lg-n5,.mx-lg-n5{margin-right:-40px!important}.mb-lg-n5,.my-lg-n5{margin-bottom:-40px!important}.ml-lg-n5,.mx-lg-n5{margin-left:-40px!important}.m-lg-n6{margin:-48px!important}.mt-lg-n6,.my-lg-n6{margin-top:-48px!important}.mr-lg-n6,.mx-lg-n6{margin-right:-48px!important}.mb-lg-n6,.my-lg-n6{margin-bottom:-48px!important}.ml-lg-n6,.mx-lg-n6{margin-left:-48px!important}.m-lg-n7{margin:-56px!important}.mt-lg-n7,.my-lg-n7{margin-top:-56px!important}.mr-lg-n7,.mx-lg-n7{margin-right:-56px!important}.mb-lg-n7,.my-lg-n7{margin-bottom:-56px!important}.ml-lg-n7,.mx-lg-n7{margin-left:-56px!important}.m-lg-n8{margin:-64px!important}.mt-lg-n8,.my-lg-n8{margin-top:-64px!important}.mr-lg-n8,.mx-lg-n8{margin-right:-64px!important}.mb-lg-n8,.my-lg-n8{margin-bottom:-64px!important}.ml-lg-n8,.mx-lg-n8{margin-left:-64px!important}.m-lg-n9{margin:-72px!important}.mt-lg-n9,.my-lg-n9{margin-top:-72px!important}.mr-lg-n9,.mx-lg-n9{margin-right:-72px!important}.mb-lg-n9,.my-lg-n9{margin-bottom:-72px!important}.ml-lg-n9,.mx-lg-n9{margin-left:-72px!important}.m-lg-n10{margin:-80px!important}.mt-lg-n10,.my-lg-n10{margin-top:-80px!important}.mr-lg-n10,.mx-lg-n10{margin-right:-80px!important}.mb-lg-n10,.my-lg-n10{margin-bottom:-80px!important}.ml-lg-n10,.mx-lg-n10{margin-left:-80px!important}.m-lg-n12{margin:-96px!important}.mt-lg-n12,.my-lg-n12{margin-top:-96px!important}.mr-lg-n12,.mx-lg-n12{margin-right:-96px!important}.mb-lg-n12,.my-lg-n12{margin-bottom:-96px!important}.ml-lg-n12,.mx-lg-n12{margin-left:-96px!important}.m-lg-n15{margin:-120px!important}.mt-lg-n15,.my-lg-n15{margin-top:-120px!important}.mr-lg-n15,.mx-lg-n15{margin-right:-120px!important}.mb-lg-n15,.my-lg-n15{margin-bottom:-120px!important}.ml-lg-n15,.mx-lg-n15{margin-left:-120px!important}.m-lg-auto{margin:auto!important}.mt-lg-auto,.my-lg-auto{margin-top:auto!important}.mr-lg-auto,.mx-lg-auto{margin-right:auto!important}.mb-lg-auto,.my-lg-auto{margin-bottom:auto!important}.ml-lg-auto,.mx-lg-auto{margin-left:auto!important}}@media(min-width:1240px){.m-xl-0{margin:0!important}.mt-xl-0,.my-xl-0{margin-top:0!important}.mr-xl-0,.mx-xl-0{margin-right:0!important}.mb-xl-0,.my-xl-0{margin-bottom:0!important}.ml-xl-0,.mx-xl-0{margin-left:0!important}.m-xl-1{margin:8px!important}.mt-xl-1,.my-xl-1{margin-top:8px!important}.mr-xl-1,.mx-xl-1{margin-right:8px!important}.mb-xl-1,.my-xl-1{margin-bottom:8px!important}.ml-xl-1,.mx-xl-1{margin-left:8px!important}.m-xl-2{margin:16px!important}.mt-xl-2,.my-xl-2{margin-top:16px!important}.mr-xl-2,.mx-xl-2{margin-right:16px!important}.mb-xl-2,.my-xl-2{margin-bottom:16px!important}.ml-xl-2,.mx-xl-2{margin-left:16px!important}.m-xl-3{margin:24px!important}.mt-xl-3,.my-xl-3{margin-top:24px!important}.mr-xl-3,.mx-xl-3{margin-right:24px!important}.mb-xl-3,.my-xl-3{margin-bottom:24px!important}.ml-xl-3,.mx-xl-3{margin-left:24px!important}.m-xl-4{margin:32px!important}.mt-xl-4,.my-xl-4{margin-top:32px!important}.mr-xl-4,.mx-xl-4{margin-right:32px!important}.mb-xl-4,.my-xl-4{margin-bottom:32px!important}.ml-xl-4,.mx-xl-4{margin-left:32px!important}.m-xl-5{margin:40px!important}.mt-xl-5,.my-xl-5{margin-top:40px!important}.mr-xl-5,.mx-xl-5{margin-right:40px!important}.mb-xl-5,.my-xl-5{margin-bottom:40px!important}.ml-xl-5,.mx-xl-5{margin-left:40px!important}.m-xl-6{margin:48px!important}.mt-xl-6,.my-xl-6{margin-top:48px!important}.mr-xl-6,.mx-xl-6{margin-right:48px!important}.mb-xl-6,.my-xl-6{margin-bottom:48px!important}.ml-xl-6,.mx-xl-6{margin-left:48px!important}.m-xl-7{margin:56px!important}.mt-xl-7,.my-xl-7{margin-top:56px!important}.mr-xl-7,.mx-xl-7{margin-right:56px!important}.mb-xl-7,.my-xl-7{margin-bottom:56px!important}.ml-xl-7,.mx-xl-7{margin-left:56px!important}.m-xl-8{margin:64px!important}.mt-xl-8,.my-xl-8{margin-top:64px!important}.mr-xl-8,.mx-xl-8{margin-right:64px!important}.mb-xl-8,.my-xl-8{margin-bottom:64px!important}.ml-xl-8,.mx-xl-8{margin-left:64px!important}.m-xl-9{margin:72px!important}.mt-xl-9,.my-xl-9{margin-top:72px!important}.mr-xl-9,.mx-xl-9{margin-right:72px!important}.mb-xl-9,.my-xl-9{margin-bottom:72px!important}.ml-xl-9,.mx-xl-9{margin-left:72px!important}.m-xl-10{margin:80px!important}.mt-xl-10,.my-xl-10{margin-top:80px!important}.mr-xl-10,.mx-xl-10{margin-right:80px!important}.mb-xl-10,.my-xl-10{margin-bottom:80px!important}.ml-xl-10,.mx-xl-10{margin-left:80px!important}.m-xl-12{margin:96px!important}.mt-xl-12,.my-xl-12{margin-top:96px!important}.mr-xl-12,.mx-xl-12{margin-right:96px!important}.mb-xl-12,.my-xl-12{margin-bottom:96px!important}.ml-xl-12,.mx-xl-12{margin-left:96px!important}.m-xl-15{margin:120px!important}.mt-xl-15,.my-xl-15{margin-top:120px!important}.mr-xl-15,.mx-xl-15{margin-right:120px!important}.mb-xl-15,.my-xl-15{margin-bottom:120px!important}.ml-xl-15,.mx-xl-15{margin-left:120px!important}.p-xl-0{padding:0!important}.pt-xl-0,.py-xl-0{padding-top:0!important}.pr-xl-0,.px-xl-0{padding-right:0!important}.pb-xl-0,.py-xl-0{padding-bottom:0!important}.pl-xl-0,.px-xl-0{padding-left:0!important}.p-xl-1{padding:8px!important}.pt-xl-1,.py-xl-1{padding-top:8px!important}.pr-xl-1,.px-xl-1{padding-right:8px!important}.pb-xl-1,.py-xl-1{padding-bottom:8px!important}.pl-xl-1,.px-xl-1{padding-left:8px!important}.p-xl-2{padding:16px!important}.pt-xl-2,.py-xl-2{padding-top:16px!important}.pr-xl-2,.px-xl-2{padding-right:16px!important}.pb-xl-2,.py-xl-2{padding-bottom:16px!important}.pl-xl-2,.px-xl-2{padding-left:16px!important}.p-xl-3{padding:24px!important}.pt-xl-3,.py-xl-3{padding-top:24px!important}.pr-xl-3,.px-xl-3{padding-right:24px!important}.pb-xl-3,.py-xl-3{padding-bottom:24px!important}.pl-xl-3,.px-xl-3{padding-left:24px!important}.p-xl-4{padding:32px!important}.pt-xl-4,.py-xl-4{padding-top:32px!important}.pr-xl-4,.px-xl-4{padding-right:32px!important}.pb-xl-4,.py-xl-4{padding-bottom:32px!important}.pl-xl-4,.px-xl-4{padding-left:32px!important}.p-xl-5{padding:40px!important}.pt-xl-5,.py-xl-5{padding-top:40px!important}.pr-xl-5,.px-xl-5{padding-right:40px!important}.pb-xl-5,.py-xl-5{padding-bottom:40px!important}.pl-xl-5,.px-xl-5{padding-left:40px!important}.p-xl-6{padding:48px!important}.pt-xl-6,.py-xl-6{padding-top:48px!important}.pr-xl-6,.px-xl-6{padding-right:48px!important}.pb-xl-6,.py-xl-6{padding-bottom:48px!important}.pl-xl-6,.px-xl-6{padding-left:48px!important}.p-xl-7{padding:56px!important}.pt-xl-7,.py-xl-7{padding-top:56px!important}.pr-xl-7,.px-xl-7{padding-right:56px!important}.pb-xl-7,.py-xl-7{padding-bottom:56px!important}.pl-xl-7,.px-xl-7{padding-left:56px!important}.p-xl-8{padding:64px!important}.pt-xl-8,.py-xl-8{padding-top:64px!important}.pr-xl-8,.px-xl-8{padding-right:64px!important}.pb-xl-8,.py-xl-8{padding-bottom:64px!important}.pl-xl-8,.px-xl-8{padding-left:64px!important}.p-xl-9{padding:72px!important}.pt-xl-9,.py-xl-9{padding-top:72px!important}.pr-xl-9,.px-xl-9{padding-right:72px!important}.pb-xl-9,.py-xl-9{padding-bottom:72px!important}.pl-xl-9,.px-xl-9{padding-left:72px!important}.p-xl-10{padding:80px!important}.pt-xl-10,.py-xl-10{padding-top:80px!important}.pr-xl-10,.px-xl-10{padding-right:80px!important}.pb-xl-10,.py-xl-10{padding-bottom:80px!important}.pl-xl-10,.px-xl-10{padding-left:80px!important}.p-xl-12{padding:96px!important}.pt-xl-12,.py-xl-12{padding-top:96px!important}.pr-xl-12,.px-xl-12{padding-right:96px!important}.pb-xl-12,.py-xl-12{padding-bottom:96px!important}.pl-xl-12,.px-xl-12{padding-left:96px!important}.p-xl-15{padding:120px!important}.pt-xl-15,.py-xl-15{padding-top:120px!important}.pr-xl-15,.px-xl-15{padding-right:120px!important}.pb-xl-15,.py-xl-15{padding-bottom:120px!important}.pl-xl-15,.px-xl-15{padding-left:120px!important}.m-xl-n1{margin:-8px!important}.mt-xl-n1,.my-xl-n1{margin-top:-8px!important}.mr-xl-n1,.mx-xl-n1{margin-right:-8px!important}.mb-xl-n1,.my-xl-n1{margin-bottom:-8px!important}.ml-xl-n1,.mx-xl-n1{margin-left:-8px!important}.m-xl-n2{margin:-16px!important}.mt-xl-n2,.my-xl-n2{margin-top:-16px!important}.mr-xl-n2,.mx-xl-n2{margin-right:-16px!important}.mb-xl-n2,.my-xl-n2{margin-bottom:-16px!important}.ml-xl-n2,.mx-xl-n2{margin-left:-16px!important}.m-xl-n3{margin:-24px!important}.mt-xl-n3,.my-xl-n3{margin-top:-24px!important}.mr-xl-n3,.mx-xl-n3{margin-right:-24px!important}.mb-xl-n3,.my-xl-n3{margin-bottom:-24px!important}.ml-xl-n3,.mx-xl-n3{margin-left:-24px!important}.m-xl-n4{margin:-32px!important}.mt-xl-n4,.my-xl-n4{margin-top:-32px!important}.mr-xl-n4,.mx-xl-n4{margin-right:-32px!important}.mb-xl-n4,.my-xl-n4{margin-bottom:-32px!important}.ml-xl-n4,.mx-xl-n4{margin-left:-32px!important}.m-xl-n5{margin:-40px!important}.mt-xl-n5,.my-xl-n5{margin-top:-40px!important}.mr-xl-n5,.mx-xl-n5{margin-right:-40px!important}.mb-xl-n5,.my-xl-n5{margin-bottom:-40px!important}.ml-xl-n5,.mx-xl-n5{margin-left:-40px!important}.m-xl-n6{margin:-48px!important}.mt-xl-n6,.my-xl-n6{margin-top:-48px!important}.mr-xl-n6,.mx-xl-n6{margin-right:-48px!important}.mb-xl-n6,.my-xl-n6{margin-bottom:-48px!important}.ml-xl-n6,.mx-xl-n6{margin-left:-48px!important}.m-xl-n7{margin:-56px!important}.mt-xl-n7,.my-xl-n7{margin-top:-56px!important}.mr-xl-n7,.mx-xl-n7{margin-right:-56px!important}.mb-xl-n7,.my-xl-n7{margin-bottom:-56px!important}.ml-xl-n7,.mx-xl-n7{margin-left:-56px!important}.m-xl-n8{margin:-64px!important}.mt-xl-n8,.my-xl-n8{margin-top:-64px!important}.mr-xl-n8,.mx-xl-n8{margin-right:-64px!important}.mb-xl-n8,.my-xl-n8{margin-bottom:-64px!important}.ml-xl-n8,.mx-xl-n8{margin-left:-64px!important}.m-xl-n9{margin:-72px!important}.mt-xl-n9,.my-xl-n9{margin-top:-72px!important}.mr-xl-n9,.mx-xl-n9{margin-right:-72px!important}.mb-xl-n9,.my-xl-n9{margin-bottom:-72px!important}.ml-xl-n9,.mx-xl-n9{margin-left:-72px!important}.m-xl-n10{margin:-80px!important}.mt-xl-n10,.my-xl-n10{margin-top:-80px!important}.mr-xl-n10,.mx-xl-n10{margin-right:-80px!important}.mb-xl-n10,.my-xl-n10{margin-bottom:-80px!important}.ml-xl-n10,.mx-xl-n10{margin-left:-80px!important}.m-xl-n12{margin:-96px!important}.mt-xl-n12,.my-xl-n12{margin-top:-96px!important}.mr-xl-n12,.mx-xl-n12{margin-right:-96px!important}.mb-xl-n12,.my-xl-n12{margin-bottom:-96px!important}.ml-xl-n12,.mx-xl-n12{margin-left:-96px!important}.m-xl-n15{margin:-120px!important}.mt-xl-n15,.my-xl-n15{margin-top:-120px!important}.mr-xl-n15,.mx-xl-n15{margin-right:-120px!important}.mb-xl-n15,.my-xl-n15{margin-bottom:-120px!important}.ml-xl-n15,.mx-xl-n15{margin-left:-120px!important}.m-xl-auto{margin:auto!important}.mt-xl-auto,.my-xl-auto{margin-top:auto!important}.mr-xl-auto,.mx-xl-auto{margin-right:auto!important}.mb-xl-auto,.my-xl-auto{margin-bottom:auto!important}.ml-xl-auto,.mx-xl-auto{margin-left:auto!important}}.text-monospace{font-family:SFMono-Regular,Menlo,Monaco,Consolas,Liberation Mono,Courier New,monospace!important}.text-justify{text-align:justify!important}.text-wrap{white-space:normal!important}.text-nowrap{white-space:nowrap!important}.text-truncate{overflow:hidden;text-overflow:ellipsis;white-space:nowrap}.text-left{text-align:left!important}.text-right{text-align:right!important}.text-center{text-align:center!important}@media(min-width:400px){.text-xs-left{text-align:left!important}.text-xs-right{text-align:right!important}.text-xs-center{text-align:center!important}}@media(min-width:616px){.text-sm-left{text-align:left!important}.text-sm-right{text-align:right!important}.text-sm-center{text-align:center!important}}@media(min-width:768px){.text-md-left{text-align:left!important}.text-md-right{text-align:right!important}.text-md-center{text-align:center!important}}@media(min-width:980px){.text-lg-left{text-align:left!important}.text-lg-right{text-align:right!important}.text-lg-center{text-align:center!important}}@media(min-width:1240px){.text-xl-left{text-align:left!important}.text-xl-right{text-align:right!important}.text-xl-center{text-align:center!important}}.text-lowercase{text-transform:lowercase!important}.text-uppercase{text-transform:uppercase!important}.text-capitalize{text-transform:capitalize!important}.font-weight-light{font-weight:300!important}.font-weight-lighter{font-weight:lighter!important}.font-weight-normal{font-weight:400!important}.font-weight-bold{font-weight:700!important}.font-weight-bolder{font-weight:bolder!important}.font-italic{font-style:italic!important}.text-primary{color:#fc0!important}a.text-primary:focus,a.text-primary:hover{color:#b38f00!important}.text-secondary{color:#212529!important}a.text-secondary:focus,a.text-secondary:hover{color:#000!important}.text-success{color:#28a745!important}a.text-success:focus,a.text-success:hover{color:#19692c!important}.text-info{color:#17a2b8!important}a.text-info:focus,a.text-info:hover{color:#0f6674!important}.text-warning{color:#ffc107!important}a.text-warning:focus,a.text-warning:hover{color:#ba8b00!important}.text-danger{color:#dc3545!important}a.text-danger:focus,a.text-danger:hover{color:#a71d2a!important}.text-light{color:#f1f6f9!important}a.text-light:focus,a.text-light:hover{color:#bbd4e2!important}.text-dark{color:#495057!important}a.text-dark:focus,a.text-dark:hover{color:#262a2d!important}.text-primary-light{color:#fffaf0!important}a.text-primary-light:focus,a.text-primary-light:hover{color:#ffe1a4!important}.text-secondary-light{color:#fff!important}a.text-secondary-light:focus,a.text-secondary-light:hover{color:#d9d9d9!important}.text-tertiary{color:#257af4!important}a.text-tertiary:focus,a.text-tertiary:hover{color:#0a56c3!important}.text-tertiary-light{color:#e3f1fe!important}a.text-tertiary-light:focus,a.text-tertiary-light:hover{color:#99ccfb!important}.text-white{color:#fff!important}a.text-white:focus,a.text-white:hover{color:#d9d9d9!important}.text-black{color:#212529!important}a.text-black:focus,a.text-black:hover{color:#000!important}.text-blue{color:#257af4!important}a.text-blue:focus,a.text-blue:hover{color:#0a56c3!important}.text-light-blue{color:#e3f1fe!important}a.text-light-blue:focus,a.text-light-blue:hover{color:#99ccfb!important}.text-yellow{color:#fc0!important}a.text-yellow:focus,a.text-yellow:hover{color:#b38f00!important}.text-light-yellow{color:#fffaf0!important}a.text-light-yellow:focus,a.text-light-yellow:hover{color:#ffe1a4!important}.text-orange{color:#ff8c00!important}a.text-orange:focus,a.text-orange:hover{color:#b36200!important}.text-light-orange{color:#ffe4b5!important}a.text-light-orange:focus,a.text-light-orange:hover{color:#ffc869!important}.text-red{color:#ff3939!important}a.text-red:focus,a.text-red:hover{color:#ec0000!important}.text-light-red{color:#ffe4e1!important}a.text-light-red:focus,a.text-light-red:hover{color:#ff9f95!important}.text-medium{color:#d6dbdf!important}a.text-medium:focus,a.text-medium:hover{color:#abb5bd!important}.text-body{color:#212529!important}.text-muted{color:#6c757d!important}.text-black-50{color:rgba(33,37,41,.5)!important}.text-white-50{color:hsla(0,0%,100%,.5)!important}.text-hide{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.text-decoration-none{text-decoration:none!important}.text-break{word-break:break-word!important;overflow-wrap:break-word!important}.text-reset{color:inherit!important}.visible{visibility:visible!important}.invisible{visibility:hidden!important}@media print{*,:after,:before{text-shadow:none!important;box-shadow:none!important}a:not(.btn){text-decoration:underline}abbr[title]:after{content:" (" attr(title) ")"}pre{white-space:pre-wrap!important}blockquote,pre{border:1px solid #d6dbdf;page-break-inside:avoid}thead{display:table-header-group}img,tr{page-break-inside:avoid}h2,h3,p{orphans:3;widows:3}h2,h3{page-break-after:avoid}@page{size:a3}.container,body{min-width:980px!important}.navbar{display:none}.badge{border:1px solid #212529}.table{border-collapse:collapse!important}.table td,.table th{background-color:#fff!important}.table-bordered td,.table-bordered th{border:1px solid #dee2e6!important}.table-dark{color:inherit}.table-dark tbody+tbody,.table-dark td,.table-dark th,.table-dark thead th{border-color:#d6dbdf}.table .thead-dark th{color:inherit;border-color:#d6dbdf}} \ No newline at end of file + */:root{--blue:#007bff;--indigo:#6610f2;--purple:#6f42c1;--pink:#e83e8c;--red:#dc3545;--orange:#fd7e14;--yellow:#ffc107;--green:#28a745;--teal:#20c997;--cyan:#17a2b8;--gray:#6c757d;--gray-dark:#343a40;--brand-primary:#fc0;--brand-secondary:#ff3939;--primary-accent-yellow:#fc0;--primary-accent-light-yellow:#fffaf0;--primary-accent-blue:#257af4;--primary-accent-light-blue:#e3f1fe;--secondary-accent-orange:#ff8c00;--secondary-accent-light-orange:#ffe4b5;--secondary-accent-red:#ff3939;--secondary-accent-light-red:#ffe4e1;--primary:#fc0;--secondary:#212529;--success:#28a745;--info:#17a2b8;--warning:#ffc107;--danger:#dc3545;--light:#f1f6f9;--dark:#495057;--primary-light:#fffaf0;--secondary-light:#fff;--tertiary:#257af4;--tertiary-light:#e3f1fe;--white:#fff;--black:#212529;--blue:#257af4;--light-blue:#e3f1fe;--yellow:#fc0;--light-yellow:#fffaf0;--orange:#ff8c00;--light-orange:#ffe4b5;--red:#ff3939;--light-red:#ffe4e1;--medium:#d6dbdf;--breakpoint-xxs:0;--breakpoint-xs:400px;--breakpoint-sm:616px;--breakpoint-md:768px;--breakpoint-lg:980px;--breakpoint-xl:1240px;--font-family-sans-serif:"Noto Sans",sans-serif;--font-family-monospace:SFMono-Regular,Menlo,Monaco,Consolas,"Liberation Mono","Courier New",monospace}*,:after,:before{box-sizing:border-box}html{font-family:sans-serif;line-height:1.15;-webkit-text-size-adjust:100%;-webkit-tap-highlight-color:rgba(33,37,41,0)}article,aside,figcaption,figure,footer,header,hgroup,main,nav,section{display:block}body{margin:0;font-family:Noto Sans,sans-serif;font-size:1rem;font-weight:400;line-height:1.5;color:#212529;text-align:left;background-color:#fff}[tabindex="-1"]:focus:not(:focus-visible){outline:0!important}hr{box-sizing:content-box;height:0;overflow:visible}h1,h2,h3,h4,h5,h6{margin-top:0;margin-bottom:16px}p{margin-top:0;margin-bottom:1rem}abbr[data-original-title],abbr[title]{text-decoration:underline;-webkit-text-decoration:underline dotted;text-decoration:underline dotted;cursor:help;border-bottom:0;-webkit-text-decoration-skip-ink:none;text-decoration-skip-ink:none}address{font-style:normal;line-height:inherit}address,dl,ol,ul{margin-bottom:1rem}dl,ol,ul{margin-top:0}ol ol,ol ul,ul ol,ul ul{margin-bottom:0}dt{font-weight:700}dd{margin-bottom:.5rem;margin-left:0}blockquote{margin:0 0 1rem}b,strong{font-weight:bolder}small{font-size:80%}sub,sup{position:relative;font-size:75%;line-height:0;vertical-align:baseline}sub{bottom:-.25em}sup{top:-.5em}a{text-decoration:none;background-color:transparent}a,a:hover{color:#ff8c00}a:hover{text-decoration:underline}a:not([href]),a:not([href]):hover{color:inherit;text-decoration:none}code,kbd,pre,samp{font-family:SFMono-Regular,Menlo,Monaco,Consolas,Liberation Mono,Courier New,monospace;font-size:1em}pre{margin-top:0;margin-bottom:1rem;overflow:auto}figure{margin:0 0 1rem}img{border-style:none}img,svg{vertical-align:middle}svg{overflow:hidden}table{border-collapse:collapse}caption{padding-top:.75rem;padding-bottom:.75rem;color:#6c757d;text-align:left;caption-side:bottom}th{text-align:inherit}label{display:inline-block;margin-bottom:.5rem}button{border-radius:0}button:focus{outline:1px dotted;outline:5px auto -webkit-focus-ring-color}button,input,optgroup,select,textarea{margin:0;font-family:inherit;font-size:inherit;line-height:inherit}button,input{overflow:visible}button,select{text-transform:none}select{word-wrap:normal}[type=button],[type=reset],[type=submit],button{-webkit-appearance:button}[type=button]:not(:disabled),[type=reset]:not(:disabled),[type=submit]:not(:disabled),button:not(:disabled){cursor:pointer}[type=button]::-moz-focus-inner,[type=reset]::-moz-focus-inner,[type=submit]::-moz-focus-inner,button::-moz-focus-inner{padding:0;border-style:none}input[type=checkbox],input[type=radio]{box-sizing:border-box;padding:0}input[type=date],input[type=datetime-local],input[type=month],input[type=time]{-webkit-appearance:listbox}textarea{overflow:auto;resize:vertical}fieldset{min-width:0;padding:0;margin:0;border:0}legend{display:block;width:100%;max-width:100%;padding:0;margin-bottom:.5rem;font-size:1.5rem;line-height:inherit;color:inherit;white-space:normal}@media(max-width:1200px){legend{font-size:calc(1.275rem + .3vw)}}progress{vertical-align:baseline}[type=number]::-webkit-inner-spin-button,[type=number]::-webkit-outer-spin-button{height:auto}[type=search]{outline-offset:-2px;-webkit-appearance:none}[type=search]::-webkit-search-decoration{-webkit-appearance:none}::-webkit-file-upload-button{font:inherit;-webkit-appearance:button}output{display:inline-block}summary{display:list-item;cursor:pointer}template{display:none}[hidden]{display:none!important}.h1,.h2,.h3,.h4,.h5,.h6,h1,h2,h3,h4,h5,h6{margin-bottom:16px;font-family:Hind Siliguri,sans-serif;font-weight:500;line-height:1.125}.h1,h1{font-size:2.5rem}@media(max-width:1200px){.h1,h1{font-size:calc(1.375rem + 1.5vw)}}.h2,h2{font-size:2rem}@media(max-width:1200px){.h2,h2{font-size:calc(1.325rem + .9vw)}}.h3,h3{font-size:1.75rem}@media(max-width:1200px){.h3,h3{font-size:calc(1.3rem + .6vw)}}.h4,h4{font-size:1.5rem}@media(max-width:1200px){.h4,h4{font-size:calc(1.275rem + .3vw)}}.h5,h5{font-size:1.125rem}.h6,h6{font-size:.875rem}.lead{font-size:1.375rem;font-weight:400}@media(max-width:1200px){.lead{font-size:calc(1.2625rem + .15vw)}}.display-1{font-size:4rem;font-weight:600;line-height:1.125}@media(max-width:1200px){.display-1{font-size:calc(1.525rem + 3.3vw)}}.display-2{font-size:2.5rem;font-weight:600;line-height:1.125}@media(max-width:1200px){.display-2{font-size:calc(1.375rem + 1.5vw)}}.display-3{font-size:2rem;font-weight:500;line-height:1.125}@media(max-width:1200px){.display-3{font-size:calc(1.325rem + .9vw)}}.display-4{font-size:1.75rem;font-weight:500;line-height:1.125}@media(max-width:1200px){.display-4{font-size:calc(1.3rem + .6vw)}}hr{margin-top:8px;margin-bottom:8px;border:0;border-top:1px solid rgba(33,37,41,.1)}.small,small{font-size:80%;font-weight:400}.mark,mark{padding:.2em;background-color:#fcf8e3}.list-inline,.list-unstyled{padding-left:0;list-style:none}.list-inline-item{display:inline-block}.list-inline-item:not(:last-child){margin-right:.5rem}.initialism{font-size:90%;text-transform:uppercase}.blockquote{margin-bottom:8px;font-size:1.25rem}.blockquote-footer{display:block;font-size:80%;color:#6c757d}.blockquote-footer:before{content:"— "}.img-fluid,.img-thumbnail{max-width:100%;height:auto}.img-thumbnail{padding:.25rem;background-color:#fff;border:1px solid #dee2e6;border-radius:8px}.figure{display:inline-block}.figure-img{margin-bottom:4px;line-height:1}.figure-caption{font-size:90%;color:#6c757d}code{font-size:87.5%;color:#e83e8c;word-wrap:break-word}a>code{color:inherit}kbd{padding:.2rem .4rem;font-size:87.5%;color:#fff;background-color:#495057;border-radius:4px}kbd kbd{padding:0;font-size:100%;font-weight:700}pre{display:block;font-size:87.5%;color:#495057}pre code{font-size:inherit;color:inherit;word-break:normal}.pre-scrollable{max-height:340px;overflow-y:scroll}.container{width:100%;padding-right:20px;padding-left:20px;margin-right:auto;margin-left:auto}@media(min-width:400px){.container{max-width:576px}}@media(min-width:616px){.container{max-width:576px}}@media(min-width:768px){.container{max-width:958px}}@media(min-width:980px){.container{max-width:1008px}}@media(min-width:1240px){.container{max-width:1118px}}.container-fluid,.container-lg,.container-md,.container-sm,.container-xl,.container-xs{width:100%;padding-right:20px;padding-left:20px;margin-right:auto;margin-left:auto}@media(min-width:400px){.container,.container-xs{max-width:576px}}@media(min-width:616px){.container,.container-sm,.container-xs{max-width:576px}}@media(min-width:768px){.container,.container-md,.container-sm,.container-xs{max-width:958px}}@media(min-width:980px){.container,.container-lg,.container-md,.container-sm,.container-xs{max-width:1008px}}@media(min-width:1240px){.container,.container-lg,.container-md,.container-sm,.container-xl,.container-xs{max-width:1118px}}.row{display:flex;flex-wrap:wrap;margin-right:-20px;margin-left:-20px}.no-gutters{margin-right:0;margin-left:0}.no-gutters>.col,.no-gutters>[class*=col-]{padding-right:0;padding-left:0}.col,.col-1,.col-2,.col-3,.col-4,.col-5,.col-6,.col-7,.col-8,.col-9,.col-10,.col-11,.col-12,.col-auto,.col-lg,.col-lg-1,.col-lg-2,.col-lg-3,.col-lg-4,.col-lg-5,.col-lg-6,.col-lg-7,.col-lg-8,.col-lg-9,.col-lg-10,.col-lg-11,.col-lg-12,.col-lg-auto,.col-md,.col-md-1,.col-md-2,.col-md-3,.col-md-4,.col-md-5,.col-md-6,.col-md-7,.col-md-8,.col-md-9,.col-md-10,.col-md-11,.col-md-12,.col-md-auto,.col-sm,.col-sm-1,.col-sm-2,.col-sm-3,.col-sm-4,.col-sm-5,.col-sm-6,.col-sm-7,.col-sm-8,.col-sm-9,.col-sm-10,.col-sm-11,.col-sm-12,.col-sm-auto,.col-xl,.col-xl-1,.col-xl-2,.col-xl-3,.col-xl-4,.col-xl-5,.col-xl-6,.col-xl-7,.col-xl-8,.col-xl-9,.col-xl-10,.col-xl-11,.col-xl-12,.col-xl-auto,.col-xs,.col-xs-1,.col-xs-2,.col-xs-3,.col-xs-4,.col-xs-5,.col-xs-6,.col-xs-7,.col-xs-8,.col-xs-9,.col-xs-10,.col-xs-11,.col-xs-12,.col-xs-auto{position:relative;width:100%;padding-right:20px;padding-left:20px}.col{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-1>*{flex:0 0 100%;max-width:100%}.row-cols-2>*{flex:0 0 50%;max-width:50%}.row-cols-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-4>*{flex:0 0 25%;max-width:25%}.row-cols-5>*{flex:0 0 20%;max-width:20%}.row-cols-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-auto{flex:0 0 auto;width:auto;max-width:100%}.col-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-3{flex:0 0 25%;max-width:25%}.col-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-6{flex:0 0 50%;max-width:50%}.col-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-9{flex:0 0 75%;max-width:75%}.col-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-12{flex:0 0 100%;max-width:100%}.order-first{order:-1}.order-last{order:13}.order-0{order:0}.order-1{order:1}.order-2{order:2}.order-3{order:3}.order-4{order:4}.order-5{order:5}.order-6{order:6}.order-7{order:7}.order-8{order:8}.order-9{order:9}.order-10{order:10}.order-11{order:11}.order-12{order:12}.offset-1{margin-left:8.3333333333%}.offset-2{margin-left:16.6666666667%}.offset-3{margin-left:25%}.offset-4{margin-left:33.3333333333%}.offset-5{margin-left:41.6666666667%}.offset-6{margin-left:50%}.offset-7{margin-left:58.3333333333%}.offset-8{margin-left:66.6666666667%}.offset-9{margin-left:75%}.offset-10{margin-left:83.3333333333%}.offset-11{margin-left:91.6666666667%}@media(min-width:400px){.col-xs{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-xs-1>*{flex:0 0 100%;max-width:100%}.row-cols-xs-2>*{flex:0 0 50%;max-width:50%}.row-cols-xs-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-xs-4>*{flex:0 0 25%;max-width:25%}.row-cols-xs-5>*{flex:0 0 20%;max-width:20%}.row-cols-xs-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-xs-auto{flex:0 0 auto;width:auto;max-width:100%}.col-xs-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-xs-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-xs-3{flex:0 0 25%;max-width:25%}.col-xs-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-xs-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-xs-6{flex:0 0 50%;max-width:50%}.col-xs-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-xs-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-xs-9{flex:0 0 75%;max-width:75%}.col-xs-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-xs-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-xs-12{flex:0 0 100%;max-width:100%}.order-xs-first{order:-1}.order-xs-last{order:13}.order-xs-0{order:0}.order-xs-1{order:1}.order-xs-2{order:2}.order-xs-3{order:3}.order-xs-4{order:4}.order-xs-5{order:5}.order-xs-6{order:6}.order-xs-7{order:7}.order-xs-8{order:8}.order-xs-9{order:9}.order-xs-10{order:10}.order-xs-11{order:11}.order-xs-12{order:12}.offset-xs-0{margin-left:0}.offset-xs-1{margin-left:8.3333333333%}.offset-xs-2{margin-left:16.6666666667%}.offset-xs-3{margin-left:25%}.offset-xs-4{margin-left:33.3333333333%}.offset-xs-5{margin-left:41.6666666667%}.offset-xs-6{margin-left:50%}.offset-xs-7{margin-left:58.3333333333%}.offset-xs-8{margin-left:66.6666666667%}.offset-xs-9{margin-left:75%}.offset-xs-10{margin-left:83.3333333333%}.offset-xs-11{margin-left:91.6666666667%}}@media(min-width:616px){.col-sm{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-sm-1>*{flex:0 0 100%;max-width:100%}.row-cols-sm-2>*{flex:0 0 50%;max-width:50%}.row-cols-sm-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-sm-4>*{flex:0 0 25%;max-width:25%}.row-cols-sm-5>*{flex:0 0 20%;max-width:20%}.row-cols-sm-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-sm-auto{flex:0 0 auto;width:auto;max-width:100%}.col-sm-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-sm-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-sm-3{flex:0 0 25%;max-width:25%}.col-sm-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-sm-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-sm-6{flex:0 0 50%;max-width:50%}.col-sm-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-sm-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-sm-9{flex:0 0 75%;max-width:75%}.col-sm-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-sm-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-sm-12{flex:0 0 100%;max-width:100%}.order-sm-first{order:-1}.order-sm-last{order:13}.order-sm-0{order:0}.order-sm-1{order:1}.order-sm-2{order:2}.order-sm-3{order:3}.order-sm-4{order:4}.order-sm-5{order:5}.order-sm-6{order:6}.order-sm-7{order:7}.order-sm-8{order:8}.order-sm-9{order:9}.order-sm-10{order:10}.order-sm-11{order:11}.order-sm-12{order:12}.offset-sm-0{margin-left:0}.offset-sm-1{margin-left:8.3333333333%}.offset-sm-2{margin-left:16.6666666667%}.offset-sm-3{margin-left:25%}.offset-sm-4{margin-left:33.3333333333%}.offset-sm-5{margin-left:41.6666666667%}.offset-sm-6{margin-left:50%}.offset-sm-7{margin-left:58.3333333333%}.offset-sm-8{margin-left:66.6666666667%}.offset-sm-9{margin-left:75%}.offset-sm-10{margin-left:83.3333333333%}.offset-sm-11{margin-left:91.6666666667%}}@media(min-width:768px){.col-md{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-md-1>*{flex:0 0 100%;max-width:100%}.row-cols-md-2>*{flex:0 0 50%;max-width:50%}.row-cols-md-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-md-4>*{flex:0 0 25%;max-width:25%}.row-cols-md-5>*{flex:0 0 20%;max-width:20%}.row-cols-md-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-md-auto{flex:0 0 auto;width:auto;max-width:100%}.col-md-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-md-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-md-3{flex:0 0 25%;max-width:25%}.col-md-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-md-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-md-6{flex:0 0 50%;max-width:50%}.col-md-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-md-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-md-9{flex:0 0 75%;max-width:75%}.col-md-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-md-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-md-12{flex:0 0 100%;max-width:100%}.order-md-first{order:-1}.order-md-last{order:13}.order-md-0{order:0}.order-md-1{order:1}.order-md-2{order:2}.order-md-3{order:3}.order-md-4{order:4}.order-md-5{order:5}.order-md-6{order:6}.order-md-7{order:7}.order-md-8{order:8}.order-md-9{order:9}.order-md-10{order:10}.order-md-11{order:11}.order-md-12{order:12}.offset-md-0{margin-left:0}.offset-md-1{margin-left:8.3333333333%}.offset-md-2{margin-left:16.6666666667%}.offset-md-3{margin-left:25%}.offset-md-4{margin-left:33.3333333333%}.offset-md-5{margin-left:41.6666666667%}.offset-md-6{margin-left:50%}.offset-md-7{margin-left:58.3333333333%}.offset-md-8{margin-left:66.6666666667%}.offset-md-9{margin-left:75%}.offset-md-10{margin-left:83.3333333333%}.offset-md-11{margin-left:91.6666666667%}}@media(min-width:980px){.col-lg{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-lg-1>*{flex:0 0 100%;max-width:100%}.row-cols-lg-2>*{flex:0 0 50%;max-width:50%}.row-cols-lg-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-lg-4>*{flex:0 0 25%;max-width:25%}.row-cols-lg-5>*{flex:0 0 20%;max-width:20%}.row-cols-lg-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-lg-auto{flex:0 0 auto;width:auto;max-width:100%}.col-lg-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-lg-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-lg-3{flex:0 0 25%;max-width:25%}.col-lg-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-lg-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-lg-6{flex:0 0 50%;max-width:50%}.col-lg-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-lg-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-lg-9{flex:0 0 75%;max-width:75%}.col-lg-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-lg-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-lg-12{flex:0 0 100%;max-width:100%}.order-lg-first{order:-1}.order-lg-last{order:13}.order-lg-0{order:0}.order-lg-1{order:1}.order-lg-2{order:2}.order-lg-3{order:3}.order-lg-4{order:4}.order-lg-5{order:5}.order-lg-6{order:6}.order-lg-7{order:7}.order-lg-8{order:8}.order-lg-9{order:9}.order-lg-10{order:10}.order-lg-11{order:11}.order-lg-12{order:12}.offset-lg-0{margin-left:0}.offset-lg-1{margin-left:8.3333333333%}.offset-lg-2{margin-left:16.6666666667%}.offset-lg-3{margin-left:25%}.offset-lg-4{margin-left:33.3333333333%}.offset-lg-5{margin-left:41.6666666667%}.offset-lg-6{margin-left:50%}.offset-lg-7{margin-left:58.3333333333%}.offset-lg-8{margin-left:66.6666666667%}.offset-lg-9{margin-left:75%}.offset-lg-10{margin-left:83.3333333333%}.offset-lg-11{margin-left:91.6666666667%}}@media(min-width:1240px){.col-xl{flex-basis:0;flex-grow:1;max-width:100%}.row-cols-xl-1>*{flex:0 0 100%;max-width:100%}.row-cols-xl-2>*{flex:0 0 50%;max-width:50%}.row-cols-xl-3>*{flex:0 0 33.3333333333%;max-width:33.3333333333%}.row-cols-xl-4>*{flex:0 0 25%;max-width:25%}.row-cols-xl-5>*{flex:0 0 20%;max-width:20%}.row-cols-xl-6>*{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-xl-auto{flex:0 0 auto;width:auto;max-width:100%}.col-xl-1{flex:0 0 8.3333333333%;max-width:8.3333333333%}.col-xl-2{flex:0 0 16.6666666667%;max-width:16.6666666667%}.col-xl-3{flex:0 0 25%;max-width:25%}.col-xl-4{flex:0 0 33.3333333333%;max-width:33.3333333333%}.col-xl-5{flex:0 0 41.6666666667%;max-width:41.6666666667%}.col-xl-6{flex:0 0 50%;max-width:50%}.col-xl-7{flex:0 0 58.3333333333%;max-width:58.3333333333%}.col-xl-8{flex:0 0 66.6666666667%;max-width:66.6666666667%}.col-xl-9{flex:0 0 75%;max-width:75%}.col-xl-10{flex:0 0 83.3333333333%;max-width:83.3333333333%}.col-xl-11{flex:0 0 91.6666666667%;max-width:91.6666666667%}.col-xl-12{flex:0 0 100%;max-width:100%}.order-xl-first{order:-1}.order-xl-last{order:13}.order-xl-0{order:0}.order-xl-1{order:1}.order-xl-2{order:2}.order-xl-3{order:3}.order-xl-4{order:4}.order-xl-5{order:5}.order-xl-6{order:6}.order-xl-7{order:7}.order-xl-8{order:8}.order-xl-9{order:9}.order-xl-10{order:10}.order-xl-11{order:11}.order-xl-12{order:12}.offset-xl-0{margin-left:0}.offset-xl-1{margin-left:8.3333333333%}.offset-xl-2{margin-left:16.6666666667%}.offset-xl-3{margin-left:25%}.offset-xl-4{margin-left:33.3333333333%}.offset-xl-5{margin-left:41.6666666667%}.offset-xl-6{margin-left:50%}.offset-xl-7{margin-left:58.3333333333%}.offset-xl-8{margin-left:66.6666666667%}.offset-xl-9{margin-left:75%}.offset-xl-10{margin-left:83.3333333333%}.offset-xl-11{margin-left:91.6666666667%}}.table{width:100%;margin-bottom:8px;color:#212529}.table td,.table th{padding:.75rem;vertical-align:top;border-top:1px solid #d6dbdf}.table thead th{vertical-align:bottom;border-bottom:2px solid #d6dbdf}.table tbody+tbody{border-top:2px solid #d6dbdf}.table-sm td,.table-sm th{padding:.3rem}.table-bordered,.table-bordered td,.table-bordered th{border:1px solid #d6dbdf}.table-bordered thead td,.table-bordered thead th{border-bottom-width:2px}.table-borderless tbody+tbody,.table-borderless td,.table-borderless th,.table-borderless thead th{border:0}.table-striped tbody tr:nth-of-type(odd){background-color:rgba(33,37,41,.05)}.table-hover tbody tr:hover{color:#212529;background-color:rgba(33,37,41,.075)}.table-primary,.table-primary>td,.table-primary>th{background-color:#fff1b8}.table-primary tbody+tbody,.table-primary td,.table-primary th,.table-primary thead th{border-color:#ffe47a}.table-hover .table-primary:hover,.table-hover .table-primary:hover>td,.table-hover .table-primary:hover>th{background-color:#ffec9f}.table-secondary,.table-secondary>td,.table-secondary>th{background-color:#c1c2c3}.table-secondary tbody+tbody,.table-secondary td,.table-secondary th,.table-secondary thead th{border-color:#8c8e90}.table-hover .table-secondary:hover,.table-hover .table-secondary:hover>td,.table-hover .table-secondary:hover>th{background-color:#b4b5b6}.table-success,.table-success>td,.table-success>th{background-color:#c3e6cb}.table-success tbody+tbody,.table-success td,.table-success th,.table-success thead th{border-color:#8fd19e}.table-hover .table-success:hover,.table-hover .table-success:hover>td,.table-hover .table-success:hover>th{background-color:#b1dfbb}.table-info,.table-info>td,.table-info>th{background-color:#bee5eb}.table-info tbody+tbody,.table-info td,.table-info th,.table-info thead th{border-color:#86cfda}.table-hover .table-info:hover,.table-hover .table-info:hover>td,.table-hover .table-info:hover>th{background-color:#abdde5}.table-warning,.table-warning>td,.table-warning>th{background-color:#ffeeba}.table-warning tbody+tbody,.table-warning td,.table-warning th,.table-warning thead th{border-color:#ffdf7e}.table-hover .table-warning:hover,.table-hover .table-warning:hover>td,.table-hover .table-warning:hover>th{background-color:#ffe8a1}.table-danger,.table-danger>td,.table-danger>th{background-color:#f5c6cb}.table-danger tbody+tbody,.table-danger td,.table-danger th,.table-danger thead th{border-color:#ed969e}.table-hover .table-danger:hover,.table-hover .table-danger:hover>td,.table-hover .table-danger:hover>th{background-color:#f1b0b7}.table-light,.table-light>td,.table-light>th{background-color:#fbfcfd}.table-light tbody+tbody,.table-light td,.table-light th,.table-light thead th{border-color:#f8fafc}.table-hover .table-light:hover,.table-hover .table-light:hover>td,.table-hover .table-light:hover>th{background-color:#eaeff5}.table-dark,.table-dark>td,.table-dark>th{background-color:#ccced0}.table-dark tbody+tbody,.table-dark td,.table-dark th,.table-dark thead th{border-color:#a0a4a8}.table-hover .table-dark:hover,.table-hover .table-dark:hover>td,.table-hover .table-dark:hover>th{background-color:#bfc1c4}.table-primary-light,.table-primary-light>td,.table-primary-light>th{background-color:#fffefb}.table-primary-light tbody+tbody,.table-primary-light td,.table-primary-light th,.table-primary-light thead th{border-color:#fffcf7}.table-hover .table-primary-light:hover,.table-hover .table-primary-light:hover>td,.table-hover .table-primary-light:hover>th{background-color:#fff8e2}.table-secondary-light,.table-secondary-light>td,.table-secondary-light>th{background-color:#fff}.table-secondary-light tbody+tbody,.table-secondary-light td,.table-secondary-light th,.table-secondary-light thead th{border-color:#fff}.table-hover .table-secondary-light:hover,.table-hover .table-secondary-light:hover>td,.table-hover .table-secondary-light:hover>th{background-color:#f2f2f2}.table-tertiary,.table-tertiary>td,.table-tertiary>th{background-color:#c2dafc}.table-tertiary tbody+tbody,.table-tertiary td,.table-tertiary th,.table-tertiary thead th{border-color:#8ebaf9}.table-hover .table-tertiary:hover,.table-hover .table-tertiary:hover>td,.table-hover .table-tertiary:hover>th{background-color:#aacbfb}.table-tertiary-light,.table-tertiary-light>td,.table-tertiary-light>th{background-color:#f7fbff}.table-tertiary-light tbody+tbody,.table-tertiary-light td,.table-tertiary-light th,.table-tertiary-light thead th{border-color:#f0f8fe}.table-hover .table-tertiary-light:hover,.table-hover .table-tertiary-light:hover>td,.table-hover .table-tertiary-light:hover>th{background-color:#deeeff}.table-white,.table-white>td,.table-white>th{background-color:#fff}.table-white tbody+tbody,.table-white td,.table-white th,.table-white thead th{border-color:#fff}.table-hover .table-white:hover,.table-hover .table-white:hover>td,.table-hover .table-white:hover>th{background-color:#f2f2f2}.table-black,.table-black>td,.table-black>th{background-color:#c1c2c3}.table-black tbody+tbody,.table-black td,.table-black th,.table-black thead th{border-color:#8c8e90}.table-hover .table-black:hover,.table-hover .table-black:hover>td,.table-hover .table-black:hover>th{background-color:#b4b5b6}.table-blue,.table-blue>td,.table-blue>th{background-color:#c2dafc}.table-blue tbody+tbody,.table-blue td,.table-blue th,.table-blue thead th{border-color:#8ebaf9}.table-hover .table-blue:hover,.table-hover .table-blue:hover>td,.table-hover .table-blue:hover>th{background-color:#aacbfb}.table-light-blue,.table-light-blue>td,.table-light-blue>th{background-color:#f7fbff}.table-light-blue tbody+tbody,.table-light-blue td,.table-light-blue th,.table-light-blue thead th{border-color:#f0f8fe}.table-hover .table-light-blue:hover,.table-hover .table-light-blue:hover>td,.table-hover .table-light-blue:hover>th{background-color:#deeeff}.table-yellow,.table-yellow>td,.table-yellow>th{background-color:#fff1b8}.table-yellow tbody+tbody,.table-yellow td,.table-yellow th,.table-yellow thead th{border-color:#ffe47a}.table-hover .table-yellow:hover,.table-hover .table-yellow:hover>td,.table-hover .table-yellow:hover>th{background-color:#ffec9f}.table-light-yellow,.table-light-yellow>td,.table-light-yellow>th{background-color:#fffefb}.table-light-yellow tbody+tbody,.table-light-yellow td,.table-light-yellow th,.table-light-yellow thead th{border-color:#fffcf7}.table-hover .table-light-yellow:hover,.table-hover .table-light-yellow:hover>td,.table-hover .table-light-yellow:hover>th{background-color:#fff8e2}.table-orange,.table-orange>td,.table-orange>th{background-color:#ffdfb8}.table-orange tbody+tbody,.table-orange td,.table-orange th,.table-orange thead th{border-color:#ffc37a}.table-hover .table-orange:hover,.table-hover .table-orange:hover>td,.table-hover .table-orange:hover>th{background-color:#ffd49f}.table-light-orange,.table-light-orange>td,.table-light-orange>th{background-color:#fff7ea}.table-light-orange tbody+tbody,.table-light-orange td,.table-light-orange th,.table-light-orange thead th{border-color:#fff1d9}.table-hover .table-light-orange:hover,.table-hover .table-light-orange:hover>td,.table-hover .table-light-orange:hover>th{background-color:#ffedd1}.table-red,.table-red>td,.table-red>th{background-color:#ffc8c8}.table-red tbody+tbody,.table-red td,.table-red th,.table-red thead th{border-color:#ff9898}.table-hover .table-red:hover,.table-hover .table-red:hover>td,.table-hover .table-red:hover>th{background-color:#ffafaf}.table-light-red,.table-light-red>td,.table-light-red>th{background-color:#fff7f7}.table-light-red tbody+tbody,.table-light-red td,.table-light-red th,.table-light-red thead th{border-color:#fff1ef}.table-hover .table-light-red:hover,.table-hover .table-light-red:hover>td,.table-hover .table-light-red:hover>th{background-color:#ffdede}.table-medium,.table-medium>td,.table-medium>th{background-color:#f4f5f6}.table-medium tbody+tbody,.table-medium td,.table-medium th,.table-medium thead th{border-color:#eaecee}.table-hover .table-medium:hover,.table-hover .table-medium:hover>td,.table-hover .table-medium:hover>th{background-color:#e6e8eb}.table-active,.table-active>td,.table-active>th{background-color:rgba(33,37,41,.075)}.table-hover .table-active:hover,.table-hover .table-active:hover>td,.table-hover .table-active:hover>th{background-color:rgba(22,24,27,.075)}.table .thead-dark th{color:#fff;background-color:#343a40;border-color:#454d55}.table .thead-light th{color:#6c757d;background-color:#e9ecef;border-color:#d6dbdf}.table-dark{color:#fff;background-color:#343a40}.table-dark td,.table-dark th,.table-dark thead th{border-color:#454d55}.table-dark.table-bordered{border:0}.table-dark.table-striped tbody tr:nth-of-type(odd){background-color:hsla(0,0%,100%,.05)}.table-dark.table-hover tbody tr:hover{color:#fff;background-color:hsla(0,0%,100%,.075)}@media(max-width:399.98px){.table-responsive-xs{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive-xs>.table-bordered{border:0}}@media(max-width:615.98px){.table-responsive-sm{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive-sm>.table-bordered{border:0}}@media(max-width:767.98px){.table-responsive-md{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive-md>.table-bordered{border:0}}@media(max-width:979.98px){.table-responsive-lg{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive-lg>.table-bordered{border:0}}@media(max-width:1239.98px){.table-responsive-xl{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive-xl>.table-bordered{border:0}}.table-responsive{display:block;width:100%;overflow-x:auto;-webkit-overflow-scrolling:touch}.table-responsive>.table-bordered{border:0}.form-control{display:block;width:100%;height:calc(1.5em + .75rem + 2px);padding:.375rem .75rem;font-size:1rem;font-weight:400;line-height:1.5;color:#6c757d;background-color:#fff;background-clip:padding-box;border:1px solid #ced4da;border-radius:8px;transition:border-color .15s ease-in-out,box-shadow .15s ease-in-out}@media(prefers-reduced-motion:reduce){.form-control{transition:none}}.form-control::-ms-expand{background-color:transparent;border:0}.form-control:-moz-focusring{color:transparent;text-shadow:0 0 0 #6c757d}.form-control:focus{color:#6c757d;background-color:#fff;border-color:#ffe680;outline:0;box-shadow:0 0 0 .2rem rgba(255,204,0,.25)}.form-control::-moz-placeholder{color:#6c757d;opacity:1}.form-control:-ms-input-placeholder{color:#6c757d;opacity:1}.form-control::placeholder{color:#6c757d;opacity:1}.form-control:disabled,.form-control[readonly]{background-color:#e9ecef;opacity:1}select.form-control:focus::-ms-value{color:#6c757d;background-color:#fff}.form-control-file,.form-control-range{display:block;width:100%}.col-form-label{padding-top:calc(.375rem + 1px);padding-bottom:calc(.375rem + 1px);margin-bottom:0;font-size:inherit;line-height:1.5}.col-form-label-lg{padding-top:calc(.5rem + 1px);padding-bottom:calc(.5rem + 1px);font-size:1.125rem;line-height:1.5}.col-form-label-sm{padding-top:calc(.25rem + 1px);padding-bottom:calc(.25rem + 1px);font-size:.875rem;line-height:1.5}.form-control-plaintext{display:block;width:100%;padding:.375rem 0;margin-bottom:0;font-size:1rem;line-height:1.5;color:#212529;background-color:transparent;border:solid transparent;border-width:1px 0}.form-control-plaintext.form-control-lg,.form-control-plaintext.form-control-sm{padding-right:0;padding-left:0}.form-control-sm{height:calc(1.5em + .5rem + 2px);padding:.25rem .5rem;font-size:.875rem;line-height:1.5;border-radius:4px}.form-control-lg{height:calc(1.5em + 1rem + 2px);padding:.5rem 1rem;font-size:1.125rem;line-height:1.5;border-radius:8px}select.form-control[multiple],select.form-control[size],textarea.form-control{height:auto}.form-group{margin-bottom:1rem}.form-text{display:block;margin-top:.25rem}.form-row{display:flex;flex-wrap:wrap;margin-right:-5px;margin-left:-5px}.form-row>.col,.form-row>[class*=col-]{padding-right:5px;padding-left:5px}.form-check{position:relative;display:block;padding-left:1.25rem}.form-check-input{position:absolute;margin-top:.3rem;margin-left:-1.25rem}.form-check-input:disabled~.form-check-label,.form-check-input[disabled]~.form-check-label{color:#6c757d}.form-check-label{margin-bottom:0}.form-check-inline{display:inline-flex;align-items:center;padding-left:0;margin-right:.75rem}.form-check-inline .form-check-input{position:static;margin-top:0;margin-right:.3125rem;margin-left:0}.valid-feedback{display:none;width:100%;margin-top:.25rem;font-size:80%;color:#28a745}.valid-tooltip{position:absolute;top:100%;z-index:5;display:none;max-width:100%;padding:.25rem .5rem;margin-top:.1rem;font-size:.875rem;line-height:1.5;color:#fff;background-color:rgba(40,167,69,.9);border-radius:8px}.is-valid~.valid-feedback,.is-valid~.valid-tooltip,.was-validated :valid~.valid-feedback,.was-validated :valid~.valid-tooltip{display:block}.form-control.is-valid,.was-validated .form-control:valid{border-color:#28a745;padding-right:calc(1.5em + .75rem);background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='8' height='8'%3E%3Cpath fill='%2328a745' d='M2.3 6.73L.6 4.53c-.4-1.04.46-1.4 1.1-.8l1.1 1.4 3.4-3.8c.6-.63 1.6-.27 1.2.7l-4 4.6c-.43.5-.8.4-1.1.1z'/%3E%3C/svg%3E");background-repeat:no-repeat;background-position:right calc(.375em + .1875rem) center;background-size:calc(.75em + .375rem) calc(.75em + .375rem)}.form-control.is-valid:focus,.was-validated .form-control:valid:focus{border-color:#28a745;box-shadow:0 0 0 .2rem rgba(40,167,69,.25)}.was-validated textarea.form-control:valid,textarea.form-control.is-valid{padding-right:calc(1.5em + .75rem);background-position:top calc(.375em + .1875rem) right calc(.375em + .1875rem)}.custom-select.is-valid,.was-validated .custom-select:valid{border-color:#28a745;padding-right:calc(.75em + 2.3125rem);background:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='4' height='5'%3E%3Cpath fill='%23343a40' d='M2 0L0 2h4zm0 5L0 3h4z'/%3E%3C/svg%3E") no-repeat right .75rem center/8px 10px,url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='8' height='8'%3E%3Cpath fill='%2328a745' d='M2.3 6.73L.6 4.53c-.4-1.04.46-1.4 1.1-.8l1.1 1.4 3.4-3.8c.6-.63 1.6-.27 1.2.7l-4 4.6c-.43.5-.8.4-1.1.1z'/%3E%3C/svg%3E") #fff no-repeat center right 1.75rem/calc(.75em + .375rem) calc(.75em + .375rem)}.custom-select.is-valid:focus,.was-validated .custom-select:valid:focus{border-color:#28a745;box-shadow:0 0 0 .2rem rgba(40,167,69,.25)}.form-check-input.is-valid~.form-check-label,.was-validated .form-check-input:valid~.form-check-label{color:#28a745}.form-check-input.is-valid~.valid-feedback,.form-check-input.is-valid~.valid-tooltip,.was-validated .form-check-input:valid~.valid-feedback,.was-validated .form-check-input:valid~.valid-tooltip{display:block}.custom-control-input.is-valid~.custom-control-label,.was-validated .custom-control-input:valid~.custom-control-label{color:#28a745}.custom-control-input.is-valid~.custom-control-label:before,.was-validated .custom-control-input:valid~.custom-control-label:before{border-color:#28a745}.custom-control-input.is-valid:checked~.custom-control-label:before,.was-validated .custom-control-input:valid:checked~.custom-control-label:before{border-color:#34ce57;background-color:#34ce57}.custom-control-input.is-valid:focus~.custom-control-label:before,.was-validated .custom-control-input:valid:focus~.custom-control-label:before{box-shadow:0 0 0 .2rem rgba(40,167,69,.25)}.custom-control-input.is-valid:focus:not(:checked)~.custom-control-label:before,.custom-file-input.is-valid~.custom-file-label,.was-validated .custom-control-input:valid:focus:not(:checked)~.custom-control-label:before,.was-validated .custom-file-input:valid~.custom-file-label{border-color:#28a745}.custom-file-input.is-valid:focus~.custom-file-label,.was-validated .custom-file-input:valid:focus~.custom-file-label{border-color:#28a745;box-shadow:0 0 0 .2rem rgba(40,167,69,.25)}.invalid-feedback{display:none;width:100%;margin-top:.25rem;font-size:80%;color:#dc3545}.invalid-tooltip{position:absolute;top:100%;z-index:5;display:none;max-width:100%;padding:.25rem .5rem;margin-top:.1rem;font-size:.875rem;line-height:1.5;color:#fff;background-color:rgba(220,53,69,.9);border-radius:8px}.is-invalid~.invalid-feedback,.is-invalid~.invalid-tooltip,.was-validated :invalid~.invalid-feedback,.was-validated :invalid~.invalid-tooltip{display:block}.form-control.is-invalid,.was-validated .form-control:invalid{border-color:#dc3545;padding-right:calc(1.5em + .75rem);background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='12' height='12' fill='none' stroke='%23dc3545'%3E%3Ccircle cx='6' cy='6' r='4.5'/%3E%3Cpath stroke-linejoin='round' d='M5.8 3.6h.4L6 6.5z'/%3E%3Ccircle cx='6' cy='8.2' r='.6' fill='%23dc3545' stroke='none'/%3E%3C/svg%3E");background-repeat:no-repeat;background-position:right calc(.375em + .1875rem) center;background-size:calc(.75em + .375rem) calc(.75em + .375rem)}.form-control.is-invalid:focus,.was-validated .form-control:invalid:focus{border-color:#dc3545;box-shadow:0 0 0 .2rem rgba(220,53,69,.25)}.was-validated textarea.form-control:invalid,textarea.form-control.is-invalid{padding-right:calc(1.5em + .75rem);background-position:top calc(.375em + .1875rem) right calc(.375em + .1875rem)}.custom-select.is-invalid,.was-validated .custom-select:invalid{border-color:#dc3545;padding-right:calc(.75em + 2.3125rem);background:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='4' height='5'%3E%3Cpath fill='%23343a40' d='M2 0L0 2h4zm0 5L0 3h4z'/%3E%3C/svg%3E") no-repeat right .75rem center/8px 10px,url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='12' height='12' fill='none' stroke='%23dc3545'%3E%3Ccircle cx='6' cy='6' r='4.5'/%3E%3Cpath stroke-linejoin='round' d='M5.8 3.6h.4L6 6.5z'/%3E%3Ccircle cx='6' cy='8.2' r='.6' fill='%23dc3545' stroke='none'/%3E%3C/svg%3E") #fff no-repeat center right 1.75rem/calc(.75em + .375rem) calc(.75em + .375rem)}.custom-select.is-invalid:focus,.was-validated .custom-select:invalid:focus{border-color:#dc3545;box-shadow:0 0 0 .2rem rgba(220,53,69,.25)}.form-check-input.is-invalid~.form-check-label,.was-validated .form-check-input:invalid~.form-check-label{color:#dc3545}.form-check-input.is-invalid~.invalid-feedback,.form-check-input.is-invalid~.invalid-tooltip,.was-validated .form-check-input:invalid~.invalid-feedback,.was-validated .form-check-input:invalid~.invalid-tooltip{display:block}.custom-control-input.is-invalid~.custom-control-label,.was-validated .custom-control-input:invalid~.custom-control-label{color:#dc3545}.custom-control-input.is-invalid~.custom-control-label:before,.was-validated .custom-control-input:invalid~.custom-control-label:before{border-color:#dc3545}.custom-control-input.is-invalid:checked~.custom-control-label:before,.was-validated .custom-control-input:invalid:checked~.custom-control-label:before{border-color:#e4606d;background-color:#e4606d}.custom-control-input.is-invalid:focus~.custom-control-label:before,.was-validated .custom-control-input:invalid:focus~.custom-control-label:before{box-shadow:0 0 0 .2rem rgba(220,53,69,.25)}.custom-control-input.is-invalid:focus:not(:checked)~.custom-control-label:before,.custom-file-input.is-invalid~.custom-file-label,.was-validated .custom-control-input:invalid:focus:not(:checked)~.custom-control-label:before,.was-validated .custom-file-input:invalid~.custom-file-label{border-color:#dc3545}.custom-file-input.is-invalid:focus~.custom-file-label,.was-validated .custom-file-input:invalid:focus~.custom-file-label{border-color:#dc3545;box-shadow:0 0 0 .2rem rgba(220,53,69,.25)}.form-inline{display:flex;flex-flow:row wrap;align-items:center}.form-inline .form-check{width:100%}@media(min-width:616px){.form-inline label{justify-content:center}.form-inline .form-group,.form-inline label{display:flex;align-items:center;margin-bottom:0}.form-inline .form-group{flex:0 0 auto;flex-flow:row wrap}.form-inline .form-control{display:inline-block;width:auto;vertical-align:middle}.form-inline .form-control-plaintext{display:inline-block}.form-inline .custom-select,.form-inline .input-group{width:auto}.form-inline .form-check{display:flex;align-items:center;justify-content:center;width:auto;padding-left:0}.form-inline .form-check-input{position:relative;flex-shrink:0;margin-top:0;margin-right:.25rem;margin-left:0}.form-inline .custom-control{align-items:center;justify-content:center}.form-inline .custom-control-label{margin-bottom:0}}.btn{display:inline-block;font-family:inherit;font-weight:700;color:#212529;text-align:center;vertical-align:middle;cursor:pointer;-webkit-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none;background-color:transparent;border:1px solid transparent;padding:12px 32px;font-size:.875rem;line-height:20px;border-radius:8px;transition:color .15s ease-in-out,background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out}@media(prefers-reduced-motion:reduce){.btn{transition:none}}.btn:hover{color:#212529;text-decoration:none}.btn.focus,.btn:focus{outline:0;box-shadow:none}.btn.disabled,.btn:disabled{opacity:.65}a.btn.disabled,fieldset:disabled a.btn{pointer-events:none}.btn-primary{color:#495057;background-color:#fc0;border-color:#fc0}.btn-primary.focus,.btn-primary:focus,.btn-primary:hover{color:#495057;background-color:#d9ad00;border-color:#cca300}.btn-primary.focus,.btn-primary:focus{box-shadow:0 0 0 0 rgba(228,185,13,.5)}.btn-primary.disabled,.btn-primary:disabled{color:#495057;background-color:#fc0;border-color:#fc0}.btn-primary:not(:disabled):not(.disabled).active,.btn-primary:not(:disabled):not(.disabled):active,.show>.btn-primary.dropdown-toggle{color:#495057;background-color:#cca300;border-color:#bf9900}.btn-primary:not(:disabled):not(.disabled).active:focus,.btn-primary:not(:disabled):not(.disabled):active:focus,.show>.btn-primary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,185,13,.5)}.btn-secondary{color:#fff;background-color:#212529;border-color:#212529}.btn-secondary.focus,.btn-secondary:focus,.btn-secondary:hover{color:#fff;background-color:#101214;border-color:#0a0c0d}.btn-secondary.focus,.btn-secondary:focus{box-shadow:0 0 0 0 rgba(66,70,73,.5)}.btn-secondary.disabled,.btn-secondary:disabled{color:#fff;background-color:#212529;border-color:#212529}.btn-secondary:not(:disabled):not(.disabled).active,.btn-secondary:not(:disabled):not(.disabled):active,.show>.btn-secondary.dropdown-toggle{color:#fff;background-color:#0a0c0d;border-color:#050506}.btn-secondary:not(:disabled):not(.disabled).active:focus,.btn-secondary:not(:disabled):not(.disabled):active:focus,.show>.btn-secondary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(66,70,73,.5)}.btn-success{color:#fff;background-color:#28a745;border-color:#28a745}.btn-success.focus,.btn-success:focus,.btn-success:hover{color:#fff;background-color:#218838;border-color:#1e7e34}.btn-success.focus,.btn-success:focus{box-shadow:0 0 0 0 rgba(72,180,97,.5)}.btn-success.disabled,.btn-success:disabled{color:#fff;background-color:#28a745;border-color:#28a745}.btn-success:not(:disabled):not(.disabled).active,.btn-success:not(:disabled):not(.disabled):active,.show>.btn-success.dropdown-toggle{color:#fff;background-color:#1e7e34;border-color:#1c7430}.btn-success:not(:disabled):not(.disabled).active:focus,.btn-success:not(:disabled):not(.disabled):active:focus,.show>.btn-success.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(72,180,97,.5)}.btn-info{color:#fff;background-color:#17a2b8;border-color:#17a2b8}.btn-info.focus,.btn-info:focus,.btn-info:hover{color:#fff;background-color:#138496;border-color:#117a8b}.btn-info.focus,.btn-info:focus{box-shadow:0 0 0 0 rgba(58,176,195,.5)}.btn-info.disabled,.btn-info:disabled{color:#fff;background-color:#17a2b8;border-color:#17a2b8}.btn-info:not(:disabled):not(.disabled).active,.btn-info:not(:disabled):not(.disabled):active,.show>.btn-info.dropdown-toggle{color:#fff;background-color:#117a8b;border-color:#10707f}.btn-info:not(:disabled):not(.disabled).active:focus,.btn-info:not(:disabled):not(.disabled):active:focus,.show>.btn-info.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(58,176,195,.5)}.btn-warning{color:#495057;background-color:#ffc107;border-color:#ffc107}.btn-warning.focus,.btn-warning:focus,.btn-warning:hover{color:#495057;background-color:#e0a800;border-color:#d39e00}.btn-warning.focus,.btn-warning:focus{box-shadow:0 0 0 0 rgba(228,176,19,.5)}.btn-warning.disabled,.btn-warning:disabled{color:#495057;background-color:#ffc107;border-color:#ffc107}.btn-warning:not(:disabled):not(.disabled).active,.btn-warning:not(:disabled):not(.disabled):active,.show>.btn-warning.dropdown-toggle{color:#495057;background-color:#d39e00;border-color:#c69500}.btn-warning:not(:disabled):not(.disabled).active:focus,.btn-warning:not(:disabled):not(.disabled):active:focus,.show>.btn-warning.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,176,19,.5)}.btn-danger{color:#fff;background-color:#dc3545;border-color:#dc3545}.btn-danger.focus,.btn-danger:focus,.btn-danger:hover{color:#fff;background-color:#c82333;border-color:#bd2130}.btn-danger.focus,.btn-danger:focus{box-shadow:0 0 0 0 rgba(225,83,97,.5)}.btn-danger.disabled,.btn-danger:disabled{color:#fff;background-color:#dc3545;border-color:#dc3545}.btn-danger:not(:disabled):not(.disabled).active,.btn-danger:not(:disabled):not(.disabled):active,.show>.btn-danger.dropdown-toggle{color:#fff;background-color:#bd2130;border-color:#b21f2d}.btn-danger:not(:disabled):not(.disabled).active:focus,.btn-danger:not(:disabled):not(.disabled):active:focus,.show>.btn-danger.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(225,83,97,.5)}.btn-light{color:#495057;background-color:#f1f6f9;border-color:#f1f6f9}.btn-light.focus,.btn-light:focus,.btn-light:hover{color:#495057;background-color:#d6e5ee;border-color:#cddfea}.btn-light.focus,.btn-light:focus{box-shadow:0 0 0 0 rgba(216,221,225,.5)}.btn-light.disabled,.btn-light:disabled{color:#495057;background-color:#f1f6f9;border-color:#f1f6f9}.btn-light:not(:disabled):not(.disabled).active,.btn-light:not(:disabled):not(.disabled):active,.show>.btn-light.dropdown-toggle{color:#495057;background-color:#cddfea;border-color:#c4d9e6}.btn-light:not(:disabled):not(.disabled).active:focus,.btn-light:not(:disabled):not(.disabled):active:focus,.show>.btn-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(216,221,225,.5)}.btn-dark{color:#fff;background-color:#495057;border-color:#495057}.btn-dark.focus,.btn-dark:focus,.btn-dark:hover{color:#fff;background-color:#383d42;border-color:#32373b}.btn-dark.focus,.btn-dark:focus{box-shadow:0 0 0 0 rgba(100,106,112,.5)}.btn-dark.disabled,.btn-dark:disabled{color:#fff;background-color:#495057;border-color:#495057}.btn-dark:not(:disabled):not(.disabled).active,.btn-dark:not(:disabled):not(.disabled):active,.show>.btn-dark.dropdown-toggle{color:#fff;background-color:#32373b;border-color:#2c3034}.btn-dark:not(:disabled):not(.disabled).active:focus,.btn-dark:not(:disabled):not(.disabled):active:focus,.show>.btn-dark.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(100,106,112,.5)}.btn-primary-light{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-primary-light.focus,.btn-primary-light:focus,.btn-primary-light:hover{color:#495057;background-color:#ffedca;border-color:#ffe9bd}.btn-primary-light.focus,.btn-primary-light:focus{box-shadow:0 0 0 0 rgba(228,225,217,.5)}.btn-primary-light.disabled,.btn-primary-light:disabled{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-primary-light:not(:disabled):not(.disabled).active,.btn-primary-light:not(:disabled):not(.disabled):active,.show>.btn-primary-light.dropdown-toggle{color:#495057;background-color:#ffe9bd;border-color:#ffe5b0}.btn-primary-light:not(:disabled):not(.disabled).active:focus,.btn-primary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-primary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,225,217,.5)}.btn-secondary-light{color:#495057;background-color:#fff;border-color:#fff}.btn-secondary-light.focus,.btn-secondary-light:focus,.btn-secondary-light:hover{color:#495057;background-color:#ececec;border-color:#e6e6e6}.btn-secondary-light.focus,.btn-secondary-light:focus{box-shadow:0 0 0 0 rgba(228,229,230,.5)}.btn-secondary-light.disabled,.btn-secondary-light:disabled{color:#495057;background-color:#fff;border-color:#fff}.btn-secondary-light:not(:disabled):not(.disabled).active,.btn-secondary-light:not(:disabled):not(.disabled):active,.show>.btn-secondary-light.dropdown-toggle{color:#495057;background-color:#e6e6e6;border-color:#dfdfdf}.btn-secondary-light:not(:disabled):not(.disabled).active:focus,.btn-secondary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-secondary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,229,230,.5)}.btn-tertiary{color:#fff;background-color:#257af4;border-color:#257af4}.btn-tertiary.focus,.btn-tertiary:focus,.btn-tertiary:hover{color:#fff;background-color:#0c66e7;border-color:#0b60db}.btn-tertiary.focus,.btn-tertiary:focus{box-shadow:0 0 0 0 rgba(70,142,246,.5)}.btn-tertiary.disabled,.btn-tertiary:disabled{color:#fff;background-color:#257af4;border-color:#257af4}.btn-tertiary:not(:disabled):not(.disabled).active,.btn-tertiary:not(:disabled):not(.disabled):active,.show>.btn-tertiary.dropdown-toggle{color:#fff;background-color:#0b60db;border-color:#0a5bcf}.btn-tertiary:not(:disabled):not(.disabled).active:focus,.btn-tertiary:not(:disabled):not(.disabled):active:focus,.show>.btn-tertiary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(70,142,246,.5)}.btn-tertiary-light{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-tertiary-light.focus,.btn-tertiary-light:focus,.btn-tertiary-light:hover{color:#495057;background-color:#bedffd;border-color:#b2d8fc}.btn-tertiary-light.focus,.btn-tertiary-light:focus{box-shadow:0 0 0 0 rgba(204,217,229,.5)}.btn-tertiary-light.disabled,.btn-tertiary-light:disabled{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-tertiary-light:not(:disabled):not(.disabled).active,.btn-tertiary-light:not(:disabled):not(.disabled):active,.show>.btn-tertiary-light.dropdown-toggle{color:#495057;background-color:#b2d8fc;border-color:#a5d2fc}.btn-tertiary-light:not(:disabled):not(.disabled).active:focus,.btn-tertiary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-tertiary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(204,217,229,.5)}.btn-white{color:#495057;background-color:#fff;border-color:#fff}.btn-white.focus,.btn-white:focus,.btn-white:hover{color:#495057;background-color:#ececec;border-color:#e6e6e6}.btn-white.focus,.btn-white:focus{box-shadow:0 0 0 0 rgba(228,229,230,.5)}.btn-white.disabled,.btn-white:disabled{color:#495057;background-color:#fff;border-color:#fff}.btn-white:not(:disabled):not(.disabled).active,.btn-white:not(:disabled):not(.disabled):active,.show>.btn-white.dropdown-toggle{color:#495057;background-color:#e6e6e6;border-color:#dfdfdf}.btn-white:not(:disabled):not(.disabled).active:focus,.btn-white:not(:disabled):not(.disabled):active:focus,.show>.btn-white.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,229,230,.5)}.btn-black{color:#fff;background-color:#212529;border-color:#212529}.btn-black.focus,.btn-black:focus,.btn-black:hover{color:#fff;background-color:#101214;border-color:#0a0c0d}.btn-black.focus,.btn-black:focus{box-shadow:0 0 0 0 rgba(66,70,73,.5)}.btn-black.disabled,.btn-black:disabled{color:#fff;background-color:#212529;border-color:#212529}.btn-black:not(:disabled):not(.disabled).active,.btn-black:not(:disabled):not(.disabled):active,.show>.btn-black.dropdown-toggle{color:#fff;background-color:#0a0c0d;border-color:#050506}.btn-black:not(:disabled):not(.disabled).active:focus,.btn-black:not(:disabled):not(.disabled):active:focus,.show>.btn-black.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(66,70,73,.5)}.btn-blue{color:#fff;background-color:#257af4;border-color:#257af4}.btn-blue.focus,.btn-blue:focus,.btn-blue:hover{color:#fff;background-color:#0c66e7;border-color:#0b60db}.btn-blue.focus,.btn-blue:focus{box-shadow:0 0 0 0 rgba(70,142,246,.5)}.btn-blue.disabled,.btn-blue:disabled{color:#fff;background-color:#257af4;border-color:#257af4}.btn-blue:not(:disabled):not(.disabled).active,.btn-blue:not(:disabled):not(.disabled):active,.show>.btn-blue.dropdown-toggle{color:#fff;background-color:#0b60db;border-color:#0a5bcf}.btn-blue:not(:disabled):not(.disabled).active:focus,.btn-blue:not(:disabled):not(.disabled):active:focus,.show>.btn-blue.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(70,142,246,.5)}.btn-light-blue{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-light-blue.focus,.btn-light-blue:focus,.btn-light-blue:hover{color:#495057;background-color:#bedffd;border-color:#b2d8fc}.btn-light-blue.focus,.btn-light-blue:focus{box-shadow:0 0 0 0 rgba(204,217,229,.5)}.btn-light-blue.disabled,.btn-light-blue:disabled{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-light-blue:not(:disabled):not(.disabled).active,.btn-light-blue:not(:disabled):not(.disabled):active,.show>.btn-light-blue.dropdown-toggle{color:#495057;background-color:#b2d8fc;border-color:#a5d2fc}.btn-light-blue:not(:disabled):not(.disabled).active:focus,.btn-light-blue:not(:disabled):not(.disabled):active:focus,.show>.btn-light-blue.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(204,217,229,.5)}.btn-yellow{color:#495057;background-color:#fc0;border-color:#fc0}.btn-yellow.focus,.btn-yellow:focus,.btn-yellow:hover{color:#495057;background-color:#d9ad00;border-color:#cca300}.btn-yellow.focus,.btn-yellow:focus{box-shadow:0 0 0 0 rgba(228,185,13,.5)}.btn-yellow.disabled,.btn-yellow:disabled{color:#495057;background-color:#fc0;border-color:#fc0}.btn-yellow:not(:disabled):not(.disabled).active,.btn-yellow:not(:disabled):not(.disabled):active,.show>.btn-yellow.dropdown-toggle{color:#495057;background-color:#cca300;border-color:#bf9900}.btn-yellow:not(:disabled):not(.disabled).active:focus,.btn-yellow:not(:disabled):not(.disabled):active:focus,.show>.btn-yellow.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,185,13,.5)}.btn-light-yellow{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-light-yellow.focus,.btn-light-yellow:focus,.btn-light-yellow:hover{color:#495057;background-color:#ffedca;border-color:#ffe9bd}.btn-light-yellow.focus,.btn-light-yellow:focus{box-shadow:0 0 0 0 rgba(228,225,217,.5)}.btn-light-yellow.disabled,.btn-light-yellow:disabled{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-light-yellow:not(:disabled):not(.disabled).active,.btn-light-yellow:not(:disabled):not(.disabled):active,.show>.btn-light-yellow.dropdown-toggle{color:#495057;background-color:#ffe9bd;border-color:#ffe5b0}.btn-light-yellow:not(:disabled):not(.disabled).active:focus,.btn-light-yellow:not(:disabled):not(.disabled):active:focus,.show>.btn-light-yellow.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,225,217,.5)}.btn-orange{color:#495057;background-color:#ff8c00;border-color:#ff8c00}.btn-orange.focus,.btn-orange:focus,.btn-orange:hover{color:#fff;background-color:#d97700;border-color:#cc7000}.btn-orange.focus,.btn-orange:focus{box-shadow:0 0 0 0 rgba(228,131,13,.5)}.btn-orange.disabled,.btn-orange:disabled{color:#495057;background-color:#ff8c00;border-color:#ff8c00}.btn-orange:not(:disabled):not(.disabled).active,.btn-orange:not(:disabled):not(.disabled):active,.show>.btn-orange.dropdown-toggle{color:#fff;background-color:#cc7000;border-color:#bf6900}.btn-orange:not(:disabled):not(.disabled).active:focus,.btn-orange:not(:disabled):not(.disabled):active:focus,.show>.btn-orange.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,131,13,.5)}.btn-light-orange{color:#495057;background-color:#ffe4b5;border-color:#ffe4b5}.btn-light-orange.focus,.btn-light-orange:focus,.btn-light-orange:hover{color:#495057;background-color:#ffd68f;border-color:#ffd182}.btn-light-orange.focus,.btn-light-orange:focus{box-shadow:0 0 0 0 rgba(228,206,167,.5)}.btn-light-orange.disabled,.btn-light-orange:disabled{color:#495057;background-color:#ffe4b5;border-color:#ffe4b5}.btn-light-orange:not(:disabled):not(.disabled).active,.btn-light-orange:not(:disabled):not(.disabled):active,.show>.btn-light-orange.dropdown-toggle{color:#495057;background-color:#ffd182;border-color:#ffcd75}.btn-light-orange:not(:disabled):not(.disabled).active:focus,.btn-light-orange:not(:disabled):not(.disabled):active:focus,.show>.btn-light-orange.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,206,167,.5)}.btn-red{color:#fff;background-color:#ff3939;border-color:#ff3939}.btn-red.focus,.btn-red:focus,.btn-red:hover{color:#fff;background-color:#ff1313;border-color:#ff0606}.btn-red.focus,.btn-red:focus{box-shadow:0 0 0 0 rgba(255,87,87,.5)}.btn-red.disabled,.btn-red:disabled{color:#fff;background-color:#ff3939;border-color:#ff3939}.btn-red:not(:disabled):not(.disabled).active,.btn-red:not(:disabled):not(.disabled):active,.show>.btn-red.dropdown-toggle{color:#fff;background-color:#ff0606;border-color:#f80000}.btn-red:not(:disabled):not(.disabled).active:focus,.btn-red:not(:disabled):not(.disabled):active:focus,.show>.btn-red.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,87,87,.5)}.btn-light-red{color:#495057;background-color:#ffe4e1;border-color:#ffe4e1}.btn-light-red.focus,.btn-light-red:focus,.btn-light-red:hover{color:#495057;background-color:#ffc2bb;border-color:#ffb6ae}.btn-light-red.focus,.btn-light-red:focus{box-shadow:0 0 0 0 rgba(228,206,204,.5)}.btn-light-red.disabled,.btn-light-red:disabled{color:#495057;background-color:#ffe4e1;border-color:#ffe4e1}.btn-light-red:not(:disabled):not(.disabled).active,.btn-light-red:not(:disabled):not(.disabled):active,.show>.btn-light-red.dropdown-toggle{color:#495057;background-color:#ffb6ae;border-color:#ffaba1}.btn-light-red:not(:disabled):not(.disabled).active:focus,.btn-light-red:not(:disabled):not(.disabled):active:focus,.show>.btn-light-red.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(228,206,204,.5)}.btn-medium{color:#495057;background-color:#d6dbdf;border-color:#d6dbdf}.btn-medium.focus,.btn-medium:focus,.btn-medium:hover{color:#495057;background-color:#c1c8ce;border-color:#b9c2c9}.btn-medium.focus,.btn-medium:focus{box-shadow:0 0 0 0 rgba(193,198,203,.5)}.btn-medium.disabled,.btn-medium:disabled{color:#495057;background-color:#d6dbdf;border-color:#d6dbdf}.btn-medium:not(:disabled):not(.disabled).active,.btn-medium:not(:disabled):not(.disabled):active,.show>.btn-medium.dropdown-toggle{color:#495057;background-color:#b9c2c9;border-color:#b2bcc3}.btn-medium:not(:disabled):not(.disabled).active:focus,.btn-medium:not(:disabled):not(.disabled):active:focus,.show>.btn-medium.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(193,198,203,.5)}.btn-outline-primary{color:#fc0;border-color:#fc0}.btn-outline-primary:hover{color:#495057;background-color:#fc0;border-color:#fc0}.btn-outline-primary.focus,.btn-outline-primary:focus{box-shadow:0 0 0 0 rgba(255,204,0,.5)}.btn-outline-primary.disabled,.btn-outline-primary:disabled{color:#fc0;background-color:transparent}.btn-outline-primary:not(:disabled):not(.disabled).active,.btn-outline-primary:not(:disabled):not(.disabled):active,.show>.btn-outline-primary.dropdown-toggle{color:#495057;background-color:#fc0;border-color:#fc0}.btn-outline-primary:not(:disabled):not(.disabled).active:focus,.btn-outline-primary:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-primary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,204,0,.5)}.btn-outline-secondary{color:#212529;border-color:#212529}.btn-outline-secondary:hover{color:#fff;background-color:#212529;border-color:#212529}.btn-outline-secondary.focus,.btn-outline-secondary:focus{box-shadow:0 0 0 0 rgba(33,37,41,.5)}.btn-outline-secondary.disabled,.btn-outline-secondary:disabled{color:#212529;background-color:transparent}.btn-outline-secondary:not(:disabled):not(.disabled).active,.btn-outline-secondary:not(:disabled):not(.disabled):active,.show>.btn-outline-secondary.dropdown-toggle{color:#fff;background-color:#212529;border-color:#212529}.btn-outline-secondary:not(:disabled):not(.disabled).active:focus,.btn-outline-secondary:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-secondary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(33,37,41,.5)}.btn-outline-success{color:#28a745;border-color:#28a745}.btn-outline-success:hover{color:#fff;background-color:#28a745;border-color:#28a745}.btn-outline-success.focus,.btn-outline-success:focus{box-shadow:0 0 0 0 rgba(40,167,69,.5)}.btn-outline-success.disabled,.btn-outline-success:disabled{color:#28a745;background-color:transparent}.btn-outline-success:not(:disabled):not(.disabled).active,.btn-outline-success:not(:disabled):not(.disabled):active,.show>.btn-outline-success.dropdown-toggle{color:#fff;background-color:#28a745;border-color:#28a745}.btn-outline-success:not(:disabled):not(.disabled).active:focus,.btn-outline-success:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-success.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(40,167,69,.5)}.btn-outline-info{color:#17a2b8;border-color:#17a2b8}.btn-outline-info:hover{color:#fff;background-color:#17a2b8;border-color:#17a2b8}.btn-outline-info.focus,.btn-outline-info:focus{box-shadow:0 0 0 0 rgba(23,162,184,.5)}.btn-outline-info.disabled,.btn-outline-info:disabled{color:#17a2b8;background-color:transparent}.btn-outline-info:not(:disabled):not(.disabled).active,.btn-outline-info:not(:disabled):not(.disabled):active,.show>.btn-outline-info.dropdown-toggle{color:#fff;background-color:#17a2b8;border-color:#17a2b8}.btn-outline-info:not(:disabled):not(.disabled).active:focus,.btn-outline-info:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-info.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(23,162,184,.5)}.btn-outline-warning{color:#ffc107;border-color:#ffc107}.btn-outline-warning:hover{color:#495057;background-color:#ffc107;border-color:#ffc107}.btn-outline-warning.focus,.btn-outline-warning:focus{box-shadow:0 0 0 0 rgba(255,193,7,.5)}.btn-outline-warning.disabled,.btn-outline-warning:disabled{color:#ffc107;background-color:transparent}.btn-outline-warning:not(:disabled):not(.disabled).active,.btn-outline-warning:not(:disabled):not(.disabled):active,.show>.btn-outline-warning.dropdown-toggle{color:#495057;background-color:#ffc107;border-color:#ffc107}.btn-outline-warning:not(:disabled):not(.disabled).active:focus,.btn-outline-warning:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-warning.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,193,7,.5)}.btn-outline-danger{color:#dc3545;border-color:#dc3545}.btn-outline-danger:hover{color:#fff;background-color:#dc3545;border-color:#dc3545}.btn-outline-danger.focus,.btn-outline-danger:focus{box-shadow:0 0 0 0 rgba(220,53,69,.5)}.btn-outline-danger.disabled,.btn-outline-danger:disabled{color:#dc3545;background-color:transparent}.btn-outline-danger:not(:disabled):not(.disabled).active,.btn-outline-danger:not(:disabled):not(.disabled):active,.show>.btn-outline-danger.dropdown-toggle{color:#fff;background-color:#dc3545;border-color:#dc3545}.btn-outline-danger:not(:disabled):not(.disabled).active:focus,.btn-outline-danger:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-danger.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(220,53,69,.5)}.btn-outline-light{color:#f1f6f9;border-color:#f1f6f9}.btn-outline-light:hover{color:#495057;background-color:#f1f6f9;border-color:#f1f6f9}.btn-outline-light.focus,.btn-outline-light:focus{box-shadow:0 0 0 0 rgba(241,246,249,.5)}.btn-outline-light.disabled,.btn-outline-light:disabled{color:#f1f6f9;background-color:transparent}.btn-outline-light:not(:disabled):not(.disabled).active,.btn-outline-light:not(:disabled):not(.disabled):active,.show>.btn-outline-light.dropdown-toggle{color:#495057;background-color:#f1f6f9;border-color:#f1f6f9}.btn-outline-light:not(:disabled):not(.disabled).active:focus,.btn-outline-light:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(241,246,249,.5)}.btn-outline-dark{color:#495057;border-color:#495057}.btn-outline-dark:hover{color:#fff;background-color:#495057;border-color:#495057}.btn-outline-dark.focus,.btn-outline-dark:focus{box-shadow:0 0 0 0 rgba(73,80,87,.5)}.btn-outline-dark.disabled,.btn-outline-dark:disabled{color:#495057;background-color:transparent}.btn-outline-dark:not(:disabled):not(.disabled).active,.btn-outline-dark:not(:disabled):not(.disabled):active,.show>.btn-outline-dark.dropdown-toggle{color:#fff;background-color:#495057;border-color:#495057}.btn-outline-dark:not(:disabled):not(.disabled).active:focus,.btn-outline-dark:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-dark.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(73,80,87,.5)}.btn-outline-primary-light{color:#fffaf0;border-color:#fffaf0}.btn-outline-primary-light:hover{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-outline-primary-light.focus,.btn-outline-primary-light:focus{box-shadow:0 0 0 0 rgba(255,250,240,.5)}.btn-outline-primary-light.disabled,.btn-outline-primary-light:disabled{color:#fffaf0;background-color:transparent}.btn-outline-primary-light:not(:disabled):not(.disabled).active,.btn-outline-primary-light:not(:disabled):not(.disabled):active,.show>.btn-outline-primary-light.dropdown-toggle{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-outline-primary-light:not(:disabled):not(.disabled).active:focus,.btn-outline-primary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-primary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,250,240,.5)}.btn-outline-secondary-light{color:#fff;border-color:#fff}.btn-outline-secondary-light:hover{color:#495057;background-color:#fff;border-color:#fff}.btn-outline-secondary-light.focus,.btn-outline-secondary-light:focus{box-shadow:0 0 0 0 hsla(0,0%,100%,.5)}.btn-outline-secondary-light.disabled,.btn-outline-secondary-light:disabled{color:#fff;background-color:transparent}.btn-outline-secondary-light:not(:disabled):not(.disabled).active,.btn-outline-secondary-light:not(:disabled):not(.disabled):active,.show>.btn-outline-secondary-light.dropdown-toggle{color:#495057;background-color:#fff;border-color:#fff}.btn-outline-secondary-light:not(:disabled):not(.disabled).active:focus,.btn-outline-secondary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-secondary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 hsla(0,0%,100%,.5)}.btn-outline-tertiary{color:#257af4;border-color:#257af4}.btn-outline-tertiary:hover{color:#fff;background-color:#257af4;border-color:#257af4}.btn-outline-tertiary.focus,.btn-outline-tertiary:focus{box-shadow:0 0 0 0 rgba(37,122,244,.5)}.btn-outline-tertiary.disabled,.btn-outline-tertiary:disabled{color:#257af4;background-color:transparent}.btn-outline-tertiary:not(:disabled):not(.disabled).active,.btn-outline-tertiary:not(:disabled):not(.disabled):active,.show>.btn-outline-tertiary.dropdown-toggle{color:#fff;background-color:#257af4;border-color:#257af4}.btn-outline-tertiary:not(:disabled):not(.disabled).active:focus,.btn-outline-tertiary:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-tertiary.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(37,122,244,.5)}.btn-outline-tertiary-light{color:#e3f1fe;border-color:#e3f1fe}.btn-outline-tertiary-light:hover{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-outline-tertiary-light.focus,.btn-outline-tertiary-light:focus{box-shadow:0 0 0 0 rgba(227,241,254,.5)}.btn-outline-tertiary-light.disabled,.btn-outline-tertiary-light:disabled{color:#e3f1fe;background-color:transparent}.btn-outline-tertiary-light:not(:disabled):not(.disabled).active,.btn-outline-tertiary-light:not(:disabled):not(.disabled):active,.show>.btn-outline-tertiary-light.dropdown-toggle{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-outline-tertiary-light:not(:disabled):not(.disabled).active:focus,.btn-outline-tertiary-light:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-tertiary-light.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(227,241,254,.5)}.btn-outline-white{color:#fff;border-color:#fff}.btn-outline-white:hover{color:#495057;background-color:#fff;border-color:#fff}.btn-outline-white.focus,.btn-outline-white:focus{box-shadow:0 0 0 0 hsla(0,0%,100%,.5)}.btn-outline-white.disabled,.btn-outline-white:disabled{color:#fff;background-color:transparent}.btn-outline-white:not(:disabled):not(.disabled).active,.btn-outline-white:not(:disabled):not(.disabled):active,.show>.btn-outline-white.dropdown-toggle{color:#495057;background-color:#fff;border-color:#fff}.btn-outline-white:not(:disabled):not(.disabled).active:focus,.btn-outline-white:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-white.dropdown-toggle:focus{box-shadow:0 0 0 0 hsla(0,0%,100%,.5)}.btn-outline-black{color:#212529;border-color:#212529}.btn-outline-black:hover{color:#fff;background-color:#212529;border-color:#212529}.btn-outline-black.focus,.btn-outline-black:focus{box-shadow:0 0 0 0 rgba(33,37,41,.5)}.btn-outline-black.disabled,.btn-outline-black:disabled{color:#212529;background-color:transparent}.btn-outline-black:not(:disabled):not(.disabled).active,.btn-outline-black:not(:disabled):not(.disabled):active,.show>.btn-outline-black.dropdown-toggle{color:#fff;background-color:#212529;border-color:#212529}.btn-outline-black:not(:disabled):not(.disabled).active:focus,.btn-outline-black:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-black.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(33,37,41,.5)}.btn-outline-blue{color:#257af4;border-color:#257af4}.btn-outline-blue:hover{color:#fff;background-color:#257af4;border-color:#257af4}.btn-outline-blue.focus,.btn-outline-blue:focus{box-shadow:0 0 0 0 rgba(37,122,244,.5)}.btn-outline-blue.disabled,.btn-outline-blue:disabled{color:#257af4;background-color:transparent}.btn-outline-blue:not(:disabled):not(.disabled).active,.btn-outline-blue:not(:disabled):not(.disabled):active,.show>.btn-outline-blue.dropdown-toggle{color:#fff;background-color:#257af4;border-color:#257af4}.btn-outline-blue:not(:disabled):not(.disabled).active:focus,.btn-outline-blue:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-blue.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(37,122,244,.5)}.btn-outline-light-blue{color:#e3f1fe;border-color:#e3f1fe}.btn-outline-light-blue:hover{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-outline-light-blue.focus,.btn-outline-light-blue:focus{box-shadow:0 0 0 0 rgba(227,241,254,.5)}.btn-outline-light-blue.disabled,.btn-outline-light-blue:disabled{color:#e3f1fe;background-color:transparent}.btn-outline-light-blue:not(:disabled):not(.disabled).active,.btn-outline-light-blue:not(:disabled):not(.disabled):active,.show>.btn-outline-light-blue.dropdown-toggle{color:#495057;background-color:#e3f1fe;border-color:#e3f1fe}.btn-outline-light-blue:not(:disabled):not(.disabled).active:focus,.btn-outline-light-blue:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-light-blue.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(227,241,254,.5)}.btn-outline-yellow{color:#fc0;border-color:#fc0}.btn-outline-yellow:hover{color:#495057;background-color:#fc0;border-color:#fc0}.btn-outline-yellow.focus,.btn-outline-yellow:focus{box-shadow:0 0 0 0 rgba(255,204,0,.5)}.btn-outline-yellow.disabled,.btn-outline-yellow:disabled{color:#fc0;background-color:transparent}.btn-outline-yellow:not(:disabled):not(.disabled).active,.btn-outline-yellow:not(:disabled):not(.disabled):active,.show>.btn-outline-yellow.dropdown-toggle{color:#495057;background-color:#fc0;border-color:#fc0}.btn-outline-yellow:not(:disabled):not(.disabled).active:focus,.btn-outline-yellow:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-yellow.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,204,0,.5)}.btn-outline-light-yellow{color:#fffaf0;border-color:#fffaf0}.btn-outline-light-yellow:hover{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-outline-light-yellow.focus,.btn-outline-light-yellow:focus{box-shadow:0 0 0 0 rgba(255,250,240,.5)}.btn-outline-light-yellow.disabled,.btn-outline-light-yellow:disabled{color:#fffaf0;background-color:transparent}.btn-outline-light-yellow:not(:disabled):not(.disabled).active,.btn-outline-light-yellow:not(:disabled):not(.disabled):active,.show>.btn-outline-light-yellow.dropdown-toggle{color:#495057;background-color:#fffaf0;border-color:#fffaf0}.btn-outline-light-yellow:not(:disabled):not(.disabled).active:focus,.btn-outline-light-yellow:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-light-yellow.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,250,240,.5)}.btn-outline-orange{color:#ff8c00;border-color:#ff8c00}.btn-outline-orange:hover{color:#495057;background-color:#ff8c00;border-color:#ff8c00}.btn-outline-orange.focus,.btn-outline-orange:focus{box-shadow:0 0 0 0 rgba(255,140,0,.5)}.btn-outline-orange.disabled,.btn-outline-orange:disabled{color:#ff8c00;background-color:transparent}.btn-outline-orange:not(:disabled):not(.disabled).active,.btn-outline-orange:not(:disabled):not(.disabled):active,.show>.btn-outline-orange.dropdown-toggle{color:#495057;background-color:#ff8c00;border-color:#ff8c00}.btn-outline-orange:not(:disabled):not(.disabled).active:focus,.btn-outline-orange:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-orange.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,140,0,.5)}.btn-outline-light-orange{color:#ffe4b5;border-color:#ffe4b5}.btn-outline-light-orange:hover{color:#495057;background-color:#ffe4b5;border-color:#ffe4b5}.btn-outline-light-orange.focus,.btn-outline-light-orange:focus{box-shadow:0 0 0 0 rgba(255,228,181,.5)}.btn-outline-light-orange.disabled,.btn-outline-light-orange:disabled{color:#ffe4b5;background-color:transparent}.btn-outline-light-orange:not(:disabled):not(.disabled).active,.btn-outline-light-orange:not(:disabled):not(.disabled):active,.show>.btn-outline-light-orange.dropdown-toggle{color:#495057;background-color:#ffe4b5;border-color:#ffe4b5}.btn-outline-light-orange:not(:disabled):not(.disabled).active:focus,.btn-outline-light-orange:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-light-orange.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,228,181,.5)}.btn-outline-red{color:#ff3939;border-color:#ff3939}.btn-outline-red:hover{color:#fff;background-color:#ff3939;border-color:#ff3939}.btn-outline-red.focus,.btn-outline-red:focus{box-shadow:0 0 0 0 rgba(255,57,57,.5)}.btn-outline-red.disabled,.btn-outline-red:disabled{color:#ff3939;background-color:transparent}.btn-outline-red:not(:disabled):not(.disabled).active,.btn-outline-red:not(:disabled):not(.disabled):active,.show>.btn-outline-red.dropdown-toggle{color:#fff;background-color:#ff3939;border-color:#ff3939}.btn-outline-red:not(:disabled):not(.disabled).active:focus,.btn-outline-red:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-red.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,57,57,.5)}.btn-outline-light-red{color:#ffe4e1;border-color:#ffe4e1}.btn-outline-light-red:hover{color:#495057;background-color:#ffe4e1;border-color:#ffe4e1}.btn-outline-light-red.focus,.btn-outline-light-red:focus{box-shadow:0 0 0 0 rgba(255,228,225,.5)}.btn-outline-light-red.disabled,.btn-outline-light-red:disabled{color:#ffe4e1;background-color:transparent}.btn-outline-light-red:not(:disabled):not(.disabled).active,.btn-outline-light-red:not(:disabled):not(.disabled):active,.show>.btn-outline-light-red.dropdown-toggle{color:#495057;background-color:#ffe4e1;border-color:#ffe4e1}.btn-outline-light-red:not(:disabled):not(.disabled).active:focus,.btn-outline-light-red:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-light-red.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(255,228,225,.5)}.btn-outline-medium{color:#d6dbdf;border-color:#d6dbdf}.btn-outline-medium:hover{color:#495057;background-color:#d6dbdf;border-color:#d6dbdf}.btn-outline-medium.focus,.btn-outline-medium:focus{box-shadow:0 0 0 0 rgba(214,219,223,.5)}.btn-outline-medium.disabled,.btn-outline-medium:disabled{color:#d6dbdf;background-color:transparent}.btn-outline-medium:not(:disabled):not(.disabled).active,.btn-outline-medium:not(:disabled):not(.disabled):active,.show>.btn-outline-medium.dropdown-toggle{color:#495057;background-color:#d6dbdf;border-color:#d6dbdf}.btn-outline-medium:not(:disabled):not(.disabled).active:focus,.btn-outline-medium:not(:disabled):not(.disabled):active:focus,.show>.btn-outline-medium.dropdown-toggle:focus{box-shadow:0 0 0 0 rgba(214,219,223,.5)}.btn-link{font-weight:400;color:#ff8c00;text-decoration:none}.btn-link:hover{color:#ff8c00;text-decoration:underline}.btn-link.focus,.btn-link:focus{text-decoration:underline;box-shadow:none}.btn-link.disabled,.btn-link:disabled{color:#d6dbdf;pointer-events:none}.btn-group-lg>.btn,.btn-lg{padding:16px 32px;font-size:1.125rem;line-height:26px;border-radius:8px}.btn-group-sm>.btn,.btn-sm{padding:6px 12px;font-size:.75rem;line-height:16px;border-radius:4px}.btn-block{display:block;width:100%}.btn-block+.btn-block{margin-top:24px}input[type=button].btn-block,input[type=reset].btn-block,input[type=submit].btn-block{width:100%}.fade{transition:opacity .15s linear}@media(prefers-reduced-motion:reduce){.fade{transition:none}}.fade:not(.show){opacity:0}.collapse:not(.show){display:none}.collapsing{position:relative;height:0;overflow:hidden;transition:height .35s ease}@media(prefers-reduced-motion:reduce){.collapsing{transition:none}}.dropdown,.dropleft,.dropright,.dropup{position:relative}.dropdown-toggle{white-space:nowrap}.dropdown-toggle:after{display:inline-block;margin-left:.255em;vertical-align:.255em;content:"";border-top:.3em solid;border-right:.3em solid transparent;border-bottom:0;border-left:.3em solid transparent}.dropdown-toggle:empty:after{margin-left:0}.dropdown-menu{position:absolute;top:100%;left:0;z-index:1000;display:none;float:left;min-width:10rem;padding:.5rem 0;margin:.125rem 0 0;font-size:1rem;color:#212529;text-align:left;list-style:none;background-color:#fff;background-clip:padding-box;border:1px solid rgba(33,37,41,.15);border-radius:8px}.dropdown-menu-left{right:auto;left:0}.dropdown-menu-right{right:0;left:auto}@media(min-width:400px){.dropdown-menu-xs-left{right:auto;left:0}.dropdown-menu-xs-right{right:0;left:auto}}@media(min-width:616px){.dropdown-menu-sm-left{right:auto;left:0}.dropdown-menu-sm-right{right:0;left:auto}}@media(min-width:768px){.dropdown-menu-md-left{right:auto;left:0}.dropdown-menu-md-right{right:0;left:auto}}@media(min-width:980px){.dropdown-menu-lg-left{right:auto;left:0}.dropdown-menu-lg-right{right:0;left:auto}}@media(min-width:1240px){.dropdown-menu-xl-left{right:auto;left:0}.dropdown-menu-xl-right{right:0;left:auto}}.dropup .dropdown-menu{top:auto;bottom:100%;margin-top:0;margin-bottom:.125rem}.dropup .dropdown-toggle:after{display:inline-block;margin-left:.255em;vertical-align:.255em;content:"";border-top:0;border-right:.3em solid transparent;border-bottom:.3em solid;border-left:.3em solid transparent}.dropup .dropdown-toggle:empty:after{margin-left:0}.dropright .dropdown-menu{top:0;right:auto;left:100%;margin-top:0;margin-left:.125rem}.dropright .dropdown-toggle:after{display:inline-block;margin-left:.255em;vertical-align:.255em;content:"";border-top:.3em solid transparent;border-right:0;border-bottom:.3em solid transparent;border-left:.3em solid}.dropright .dropdown-toggle:empty:after{margin-left:0}.dropright .dropdown-toggle:after{vertical-align:0}.dropleft .dropdown-menu{top:0;right:100%;left:auto;margin-top:0;margin-right:.125rem}.dropleft .dropdown-toggle:after{display:inline-block;margin-left:.255em;vertical-align:.255em;content:"";display:none}.dropleft .dropdown-toggle:before{display:inline-block;margin-right:.255em;vertical-align:.255em;content:"";border-top:.3em solid transparent;border-right:.3em solid;border-bottom:.3em solid transparent}.dropleft .dropdown-toggle:empty:after{margin-left:0}.dropleft .dropdown-toggle:before{vertical-align:0}.dropdown-menu[x-placement^=bottom],.dropdown-menu[x-placement^=left],.dropdown-menu[x-placement^=right],.dropdown-menu[x-placement^=top]{right:auto;bottom:auto}.dropdown-divider{height:0;margin:4px 0;overflow:hidden;border-top:1px solid #e9ecef}.dropdown-item{display:block;width:100%;padding:.25rem 1.5rem;clear:both;font-weight:400;color:#495057;text-align:inherit;white-space:nowrap;background-color:transparent;border:0}.dropdown-item:focus,.dropdown-item:hover{color:#3d4349;text-decoration:none;background-color:#f1f6f9}.dropdown-item.active,.dropdown-item:active{color:#fff;text-decoration:none;background-color:#fc0}.dropdown-item.disabled,.dropdown-item:disabled{color:#6c757d;pointer-events:none;background-color:transparent}.dropdown-menu.show{display:block}.dropdown-header{display:block;padding:.5rem 1.5rem;margin-bottom:0;font-size:.875rem;color:#6c757d;white-space:nowrap}.dropdown-item-text{display:block;padding:.25rem 1.5rem;color:#495057}.btn-group,.btn-group-vertical{position:relative;display:inline-flex;vertical-align:middle}.btn-group-vertical>.btn,.btn-group>.btn{position:relative;flex:1 1 auto}.btn-group-vertical>.btn.active,.btn-group-vertical>.btn:active,.btn-group-vertical>.btn:focus,.btn-group-vertical>.btn:hover,.btn-group>.btn.active,.btn-group>.btn:active,.btn-group>.btn:focus,.btn-group>.btn:hover{z-index:1}.btn-toolbar{display:flex;flex-wrap:wrap;justify-content:flex-start}.btn-toolbar .input-group{width:auto}.btn-group>.btn-group:not(:first-child),.btn-group>.btn:not(:first-child){margin-left:-1px}.btn-group>.btn-group:not(:last-child)>.btn,.btn-group>.btn:not(:last-child):not(.dropdown-toggle){border-top-right-radius:0;border-bottom-right-radius:0}.btn-group>.btn-group:not(:first-child)>.btn,.btn-group>.btn:not(:first-child){border-top-left-radius:0;border-bottom-left-radius:0}.dropdown-toggle-split{padding-right:24px;padding-left:24px}.dropdown-toggle-split:after,.dropright .dropdown-toggle-split:after,.dropup .dropdown-toggle-split:after{margin-left:0}.dropleft .dropdown-toggle-split:before{margin-right:0}.btn-group-sm>.btn+.dropdown-toggle-split,.btn-sm+.dropdown-toggle-split{padding-right:9px;padding-left:9px}.btn-group-lg>.btn+.dropdown-toggle-split,.btn-lg+.dropdown-toggle-split{padding-right:24px;padding-left:24px}.btn-group-vertical{flex-direction:column;align-items:flex-start;justify-content:center}.btn-group-vertical>.btn,.btn-group-vertical>.btn-group{width:100%}.btn-group-vertical>.btn-group:not(:first-child),.btn-group-vertical>.btn:not(:first-child){margin-top:-1px}.btn-group-vertical>.btn-group:not(:last-child)>.btn,.btn-group-vertical>.btn:not(:last-child):not(.dropdown-toggle){border-bottom-right-radius:0;border-bottom-left-radius:0}.btn-group-vertical>.btn-group:not(:first-child)>.btn,.btn-group-vertical>.btn:not(:first-child){border-top-left-radius:0;border-top-right-radius:0}.btn-group-toggle>.btn,.btn-group-toggle>.btn-group>.btn{margin-bottom:0}.btn-group-toggle>.btn-group>.btn input[type=checkbox],.btn-group-toggle>.btn-group>.btn input[type=radio],.btn-group-toggle>.btn input[type=checkbox],.btn-group-toggle>.btn input[type=radio]{position:absolute;clip:rect(0,0,0,0);pointer-events:none}.input-group{position:relative;display:flex;flex-wrap:wrap;align-items:stretch;width:100%}.input-group>.custom-file,.input-group>.custom-select,.input-group>.form-control,.input-group>.form-control-plaintext{position:relative;flex:1 1 0%;min-width:0;margin-bottom:0}.input-group>.custom-file+.custom-file,.input-group>.custom-file+.custom-select,.input-group>.custom-file+.form-control,.input-group>.custom-select+.custom-file,.input-group>.custom-select+.custom-select,.input-group>.custom-select+.form-control,.input-group>.form-control+.custom-file,.input-group>.form-control+.custom-select,.input-group>.form-control+.form-control,.input-group>.form-control-plaintext+.custom-file,.input-group>.form-control-plaintext+.custom-select,.input-group>.form-control-plaintext+.form-control{margin-left:-1px}.input-group>.custom-file .custom-file-input:focus~.custom-file-label,.input-group>.custom-select:focus,.input-group>.form-control:focus{z-index:3}.input-group>.custom-file .custom-file-input:focus{z-index:4}.input-group>.custom-select:not(:last-child),.input-group>.form-control:not(:last-child){border-top-right-radius:0;border-bottom-right-radius:0}.input-group>.custom-select:not(:first-child),.input-group>.form-control:not(:first-child){border-top-left-radius:0;border-bottom-left-radius:0}.input-group>.custom-file{display:flex;align-items:center}.input-group>.custom-file:not(:last-child) .custom-file-label,.input-group>.custom-file:not(:last-child) .custom-file-label:after{border-top-right-radius:0;border-bottom-right-radius:0}.input-group>.custom-file:not(:first-child) .custom-file-label{border-top-left-radius:0;border-bottom-left-radius:0}.input-group-append,.input-group-prepend{display:flex}.input-group-append .btn,.input-group-prepend .btn{position:relative;z-index:2}.input-group-append .btn:focus,.input-group-prepend .btn:focus{z-index:3}.input-group-append .btn+.btn,.input-group-append .btn+.input-group-text,.input-group-append .input-group-text+.btn,.input-group-append .input-group-text+.input-group-text,.input-group-prepend .btn+.btn,.input-group-prepend .btn+.input-group-text,.input-group-prepend .input-group-text+.btn,.input-group-prepend .input-group-text+.input-group-text{margin-left:-1px}.input-group-prepend{margin-right:-1px}.input-group-append{margin-left:-1px}.input-group-text{display:flex;align-items:center;padding:.375rem .75rem;margin-bottom:0;font-size:1rem;font-weight:400;line-height:1.5;color:#6c757d;text-align:center;white-space:nowrap;background-color:#e9ecef;border:1px solid #ced4da;border-radius:8px}.input-group-text input[type=checkbox],.input-group-text input[type=radio]{margin-top:0}.input-group-lg>.custom-select,.input-group-lg>.form-control:not(textarea){height:calc(1.5em + 1rem + 2px)}.input-group-lg>.custom-select,.input-group-lg>.form-control,.input-group-lg>.input-group-append>.btn,.input-group-lg>.input-group-append>.input-group-text,.input-group-lg>.input-group-prepend>.btn,.input-group-lg>.input-group-prepend>.input-group-text{padding:.5rem 1rem;font-size:1.125rem;line-height:1.5;border-radius:8px}.input-group-sm>.custom-select,.input-group-sm>.form-control:not(textarea){height:calc(1.5em + .5rem + 2px)}.input-group-sm>.custom-select,.input-group-sm>.form-control,.input-group-sm>.input-group-append>.btn,.input-group-sm>.input-group-append>.input-group-text,.input-group-sm>.input-group-prepend>.btn,.input-group-sm>.input-group-prepend>.input-group-text{padding:.25rem .5rem;font-size:.875rem;line-height:1.5;border-radius:4px}.input-group-lg>.custom-select,.input-group-sm>.custom-select{padding-right:1.75rem}.input-group>.input-group-append:last-child>.btn:not(:last-child):not(.dropdown-toggle),.input-group>.input-group-append:last-child>.input-group-text:not(:last-child),.input-group>.input-group-append:not(:last-child)>.btn,.input-group>.input-group-append:not(:last-child)>.input-group-text,.input-group>.input-group-prepend>.btn,.input-group>.input-group-prepend>.input-group-text{border-top-right-radius:0;border-bottom-right-radius:0}.input-group>.input-group-append>.btn,.input-group>.input-group-append>.input-group-text,.input-group>.input-group-prepend:first-child>.btn:not(:first-child),.input-group>.input-group-prepend:first-child>.input-group-text:not(:first-child),.input-group>.input-group-prepend:not(:first-child)>.btn,.input-group>.input-group-prepend:not(:first-child)>.input-group-text{border-top-left-radius:0;border-bottom-left-radius:0}.custom-control{position:relative;display:block;min-height:1.5rem;padding-left:1.5rem}.custom-control-inline{display:inline-flex;margin-right:1rem}.custom-control-input{position:absolute;left:0;z-index:-1;width:1rem;height:1.25rem;opacity:0}.custom-control-input:checked~.custom-control-label:before{color:#fff;border-color:#fc0;background-color:#fc0}.custom-control-input:focus~.custom-control-label:before{box-shadow:0 0 0 .2rem rgba(255,204,0,.25)}.custom-control-input:focus:not(:checked)~.custom-control-label:before{border-color:#ffe680}.custom-control-input:not(:disabled):active~.custom-control-label:before{color:#fff;background-color:#fff0b3;border-color:#fff0b3}.custom-control-input:disabled~.custom-control-label,.custom-control-input[disabled]~.custom-control-label{color:#6c757d}.custom-control-input:disabled~.custom-control-label:before,.custom-control-input[disabled]~.custom-control-label:before{background-color:#e9ecef}.custom-control-label{position:relative;margin-bottom:0;vertical-align:top}.custom-control-label:before{pointer-events:none;background-color:#fff;border:1px solid #d6dbdf}.custom-control-label:after,.custom-control-label:before{position:absolute;top:.25rem;left:-1.5rem;display:block;width:1rem;height:1rem;content:""}.custom-control-label:after{background:no-repeat 50%/50% 50%}.custom-checkbox .custom-control-label:before{border-radius:8px}.custom-checkbox .custom-control-input:checked~.custom-control-label:after{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='8' height='8'%3E%3Cpath fill='%23fff' d='M6.564.75l-3.59 3.612-1.538-1.55L0 4.26l2.974 2.99L8 2.193z'/%3E%3C/svg%3E")}.custom-checkbox .custom-control-input:indeterminate~.custom-control-label:before{border-color:#fc0;background-color:#fc0}.custom-checkbox .custom-control-input:indeterminate~.custom-control-label:after{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='4' height='4'%3E%3Cpath stroke='%23fff' d='M0 2h4'/%3E%3C/svg%3E")}.custom-checkbox .custom-control-input:disabled:checked~.custom-control-label:before{background-color:rgba(255,204,0,.5)}.custom-checkbox .custom-control-input:disabled:indeterminate~.custom-control-label:before{background-color:rgba(255,204,0,.5)}.custom-radio .custom-control-label:before{border-radius:50%}.custom-radio .custom-control-input:checked~.custom-control-label:after{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='12' height='12' viewBox='-4 -4 8 8'%3E%3Ccircle r='3' fill='%23fff'/%3E%3C/svg%3E")}.custom-radio .custom-control-input:disabled:checked~.custom-control-label:before{background-color:rgba(255,204,0,.5)}.custom-switch{padding-left:2.25rem}.custom-switch .custom-control-label:before{left:-2.25rem;width:1.75rem;pointer-events:all;border-radius:.5rem}.custom-switch .custom-control-label:after{top:calc(.25rem + 2px);left:calc(-2.25rem + 2px);width:calc(1rem - 4px);height:calc(1rem - 4px);background-color:#d6dbdf;border-radius:.5rem;transition:transform .15s ease-in-out,background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out}@media(prefers-reduced-motion:reduce){.custom-switch .custom-control-label:after{transition:none}}.custom-switch .custom-control-input:checked~.custom-control-label:after{background-color:#fff;transform:translateX(.75rem)}.custom-switch .custom-control-input:disabled:checked~.custom-control-label:before{background-color:rgba(255,204,0,.5)}.custom-select{display:inline-block;width:100%;height:calc(1.5em + .75rem + 2px);padding:.375rem 1.75rem .375rem .75rem;font-size:1rem;font-weight:400;line-height:1.5;color:#6c757d;vertical-align:middle;background:#fff url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='4' height='5'%3E%3Cpath fill='%23343a40' d='M2 0L0 2h4zm0 5L0 3h4z'/%3E%3C/svg%3E") no-repeat right .75rem center/8px 10px;border:1px solid #ced4da;border-radius:8px;-webkit-appearance:none;-moz-appearance:none;appearance:none}.custom-select:focus{border-color:#ffe680;outline:0;box-shadow:0 0 0 .2rem rgba(255,204,0,.25)}.custom-select:focus::-ms-value{color:#6c757d;background-color:#fff}.custom-select[multiple],.custom-select[size]:not([size="1"]){height:auto;padding-right:.75rem;background-image:none}.custom-select:disabled{color:#6c757d;background-color:#e9ecef}.custom-select::-ms-expand{display:none}.custom-select:-moz-focusring{color:transparent;text-shadow:0 0 0 #6c757d}.custom-select-sm{height:calc(1.5em + .5rem + 2px);padding-top:.25rem;padding-bottom:.25rem;padding-left:.5rem;font-size:.875rem}.custom-select-lg{height:calc(1.5em + 1rem + 2px);padding-top:.5rem;padding-bottom:.5rem;padding-left:1rem;font-size:1.125rem}.custom-file{display:inline-block;margin-bottom:0}.custom-file,.custom-file-input{position:relative;width:100%;height:calc(1.5em + .75rem + 2px)}.custom-file-input{z-index:2;margin:0;opacity:0}.custom-file-input:focus~.custom-file-label{border-color:#ffe680;box-shadow:0 0 0 .2rem rgba(255,204,0,.25)}.custom-file-input:disabled~.custom-file-label,.custom-file-input[disabled]~.custom-file-label{background-color:#e9ecef}.custom-file-input:lang(en)~.custom-file-label:after{content:"Browse"}.custom-file-input~.custom-file-label[data-browse]:after{content:attr(data-browse)}.custom-file-label{left:0;z-index:1;height:calc(1.5em + .75rem + 2px);font-weight:400;background-color:#fff;border:1px solid #ced4da;border-radius:8px}.custom-file-label,.custom-file-label:after{position:absolute;top:0;right:0;padding:.375rem .75rem;line-height:1.5;color:#6c757d}.custom-file-label:after{bottom:0;z-index:3;display:block;height:calc(1.5em + .75rem);content:"Browse";background-color:#e9ecef;border-left:inherit;border-radius:0 8px 8px 0}.custom-range{width:100%;height:1.4rem;padding:0;background-color:transparent;-webkit-appearance:none;-moz-appearance:none;appearance:none}.custom-range:focus{outline:none}.custom-range:focus::-webkit-slider-thumb{box-shadow:0 0 0 1px #fff,0 0 0 .2rem rgba(255,204,0,.25)}.custom-range:focus::-moz-range-thumb{box-shadow:0 0 0 1px #fff,0 0 0 .2rem rgba(255,204,0,.25)}.custom-range:focus::-ms-thumb{box-shadow:0 0 0 1px #fff,0 0 0 .2rem rgba(255,204,0,.25)}.custom-range::-moz-focus-outer{border:0}.custom-range::-webkit-slider-thumb{width:1rem;height:1rem;margin-top:-.25rem;background-color:#fc0;border:0;border-radius:1rem;-webkit-transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;-webkit-appearance:none;appearance:none}@media(prefers-reduced-motion:reduce){.custom-range::-webkit-slider-thumb{-webkit-transition:none;transition:none}}.custom-range::-webkit-slider-thumb:active{background-color:#fff0b3}.custom-range::-webkit-slider-runnable-track{width:100%;height:.5rem;color:transparent;cursor:pointer;background-color:#dee2e6;border-color:transparent;border-radius:1rem}.custom-range::-moz-range-thumb{width:1rem;height:1rem;background-color:#fc0;border:0;border-radius:1rem;-moz-transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;-moz-appearance:none;appearance:none}@media(prefers-reduced-motion:reduce){.custom-range::-moz-range-thumb{-moz-transition:none;transition:none}}.custom-range::-moz-range-thumb:active{background-color:#fff0b3}.custom-range::-moz-range-track{width:100%;height:.5rem;color:transparent;cursor:pointer;background-color:#dee2e6;border-color:transparent;border-radius:1rem}.custom-range::-ms-thumb{width:1rem;height:1rem;margin-top:0;margin-right:.2rem;margin-left:.2rem;background-color:#fc0;border:0;border-radius:1rem;-ms-transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out;appearance:none}@media(prefers-reduced-motion:reduce){.custom-range::-ms-thumb{-ms-transition:none;transition:none}}.custom-range::-ms-thumb:active{background-color:#fff0b3}.custom-range::-ms-track{width:100%;height:.5rem;color:transparent;cursor:pointer;background-color:transparent;border-color:transparent;border-width:.5rem}.custom-range::-ms-fill-lower,.custom-range::-ms-fill-upper{background-color:#dee2e6;border-radius:1rem}.custom-range::-ms-fill-upper{margin-right:15px}.custom-range:disabled::-webkit-slider-thumb{background-color:#d6dbdf}.custom-range:disabled::-webkit-slider-runnable-track{cursor:default}.custom-range:disabled::-moz-range-thumb{background-color:#d6dbdf}.custom-range:disabled::-moz-range-track{cursor:default}.custom-range:disabled::-ms-thumb{background-color:#d6dbdf}.custom-control-label:before,.custom-file-label,.custom-select{transition:background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out}@media(prefers-reduced-motion:reduce){.custom-control-label:before,.custom-file-label,.custom-select{transition:none}}.nav{display:flex;flex-wrap:wrap;padding-left:0;margin-bottom:0;list-style:none}.nav-link{display:block;padding:0}.nav-link:focus,.nav-link:hover{text-decoration:none}.nav-link.disabled{color:#d6dbdf;pointer-events:none;cursor:default}.nav-tabs{border-bottom:1px solid #6c757d}.nav-tabs .nav-item{margin-bottom:-1px}.nav-tabs .nav-link{border:1px solid transparent;border-top-left-radius:8px;border-top-right-radius:8px}.nav-tabs .nav-link:focus,.nav-tabs .nav-link:hover{border-color:transparent}.nav-tabs .nav-link.disabled{color:#d6dbdf;background-color:transparent;border-color:transparent}.nav-tabs .nav-item.show .nav-link,.nav-tabs .nav-link.active{color:#257af4;background-color:#fff;border-color:#6c757d}.nav-tabs .dropdown-menu{margin-top:-1px;border-top-left-radius:0;border-top-right-radius:0}.nav-pills .nav-link{border-radius:8px}.nav-pills .nav-link.active,.nav-pills .show>.nav-link{color:#fff;background-color:#fc0}.nav-fill .nav-item{flex:1 1 auto;text-align:center}.nav-justified .nav-item{flex-basis:0;flex-grow:1;text-align:center}.tab-content>.tab-pane{display:none}.tab-content>.active{display:block}.navbar{position:relative;padding:24px 0}.navbar,.navbar .container,.navbar .container-fluid,.navbar .container-lg,.navbar .container-md,.navbar .container-sm,.navbar .container-xl,.navbar .container-xs{display:flex;flex-wrap:wrap;align-items:center;justify-content:space-between}.navbar-brand{display:inline-block;padding-top:-.09375rem;padding-bottom:-.09375rem;margin-right:0;font-size:1.125rem;line-height:inherit;white-space:nowrap}.navbar-brand:focus,.navbar-brand:hover{text-decoration:none}.navbar-nav{display:flex;flex-direction:column;padding-left:0;margin-bottom:0;list-style:none}.navbar-nav .nav-link{padding-right:0;padding-left:0}.navbar-nav .dropdown-menu{position:static;float:none}.navbar-text{display:inline-block;padding-top:0;padding-bottom:0}.navbar-collapse{flex-basis:100%;flex-grow:1;align-items:center}.navbar-toggler{padding:.25rem .75rem;font-size:1.125rem;line-height:1;background-color:transparent;border:1px solid transparent;border-radius:8px}.navbar-toggler:focus,.navbar-toggler:hover{text-decoration:none}.navbar-toggler-icon{display:inline-block;width:1.5em;height:1.5em;vertical-align:middle;content:"";background:no-repeat 50%;background-size:100% 100%}@media(max-width:399.98px){.navbar-expand-xs>.container,.navbar-expand-xs>.container-fluid,.navbar-expand-xs>.container-lg,.navbar-expand-xs>.container-md,.navbar-expand-xs>.container-sm,.navbar-expand-xs>.container-xl,.navbar-expand-xs>.container-xs{padding-right:0;padding-left:0}}@media(min-width:400px){.navbar-expand-xs{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand-xs .navbar-nav{flex-direction:row}.navbar-expand-xs .navbar-nav .dropdown-menu{position:absolute}.navbar-expand-xs .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand-xs>.container,.navbar-expand-xs>.container-fluid,.navbar-expand-xs>.container-lg,.navbar-expand-xs>.container-md,.navbar-expand-xs>.container-sm,.navbar-expand-xs>.container-xl,.navbar-expand-xs>.container-xs{flex-wrap:nowrap}.navbar-expand-xs .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand-xs .navbar-toggler{display:none}}@media(max-width:615.98px){.navbar-expand-sm>.container,.navbar-expand-sm>.container-fluid,.navbar-expand-sm>.container-lg,.navbar-expand-sm>.container-md,.navbar-expand-sm>.container-sm,.navbar-expand-sm>.container-xl,.navbar-expand-sm>.container-xs{padding-right:0;padding-left:0}}@media(min-width:616px){.navbar-expand-sm{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand-sm .navbar-nav{flex-direction:row}.navbar-expand-sm .navbar-nav .dropdown-menu{position:absolute}.navbar-expand-sm .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand-sm>.container,.navbar-expand-sm>.container-fluid,.navbar-expand-sm>.container-lg,.navbar-expand-sm>.container-md,.navbar-expand-sm>.container-sm,.navbar-expand-sm>.container-xl,.navbar-expand-sm>.container-xs{flex-wrap:nowrap}.navbar-expand-sm .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand-sm .navbar-toggler{display:none}}@media(max-width:767.98px){.navbar-expand-md>.container,.navbar-expand-md>.container-fluid,.navbar-expand-md>.container-lg,.navbar-expand-md>.container-md,.navbar-expand-md>.container-sm,.navbar-expand-md>.container-xl,.navbar-expand-md>.container-xs{padding-right:0;padding-left:0}}@media(min-width:768px){.navbar-expand-md{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand-md .navbar-nav{flex-direction:row}.navbar-expand-md .navbar-nav .dropdown-menu{position:absolute}.navbar-expand-md .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand-md>.container,.navbar-expand-md>.container-fluid,.navbar-expand-md>.container-lg,.navbar-expand-md>.container-md,.navbar-expand-md>.container-sm,.navbar-expand-md>.container-xl,.navbar-expand-md>.container-xs{flex-wrap:nowrap}.navbar-expand-md .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand-md .navbar-toggler{display:none}}@media(max-width:979.98px){.navbar-expand-lg>.container,.navbar-expand-lg>.container-fluid,.navbar-expand-lg>.container-lg,.navbar-expand-lg>.container-md,.navbar-expand-lg>.container-sm,.navbar-expand-lg>.container-xl,.navbar-expand-lg>.container-xs{padding-right:0;padding-left:0}}@media(min-width:980px){.navbar-expand-lg{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand-lg .navbar-nav{flex-direction:row}.navbar-expand-lg .navbar-nav .dropdown-menu{position:absolute}.navbar-expand-lg .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand-lg>.container,.navbar-expand-lg>.container-fluid,.navbar-expand-lg>.container-lg,.navbar-expand-lg>.container-md,.navbar-expand-lg>.container-sm,.navbar-expand-lg>.container-xl,.navbar-expand-lg>.container-xs{flex-wrap:nowrap}.navbar-expand-lg .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand-lg .navbar-toggler{display:none}}@media(max-width:1239.98px){.navbar-expand-xl>.container,.navbar-expand-xl>.container-fluid,.navbar-expand-xl>.container-lg,.navbar-expand-xl>.container-md,.navbar-expand-xl>.container-sm,.navbar-expand-xl>.container-xl,.navbar-expand-xl>.container-xs{padding-right:0;padding-left:0}}@media(min-width:1240px){.navbar-expand-xl{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand-xl .navbar-nav{flex-direction:row}.navbar-expand-xl .navbar-nav .dropdown-menu{position:absolute}.navbar-expand-xl .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand-xl>.container,.navbar-expand-xl>.container-fluid,.navbar-expand-xl>.container-lg,.navbar-expand-xl>.container-md,.navbar-expand-xl>.container-sm,.navbar-expand-xl>.container-xl,.navbar-expand-xl>.container-xs{flex-wrap:nowrap}.navbar-expand-xl .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand-xl .navbar-toggler{display:none}}.navbar-expand{flex-flow:row nowrap;justify-content:flex-start}.navbar-expand>.container,.navbar-expand>.container-fluid,.navbar-expand>.container-lg,.navbar-expand>.container-md,.navbar-expand>.container-sm,.navbar-expand>.container-xl,.navbar-expand>.container-xs{padding-right:0;padding-left:0}.navbar-expand .navbar-nav{flex-direction:row}.navbar-expand .navbar-nav .dropdown-menu{position:absolute}.navbar-expand .navbar-nav .nav-link{padding-right:.5rem;padding-left:.5rem}.navbar-expand>.container,.navbar-expand>.container-fluid,.navbar-expand>.container-lg,.navbar-expand>.container-md,.navbar-expand>.container-sm,.navbar-expand>.container-xl,.navbar-expand>.container-xs{flex-wrap:nowrap}.navbar-expand .navbar-collapse{display:flex!important;flex-basis:auto}.navbar-expand .navbar-toggler{display:none}.navbar-light .navbar-brand,.navbar-light .navbar-brand:focus,.navbar-light .navbar-brand:hover{color:rgba(33,37,41,.9)}.navbar-light .navbar-nav .nav-link{color:rgba(33,37,41,.5)}.navbar-light .navbar-nav .nav-link:focus,.navbar-light .navbar-nav .nav-link:hover{color:rgba(33,37,41,.7)}.navbar-light .navbar-nav .nav-link.disabled{color:rgba(33,37,41,.3)}.navbar-light .navbar-nav .active>.nav-link,.navbar-light .navbar-nav .nav-link.active,.navbar-light .navbar-nav .nav-link.show,.navbar-light .navbar-nav .show>.nav-link{color:rgba(33,37,41,.9)}.navbar-light .navbar-toggler{color:rgba(33,37,41,.5);border-color:rgba(33,37,41,.1)}.navbar-light .navbar-toggler-icon{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='30' height='30'%3E%3Cpath stroke='rgba(33, 37, 41, 0.5)' stroke-linecap='round' stroke-miterlimit='10' stroke-width='2' d='M4 7h22M4 15h22M4 23h22'/%3E%3C/svg%3E")}.navbar-light .navbar-text{color:rgba(33,37,41,.5)}.navbar-light .navbar-text a,.navbar-light .navbar-text a:focus,.navbar-light .navbar-text a:hover{color:rgba(33,37,41,.9)}.navbar-dark .navbar-brand,.navbar-dark .navbar-brand:focus,.navbar-dark .navbar-brand:hover{color:#fff}.navbar-dark .navbar-nav .nav-link{color:hsla(0,0%,100%,.5)}.navbar-dark .navbar-nav .nav-link:focus,.navbar-dark .navbar-nav .nav-link:hover{color:hsla(0,0%,100%,.75)}.navbar-dark .navbar-nav .nav-link.disabled{color:hsla(0,0%,100%,.25)}.navbar-dark .navbar-nav .active>.nav-link,.navbar-dark .navbar-nav .nav-link.active,.navbar-dark .navbar-nav .nav-link.show,.navbar-dark .navbar-nav .show>.nav-link{color:#fff}.navbar-dark .navbar-toggler{color:hsla(0,0%,100%,.5);border-color:hsla(0,0%,100%,.1)}.navbar-dark .navbar-toggler-icon{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' width='30' height='30'%3E%3Cpath stroke='rgba(255, 255, 255, 0.5)' stroke-linecap='round' stroke-miterlimit='10' stroke-width='2' d='M4 7h22M4 15h22M4 23h22'/%3E%3C/svg%3E")}.navbar-dark .navbar-text{color:hsla(0,0%,100%,.5)}.navbar-dark .navbar-text a,.navbar-dark .navbar-text a:focus,.navbar-dark .navbar-text a:hover{color:#fff}.card{position:relative;display:flex;flex-direction:column;min-width:0;word-wrap:break-word;background-color:#fff;background-clip:border-box;border:1px solid #d6dbdf;border-radius:8px}.card>hr{margin-right:0;margin-left:0}.card>.list-group:first-child .list-group-item:first-child{border-top-left-radius:8px;border-top-right-radius:8px}.card>.list-group:last-child .list-group-item:last-child{border-bottom-right-radius:8px;border-bottom-left-radius:8px}.card-body{flex:1 1 auto;min-height:1px;padding:24px}.card-title{margin-bottom:24px}.card-subtitle{margin-top:-12px}.card-subtitle,.card-text:last-child{margin-bottom:0}.card-link:hover{text-decoration:none}.card-link+.card-link{margin-left:24px}.card-header{padding:24px;margin-bottom:0;background-color:#f1f6f9;border-bottom:1px solid #d6dbdf}.card-header:first-child{border-radius:subtract(8px,1px) subtract(8px,1px) 0 0}.card-header+.list-group .list-group-item:first-child{border-top:0}.card-footer{padding:24px;background-color:#f1f6f9;border-top:1px solid #d6dbdf}.card-footer:last-child{border-radius:0 0 subtract(8px,1px) subtract(8px,1px)}.card-header-tabs{margin-bottom:-24px;border-bottom:0}.card-header-pills,.card-header-tabs{margin-right:-12px;margin-left:-12px}.card-img-overlay{position:absolute;top:0;right:0;bottom:0;left:0;padding:24px}.card-img,.card-img-bottom,.card-img-top{flex-shrink:0;width:100%}.card-img,.card-img-top{border-top-left-radius:subtract(8px,1px);border-top-right-radius:subtract(8px,1px)}.card-img,.card-img-bottom{border-bottom-right-radius:subtract(8px,1px);border-bottom-left-radius:subtract(8px,1px)}.card-deck .card{margin-bottom:20px}@media(min-width:616px){.card-deck{display:flex;flex-flow:row wrap;margin-right:-20px;margin-left:-20px}.card-deck .card{flex:1 0 0%;margin-right:20px;margin-bottom:0;margin-left:20px}}.card-group>.card{margin-bottom:20px}@media(min-width:616px){.card-group{display:flex;flex-flow:row wrap}.card-group>.card{flex:1 0 0%;margin-bottom:0}.card-group>.card+.card{margin-left:0;border-left:0}.card-group>.card:not(:last-child){border-top-right-radius:0;border-bottom-right-radius:0}.card-group>.card:not(:last-child) .card-header,.card-group>.card:not(:last-child) .card-img-top{border-top-right-radius:0}.card-group>.card:not(:last-child) .card-footer,.card-group>.card:not(:last-child) .card-img-bottom{border-bottom-right-radius:0}.card-group>.card:not(:first-child){border-top-left-radius:0;border-bottom-left-radius:0}.card-group>.card:not(:first-child) .card-header,.card-group>.card:not(:first-child) .card-img-top{border-top-left-radius:0}.card-group>.card:not(:first-child) .card-footer,.card-group>.card:not(:first-child) .card-img-bottom{border-bottom-left-radius:0}}.card-columns .card{margin-bottom:40px}@media(min-width:616px){.card-columns{-moz-column-count:3;column-count:3;-moz-column-gap:40px;column-gap:40px;orphans:1;widows:1}.card-columns .card{display:inline-block;width:100%}}.accordion>.card{overflow:hidden}.accordion>.card:not(:last-of-type){border-bottom:0;border-bottom-right-radius:0;border-bottom-left-radius:0}.accordion>.card:not(:first-of-type){border-top-left-radius:0;border-top-right-radius:0}.accordion>.card>.card-header{border-radius:0;margin-bottom:-1px}.breadcrumb{display:flex;flex-wrap:wrap;padding:.75rem 1rem;margin-bottom:1rem;list-style:none;background-color:#e9ecef;border-radius:8px}.breadcrumb-item+.breadcrumb-item{padding-left:.5rem}.breadcrumb-item+.breadcrumb-item:before{display:inline-block;padding-right:.5rem;color:#6c757d;content:"/"}.breadcrumb-item+.breadcrumb-item:hover:before{text-decoration:underline;text-decoration:none}.breadcrumb-item.active{color:#6c757d}.pagination{display:flex;padding-left:0;list-style:none;border-radius:8px}.page-link{position:relative;display:block;padding:.5rem .75rem;margin-left:-1px;line-height:1.25;color:#ff8c00;background-color:#fff;border:1px solid #dee2e6}.page-link:hover{z-index:2;color:#ff8c00;text-decoration:none;background-color:#e9ecef;border-color:#dee2e6}.page-link:focus{z-index:3;outline:0;box-shadow:0 0 0 .2rem rgba(255,204,0,.25)}.page-item:first-child .page-link{margin-left:0;border-top-left-radius:8px;border-bottom-left-radius:8px}.page-item:last-child .page-link{border-top-right-radius:8px;border-bottom-right-radius:8px}.page-item.active .page-link{z-index:3;color:#fff;background-color:#fc0;border-color:#fc0}.page-item.disabled .page-link{color:#6c757d;pointer-events:none;cursor:auto;background-color:#fff;border-color:#dee2e6}.pagination-lg .page-link{padding:.75rem 1.5rem;font-size:1.125rem;line-height:1.5}.pagination-lg .page-item:first-child .page-link{border-top-left-radius:8px;border-bottom-left-radius:8px}.pagination-lg .page-item:last-child .page-link{border-top-right-radius:8px;border-bottom-right-radius:8px}.pagination-sm .page-link{padding:.25rem .5rem;font-size:.875rem;line-height:1.5}.pagination-sm .page-item:first-child .page-link{border-top-left-radius:4px;border-bottom-left-radius:4px}.pagination-sm .page-item:last-child .page-link{border-top-right-radius:4px;border-bottom-right-radius:4px}.badge{display:inline-block;padding:.25em .4em;font-size:75%;font-weight:700;line-height:1;text-align:center;white-space:nowrap;vertical-align:baseline;border-radius:8px;transition:color .15s ease-in-out,background-color .15s ease-in-out,border-color .15s ease-in-out,box-shadow .15s ease-in-out}@media(prefers-reduced-motion:reduce){.badge{transition:none}}a.badge:focus,a.badge:hover{text-decoration:none}.badge:empty{display:none}.btn .badge{position:relative;top:-1px}.badge-pill{padding-right:.6em;padding-left:.6em;border-radius:10rem}.badge-primary{color:#495057;background-color:#fc0}a.badge-primary:focus,a.badge-primary:hover{color:#495057;background-color:#cca300}a.badge-primary.focus,a.badge-primary:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,204,0,.5)}.badge-secondary{color:#fff;background-color:#212529}a.badge-secondary:focus,a.badge-secondary:hover{color:#fff;background-color:#0a0c0d}a.badge-secondary.focus,a.badge-secondary:focus{outline:0;box-shadow:0 0 0 .2rem rgba(33,37,41,.5)}.badge-success{color:#fff;background-color:#28a745}a.badge-success:focus,a.badge-success:hover{color:#fff;background-color:#1e7e34}a.badge-success.focus,a.badge-success:focus{outline:0;box-shadow:0 0 0 .2rem rgba(40,167,69,.5)}.badge-info{color:#fff;background-color:#17a2b8}a.badge-info:focus,a.badge-info:hover{color:#fff;background-color:#117a8b}a.badge-info.focus,a.badge-info:focus{outline:0;box-shadow:0 0 0 .2rem rgba(23,162,184,.5)}.badge-warning{color:#495057;background-color:#ffc107}a.badge-warning:focus,a.badge-warning:hover{color:#495057;background-color:#d39e00}a.badge-warning.focus,a.badge-warning:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,193,7,.5)}.badge-danger{color:#fff;background-color:#dc3545}a.badge-danger:focus,a.badge-danger:hover{color:#fff;background-color:#bd2130}a.badge-danger.focus,a.badge-danger:focus{outline:0;box-shadow:0 0 0 .2rem rgba(220,53,69,.5)}.badge-light{color:#495057;background-color:#f1f6f9}a.badge-light:focus,a.badge-light:hover{color:#495057;background-color:#cddfea}a.badge-light.focus,a.badge-light:focus{outline:0;box-shadow:0 0 0 .2rem rgba(241,246,249,.5)}.badge-dark{color:#fff;background-color:#495057}a.badge-dark:focus,a.badge-dark:hover{color:#fff;background-color:#32373b}a.badge-dark.focus,a.badge-dark:focus{outline:0;box-shadow:0 0 0 .2rem rgba(73,80,87,.5)}.badge-primary-light{color:#495057;background-color:#fffaf0}a.badge-primary-light:focus,a.badge-primary-light:hover{color:#495057;background-color:#ffe9bd}a.badge-primary-light.focus,a.badge-primary-light:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,250,240,.5)}.badge-secondary-light{color:#495057;background-color:#fff}a.badge-secondary-light:focus,a.badge-secondary-light:hover{color:#495057;background-color:#e6e6e6}a.badge-secondary-light.focus,a.badge-secondary-light:focus{outline:0;box-shadow:0 0 0 .2rem hsla(0,0%,100%,.5)}.badge-tertiary{color:#fff;background-color:#257af4}a.badge-tertiary:focus,a.badge-tertiary:hover{color:#fff;background-color:#0b60db}a.badge-tertiary.focus,a.badge-tertiary:focus{outline:0;box-shadow:0 0 0 .2rem rgba(37,122,244,.5)}.badge-tertiary-light{color:#495057;background-color:#e3f1fe}a.badge-tertiary-light:focus,a.badge-tertiary-light:hover{color:#495057;background-color:#b2d8fc}a.badge-tertiary-light.focus,a.badge-tertiary-light:focus{outline:0;box-shadow:0 0 0 .2rem rgba(227,241,254,.5)}.badge-white{color:#495057;background-color:#fff}a.badge-white:focus,a.badge-white:hover{color:#495057;background-color:#e6e6e6}a.badge-white.focus,a.badge-white:focus{outline:0;box-shadow:0 0 0 .2rem hsla(0,0%,100%,.5)}.badge-black{color:#fff;background-color:#212529}a.badge-black:focus,a.badge-black:hover{color:#fff;background-color:#0a0c0d}a.badge-black.focus,a.badge-black:focus{outline:0;box-shadow:0 0 0 .2rem rgba(33,37,41,.5)}.badge-blue{color:#fff;background-color:#257af4}a.badge-blue:focus,a.badge-blue:hover{color:#fff;background-color:#0b60db}a.badge-blue.focus,a.badge-blue:focus{outline:0;box-shadow:0 0 0 .2rem rgba(37,122,244,.5)}.badge-light-blue{color:#495057;background-color:#e3f1fe}a.badge-light-blue:focus,a.badge-light-blue:hover{color:#495057;background-color:#b2d8fc}a.badge-light-blue.focus,a.badge-light-blue:focus{outline:0;box-shadow:0 0 0 .2rem rgba(227,241,254,.5)}.badge-yellow{color:#495057;background-color:#fc0}a.badge-yellow:focus,a.badge-yellow:hover{color:#495057;background-color:#cca300}a.badge-yellow.focus,a.badge-yellow:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,204,0,.5)}.badge-light-yellow{color:#495057;background-color:#fffaf0}a.badge-light-yellow:focus,a.badge-light-yellow:hover{color:#495057;background-color:#ffe9bd}a.badge-light-yellow.focus,a.badge-light-yellow:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,250,240,.5)}.badge-orange{color:#495057;background-color:#ff8c00}a.badge-orange:focus,a.badge-orange:hover{color:#495057;background-color:#cc7000}a.badge-orange.focus,a.badge-orange:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,140,0,.5)}.badge-light-orange{color:#495057;background-color:#ffe4b5}a.badge-light-orange:focus,a.badge-light-orange:hover{color:#495057;background-color:#ffd182}a.badge-light-orange.focus,a.badge-light-orange:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,228,181,.5)}.badge-red{color:#fff;background-color:#ff3939}a.badge-red:focus,a.badge-red:hover{color:#fff;background-color:#ff0606}a.badge-red.focus,a.badge-red:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,57,57,.5)}.badge-light-red{color:#495057;background-color:#ffe4e1}a.badge-light-red:focus,a.badge-light-red:hover{color:#495057;background-color:#ffb6ae}a.badge-light-red.focus,a.badge-light-red:focus{outline:0;box-shadow:0 0 0 .2rem rgba(255,228,225,.5)}.badge-medium{color:#495057;background-color:#d6dbdf}a.badge-medium:focus,a.badge-medium:hover{color:#495057;background-color:#b9c2c9}a.badge-medium.focus,a.badge-medium:focus{outline:0;box-shadow:0 0 0 .2rem rgba(214,219,223,.5)}.jumbotron{padding:2rem 1rem;margin-bottom:2rem;background-color:#e9ecef;border-radius:8px}@media(min-width:616px){.jumbotron{padding:4rem 2rem}}.jumbotron-fluid{padding-right:0;padding-left:0;border-radius:0}.alert{position:relative;padding:.75rem 1.25rem;margin-bottom:1rem;border:1px solid transparent;border-radius:8px}.alert-heading{color:inherit}.alert-link{font-weight:700}.alert-dismissible{padding-right:4rem}.alert-dismissible .close{position:absolute;top:0;right:0;padding:.75rem 1.25rem;color:inherit}.alert-primary{color:#947c14;background-color:#fff5cc;border-color:#fff1b8}.alert-primary hr{border-top-color:#ffec9f}.alert-primary .alert-link{color:#67560e}.alert-secondary{color:#212529;background-color:#d3d3d4;border-color:#c1c2c3}.alert-secondary hr{border-top-color:#b4b5b6}.alert-secondary .alert-link{color:#0a0c0d}.alert-success{color:#256938;background-color:#d4edda;border-color:#c3e6cb}.alert-success hr{border-top-color:#b1dfbb}.alert-success .alert-link{color:#184324}.alert-info{color:#1c6673;background-color:#d1ecf1;border-color:#bee5eb}.alert-info hr{border-top-color:#abdde5}.alert-info .alert-link{color:#12424a}.alert-warning{color:#947617;background-color:#fff3cd;border-color:#ffeeba}.alert-warning hr{border-top-color:#ffe8a1}.alert-warning .alert-link{color:#685310}.alert-danger{color:#822d38;background-color:#f8d7da;border-color:#f5c6cb}.alert-danger hr{border-top-color:#f1b0b7}.alert-danger .alert-link{color:#5c2028}.alert-light{color:#8d9295;background-color:#fcfdfe;border-color:#fbfcfd}.alert-light hr{border-top-color:#eaeff5}.alert-light .alert-link{color:#73797c}.alert-dark{color:#363b41;background-color:#dbdcdd;border-color:#ccced0}.alert-dark hr{border-top-color:#bfc1c4}.alert-dark .alert-link{color:#1f2225}.alert-primary-light{color:#949490;background-color:#fffefc;border-color:#fffefb}.alert-primary-light hr{border-top-color:#fff8e2}.alert-primary-light .alert-link{color:#7b7b76}.alert-secondary-light{color:#949698;background-color:#fff;border-color:#fff}.alert-secondary-light hr{border-top-color:#f2f2f2}.alert-secondary-light .alert-link{color:#7a7d7f}.alert-tertiary{color:#235193;background-color:#d3e4fd;border-color:#c2dafc}.alert-tertiary hr{border-top-color:#aacbfb}.alert-tertiary .alert-link{color:#193a6a}.alert-tertiary-light{color:#868f98;background-color:#f9fcff;border-color:#f7fbff}.alert-tertiary-light hr{border-top-color:#deeeff}.alert-tertiary-light .alert-link{color:#6c767f}.alert-white{color:#949698;background-color:#fff;border-color:#fff}.alert-white hr{border-top-color:#f2f2f2}.alert-white .alert-link{color:#7a7d7f}.alert-black{color:#212529;background-color:#d3d3d4;border-color:#c1c2c3}.alert-black hr{border-top-color:#b4b5b6}.alert-black .alert-link{color:#0a0c0d}.alert-blue{color:#235193;background-color:#d3e4fd;border-color:#c2dafc}.alert-blue hr{border-top-color:#aacbfb}.alert-blue .alert-link{color:#193a6a}.alert-light-blue{color:#868f98;background-color:#f9fcff;border-color:#f7fbff}.alert-light-blue hr{border-top-color:#deeeff}.alert-light-blue .alert-link{color:#6c767f}.alert-yellow{color:#947c14;background-color:#fff5cc;border-color:#fff1b8}.alert-yellow hr{border-top-color:#ffec9f}.alert-yellow .alert-link{color:#67560e}.alert-light-yellow{color:#949490;background-color:#fffefc;border-color:#fffefb}.alert-light-yellow hr{border-top-color:#fff8e2}.alert-light-yellow .alert-link{color:#7b7b76}.alert-orange{color:#945b14;background-color:#ffe8cc;border-color:#ffdfb8}.alert-orange hr{border-top-color:#ffd49f}.alert-orange .alert-link{color:#673f0e}.alert-light-orange{color:#948872;background-color:#fffaf0;border-color:#fff7ea}.alert-light-orange hr{border-top-color:#ffedd1}.alert-light-orange .alert-link{color:#786e5b}.alert-red{color:#942f31;background-color:#ffd7d7;border-color:#ffc8c8}.alert-red hr{border-top-color:#ffafaf}.alert-red .alert-link{color:#6d2324}.alert-light-red{color:#948889;background-color:#fffaf9;border-color:#fff7f7}.alert-light-red hr{border-top-color:#ffdede}.alert-light-red .alert-link{color:#7b6e6f}.alert-medium{color:#7f8488;background-color:#f7f8f9;border-color:#f4f5f6}.alert-medium hr{border-top-color:#e6e8eb}.alert-medium .alert-link{color:#666a6e}@-webkit-keyframes progress-bar-stripes{0%{background-position:1rem 0}to{background-position:0 0}}@keyframes progress-bar-stripes{0%{background-position:1rem 0}to{background-position:0 0}}.progress{height:1rem;font-size:.75rem;background-color:#e9ecef;border-radius:8px}.progress,.progress-bar{display:flex;overflow:hidden}.progress-bar{flex-direction:column;justify-content:center;color:#fff;text-align:center;white-space:nowrap;background-color:#fc0;transition:width .6s ease}@media(prefers-reduced-motion:reduce){.progress-bar{transition:none}}.progress-bar-striped{background-image:linear-gradient(45deg,hsla(0,0%,100%,.15) 25%,transparent 0,transparent 50%,hsla(0,0%,100%,.15) 0,hsla(0,0%,100%,.15) 75%,transparent 0,transparent);background-size:1rem 1rem}.progress-bar-animated{-webkit-animation:progress-bar-stripes 1s linear infinite;animation:progress-bar-stripes 1s linear infinite}@media(prefers-reduced-motion:reduce){.progress-bar-animated{-webkit-animation:none;animation:none}}.media{display:flex;align-items:flex-start}.media-body{flex:1}.list-group{display:flex;flex-direction:column;padding-left:0;margin-bottom:0}.list-group-item-action{width:100%;color:#6c757d;text-align:inherit}.list-group-item-action:focus,.list-group-item-action:hover{z-index:1;color:#6c757d;text-decoration:none;background-color:#f1f6f9}.list-group-item-action:active{color:#212529;background-color:#e9ecef}.list-group-item{position:relative;display:block;padding:.75rem 1.25rem;background-color:#fff;border:1px solid rgba(33,37,41,.125)}.list-group-item:first-child{border-top-left-radius:8px;border-top-right-radius:8px}.list-group-item:last-child{border-bottom-right-radius:8px;border-bottom-left-radius:8px}.list-group-item.disabled,.list-group-item:disabled{color:#6c757d;pointer-events:none;background-color:#fff}.list-group-item.active{z-index:2;color:#fff;background-color:#fc0;border-color:#fc0}.list-group-item+.list-group-item{border-top-width:0}.list-group-item+.list-group-item.active{margin-top:-1px;border-top-width:1px}.list-group-horizontal{flex-direction:row}.list-group-horizontal .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal .list-group-item.active{margin-top:0}.list-group-horizontal .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}@media(min-width:400px){.list-group-horizontal-xs{flex-direction:row}.list-group-horizontal-xs .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal-xs .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal-xs .list-group-item.active{margin-top:0}.list-group-horizontal-xs .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal-xs .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}}@media(min-width:616px){.list-group-horizontal-sm{flex-direction:row}.list-group-horizontal-sm .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal-sm .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal-sm .list-group-item.active{margin-top:0}.list-group-horizontal-sm .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal-sm .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}}@media(min-width:768px){.list-group-horizontal-md{flex-direction:row}.list-group-horizontal-md .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal-md .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal-md .list-group-item.active{margin-top:0}.list-group-horizontal-md .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal-md .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}}@media(min-width:980px){.list-group-horizontal-lg{flex-direction:row}.list-group-horizontal-lg .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal-lg .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal-lg .list-group-item.active{margin-top:0}.list-group-horizontal-lg .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal-lg .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}}@media(min-width:1240px){.list-group-horizontal-xl{flex-direction:row}.list-group-horizontal-xl .list-group-item:first-child{border-bottom-left-radius:8px;border-top-right-radius:0}.list-group-horizontal-xl .list-group-item:last-child{border-top-right-radius:8px;border-bottom-left-radius:0}.list-group-horizontal-xl .list-group-item.active{margin-top:0}.list-group-horizontal-xl .list-group-item+.list-group-item{border-top-width:1px;border-left-width:0}.list-group-horizontal-xl .list-group-item+.list-group-item.active{margin-left:-1px;border-left-width:1px}}.list-group-flush .list-group-item{border-right-width:0;border-left-width:0;border-radius:0}.list-group-flush .list-group-item:first-child{border-top-width:0}.list-group-flush:last-child .list-group-item:last-child{border-bottom-width:0}.list-group-item-primary{color:#947c14;background-color:#fff1b8}.list-group-item-primary.list-group-item-action:focus,.list-group-item-primary.list-group-item-action:hover{color:#947c14;background-color:#ffec9f}.list-group-item-primary.list-group-item-action.active{color:#fff;background-color:#947c14;border-color:#947c14}.list-group-item-secondary{color:#212529;background-color:#c1c2c3}.list-group-item-secondary.list-group-item-action:focus,.list-group-item-secondary.list-group-item-action:hover{color:#212529;background-color:#b4b5b6}.list-group-item-secondary.list-group-item-action.active{color:#fff;background-color:#212529;border-color:#212529}.list-group-item-success{color:#256938;background-color:#c3e6cb}.list-group-item-success.list-group-item-action:focus,.list-group-item-success.list-group-item-action:hover{color:#256938;background-color:#b1dfbb}.list-group-item-success.list-group-item-action.active{color:#fff;background-color:#256938;border-color:#256938}.list-group-item-info{color:#1c6673;background-color:#bee5eb}.list-group-item-info.list-group-item-action:focus,.list-group-item-info.list-group-item-action:hover{color:#1c6673;background-color:#abdde5}.list-group-item-info.list-group-item-action.active{color:#fff;background-color:#1c6673;border-color:#1c6673}.list-group-item-warning{color:#947617;background-color:#ffeeba}.list-group-item-warning.list-group-item-action:focus,.list-group-item-warning.list-group-item-action:hover{color:#947617;background-color:#ffe8a1}.list-group-item-warning.list-group-item-action.active{color:#fff;background-color:#947617;border-color:#947617}.list-group-item-danger{color:#822d38;background-color:#f5c6cb}.list-group-item-danger.list-group-item-action:focus,.list-group-item-danger.list-group-item-action:hover{color:#822d38;background-color:#f1b0b7}.list-group-item-danger.list-group-item-action.active{color:#fff;background-color:#822d38;border-color:#822d38}.list-group-item-light{color:#8d9295;background-color:#fbfcfd}.list-group-item-light.list-group-item-action:focus,.list-group-item-light.list-group-item-action:hover{color:#8d9295;background-color:#eaeff5}.list-group-item-light.list-group-item-action.active{color:#fff;background-color:#8d9295;border-color:#8d9295}.list-group-item-dark{color:#363b41;background-color:#ccced0}.list-group-item-dark.list-group-item-action:focus,.list-group-item-dark.list-group-item-action:hover{color:#363b41;background-color:#bfc1c4}.list-group-item-dark.list-group-item-action.active{color:#fff;background-color:#363b41;border-color:#363b41}.list-group-item-primary-light{color:#949490;background-color:#fffefb}.list-group-item-primary-light.list-group-item-action:focus,.list-group-item-primary-light.list-group-item-action:hover{color:#949490;background-color:#fff8e2}.list-group-item-primary-light.list-group-item-action.active{color:#fff;background-color:#949490;border-color:#949490}.list-group-item-secondary-light{color:#949698;background-color:#fff}.list-group-item-secondary-light.list-group-item-action:focus,.list-group-item-secondary-light.list-group-item-action:hover{color:#949698;background-color:#f2f2f2}.list-group-item-secondary-light.list-group-item-action.active{color:#fff;background-color:#949698;border-color:#949698}.list-group-item-tertiary{color:#235193;background-color:#c2dafc}.list-group-item-tertiary.list-group-item-action:focus,.list-group-item-tertiary.list-group-item-action:hover{color:#235193;background-color:#aacbfb}.list-group-item-tertiary.list-group-item-action.active{color:#fff;background-color:#235193;border-color:#235193}.list-group-item-tertiary-light{color:#868f98;background-color:#f7fbff}.list-group-item-tertiary-light.list-group-item-action:focus,.list-group-item-tertiary-light.list-group-item-action:hover{color:#868f98;background-color:#deeeff}.list-group-item-tertiary-light.list-group-item-action.active{color:#fff;background-color:#868f98;border-color:#868f98}.list-group-item-white{color:#949698;background-color:#fff}.list-group-item-white.list-group-item-action:focus,.list-group-item-white.list-group-item-action:hover{color:#949698;background-color:#f2f2f2}.list-group-item-white.list-group-item-action.active{color:#fff;background-color:#949698;border-color:#949698}.list-group-item-black{color:#212529;background-color:#c1c2c3}.list-group-item-black.list-group-item-action:focus,.list-group-item-black.list-group-item-action:hover{color:#212529;background-color:#b4b5b6}.list-group-item-black.list-group-item-action.active{color:#fff;background-color:#212529;border-color:#212529}.list-group-item-blue{color:#235193;background-color:#c2dafc}.list-group-item-blue.list-group-item-action:focus,.list-group-item-blue.list-group-item-action:hover{color:#235193;background-color:#aacbfb}.list-group-item-blue.list-group-item-action.active{color:#fff;background-color:#235193;border-color:#235193}.list-group-item-light-blue{color:#868f98;background-color:#f7fbff}.list-group-item-light-blue.list-group-item-action:focus,.list-group-item-light-blue.list-group-item-action:hover{color:#868f98;background-color:#deeeff}.list-group-item-light-blue.list-group-item-action.active{color:#fff;background-color:#868f98;border-color:#868f98}.list-group-item-yellow{color:#947c14;background-color:#fff1b8}.list-group-item-yellow.list-group-item-action:focus,.list-group-item-yellow.list-group-item-action:hover{color:#947c14;background-color:#ffec9f}.list-group-item-yellow.list-group-item-action.active{color:#fff;background-color:#947c14;border-color:#947c14}.list-group-item-light-yellow{color:#949490;background-color:#fffefb}.list-group-item-light-yellow.list-group-item-action:focus,.list-group-item-light-yellow.list-group-item-action:hover{color:#949490;background-color:#fff8e2}.list-group-item-light-yellow.list-group-item-action.active{color:#fff;background-color:#949490;border-color:#949490}.list-group-item-orange{color:#945b14;background-color:#ffdfb8}.list-group-item-orange.list-group-item-action:focus,.list-group-item-orange.list-group-item-action:hover{color:#945b14;background-color:#ffd49f}.list-group-item-orange.list-group-item-action.active{color:#fff;background-color:#945b14;border-color:#945b14}.list-group-item-light-orange{color:#948872;background-color:#fff7ea}.list-group-item-light-orange.list-group-item-action:focus,.list-group-item-light-orange.list-group-item-action:hover{color:#948872;background-color:#ffedd1}.list-group-item-light-orange.list-group-item-action.active{color:#fff;background-color:#948872;border-color:#948872}.list-group-item-red{color:#942f31;background-color:#ffc8c8}.list-group-item-red.list-group-item-action:focus,.list-group-item-red.list-group-item-action:hover{color:#942f31;background-color:#ffafaf}.list-group-item-red.list-group-item-action.active{color:#fff;background-color:#942f31;border-color:#942f31}.list-group-item-light-red{color:#948889;background-color:#fff7f7}.list-group-item-light-red.list-group-item-action:focus,.list-group-item-light-red.list-group-item-action:hover{color:#948889;background-color:#ffdede}.list-group-item-light-red.list-group-item-action.active{color:#fff;background-color:#948889;border-color:#948889}.list-group-item-medium{color:#7f8488;background-color:#f4f5f6}.list-group-item-medium.list-group-item-action:focus,.list-group-item-medium.list-group-item-action:hover{color:#7f8488;background-color:#e6e8eb}.list-group-item-medium.list-group-item-action.active{color:#fff;background-color:#7f8488;border-color:#7f8488}.close{float:right;font-size:1.5rem;font-weight:700;line-height:1;color:#212529;text-shadow:0 1px 0 #fff;opacity:.5}@media(max-width:1200px){.close{font-size:calc(1.275rem + .3vw)}}.close:hover{color:#212529;text-decoration:none}.close:not(:disabled):not(.disabled):focus,.close:not(:disabled):not(.disabled):hover{opacity:.75}button.close{padding:0;background-color:transparent;border:0;-webkit-appearance:none;-moz-appearance:none;appearance:none}a.close.disabled{pointer-events:none}.toast{max-width:350px;overflow:hidden;font-size:.875rem;background-color:hsla(0,0%,100%,.85);background-clip:padding-box;border:1px solid rgba(0,0,0,.1);box-shadow:0 .25rem .75rem rgba(33,37,41,.1);-webkit-backdrop-filter:blur(10px);backdrop-filter:blur(10px);opacity:0;border-radius:.25rem}.toast:not(:last-child){margin-bottom:.75rem}.toast.showing{opacity:1}.toast.show{display:block;opacity:1}.toast.hide{display:none}.toast-header{display:flex;align-items:center;padding:.25rem .75rem;color:#6c757d;background-color:hsla(0,0%,100%,.85);background-clip:padding-box;border-bottom:1px solid rgba(0,0,0,.05)}.toast-body{padding:.75rem}.modal-open{overflow:hidden}.modal-open .modal{overflow-x:hidden;overflow-y:auto}.modal{position:fixed;top:0;left:0;z-index:1050;display:none;width:100%;height:100%;overflow:hidden;outline:0}.modal-dialog{position:relative;width:auto;margin:.5rem;pointer-events:none}.modal.fade .modal-dialog{transition:transform .3s ease-out;transform:translateY(-50px)}@media(prefers-reduced-motion:reduce){.modal.fade .modal-dialog{transition:none}}.modal.show .modal-dialog{transform:none}.modal.modal-static .modal-dialog{transform:scale(1.02)}.modal-dialog-scrollable{display:flex;max-height:calc(100% - 1rem)}.modal-dialog-scrollable .modal-content{max-height:calc(100vh - 1rem);overflow:hidden}.modal-dialog-scrollable .modal-footer,.modal-dialog-scrollable .modal-header{flex-shrink:0}.modal-dialog-scrollable .modal-body{overflow-y:auto}.modal-dialog-centered{display:flex;align-items:center;min-height:calc(100% - 1rem)}.modal-dialog-centered:before{display:block;height:calc(100vh - 1rem);content:""}.modal-dialog-centered.modal-dialog-scrollable{flex-direction:column;justify-content:center;height:100%}.modal-dialog-centered.modal-dialog-scrollable .modal-content{max-height:none}.modal-dialog-centered.modal-dialog-scrollable:before{content:none}.modal-content{position:relative;display:flex;flex-direction:column;width:100%;pointer-events:auto;background-color:#fff;background-clip:padding-box;border:1px solid rgba(33,37,41,.2);border-radius:8px;outline:0}.modal-backdrop{position:fixed;top:0;left:0;z-index:1040;width:100vw;height:100vh;background-color:#212529}.modal-backdrop.fade{opacity:0}.modal-backdrop.show{opacity:.5}.modal-header{display:flex;align-items:flex-start;justify-content:space-between;padding:1rem;border-bottom:1px solid #d6dbdf;border-top-left-radius:7px;border-top-right-radius:7px}.modal-header .close{padding:1rem;margin:-1rem -1rem -1rem auto}.modal-title{margin-bottom:0;line-height:1.5}.modal-body{position:relative;flex:1 1 auto;padding:1rem}.modal-footer{display:flex;flex-wrap:wrap;align-items:center;justify-content:flex-end;padding:.75rem;border-top:1px solid #d6dbdf;border-bottom-right-radius:7px;border-bottom-left-radius:7px}.modal-footer>*{margin:.25rem}.modal-scrollbar-measure{position:absolute;top:-9999px;width:50px;height:50px;overflow:scroll}@media(min-width:616px){.modal-dialog{max-width:500px;margin:1.75rem auto}.modal-dialog-scrollable{max-height:calc(100% - 3.5rem)}.modal-dialog-scrollable .modal-content{max-height:calc(100vh - 3.5rem)}.modal-dialog-centered{min-height:calc(100% - 3.5rem)}.modal-dialog-centered:before{height:calc(100vh - 3.5rem)}.modal-sm{max-width:300px}}@media(min-width:980px){.modal-lg,.modal-xl{max-width:800px}}@media(min-width:1240px){.modal-xl{max-width:1140px}}.tooltip{position:absolute;z-index:1070;display:block;margin:0;font-family:Noto Sans,sans-serif;font-style:normal;font-weight:400;line-height:1.5;text-align:left;text-align:start;text-decoration:none;text-shadow:none;text-transform:none;letter-spacing:normal;word-break:normal;word-spacing:normal;white-space:normal;line-break:auto;font-size:.875rem;word-wrap:break-word;opacity:0}.tooltip.show{opacity:.9}.tooltip .arrow{position:absolute;display:block;width:.8rem;height:.4rem}.tooltip .arrow:before{position:absolute;content:"";border-color:transparent;border-style:solid}.bs-tooltip-auto[x-placement^=top],.bs-tooltip-top{padding:.4rem 0}.bs-tooltip-auto[x-placement^=top] .arrow,.bs-tooltip-top .arrow{bottom:0}.bs-tooltip-auto[x-placement^=top] .arrow:before,.bs-tooltip-top .arrow:before{top:0;border-width:.4rem .4rem 0;border-top-color:#212529}.bs-tooltip-auto[x-placement^=right],.bs-tooltip-right{padding:0 .4rem}.bs-tooltip-auto[x-placement^=right] .arrow,.bs-tooltip-right .arrow{left:0;width:.4rem;height:.8rem}.bs-tooltip-auto[x-placement^=right] .arrow:before,.bs-tooltip-right .arrow:before{right:0;border-width:.4rem .4rem .4rem 0;border-right-color:#212529}.bs-tooltip-auto[x-placement^=bottom],.bs-tooltip-bottom{padding:.4rem 0}.bs-tooltip-auto[x-placement^=bottom] .arrow,.bs-tooltip-bottom .arrow{top:0}.bs-tooltip-auto[x-placement^=bottom] .arrow:before,.bs-tooltip-bottom .arrow:before{bottom:0;border-width:0 .4rem .4rem;border-bottom-color:#212529}.bs-tooltip-auto[x-placement^=left],.bs-tooltip-left{padding:0 .4rem}.bs-tooltip-auto[x-placement^=left] .arrow,.bs-tooltip-left .arrow{right:0;width:.4rem;height:.8rem}.bs-tooltip-auto[x-placement^=left] .arrow:before,.bs-tooltip-left .arrow:before{left:0;border-width:.4rem 0 .4rem .4rem;border-left-color:#212529}.tooltip-inner{max-width:200px;padding:.25rem .5rem;color:#fff;text-align:center;background-color:#212529;border-radius:8px}.popover{top:0;left:0;z-index:1060;max-width:276px;font-family:Noto Sans,sans-serif;font-style:normal;font-weight:400;line-height:1.5;text-align:left;text-align:start;text-decoration:none;text-shadow:none;text-transform:none;letter-spacing:normal;word-break:normal;word-spacing:normal;white-space:normal;line-break:auto;font-size:.875rem;word-wrap:break-word;background-color:#fff;background-clip:padding-box;border:1px solid rgba(33,37,41,.2);border-radius:8px}.popover,.popover .arrow{position:absolute;display:block}.popover .arrow{width:1rem;height:.5rem;margin:0 8px}.popover .arrow:after,.popover .arrow:before{position:absolute;display:block;content:"";border-color:transparent;border-style:solid}.bs-popover-auto[x-placement^=top],.bs-popover-top{margin-bottom:.5rem}.bs-popover-auto[x-placement^=top]>.arrow,.bs-popover-top>.arrow{bottom:calc(-.5rem - 1px)}.bs-popover-auto[x-placement^=top]>.arrow:before,.bs-popover-top>.arrow:before{bottom:0;border-width:.5rem .5rem 0;border-top-color:rgba(33,37,41,.25)}.bs-popover-auto[x-placement^=top]>.arrow:after,.bs-popover-top>.arrow:after{bottom:1px;border-width:.5rem .5rem 0;border-top-color:#fff}.bs-popover-auto[x-placement^=right],.bs-popover-right{margin-left:.5rem}.bs-popover-auto[x-placement^=right]>.arrow,.bs-popover-right>.arrow{left:calc(-.5rem - 1px);width:.5rem;height:1rem;margin:8px 0}.bs-popover-auto[x-placement^=right]>.arrow:before,.bs-popover-right>.arrow:before{left:0;border-width:.5rem .5rem .5rem 0;border-right-color:rgba(33,37,41,.25)}.bs-popover-auto[x-placement^=right]>.arrow:after,.bs-popover-right>.arrow:after{left:1px;border-width:.5rem .5rem .5rem 0;border-right-color:#fff}.bs-popover-auto[x-placement^=bottom],.bs-popover-bottom{margin-top:.5rem}.bs-popover-auto[x-placement^=bottom]>.arrow,.bs-popover-bottom>.arrow{top:calc(-.5rem - 1px)}.bs-popover-auto[x-placement^=bottom]>.arrow:before,.bs-popover-bottom>.arrow:before{top:0;border-width:0 .5rem .5rem;border-bottom-color:rgba(33,37,41,.25)}.bs-popover-auto[x-placement^=bottom]>.arrow:after,.bs-popover-bottom>.arrow:after{top:1px;border-width:0 .5rem .5rem;border-bottom-color:#fff}.bs-popover-auto[x-placement^=bottom] .popover-header:before,.bs-popover-bottom .popover-header:before{position:absolute;top:0;left:50%;display:block;width:1rem;margin-left:-.5rem;content:"";border-bottom:1px solid #f7f7f7}.bs-popover-auto[x-placement^=left],.bs-popover-left{margin-right:.5rem}.bs-popover-auto[x-placement^=left]>.arrow,.bs-popover-left>.arrow{right:calc(-.5rem - 1px);width:.5rem;height:1rem;margin:8px 0}.bs-popover-auto[x-placement^=left]>.arrow:before,.bs-popover-left>.arrow:before{right:0;border-width:.5rem 0 .5rem .5rem;border-left-color:rgba(33,37,41,.25)}.bs-popover-auto[x-placement^=left]>.arrow:after,.bs-popover-left>.arrow:after{right:1px;border-width:.5rem 0 .5rem .5rem;border-left-color:#fff}.popover-header{padding:.5rem .75rem;margin-bottom:0;font-size:1rem;background-color:#f7f7f7;border-bottom:1px solid #ebebeb;border-top-left-radius:7px;border-top-right-radius:7px}.popover-header:empty{display:none}.popover-body{padding:.5rem .75rem;color:#212529}.carousel{position:relative}.carousel.pointer-event{touch-action:pan-y}.carousel-inner{position:relative;width:100%;overflow:hidden}.carousel-inner:after{display:block;clear:both;content:""}.carousel-item{position:relative;display:none;float:left;width:100%;margin-right:-100%;-webkit-backface-visibility:hidden;backface-visibility:hidden;transition:transform .6s ease-in-out}@media(prefers-reduced-motion:reduce){.carousel-item{transition:none}}.carousel-item-next,.carousel-item-prev,.carousel-item.active{display:block}.active.carousel-item-right,.carousel-item-next:not(.carousel-item-left){transform:translateX(100%)}.active.carousel-item-left,.carousel-item-prev:not(.carousel-item-right){transform:translateX(-100%)}.carousel-fade .carousel-item{opacity:0;transition-property:opacity;transform:none}.carousel-fade .carousel-item-next.carousel-item-left,.carousel-fade .carousel-item-prev.carousel-item-right,.carousel-fade .carousel-item.active{z-index:1;opacity:1}.carousel-fade .active.carousel-item-left,.carousel-fade .active.carousel-item-right{z-index:0;opacity:0;transition:opacity 0s .6s}@media(prefers-reduced-motion:reduce){.carousel-fade .active.carousel-item-left,.carousel-fade .active.carousel-item-right{transition:none}}.carousel-control-next,.carousel-control-prev{position:absolute;top:0;bottom:0;z-index:1;display:flex;align-items:center;justify-content:center;width:15%;color:#fff;text-align:center;opacity:.5;transition:opacity .15s ease}@media(prefers-reduced-motion:reduce){.carousel-control-next,.carousel-control-prev{transition:none}}.carousel-control-next:focus,.carousel-control-next:hover,.carousel-control-prev:focus,.carousel-control-prev:hover{color:#fff;text-decoration:none;outline:0;opacity:.9}.carousel-control-prev{left:0}.carousel-control-next{right:0}.carousel-control-next-icon,.carousel-control-prev-icon{display:inline-block;width:20px;height:20px;background:no-repeat 50%/100% 100%}.carousel-control-prev-icon{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' fill='%23fff' width='8' height='8'%3E%3Cpath d='M5.25 0l-4 4 4 4 1.5-1.5L4.25 4l2.5-2.5L5.25 0z'/%3E%3C/svg%3E")}.carousel-control-next-icon{background-image:url("data:image/svg+xml;charset=utf-8,%3Csvg xmlns='http://www.w3.org/2000/svg' fill='%23fff' width='8' height='8'%3E%3Cpath d='M2.75 0l-1.5 1.5L3.75 4l-2.5 2.5L2.75 8l4-4-4-4z'/%3E%3C/svg%3E")}.carousel-indicators{position:absolute;right:0;bottom:0;left:0;z-index:15;display:flex;justify-content:center;padding-left:0;margin-right:15%;margin-left:15%;list-style:none}.carousel-indicators li{box-sizing:content-box;flex:0 1 auto;width:30px;height:3px;margin-right:3px;margin-left:3px;text-indent:-999px;cursor:pointer;background-color:#fff;background-clip:padding-box;border-top:10px solid transparent;border-bottom:10px solid transparent;opacity:.5;transition:opacity .6s ease}@media(prefers-reduced-motion:reduce){.carousel-indicators li{transition:none}}.carousel-indicators .active{opacity:1}.carousel-caption{position:absolute;right:15%;bottom:20px;left:15%;z-index:10;padding-top:20px;padding-bottom:20px;color:#fff;text-align:center}@-webkit-keyframes spinner-border{to{transform:rotate(1turn)}}@keyframes spinner-border{to{transform:rotate(1turn)}}.spinner-border{display:inline-block;width:2rem;height:2rem;vertical-align:text-bottom;border:.25em solid;border-right:.25em solid transparent;border-radius:50%;-webkit-animation:spinner-border .75s linear infinite;animation:spinner-border .75s linear infinite}.spinner-border-sm{width:1rem;height:1rem;border-width:.2em}@-webkit-keyframes spinner-grow{0%{transform:scale(0)}50%{opacity:1}}@keyframes spinner-grow{0%{transform:scale(0)}50%{opacity:1}}.spinner-grow{display:inline-block;width:2rem;height:2rem;vertical-align:text-bottom;background-color:currentColor;border-radius:50%;opacity:0;-webkit-animation:spinner-grow .75s linear infinite;animation:spinner-grow .75s linear infinite}.spinner-grow-sm{width:1rem;height:1rem}.align-baseline{vertical-align:baseline!important}.align-top{vertical-align:top!important}.align-middle{vertical-align:middle!important}.align-bottom{vertical-align:bottom!important}.align-text-bottom{vertical-align:text-bottom!important}.align-text-top{vertical-align:text-top!important}.bg-primary{background-color:#fc0!important}a.bg-primary:focus,a.bg-primary:hover,button.bg-primary:focus,button.bg-primary:hover{background-color:#cca300!important}.bg-secondary{background-color:#212529!important}a.bg-secondary:focus,a.bg-secondary:hover,button.bg-secondary:focus,button.bg-secondary:hover{background-color:#0a0c0d!important}.bg-success{background-color:#28a745!important}a.bg-success:focus,a.bg-success:hover,button.bg-success:focus,button.bg-success:hover{background-color:#1e7e34!important}.bg-info{background-color:#17a2b8!important}a.bg-info:focus,a.bg-info:hover,button.bg-info:focus,button.bg-info:hover{background-color:#117a8b!important}.bg-warning{background-color:#ffc107!important}a.bg-warning:focus,a.bg-warning:hover,button.bg-warning:focus,button.bg-warning:hover{background-color:#d39e00!important}.bg-danger{background-color:#dc3545!important}a.bg-danger:focus,a.bg-danger:hover,button.bg-danger:focus,button.bg-danger:hover{background-color:#bd2130!important}.bg-light{background-color:#f1f6f9!important}a.bg-light:focus,a.bg-light:hover,button.bg-light:focus,button.bg-light:hover{background-color:#cddfea!important}.bg-dark{background-color:#495057!important}a.bg-dark:focus,a.bg-dark:hover,button.bg-dark:focus,button.bg-dark:hover{background-color:#32373b!important}.bg-primary-light{background-color:#fffaf0!important}a.bg-primary-light:focus,a.bg-primary-light:hover,button.bg-primary-light:focus,button.bg-primary-light:hover{background-color:#ffe9bd!important}.bg-secondary-light{background-color:#fff!important}a.bg-secondary-light:focus,a.bg-secondary-light:hover,button.bg-secondary-light:focus,button.bg-secondary-light:hover{background-color:#e6e6e6!important}.bg-tertiary{background-color:#257af4!important}a.bg-tertiary:focus,a.bg-tertiary:hover,button.bg-tertiary:focus,button.bg-tertiary:hover{background-color:#0b60db!important}.bg-tertiary-light{background-color:#e3f1fe!important}a.bg-tertiary-light:focus,a.bg-tertiary-light:hover,button.bg-tertiary-light:focus,button.bg-tertiary-light:hover{background-color:#b2d8fc!important}a.bg-white:focus,a.bg-white:hover,button.bg-white:focus,button.bg-white:hover{background-color:#e6e6e6!important}.bg-black{background-color:#212529!important}a.bg-black:focus,a.bg-black:hover,button.bg-black:focus,button.bg-black:hover{background-color:#0a0c0d!important}.bg-blue{background-color:#257af4!important}a.bg-blue:focus,a.bg-blue:hover,button.bg-blue:focus,button.bg-blue:hover{background-color:#0b60db!important}.bg-light-blue{background-color:#e3f1fe!important}a.bg-light-blue:focus,a.bg-light-blue:hover,button.bg-light-blue:focus,button.bg-light-blue:hover{background-color:#b2d8fc!important}.bg-yellow{background-color:#fc0!important}a.bg-yellow:focus,a.bg-yellow:hover,button.bg-yellow:focus,button.bg-yellow:hover{background-color:#cca300!important}.bg-light-yellow{background-color:#fffaf0!important}a.bg-light-yellow:focus,a.bg-light-yellow:hover,button.bg-light-yellow:focus,button.bg-light-yellow:hover{background-color:#ffe9bd!important}.bg-orange{background-color:#ff8c00!important}a.bg-orange:focus,a.bg-orange:hover,button.bg-orange:focus,button.bg-orange:hover{background-color:#cc7000!important}.bg-light-orange{background-color:#ffe4b5!important}a.bg-light-orange:focus,a.bg-light-orange:hover,button.bg-light-orange:focus,button.bg-light-orange:hover{background-color:#ffd182!important}.bg-red{background-color:#ff3939!important}a.bg-red:focus,a.bg-red:hover,button.bg-red:focus,button.bg-red:hover{background-color:#ff0606!important}.bg-light-red{background-color:#ffe4e1!important}a.bg-light-red:focus,a.bg-light-red:hover,button.bg-light-red:focus,button.bg-light-red:hover{background-color:#ffb6ae!important}.bg-medium{background-color:#d6dbdf!important}a.bg-medium:focus,a.bg-medium:hover,button.bg-medium:focus,button.bg-medium:hover{background-color:#b9c2c9!important}.bg-white{background-color:#fff!important}.bg-transparent{background-color:transparent!important}.border{border:1px solid #d6dbdf!important}.border-top{border-top:1px solid #d6dbdf!important}.border-right{border-right:1px solid #d6dbdf!important}.border-bottom{border-bottom:1px solid #d6dbdf!important}.border-left{border-left:1px solid #d6dbdf!important}.border-0{border:0!important}.border-top-0{border-top:0!important}.border-right-0{border-right:0!important}.border-bottom-0{border-bottom:0!important}.border-left-0{border-left:0!important}.border-primary{border-color:#fc0!important}.border-secondary{border-color:#212529!important}.border-success{border-color:#28a745!important}.border-info{border-color:#17a2b8!important}.border-warning{border-color:#ffc107!important}.border-danger{border-color:#dc3545!important}.border-light{border-color:#f1f6f9!important}.border-dark{border-color:#495057!important}.border-primary-light{border-color:#fffaf0!important}.border-secondary-light{border-color:#fff!important}.border-tertiary{border-color:#257af4!important}.border-tertiary-light{border-color:#e3f1fe!important}.border-black{border-color:#212529!important}.border-blue{border-color:#257af4!important}.border-light-blue{border-color:#e3f1fe!important}.border-yellow{border-color:#fc0!important}.border-light-yellow{border-color:#fffaf0!important}.border-orange{border-color:#ff8c00!important}.border-light-orange{border-color:#ffe4b5!important}.border-red{border-color:#ff3939!important}.border-light-red{border-color:#ffe4e1!important}.border-medium{border-color:#d6dbdf!important}.border-white{border-color:#fff!important}.rounded-sm{border-radius:4px!important}.rounded{border-radius:8px!important}.rounded-top{border-top-left-radius:8px!important}.rounded-right,.rounded-top{border-top-right-radius:8px!important}.rounded-bottom,.rounded-right{border-bottom-right-radius:8px!important}.rounded-bottom,.rounded-left{border-bottom-left-radius:8px!important}.rounded-left{border-top-left-radius:8px!important}.rounded-lg{border-radius:8px!important}.rounded-circle{border-radius:50%!important}.rounded-pill{border-radius:50rem!important}.rounded-0{border-radius:0!important}.clearfix:after{display:block;clear:both;content:""}.d-none{display:none!important}.d-inline{display:inline!important}.d-inline-block{display:inline-block!important}.d-block{display:block!important}.d-table{display:table!important}.d-table-row{display:table-row!important}.d-table-cell{display:table-cell!important}.d-flex{display:flex!important}.d-inline-flex{display:inline-flex!important}@media(min-width:400px){.d-xs-none{display:none!important}.d-xs-inline{display:inline!important}.d-xs-inline-block{display:inline-block!important}.d-xs-block{display:block!important}.d-xs-table{display:table!important}.d-xs-table-row{display:table-row!important}.d-xs-table-cell{display:table-cell!important}.d-xs-flex{display:flex!important}.d-xs-inline-flex{display:inline-flex!important}}@media(min-width:616px){.d-sm-none{display:none!important}.d-sm-inline{display:inline!important}.d-sm-inline-block{display:inline-block!important}.d-sm-block{display:block!important}.d-sm-table{display:table!important}.d-sm-table-row{display:table-row!important}.d-sm-table-cell{display:table-cell!important}.d-sm-flex{display:flex!important}.d-sm-inline-flex{display:inline-flex!important}}@media(min-width:768px){.d-md-none{display:none!important}.d-md-inline{display:inline!important}.d-md-inline-block{display:inline-block!important}.d-md-block{display:block!important}.d-md-table{display:table!important}.d-md-table-row{display:table-row!important}.d-md-table-cell{display:table-cell!important}.d-md-flex{display:flex!important}.d-md-inline-flex{display:inline-flex!important}}@media(min-width:980px){.d-lg-none{display:none!important}.d-lg-inline{display:inline!important}.d-lg-inline-block{display:inline-block!important}.d-lg-block{display:block!important}.d-lg-table{display:table!important}.d-lg-table-row{display:table-row!important}.d-lg-table-cell{display:table-cell!important}.d-lg-flex{display:flex!important}.d-lg-inline-flex{display:inline-flex!important}}@media(min-width:1240px){.d-xl-none{display:none!important}.d-xl-inline{display:inline!important}.d-xl-inline-block{display:inline-block!important}.d-xl-block{display:block!important}.d-xl-table{display:table!important}.d-xl-table-row{display:table-row!important}.d-xl-table-cell{display:table-cell!important}.d-xl-flex{display:flex!important}.d-xl-inline-flex{display:inline-flex!important}}@media print{.d-print-none{display:none!important}.d-print-inline{display:inline!important}.d-print-inline-block{display:inline-block!important}.d-print-block{display:block!important}.d-print-table{display:table!important}.d-print-table-row{display:table-row!important}.d-print-table-cell{display:table-cell!important}.d-print-flex{display:flex!important}.d-print-inline-flex{display:inline-flex!important}}.embed-responsive{position:relative;display:block;width:100%;padding:0;overflow:hidden}.embed-responsive:before{display:block;content:""}.embed-responsive .embed-responsive-item,.embed-responsive embed,.embed-responsive iframe,.embed-responsive object,.embed-responsive video{position:absolute;top:0;bottom:0;left:0;width:100%;height:100%;border:0}.embed-responsive-21by9:before{padding-top:42.8571428571%}.embed-responsive-16by9:before{padding-top:56.25%}.embed-responsive-4by3:before{padding-top:75%}.embed-responsive-1by1:before{padding-top:100%}.flex-row{flex-direction:row!important}.flex-column{flex-direction:column!important}.flex-row-reverse{flex-direction:row-reverse!important}.flex-column-reverse{flex-direction:column-reverse!important}.flex-wrap{flex-wrap:wrap!important}.flex-nowrap{flex-wrap:nowrap!important}.flex-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-fill{flex:1 1 auto!important}.flex-grow-0{flex-grow:0!important}.flex-grow-1{flex-grow:1!important}.flex-shrink-0{flex-shrink:0!important}.flex-shrink-1{flex-shrink:1!important}.justify-content-start{justify-content:flex-start!important}.justify-content-end{justify-content:flex-end!important}.justify-content-center{justify-content:center!important}.justify-content-between{justify-content:space-between!important}.justify-content-around{justify-content:space-around!important}.align-items-start{align-items:flex-start!important}.align-items-end{align-items:flex-end!important}.align-items-center{align-items:center!important}.align-items-baseline{align-items:baseline!important}.align-items-stretch{align-items:stretch!important}.align-content-start{align-content:flex-start!important}.align-content-end{align-content:flex-end!important}.align-content-center{align-content:center!important}.align-content-between{align-content:space-between!important}.align-content-around{align-content:space-around!important}.align-content-stretch{align-content:stretch!important}.align-self-auto{align-self:auto!important}.align-self-start{align-self:flex-start!important}.align-self-end{align-self:flex-end!important}.align-self-center{align-self:center!important}.align-self-baseline{align-self:baseline!important}.align-self-stretch{align-self:stretch!important}@media(min-width:400px){.flex-xs-row{flex-direction:row!important}.flex-xs-column{flex-direction:column!important}.flex-xs-row-reverse{flex-direction:row-reverse!important}.flex-xs-column-reverse{flex-direction:column-reverse!important}.flex-xs-wrap{flex-wrap:wrap!important}.flex-xs-nowrap{flex-wrap:nowrap!important}.flex-xs-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-xs-fill{flex:1 1 auto!important}.flex-xs-grow-0{flex-grow:0!important}.flex-xs-grow-1{flex-grow:1!important}.flex-xs-shrink-0{flex-shrink:0!important}.flex-xs-shrink-1{flex-shrink:1!important}.justify-content-xs-start{justify-content:flex-start!important}.justify-content-xs-end{justify-content:flex-end!important}.justify-content-xs-center{justify-content:center!important}.justify-content-xs-between{justify-content:space-between!important}.justify-content-xs-around{justify-content:space-around!important}.align-items-xs-start{align-items:flex-start!important}.align-items-xs-end{align-items:flex-end!important}.align-items-xs-center{align-items:center!important}.align-items-xs-baseline{align-items:baseline!important}.align-items-xs-stretch{align-items:stretch!important}.align-content-xs-start{align-content:flex-start!important}.align-content-xs-end{align-content:flex-end!important}.align-content-xs-center{align-content:center!important}.align-content-xs-between{align-content:space-between!important}.align-content-xs-around{align-content:space-around!important}.align-content-xs-stretch{align-content:stretch!important}.align-self-xs-auto{align-self:auto!important}.align-self-xs-start{align-self:flex-start!important}.align-self-xs-end{align-self:flex-end!important}.align-self-xs-center{align-self:center!important}.align-self-xs-baseline{align-self:baseline!important}.align-self-xs-stretch{align-self:stretch!important}}@media(min-width:616px){.flex-sm-row{flex-direction:row!important}.flex-sm-column{flex-direction:column!important}.flex-sm-row-reverse{flex-direction:row-reverse!important}.flex-sm-column-reverse{flex-direction:column-reverse!important}.flex-sm-wrap{flex-wrap:wrap!important}.flex-sm-nowrap{flex-wrap:nowrap!important}.flex-sm-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-sm-fill{flex:1 1 auto!important}.flex-sm-grow-0{flex-grow:0!important}.flex-sm-grow-1{flex-grow:1!important}.flex-sm-shrink-0{flex-shrink:0!important}.flex-sm-shrink-1{flex-shrink:1!important}.justify-content-sm-start{justify-content:flex-start!important}.justify-content-sm-end{justify-content:flex-end!important}.justify-content-sm-center{justify-content:center!important}.justify-content-sm-between{justify-content:space-between!important}.justify-content-sm-around{justify-content:space-around!important}.align-items-sm-start{align-items:flex-start!important}.align-items-sm-end{align-items:flex-end!important}.align-items-sm-center{align-items:center!important}.align-items-sm-baseline{align-items:baseline!important}.align-items-sm-stretch{align-items:stretch!important}.align-content-sm-start{align-content:flex-start!important}.align-content-sm-end{align-content:flex-end!important}.align-content-sm-center{align-content:center!important}.align-content-sm-between{align-content:space-between!important}.align-content-sm-around{align-content:space-around!important}.align-content-sm-stretch{align-content:stretch!important}.align-self-sm-auto{align-self:auto!important}.align-self-sm-start{align-self:flex-start!important}.align-self-sm-end{align-self:flex-end!important}.align-self-sm-center{align-self:center!important}.align-self-sm-baseline{align-self:baseline!important}.align-self-sm-stretch{align-self:stretch!important}}@media(min-width:768px){.flex-md-row{flex-direction:row!important}.flex-md-column{flex-direction:column!important}.flex-md-row-reverse{flex-direction:row-reverse!important}.flex-md-column-reverse{flex-direction:column-reverse!important}.flex-md-wrap{flex-wrap:wrap!important}.flex-md-nowrap{flex-wrap:nowrap!important}.flex-md-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-md-fill{flex:1 1 auto!important}.flex-md-grow-0{flex-grow:0!important}.flex-md-grow-1{flex-grow:1!important}.flex-md-shrink-0{flex-shrink:0!important}.flex-md-shrink-1{flex-shrink:1!important}.justify-content-md-start{justify-content:flex-start!important}.justify-content-md-end{justify-content:flex-end!important}.justify-content-md-center{justify-content:center!important}.justify-content-md-between{justify-content:space-between!important}.justify-content-md-around{justify-content:space-around!important}.align-items-md-start{align-items:flex-start!important}.align-items-md-end{align-items:flex-end!important}.align-items-md-center{align-items:center!important}.align-items-md-baseline{align-items:baseline!important}.align-items-md-stretch{align-items:stretch!important}.align-content-md-start{align-content:flex-start!important}.align-content-md-end{align-content:flex-end!important}.align-content-md-center{align-content:center!important}.align-content-md-between{align-content:space-between!important}.align-content-md-around{align-content:space-around!important}.align-content-md-stretch{align-content:stretch!important}.align-self-md-auto{align-self:auto!important}.align-self-md-start{align-self:flex-start!important}.align-self-md-end{align-self:flex-end!important}.align-self-md-center{align-self:center!important}.align-self-md-baseline{align-self:baseline!important}.align-self-md-stretch{align-self:stretch!important}}@media(min-width:980px){.flex-lg-row{flex-direction:row!important}.flex-lg-column{flex-direction:column!important}.flex-lg-row-reverse{flex-direction:row-reverse!important}.flex-lg-column-reverse{flex-direction:column-reverse!important}.flex-lg-wrap{flex-wrap:wrap!important}.flex-lg-nowrap{flex-wrap:nowrap!important}.flex-lg-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-lg-fill{flex:1 1 auto!important}.flex-lg-grow-0{flex-grow:0!important}.flex-lg-grow-1{flex-grow:1!important}.flex-lg-shrink-0{flex-shrink:0!important}.flex-lg-shrink-1{flex-shrink:1!important}.justify-content-lg-start{justify-content:flex-start!important}.justify-content-lg-end{justify-content:flex-end!important}.justify-content-lg-center{justify-content:center!important}.justify-content-lg-between{justify-content:space-between!important}.justify-content-lg-around{justify-content:space-around!important}.align-items-lg-start{align-items:flex-start!important}.align-items-lg-end{align-items:flex-end!important}.align-items-lg-center{align-items:center!important}.align-items-lg-baseline{align-items:baseline!important}.align-items-lg-stretch{align-items:stretch!important}.align-content-lg-start{align-content:flex-start!important}.align-content-lg-end{align-content:flex-end!important}.align-content-lg-center{align-content:center!important}.align-content-lg-between{align-content:space-between!important}.align-content-lg-around{align-content:space-around!important}.align-content-lg-stretch{align-content:stretch!important}.align-self-lg-auto{align-self:auto!important}.align-self-lg-start{align-self:flex-start!important}.align-self-lg-end{align-self:flex-end!important}.align-self-lg-center{align-self:center!important}.align-self-lg-baseline{align-self:baseline!important}.align-self-lg-stretch{align-self:stretch!important}}@media(min-width:1240px){.flex-xl-row{flex-direction:row!important}.flex-xl-column{flex-direction:column!important}.flex-xl-row-reverse{flex-direction:row-reverse!important}.flex-xl-column-reverse{flex-direction:column-reverse!important}.flex-xl-wrap{flex-wrap:wrap!important}.flex-xl-nowrap{flex-wrap:nowrap!important}.flex-xl-wrap-reverse{flex-wrap:wrap-reverse!important}.flex-xl-fill{flex:1 1 auto!important}.flex-xl-grow-0{flex-grow:0!important}.flex-xl-grow-1{flex-grow:1!important}.flex-xl-shrink-0{flex-shrink:0!important}.flex-xl-shrink-1{flex-shrink:1!important}.justify-content-xl-start{justify-content:flex-start!important}.justify-content-xl-end{justify-content:flex-end!important}.justify-content-xl-center{justify-content:center!important}.justify-content-xl-between{justify-content:space-between!important}.justify-content-xl-around{justify-content:space-around!important}.align-items-xl-start{align-items:flex-start!important}.align-items-xl-end{align-items:flex-end!important}.align-items-xl-center{align-items:center!important}.align-items-xl-baseline{align-items:baseline!important}.align-items-xl-stretch{align-items:stretch!important}.align-content-xl-start{align-content:flex-start!important}.align-content-xl-end{align-content:flex-end!important}.align-content-xl-center{align-content:center!important}.align-content-xl-between{align-content:space-between!important}.align-content-xl-around{align-content:space-around!important}.align-content-xl-stretch{align-content:stretch!important}.align-self-xl-auto{align-self:auto!important}.align-self-xl-start{align-self:flex-start!important}.align-self-xl-end{align-self:flex-end!important}.align-self-xl-center{align-self:center!important}.align-self-xl-baseline{align-self:baseline!important}.align-self-xl-stretch{align-self:stretch!important}}.float-left{float:left!important}.float-right{float:right!important}.float-none{float:none!important}@media(min-width:400px){.float-xs-left{float:left!important}.float-xs-right{float:right!important}.float-xs-none{float:none!important}}@media(min-width:616px){.float-sm-left{float:left!important}.float-sm-right{float:right!important}.float-sm-none{float:none!important}}@media(min-width:768px){.float-md-left{float:left!important}.float-md-right{float:right!important}.float-md-none{float:none!important}}@media(min-width:980px){.float-lg-left{float:left!important}.float-lg-right{float:right!important}.float-lg-none{float:none!important}}@media(min-width:1240px){.float-xl-left{float:left!important}.float-xl-right{float:right!important}.float-xl-none{float:none!important}}.overflow-auto{overflow:auto!important}.overflow-hidden{overflow:hidden!important}.position-static{position:static!important}.position-relative{position:relative!important}.position-absolute{position:absolute!important}.position-fixed{position:fixed!important}.position-sticky{position:sticky!important}.fixed-top{top:0}.fixed-bottom,.fixed-top{position:fixed;right:0;left:0;z-index:1030}.fixed-bottom{bottom:0}@supports(position:sticky){.sticky-top{position:sticky;top:0;z-index:1020}}.sr-only{position:absolute;width:1px;height:1px;padding:0;margin:-1px;overflow:hidden;clip:rect(0,0,0,0);white-space:nowrap;border:0}.sr-only-focusable:active,.sr-only-focusable:focus{position:static;width:auto;height:auto;overflow:visible;clip:auto;white-space:normal}.shadow-sm{box-shadow:0 2px 14px rgba(108,117,125,.2)!important}.shadow{box-shadow:0 8px 20px rgba(108,117,125,.2)!important}.shadow-lg{box-shadow:0 12px 32px rgba(108,117,125,.2)!important}.shadow-none{box-shadow:none!important}.w-25{width:25%!important}.w-50{width:50%!important}.w-75{width:75%!important}.w-100{width:100%!important}.w-auto{width:auto!important}.h-25{height:25%!important}.h-50{height:50%!important}.h-75{height:75%!important}.h-100{height:100%!important}.h-auto{height:auto!important}.mw-100{max-width:100%!important}.mh-100{max-height:100%!important}.min-vw-100{min-width:100vw!important}.min-vh-100{min-height:100vh!important}.vw-100{width:100vw!important}.vh-100{height:100vh!important}.stretched-link:after{position:absolute;top:0;right:0;bottom:0;left:0;z-index:1;pointer-events:auto;content:"";background-color:transparent}.m-0{margin:0!important}.mt-0,.my-0{margin-top:0!important}.mr-0,.mx-0{margin-right:0!important}.mb-0,.my-0{margin-bottom:0!important}.ml-0,.mx-0{margin-left:0!important}.m-1{margin:8px!important}.mt-1,.my-1{margin-top:8px!important}.mr-1,.mx-1{margin-right:8px!important}.mb-1,.my-1{margin-bottom:8px!important}.ml-1,.mx-1{margin-left:8px!important}.m-2{margin:16px!important}.mt-2,.my-2{margin-top:16px!important}.mr-2,.mx-2{margin-right:16px!important}.mb-2,.my-2{margin-bottom:16px!important}.ml-2,.mx-2{margin-left:16px!important}.m-3{margin:24px!important}.mt-3,.my-3{margin-top:24px!important}.mr-3,.mx-3{margin-right:24px!important}.mb-3,.my-3{margin-bottom:24px!important}.ml-3,.mx-3{margin-left:24px!important}.m-4{margin:32px!important}.mt-4,.my-4{margin-top:32px!important}.mr-4,.mx-4{margin-right:32px!important}.mb-4,.my-4{margin-bottom:32px!important}.ml-4,.mx-4{margin-left:32px!important}.m-5{margin:40px!important}.mt-5,.my-5{margin-top:40px!important}.mr-5,.mx-5{margin-right:40px!important}.mb-5,.my-5{margin-bottom:40px!important}.ml-5,.mx-5{margin-left:40px!important}.m-6{margin:48px!important}.mt-6,.my-6{margin-top:48px!important}.mr-6,.mx-6{margin-right:48px!important}.mb-6,.my-6{margin-bottom:48px!important}.ml-6,.mx-6{margin-left:48px!important}.m-7{margin:56px!important}.mt-7,.my-7{margin-top:56px!important}.mr-7,.mx-7{margin-right:56px!important}.mb-7,.my-7{margin-bottom:56px!important}.ml-7,.mx-7{margin-left:56px!important}.m-8{margin:64px!important}.mt-8,.my-8{margin-top:64px!important}.mr-8,.mx-8{margin-right:64px!important}.mb-8,.my-8{margin-bottom:64px!important}.ml-8,.mx-8{margin-left:64px!important}.m-9{margin:72px!important}.mt-9,.my-9{margin-top:72px!important}.mr-9,.mx-9{margin-right:72px!important}.mb-9,.my-9{margin-bottom:72px!important}.ml-9,.mx-9{margin-left:72px!important}.m-10{margin:80px!important}.mt-10,.my-10{margin-top:80px!important}.mr-10,.mx-10{margin-right:80px!important}.mb-10,.my-10{margin-bottom:80px!important}.ml-10,.mx-10{margin-left:80px!important}.m-12{margin:96px!important}.mt-12,.my-12{margin-top:96px!important}.mr-12,.mx-12{margin-right:96px!important}.mb-12,.my-12{margin-bottom:96px!important}.ml-12,.mx-12{margin-left:96px!important}.m-15{margin:120px!important}.mt-15,.my-15{margin-top:120px!important}.mr-15,.mx-15{margin-right:120px!important}.mb-15,.my-15{margin-bottom:120px!important}.ml-15,.mx-15{margin-left:120px!important}.p-0{padding:0!important}.pt-0,.py-0{padding-top:0!important}.pr-0,.px-0{padding-right:0!important}.pb-0,.py-0{padding-bottom:0!important}.pl-0,.px-0{padding-left:0!important}.p-1{padding:8px!important}.pt-1,.py-1{padding-top:8px!important}.pr-1,.px-1{padding-right:8px!important}.pb-1,.py-1{padding-bottom:8px!important}.pl-1,.px-1{padding-left:8px!important}.p-2{padding:16px!important}.pt-2,.py-2{padding-top:16px!important}.pr-2,.px-2{padding-right:16px!important}.pb-2,.py-2{padding-bottom:16px!important}.pl-2,.px-2{padding-left:16px!important}.p-3{padding:24px!important}.pt-3,.py-3{padding-top:24px!important}.pr-3,.px-3{padding-right:24px!important}.pb-3,.py-3{padding-bottom:24px!important}.pl-3,.px-3{padding-left:24px!important}.p-4{padding:32px!important}.pt-4,.py-4{padding-top:32px!important}.pr-4,.px-4{padding-right:32px!important}.pb-4,.py-4{padding-bottom:32px!important}.pl-4,.px-4{padding-left:32px!important}.p-5{padding:40px!important}.pt-5,.py-5{padding-top:40px!important}.pr-5,.px-5{padding-right:40px!important}.pb-5,.py-5{padding-bottom:40px!important}.pl-5,.px-5{padding-left:40px!important}.p-6{padding:48px!important}.pt-6,.py-6{padding-top:48px!important}.pr-6,.px-6{padding-right:48px!important}.pb-6,.py-6{padding-bottom:48px!important}.pl-6,.px-6{padding-left:48px!important}.p-7{padding:56px!important}.pt-7,.py-7{padding-top:56px!important}.pr-7,.px-7{padding-right:56px!important}.pb-7,.py-7{padding-bottom:56px!important}.pl-7,.px-7{padding-left:56px!important}.p-8{padding:64px!important}.pt-8,.py-8{padding-top:64px!important}.pr-8,.px-8{padding-right:64px!important}.pb-8,.py-8{padding-bottom:64px!important}.pl-8,.px-8{padding-left:64px!important}.p-9{padding:72px!important}.pt-9,.py-9{padding-top:72px!important}.pr-9,.px-9{padding-right:72px!important}.pb-9,.py-9{padding-bottom:72px!important}.pl-9,.px-9{padding-left:72px!important}.p-10{padding:80px!important}.pt-10,.py-10{padding-top:80px!important}.pr-10,.px-10{padding-right:80px!important}.pb-10,.py-10{padding-bottom:80px!important}.pl-10,.px-10{padding-left:80px!important}.p-12{padding:96px!important}.pt-12,.py-12{padding-top:96px!important}.pr-12,.px-12{padding-right:96px!important}.pb-12,.py-12{padding-bottom:96px!important}.pl-12,.px-12{padding-left:96px!important}.p-15{padding:120px!important}.pt-15,.py-15{padding-top:120px!important}.pr-15,.px-15{padding-right:120px!important}.pb-15,.py-15{padding-bottom:120px!important}.pl-15,.px-15{padding-left:120px!important}.m-n1{margin:-8px!important}.mt-n1,.my-n1{margin-top:-8px!important}.mr-n1,.mx-n1{margin-right:-8px!important}.mb-n1,.my-n1{margin-bottom:-8px!important}.ml-n1,.mx-n1{margin-left:-8px!important}.m-n2{margin:-16px!important}.mt-n2,.my-n2{margin-top:-16px!important}.mr-n2,.mx-n2{margin-right:-16px!important}.mb-n2,.my-n2{margin-bottom:-16px!important}.ml-n2,.mx-n2{margin-left:-16px!important}.m-n3{margin:-24px!important}.mt-n3,.my-n3{margin-top:-24px!important}.mr-n3,.mx-n3{margin-right:-24px!important}.mb-n3,.my-n3{margin-bottom:-24px!important}.ml-n3,.mx-n3{margin-left:-24px!important}.m-n4{margin:-32px!important}.mt-n4,.my-n4{margin-top:-32px!important}.mr-n4,.mx-n4{margin-right:-32px!important}.mb-n4,.my-n4{margin-bottom:-32px!important}.ml-n4,.mx-n4{margin-left:-32px!important}.m-n5{margin:-40px!important}.mt-n5,.my-n5{margin-top:-40px!important}.mr-n5,.mx-n5{margin-right:-40px!important}.mb-n5,.my-n5{margin-bottom:-40px!important}.ml-n5,.mx-n5{margin-left:-40px!important}.m-n6{margin:-48px!important}.mt-n6,.my-n6{margin-top:-48px!important}.mr-n6,.mx-n6{margin-right:-48px!important}.mb-n6,.my-n6{margin-bottom:-48px!important}.ml-n6,.mx-n6{margin-left:-48px!important}.m-n7{margin:-56px!important}.mt-n7,.my-n7{margin-top:-56px!important}.mr-n7,.mx-n7{margin-right:-56px!important}.mb-n7,.my-n7{margin-bottom:-56px!important}.ml-n7,.mx-n7{margin-left:-56px!important}.m-n8{margin:-64px!important}.mt-n8,.my-n8{margin-top:-64px!important}.mr-n8,.mx-n8{margin-right:-64px!important}.mb-n8,.my-n8{margin-bottom:-64px!important}.ml-n8,.mx-n8{margin-left:-64px!important}.m-n9{margin:-72px!important}.mt-n9,.my-n9{margin-top:-72px!important}.mr-n9,.mx-n9{margin-right:-72px!important}.mb-n9,.my-n9{margin-bottom:-72px!important}.ml-n9,.mx-n9{margin-left:-72px!important}.m-n10{margin:-80px!important}.mt-n10,.my-n10{margin-top:-80px!important}.mr-n10,.mx-n10{margin-right:-80px!important}.mb-n10,.my-n10{margin-bottom:-80px!important}.ml-n10,.mx-n10{margin-left:-80px!important}.m-n12{margin:-96px!important}.mt-n12,.my-n12{margin-top:-96px!important}.mr-n12,.mx-n12{margin-right:-96px!important}.mb-n12,.my-n12{margin-bottom:-96px!important}.ml-n12,.mx-n12{margin-left:-96px!important}.m-n15{margin:-120px!important}.mt-n15,.my-n15{margin-top:-120px!important}.mr-n15,.mx-n15{margin-right:-120px!important}.mb-n15,.my-n15{margin-bottom:-120px!important}.ml-n15,.mx-n15{margin-left:-120px!important}.m-auto{margin:auto!important}.mt-auto,.my-auto{margin-top:auto!important}.mr-auto,.mx-auto{margin-right:auto!important}.mb-auto,.my-auto{margin-bottom:auto!important}.ml-auto,.mx-auto{margin-left:auto!important}@media(min-width:400px){.m-xs-0{margin:0!important}.mt-xs-0,.my-xs-0{margin-top:0!important}.mr-xs-0,.mx-xs-0{margin-right:0!important}.mb-xs-0,.my-xs-0{margin-bottom:0!important}.ml-xs-0,.mx-xs-0{margin-left:0!important}.m-xs-1{margin:8px!important}.mt-xs-1,.my-xs-1{margin-top:8px!important}.mr-xs-1,.mx-xs-1{margin-right:8px!important}.mb-xs-1,.my-xs-1{margin-bottom:8px!important}.ml-xs-1,.mx-xs-1{margin-left:8px!important}.m-xs-2{margin:16px!important}.mt-xs-2,.my-xs-2{margin-top:16px!important}.mr-xs-2,.mx-xs-2{margin-right:16px!important}.mb-xs-2,.my-xs-2{margin-bottom:16px!important}.ml-xs-2,.mx-xs-2{margin-left:16px!important}.m-xs-3{margin:24px!important}.mt-xs-3,.my-xs-3{margin-top:24px!important}.mr-xs-3,.mx-xs-3{margin-right:24px!important}.mb-xs-3,.my-xs-3{margin-bottom:24px!important}.ml-xs-3,.mx-xs-3{margin-left:24px!important}.m-xs-4{margin:32px!important}.mt-xs-4,.my-xs-4{margin-top:32px!important}.mr-xs-4,.mx-xs-4{margin-right:32px!important}.mb-xs-4,.my-xs-4{margin-bottom:32px!important}.ml-xs-4,.mx-xs-4{margin-left:32px!important}.m-xs-5{margin:40px!important}.mt-xs-5,.my-xs-5{margin-top:40px!important}.mr-xs-5,.mx-xs-5{margin-right:40px!important}.mb-xs-5,.my-xs-5{margin-bottom:40px!important}.ml-xs-5,.mx-xs-5{margin-left:40px!important}.m-xs-6{margin:48px!important}.mt-xs-6,.my-xs-6{margin-top:48px!important}.mr-xs-6,.mx-xs-6{margin-right:48px!important}.mb-xs-6,.my-xs-6{margin-bottom:48px!important}.ml-xs-6,.mx-xs-6{margin-left:48px!important}.m-xs-7{margin:56px!important}.mt-xs-7,.my-xs-7{margin-top:56px!important}.mr-xs-7,.mx-xs-7{margin-right:56px!important}.mb-xs-7,.my-xs-7{margin-bottom:56px!important}.ml-xs-7,.mx-xs-7{margin-left:56px!important}.m-xs-8{margin:64px!important}.mt-xs-8,.my-xs-8{margin-top:64px!important}.mr-xs-8,.mx-xs-8{margin-right:64px!important}.mb-xs-8,.my-xs-8{margin-bottom:64px!important}.ml-xs-8,.mx-xs-8{margin-left:64px!important}.m-xs-9{margin:72px!important}.mt-xs-9,.my-xs-9{margin-top:72px!important}.mr-xs-9,.mx-xs-9{margin-right:72px!important}.mb-xs-9,.my-xs-9{margin-bottom:72px!important}.ml-xs-9,.mx-xs-9{margin-left:72px!important}.m-xs-10{margin:80px!important}.mt-xs-10,.my-xs-10{margin-top:80px!important}.mr-xs-10,.mx-xs-10{margin-right:80px!important}.mb-xs-10,.my-xs-10{margin-bottom:80px!important}.ml-xs-10,.mx-xs-10{margin-left:80px!important}.m-xs-12{margin:96px!important}.mt-xs-12,.my-xs-12{margin-top:96px!important}.mr-xs-12,.mx-xs-12{margin-right:96px!important}.mb-xs-12,.my-xs-12{margin-bottom:96px!important}.ml-xs-12,.mx-xs-12{margin-left:96px!important}.m-xs-15{margin:120px!important}.mt-xs-15,.my-xs-15{margin-top:120px!important}.mr-xs-15,.mx-xs-15{margin-right:120px!important}.mb-xs-15,.my-xs-15{margin-bottom:120px!important}.ml-xs-15,.mx-xs-15{margin-left:120px!important}.p-xs-0{padding:0!important}.pt-xs-0,.py-xs-0{padding-top:0!important}.pr-xs-0,.px-xs-0{padding-right:0!important}.pb-xs-0,.py-xs-0{padding-bottom:0!important}.pl-xs-0,.px-xs-0{padding-left:0!important}.p-xs-1{padding:8px!important}.pt-xs-1,.py-xs-1{padding-top:8px!important}.pr-xs-1,.px-xs-1{padding-right:8px!important}.pb-xs-1,.py-xs-1{padding-bottom:8px!important}.pl-xs-1,.px-xs-1{padding-left:8px!important}.p-xs-2{padding:16px!important}.pt-xs-2,.py-xs-2{padding-top:16px!important}.pr-xs-2,.px-xs-2{padding-right:16px!important}.pb-xs-2,.py-xs-2{padding-bottom:16px!important}.pl-xs-2,.px-xs-2{padding-left:16px!important}.p-xs-3{padding:24px!important}.pt-xs-3,.py-xs-3{padding-top:24px!important}.pr-xs-3,.px-xs-3{padding-right:24px!important}.pb-xs-3,.py-xs-3{padding-bottom:24px!important}.pl-xs-3,.px-xs-3{padding-left:24px!important}.p-xs-4{padding:32px!important}.pt-xs-4,.py-xs-4{padding-top:32px!important}.pr-xs-4,.px-xs-4{padding-right:32px!important}.pb-xs-4,.py-xs-4{padding-bottom:32px!important}.pl-xs-4,.px-xs-4{padding-left:32px!important}.p-xs-5{padding:40px!important}.pt-xs-5,.py-xs-5{padding-top:40px!important}.pr-xs-5,.px-xs-5{padding-right:40px!important}.pb-xs-5,.py-xs-5{padding-bottom:40px!important}.pl-xs-5,.px-xs-5{padding-left:40px!important}.p-xs-6{padding:48px!important}.pt-xs-6,.py-xs-6{padding-top:48px!important}.pr-xs-6,.px-xs-6{padding-right:48px!important}.pb-xs-6,.py-xs-6{padding-bottom:48px!important}.pl-xs-6,.px-xs-6{padding-left:48px!important}.p-xs-7{padding:56px!important}.pt-xs-7,.py-xs-7{padding-top:56px!important}.pr-xs-7,.px-xs-7{padding-right:56px!important}.pb-xs-7,.py-xs-7{padding-bottom:56px!important}.pl-xs-7,.px-xs-7{padding-left:56px!important}.p-xs-8{padding:64px!important}.pt-xs-8,.py-xs-8{padding-top:64px!important}.pr-xs-8,.px-xs-8{padding-right:64px!important}.pb-xs-8,.py-xs-8{padding-bottom:64px!important}.pl-xs-8,.px-xs-8{padding-left:64px!important}.p-xs-9{padding:72px!important}.pt-xs-9,.py-xs-9{padding-top:72px!important}.pr-xs-9,.px-xs-9{padding-right:72px!important}.pb-xs-9,.py-xs-9{padding-bottom:72px!important}.pl-xs-9,.px-xs-9{padding-left:72px!important}.p-xs-10{padding:80px!important}.pt-xs-10,.py-xs-10{padding-top:80px!important}.pr-xs-10,.px-xs-10{padding-right:80px!important}.pb-xs-10,.py-xs-10{padding-bottom:80px!important}.pl-xs-10,.px-xs-10{padding-left:80px!important}.p-xs-12{padding:96px!important}.pt-xs-12,.py-xs-12{padding-top:96px!important}.pr-xs-12,.px-xs-12{padding-right:96px!important}.pb-xs-12,.py-xs-12{padding-bottom:96px!important}.pl-xs-12,.px-xs-12{padding-left:96px!important}.p-xs-15{padding:120px!important}.pt-xs-15,.py-xs-15{padding-top:120px!important}.pr-xs-15,.px-xs-15{padding-right:120px!important}.pb-xs-15,.py-xs-15{padding-bottom:120px!important}.pl-xs-15,.px-xs-15{padding-left:120px!important}.m-xs-n1{margin:-8px!important}.mt-xs-n1,.my-xs-n1{margin-top:-8px!important}.mr-xs-n1,.mx-xs-n1{margin-right:-8px!important}.mb-xs-n1,.my-xs-n1{margin-bottom:-8px!important}.ml-xs-n1,.mx-xs-n1{margin-left:-8px!important}.m-xs-n2{margin:-16px!important}.mt-xs-n2,.my-xs-n2{margin-top:-16px!important}.mr-xs-n2,.mx-xs-n2{margin-right:-16px!important}.mb-xs-n2,.my-xs-n2{margin-bottom:-16px!important}.ml-xs-n2,.mx-xs-n2{margin-left:-16px!important}.m-xs-n3{margin:-24px!important}.mt-xs-n3,.my-xs-n3{margin-top:-24px!important}.mr-xs-n3,.mx-xs-n3{margin-right:-24px!important}.mb-xs-n3,.my-xs-n3{margin-bottom:-24px!important}.ml-xs-n3,.mx-xs-n3{margin-left:-24px!important}.m-xs-n4{margin:-32px!important}.mt-xs-n4,.my-xs-n4{margin-top:-32px!important}.mr-xs-n4,.mx-xs-n4{margin-right:-32px!important}.mb-xs-n4,.my-xs-n4{margin-bottom:-32px!important}.ml-xs-n4,.mx-xs-n4{margin-left:-32px!important}.m-xs-n5{margin:-40px!important}.mt-xs-n5,.my-xs-n5{margin-top:-40px!important}.mr-xs-n5,.mx-xs-n5{margin-right:-40px!important}.mb-xs-n5,.my-xs-n5{margin-bottom:-40px!important}.ml-xs-n5,.mx-xs-n5{margin-left:-40px!important}.m-xs-n6{margin:-48px!important}.mt-xs-n6,.my-xs-n6{margin-top:-48px!important}.mr-xs-n6,.mx-xs-n6{margin-right:-48px!important}.mb-xs-n6,.my-xs-n6{margin-bottom:-48px!important}.ml-xs-n6,.mx-xs-n6{margin-left:-48px!important}.m-xs-n7{margin:-56px!important}.mt-xs-n7,.my-xs-n7{margin-top:-56px!important}.mr-xs-n7,.mx-xs-n7{margin-right:-56px!important}.mb-xs-n7,.my-xs-n7{margin-bottom:-56px!important}.ml-xs-n7,.mx-xs-n7{margin-left:-56px!important}.m-xs-n8{margin:-64px!important}.mt-xs-n8,.my-xs-n8{margin-top:-64px!important}.mr-xs-n8,.mx-xs-n8{margin-right:-64px!important}.mb-xs-n8,.my-xs-n8{margin-bottom:-64px!important}.ml-xs-n8,.mx-xs-n8{margin-left:-64px!important}.m-xs-n9{margin:-72px!important}.mt-xs-n9,.my-xs-n9{margin-top:-72px!important}.mr-xs-n9,.mx-xs-n9{margin-right:-72px!important}.mb-xs-n9,.my-xs-n9{margin-bottom:-72px!important}.ml-xs-n9,.mx-xs-n9{margin-left:-72px!important}.m-xs-n10{margin:-80px!important}.mt-xs-n10,.my-xs-n10{margin-top:-80px!important}.mr-xs-n10,.mx-xs-n10{margin-right:-80px!important}.mb-xs-n10,.my-xs-n10{margin-bottom:-80px!important}.ml-xs-n10,.mx-xs-n10{margin-left:-80px!important}.m-xs-n12{margin:-96px!important}.mt-xs-n12,.my-xs-n12{margin-top:-96px!important}.mr-xs-n12,.mx-xs-n12{margin-right:-96px!important}.mb-xs-n12,.my-xs-n12{margin-bottom:-96px!important}.ml-xs-n12,.mx-xs-n12{margin-left:-96px!important}.m-xs-n15{margin:-120px!important}.mt-xs-n15,.my-xs-n15{margin-top:-120px!important}.mr-xs-n15,.mx-xs-n15{margin-right:-120px!important}.mb-xs-n15,.my-xs-n15{margin-bottom:-120px!important}.ml-xs-n15,.mx-xs-n15{margin-left:-120px!important}.m-xs-auto{margin:auto!important}.mt-xs-auto,.my-xs-auto{margin-top:auto!important}.mr-xs-auto,.mx-xs-auto{margin-right:auto!important}.mb-xs-auto,.my-xs-auto{margin-bottom:auto!important}.ml-xs-auto,.mx-xs-auto{margin-left:auto!important}}@media(min-width:616px){.m-sm-0{margin:0!important}.mt-sm-0,.my-sm-0{margin-top:0!important}.mr-sm-0,.mx-sm-0{margin-right:0!important}.mb-sm-0,.my-sm-0{margin-bottom:0!important}.ml-sm-0,.mx-sm-0{margin-left:0!important}.m-sm-1{margin:8px!important}.mt-sm-1,.my-sm-1{margin-top:8px!important}.mr-sm-1,.mx-sm-1{margin-right:8px!important}.mb-sm-1,.my-sm-1{margin-bottom:8px!important}.ml-sm-1,.mx-sm-1{margin-left:8px!important}.m-sm-2{margin:16px!important}.mt-sm-2,.my-sm-2{margin-top:16px!important}.mr-sm-2,.mx-sm-2{margin-right:16px!important}.mb-sm-2,.my-sm-2{margin-bottom:16px!important}.ml-sm-2,.mx-sm-2{margin-left:16px!important}.m-sm-3{margin:24px!important}.mt-sm-3,.my-sm-3{margin-top:24px!important}.mr-sm-3,.mx-sm-3{margin-right:24px!important}.mb-sm-3,.my-sm-3{margin-bottom:24px!important}.ml-sm-3,.mx-sm-3{margin-left:24px!important}.m-sm-4{margin:32px!important}.mt-sm-4,.my-sm-4{margin-top:32px!important}.mr-sm-4,.mx-sm-4{margin-right:32px!important}.mb-sm-4,.my-sm-4{margin-bottom:32px!important}.ml-sm-4,.mx-sm-4{margin-left:32px!important}.m-sm-5{margin:40px!important}.mt-sm-5,.my-sm-5{margin-top:40px!important}.mr-sm-5,.mx-sm-5{margin-right:40px!important}.mb-sm-5,.my-sm-5{margin-bottom:40px!important}.ml-sm-5,.mx-sm-5{margin-left:40px!important}.m-sm-6{margin:48px!important}.mt-sm-6,.my-sm-6{margin-top:48px!important}.mr-sm-6,.mx-sm-6{margin-right:48px!important}.mb-sm-6,.my-sm-6{margin-bottom:48px!important}.ml-sm-6,.mx-sm-6{margin-left:48px!important}.m-sm-7{margin:56px!important}.mt-sm-7,.my-sm-7{margin-top:56px!important}.mr-sm-7,.mx-sm-7{margin-right:56px!important}.mb-sm-7,.my-sm-7{margin-bottom:56px!important}.ml-sm-7,.mx-sm-7{margin-left:56px!important}.m-sm-8{margin:64px!important}.mt-sm-8,.my-sm-8{margin-top:64px!important}.mr-sm-8,.mx-sm-8{margin-right:64px!important}.mb-sm-8,.my-sm-8{margin-bottom:64px!important}.ml-sm-8,.mx-sm-8{margin-left:64px!important}.m-sm-9{margin:72px!important}.mt-sm-9,.my-sm-9{margin-top:72px!important}.mr-sm-9,.mx-sm-9{margin-right:72px!important}.mb-sm-9,.my-sm-9{margin-bottom:72px!important}.ml-sm-9,.mx-sm-9{margin-left:72px!important}.m-sm-10{margin:80px!important}.mt-sm-10,.my-sm-10{margin-top:80px!important}.mr-sm-10,.mx-sm-10{margin-right:80px!important}.mb-sm-10,.my-sm-10{margin-bottom:80px!important}.ml-sm-10,.mx-sm-10{margin-left:80px!important}.m-sm-12{margin:96px!important}.mt-sm-12,.my-sm-12{margin-top:96px!important}.mr-sm-12,.mx-sm-12{margin-right:96px!important}.mb-sm-12,.my-sm-12{margin-bottom:96px!important}.ml-sm-12,.mx-sm-12{margin-left:96px!important}.m-sm-15{margin:120px!important}.mt-sm-15,.my-sm-15{margin-top:120px!important}.mr-sm-15,.mx-sm-15{margin-right:120px!important}.mb-sm-15,.my-sm-15{margin-bottom:120px!important}.ml-sm-15,.mx-sm-15{margin-left:120px!important}.p-sm-0{padding:0!important}.pt-sm-0,.py-sm-0{padding-top:0!important}.pr-sm-0,.px-sm-0{padding-right:0!important}.pb-sm-0,.py-sm-0{padding-bottom:0!important}.pl-sm-0,.px-sm-0{padding-left:0!important}.p-sm-1{padding:8px!important}.pt-sm-1,.py-sm-1{padding-top:8px!important}.pr-sm-1,.px-sm-1{padding-right:8px!important}.pb-sm-1,.py-sm-1{padding-bottom:8px!important}.pl-sm-1,.px-sm-1{padding-left:8px!important}.p-sm-2{padding:16px!important}.pt-sm-2,.py-sm-2{padding-top:16px!important}.pr-sm-2,.px-sm-2{padding-right:16px!important}.pb-sm-2,.py-sm-2{padding-bottom:16px!important}.pl-sm-2,.px-sm-2{padding-left:16px!important}.p-sm-3{padding:24px!important}.pt-sm-3,.py-sm-3{padding-top:24px!important}.pr-sm-3,.px-sm-3{padding-right:24px!important}.pb-sm-3,.py-sm-3{padding-bottom:24px!important}.pl-sm-3,.px-sm-3{padding-left:24px!important}.p-sm-4{padding:32px!important}.pt-sm-4,.py-sm-4{padding-top:32px!important}.pr-sm-4,.px-sm-4{padding-right:32px!important}.pb-sm-4,.py-sm-4{padding-bottom:32px!important}.pl-sm-4,.px-sm-4{padding-left:32px!important}.p-sm-5{padding:40px!important}.pt-sm-5,.py-sm-5{padding-top:40px!important}.pr-sm-5,.px-sm-5{padding-right:40px!important}.pb-sm-5,.py-sm-5{padding-bottom:40px!important}.pl-sm-5,.px-sm-5{padding-left:40px!important}.p-sm-6{padding:48px!important}.pt-sm-6,.py-sm-6{padding-top:48px!important}.pr-sm-6,.px-sm-6{padding-right:48px!important}.pb-sm-6,.py-sm-6{padding-bottom:48px!important}.pl-sm-6,.px-sm-6{padding-left:48px!important}.p-sm-7{padding:56px!important}.pt-sm-7,.py-sm-7{padding-top:56px!important}.pr-sm-7,.px-sm-7{padding-right:56px!important}.pb-sm-7,.py-sm-7{padding-bottom:56px!important}.pl-sm-7,.px-sm-7{padding-left:56px!important}.p-sm-8{padding:64px!important}.pt-sm-8,.py-sm-8{padding-top:64px!important}.pr-sm-8,.px-sm-8{padding-right:64px!important}.pb-sm-8,.py-sm-8{padding-bottom:64px!important}.pl-sm-8,.px-sm-8{padding-left:64px!important}.p-sm-9{padding:72px!important}.pt-sm-9,.py-sm-9{padding-top:72px!important}.pr-sm-9,.px-sm-9{padding-right:72px!important}.pb-sm-9,.py-sm-9{padding-bottom:72px!important}.pl-sm-9,.px-sm-9{padding-left:72px!important}.p-sm-10{padding:80px!important}.pt-sm-10,.py-sm-10{padding-top:80px!important}.pr-sm-10,.px-sm-10{padding-right:80px!important}.pb-sm-10,.py-sm-10{padding-bottom:80px!important}.pl-sm-10,.px-sm-10{padding-left:80px!important}.p-sm-12{padding:96px!important}.pt-sm-12,.py-sm-12{padding-top:96px!important}.pr-sm-12,.px-sm-12{padding-right:96px!important}.pb-sm-12,.py-sm-12{padding-bottom:96px!important}.pl-sm-12,.px-sm-12{padding-left:96px!important}.p-sm-15{padding:120px!important}.pt-sm-15,.py-sm-15{padding-top:120px!important}.pr-sm-15,.px-sm-15{padding-right:120px!important}.pb-sm-15,.py-sm-15{padding-bottom:120px!important}.pl-sm-15,.px-sm-15{padding-left:120px!important}.m-sm-n1{margin:-8px!important}.mt-sm-n1,.my-sm-n1{margin-top:-8px!important}.mr-sm-n1,.mx-sm-n1{margin-right:-8px!important}.mb-sm-n1,.my-sm-n1{margin-bottom:-8px!important}.ml-sm-n1,.mx-sm-n1{margin-left:-8px!important}.m-sm-n2{margin:-16px!important}.mt-sm-n2,.my-sm-n2{margin-top:-16px!important}.mr-sm-n2,.mx-sm-n2{margin-right:-16px!important}.mb-sm-n2,.my-sm-n2{margin-bottom:-16px!important}.ml-sm-n2,.mx-sm-n2{margin-left:-16px!important}.m-sm-n3{margin:-24px!important}.mt-sm-n3,.my-sm-n3{margin-top:-24px!important}.mr-sm-n3,.mx-sm-n3{margin-right:-24px!important}.mb-sm-n3,.my-sm-n3{margin-bottom:-24px!important}.ml-sm-n3,.mx-sm-n3{margin-left:-24px!important}.m-sm-n4{margin:-32px!important}.mt-sm-n4,.my-sm-n4{margin-top:-32px!important}.mr-sm-n4,.mx-sm-n4{margin-right:-32px!important}.mb-sm-n4,.my-sm-n4{margin-bottom:-32px!important}.ml-sm-n4,.mx-sm-n4{margin-left:-32px!important}.m-sm-n5{margin:-40px!important}.mt-sm-n5,.my-sm-n5{margin-top:-40px!important}.mr-sm-n5,.mx-sm-n5{margin-right:-40px!important}.mb-sm-n5,.my-sm-n5{margin-bottom:-40px!important}.ml-sm-n5,.mx-sm-n5{margin-left:-40px!important}.m-sm-n6{margin:-48px!important}.mt-sm-n6,.my-sm-n6{margin-top:-48px!important}.mr-sm-n6,.mx-sm-n6{margin-right:-48px!important}.mb-sm-n6,.my-sm-n6{margin-bottom:-48px!important}.ml-sm-n6,.mx-sm-n6{margin-left:-48px!important}.m-sm-n7{margin:-56px!important}.mt-sm-n7,.my-sm-n7{margin-top:-56px!important}.mr-sm-n7,.mx-sm-n7{margin-right:-56px!important}.mb-sm-n7,.my-sm-n7{margin-bottom:-56px!important}.ml-sm-n7,.mx-sm-n7{margin-left:-56px!important}.m-sm-n8{margin:-64px!important}.mt-sm-n8,.my-sm-n8{margin-top:-64px!important}.mr-sm-n8,.mx-sm-n8{margin-right:-64px!important}.mb-sm-n8,.my-sm-n8{margin-bottom:-64px!important}.ml-sm-n8,.mx-sm-n8{margin-left:-64px!important}.m-sm-n9{margin:-72px!important}.mt-sm-n9,.my-sm-n9{margin-top:-72px!important}.mr-sm-n9,.mx-sm-n9{margin-right:-72px!important}.mb-sm-n9,.my-sm-n9{margin-bottom:-72px!important}.ml-sm-n9,.mx-sm-n9{margin-left:-72px!important}.m-sm-n10{margin:-80px!important}.mt-sm-n10,.my-sm-n10{margin-top:-80px!important}.mr-sm-n10,.mx-sm-n10{margin-right:-80px!important}.mb-sm-n10,.my-sm-n10{margin-bottom:-80px!important}.ml-sm-n10,.mx-sm-n10{margin-left:-80px!important}.m-sm-n12{margin:-96px!important}.mt-sm-n12,.my-sm-n12{margin-top:-96px!important}.mr-sm-n12,.mx-sm-n12{margin-right:-96px!important}.mb-sm-n12,.my-sm-n12{margin-bottom:-96px!important}.ml-sm-n12,.mx-sm-n12{margin-left:-96px!important}.m-sm-n15{margin:-120px!important}.mt-sm-n15,.my-sm-n15{margin-top:-120px!important}.mr-sm-n15,.mx-sm-n15{margin-right:-120px!important}.mb-sm-n15,.my-sm-n15{margin-bottom:-120px!important}.ml-sm-n15,.mx-sm-n15{margin-left:-120px!important}.m-sm-auto{margin:auto!important}.mt-sm-auto,.my-sm-auto{margin-top:auto!important}.mr-sm-auto,.mx-sm-auto{margin-right:auto!important}.mb-sm-auto,.my-sm-auto{margin-bottom:auto!important}.ml-sm-auto,.mx-sm-auto{margin-left:auto!important}}@media(min-width:768px){.m-md-0{margin:0!important}.mt-md-0,.my-md-0{margin-top:0!important}.mr-md-0,.mx-md-0{margin-right:0!important}.mb-md-0,.my-md-0{margin-bottom:0!important}.ml-md-0,.mx-md-0{margin-left:0!important}.m-md-1{margin:8px!important}.mt-md-1,.my-md-1{margin-top:8px!important}.mr-md-1,.mx-md-1{margin-right:8px!important}.mb-md-1,.my-md-1{margin-bottom:8px!important}.ml-md-1,.mx-md-1{margin-left:8px!important}.m-md-2{margin:16px!important}.mt-md-2,.my-md-2{margin-top:16px!important}.mr-md-2,.mx-md-2{margin-right:16px!important}.mb-md-2,.my-md-2{margin-bottom:16px!important}.ml-md-2,.mx-md-2{margin-left:16px!important}.m-md-3{margin:24px!important}.mt-md-3,.my-md-3{margin-top:24px!important}.mr-md-3,.mx-md-3{margin-right:24px!important}.mb-md-3,.my-md-3{margin-bottom:24px!important}.ml-md-3,.mx-md-3{margin-left:24px!important}.m-md-4{margin:32px!important}.mt-md-4,.my-md-4{margin-top:32px!important}.mr-md-4,.mx-md-4{margin-right:32px!important}.mb-md-4,.my-md-4{margin-bottom:32px!important}.ml-md-4,.mx-md-4{margin-left:32px!important}.m-md-5{margin:40px!important}.mt-md-5,.my-md-5{margin-top:40px!important}.mr-md-5,.mx-md-5{margin-right:40px!important}.mb-md-5,.my-md-5{margin-bottom:40px!important}.ml-md-5,.mx-md-5{margin-left:40px!important}.m-md-6{margin:48px!important}.mt-md-6,.my-md-6{margin-top:48px!important}.mr-md-6,.mx-md-6{margin-right:48px!important}.mb-md-6,.my-md-6{margin-bottom:48px!important}.ml-md-6,.mx-md-6{margin-left:48px!important}.m-md-7{margin:56px!important}.mt-md-7,.my-md-7{margin-top:56px!important}.mr-md-7,.mx-md-7{margin-right:56px!important}.mb-md-7,.my-md-7{margin-bottom:56px!important}.ml-md-7,.mx-md-7{margin-left:56px!important}.m-md-8{margin:64px!important}.mt-md-8,.my-md-8{margin-top:64px!important}.mr-md-8,.mx-md-8{margin-right:64px!important}.mb-md-8,.my-md-8{margin-bottom:64px!important}.ml-md-8,.mx-md-8{margin-left:64px!important}.m-md-9{margin:72px!important}.mt-md-9,.my-md-9{margin-top:72px!important}.mr-md-9,.mx-md-9{margin-right:72px!important}.mb-md-9,.my-md-9{margin-bottom:72px!important}.ml-md-9,.mx-md-9{margin-left:72px!important}.m-md-10{margin:80px!important}.mt-md-10,.my-md-10{margin-top:80px!important}.mr-md-10,.mx-md-10{margin-right:80px!important}.mb-md-10,.my-md-10{margin-bottom:80px!important}.ml-md-10,.mx-md-10{margin-left:80px!important}.m-md-12{margin:96px!important}.mt-md-12,.my-md-12{margin-top:96px!important}.mr-md-12,.mx-md-12{margin-right:96px!important}.mb-md-12,.my-md-12{margin-bottom:96px!important}.ml-md-12,.mx-md-12{margin-left:96px!important}.m-md-15{margin:120px!important}.mt-md-15,.my-md-15{margin-top:120px!important}.mr-md-15,.mx-md-15{margin-right:120px!important}.mb-md-15,.my-md-15{margin-bottom:120px!important}.ml-md-15,.mx-md-15{margin-left:120px!important}.p-md-0{padding:0!important}.pt-md-0,.py-md-0{padding-top:0!important}.pr-md-0,.px-md-0{padding-right:0!important}.pb-md-0,.py-md-0{padding-bottom:0!important}.pl-md-0,.px-md-0{padding-left:0!important}.p-md-1{padding:8px!important}.pt-md-1,.py-md-1{padding-top:8px!important}.pr-md-1,.px-md-1{padding-right:8px!important}.pb-md-1,.py-md-1{padding-bottom:8px!important}.pl-md-1,.px-md-1{padding-left:8px!important}.p-md-2{padding:16px!important}.pt-md-2,.py-md-2{padding-top:16px!important}.pr-md-2,.px-md-2{padding-right:16px!important}.pb-md-2,.py-md-2{padding-bottom:16px!important}.pl-md-2,.px-md-2{padding-left:16px!important}.p-md-3{padding:24px!important}.pt-md-3,.py-md-3{padding-top:24px!important}.pr-md-3,.px-md-3{padding-right:24px!important}.pb-md-3,.py-md-3{padding-bottom:24px!important}.pl-md-3,.px-md-3{padding-left:24px!important}.p-md-4{padding:32px!important}.pt-md-4,.py-md-4{padding-top:32px!important}.pr-md-4,.px-md-4{padding-right:32px!important}.pb-md-4,.py-md-4{padding-bottom:32px!important}.pl-md-4,.px-md-4{padding-left:32px!important}.p-md-5{padding:40px!important}.pt-md-5,.py-md-5{padding-top:40px!important}.pr-md-5,.px-md-5{padding-right:40px!important}.pb-md-5,.py-md-5{padding-bottom:40px!important}.pl-md-5,.px-md-5{padding-left:40px!important}.p-md-6{padding:48px!important}.pt-md-6,.py-md-6{padding-top:48px!important}.pr-md-6,.px-md-6{padding-right:48px!important}.pb-md-6,.py-md-6{padding-bottom:48px!important}.pl-md-6,.px-md-6{padding-left:48px!important}.p-md-7{padding:56px!important}.pt-md-7,.py-md-7{padding-top:56px!important}.pr-md-7,.px-md-7{padding-right:56px!important}.pb-md-7,.py-md-7{padding-bottom:56px!important}.pl-md-7,.px-md-7{padding-left:56px!important}.p-md-8{padding:64px!important}.pt-md-8,.py-md-8{padding-top:64px!important}.pr-md-8,.px-md-8{padding-right:64px!important}.pb-md-8,.py-md-8{padding-bottom:64px!important}.pl-md-8,.px-md-8{padding-left:64px!important}.p-md-9{padding:72px!important}.pt-md-9,.py-md-9{padding-top:72px!important}.pr-md-9,.px-md-9{padding-right:72px!important}.pb-md-9,.py-md-9{padding-bottom:72px!important}.pl-md-9,.px-md-9{padding-left:72px!important}.p-md-10{padding:80px!important}.pt-md-10,.py-md-10{padding-top:80px!important}.pr-md-10,.px-md-10{padding-right:80px!important}.pb-md-10,.py-md-10{padding-bottom:80px!important}.pl-md-10,.px-md-10{padding-left:80px!important}.p-md-12{padding:96px!important}.pt-md-12,.py-md-12{padding-top:96px!important}.pr-md-12,.px-md-12{padding-right:96px!important}.pb-md-12,.py-md-12{padding-bottom:96px!important}.pl-md-12,.px-md-12{padding-left:96px!important}.p-md-15{padding:120px!important}.pt-md-15,.py-md-15{padding-top:120px!important}.pr-md-15,.px-md-15{padding-right:120px!important}.pb-md-15,.py-md-15{padding-bottom:120px!important}.pl-md-15,.px-md-15{padding-left:120px!important}.m-md-n1{margin:-8px!important}.mt-md-n1,.my-md-n1{margin-top:-8px!important}.mr-md-n1,.mx-md-n1{margin-right:-8px!important}.mb-md-n1,.my-md-n1{margin-bottom:-8px!important}.ml-md-n1,.mx-md-n1{margin-left:-8px!important}.m-md-n2{margin:-16px!important}.mt-md-n2,.my-md-n2{margin-top:-16px!important}.mr-md-n2,.mx-md-n2{margin-right:-16px!important}.mb-md-n2,.my-md-n2{margin-bottom:-16px!important}.ml-md-n2,.mx-md-n2{margin-left:-16px!important}.m-md-n3{margin:-24px!important}.mt-md-n3,.my-md-n3{margin-top:-24px!important}.mr-md-n3,.mx-md-n3{margin-right:-24px!important}.mb-md-n3,.my-md-n3{margin-bottom:-24px!important}.ml-md-n3,.mx-md-n3{margin-left:-24px!important}.m-md-n4{margin:-32px!important}.mt-md-n4,.my-md-n4{margin-top:-32px!important}.mr-md-n4,.mx-md-n4{margin-right:-32px!important}.mb-md-n4,.my-md-n4{margin-bottom:-32px!important}.ml-md-n4,.mx-md-n4{margin-left:-32px!important}.m-md-n5{margin:-40px!important}.mt-md-n5,.my-md-n5{margin-top:-40px!important}.mr-md-n5,.mx-md-n5{margin-right:-40px!important}.mb-md-n5,.my-md-n5{margin-bottom:-40px!important}.ml-md-n5,.mx-md-n5{margin-left:-40px!important}.m-md-n6{margin:-48px!important}.mt-md-n6,.my-md-n6{margin-top:-48px!important}.mr-md-n6,.mx-md-n6{margin-right:-48px!important}.mb-md-n6,.my-md-n6{margin-bottom:-48px!important}.ml-md-n6,.mx-md-n6{margin-left:-48px!important}.m-md-n7{margin:-56px!important}.mt-md-n7,.my-md-n7{margin-top:-56px!important}.mr-md-n7,.mx-md-n7{margin-right:-56px!important}.mb-md-n7,.my-md-n7{margin-bottom:-56px!important}.ml-md-n7,.mx-md-n7{margin-left:-56px!important}.m-md-n8{margin:-64px!important}.mt-md-n8,.my-md-n8{margin-top:-64px!important}.mr-md-n8,.mx-md-n8{margin-right:-64px!important}.mb-md-n8,.my-md-n8{margin-bottom:-64px!important}.ml-md-n8,.mx-md-n8{margin-left:-64px!important}.m-md-n9{margin:-72px!important}.mt-md-n9,.my-md-n9{margin-top:-72px!important}.mr-md-n9,.mx-md-n9{margin-right:-72px!important}.mb-md-n9,.my-md-n9{margin-bottom:-72px!important}.ml-md-n9,.mx-md-n9{margin-left:-72px!important}.m-md-n10{margin:-80px!important}.mt-md-n10,.my-md-n10{margin-top:-80px!important}.mr-md-n10,.mx-md-n10{margin-right:-80px!important}.mb-md-n10,.my-md-n10{margin-bottom:-80px!important}.ml-md-n10,.mx-md-n10{margin-left:-80px!important}.m-md-n12{margin:-96px!important}.mt-md-n12,.my-md-n12{margin-top:-96px!important}.mr-md-n12,.mx-md-n12{margin-right:-96px!important}.mb-md-n12,.my-md-n12{margin-bottom:-96px!important}.ml-md-n12,.mx-md-n12{margin-left:-96px!important}.m-md-n15{margin:-120px!important}.mt-md-n15,.my-md-n15{margin-top:-120px!important}.mr-md-n15,.mx-md-n15{margin-right:-120px!important}.mb-md-n15,.my-md-n15{margin-bottom:-120px!important}.ml-md-n15,.mx-md-n15{margin-left:-120px!important}.m-md-auto{margin:auto!important}.mt-md-auto,.my-md-auto{margin-top:auto!important}.mr-md-auto,.mx-md-auto{margin-right:auto!important}.mb-md-auto,.my-md-auto{margin-bottom:auto!important}.ml-md-auto,.mx-md-auto{margin-left:auto!important}}@media(min-width:980px){.m-lg-0{margin:0!important}.mt-lg-0,.my-lg-0{margin-top:0!important}.mr-lg-0,.mx-lg-0{margin-right:0!important}.mb-lg-0,.my-lg-0{margin-bottom:0!important}.ml-lg-0,.mx-lg-0{margin-left:0!important}.m-lg-1{margin:8px!important}.mt-lg-1,.my-lg-1{margin-top:8px!important}.mr-lg-1,.mx-lg-1{margin-right:8px!important}.mb-lg-1,.my-lg-1{margin-bottom:8px!important}.ml-lg-1,.mx-lg-1{margin-left:8px!important}.m-lg-2{margin:16px!important}.mt-lg-2,.my-lg-2{margin-top:16px!important}.mr-lg-2,.mx-lg-2{margin-right:16px!important}.mb-lg-2,.my-lg-2{margin-bottom:16px!important}.ml-lg-2,.mx-lg-2{margin-left:16px!important}.m-lg-3{margin:24px!important}.mt-lg-3,.my-lg-3{margin-top:24px!important}.mr-lg-3,.mx-lg-3{margin-right:24px!important}.mb-lg-3,.my-lg-3{margin-bottom:24px!important}.ml-lg-3,.mx-lg-3{margin-left:24px!important}.m-lg-4{margin:32px!important}.mt-lg-4,.my-lg-4{margin-top:32px!important}.mr-lg-4,.mx-lg-4{margin-right:32px!important}.mb-lg-4,.my-lg-4{margin-bottom:32px!important}.ml-lg-4,.mx-lg-4{margin-left:32px!important}.m-lg-5{margin:40px!important}.mt-lg-5,.my-lg-5{margin-top:40px!important}.mr-lg-5,.mx-lg-5{margin-right:40px!important}.mb-lg-5,.my-lg-5{margin-bottom:40px!important}.ml-lg-5,.mx-lg-5{margin-left:40px!important}.m-lg-6{margin:48px!important}.mt-lg-6,.my-lg-6{margin-top:48px!important}.mr-lg-6,.mx-lg-6{margin-right:48px!important}.mb-lg-6,.my-lg-6{margin-bottom:48px!important}.ml-lg-6,.mx-lg-6{margin-left:48px!important}.m-lg-7{margin:56px!important}.mt-lg-7,.my-lg-7{margin-top:56px!important}.mr-lg-7,.mx-lg-7{margin-right:56px!important}.mb-lg-7,.my-lg-7{margin-bottom:56px!important}.ml-lg-7,.mx-lg-7{margin-left:56px!important}.m-lg-8{margin:64px!important}.mt-lg-8,.my-lg-8{margin-top:64px!important}.mr-lg-8,.mx-lg-8{margin-right:64px!important}.mb-lg-8,.my-lg-8{margin-bottom:64px!important}.ml-lg-8,.mx-lg-8{margin-left:64px!important}.m-lg-9{margin:72px!important}.mt-lg-9,.my-lg-9{margin-top:72px!important}.mr-lg-9,.mx-lg-9{margin-right:72px!important}.mb-lg-9,.my-lg-9{margin-bottom:72px!important}.ml-lg-9,.mx-lg-9{margin-left:72px!important}.m-lg-10{margin:80px!important}.mt-lg-10,.my-lg-10{margin-top:80px!important}.mr-lg-10,.mx-lg-10{margin-right:80px!important}.mb-lg-10,.my-lg-10{margin-bottom:80px!important}.ml-lg-10,.mx-lg-10{margin-left:80px!important}.m-lg-12{margin:96px!important}.mt-lg-12,.my-lg-12{margin-top:96px!important}.mr-lg-12,.mx-lg-12{margin-right:96px!important}.mb-lg-12,.my-lg-12{margin-bottom:96px!important}.ml-lg-12,.mx-lg-12{margin-left:96px!important}.m-lg-15{margin:120px!important}.mt-lg-15,.my-lg-15{margin-top:120px!important}.mr-lg-15,.mx-lg-15{margin-right:120px!important}.mb-lg-15,.my-lg-15{margin-bottom:120px!important}.ml-lg-15,.mx-lg-15{margin-left:120px!important}.p-lg-0{padding:0!important}.pt-lg-0,.py-lg-0{padding-top:0!important}.pr-lg-0,.px-lg-0{padding-right:0!important}.pb-lg-0,.py-lg-0{padding-bottom:0!important}.pl-lg-0,.px-lg-0{padding-left:0!important}.p-lg-1{padding:8px!important}.pt-lg-1,.py-lg-1{padding-top:8px!important}.pr-lg-1,.px-lg-1{padding-right:8px!important}.pb-lg-1,.py-lg-1{padding-bottom:8px!important}.pl-lg-1,.px-lg-1{padding-left:8px!important}.p-lg-2{padding:16px!important}.pt-lg-2,.py-lg-2{padding-top:16px!important}.pr-lg-2,.px-lg-2{padding-right:16px!important}.pb-lg-2,.py-lg-2{padding-bottom:16px!important}.pl-lg-2,.px-lg-2{padding-left:16px!important}.p-lg-3{padding:24px!important}.pt-lg-3,.py-lg-3{padding-top:24px!important}.pr-lg-3,.px-lg-3{padding-right:24px!important}.pb-lg-3,.py-lg-3{padding-bottom:24px!important}.pl-lg-3,.px-lg-3{padding-left:24px!important}.p-lg-4{padding:32px!important}.pt-lg-4,.py-lg-4{padding-top:32px!important}.pr-lg-4,.px-lg-4{padding-right:32px!important}.pb-lg-4,.py-lg-4{padding-bottom:32px!important}.pl-lg-4,.px-lg-4{padding-left:32px!important}.p-lg-5{padding:40px!important}.pt-lg-5,.py-lg-5{padding-top:40px!important}.pr-lg-5,.px-lg-5{padding-right:40px!important}.pb-lg-5,.py-lg-5{padding-bottom:40px!important}.pl-lg-5,.px-lg-5{padding-left:40px!important}.p-lg-6{padding:48px!important}.pt-lg-6,.py-lg-6{padding-top:48px!important}.pr-lg-6,.px-lg-6{padding-right:48px!important}.pb-lg-6,.py-lg-6{padding-bottom:48px!important}.pl-lg-6,.px-lg-6{padding-left:48px!important}.p-lg-7{padding:56px!important}.pt-lg-7,.py-lg-7{padding-top:56px!important}.pr-lg-7,.px-lg-7{padding-right:56px!important}.pb-lg-7,.py-lg-7{padding-bottom:56px!important}.pl-lg-7,.px-lg-7{padding-left:56px!important}.p-lg-8{padding:64px!important}.pt-lg-8,.py-lg-8{padding-top:64px!important}.pr-lg-8,.px-lg-8{padding-right:64px!important}.pb-lg-8,.py-lg-8{padding-bottom:64px!important}.pl-lg-8,.px-lg-8{padding-left:64px!important}.p-lg-9{padding:72px!important}.pt-lg-9,.py-lg-9{padding-top:72px!important}.pr-lg-9,.px-lg-9{padding-right:72px!important}.pb-lg-9,.py-lg-9{padding-bottom:72px!important}.pl-lg-9,.px-lg-9{padding-left:72px!important}.p-lg-10{padding:80px!important}.pt-lg-10,.py-lg-10{padding-top:80px!important}.pr-lg-10,.px-lg-10{padding-right:80px!important}.pb-lg-10,.py-lg-10{padding-bottom:80px!important}.pl-lg-10,.px-lg-10{padding-left:80px!important}.p-lg-12{padding:96px!important}.pt-lg-12,.py-lg-12{padding-top:96px!important}.pr-lg-12,.px-lg-12{padding-right:96px!important}.pb-lg-12,.py-lg-12{padding-bottom:96px!important}.pl-lg-12,.px-lg-12{padding-left:96px!important}.p-lg-15{padding:120px!important}.pt-lg-15,.py-lg-15{padding-top:120px!important}.pr-lg-15,.px-lg-15{padding-right:120px!important}.pb-lg-15,.py-lg-15{padding-bottom:120px!important}.pl-lg-15,.px-lg-15{padding-left:120px!important}.m-lg-n1{margin:-8px!important}.mt-lg-n1,.my-lg-n1{margin-top:-8px!important}.mr-lg-n1,.mx-lg-n1{margin-right:-8px!important}.mb-lg-n1,.my-lg-n1{margin-bottom:-8px!important}.ml-lg-n1,.mx-lg-n1{margin-left:-8px!important}.m-lg-n2{margin:-16px!important}.mt-lg-n2,.my-lg-n2{margin-top:-16px!important}.mr-lg-n2,.mx-lg-n2{margin-right:-16px!important}.mb-lg-n2,.my-lg-n2{margin-bottom:-16px!important}.ml-lg-n2,.mx-lg-n2{margin-left:-16px!important}.m-lg-n3{margin:-24px!important}.mt-lg-n3,.my-lg-n3{margin-top:-24px!important}.mr-lg-n3,.mx-lg-n3{margin-right:-24px!important}.mb-lg-n3,.my-lg-n3{margin-bottom:-24px!important}.ml-lg-n3,.mx-lg-n3{margin-left:-24px!important}.m-lg-n4{margin:-32px!important}.mt-lg-n4,.my-lg-n4{margin-top:-32px!important}.mr-lg-n4,.mx-lg-n4{margin-right:-32px!important}.mb-lg-n4,.my-lg-n4{margin-bottom:-32px!important}.ml-lg-n4,.mx-lg-n4{margin-left:-32px!important}.m-lg-n5{margin:-40px!important}.mt-lg-n5,.my-lg-n5{margin-top:-40px!important}.mr-lg-n5,.mx-lg-n5{margin-right:-40px!important}.mb-lg-n5,.my-lg-n5{margin-bottom:-40px!important}.ml-lg-n5,.mx-lg-n5{margin-left:-40px!important}.m-lg-n6{margin:-48px!important}.mt-lg-n6,.my-lg-n6{margin-top:-48px!important}.mr-lg-n6,.mx-lg-n6{margin-right:-48px!important}.mb-lg-n6,.my-lg-n6{margin-bottom:-48px!important}.ml-lg-n6,.mx-lg-n6{margin-left:-48px!important}.m-lg-n7{margin:-56px!important}.mt-lg-n7,.my-lg-n7{margin-top:-56px!important}.mr-lg-n7,.mx-lg-n7{margin-right:-56px!important}.mb-lg-n7,.my-lg-n7{margin-bottom:-56px!important}.ml-lg-n7,.mx-lg-n7{margin-left:-56px!important}.m-lg-n8{margin:-64px!important}.mt-lg-n8,.my-lg-n8{margin-top:-64px!important}.mr-lg-n8,.mx-lg-n8{margin-right:-64px!important}.mb-lg-n8,.my-lg-n8{margin-bottom:-64px!important}.ml-lg-n8,.mx-lg-n8{margin-left:-64px!important}.m-lg-n9{margin:-72px!important}.mt-lg-n9,.my-lg-n9{margin-top:-72px!important}.mr-lg-n9,.mx-lg-n9{margin-right:-72px!important}.mb-lg-n9,.my-lg-n9{margin-bottom:-72px!important}.ml-lg-n9,.mx-lg-n9{margin-left:-72px!important}.m-lg-n10{margin:-80px!important}.mt-lg-n10,.my-lg-n10{margin-top:-80px!important}.mr-lg-n10,.mx-lg-n10{margin-right:-80px!important}.mb-lg-n10,.my-lg-n10{margin-bottom:-80px!important}.ml-lg-n10,.mx-lg-n10{margin-left:-80px!important}.m-lg-n12{margin:-96px!important}.mt-lg-n12,.my-lg-n12{margin-top:-96px!important}.mr-lg-n12,.mx-lg-n12{margin-right:-96px!important}.mb-lg-n12,.my-lg-n12{margin-bottom:-96px!important}.ml-lg-n12,.mx-lg-n12{margin-left:-96px!important}.m-lg-n15{margin:-120px!important}.mt-lg-n15,.my-lg-n15{margin-top:-120px!important}.mr-lg-n15,.mx-lg-n15{margin-right:-120px!important}.mb-lg-n15,.my-lg-n15{margin-bottom:-120px!important}.ml-lg-n15,.mx-lg-n15{margin-left:-120px!important}.m-lg-auto{margin:auto!important}.mt-lg-auto,.my-lg-auto{margin-top:auto!important}.mr-lg-auto,.mx-lg-auto{margin-right:auto!important}.mb-lg-auto,.my-lg-auto{margin-bottom:auto!important}.ml-lg-auto,.mx-lg-auto{margin-left:auto!important}}@media(min-width:1240px){.m-xl-0{margin:0!important}.mt-xl-0,.my-xl-0{margin-top:0!important}.mr-xl-0,.mx-xl-0{margin-right:0!important}.mb-xl-0,.my-xl-0{margin-bottom:0!important}.ml-xl-0,.mx-xl-0{margin-left:0!important}.m-xl-1{margin:8px!important}.mt-xl-1,.my-xl-1{margin-top:8px!important}.mr-xl-1,.mx-xl-1{margin-right:8px!important}.mb-xl-1,.my-xl-1{margin-bottom:8px!important}.ml-xl-1,.mx-xl-1{margin-left:8px!important}.m-xl-2{margin:16px!important}.mt-xl-2,.my-xl-2{margin-top:16px!important}.mr-xl-2,.mx-xl-2{margin-right:16px!important}.mb-xl-2,.my-xl-2{margin-bottom:16px!important}.ml-xl-2,.mx-xl-2{margin-left:16px!important}.m-xl-3{margin:24px!important}.mt-xl-3,.my-xl-3{margin-top:24px!important}.mr-xl-3,.mx-xl-3{margin-right:24px!important}.mb-xl-3,.my-xl-3{margin-bottom:24px!important}.ml-xl-3,.mx-xl-3{margin-left:24px!important}.m-xl-4{margin:32px!important}.mt-xl-4,.my-xl-4{margin-top:32px!important}.mr-xl-4,.mx-xl-4{margin-right:32px!important}.mb-xl-4,.my-xl-4{margin-bottom:32px!important}.ml-xl-4,.mx-xl-4{margin-left:32px!important}.m-xl-5{margin:40px!important}.mt-xl-5,.my-xl-5{margin-top:40px!important}.mr-xl-5,.mx-xl-5{margin-right:40px!important}.mb-xl-5,.my-xl-5{margin-bottom:40px!important}.ml-xl-5,.mx-xl-5{margin-left:40px!important}.m-xl-6{margin:48px!important}.mt-xl-6,.my-xl-6{margin-top:48px!important}.mr-xl-6,.mx-xl-6{margin-right:48px!important}.mb-xl-6,.my-xl-6{margin-bottom:48px!important}.ml-xl-6,.mx-xl-6{margin-left:48px!important}.m-xl-7{margin:56px!important}.mt-xl-7,.my-xl-7{margin-top:56px!important}.mr-xl-7,.mx-xl-7{margin-right:56px!important}.mb-xl-7,.my-xl-7{margin-bottom:56px!important}.ml-xl-7,.mx-xl-7{margin-left:56px!important}.m-xl-8{margin:64px!important}.mt-xl-8,.my-xl-8{margin-top:64px!important}.mr-xl-8,.mx-xl-8{margin-right:64px!important}.mb-xl-8,.my-xl-8{margin-bottom:64px!important}.ml-xl-8,.mx-xl-8{margin-left:64px!important}.m-xl-9{margin:72px!important}.mt-xl-9,.my-xl-9{margin-top:72px!important}.mr-xl-9,.mx-xl-9{margin-right:72px!important}.mb-xl-9,.my-xl-9{margin-bottom:72px!important}.ml-xl-9,.mx-xl-9{margin-left:72px!important}.m-xl-10{margin:80px!important}.mt-xl-10,.my-xl-10{margin-top:80px!important}.mr-xl-10,.mx-xl-10{margin-right:80px!important}.mb-xl-10,.my-xl-10{margin-bottom:80px!important}.ml-xl-10,.mx-xl-10{margin-left:80px!important}.m-xl-12{margin:96px!important}.mt-xl-12,.my-xl-12{margin-top:96px!important}.mr-xl-12,.mx-xl-12{margin-right:96px!important}.mb-xl-12,.my-xl-12{margin-bottom:96px!important}.ml-xl-12,.mx-xl-12{margin-left:96px!important}.m-xl-15{margin:120px!important}.mt-xl-15,.my-xl-15{margin-top:120px!important}.mr-xl-15,.mx-xl-15{margin-right:120px!important}.mb-xl-15,.my-xl-15{margin-bottom:120px!important}.ml-xl-15,.mx-xl-15{margin-left:120px!important}.p-xl-0{padding:0!important}.pt-xl-0,.py-xl-0{padding-top:0!important}.pr-xl-0,.px-xl-0{padding-right:0!important}.pb-xl-0,.py-xl-0{padding-bottom:0!important}.pl-xl-0,.px-xl-0{padding-left:0!important}.p-xl-1{padding:8px!important}.pt-xl-1,.py-xl-1{padding-top:8px!important}.pr-xl-1,.px-xl-1{padding-right:8px!important}.pb-xl-1,.py-xl-1{padding-bottom:8px!important}.pl-xl-1,.px-xl-1{padding-left:8px!important}.p-xl-2{padding:16px!important}.pt-xl-2,.py-xl-2{padding-top:16px!important}.pr-xl-2,.px-xl-2{padding-right:16px!important}.pb-xl-2,.py-xl-2{padding-bottom:16px!important}.pl-xl-2,.px-xl-2{padding-left:16px!important}.p-xl-3{padding:24px!important}.pt-xl-3,.py-xl-3{padding-top:24px!important}.pr-xl-3,.px-xl-3{padding-right:24px!important}.pb-xl-3,.py-xl-3{padding-bottom:24px!important}.pl-xl-3,.px-xl-3{padding-left:24px!important}.p-xl-4{padding:32px!important}.pt-xl-4,.py-xl-4{padding-top:32px!important}.pr-xl-4,.px-xl-4{padding-right:32px!important}.pb-xl-4,.py-xl-4{padding-bottom:32px!important}.pl-xl-4,.px-xl-4{padding-left:32px!important}.p-xl-5{padding:40px!important}.pt-xl-5,.py-xl-5{padding-top:40px!important}.pr-xl-5,.px-xl-5{padding-right:40px!important}.pb-xl-5,.py-xl-5{padding-bottom:40px!important}.pl-xl-5,.px-xl-5{padding-left:40px!important}.p-xl-6{padding:48px!important}.pt-xl-6,.py-xl-6{padding-top:48px!important}.pr-xl-6,.px-xl-6{padding-right:48px!important}.pb-xl-6,.py-xl-6{padding-bottom:48px!important}.pl-xl-6,.px-xl-6{padding-left:48px!important}.p-xl-7{padding:56px!important}.pt-xl-7,.py-xl-7{padding-top:56px!important}.pr-xl-7,.px-xl-7{padding-right:56px!important}.pb-xl-7,.py-xl-7{padding-bottom:56px!important}.pl-xl-7,.px-xl-7{padding-left:56px!important}.p-xl-8{padding:64px!important}.pt-xl-8,.py-xl-8{padding-top:64px!important}.pr-xl-8,.px-xl-8{padding-right:64px!important}.pb-xl-8,.py-xl-8{padding-bottom:64px!important}.pl-xl-8,.px-xl-8{padding-left:64px!important}.p-xl-9{padding:72px!important}.pt-xl-9,.py-xl-9{padding-top:72px!important}.pr-xl-9,.px-xl-9{padding-right:72px!important}.pb-xl-9,.py-xl-9{padding-bottom:72px!important}.pl-xl-9,.px-xl-9{padding-left:72px!important}.p-xl-10{padding:80px!important}.pt-xl-10,.py-xl-10{padding-top:80px!important}.pr-xl-10,.px-xl-10{padding-right:80px!important}.pb-xl-10,.py-xl-10{padding-bottom:80px!important}.pl-xl-10,.px-xl-10{padding-left:80px!important}.p-xl-12{padding:96px!important}.pt-xl-12,.py-xl-12{padding-top:96px!important}.pr-xl-12,.px-xl-12{padding-right:96px!important}.pb-xl-12,.py-xl-12{padding-bottom:96px!important}.pl-xl-12,.px-xl-12{padding-left:96px!important}.p-xl-15{padding:120px!important}.pt-xl-15,.py-xl-15{padding-top:120px!important}.pr-xl-15,.px-xl-15{padding-right:120px!important}.pb-xl-15,.py-xl-15{padding-bottom:120px!important}.pl-xl-15,.px-xl-15{padding-left:120px!important}.m-xl-n1{margin:-8px!important}.mt-xl-n1,.my-xl-n1{margin-top:-8px!important}.mr-xl-n1,.mx-xl-n1{margin-right:-8px!important}.mb-xl-n1,.my-xl-n1{margin-bottom:-8px!important}.ml-xl-n1,.mx-xl-n1{margin-left:-8px!important}.m-xl-n2{margin:-16px!important}.mt-xl-n2,.my-xl-n2{margin-top:-16px!important}.mr-xl-n2,.mx-xl-n2{margin-right:-16px!important}.mb-xl-n2,.my-xl-n2{margin-bottom:-16px!important}.ml-xl-n2,.mx-xl-n2{margin-left:-16px!important}.m-xl-n3{margin:-24px!important}.mt-xl-n3,.my-xl-n3{margin-top:-24px!important}.mr-xl-n3,.mx-xl-n3{margin-right:-24px!important}.mb-xl-n3,.my-xl-n3{margin-bottom:-24px!important}.ml-xl-n3,.mx-xl-n3{margin-left:-24px!important}.m-xl-n4{margin:-32px!important}.mt-xl-n4,.my-xl-n4{margin-top:-32px!important}.mr-xl-n4,.mx-xl-n4{margin-right:-32px!important}.mb-xl-n4,.my-xl-n4{margin-bottom:-32px!important}.ml-xl-n4,.mx-xl-n4{margin-left:-32px!important}.m-xl-n5{margin:-40px!important}.mt-xl-n5,.my-xl-n5{margin-top:-40px!important}.mr-xl-n5,.mx-xl-n5{margin-right:-40px!important}.mb-xl-n5,.my-xl-n5{margin-bottom:-40px!important}.ml-xl-n5,.mx-xl-n5{margin-left:-40px!important}.m-xl-n6{margin:-48px!important}.mt-xl-n6,.my-xl-n6{margin-top:-48px!important}.mr-xl-n6,.mx-xl-n6{margin-right:-48px!important}.mb-xl-n6,.my-xl-n6{margin-bottom:-48px!important}.ml-xl-n6,.mx-xl-n6{margin-left:-48px!important}.m-xl-n7{margin:-56px!important}.mt-xl-n7,.my-xl-n7{margin-top:-56px!important}.mr-xl-n7,.mx-xl-n7{margin-right:-56px!important}.mb-xl-n7,.my-xl-n7{margin-bottom:-56px!important}.ml-xl-n7,.mx-xl-n7{margin-left:-56px!important}.m-xl-n8{margin:-64px!important}.mt-xl-n8,.my-xl-n8{margin-top:-64px!important}.mr-xl-n8,.mx-xl-n8{margin-right:-64px!important}.mb-xl-n8,.my-xl-n8{margin-bottom:-64px!important}.ml-xl-n8,.mx-xl-n8{margin-left:-64px!important}.m-xl-n9{margin:-72px!important}.mt-xl-n9,.my-xl-n9{margin-top:-72px!important}.mr-xl-n9,.mx-xl-n9{margin-right:-72px!important}.mb-xl-n9,.my-xl-n9{margin-bottom:-72px!important}.ml-xl-n9,.mx-xl-n9{margin-left:-72px!important}.m-xl-n10{margin:-80px!important}.mt-xl-n10,.my-xl-n10{margin-top:-80px!important}.mr-xl-n10,.mx-xl-n10{margin-right:-80px!important}.mb-xl-n10,.my-xl-n10{margin-bottom:-80px!important}.ml-xl-n10,.mx-xl-n10{margin-left:-80px!important}.m-xl-n12{margin:-96px!important}.mt-xl-n12,.my-xl-n12{margin-top:-96px!important}.mr-xl-n12,.mx-xl-n12{margin-right:-96px!important}.mb-xl-n12,.my-xl-n12{margin-bottom:-96px!important}.ml-xl-n12,.mx-xl-n12{margin-left:-96px!important}.m-xl-n15{margin:-120px!important}.mt-xl-n15,.my-xl-n15{margin-top:-120px!important}.mr-xl-n15,.mx-xl-n15{margin-right:-120px!important}.mb-xl-n15,.my-xl-n15{margin-bottom:-120px!important}.ml-xl-n15,.mx-xl-n15{margin-left:-120px!important}.m-xl-auto{margin:auto!important}.mt-xl-auto,.my-xl-auto{margin-top:auto!important}.mr-xl-auto,.mx-xl-auto{margin-right:auto!important}.mb-xl-auto,.my-xl-auto{margin-bottom:auto!important}.ml-xl-auto,.mx-xl-auto{margin-left:auto!important}}.text-monospace{font-family:SFMono-Regular,Menlo,Monaco,Consolas,Liberation Mono,Courier New,monospace!important}.text-justify{text-align:justify!important}.text-wrap{white-space:normal!important}.text-nowrap{white-space:nowrap!important}.text-truncate{overflow:hidden;text-overflow:ellipsis;white-space:nowrap}.text-left{text-align:left!important}.text-right{text-align:right!important}.text-center{text-align:center!important}@media(min-width:400px){.text-xs-left{text-align:left!important}.text-xs-right{text-align:right!important}.text-xs-center{text-align:center!important}}@media(min-width:616px){.text-sm-left{text-align:left!important}.text-sm-right{text-align:right!important}.text-sm-center{text-align:center!important}}@media(min-width:768px){.text-md-left{text-align:left!important}.text-md-right{text-align:right!important}.text-md-center{text-align:center!important}}@media(min-width:980px){.text-lg-left{text-align:left!important}.text-lg-right{text-align:right!important}.text-lg-center{text-align:center!important}}@media(min-width:1240px){.text-xl-left{text-align:left!important}.text-xl-right{text-align:right!important}.text-xl-center{text-align:center!important}}.text-lowercase{text-transform:lowercase!important}.text-uppercase{text-transform:uppercase!important}.text-capitalize{text-transform:capitalize!important}.font-weight-light{font-weight:300!important}.font-weight-lighter{font-weight:lighter!important}.font-weight-normal{font-weight:400!important}.font-weight-bold{font-weight:700!important}.font-weight-bolder{font-weight:bolder!important}.font-italic{font-style:italic!important}.text-primary{color:#fc0!important}a.text-primary:focus,a.text-primary:hover{color:#b38f00!important}.text-secondary{color:#212529!important}a.text-secondary:focus,a.text-secondary:hover{color:#000!important}.text-success{color:#28a745!important}a.text-success:focus,a.text-success:hover{color:#19692c!important}.text-info{color:#17a2b8!important}a.text-info:focus,a.text-info:hover{color:#0f6674!important}.text-warning{color:#ffc107!important}a.text-warning:focus,a.text-warning:hover{color:#ba8b00!important}.text-danger{color:#dc3545!important}a.text-danger:focus,a.text-danger:hover{color:#a71d2a!important}.text-light{color:#f1f6f9!important}a.text-light:focus,a.text-light:hover{color:#bbd4e2!important}.text-dark{color:#495057!important}a.text-dark:focus,a.text-dark:hover{color:#262a2d!important}.text-primary-light{color:#fffaf0!important}a.text-primary-light:focus,a.text-primary-light:hover{color:#ffe1a4!important}.text-secondary-light{color:#fff!important}a.text-secondary-light:focus,a.text-secondary-light:hover{color:#d9d9d9!important}.text-tertiary{color:#257af4!important}a.text-tertiary:focus,a.text-tertiary:hover{color:#0a56c3!important}.text-tertiary-light{color:#e3f1fe!important}a.text-tertiary-light:focus,a.text-tertiary-light:hover{color:#99ccfb!important}.text-white{color:#fff!important}a.text-white:focus,a.text-white:hover{color:#d9d9d9!important}.text-black{color:#212529!important}a.text-black:focus,a.text-black:hover{color:#000!important}.text-blue{color:#257af4!important}a.text-blue:focus,a.text-blue:hover{color:#0a56c3!important}.text-light-blue{color:#e3f1fe!important}a.text-light-blue:focus,a.text-light-blue:hover{color:#99ccfb!important}.text-yellow{color:#fc0!important}a.text-yellow:focus,a.text-yellow:hover{color:#b38f00!important}.text-light-yellow{color:#fffaf0!important}a.text-light-yellow:focus,a.text-light-yellow:hover{color:#ffe1a4!important}.text-orange{color:#ff8c00!important}a.text-orange:focus,a.text-orange:hover{color:#b36200!important}.text-light-orange{color:#ffe4b5!important}a.text-light-orange:focus,a.text-light-orange:hover{color:#ffc869!important}.text-red{color:#ff3939!important}a.text-red:focus,a.text-red:hover{color:#ec0000!important}.text-light-red{color:#ffe4e1!important}a.text-light-red:focus,a.text-light-red:hover{color:#ff9f95!important}.text-medium{color:#d6dbdf!important}a.text-medium:focus,a.text-medium:hover{color:#abb5bd!important}.text-body{color:#212529!important}.text-muted{color:#6c757d!important}.text-black-50{color:rgba(33,37,41,.5)!important}.text-white-50{color:hsla(0,0%,100%,.5)!important}.text-hide{font:0/0 a;color:transparent;text-shadow:none;background-color:transparent;border:0}.text-decoration-none{text-decoration:none!important}.text-break{word-break:break-word!important;overflow-wrap:break-word!important}.text-reset{color:inherit!important}.visible{visibility:visible!important}.invisible{visibility:hidden!important}@media print{*,:after,:before{text-shadow:none!important;box-shadow:none!important}a:not(.btn){text-decoration:underline}abbr[title]:after{content:" (" attr(title) ")"}pre{white-space:pre-wrap!important}blockquote,pre{border:1px solid #d6dbdf;page-break-inside:avoid}thead{display:table-header-group}img,tr{page-break-inside:avoid}h2,h3,p{orphans:3;widows:3}h2,h3{page-break-after:avoid}@page{size:a3}.container,body{min-width:980px!important}.navbar{display:none}.badge{border:1px solid #212529}.table{border-collapse:collapse!important}.table td,.table th{background-color:#fff!important}.table-bordered td,.table-bordered th{border:1px solid #dee2e6!important}.table-dark{color:inherit}.table-dark tbody+tbody,.table-dark td,.table-dark th,.table-dark thead th{border-color:#d6dbdf}.table .thead-dark th{color:inherit;border-color:#d6dbdf}} \ No newline at end of file diff --git a/website/css/main.css b/website/css/main.css index 614422689a9..a505beb20bb 100644 --- a/website/css/main.css +++ b/website/css/main.css @@ -1 +1 @@ -@media screen and (max-width:978.98px){.btn{padding:8px 16px}}@media screen and (max-width:978.98px){.btn-lg{padding:12px 24px}}.btn-primary,.btn-primary:active,.btn-primary:hover{color:#212529}.btn-outline-primary{background:#fffaf0;border-color:#fc0;color:#212529}.btn-outline-primary:active,.btn-outline-primary:hover{background:#fc0;border-color:#fc0;color:#212529}.btn-secondary{border-color:#212529;color:#fff}.btn-outline-secondary,.btn-secondary:active,.btn-secondary:hover{background:#fff;border-color:#212529;color:#212529}.btn-outline-secondary:active,.btn-outline-secondary:hover{background:#212529;border-color:#212529;color:#fff}.btn-tertiary{border-color:#257af4;color:#fff}.btn-tertiary:active,.btn-tertiary:hover{background:#257af4;border-color:#257af4;color:#fff}.btn-outline-tertiary{background:#e3f1fe;color:#257af4}.btn-outline-tertiary:active,.btn-outline-tertiary:hover{background:#257af4;color:#fff}.btns{align-items:center;display:grid;-moz-column-gap:24px;column-gap:24px;row-gap:16px;grid-auto-flow:column;justify-content:center}@media screen and (max-width:767.98px){.btns{grid-auto-flow:row}}.btns.btns-lg{-moz-column-gap:40px;column-gap:40px}.btns.is-2{grid-template-columns:1fr 1fr}@media screen and (max-width:767.98px){.btns.is-2{grid-template-columns:1fr}}.btns.is-3{grid-template-columns:1fr 1fr 1fr}@media screen and (max-width:767.98px){.btns.is-3{grid-template-columns:1fr}}.card{box-shadow:0 8px 20px rgba(108,117,125,.2);overflow:hidden;transition:box-shadow .2s,transform .2s;width:100%}.card,.card-body{position:relative}.card-body{z-index:10}.card.is-large .card-body{padding:40px}.card.bg-primary-light{border-color:#fc0}.card.has-dark-border{border-color:#6c757d}.card.has-pattern:after,.card.has-pattern:before{background-repeat:no-repeat;background-size:auto 100%;bottom:0;content:"";display:block;position:absolute;top:0;width:72px}.card.has-pattern:before{background-image:url(../images/backgrounds/bg-card-pattern-blue-1.png);background-position:0 0;left:0}.card.has-pattern:after{background-image:url(../images/backgrounds/bg-card-pattern-blue-2.png);background-position:100% 0;right:0}.card.has-hover:active,.card.has-hover:hover,a.card:active,a.card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2);transform:translateY(-8px)}.card.has-highlight:after,.card.has-hover:after,a.card:after{content:"";display:block;height:8px;margin-top:auto;transition:background .2s;width:100%}.card.has-highlight:after,.card.has-hover:active:after,.card.has-hover:hover:after,a.card:active:after,a.card:hover:after{background:#e3f1fe}.case-study-cards{-moz-column-gap:40px;column-gap:40px;display:grid;grid-template-columns:1fr;row-gap:40px;padding-bottom:40px;position:relative}.case-study-cards>div{align-items:stretch;display:flex}.case-study-cards:before{background:#d6dbdf;bottom:0;content:"";display:block;left:20px;position:absolute;top:40px;width:100vw}@media screen and (min-width:980px){.case-study-cards{grid-template-columns:repeat(2,minmax(0,1fr));row-gap:80px;padding-bottom:120px}.case-study-cards:before{left:-40px;top:120px}}.case-study-card{align-items:stretch;flex-direction:row;flex-shrink:0;left:0;transition:box-shadow .2s,left .4s,width .4s,z-index 0s;transition-delay:0s,.6s,.6s,0s;width:100%;z-index:2}@media screen and (max-width:979.98px){.case-study-card .row{min-height:0!important}}@media screen and (min-width:980px){.case-study-card:active,.case-study-card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2)}.case-study-card:not(.is-open){cursor:pointer}.case-study-card.is-open{transform:none!important;transition-delay:0s,0s,0s,0s;width:calc(200% + 40px);z-index:10}.case-study-card.is-closing{z-index:10}.case-study-card.open-left.is-open{left:calc(-100% - 40px)}.case-study-card:before{background:no-repeat url(../images/backgrounds/bg-card-pattern-red.png);background-position:100%;background-size:contain;content:"";display:block;height:calc(100% - 80px);max-height:224px;max-width:234px;position:absolute;right:0;top:40px;transform:translateX(30%);transition:transform .4s;transition-delay:.6s;width:100%;z-index:1}}@media screen and (min-width:980px)and (min-width:1240px){.case-study-card:before{transform:translateX(50%)}}@media screen and (min-width:980px){.case-study-card.is-open:before{transform:translateX(70%);transition-delay:0s}}@media screen and (min-width:980px){.case-study-card-wrap{align-items:stretch;display:flex;flex-shrink:0;min-height:304px;position:relative;transition:width .4s;transition-delay:.6s;width:calc(200% + 42px);z-index:2}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-wrap{transition-delay:0s;width:100%}}@media screen and (min-width:980px){.case-study-card-body{display:flex;flex-direction:column;padding-right:80px!important}.case-study-card-body>.row{align-self:stretch}}@media screen and (min-width:980px){.case-study-card-toggle{background:#fff;box-shadow:0 8px 20px rgba(108,117,125,.2);border-radius:100%;cursor:pointer;height:56px;position:relative;width:56px}.case-study-card-toggle:after,.case-study-card-toggle:before{background:#257af4;content:"";display:block;height:4px;left:calc(50% - 15px);position:absolute;top:calc(50% - 2px);transition:opacity .2s,transform .2s;width:30px}.case-study-card-toggle:after{transform:rotate(90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:before{opacity:0;transform:rotate(-90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:after{transform:rotate(0)}}@media screen and (min-width:980px){.case-study-card .col-lg-3{left:-60%;position:relative;transition:left .4s;transition-delay:.6s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-3{flex:0 0 250px;max-width:250px;width:250px}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-3{left:0;transition-delay:0s}}@media screen and (min-width:980px){.case-study-card .col-lg-auto{opacity:0;transform:translateX(24px);transition:opacity .4s,transform .4s;transition-delay:.2s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-auto{max-width:605px;width:calc(100% - 319px)}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-auto{opacity:1;transform:none;transition-delay:.2s}}.footer-copy{white-space:nowrap}form .form-control{border:1px solid #6c757d;border-radius:6px;height:auto;line-height:20px;min-height:44px;padding:12px 16px;width:100%}form .form-control,form .form-control:focus{box-shadow:0 8px 20px rgba(108,117,125,.2);color:#212529}form .form-control:focus{border-color:#212529}form .form-control::-moz-placeholder{color:#6c757d}form .form-control:-ms-input-placeholder{color:#6c757d}form .form-control::placeholder{color:#6c757d}form select.form-control{-webkit-appearance:none;-moz-appearance:none;appearance:none}form select.form-control:not([data-chosen]){color:#6c757d}form .btn-secondary:active,form .btn-secondary:hover{color:#212529;background:#fc0;border-color:#fc0}.hero{overflow:visible;position:relative}.hero,.hero-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.hero-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.hero>.container{position:relative;z-index:2}.hero.has-offset{margin-bottom:-160px;padding-bottom:160px}.base-hero{height:22.5vw;max-height:324px;min-height:280px}.index-hero{background-image:url(/images/backgrounds/bg-hero-home.svg);height:68vw;max-height:980px}.index-hero,.other-hero{max-width:2448px;width:170vw}.other-hero{background-image:url(/images/backgrounds/bg-hero.svg)}.bg-footer-cta{background-image:url(/images/backgrounds/bg-footer-cta.svg);width:2448px}.quickstart-bg{background-image:url(/images/backgrounds/bg-quick-start.svg);height:40vw;top:220px;width:170vw}hr{background:#f1f6f9;border:0;display:block;height:4px;margin:0;width:100%}hr.is-small{height:2px}hr.is-large{height:8px}hr.is-medium{background:#d6dbdf}hr.is-dark{background:#495057}hr.is-yellow{background:linear-gradient(90deg,#ff8c00,#ff8c00 8px,#fc0 16px,rgba(255,204,0,0));-webkit-clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);height:8px}.icon{display:block;height:48px;margin-bottom:24px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center}@media screen and (min-width:576px){.icon{height:64px}}@media screen and (min-width:980px){.icon{height:80px}}img{max-width:100%}.kicker{color:#6c757d;font-family:Hind Siliguri,sans-serif;font-size:.875rem;font-weight:600;letter-spacing:1px;margin:0}@media screen and (max-width:978.98px){.lead{font-size:1.125rem}}.logo{display:block;height:36px;max-width:220px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;width:100%}.navbar-clickhouse{border-bottom:4px solid #f1f6f9;height:142px}.navbar-clickhouse>.container{flex-wrap:wrap}.navbar-super{flex-shrink:0;width:100%}.navbar-super ul{list-style:none}.navbar-super li:not(:last-child){margin-bottom:0;margin-right:24px}.navbar-super a{align-items:center;color:#212529;display:flex;font-size:.875rem}.navbar-super a:active,.navbar-super a:hover{color:#257af4;text-decoration:none}.navbar-super img{flex-shrink:0;margin-right:4px}.navbar-brand-clickhouse{background:no-repeat url(../images/logo-clickhouse.svg);background-size:contain;flex-shrink:0;height:28px;margin-right:48px;padding:0;width:180px}.navbar-nav{align-items:center;height:46px}.navbar .nav-item:not(:last-child){margin-bottom:0;margin-right:24px}.navbar .nav-link{color:#212529}.navbar .nav-link:active,.navbar .nav-link:hover{color:#257af4}.navbar .navbar-nav{flex-direction:row}@media screen and (max-width:978.98px){.navbar>.container{padding-left:20px;padding-right:20px}.navbar .navbar-toggler{height:24px;padding:0;width:24px}.navbar .navbar-toggler:focus{outline:none}.navbar .navbar-toggler-icon{background:no-repeat url(../images/icons/icon-menu.svg);background-position:50%;background-size:contain;height:24px;width:24px}.navbar .navbar-collapse{background:#fff;border-bottom:4px solid #f1f6f9;height:56px;left:0;padding:0 20px 16px;position:absolute;right:0;top:100%}.navbar .nav-link{font-size:.875rem;white-space:nowrap}}@media screen and (max-width:615.98px){.navbar .navbar-collapse{height:auto}.navbar .navbar-nav{flex-direction:column;height:auto}.navbar .nav-item:not(:last-child){margin-bottom:16px;margin-right:0}}@media screen and (max-width:399.98px){.navbar{height:80px}}.page,.photo-frame{overflow:hidden;width:100%}.photo-frame{background:hsla(0,0%,100%,.6);border-radius:100%;box-shadow:0 8px 20px rgba(108,117,125,.2);display:block;margin-bottom:24px;max-width:160px;position:relative}.photo-frame:before{content:"";display:block;padding-bottom:100%;width:100%}.photo-frame img{display:block;height:100%;left:0;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;position:absolute;top:0;width:100%}.pullquote{position:relative;width:70%}.pullquote:before{background:no-repeat url(../images/backgrounds/bg-quotes.svg);background-position:50%;background-size:100%;content:"";mix-blend-mode:multiply;right:56px;width:calc(100% - 16px);z-index:2}.pullquote-bg,.pullquote:before{bottom:0;display:block;position:absolute;top:0}.pullquote-bg{right:0;width:calc(50vw + 28.57143%);z-index:1}.pullquote-body{padding:64px 40px 64px 0;position:relative;z-index:3}.pullquote-quote{font-family:Hind Siliguri,sans-serif;font-size:32px;font-weight:700}.pullquote-citation{font-size:1.125rem}.section{overflow:visible;position:relative}.section,.section-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.section-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.section>.container{position:relative;z-index:2}.social-icons{align-items:center;display:flex}.social-icons>a{aspect-ratio:24/24;background:#6c757d;display:block;height:24px;width:24px;-webkit-mask-position:center;mask-position:center;-webkit-mask-repeat:no-repeat;mask-repeat:no-repeat;-webkit-mask-size:contain;mask-size:contain;transition:background .2s}.social-icons>a:active,.social-icons>a:hover{background:#212529}.social-icons>a+a{margin-left:32px}.social-icons-facebook{-webkit-mask-image:url(/images/icons/icon-facebook-gray.svg);mask-image:url(/images/icons/icon-facebook-gray.svg)}.social-icons-twitter{-webkit-mask-image:url(/images/icons/icon-twitter-gray.svg);mask-image:url(/images/icons/icon-twitter-gray.svg);width:31px}.social-icons-linkedin{-webkit-mask-image:url(/images/icons/icon-linkedin-gray.svg);mask-image:url(/images/icons/icon-linkedin-gray.svg)}.social-icons-linkedin-alt{-webkit-mask-image:url(/images/icons/icon-linkedin-alt-gray.svg);mask-image:url(/images/icons/icon-linkedin-alt-gray.svg)}.social-icons.size-small>a{height:20px;width:20px}.social-icons.size-small>a:active,.social-icons.size-small>a:hover{background:#212529}.social-icons.size-small>a+a{margin-left:16px}.tabs{position:relative}.tabs:before{background:#fff;border-radius:7px 7px 0 0;content:"";display:block;height:8px;left:1px;position:absolute;right:1px;top:68px;z-index:10}@media screen and (min-width:1240px){.tabs:before{top:76px}}.tabs-body{background:#fff;border-radius:8px;border:1px solid #6c757d;box-shadow:0 8px 20px rgba(108,117,125,.2);padding:24px}@media screen and (min-width:980px){.tabs-body{padding:32px}}@media screen and (min-width:1240px){.tabs-body{padding:40px}}.tabs .nav-tabs{border-bottom:0;flex-wrap:nowrap;height:76px;margin:-20px -20px -9px;-webkit-mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);overflow:scroll;overflow-x:scroll;overflow-y:visible;padding:20px 20px 0;position:relative}@media screen and (min-width:940px){.tabs .nav-tabs{overflow:visible}}@media screen and (min-width:1240px){.tabs .nav-tabs{height:84px}}.tabs .nav-link{align-items:center;border-bottom:0;color:#6c757d;display:flex;font-size:.875rem;flex-shrink:0;height:56px;justify-content:center;padding:0 12px 8px;text-align:center;white-space:nowrap}@media screen and (min-width:1240px){.tabs .nav-link{height:64px;padding:0 16px 8px}}.tabs .nav-link.active{background:#fff;box-shadow:0 -4px 8px rgba(108,117,125,.1);font-weight:700;padding:0 16px 8px}@media screen and (min-width:980px){.tabs .nav-link.active{padding:0 24px 8px}}@media screen and (min-width:1240px){.tabs .nav-link.active{padding:0 32px 8px}}.tab-pane pre{background:#212529;border-radius:16px;color:#fff;padding:24px 16px}@media screen and (min-width:1240px){.tab-pane pre{padding:32px 24px}}.trailing-link{align-items:center;color:#212529;display:flex;font-size:.875rem;font-weight:700}.trailing-link:after{background:no-repeat url(../images/icons/icon-arrow.svg);background-position:100%;background-size:contain;content:"";display:block;height:12px;transition:transform .2s;width:20px}.trailing-link:active,.trailing-link:hover{color:#212529;text-decoration:none}.trailing-link:active:after,.trailing-link:hover:after{transform:translateX(8px)}.trailing-link.span-full:after{margin-left:auto}ul{list-style-type:square;padding-left:1.25em}ul li:not(:last-child){margin-bottom:16px}ul li::marker{color:#ff3939}ul.has-separators{list-style:none;padding:0}ul.has-separators li:not(:last-child){border-bottom:4px solid #f1f6f9;margin-bottom:24px;padding-bottom:24px}.bg-gradient-secondary{background-image:linear-gradient(58deg,#ff6443 3%,#fe561d 24%,#e32f0d 93%)}.bg-gradient-light-orange{background-image:linear-gradient(90deg,rgba(255,203,128,0),#ffcb80)}.bg-offset-right{bottom:0;left:-24px;position:absolute;top:0;width:calc(100vw + 24px);z-index:-1}@media screen and (min-width:1240px){.bg-offset-right{left:-96px;width:calc(100vw + 96px)}}.bg-inset-right{bottom:0;left:40px;position:absolute;top:0;width:calc(100vw - 40px);z-index:-1}@media screen and (min-width:980px){.bg-inset-right{left:96px;width:calc(100vw - 96px)}}.has-border-left{border-left:8px solid #f1f6f9;padding-left:16px}.font-xl{font-size:1.25rem}.font-lg{font-size:1.125rem}.font-sm{font-size:.875rem}.font-xs{font-size:.625rem}.font-weight-semibold{font-weight:600}.display-5{color:#212529;font-size:20px;font-weight:500}.display-6{color:#212529;font-size:14px;font-weight:700}.overflow-auto{overflow:auto}.text-decoration-underline{text-decoration:underline}.text-upper{text-transform:uppercase} +@media screen and (max-width:978.98px){.btn{padding:8px 16px}}@media screen and (max-width:978.98px){.btn-lg{padding:12px 24px}}.btn-primary,.btn-primary:active,.btn-primary:hover{color:#212529}.btn-outline-primary{background:#fffaf0;border-color:#fc0;color:#212529}.btn-outline-primary:active,.btn-outline-primary:hover{background:#fc0;border-color:#fc0;color:#212529}.btn-secondary{border-color:#212529;color:#fff}.btn-outline-secondary,.btn-secondary:active,.btn-secondary:hover{background:#fff;border-color:#212529;color:#212529}.btn-outline-secondary:active,.btn-outline-secondary:hover{background:#212529;border-color:#212529;color:#fff}.btn-tertiary{border-color:#257af4;color:#fff}.btn-tertiary:active,.btn-tertiary:hover{background:#257af4;border-color:#257af4;color:#fff}.btn-outline-tertiary{background:#e3f1fe;color:#257af4}.btn-outline-tertiary:active,.btn-outline-tertiary:hover{background:#257af4;color:#fff}.btns{align-items:center;display:grid;-moz-column-gap:24px;column-gap:24px;row-gap:16px;grid-auto-flow:column;justify-content:center}@media screen and (max-width:767.98px){.btns{grid-auto-flow:row}}.btns.btns-lg{-moz-column-gap:40px;column-gap:40px}.btns.is-2{grid-template-columns:1fr 1fr}@media screen and (max-width:767.98px){.btns.is-2{grid-template-columns:1fr}}.btns.is-3{grid-template-columns:1fr 1fr 1fr}@media screen and (max-width:767.98px){.btns.is-3{grid-template-columns:1fr}}.card{box-shadow:0 8px 20px rgba(108,117,125,.2);overflow:hidden;transition:box-shadow .2s,transform .2s;width:100%}.card,.card-body{position:relative}.card-body{z-index:10}.card.is-large .card-body{padding:40px}.card.bg-primary-light{border-color:#fc0}.card.has-dark-border{border-color:#6c757d}.card.has-pattern:after,.card.has-pattern:before{background-repeat:no-repeat;background-size:auto 100%;bottom:0;content:"";display:block;position:absolute;top:0;width:72px}.card.has-pattern:before{background-image:url(../images/backgrounds/bg-card-pattern-blue-1.png);background-position:0 0;left:0}.card.has-pattern:after{background-image:url(../images/backgrounds/bg-card-pattern-blue-2.png);background-position:100% 0;right:0}.card.has-hover:active,.card.has-hover:hover,a.card:active,a.card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2);transform:translateY(-8px)}.card.has-highlight:after,.card.has-hover:after,a.card:after{content:"";display:block;height:8px;margin-top:auto;transition:background .2s;width:100%}.card.has-highlight:after,.card.has-hover:active:after,.card.has-hover:hover:after,a.card:active:after,a.card:hover:after{background:#e3f1fe}.case-study-cards{-moz-column-gap:40px;column-gap:40px;display:grid;grid-template-columns:1fr;row-gap:40px;padding-bottom:40px;position:relative}.case-study-cards>div{align-items:stretch;display:flex}.case-study-cards:before{background:#d6dbdf;bottom:0;content:"";display:block;left:20px;position:absolute;top:40px;width:100vw}@media screen and (min-width:980px){.case-study-cards{grid-template-columns:repeat(2,minmax(0,1fr));row-gap:80px;padding-bottom:120px}.case-study-cards:before{left:-40px;top:120px}}.case-study-card{align-items:stretch;flex-direction:row;flex-shrink:0;left:0;transition:box-shadow .2s,left .4s,width .4s,z-index 0s;transition-delay:0s,.6s,.6s,0s;width:100%;z-index:2}@media screen and (max-width:979.98px){.case-study-card .row{min-height:0!important}}@media screen and (min-width:980px){.case-study-card:active,.case-study-card:hover{box-shadow:0 12px 32px rgba(108,117,125,.2)}.case-study-card:not(.is-open){cursor:pointer}.case-study-card.is-open{transform:none!important;transition-delay:0s,0s,0s,0s;width:calc(200% + 40px);z-index:10}.case-study-card.is-closing{z-index:10}.case-study-card.open-left.is-open{left:calc(-100% - 40px)}.case-study-card:before{background:no-repeat url(../images/backgrounds/bg-card-pattern-red.png);background-position:100%;background-size:contain;content:"";display:block;height:calc(100% - 80px);max-height:224px;max-width:234px;position:absolute;right:0;top:40px;transform:translateX(30%);transition:transform .4s;transition-delay:.6s;width:100%;z-index:1}}@media screen and (min-width:980px)and (min-width:1240px){.case-study-card:before{transform:translateX(50%)}}@media screen and (min-width:980px){.case-study-card.is-open:before{transform:translateX(70%);transition-delay:0s}}@media screen and (min-width:980px){.case-study-card-wrap{align-items:stretch;display:flex;flex-shrink:0;min-height:304px;position:relative;transition:width .4s;transition-delay:.6s;width:calc(200% + 42px);z-index:2}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-wrap{transition-delay:0s;width:100%}}@media screen and (min-width:980px){.case-study-card-body{display:flex;flex-direction:column;padding-right:80px!important}.case-study-card-body>.row{align-self:stretch}}@media screen and (min-width:980px){.case-study-card-toggle{background:#fff;box-shadow:0 8px 20px rgba(108,117,125,.2);border-radius:100%;cursor:pointer;height:56px;position:relative;width:56px}.case-study-card-toggle:after,.case-study-card-toggle:before{background:#257af4;content:"";display:block;height:4px;left:calc(50% - 15px);position:absolute;top:calc(50% - 2px);transition:opacity .2s,transform .2s;width:30px}.case-study-card-toggle:after{transform:rotate(90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:before{opacity:0;transform:rotate(-90deg)}}@media screen and (min-width:980px){.case-study-card.is-open .case-study-card-toggle:after{transform:rotate(0)}}@media screen and (min-width:980px){.case-study-card .col-lg-3{left:-60%;position:relative;transition:left .4s;transition-delay:.6s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-3{flex:0 0 250px;max-width:250px;width:250px}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-3{left:0;transition-delay:0s}}@media screen and (min-width:980px){.case-study-card .col-lg-auto{opacity:0;transform:translateX(24px);transition:opacity .4s,transform .4s;transition-delay:.2s}}@media screen and (min-width:980px)and (min-width:980px){.case-study-card .col-lg-auto{max-width:605px;width:calc(100% - 319px)}}@media screen and (min-width:980px){.case-study-card.is-open .col-lg-auto{opacity:1;transform:none;transition-delay:.2s}}.footer-copy{white-space:nowrap}form .form-control{border:1px solid #6c757d;border-radius:6px;height:auto;line-height:20px;min-height:44px;padding:12px 16px;width:100%}form .form-control,form .form-control:focus{box-shadow:0 8px 20px rgba(108,117,125,.2);color:#212529}form .form-control:focus{border-color:#212529}form .form-control::-moz-placeholder{color:#6c757d}form .form-control:-ms-input-placeholder{color:#6c757d}form .form-control::placeholder{color:#6c757d}form select.form-control{-webkit-appearance:none;-moz-appearance:none;appearance:none}form select.form-control:not([data-chosen]){color:#6c757d}form .btn-secondary:active,form .btn-secondary:hover{color:#212529;background:#fc0;border-color:#fc0}.hero{overflow:visible;position:relative}.hero,.hero-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.hero-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.hero>.container{position:relative;z-index:2}.hero.has-offset{margin-bottom:-160px;padding-bottom:160px}.base-hero{height:22.5vw;max-height:324px;min-height:280px}.index-hero{background-image:url(/images/backgrounds/bg-hero-home.svg);height:68vw;max-height:980px}.index-hero,.other-hero{max-width:2448px;width:170vw}.other-hero{background-image:url(/images/backgrounds/bg-hero.svg)}.bg-footer-cta{background-image:url(/images/backgrounds/bg-footer-cta.svg);width:2448px}.quickstart-bg{background-image:url(/images/backgrounds/bg-quick-start.svg);height:40vw;top:220px;width:170vw}hr{background:#f1f6f9;border:0;display:block;height:4px;margin:0;width:100%}hr.is-small{height:2px}hr.is-large{height:8px}hr.is-medium{background:#d6dbdf}hr.is-dark{background:#495057}hr.is-yellow{background:linear-gradient(90deg,#ff8c00,#ff8c00 8px,#fc0 16px,rgba(255,204,0,0));-webkit-clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);clip-path:polygon(8px 100%,0 100%,0 0,8px 0,8px 100%,16px 100%,16px 0,100% 0,100% 100%);height:8px}.icon{display:block;height:48px;margin-bottom:24px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center}@media screen and (min-width:576px){.icon{height:64px}}@media screen and (min-width:980px){.icon{height:80px}}img{max-width:100%}.kicker{color:#6c757d;font-family:Hind Siliguri,sans-serif;font-size:.875rem;font-weight:600;letter-spacing:1px;margin:0}@media screen and (max-width:978.98px){.lead{font-size:1.125rem}}.logo{display:block;height:36px;max-width:220px;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;width:100%}.navbar-clickhouse{border-bottom:4px solid #f1f6f9;height:142px}.navbar-clickhouse>.container{flex-wrap:wrap}.navbar-super{flex-shrink:0;width:100%}.navbar-super ul{list-style:none}.navbar-super li:not(:last-child){margin-bottom:0;margin-right:24px}.navbar-super a{align-items:center;color:#212529;display:flex;font-size:.875rem}.navbar-super a:active,.navbar-super a:hover{color:#257af4;text-decoration:none}.navbar-super img{flex-shrink:0;margin-right:4px}.navbar-brand-clickhouse{background:no-repeat url(../images/logo-clickhouse.svg);background-size:contain;flex-shrink:0;height:28px;margin-right:48px;padding:0;width:180px}.navbar-nav{align-items:center;height:46px}.navbar .nav-item:not(:last-child){margin-bottom:0;margin-right:24px}.navbar .nav-link{color:#212529}.navbar .nav-link:active,.navbar .nav-link:hover{color:#257af4}.navbar .navbar-nav{flex-direction:row}@media screen and (max-width:978.98px){.navbar>.container{padding-left:20px;padding-right:20px}.navbar .navbar-toggler{height:24px;padding:0;width:24px}.navbar .navbar-toggler:focus{outline:none}.navbar .navbar-toggler-icon{background:no-repeat url(../images/icons/icon-menu.svg);background-position:50%;background-size:contain;height:24px;width:24px}.navbar .navbar-collapse{background:#fff;border-bottom:4px solid #f1f6f9;height:56px;left:0;padding:0 20px 16px;position:absolute;right:0;top:100%}.navbar .nav-link{font-size:.875rem;white-space:nowrap}}@media screen and (max-width:615.98px){.navbar .navbar-collapse{height:auto}.navbar .navbar-nav{flex-direction:column;height:auto}.navbar .nav-item:not(:last-child){margin-bottom:16px;margin-right:0}}@media screen and (max-width:399.98px){.navbar{height:80px}}.page,.photo-frame{overflow:hidden;width:100%}.photo-frame{background:hsla(0,0%,100%,.6);border-radius:100%;box-shadow:0 8px 20px rgba(108,117,125,.2);display:block;margin-bottom:24px;max-width:160px;position:relative}.photo-frame:before{content:"";display:block;padding-bottom:100%;width:100%}.photo-frame img{display:block;height:100%;left:0;-o-object-fit:contain;object-fit:contain;-o-object-position:center;object-position:center;position:absolute;top:0;width:100%}.pullquote{position:relative;width:70%}.pullquote:before{background:no-repeat url(../images/backgrounds/bg-quotes.svg);background-position:50%;background-size:100%;content:"";mix-blend-mode:multiply;right:56px;width:calc(100% - 16px);z-index:2}.pullquote-bg,.pullquote:before{bottom:0;display:block;position:absolute;top:0}.pullquote-bg{right:0;width:calc(50vw + 28.57143%);z-index:1}.pullquote-body{padding:64px 40px 64px 0;position:relative;z-index:3}.pullquote-quote{font-family:Hind Siliguri,sans-serif;font-size:32px;font-weight:700}.pullquote-citation{font-size:1.125rem}.section{overflow:visible;position:relative}.section,.section-bg{background-repeat:no-repeat;background-position:50%;background-size:cover}.section-bg{display:block;height:100%;left:50%;position:absolute;top:0;transform:translateX(-50%);z-index:1}.section>.container{position:relative;z-index:2}.social-icons{align-items:center;display:flex}.social-icons>a{aspect-ratio:24/24;background:#6c757d;display:block;height:24px;width:24px;-webkit-mask-position:center;mask-position:center;-webkit-mask-repeat:no-repeat;mask-repeat:no-repeat;-webkit-mask-size:contain;mask-size:contain;transition:background .2s}.social-icons>a:active,.social-icons>a:hover{background:#212529}.social-icons>a+a{margin-left:32px}.social-icons-facebook{-webkit-mask-image:url(/images/icons/icon-facebook-gray.svg);mask-image:url(/images/icons/icon-facebook-gray.svg)}.social-icons-twitter{-webkit-mask-image:url(/images/icons/icon-twitter-gray.svg);mask-image:url(/images/icons/icon-twitter-gray.svg);width:31px}.social-icons-linkedin{-webkit-mask-image:url(/images/icons/icon-linkedin-gray.svg);mask-image:url(/images/icons/icon-linkedin-gray.svg)}.social-icons-linkedin-alt{-webkit-mask-image:url(/images/icons/icon-linkedin-alt-gray.svg);mask-image:url(/images/icons/icon-linkedin-alt-gray.svg)}.social-icons.size-small>a{height:20px;width:20px}.social-icons.size-small>a:active,.social-icons.size-small>a:hover{background:#212529}.social-icons.size-small>a+a{margin-left:16px}.tabs{position:relative}.tabs:before{background:#fff;border-radius:7px 7px 0 0;content:"";display:block;height:8px;left:1px;position:absolute;right:1px;top:68px;z-index:10}@media screen and (min-width:1240px){.tabs:before{top:76px}}.tabs-body{background:#fff;border-radius:8px;border:1px solid #6c757d;box-shadow:0 8px 20px rgba(108,117,125,.2);padding:24px}@media screen and (min-width:980px){.tabs-body{padding:32px}}@media screen and (min-width:1240px){.tabs-body{padding:40px}}.tabs .nav-tabs{border-bottom:0;flex-wrap:nowrap;height:76px;margin:-20px -20px -9px;-webkit-mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);mask-image:linear-gradient(90deg,transparent,#000 20px,#000 calc(100% - 20px),transparent);overflow:scroll;overflow-x:scroll;overflow-y:visible;padding:20px 20px 0;position:relative}@media screen and (min-width:940px){.tabs .nav-tabs{overflow:visible}}@media screen and (min-width:1240px){.tabs .nav-tabs{height:84px}}.tabs .nav-link{align-items:center;border-bottom:0;color:#6c757d;display:flex;font-size:.875rem;flex-shrink:0;height:56px;justify-content:center;padding:0 12px 8px;text-align:center;white-space:nowrap}@media screen and (min-width:1240px){.tabs .nav-link{height:64px;padding:0 16px 8px}}.tabs .nav-link.active{background:#fff;box-shadow:0 -4px 8px rgba(108,117,125,.1);font-weight:700;padding:0 16px 8px}@media screen and (min-width:980px){.tabs .nav-link.active{padding:0 24px 8px}}@media screen and (min-width:1240px){.tabs .nav-link.active{padding:0 32px 8px}}.tab-pane pre{background:#212529;border-radius:16px;color:#fff;padding:24px 16px}@media screen and (min-width:1240px){.tab-pane pre{padding:32px 24px}}.trailing-link{align-items:center;color:#212529;display:flex;font-size:.875rem;font-weight:700}.trailing-link:after{background:no-repeat url(../images/icons/icon-arrow.svg);background-position:100%;background-size:contain;content:"";display:block;height:12px;transition:transform .2s;width:20px}.trailing-link:active,.trailing-link:hover{color:#212529;text-decoration:none}.trailing-link:active:after,.trailing-link:hover:after{transform:translateX(8px)}.trailing-link.span-full:after{margin-left:auto}ul{list-style-type:square;padding-left:1.25em}ul li:not(:last-child){margin-bottom:16px}ul li::marker{color:#ff3939}ul.has-separators{list-style:none;padding:0}ul.has-separators li:not(:last-child){border-bottom:4px solid #f1f6f9;margin-bottom:24px;padding-bottom:24px}.bg-gradient-secondary{background-image:linear-gradient(58deg,#ff6443 3%,#fe561d 24%,#e32f0d 93%)}.bg-gradient-light-orange{background-image:linear-gradient(90deg,rgba(255,203,128,0),#ffcb80)}.bg-offset-right{bottom:0;left:-24px;position:absolute;top:0;width:calc(100vw + 24px);z-index:-1}@media screen and (min-width:1240px){.bg-offset-right{left:-96px;width:calc(100vw + 96px)}}.bg-inset-right{bottom:0;left:40px;position:absolute;top:0;width:calc(100vw - 40px);z-index:-1}@media screen and (min-width:980px){.bg-inset-right{left:96px;width:calc(100vw - 96px)}}.has-border-left{border-left:8px solid #f1f6f9;padding-left:16px}.font-xl{font-size:1.25rem}.font-lg{font-size:1.125rem}.font-sm{font-size:.875rem}.font-xs{font-size:.625rem}.font-weight-semibold{font-weight:600}.display-5{color:#212529;font-size:20px;font-weight:500}.display-6{color:#212529;font-size:14px;font-weight:700}.overflow-auto{overflow:auto}.text-decoration-underline{text-decoration:underline}.text-upper{text-transform:uppercase} \ No newline at end of file diff --git a/website/src/scss/_variables.scss b/website/src/scss/_variables.scss index 4d5ef4f4667..d511c757055 100644 --- a/website/src/scss/_variables.scss +++ b/website/src/scss/_variables.scss @@ -165,7 +165,7 @@ $border-color: $gray-500; $border-radius: 8px; $border-radius-lg: 8px; -$border-radius-sm: 8px; +$border-radius-sm: 4px; $box-shadow-sm: 0 2px 14px rgba($gray-700, .2); $box-shadow: 0 8px 20px rgba($gray-700, .2); @@ -238,10 +238,10 @@ $btn-font-size: 14px; $btn-line-height: 20px; $btn-white-space: null; // Set to `nowrap` to prevent text wrapping -$btn-padding-y-sm: 12px; -$btn-padding-x-sm: 32px; -$btn-font-size-sm: 14px; -$btn-line-height-sm: 20px; +$btn-padding-y-sm: 6px; +$btn-padding-x-sm: 12px; +$btn-font-size-sm: 12px; +$btn-line-height-sm: 16px; $btn-padding-y-lg: 16px; $btn-padding-x-lg: 32px; From d1a14d76f1ecfc2419a8368c099e50723812f9b5 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 19 Nov 2021 02:29:35 +0800 Subject: [PATCH 139/200] fix --- .../MergeTreeBaseSelectProcessor.cpp | 8 ++------ .../MergeTree/MergeTreeBaseSelectProcessor.h | 2 -- src/Storages/MergeTree/MergeTreeData.cpp | 20 +++++++++++++++++-- .../01710_minmax_count_projection.reference | 1 + .../01710_minmax_count_projection.sql | 1 + 5 files changed, 22 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp index 80dc3f149e9..2f46543b03c 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.cpp @@ -415,7 +415,8 @@ void MergeTreeBaseSelectProcessor::injectVirtualColumns( chunk.setColumns(columns, num_rows); } -void MergeTreeBaseSelectProcessor::transformBlockViaPrewhereInfo(Block & block, const PrewhereInfoPtr & prewhere_info) +Block MergeTreeBaseSelectProcessor::transformHeader( + Block block, const PrewhereInfoPtr & prewhere_info, const DataTypePtr & partition_value_type, const Names & virtual_columns) { if (prewhere_info) { @@ -459,12 +460,7 @@ void MergeTreeBaseSelectProcessor::transformBlockViaPrewhereInfo(Block & block, ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER); } } -} -Block MergeTreeBaseSelectProcessor::transformHeader( - Block block, const PrewhereInfoPtr & prewhere_info, const DataTypePtr & partition_value_type, const Names & virtual_columns) -{ - transformBlockViaPrewhereInfo(block, prewhere_info); injectVirtualColumns(block, nullptr, partition_value_type, virtual_columns); return block; } diff --git a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h index 01e55486815..d102e4f07a4 100644 --- a/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeBaseSelectProcessor.h @@ -42,8 +42,6 @@ public: const MergeTreeReadTaskColumns & task_columns, const Block & sample_block); - static void transformBlockViaPrewhereInfo(Block & block, const PrewhereInfoPtr & prewhere_info); - protected: Chunk generate() final; diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 44ea78d3cd5..22f582dc13a 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4789,8 +4789,24 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( query_info.minmax_count_projection_block = getMinMaxCountProjectionBlock( metadata_snapshot, minmax_conut_projection_candidate->required_columns, query_info, parts, normal_parts, query_context); - MergeTreeBaseSelectProcessor::transformBlockViaPrewhereInfo( - query_info.minmax_count_projection_block, minmax_conut_projection_candidate->prewhere_info); + if (minmax_conut_projection_candidate->prewhere_info) + { + const auto & prewhere_info = minmax_conut_projection_candidate->prewhere_info; + if (prewhere_info->alias_actions) + ExpressionActions(prewhere_info->alias_actions, actions_settings).execute(query_info.minmax_count_projection_block); + + if (prewhere_info->row_level_filter) + { + ExpressionActions(prewhere_info->row_level_filter, actions_settings).execute(query_info.minmax_count_projection_block); + query_info.minmax_count_projection_block.erase(prewhere_info->row_level_column_name); + } + + if (prewhere_info->prewhere_actions) + ExpressionActions(prewhere_info->prewhere_actions, actions_settings).execute(query_info.minmax_count_projection_block); + + if (prewhere_info->remove_prewhere_column) + query_info.minmax_count_projection_block.erase(prewhere_info->prewhere_column_name); + } if (normal_parts.empty()) { diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.reference b/tests/queries/0_stateless/01710_minmax_count_projection.reference index 201156465a6..495a5e7b6fa 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.reference +++ b/tests/queries/0_stateless/01710_minmax_count_projection.reference @@ -15,3 +15,4 @@ 1 \N 2021-10-27 10:00:00 4 2021-10-24 10:00:00 +2021-10-24 10:00:00 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index 597cd653ab3..b3dad492bdc 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -54,5 +54,6 @@ select count() from d group by toDate(dt); -- fuzz crash SELECT pointInEllipses(min(j), NULL), max(dt), count('0.0000000007') FROM d WHERE toDate(dt) >= '2021-10-25'; SELECT min(dt) FROM d PREWHERE ceil(j) <= 0; +SELECT min(dt) FROM d PREWHERE ((0.9998999834060669 AND 1023) AND 255) <= ceil(j); drop table d; From f4b5d5a0d795d8e39c5133af3f8ca3ddf2736cca Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 18 Nov 2021 21:49:36 +0300 Subject: [PATCH 140/200] Update src/Access/EnabledQuota.cpp --- src/Access/EnabledQuota.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Access/EnabledQuota.cpp b/src/Access/EnabledQuota.cpp index cf78bfd0475..76708a92764 100644 --- a/src/Access/EnabledQuota.cpp +++ b/src/Access/EnabledQuota.cpp @@ -67,7 +67,7 @@ struct EnabledQuota::Impl { /// We reset counters only if the interval's end has been calculated before. /// If it hasn't we just calculate the interval's end for the first time and don't reset counters yet. - need_reset_counters = true; + need_reset_counters = (end_loaded.count() != 0); break; } end = std::chrono::system_clock::time_point{end_loaded}; From 108e86bf57b22fb13ee4053d3133b8b7bc2f5563 Mon Sep 17 00:00:00 2001 From: Alexey Date: Thu, 18 Nov 2021 19:44:38 +0000 Subject: [PATCH 141/200] Changed examples: take from dev PR --- docs/en/sql-reference/operators/exists.md | 27 +++++++++++++++++------ docs/ru/sql-reference/operators/exists.md | 27 +++++++++++++++++------ 2 files changed, 40 insertions(+), 14 deletions(-) diff --git a/docs/en/sql-reference/operators/exists.md b/docs/en/sql-reference/operators/exists.md index 45ea584c3e6..ee0c7317637 100644 --- a/docs/en/sql-reference/operators/exists.md +++ b/docs/en/sql-reference/operators/exists.md @@ -15,17 +15,30 @@ WHERE EXISTS(subquery) **Example** -Query: +Query with a subquery returning several rows: ``` sql -SELECT 'Exists' WHERE EXISTS (SELECT * FROM numbers(10) WHERE number < 2); -SELECT 'Empty subquery' WHERE EXISTS (SELECT * FROM numbers(10) WHERE number > 12); +SELECT count() FROM numbers(10) WHERE EXISTS(SELECT number FROM numbers(10) WHERE number > 8); ``` -The first query returns one row while the second query does not return rows because the result of the subquery is empty: +Result: ``` text -┌─'Exists'─┐ -│ Exists │ -└──────────┘ +┌─count()─┐ +│ 10 │ +└─────────┘ +``` + +Query with a subquery that returns an empty result: + +``` sql +SELECT count() FROM numbers(10) WHERE EXISTS(SELECT number FROM numbers(10) WHERE number > 11); +``` + +Result: + +``` text +┌─count()─┐ +│ 0 │ +└─────────┘ ``` diff --git a/docs/ru/sql-reference/operators/exists.md b/docs/ru/sql-reference/operators/exists.md index d56765286f7..3e04304e222 100644 --- a/docs/ru/sql-reference/operators/exists.md +++ b/docs/ru/sql-reference/operators/exists.md @@ -15,17 +15,30 @@ WHERE EXISTS(subquery) **Пример** -Запрос: +Запрос с подзапросом, возвращающим несколько строк: ``` sql -SELECT 'Exists' WHERE EXISTS (SELECT * FROM numbers(10) WHERE number < 2); -SELECT 'Empty subquery' WHERE EXISTS (SELECT * FROM numbers(10) WHERE number > 12); +SELECT count() FROM numbers(10) WHERE EXISTS(SELECT number FROM numbers(10) WHERE number > 8); ``` -Первый запрос возвращает одну строку, а второй запрос не возвращает строк, так как результат его подзапроса пустой: +Результат: ``` text -┌─'Exists'─┐ -│ Exists │ -└──────────┘ +┌─count()─┐ +│ 10 │ +└─────────┘ +``` + +Запрос с подзапросом, возвращающим пустой результат: + +``` sql +SELECT count() FROM numbers(10) WHERE EXISTS(SELECT number FROM numbers(10) WHERE number > 11); +``` + +Результат: + +``` text +┌─count()─┐ +│ 0 │ +└─────────┘ ``` From 1135c358452ca99bff87c8fe46f5d752b0575d49 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 18 Nov 2021 23:38:01 +0300 Subject: [PATCH 142/200] Add hardware benchmark from GCP --- website/benchmark/hardware/index.html | 1 + .../benchmark/hardware/results/gcp_n2d.json | 54 +++++++++++++++++++ 2 files changed, 55 insertions(+) create mode 100644 website/benchmark/hardware/results/gcp_n2d.json diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index a0b4001361a..e6a87385fa8 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -80,6 +80,7 @@ Results for MacBook Air M1 are from Denis Glazachev.
Results for Xeon Gold 6140 are from Shiv Iyer (ChistaDATA Labs).
Comparison of EBS and EFS is from Ramazan Polat.
Results for Hetzner and Scaleway are from Anthony Najjar Simon (Panelbear).
+Results for GCP are from Vy Nguyen Tan.

diff --git a/website/benchmark/hardware/results/gcp_n2d.json b/website/benchmark/hardware/results/gcp_n2d.json new file mode 100644 index 00000000000..25c20fb5457 --- /dev/null +++ b/website/benchmark/hardware/results/gcp_n2d.json @@ -0,0 +1,54 @@ +[ + { + "system": "GCP n2d-16-highmem" + "system_full": "GCP compute n2d-16-highmem, AMD EPYC 7B12, 16vCPU, 128 GiB RAM", + "time": "2021-11-18 00:00:00", + "kind": "cloud", + "result": + [ +[0.002, 0.001, 0.001], +[0.017, 0.013, 0.012], +[0.046, 0.032, 0.031], +[0.062, 0.048, 0.046], +[0.122, 0.112, 0.103], +[0.365, 0.313, 0.312], +[0.026, 0.029, 0.024], +[0.028, 0.014, 0.014], +[0.516, 0.473, 0.477], +[0.591, 0.545, 0.542], +[0.210, 0.178, 0.183], +[0.224, 0.208, 0.205], +[0.682, 0.629, 0.609], +[0.862, 0.804, 0.812], +[0.854, 0.769, 0.778], +[0.769, 0.771, 0.768], +[2.147, 2.171, 2.166], +[1.439, 1.380, 1.355], +[4.099, 3.974, 4.048], +[0.118, 0.053, 0.048], +[0.873, 0.785, 0.786], +[1.022, 0.908, 0.891], +[2.278, 2.079, 2.042], +[1.714, 0.962, 0.950], +[0.400, 0.287, 0.267], +[0.336, 0.248, 0.228], +[0.379, 0.273, 0.268], +[0.889, 0.816, 0.802], +[1.474, 1.391, 1.455], +[1.358, 1.355, 1.342], +[0.723, 0.628, 0.613], +[0.914, 0.756, 0.741], +[3.916, 3.967, 3.962], +[3.194, 2.998, 3.016], +[3.097, 3.050, 3.073], +[1.099, 1.111, 1.087], +[0.184, 0.168, 0.175], +[0.072, 0.066, 0.065], +[0.067, 0.063, 0.055], +[0.373, 0.374, 0.376], +[0.032, 0.027, 0.020], +[0.021, 0.015, 0.015], +[0.006, 0.008, 0.006] + ] + } +] From f2bc38ad3e99952993691f162e0043e1c7518b25 Mon Sep 17 00:00:00 2001 From: karnevil13 <90553247+karnevil13@users.noreply.github.com> Date: Thu, 18 Nov 2021 23:40:10 +0300 Subject: [PATCH 143/200] Apply suggestions from code review Co-authored-by: gyuton <40863448+gyuton@users.noreply.github.com> --- docs/ru/engines/database-engines/replicated.md | 2 +- docs/ru/interfaces/third-party/gui.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/engines/database-engines/replicated.md b/docs/ru/engines/database-engines/replicated.md index 9b4e6918633..6777529f3e0 100644 --- a/docs/ru/engines/database-engines/replicated.md +++ b/docs/ru/engines/database-engines/replicated.md @@ -36,7 +36,7 @@ DDL-запросы с базой данных `Replicated` работают по При создании новой реплики базы, эта реплика сама создаёт таблицы. Если реплика долго была недоступна и отстала от лога репликации — она сверяет свои локальные метаданные с актуальными метаданными в ZooKeeper, перекладывает лишние таблицы с данными в отдельную нереплицируемую базу (чтобы случайно не удалить что-нибудь лишнее), создаёт недостающие таблицы, обновляет имена таблиц, если были переименования. Данные реплицируются на уровне `ReplicatedMergeTree`, т.е. если таблица не реплицируемая, то данные реплицироваться не будут (база отвечает только за метаданные). -Запросы [`ALTER TABLE ATTACH|FETCH|DROP|DROP DETACHED|DETACH PARTITION|PART`](../../sql-reference/statements/alter/partition.md) допустимы, но не реплицируются. Движок базы данных может только добавить/извлечь/удалить раздел или часть нынешней реплики. Однако если сама таблица использует движок реплицируемой таблицы, тогда данные будут реплицированы после применения `ATTACH`. +Запросы [`ALTER TABLE ATTACH|FETCH|DROP|DROP DETACHED|DETACH PARTITION|PART`](../../sql-reference/statements/alter/partition.md) допустимы, но не реплицируются. Движок базы данных может только добавить/извлечь/удалить партицию или кусок нынешней реплики. Однако если сама таблица использует движок реплицируемой таблицы, тогда данные будут реплицированы после применения `ATTACH`. ## Примеры использования {#usage-example} diff --git a/docs/ru/interfaces/third-party/gui.md b/docs/ru/interfaces/third-party/gui.md index 0a3589db1d3..b80aaf7948e 100644 --- a/docs/ru/interfaces/third-party/gui.md +++ b/docs/ru/interfaces/third-party/gui.md @@ -73,7 +73,7 @@ toc_title: "Визуальные интерфейсы от сторонних р [Grafana](https://grafana.com/grafana/plugins/vertamedia-clickhouse-datasource) — платформа для мониторинга и визуализации. -"С помощью Grafana вы можете делать запросы, визуализировать, получать уведомления и разбираться в метриках, где бы они ни хранились. Создавайте, исследуйте, делитесь дашбордами с командой и прививайте культуру принятия решений на основе данных. Мы пользуемся доверием и любовью пользователей" — grafana.com. +"С помощью Grafana вы можете делать запросы, визуализировать, получать уведомления и разбираться в метриках, где бы они ни хранились. Создавайте, исследуйте, делитесь дашбордами с командой и прививайте культуру принятия решений на основе данных. Мы пользуемся доверием и любовью пользователей" — grafana.com. Плагин источника данных ClickHouse поддерживает ClickHouse в качестве бэкенд базы данных. From 2883ca012e9b96df228f91aebbe3dfbce6337fea Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 18 Nov 2021 10:45:52 +0300 Subject: [PATCH 144/200] Rename RowPolicy::NameParts -> RowPolicyName and move it to Access/Common. --- src/Access/Common/RowPolicyDefs.cpp | 23 +++++++ src/Access/Common/RowPolicyDefs.h | 22 +++++++ src/Access/RowPolicy.cpp | 30 +++++----- src/Access/RowPolicy.h | 47 +++------------ src/Access/UsersConfigAccessStorage.cpp | 2 +- .../InterpreterCreateRowPolicyQuery.cpp | 12 ++-- ...InterpreterShowCreateAccessEntityQuery.cpp | 2 +- src/Parsers/Access/ASTRowPolicyName.cpp | 60 +++++++++---------- src/Parsers/Access/ASTRowPolicyName.h | 6 +- .../Access/ParserCreateRowPolicyQuery.cpp | 2 +- src/Parsers/Access/ParserRowPolicyName.cpp | 28 ++++----- .../System/StorageSystemRowPolicies.cpp | 10 ++-- 12 files changed, 130 insertions(+), 114 deletions(-) create mode 100644 src/Access/Common/RowPolicyDefs.cpp create mode 100644 src/Access/Common/RowPolicyDefs.h diff --git a/src/Access/Common/RowPolicyDefs.cpp b/src/Access/Common/RowPolicyDefs.cpp new file mode 100644 index 00000000000..aa1d1971d7b --- /dev/null +++ b/src/Access/Common/RowPolicyDefs.cpp @@ -0,0 +1,23 @@ +#include +#include + + +namespace DB +{ + +String RowPolicyName::toString() const +{ + String name; + name.reserve(database.length() + table_name.length() + short_name.length() + 6); + name += backQuoteIfNeed(short_name); + name += " ON "; + if (!database.empty()) + { + name += backQuoteIfNeed(database); + name += '.'; + } + name += backQuoteIfNeed(table_name); + return name; +} + +} diff --git a/src/Access/Common/RowPolicyDefs.h b/src/Access/Common/RowPolicyDefs.h new file mode 100644 index 00000000000..771bb2ed738 --- /dev/null +++ b/src/Access/Common/RowPolicyDefs.h @@ -0,0 +1,22 @@ +#pragma once + +#include + + +namespace DB +{ + +struct RowPolicyName +{ + String short_name; + String database; + String table_name; + + bool empty() const { return short_name.empty(); } + String toString() const; + auto toTuple() const { return std::tie(short_name, database, table_name); } + friend bool operator ==(const RowPolicyName & left, const RowPolicyName & right) { return left.toTuple() == right.toTuple(); } + friend bool operator !=(const RowPolicyName & left, const RowPolicyName & right) { return left.toTuple() != right.toTuple(); } +}; + +} diff --git a/src/Access/RowPolicy.cpp b/src/Access/RowPolicy.cpp index 7441f915a46..3ce201f000b 100644 --- a/src/Access/RowPolicy.cpp +++ b/src/Access/RowPolicy.cpp @@ -13,34 +13,34 @@ namespace ErrorCodes void RowPolicy::setDatabase(const String & database) { - name_parts.database = database; - IAccessEntity::setName(name_parts.getName()); + full_name.database = database; + IAccessEntity::setName(full_name.toString()); } void RowPolicy::setTableName(const String & table_name) { - name_parts.table_name = table_name; - IAccessEntity::setName(name_parts.getName()); + full_name.table_name = table_name; + IAccessEntity::setName(full_name.toString()); } void RowPolicy::setShortName(const String & short_name) { - name_parts.short_name = short_name; - IAccessEntity::setName(name_parts.getName()); + full_name.short_name = short_name; + IAccessEntity::setName(full_name.toString()); } -void RowPolicy::setNameParts(const String & short_name, const String & database, const String & table_name) +void RowPolicy::setFullName(const String & short_name, const String & database, const String & table_name) { - name_parts.short_name = short_name; - name_parts.database = database; - name_parts.table_name = table_name; - IAccessEntity::setName(name_parts.getName()); + full_name.short_name = short_name; + full_name.database = database; + full_name.table_name = table_name; + IAccessEntity::setName(full_name.toString()); } -void RowPolicy::setNameParts(const NameParts & name_parts_) +void RowPolicy::setFullName(const RowPolicyName & full_name_) { - name_parts = name_parts_; - IAccessEntity::setName(name_parts.getName()); + full_name = full_name_; + IAccessEntity::setName(full_name.toString()); } void RowPolicy::setName(const String &) @@ -54,7 +54,7 @@ bool RowPolicy::equal(const IAccessEntity & other) const if (!IAccessEntity::equal(other)) return false; const auto & other_policy = typeid_cast(other); - return (name_parts == other_policy.name_parts) && boost::range::equal(conditions, other_policy.conditions) + return (full_name == other_policy.full_name) && boost::range::equal(conditions, other_policy.conditions) && restrictive == other_policy.restrictive && (to_roles == other_policy.to_roles); } diff --git a/src/Access/RowPolicy.h b/src/Access/RowPolicy.h index 723db545dbe..3e0a1bcb662 100644 --- a/src/Access/RowPolicy.h +++ b/src/Access/RowPolicy.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -18,30 +19,16 @@ namespace ErrorCodes */ struct RowPolicy : public IAccessEntity { - struct NameParts - { - String short_name; - String database; - String table_name; - - bool empty() const { return short_name.empty(); } - String getName() const; - String toString() const { return getName(); } - auto toTuple() const { return std::tie(short_name, database, table_name); } - friend bool operator ==(const NameParts & left, const NameParts & right) { return left.toTuple() == right.toTuple(); } - friend bool operator !=(const NameParts & left, const NameParts & right) { return left.toTuple() != right.toTuple(); } - }; - void setShortName(const String & short_name); void setDatabase(const String & database); void setTableName(const String & table_name); - void setNameParts(const String & short_name, const String & database, const String & table_name); - void setNameParts(const NameParts & name_parts); + void setFullName(const String & short_name, const String & database, const String & table_name); + void setFullName(const RowPolicyName & full_name_); - const String & getDatabase() const { return name_parts.database; } - const String & getTableName() const { return name_parts.table_name; } - const String & getShortName() const { return name_parts.short_name; } - const NameParts & getNameParts() const { return name_parts; } + const String & getDatabase() const { return full_name.database; } + const String & getTableName() const { return full_name.table_name; } + const String & getShortName() const { return full_name.short_name; } + const RowPolicyName & getFullName() const { return full_name; } /// Filter is a SQL conditional expression used to figure out which rows should be visible /// for user or available for modification. If the expression returns NULL or false for some rows @@ -95,9 +82,9 @@ struct RowPolicy : public IAccessEntity RolesOrUsersSet to_roles; private: - void setName(const String & name_) override; + void setName(const String &) override; - NameParts name_parts; + RowPolicyName full_name; bool restrictive = false; }; @@ -156,20 +143,4 @@ inline String toString(RowPolicy::ConditionType type) return RowPolicy::ConditionTypeInfo::get(type).raw_name; } - -inline String RowPolicy::NameParts::getName() const -{ - String name; - name.reserve(database.length() + table_name.length() + short_name.length() + 6); - name += backQuoteIfNeed(short_name); - name += " ON "; - if (!database.empty()) - { - name += backQuoteIfNeed(database); - name += '.'; - } - name += backQuoteIfNeed(table_name); - return name; -} - } diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index 2d202c5094d..41a64d7f7eb 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -351,7 +351,7 @@ namespace String filter = (it != user_to_filters.end()) ? it->second : "1"; auto policy = std::make_shared(); - policy->setNameParts(user_name, database, table_name); + policy->setFullName(user_name, database, table_name); policy->conditions[RowPolicy::SELECT_FILTER] = filter; policy->to_roles.add(generateID(EntityType::USER, user_name)); policies.push_back(policy); diff --git a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp index 37347b37619..7abe8bbf000 100644 --- a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp @@ -17,15 +17,15 @@ namespace void updateRowPolicyFromQueryImpl( RowPolicy & policy, const ASTCreateRowPolicyQuery & query, - const RowPolicy::NameParts & override_name, + const RowPolicyName & override_name, const std::optional & override_to_roles) { if (!override_name.empty()) - policy.setNameParts(override_name); + policy.setFullName(override_name); else if (!query.new_short_name.empty()) policy.setShortName(query.new_short_name); - else if (query.names->name_parts.size() == 1) - policy.setNameParts(query.names->name_parts.front()); + else if (query.names->full_names.size() == 1) + policy.setFullName(query.names->full_names.front()); if (query.is_restrictive) policy.setRestrictive(*query.is_restrictive); @@ -80,10 +80,10 @@ BlockIO InterpreterCreateRowPolicyQuery::execute() else { std::vector new_policies; - for (const auto & name_parts : query.names->name_parts) + for (const auto & full_name : query.names->full_names) { auto new_policy = std::make_shared(); - updateRowPolicyFromQueryImpl(*new_policy, query, name_parts, roles_from_query); + updateRowPolicyFromQueryImpl(*new_policy, query, full_name, roles_from_query); new_policies.emplace_back(std::move(new_policy)); } diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index ca6003e2cc0..2cef0a256cf 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -182,7 +182,7 @@ namespace { auto query = std::make_shared(); query->names = std::make_shared(); - query->names->name_parts.emplace_back(policy.getNameParts()); + query->names->full_names.emplace_back(policy.getFullName()); query->attach = attach_mode; if (policy.isRestrictive()) diff --git a/src/Parsers/Access/ASTRowPolicyName.cpp b/src/Parsers/Access/ASTRowPolicyName.cpp index c8b8107af20..8b49677c506 100644 --- a/src/Parsers/Access/ASTRowPolicyName.cpp +++ b/src/Parsers/Access/ASTRowPolicyName.cpp @@ -12,9 +12,9 @@ namespace ErrorCodes void ASTRowPolicyName::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const { - const String & database = name_parts.database; - const String & table_name = name_parts.table_name; - const String & short_name = name_parts.short_name; + const String & database = full_name.database; + const String & table_name = full_name.table_name; + const String & short_name = full_name.short_name; settings.ostr << backQuoteIfNeed(short_name) << (settings.hilite ? hilite_keyword : "") << " ON " << (settings.hilite ? hilite_none : "") << (database.empty() ? String{} : backQuoteIfNeed(database) + ".") << backQuoteIfNeed(table_name); @@ -25,21 +25,21 @@ void ASTRowPolicyName::formatImpl(const FormatSettings & settings, FormatState & void ASTRowPolicyName::replaceEmptyDatabase(const String & current_database) { - if (name_parts.database.empty()) - name_parts.database = current_database; + if (full_name.database.empty()) + full_name.database = current_database; } void ASTRowPolicyNames::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const { - if (name_parts.empty()) + if (full_names.empty()) throw Exception("No names of row policies in AST", ErrorCodes::LOGICAL_ERROR); bool same_short_name = true; - if (name_parts.size() > 1) + if (full_names.size() > 1) { - for (size_t i = 1; i != name_parts.size(); ++i) - if (name_parts[i].short_name != name_parts[0].short_name) + for (size_t i = 1; i != full_names.size(); ++i) + if (full_names[i].short_name != full_names[0].short_name) { same_short_name = false; break; @@ -47,10 +47,10 @@ void ASTRowPolicyNames::formatImpl(const FormatSettings & settings, FormatState } bool same_db_and_table_name = true; - if (name_parts.size() > 1) + if (full_names.size() > 1) { - for (size_t i = 1; i != name_parts.size(); ++i) - if ((name_parts[i].database != name_parts[0].database) || (name_parts[i].table_name != name_parts[0].table_name)) + for (size_t i = 1; i != full_names.size(); ++i) + if ((full_names[i].database != full_names[0].database) || (full_names[i].table_name != full_names[0].table_name)) { same_db_and_table_name = false; break; @@ -59,17 +59,17 @@ void ASTRowPolicyNames::formatImpl(const FormatSettings & settings, FormatState if (same_short_name) { - const String & short_name = name_parts[0].short_name; + const String & short_name = full_names[0].short_name; settings.ostr << backQuoteIfNeed(short_name) << (settings.hilite ? hilite_keyword : "") << " ON " << (settings.hilite ? hilite_none : ""); bool need_comma = false; - for (const auto & np : name_parts) + for (const auto & full_name : full_names) { if (std::exchange(need_comma, true)) settings.ostr << ", "; - const String & database = np.database; - const String & table_name = np.table_name; + const String & database = full_name.database; + const String & table_name = full_name.table_name; if (!database.empty()) settings.ostr << backQuoteIfNeed(database) + "."; settings.ostr << backQuoteIfNeed(table_name); @@ -78,16 +78,16 @@ void ASTRowPolicyNames::formatImpl(const FormatSettings & settings, FormatState else if (same_db_and_table_name) { bool need_comma = false; - for (const auto & np : name_parts) + for (const auto & full_name : full_names) { if (std::exchange(need_comma, true)) settings.ostr << ", "; - const String & short_name = np.short_name; + const String & short_name = full_name.short_name; settings.ostr << backQuoteIfNeed(short_name); } - const String & database = name_parts[0].database; - const String & table_name = name_parts[0].table_name; + const String & database = full_names[0].database; + const String & table_name = full_names[0].table_name; settings.ostr << (settings.hilite ? hilite_keyword : "") << " ON " << (settings.hilite ? hilite_none : ""); if (!database.empty()) settings.ostr << backQuoteIfNeed(database) + "."; @@ -96,13 +96,13 @@ void ASTRowPolicyNames::formatImpl(const FormatSettings & settings, FormatState else { bool need_comma = false; - for (const auto & np : name_parts) + for (const auto & full_name : full_names) { if (std::exchange(need_comma, true)) settings.ostr << ", "; - const String & short_name = np.short_name; - const String & database = np.database; - const String & table_name = np.table_name; + const String & short_name = full_name.short_name; + const String & database = full_name.database; + const String & table_name = full_name.table_name; settings.ostr << backQuoteIfNeed(short_name) << (settings.hilite ? hilite_keyword : "") << " ON " << (settings.hilite ? hilite_none : ""); if (!database.empty()) @@ -118,18 +118,18 @@ void ASTRowPolicyNames::formatImpl(const FormatSettings & settings, FormatState Strings ASTRowPolicyNames::toStrings() const { Strings res; - res.reserve(name_parts.size()); - for (const auto & np : name_parts) - res.emplace_back(np.toString()); + res.reserve(full_names.size()); + for (const auto & full_name : full_names) + res.emplace_back(full_name.toString()); return res; } void ASTRowPolicyNames::replaceEmptyDatabase(const String & current_database) { - for (auto & np : name_parts) - if (np.database.empty()) - np.database = current_database; + for (auto & full_name : full_names) + if (full_name.database.empty()) + full_name.database = current_database; } } diff --git a/src/Parsers/Access/ASTRowPolicyName.h b/src/Parsers/Access/ASTRowPolicyName.h index b195596225b..8bad6597bb8 100644 --- a/src/Parsers/Access/ASTRowPolicyName.h +++ b/src/Parsers/Access/ASTRowPolicyName.h @@ -14,8 +14,8 @@ namespace DB class ASTRowPolicyName : public IAST, public ASTQueryWithOnCluster { public: - RowPolicy::NameParts name_parts; - String toString() const { return name_parts.getName(); } + RowPolicyName full_name; + String toString() const { return full_name.toString(); } String getID(char) const override { return "RowPolicyName"; } ASTPtr clone() const override { return std::make_shared(*this); } @@ -36,7 +36,7 @@ public: class ASTRowPolicyNames : public IAST, public ASTQueryWithOnCluster { public: - std::vector name_parts; + std::vector full_names; Strings toStrings() const; String getID(char) const override { return "RowPolicyNames"; } diff --git a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp index f6a33ec84a3..dace8328fea 100644 --- a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp @@ -253,7 +253,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & while (true) { - if (alter && new_short_name.empty() && (names->name_parts.size() == 1) && parseRenameTo(pos, expected, new_short_name)) + if (alter && (names->full_names.size() == 1) && new_short_name.empty() && parseRenameTo(pos, expected, new_short_name)) continue; if (!is_restrictive) diff --git a/src/Parsers/Access/ParserRowPolicyName.cpp b/src/Parsers/Access/ParserRowPolicyName.cpp index aa159532754..7df4e5a36dc 100644 --- a/src/Parsers/Access/ParserRowPolicyName.cpp +++ b/src/Parsers/Access/ParserRowPolicyName.cpp @@ -91,7 +91,7 @@ namespace bool allow_multiple_short_names, bool allow_multiple_tables, bool allow_on_cluster, - std::vector & name_parts, + std::vector & full_names, String & cluster) { return IParserBase::wrapParseImpl(pos, [&] @@ -132,10 +132,10 @@ namespace assert(!short_names.empty()); assert(!database_and_table_names.empty()); - name_parts.clear(); + full_names.clear(); for (const String & short_name : short_names) for (const auto & [database, table_name] : database_and_table_names) - name_parts.push_back({short_name, database, table_name}); + full_names.push_back({short_name, database, table_name}); cluster = std::move(res_cluster); return true; @@ -146,14 +146,14 @@ namespace bool ParserRowPolicyName::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - std::vector name_parts; + std::vector full_names; String cluster; - if (!parseRowPolicyNamesAroundON(pos, expected, false, false, allow_on_cluster, name_parts, cluster)) + if (!parseRowPolicyNamesAroundON(pos, expected, false, false, allow_on_cluster, full_names, cluster)) return false; - assert(name_parts.size() == 1); + assert(full_names.size() == 1); auto result = std::make_shared(); - result->name_parts = std::move(name_parts.front()); + result->full_name = std::move(full_names.front()); result->cluster = std::move(cluster); node = result; return true; @@ -162,24 +162,24 @@ bool ParserRowPolicyName::parseImpl(Pos & pos, ASTPtr & node, Expected & expecte bool ParserRowPolicyNames::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { - std::vector name_parts; + std::vector full_names; size_t num_added_names_last_time = 0; String cluster; auto parse_around_on = [&] { - if (!name_parts.empty()) + if (!full_names.empty()) { if ((num_added_names_last_time != 1) || !cluster.empty()) return false; } - std::vector new_name_parts; - if (!parseRowPolicyNamesAroundON(pos, expected, name_parts.empty(), name_parts.empty(), allow_on_cluster, new_name_parts, cluster)) + std::vector new_full_names; + if (!parseRowPolicyNamesAroundON(pos, expected, full_names.empty(), full_names.empty(), allow_on_cluster, new_full_names, cluster)) return false; - num_added_names_last_time = new_name_parts.size(); - boost::range::push_back(name_parts, std::move(new_name_parts)); + num_added_names_last_time = new_full_names.size(); + boost::range::push_back(full_names, std::move(new_full_names)); return true; }; @@ -187,7 +187,7 @@ bool ParserRowPolicyNames::parseImpl(Pos & pos, ASTPtr & node, Expected & expect return false; auto result = std::make_shared(); - result->name_parts = std::move(name_parts); + result->full_names = std::move(full_names); result->cluster = std::move(cluster); node = result; return true; diff --git a/src/Storages/System/StorageSystemRowPolicies.cpp b/src/Storages/System/StorageSystemRowPolicies.cpp index 202ec5078d8..ff59d154704 100644 --- a/src/Storages/System/StorageSystemRowPolicies.cpp +++ b/src/Storages/System/StorageSystemRowPolicies.cpp @@ -82,7 +82,7 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr auto & column_apply_to_except_offsets = assert_cast(*res_columns[column_index++]).getOffsets(); auto add_row = [&](const String & name, - const RowPolicy::NameParts & name_parts, + const RowPolicyName & full_name, const UUID & id, const String & storage_name, const std::array & conditions, @@ -90,9 +90,9 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr const RolesOrUsersSet & apply_to) { column_name.insertData(name.data(), name.length()); - column_short_name.insertData(name_parts.short_name.data(), name_parts.short_name.length()); - column_database.insertData(name_parts.database.data(), name_parts.database.length()); - column_table.insertData(name_parts.table_name.data(), name_parts.table_name.length()); + column_short_name.insertData(full_name.short_name.data(), full_name.short_name.length()); + column_database.insertData(full_name.database.data(), full_name.database.length()); + column_table.insertData(full_name.table_name.data(), full_name.table_name.length()); column_id.push_back(id.toUnderType()); column_storage.insertData(storage_name.data(), storage_name.length()); @@ -134,7 +134,7 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr if (!storage) continue; - add_row(policy->getName(), policy->getNameParts(), id, storage->getStorageName(), policy->conditions, policy->isRestrictive(), policy->to_roles); + add_row(policy->getName(), policy->getFullName(), id, storage->getStorageName(), policy->conditions, policy->isRestrictive(), policy->to_roles); } } } From 33ea7a7262937913b2bd66f45dae9ff876fb9983 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 18 Nov 2021 16:04:42 +0300 Subject: [PATCH 145/200] Rename RowPolicy::ConditionType -> RowPolicyFilterType and move it to Access/Common. --- src/Access/Common/RowPolicyDefs.cpp | 58 +++++++++++ src/Access/Common/RowPolicyDefs.h | 35 +++++++ src/Access/ContextAccess.cpp | 4 +- src/Access/ContextAccess.h | 4 +- src/Access/DiskAccessStorage.cpp | 8 -- src/Access/EnabledRowPolicies.cpp | 32 +++--- src/Access/EnabledRowPolicies.h | 36 +++---- src/Access/RowPolicy.cpp | 2 +- src/Access/RowPolicy.h | 91 +---------------- src/Access/RowPolicyCache.cpp | 97 +++++++++---------- src/Access/RowPolicyCache.h | 9 +- src/Access/UsersConfigAccessStorage.cpp | 2 +- src/Disks/HDFS/DiskHDFS.cpp | 2 + .../InterpreterCreateRowPolicyQuery.cpp | 5 +- .../InterpreterDropAccessEntityQuery.cpp | 7 +- ...InterpreterShowCreateAccessEntityQuery.cpp | 11 ++- src/Interpreters/Context.cpp | 12 +-- src/Interpreters/Context.h | 15 +-- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- src/Interpreters/Session.cpp | 4 +- .../Access/ASTCreateRowPolicyQuery.cpp | 35 +++---- src/Parsers/Access/ASTCreateRowPolicyQuery.h | 4 +- src/Parsers/Access/ASTDropAccessEntityQuery.h | 2 +- src/Parsers/Access/ASTRowPolicyName.cpp | 1 + src/Parsers/Access/ASTRowPolicyName.h | 2 +- .../Access/ParserCreateRowPolicyQuery.cpp | 48 +++++---- src/Parsers/Access/ParserRowPolicyName.h | 1 - src/Storages/PartitionedSink.cpp | 2 + .../WriteBufferToRabbitMQProducer.cpp | 1 + .../System/StorageSystemRowPolicies.cpp | 38 ++++---- 30 files changed, 284 insertions(+), 286 deletions(-) diff --git a/src/Access/Common/RowPolicyDefs.cpp b/src/Access/Common/RowPolicyDefs.cpp index aa1d1971d7b..953709d9519 100644 --- a/src/Access/Common/RowPolicyDefs.cpp +++ b/src/Access/Common/RowPolicyDefs.cpp @@ -1,9 +1,15 @@ #include +#include #include +#include namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} String RowPolicyName::toString() const { @@ -20,4 +26,56 @@ String RowPolicyName::toString() const return name; } +String toString(RowPolicyFilterType type) +{ + return RowPolicyFilterTypeInfo::get(type).raw_name; +} + +const RowPolicyFilterTypeInfo & RowPolicyFilterTypeInfo::get(RowPolicyFilterType type_) +{ + static constexpr auto make_info = [](const char * raw_name_) + { + String init_name = raw_name_; + boost::to_lower(init_name); + size_t underscore_pos = init_name.find('_'); + String init_command = init_name.substr(0, underscore_pos); + boost::to_upper(init_command); + bool init_is_check = (std::string_view{init_name}.substr(underscore_pos + 1) == "check"); + return RowPolicyFilterTypeInfo{raw_name_, std::move(init_name), std::move(init_command), init_is_check}; + }; + + switch (type_) + { + case RowPolicyFilterType::SELECT_FILTER: + { + static const auto info = make_info("SELECT_FILTER"); + return info; + } +#if 0 /// Row-level security for INSERT, UPDATE, DELETE is not implemented yet. + case RowPolicyFilterType::INSERT_CHECK: + { + static const auto info = make_info("INSERT_CHECK"); + return info; + } + case RowPolicyFilterType::UPDATE_FILTER: + { + static const auto info = make_info("UPDATE_FILTER"); + return info; + } + case RowPolicyFilterType::UPDATE_CHECK: + { + static const auto info = make_info("UPDATE_CHECK"); + return info; + } + case RowPolicyFilterType::DELETE_FILTER: + { + static const auto info = make_info("DELETE_FILTER"); + return info; + } +#endif + case RowPolicyFilterType::MAX: break; + } + throw Exception("Unknown type: " + std::to_string(static_cast(type_)), ErrorCodes::LOGICAL_ERROR); +} + } diff --git a/src/Access/Common/RowPolicyDefs.h b/src/Access/Common/RowPolicyDefs.h index 771bb2ed738..792884c56df 100644 --- a/src/Access/Common/RowPolicyDefs.h +++ b/src/Access/Common/RowPolicyDefs.h @@ -6,6 +6,7 @@ namespace DB { +/// Represents the full name of a row policy, e.g. "myfilter ON mydb.mytable". struct RowPolicyName { String short_name; @@ -19,4 +20,38 @@ struct RowPolicyName friend bool operator !=(const RowPolicyName & left, const RowPolicyName & right) { return left.toTuple() != right.toTuple(); } }; + +/// Types of the filters of row policies. +/// Currently only RowPolicyFilterType::SELECT is supported. +enum class RowPolicyFilterType +{ + /// Filter is a SQL conditional expression used to figure out which rows should be visible + /// for user or available for modification. If the expression returns NULL or false for some rows + /// those rows are silently suppressed. + SELECT_FILTER, + +#if 0 /// Row-level security for INSERT, UPDATE, DELETE is not implemented yet. + /// Check is a SQL condition expression used to check whether a row can be written into + /// the table. If the expression returns NULL or false an exception is thrown. + /// If a conditional expression here is empty it means no filtering is applied. + INSERT_CHECK, + UPDATE_FILTER, + UPDATE_CHECK, + DELETE_FILTER, +#endif + + MAX +}; + +String toString(RowPolicyFilterType type); + +struct RowPolicyFilterTypeInfo +{ + const char * const raw_name; + const String name; /// Lowercased with underscores, e.g. "select_filter". + const String command; /// Uppercased without last word, e.g. "SELECT". + const bool is_check; /// E.g. false for SELECT_FILTER. + static const RowPolicyFilterTypeInfo & get(RowPolicyFilterType type); +}; + } diff --git a/src/Access/ContextAccess.cpp b/src/Access/ContextAccess.cpp index e9164b4ae44..b254a59376d 100644 --- a/src/Access/ContextAccess.cpp +++ b/src/Access/ContextAccess.cpp @@ -269,11 +269,11 @@ std::shared_ptr ContextAccess::getEnabledRowPolicies() return no_row_policies; } -ASTPtr ContextAccess::getRowPolicyCondition(const String & database, const String & table_name, RowPolicy::ConditionType index, const ASTPtr & extra_condition) const +ASTPtr ContextAccess::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const ASTPtr & combine_with_expr) const { std::lock_guard lock{mutex}; if (enabled_row_policies) - return enabled_row_policies->getCondition(database, table_name, index, extra_condition); + return enabled_row_policies->getFilter(database, table_name, filter_type, combine_with_expr); return nullptr; } diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index a7c91faf43b..05d532ae6cb 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include #include @@ -81,7 +81,7 @@ public: /// Returns the row policy filter for a specified table. /// The function returns nullptr if there is no filter to apply. - ASTPtr getRowPolicyCondition(const String & database, const String & table_name, RowPolicy::ConditionType index, const ASTPtr & extra_condition = nullptr) const; + ASTPtr getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const ASTPtr & combine_with_expr = nullptr) const; /// Returns the quota to track resource consumption. std::shared_ptr getQuota() const; diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 247bcc1ee89..4c53ba8b632 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -1,19 +1,11 @@ #include #include -#include -#include -#include -#include -#include #include #include #include #include -#include #include #include -#include -#include #include #include #include diff --git a/src/Access/EnabledRowPolicies.cpp b/src/Access/EnabledRowPolicies.cpp index 674dab3e0f0..e4f592884fe 100644 --- a/src/Access/EnabledRowPolicies.cpp +++ b/src/Access/EnabledRowPolicies.cpp @@ -6,9 +6,9 @@ namespace DB { -size_t EnabledRowPolicies::Hash::operator()(const MixedConditionKey & key) const +size_t EnabledRowPolicies::Hash::operator()(const MixedFiltersKey & key) const { - return std::hash{}(key.database) - std::hash{}(key.table_name) + static_cast(key.condition_type); + return std::hash{}(key.database) - std::hash{}(key.table_name) + static_cast(key.filter_type); } @@ -23,36 +23,36 @@ EnabledRowPolicies::EnabledRowPolicies(const Params & params_) : params(params_) EnabledRowPolicies::~EnabledRowPolicies() = default; -ASTPtr EnabledRowPolicies::getCondition(const String & database, const String & table_name, ConditionType condition_type) const +ASTPtr EnabledRowPolicies::getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const { /// We don't lock `mutex` here. - auto loaded = map_of_mixed_conditions.load(); - auto it = loaded->find({database, table_name, condition_type}); + auto loaded = mixed_filters.load(); + auto it = loaded->find({database, table_name, filter_type}); if (it == loaded->end()) return {}; - auto condition = it->second.ast; + auto filter = it->second.ast; bool value; - if (tryGetLiteralBool(condition.get(), value) && value) + if (tryGetLiteralBool(filter.get(), value) && value) return nullptr; /// The condition is always true, no need to check it. - return condition; + return filter; } -ASTPtr EnabledRowPolicies::getCondition(const String & database, const String & table_name, ConditionType type, const ASTPtr & extra_condition) const +ASTPtr EnabledRowPolicies::getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const ASTPtr & combine_with_expr) const { - ASTPtr condition = getCondition(database, table_name, type); - if (condition && extra_condition) - condition = makeASTForLogicalAnd({condition, extra_condition}); - else if (!condition) - condition = extra_condition; + ASTPtr filter = getFilter(database, table_name, filter_type); + if (filter && combine_with_expr) + filter = makeASTForLogicalAnd({filter, combine_with_expr}); + else if (!filter) + filter = combine_with_expr; bool value; - if (tryGetLiteralBool(condition.get(), value) && value) + if (tryGetLiteralBool(filter.get(), value) && value) return nullptr; /// The condition is always true, no need to check it. - return condition; + return filter; } } diff --git a/src/Access/EnabledRowPolicies.h b/src/Access/EnabledRowPolicies.h index 8ed923e98e4..7fb4e707273 100644 --- a/src/Access/EnabledRowPolicies.h +++ b/src/Access/EnabledRowPolicies.h @@ -1,8 +1,9 @@ #pragma once -#include +#include #include #include +#include #include #include #include @@ -35,43 +36,42 @@ public: EnabledRowPolicies(); ~EnabledRowPolicies(); - using ConditionType = RowPolicy::ConditionType; - /// Returns prepared filter for a specific table and operations. /// The function can return nullptr, that means there is no filters applied. /// The returned filter can be a combination of the filters defined by multiple row policies. - ASTPtr getCondition(const String & database, const String & table_name, ConditionType type) const; - ASTPtr getCondition(const String & database, const String & table_name, ConditionType type, const ASTPtr & extra_condition) const; + ASTPtr getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const; + ASTPtr getFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type, const ASTPtr & combine_with_expr) const; private: friend class RowPolicyCache; EnabledRowPolicies(const Params & params_); - struct MixedConditionKey + struct MixedFiltersKey { std::string_view database; std::string_view table_name; - ConditionType condition_type; + RowPolicyFilterType filter_type; - auto toTuple() const { return std::tie(database, table_name, condition_type); } - friend bool operator==(const MixedConditionKey & left, const MixedConditionKey & right) { return left.toTuple() == right.toTuple(); } - friend bool operator!=(const MixedConditionKey & left, const MixedConditionKey & right) { return left.toTuple() != right.toTuple(); } + auto toTuple() const { return std::tie(database, table_name, filter_type); } + friend bool operator==(const MixedFiltersKey & left, const MixedFiltersKey & right) { return left.toTuple() == right.toTuple(); } + friend bool operator!=(const MixedFiltersKey & left, const MixedFiltersKey & right) { return left.toTuple() != right.toTuple(); } }; - struct Hash - { - size_t operator()(const MixedConditionKey & key) const; - }; - - struct MixedCondition + struct MixedFiltersResult { ASTPtr ast; std::shared_ptr> database_and_table_name; }; - using MapOfMixedConditions = std::unordered_map; + + struct Hash + { + size_t operator()(const MixedFiltersKey & key) const; + }; + + using MixedFiltersMap = std::unordered_map; const Params params; - mutable boost::atomic_shared_ptr map_of_mixed_conditions; + mutable boost::atomic_shared_ptr mixed_filters; }; } diff --git a/src/Access/RowPolicy.cpp b/src/Access/RowPolicy.cpp index 3ce201f000b..c09675e0e34 100644 --- a/src/Access/RowPolicy.cpp +++ b/src/Access/RowPolicy.cpp @@ -54,7 +54,7 @@ bool RowPolicy::equal(const IAccessEntity & other) const if (!IAccessEntity::equal(other)) return false; const auto & other_policy = typeid_cast(other); - return (full_name == other_policy.full_name) && boost::range::equal(conditions, other_policy.conditions) + return (full_name == other_policy.full_name) && boost::range::equal(filters, other_policy.filters) && restrictive == other_policy.restrictive && (to_roles == other_policy.to_roles); } diff --git a/src/Access/RowPolicy.h b/src/Access/RowPolicy.h index 3e0a1bcb662..6cbf4196352 100644 --- a/src/Access/RowPolicy.h +++ b/src/Access/RowPolicy.h @@ -9,11 +9,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - /** Represents a row level security policy for a table. */ @@ -30,36 +25,9 @@ struct RowPolicy : public IAccessEntity const String & getShortName() const { return full_name.short_name; } const RowPolicyName & getFullName() const { return full_name; } - /// Filter is a SQL conditional expression used to figure out which rows should be visible - /// for user or available for modification. If the expression returns NULL or false for some rows - /// those rows are silently suppressed. - /// Check is a SQL condition expression used to check whether a row can be written into - /// the table. If the expression returns NULL or false an exception is thrown. - /// If a conditional expression here is empty it means no filtering is applied. - enum ConditionType - { - SELECT_FILTER, - -#if 0 /// Row-level security for INSERT, UPDATE, DELETE is not implemented yet. - INSERT_CHECK, - UPDATE_FILTER, - UPDATE_CHECK, - DELETE_FILTER, -#endif - - MAX_CONDITION_TYPE - }; - - struct ConditionTypeInfo - { - const char * const raw_name; - const String name; /// Lowercased with underscores, e.g. "select_filter". - const String command; /// Uppercased without last word, e.g. "SELECT". - const bool is_check; /// E.g. false for SELECT_FILTER. - static const ConditionTypeInfo & get(ConditionType type); - }; - - std::array conditions; + /// A SQL conditional expression used to figure out which rows should be visible + /// for user or available for modification. + std::array(RowPolicyFilterType::MAX)> filters; /// Sets that the policy is permissive. /// A row is only accessible if at least one of the permissive policies passes, @@ -90,57 +58,4 @@ private: using RowPolicyPtr = std::shared_ptr; - -inline const RowPolicy::ConditionTypeInfo & RowPolicy::ConditionTypeInfo::get(ConditionType type_) -{ - static constexpr auto make_info = [](const char * raw_name_) - { - String init_name = raw_name_; - boost::to_lower(init_name); - size_t underscore_pos = init_name.find('_'); - String init_command = init_name.substr(0, underscore_pos); - boost::to_upper(init_command); - bool init_is_check = (std::string_view{init_name}.substr(underscore_pos + 1) == "check"); - return ConditionTypeInfo{raw_name_, std::move(init_name), std::move(init_command), init_is_check}; - }; - - switch (type_) - { - case SELECT_FILTER: - { - static const ConditionTypeInfo info = make_info("SELECT_FILTER"); - return info; - } -#if 0 /// Row-level security for INSERT, UPDATE, DELETE is not implemented yet. - case INSERT_CHECK: - { - static const ConditionTypeInfo info = make_info("INSERT_CHECK"); - return info; - } - case UPDATE_FILTER: - { - static const ConditionTypeInfo info = make_info("UPDATE_FILTER"); - return info; - } - case UPDATE_CHECK: - { - static const ConditionTypeInfo info = make_info("UPDATE_CHECK"); - return info; - } - case DELETE_FILTER: - { - static const ConditionTypeInfo info = make_info("DELETE_FILTER"); - return info; - } -#endif - case MAX_CONDITION_TYPE: break; - } - throw Exception("Unknown type: " + std::to_string(static_cast(type_)), ErrorCodes::LOGICAL_ERROR); -} - -inline String toString(RowPolicy::ConditionType type) -{ - return RowPolicy::ConditionTypeInfo::get(type).raw_name; -} - } diff --git a/src/Access/RowPolicyCache.cpp b/src/Access/RowPolicyCache.cpp index b5b6dd99438..55bec427158 100644 --- a/src/Access/RowPolicyCache.cpp +++ b/src/Access/RowPolicyCache.cpp @@ -1,6 +1,7 @@ #include -#include #include +#include +#include #include #include #include @@ -15,35 +16,31 @@ namespace DB { namespace { - using ConditionType = RowPolicy::ConditionType; - constexpr auto MAX_CONDITION_TYPE = RowPolicy::MAX_CONDITION_TYPE; - - - /// Accumulates conditions from multiple row policies and joins them using the AND logical operation. - class ConditionsMixer + /// Accumulates filters from multiple row policies and joins them using the AND logical operation. + class FiltersMixer { public: - void add(const ASTPtr & condition, bool is_restrictive) + void add(const ASTPtr & filter, bool is_restrictive) { if (is_restrictive) - restrictions.push_back(condition); + restrictions.push_back(filter); else - permissions.push_back(condition); + permissions.push_back(filter); } ASTPtr getResult() && { - /// Process permissive conditions. + /// Process permissive filters. restrictions.push_back(makeASTForLogicalOr(std::move(permissions))); - /// Process restrictive conditions. - auto condition = makeASTForLogicalAnd(std::move(restrictions)); + /// Process restrictive filters. + auto result = makeASTForLogicalAnd(std::move(restrictions)); bool value; - if (tryGetLiteralBool(condition.get(), value) && value) - condition = nullptr; /// The condition is always true, no need to check it. + if (tryGetLiteralBool(result.get(), value) && value) + result = nullptr; /// The condition is always true, no need to check it. - return condition; + return result; } private: @@ -59,33 +56,34 @@ void RowPolicyCache::PolicyInfo::setPolicy(const RowPolicyPtr & policy_) roles = &policy->to_roles; database_and_table_name = std::make_shared>(policy->getDatabase(), policy->getTableName()); - for (auto type : collections::range(0, MAX_CONDITION_TYPE)) + for (auto filter_type : collections::range(0, RowPolicyFilterType::MAX)) { - parsed_conditions[type] = nullptr; - const String & condition = policy->conditions[type]; - if (condition.empty()) + auto filter_type_i = static_cast(filter_type); + parsed_filters[filter_type_i] = nullptr; + const String & filter = policy->filters[filter_type_i]; + if (filter.empty()) continue; - auto previous_range = std::pair(std::begin(policy->conditions), std::begin(policy->conditions) + type); - const auto * previous_it = std::find(previous_range.first, previous_range.second, condition); + auto previous_range = std::pair(std::begin(policy->filters), std::begin(policy->filters) + filter_type_i); + const auto * previous_it = std::find(previous_range.first, previous_range.second, filter); if (previous_it != previous_range.second) { - /// The condition is already parsed before. - parsed_conditions[type] = parsed_conditions[previous_it - previous_range.first]; + /// The filter is already parsed before. + parsed_filters[filter_type_i] = parsed_filters[previous_it - previous_range.first]; continue; } - /// Try to parse the condition. + /// Try to parse the filter. try { ParserExpression parser; - parsed_conditions[type] = parseQuery(parser, condition, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); + parsed_filters[filter_type_i] = parseQuery(parser, filter, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); } catch (...) { tryLogCurrentException( &Poco::Logger::get("RowPolicy"), - String("Could not parse the condition ") + toString(type) + " of row policy " + String("Could not parse the condition ") + toString(filter_type) + " of row policy " + backQuote(policy->getName())); } } @@ -119,7 +117,7 @@ std::shared_ptr RowPolicyCache::getEnabledRowPolicies( auto res = std::shared_ptr(new EnabledRowPolicies(params)); enabled_row_policies.emplace(std::move(params), res); - mixConditionsFor(*res); + mixFiltersFor(*res); return res; } @@ -165,7 +163,7 @@ void RowPolicyCache::rowPolicyAddedOrChanged(const UUID & policy_id, const RowPo auto & info = it->second; info.setPolicy(new_policy); - mixConditions(); + mixFilters(); } @@ -173,11 +171,11 @@ void RowPolicyCache::rowPolicyRemoved(const UUID & policy_id) { std::lock_guard lock{mutex}; all_policies.erase(policy_id); - mixConditions(); + mixFilters(); } -void RowPolicyCache::mixConditions() +void RowPolicyCache::mixFilters() { /// `mutex` is already locked. for (auto i = enabled_row_policies.begin(), e = enabled_row_policies.end(); i != e;) @@ -187,58 +185,59 @@ void RowPolicyCache::mixConditions() i = enabled_row_policies.erase(i); else { - mixConditionsFor(*elem); + mixFiltersFor(*elem); ++i; } } } -void RowPolicyCache::mixConditionsFor(EnabledRowPolicies & enabled) +void RowPolicyCache::mixFiltersFor(EnabledRowPolicies & enabled) { /// `mutex` is already locked. - using MapOfMixedConditions = EnabledRowPolicies::MapOfMixedConditions; - using MixedConditionKey = EnabledRowPolicies::MixedConditionKey; + using MixedFiltersMap = EnabledRowPolicies::MixedFiltersMap; + using MixedFiltersKey = EnabledRowPolicies::MixedFiltersKey; using Hash = EnabledRowPolicies::Hash; struct MixerWithNames { - ConditionsMixer mixer; + FiltersMixer mixer; std::shared_ptr> database_and_table_name; }; - std::unordered_map map_of_mixers; + std::unordered_map mixers; for (const auto & [policy_id, info] : all_policies) { const auto & policy = *info.policy; bool match = info.roles->match(enabled.params.user_id, enabled.params.enabled_roles); - MixedConditionKey key; + MixedFiltersKey key; key.database = info.database_and_table_name->first; key.table_name = info.database_and_table_name->second; - for (auto type : collections::range(0, MAX_CONDITION_TYPE)) + for (auto filter_type : collections::range(0, RowPolicyFilterType::MAX)) { - if (info.parsed_conditions[type]) + auto filter_type_i = static_cast(filter_type); + if (info.parsed_filters[filter_type_i]) { - key.condition_type = type; - auto & mixer = map_of_mixers[key]; + key.filter_type = filter_type; + auto & mixer = mixers[key]; mixer.database_and_table_name = info.database_and_table_name; if (match) - mixer.mixer.add(info.parsed_conditions[type], policy.isRestrictive()); + mixer.mixer.add(info.parsed_filters[filter_type_i], policy.isRestrictive()); } } } - auto map_of_mixed_conditions = boost::make_shared(); - for (auto & [key, mixer] : map_of_mixers) + auto mixed_filters = boost::make_shared(); + for (auto & [key, mixer] : mixers) { - auto & mixed_condition = (*map_of_mixed_conditions)[key]; - mixed_condition.database_and_table_name = mixer.database_and_table_name; - mixed_condition.ast = std::move(mixer.mixer).getResult(); + auto & mixed_filter = (*mixed_filters)[key]; + mixed_filter.database_and_table_name = mixer.database_and_table_name; + mixed_filter.ast = std::move(mixer.mixer).getResult(); } - enabled.map_of_mixed_conditions.store(map_of_mixed_conditions); + enabled.mixed_filters.store(mixed_filters); } } diff --git a/src/Access/RowPolicyCache.h b/src/Access/RowPolicyCache.h index 6834def58b6..dc416fe59f0 100644 --- a/src/Access/RowPolicyCache.h +++ b/src/Access/RowPolicyCache.h @@ -10,6 +10,9 @@ namespace DB { class AccessControl; +struct RolesOrUsersSet; +struct RowPolicy; +using RowPolicyPtr = std::shared_ptr; /// Stores read and parsed row policies. class RowPolicyCache @@ -29,14 +32,14 @@ private: RowPolicyPtr policy; const RolesOrUsersSet * roles = nullptr; std::shared_ptr> database_and_table_name; - ASTPtr parsed_conditions[RowPolicy::MAX_CONDITION_TYPE]; + ASTPtr parsed_filters[static_cast(RowPolicyFilterType::MAX)]; }; void ensureAllRowPoliciesRead(); void rowPolicyAddedOrChanged(const UUID & policy_id, const RowPolicyPtr & new_policy); void rowPolicyRemoved(const UUID & policy_id); - void mixConditions(); - void mixConditionsFor(EnabledRowPolicies & enabled); + void mixFilters(); + void mixFiltersFor(EnabledRowPolicies & enabled); const AccessControl & access_control; std::unordered_map all_policies; diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index 41a64d7f7eb..70127b22998 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -352,7 +352,7 @@ namespace auto policy = std::make_shared(); policy->setFullName(user_name, database, table_name); - policy->conditions[RowPolicy::SELECT_FILTER] = filter; + policy->filters[static_cast(RowPolicyFilterType::SELECT_FILTER)] = filter; policy->to_roles.add(generateID(EntityType::USER, user_name)); policies.push_back(policy); } diff --git a/src/Disks/HDFS/DiskHDFS.cpp b/src/Disks/HDFS/DiskHDFS.cpp index 591b0bf0347..5264e6413e7 100644 --- a/src/Disks/HDFS/DiskHDFS.cpp +++ b/src/Disks/HDFS/DiskHDFS.cpp @@ -10,6 +10,8 @@ #include #include +#include + #include #include diff --git a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp index 7abe8bbf000..c88e9c299a8 100644 --- a/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateRowPolicyQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -30,8 +31,8 @@ namespace if (query.is_restrictive) policy.setRestrictive(*query.is_restrictive); - for (const auto & [condition_type, condition] : query.conditions) - policy.conditions[condition_type] = condition ? serializeAST(*condition) : String{}; + for (const auto & [filter_type, filter] : query.filters) + policy.filters[static_cast(filter_type)] = filter ? serializeAST(*filter) : String{}; if (override_to_roles) policy.to_roles = *override_to_roles; diff --git a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp index c00bbe4f379..4cf8216931d 100644 --- a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp @@ -2,12 +2,7 @@ #include #include #include -#include -#include -#include -#include -#include -#include +#include #include #include diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index 2cef0a256cf..901025b5227 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include #include #include @@ -188,14 +189,14 @@ namespace if (policy.isRestrictive()) query->is_restrictive = policy.isRestrictive(); - for (auto type : collections::range(RowPolicy::MAX_CONDITION_TYPE)) + for (auto type : collections::range(RowPolicyFilterType::MAX)) { - const auto & condition = policy.conditions[static_cast(type)]; - if (!condition.empty()) + const auto & filter = policy.filters[static_cast(type)]; + if (!filter.empty()) { ParserExpression parser; - ASTPtr expr = parseQuery(parser, condition, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); - query->conditions.emplace_back(type, std::move(expr)); + ASTPtr expr = parseQuery(parser, filter, 0, DBMS_DEFAULT_MAX_PARSER_DEPTH); + query->filters.emplace_back(type, std::move(expr)); } } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index ab07a8ed1ad..9bf66a6ac9c 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -773,23 +773,23 @@ std::shared_ptr Context::getAccess() const return access ? access : ContextAccess::getFullAccess(); } -ASTPtr Context::getRowPolicyCondition(const String & database, const String & table_name, RowPolicy::ConditionType type) const +ASTPtr Context::getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const { auto lock = getLock(); - auto initial_condition = initial_row_policy ? initial_row_policy->getCondition(database, table_name, type) : nullptr; - return getAccess()->getRowPolicyCondition(database, table_name, type, initial_condition); + auto row_filter_of_initial_user = row_policies_of_initial_user ? row_policies_of_initial_user->getFilter(database, table_name, filter_type) : nullptr; + return getAccess()->getRowPolicyFilter(database, table_name, filter_type, row_filter_of_initial_user); } -void Context::setInitialRowPolicy() +void Context::enableRowPoliciesOfInitialUser() { auto lock = getLock(); - initial_row_policy = nullptr; + row_policies_of_initial_user = nullptr; if (client_info.initial_user == client_info.current_user) return; auto initial_user_id = getAccessControl().find(client_info.initial_user); if (!initial_user_id) return; - initial_row_policy = getAccessControl().tryGetDefaultRowPolicies(*initial_user_id); + row_policies_of_initial_user = getAccessControl().tryGetDefaultRowPolicies(*initial_user_id); } diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 0dd34db8690..5948cc7f7a7 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include #include @@ -18,6 +17,7 @@ #include "config_core.h" +#include #include #include #include @@ -43,6 +43,7 @@ struct QuotaUsage; class AccessFlags; struct AccessRightsElement; class AccessRightsElements; +enum class RowPolicyFilterType; class EmbeddedDictionaries; class ExternalDictionariesLoader; class ExternalModelsLoader; @@ -195,7 +196,7 @@ private: std::shared_ptr> current_roles; std::shared_ptr settings_constraints_and_current_profiles; std::shared_ptr access; - std::shared_ptr initial_row_policy; + std::shared_ptr row_policies_of_initial_user; String current_database; Settings settings; /// Setting for query execution. @@ -414,12 +415,14 @@ public: std::shared_ptr getAccess() const; - ASTPtr getRowPolicyCondition(const String & database, const String & table_name, RowPolicy::ConditionType type) const; + ASTPtr getRowPolicyFilter(const String & database, const String & table_name, RowPolicyFilterType filter_type) const; - /// Sets an extra row policy based on `client_info.initial_user`, if it exists. + /// Finds and sets extra row policies to be used based on `client_info.initial_user`, + /// if the initial user exists. /// TODO: we need a better solution here. It seems we should pass the initial row policy - /// because a shard is allowed to don't have the initial user or it may be another user with the same name. - void setInitialRowPolicy(); + /// because a shard is allowed to not have the initial user or it might be another user + /// with the same name. + void enableRowPoliciesOfInitialUser(); std::shared_ptr getQuota() const; std::optional getQuotaUsage() const; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f236d4ae6f9..c8f48f2ed1f 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -358,7 +358,7 @@ InterpreterSelectQuery::InterpreterSelectQuery( std::shared_ptr table_join = joined_tables.makeTableJoin(query); if (storage) - row_policy_filter = context->getRowPolicyCondition(table_id.getDatabaseName(), table_id.getTableName(), RowPolicy::SELECT_FILTER); + row_policy_filter = context->getRowPolicyFilter(table_id.getDatabaseName(), table_id.getTableName(), RowPolicyFilterType::SELECT_FILTER); StorageView * view = nullptr; if (storage) diff --git a/src/Interpreters/Session.cpp b/src/Interpreters/Session.cpp index 39d2abc9b43..c26d8b52049 100644 --- a/src/Interpreters/Session.cpp +++ b/src/Interpreters/Session.cpp @@ -468,8 +468,8 @@ ContextMutablePtr Session::makeQueryContextImpl(const ClientInfo * client_info_t res_client_info.initial_address = res_client_info.current_address; } - /// Sets that row policies from the initial user should be used too. - query_context->setInitialRowPolicy(); + /// Sets that row policies of the initial user should be used too. + query_context->enableRowPoliciesOfInitialUser(); /// Set user information for the new context: current profiles, roles, access rights. if (user_id && !query_context->getUser()) diff --git a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp index 0267379d6e5..d968fdd3250 100644 --- a/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include @@ -12,10 +13,6 @@ namespace DB { namespace { - using ConditionType = RowPolicy::ConditionType; - using ConditionTypeInfo = RowPolicy::ConditionTypeInfo; - - void formatRenameTo(const String & new_short_name, const IAST::FormatSettings & settings) { settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " RENAME TO " << (settings.hilite ? IAST::hilite_none : "") @@ -30,7 +27,7 @@ namespace } - void formatConditionalExpression(const ASTPtr & expr, const IAST::FormatSettings & settings) + void formatFilterExpression(const ASTPtr & expr, const IAST::FormatSettings & settings) { settings.ostr << " "; if (expr) @@ -59,15 +56,15 @@ namespace } - void formatForClauses(const std::vector> & conditions, bool alter, const IAST::FormatSettings & settings) + void formatForClauses(const std::vector> & filters, bool alter, const IAST::FormatSettings & settings) { - std::vector> conditions_as_strings; + std::vector> filters_as_strings; WriteBufferFromOwnString temp_buf; IAST::FormatSettings temp_settings(temp_buf, settings); - for (const auto & [condition_type, condition] : conditions) + for (const auto & [filter_type, filter] : filters) { - formatConditionalExpression(condition, temp_settings); - conditions_as_strings.emplace_back(condition_type, temp_buf.str()); + formatFilterExpression(filter, temp_settings); + filters_as_strings.emplace_back(filter_type, temp_buf.str()); temp_buf.restart(); } @@ -81,27 +78,27 @@ namespace check.clear(); /// Collect commands using the same filter and check conditions. - for (auto & [condition_type, condition] : conditions_as_strings) + for (auto & [filter_type, str] : filters_as_strings) { - if (condition.empty()) + if (str.empty()) continue; - const auto & type_info = ConditionTypeInfo::get(condition_type); + const auto & type_info = RowPolicyFilterTypeInfo::get(filter_type); if (type_info.is_check) { if (check.empty()) - check = condition; - else if (check != condition) + check = str; + else if (check != str) continue; } else { if (filter.empty()) - filter = condition; - else if (filter != condition) + filter = str; + else if (filter != str) continue; } commands.emplace(type_info.command); - condition.clear(); /// Skip this condition on the next iteration. + str.clear(); /// Skip this condition on the next iteration. } if (!filter.empty() || !check.empty()) @@ -162,7 +159,7 @@ void ASTCreateRowPolicyQuery::formatImpl(const FormatSettings & settings, Format if (is_restrictive) formatAsRestrictiveOrPermissive(*is_restrictive, settings); - formatForClauses(conditions, alter, settings); + formatForClauses(filters, alter, settings); if (roles && (!roles->empty() || alter)) formatToRoles(*roles, settings); diff --git a/src/Parsers/Access/ASTCreateRowPolicyQuery.h b/src/Parsers/Access/ASTCreateRowPolicyQuery.h index 46a7578726e..dc698c25c6d 100644 --- a/src/Parsers/Access/ASTCreateRowPolicyQuery.h +++ b/src/Parsers/Access/ASTCreateRowPolicyQuery.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include @@ -40,7 +40,7 @@ public: String new_short_name; std::optional is_restrictive; - std::vector> conditions; /// `nullptr` means set to NONE. + std::vector> filters; /// `nullptr` means set to NONE. std::shared_ptr roles; diff --git a/src/Parsers/Access/ASTDropAccessEntityQuery.h b/src/Parsers/Access/ASTDropAccessEntityQuery.h index df78acef6f4..401c3d3f5f0 100644 --- a/src/Parsers/Access/ASTDropAccessEntityQuery.h +++ b/src/Parsers/Access/ASTDropAccessEntityQuery.h @@ -1,8 +1,8 @@ #pragma once #include -#include #include +#include namespace DB diff --git a/src/Parsers/Access/ASTRowPolicyName.cpp b/src/Parsers/Access/ASTRowPolicyName.cpp index 8b49677c506..280713fe9d9 100644 --- a/src/Parsers/Access/ASTRowPolicyName.cpp +++ b/src/Parsers/Access/ASTRowPolicyName.cpp @@ -1,4 +1,5 @@ #include +#include #include diff --git a/src/Parsers/Access/ASTRowPolicyName.h b/src/Parsers/Access/ASTRowPolicyName.h index 8bad6597bb8..43270b0185d 100644 --- a/src/Parsers/Access/ASTRowPolicyName.h +++ b/src/Parsers/Access/ASTRowPolicyName.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB diff --git a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp index dace8328fea..dd01f7d024d 100644 --- a/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp +++ b/src/Parsers/Access/ParserCreateRowPolicyQuery.cpp @@ -9,8 +9,9 @@ #include #include #include -#include +#include #include +#include #include @@ -18,11 +19,6 @@ namespace DB { namespace { - using ConditionType = RowPolicy::ConditionType; - using ConditionTypeInfo = RowPolicy::ConditionTypeInfo; - constexpr auto MAX_CONDITION_TYPE = RowPolicy::MAX_CONDITION_TYPE; - - bool parseRenameTo(IParserBase::Pos & pos, Expected & expected, String & new_short_name) { return IParserBase::wrapParseImpl(pos, [&] @@ -55,7 +51,7 @@ namespace }); } - bool parseConditionalExpression(IParserBase::Pos & pos, Expected & expected, ASTPtr & expr) + bool parseFilterExpression(IParserBase::Pos & pos, Expected & expected, ASTPtr & expr) { return IParserBase::wrapParseImpl(pos, [&] { @@ -78,9 +74,9 @@ namespace void addAllCommands(boost::container::flat_set & commands) { - for (auto condition_type : collections::range(MAX_CONDITION_TYPE)) + for (auto filter_type : collections::range(RowPolicyFilterType::MAX)) { - const std::string_view & command = ConditionTypeInfo::get(condition_type).command; + const std::string_view & command = RowPolicyFilterTypeInfo::get(filter_type).command; commands.emplace(command); } } @@ -99,9 +95,9 @@ namespace return true; } - for (auto condition_type : collections::range(MAX_CONDITION_TYPE)) + for (auto filter_type : collections::range(RowPolicyFilterType::MAX)) { - const std::string_view & command = ConditionTypeInfo::get(condition_type).command; + const std::string_view & command = RowPolicyFilterTypeInfo::get(filter_type).command; if (ParserKeyword{command.data()}.ignore(pos, expected)) { res_commands.emplace(command); @@ -120,10 +116,10 @@ namespace } - bool - parseForClauses(IParserBase::Pos & pos, Expected & expected, bool alter, std::vector> & conditions) + bool parseForClauses( + IParserBase::Pos & pos, Expected & expected, bool alter, std::vector> & filters) { - std::vector> res_conditions; + std::vector> res_filters; auto parse_for_clause = [&] { @@ -141,12 +137,12 @@ namespace std::optional check; if (ParserKeyword{"USING"}.ignore(pos, expected)) { - if (!parseConditionalExpression(pos, expected, filter.emplace())) + if (!parseFilterExpression(pos, expected, filter.emplace())) return false; } if (ParserKeyword{"WITH CHECK"}.ignore(pos, expected)) { - if (!parseConditionalExpression(pos, expected, check.emplace())) + if (!parseFilterExpression(pos, expected, check.emplace())) return false; } @@ -156,15 +152,15 @@ namespace if (!check && !alter) check = filter; - for (auto condition_type : collections::range(MAX_CONDITION_TYPE)) + for (auto filter_type : collections::range(RowPolicyFilterType::MAX)) { - const auto & type_info = ConditionTypeInfo::get(condition_type); + const auto & type_info = RowPolicyFilterTypeInfo::get(filter_type); if (commands.count(type_info.command)) { if (type_info.is_check && check) - res_conditions.emplace_back(condition_type, *check); + res_filters.emplace_back(filter_type, *check); else if (filter) - res_conditions.emplace_back(condition_type, *filter); + res_filters.emplace_back(filter_type, *filter); } } @@ -174,7 +170,7 @@ namespace if (!ParserList::parseUtil(pos, expected, parse_for_clause, false)) return false; - conditions = std::move(res_conditions); + filters = std::move(res_filters); return true; } @@ -249,7 +245,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & String new_short_name; std::optional is_restrictive; - std::vector> conditions; + std::vector> filters; while (true) { @@ -266,10 +262,10 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & } } - std::vector> new_conditions; - if (parseForClauses(pos, expected, alter, new_conditions)) + std::vector> new_filters; + if (parseForClauses(pos, expected, alter, new_filters)) { - boost::range::push_back(conditions, std::move(new_conditions)); + boost::range::push_back(filters, std::move(new_filters)); continue; } @@ -297,7 +293,7 @@ bool ParserCreateRowPolicyQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & query->names = std::move(names); query->new_short_name = std::move(new_short_name); query->is_restrictive = is_restrictive; - query->conditions = std::move(conditions); + query->filters = std::move(filters); query->roles = std::move(roles); return true; diff --git a/src/Parsers/Access/ParserRowPolicyName.h b/src/Parsers/Access/ParserRowPolicyName.h index 6af0519d161..d311469faa5 100644 --- a/src/Parsers/Access/ParserRowPolicyName.h +++ b/src/Parsers/Access/ParserRowPolicyName.h @@ -1,7 +1,6 @@ #pragma once #include -#include namespace DB diff --git a/src/Storages/PartitionedSink.cpp b/src/Storages/PartitionedSink.cpp index 1d13cbe5f94..5e8f2a9e132 100644 --- a/src/Storages/PartitionedSink.cpp +++ b/src/Storages/PartitionedSink.cpp @@ -13,6 +13,8 @@ #include +#include + namespace DB { diff --git a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp index f6b12708e81..9b4f32d93df 100644 --- a/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp +++ b/src/Storages/RabbitMQ/WriteBufferToRabbitMQProducer.cpp @@ -7,6 +7,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Storages/System/StorageSystemRowPolicies.cpp b/src/Storages/System/StorageSystemRowPolicies.cpp index ff59d154704..455d715d5da 100644 --- a/src/Storages/System/StorageSystemRowPolicies.cpp +++ b/src/Storages/System/StorageSystemRowPolicies.cpp @@ -19,10 +19,6 @@ namespace DB { -using ConditionTypeInfo = RowPolicy::ConditionTypeInfo; -constexpr auto MAX_CONDITION_TYPE = RowPolicy::MAX_CONDITION_TYPE; - - NamesAndTypesList StorageSystemRowPolicies::getNamesAndTypes() { NamesAndTypesList names_and_types{ @@ -34,9 +30,9 @@ NamesAndTypesList StorageSystemRowPolicies::getNamesAndTypes() {"storage", std::make_shared()}, }; - for (auto type : collections::range(MAX_CONDITION_TYPE)) + for (auto filter_type : collections::range(RowPolicyFilterType::MAX)) { - const String & column_name = ConditionTypeInfo::get(type).name; + const String & column_name = RowPolicyFilterTypeInfo::get(filter_type).name; names_and_types.push_back({column_name, std::make_shared(std::make_shared())}); } @@ -66,12 +62,13 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr auto & column_id = assert_cast(*res_columns[column_index++]).getData(); auto & column_storage = assert_cast(*res_columns[column_index++]); - ColumnString * column_condition[MAX_CONDITION_TYPE]; - NullMap * column_condition_null_map[MAX_CONDITION_TYPE]; - for (auto condition_type : collections::range(MAX_CONDITION_TYPE)) + ColumnString * column_filter[static_cast(RowPolicyFilterType::MAX)]; + NullMap * column_filter_null_map[static_cast(RowPolicyFilterType::MAX)]; + for (auto filter_type : collections::range(RowPolicyFilterType::MAX)) { - column_condition[condition_type] = &assert_cast(assert_cast(*res_columns[column_index]).getNestedColumn()); - column_condition_null_map[condition_type] = &assert_cast(*res_columns[column_index++]).getNullMapData(); + auto filter_type_i = static_cast(filter_type); + column_filter[filter_type_i] = &assert_cast(assert_cast(*res_columns[column_index]).getNestedColumn()); + column_filter_null_map[filter_type_i] = &assert_cast(*res_columns[column_index++]).getNullMapData(); } auto & column_is_restrictive = assert_cast(*res_columns[column_index++]).getData(); @@ -85,7 +82,7 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr const RowPolicyName & full_name, const UUID & id, const String & storage_name, - const std::array & conditions, + const std::array(RowPolicyFilterType::MAX)> & filters, bool is_restrictive, const RolesOrUsersSet & apply_to) { @@ -96,18 +93,19 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr column_id.push_back(id.toUnderType()); column_storage.insertData(storage_name.data(), storage_name.length()); - for (auto condition_type : collections::range(MAX_CONDITION_TYPE)) + for (auto filter_type : collections::range(RowPolicyFilterType::MAX)) { - const String & condition = conditions[condition_type]; - if (condition.empty()) + auto filter_type_i = static_cast(filter_type); + const String & filter = filters[filter_type_i]; + if (filter.empty()) { - column_condition[condition_type]->insertDefault(); - column_condition_null_map[condition_type]->push_back(true); + column_filter[filter_type_i]->insertDefault(); + column_filter_null_map[filter_type_i]->push_back(true); } else { - column_condition[condition_type]->insertData(condition.data(), condition.length()); - column_condition_null_map[condition_type]->push_back(false); + column_filter[filter_type_i]->insertData(filter.data(), filter.length()); + column_filter_null_map[filter_type_i]->push_back(false); } } @@ -134,7 +132,7 @@ void StorageSystemRowPolicies::fillData(MutableColumns & res_columns, ContextPtr if (!storage) continue; - add_row(policy->getName(), policy->getFullName(), id, storage->getStorageName(), policy->conditions, policy->isRestrictive(), policy->to_roles); + add_row(policy->getName(), policy->getFullName(), id, storage->getStorageName(), policy->filters, policy->isRestrictive(), policy->to_roles); } } } From 6634fcbac7fdf1cf75315a6057d6eee5fb1427f0 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 18 Nov 2021 21:07:35 +0300 Subject: [PATCH 146/200] Rename Quota::ResourceType -> QuotaType and move it to Access/Common. --- programs/local/LocalServer.cpp | 1 + src/Access/Common/QuotaDefs.cpp | 104 ++++++++++++++ src/Access/Common/QuotaDefs.h | 40 ++++++ src/Access/EnabledQuota.cpp | 93 +++++++------ src/Access/EnabledQuota.h | 21 ++- src/Access/Quota.h | 128 +----------------- src/Access/QuotaCache.cpp | 15 +- src/Access/QuotaUsage.h | 8 +- src/Access/UsersConfigAccessStorage.cpp | 6 +- src/Client/ClientBase.cpp | 8 ++ src/Client/ClientBase.h | 5 +- .../Access/InterpreterCreateQuotaQuery.cpp | 7 +- ...InterpreterShowCreateAccessEntityQuery.cpp | 7 +- src/Interpreters/executeQuery.cpp | 12 +- src/Parsers/Access/ASTCreateQuotaQuery.cpp | 22 ++- src/Parsers/Access/ASTCreateQuotaQuery.h | 3 +- src/Parsers/Access/ParserCreateQuotaQuery.cpp | 49 +++---- src/Processors/Sources/SourceWithProgress.cpp | 2 +- .../Transforms/LimitsCheckingTransform.cpp | 6 +- src/QueryPipeline/Pipe.h | 2 +- src/Storages/MergeTree/DataPartsExchange.cpp | 1 + src/Storages/MergeTree/MutateTask.cpp | 2 + .../ReplicatedMergeTreeRestartingThread.cpp | 1 + .../PostgreSQLReplicationHandler.cpp | 1 + src/Storages/StorageExecutable.cpp | 2 + src/Storages/StorageReplicatedMergeTree.cpp | 6 +- .../System/StorageSystemQuotaLimits.cpp | 39 +++--- .../System/StorageSystemQuotaUsage.cpp | 60 ++++---- 28 files changed, 337 insertions(+), 314 deletions(-) create mode 100644 src/Access/Common/QuotaDefs.cpp create mode 100644 src/Access/Common/QuotaDefs.h diff --git a/programs/local/LocalServer.cpp b/programs/local/LocalServer.cpp index 0aaf78e70a0..fd35492ed39 100644 --- a/programs/local/LocalServer.cpp +++ b/programs/local/LocalServer.cpp @@ -36,6 +36,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Access/Common/QuotaDefs.cpp b/src/Access/Common/QuotaDefs.cpp new file mode 100644 index 00000000000..dca5cb6fa72 --- /dev/null +++ b/src/Access/Common/QuotaDefs.cpp @@ -0,0 +1,104 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +String QuotaTypeInfo::valueToString(QuotaValue value) const +{ + if (!(value % output_denominator)) + return std::to_string(value / output_denominator); + else + return boost::lexical_cast(static_cast(value) / output_denominator); +} + +QuotaValue QuotaTypeInfo::stringToValue(const String & str) const +{ + if (output_denominator == 1) + return static_cast(std::strtoul(str.c_str(), nullptr, 10)); + else + return static_cast(std::strtod(str.c_str(), nullptr) * output_denominator); +} + +String QuotaTypeInfo::valueToStringWithName(QuotaValue value) const +{ + String res = name; + res += " = "; + res += valueToString(value); + return res; +} + +const QuotaTypeInfo & QuotaTypeInfo::get(QuotaType type) +{ + static constexpr auto make_info = [](const char * raw_name_, UInt64 output_denominator_) + { + String init_name = raw_name_; + boost::to_lower(init_name); + String init_keyword = raw_name_; + boost::replace_all(init_keyword, "_", " "); + bool init_output_as_float = (output_denominator_ != 1); + return QuotaTypeInfo{raw_name_, std::move(init_name), std::move(init_keyword), init_output_as_float, output_denominator_}; + }; + + switch (type) + { + case QuotaType::QUERIES: + { + static const auto info = make_info("QUERIES", 1); + return info; + } + case QuotaType::QUERY_SELECTS: + { + static const auto info = make_info("QUERY_SELECTS", 1); + return info; + } + case QuotaType::QUERY_INSERTS: + { + static const auto info = make_info("QUERY_INSERTS", 1); + return info; + } + case QuotaType::ERRORS: + { + static const auto info = make_info("ERRORS", 1); + return info; + } + case QuotaType::RESULT_ROWS: + { + static const auto info = make_info("RESULT_ROWS", 1); + return info; + } + case QuotaType::RESULT_BYTES: + { + static const auto info = make_info("RESULT_BYTES", 1); + return info; + } + case QuotaType::READ_ROWS: + { + static const auto info = make_info("READ_ROWS", 1); + return info; + } + case QuotaType::READ_BYTES: + { + static const auto info = make_info("READ_BYTES", 1); + return info; + } + case QuotaType::EXECUTION_TIME: + { + static const auto info = make_info("EXECUTION_TIME", 1000000000 /* execution_time is stored in nanoseconds */); + return info; + } + case QuotaType::MAX: break; + } + throw Exception("Unexpected quota type: " + std::to_string(static_cast(type)), ErrorCodes::LOGICAL_ERROR); +} + +} diff --git a/src/Access/Common/QuotaDefs.h b/src/Access/Common/QuotaDefs.h new file mode 100644 index 00000000000..6dd9d0e5934 --- /dev/null +++ b/src/Access/Common/QuotaDefs.h @@ -0,0 +1,40 @@ +#pragma once + +#include + + +namespace DB +{ +using QuotaValue = UInt64; + +enum class QuotaType +{ + QUERIES, /// Number of queries. + QUERY_SELECTS, /// Number of select queries. + QUERY_INSERTS, /// Number of inserts queries. + ERRORS, /// Number of queries with exceptions. + RESULT_ROWS, /// Number of rows returned as result. + RESULT_BYTES, /// Number of bytes returned as result. + READ_ROWS, /// Number of rows read from tables. + READ_BYTES, /// Number of bytes read from tables. + EXECUTION_TIME, /// Total amount of query execution time in nanoseconds. + + MAX +}; + +struct QuotaTypeInfo +{ + const char * const raw_name = ""; + const String name; /// Lowercased with underscores, e.g. "result_rows". + const String keyword; /// Uppercased with spaces, e.g. "RESULT ROWS". + const bool output_as_float = false; + const UInt64 output_denominator = 1; + String valueToString(QuotaValue value) const; + QuotaValue stringToValue(const String & str) const; + String valueToStringWithName(QuotaValue value) const; + static const QuotaTypeInfo & get(QuotaType type); +}; + +String toString(QuotaType type); + +} diff --git a/src/Access/EnabledQuota.cpp b/src/Access/EnabledQuota.cpp index d776b2b851a..d1676b29e09 100644 --- a/src/Access/EnabledQuota.cpp +++ b/src/Access/EnabledQuota.cpp @@ -20,16 +20,16 @@ struct EnabledQuota::Impl [[noreturn]] static void throwQuotaExceed( const String & user_name, const String & quota_name, - ResourceType resource_type, - ResourceAmount used, - ResourceAmount max, + QuotaType quota_type, + QuotaValue used, + QuotaValue max, std::chrono::seconds duration, std::chrono::system_clock::time_point end_of_interval) { - const auto & type_info = Quota::ResourceTypeInfo::get(resource_type); + const auto & type_info = QuotaTypeInfo::get(quota_type); throw Exception( "Quota for user " + backQuote(user_name) + " for " + to_string(duration) + " has been exceeded: " - + type_info.outputWithAmount(used) + "/" + type_info.amountToString(max) + ". " + + type_info.valueToStringWithName(used) + "/" + type_info.valueToString(max) + ". " + "Interval will end at " + to_string(end_of_interval) + ". " + "Name of quota template: " + backQuote(quota_name), ErrorCodes::QUOTA_EXPIRED); } @@ -83,15 +83,16 @@ struct EnabledQuota::Impl static void used( const String & user_name, const Intervals & intervals, - ResourceType resource_type, - ResourceAmount amount, + QuotaType quota_type, + QuotaValue value, std::chrono::system_clock::time_point current_time, bool check_exceeded) { for (const auto & interval : intervals.intervals) { - ResourceAmount used = (interval.used[resource_type] += amount); - ResourceAmount max = interval.max[resource_type]; + auto quota_type_i = static_cast(quota_type); + QuotaValue used = (interval.used[quota_type_i] += value); + QuotaValue max = interval.max[quota_type_i]; if (!max) continue; if (used > max) @@ -100,12 +101,12 @@ struct EnabledQuota::Impl auto end_of_interval = getEndOfInterval(interval, current_time, counters_were_reset); if (counters_were_reset) { - used = (interval.used[resource_type] += amount); + used = (interval.used[quota_type_i] += value); if ((used > max) && check_exceeded) - throwQuotaExceed(user_name, intervals.quota_name, resource_type, used, max, interval.duration, end_of_interval); + throwQuotaExceed(user_name, intervals.quota_name, quota_type, used, max, interval.duration, end_of_interval); } else if (check_exceeded) - throwQuotaExceed(user_name, intervals.quota_name, resource_type, used, max, interval.duration, end_of_interval); + throwQuotaExceed(user_name, intervals.quota_name, quota_type, used, max, interval.duration, end_of_interval); } } } @@ -113,13 +114,14 @@ struct EnabledQuota::Impl static void checkExceeded( const String & user_name, const Intervals & intervals, - ResourceType resource_type, + QuotaType quota_type, std::chrono::system_clock::time_point current_time) { + auto quota_type_i = static_cast(quota_type); for (const auto & interval : intervals.intervals) { - ResourceAmount used = interval.used[resource_type]; - ResourceAmount max = interval.max[resource_type]; + QuotaValue used = interval.used[quota_type_i]; + QuotaValue max = interval.max[quota_type_i]; if (!max) continue; if (used > max) @@ -127,7 +129,7 @@ struct EnabledQuota::Impl bool counters_were_reset = false; std::chrono::system_clock::time_point end_of_interval = getEndOfInterval(interval, current_time, counters_were_reset); if (!counters_were_reset) - throwQuotaExceed(user_name, intervals.quota_name, resource_type, used, max, interval.duration, end_of_interval); + throwQuotaExceed(user_name, intervals.quota_name, quota_type, used, max, interval.duration, end_of_interval); } } } @@ -137,18 +139,19 @@ struct EnabledQuota::Impl const Intervals & intervals, std::chrono::system_clock::time_point current_time) { - for (auto resource_type : collections::range(Quota::MAX_RESOURCE_TYPE)) - checkExceeded(user_name, intervals, resource_type, current_time); + for (auto quota_type : collections::range(QuotaType::MAX)) + checkExceeded(user_name, intervals, quota_type, current_time); } }; EnabledQuota::Interval::Interval() { - for (auto resource_type : collections::range(MAX_RESOURCE_TYPE)) + for (auto quota_type : collections::range(QuotaType::MAX)) { - used[resource_type].store(0); - max[resource_type] = 0; + auto quota_type_i = static_cast(quota_type); + used[quota_type_i].store(0); + max[quota_type_i] = 0; } } @@ -161,10 +164,11 @@ EnabledQuota::Interval & EnabledQuota::Interval::operator =(const Interval & src randomize_interval = src.randomize_interval; duration = src.duration; end_of_interval.store(src.end_of_interval.load()); - for (auto resource_type : collections::range(MAX_RESOURCE_TYPE)) + for (auto quota_type : collections::range(QuotaType::MAX)) { - max[resource_type] = src.max[resource_type]; - used[resource_type].store(src.used[resource_type].load()); + auto quota_type_i = static_cast(quota_type); + max[quota_type_i] = src.max[quota_type_i]; + used[quota_type_i].store(src.used[quota_type_i].load()); } return *this; } @@ -187,11 +191,12 @@ std::optional EnabledQuota::Intervals::getUsage(std::chrono::system_ out.randomize_interval = in.randomize_interval; bool counters_were_reset = false; out.end_of_interval = Impl::getEndOfInterval(in, current_time, counters_were_reset); - for (auto resource_type : collections::range(MAX_RESOURCE_TYPE)) + for (auto quota_type : collections::range(QuotaType::MAX)) { - if (in.max[resource_type]) - out.max[resource_type] = in.max[resource_type]; - out.used[resource_type] = in.used[resource_type]; + auto quota_type_i = static_cast(quota_type); + if (in.max[quota_type_i]) + out.max[quota_type_i] = in.max[quota_type_i]; + out.used[quota_type_i] = in.used[quota_type_i]; } } return usage; @@ -205,45 +210,45 @@ EnabledQuota::EnabledQuota(const Params & params_) : params(params_) EnabledQuota::~EnabledQuota() = default; -void EnabledQuota::used(ResourceType resource_type, ResourceAmount amount, bool check_exceeded) const +void EnabledQuota::used(QuotaType quota_type, QuotaValue value, bool check_exceeded) const { - used({resource_type, amount}, check_exceeded); + used({quota_type, value}, check_exceeded); } -void EnabledQuota::used(const std::pair & resource, bool check_exceeded) const +void EnabledQuota::used(const std::pair & usage1, bool check_exceeded) const { auto loaded = intervals.load(); auto current_time = std::chrono::system_clock::now(); - Impl::used(getUserName(), *loaded, resource.first, resource.second, current_time, check_exceeded); + Impl::used(getUserName(), *loaded, usage1.first, usage1.second, current_time, check_exceeded); } -void EnabledQuota::used(const std::pair & resource1, const std::pair & resource2, bool check_exceeded) const +void EnabledQuota::used(const std::pair & usage1, const std::pair & usage2, bool check_exceeded) const { auto loaded = intervals.load(); auto current_time = std::chrono::system_clock::now(); - Impl::used(getUserName(), *loaded, resource1.first, resource1.second, current_time, check_exceeded); - Impl::used(getUserName(), *loaded, resource2.first, resource2.second, current_time, check_exceeded); + Impl::used(getUserName(), *loaded, usage1.first, usage1.second, current_time, check_exceeded); + Impl::used(getUserName(), *loaded, usage2.first, usage2.second, current_time, check_exceeded); } -void EnabledQuota::used(const std::pair & resource1, const std::pair & resource2, const std::pair & resource3, bool check_exceeded) const +void EnabledQuota::used(const std::pair & usage1, const std::pair & usage2, const std::pair & usage3, bool check_exceeded) const { auto loaded = intervals.load(); auto current_time = std::chrono::system_clock::now(); - Impl::used(getUserName(), *loaded, resource1.first, resource1.second, current_time, check_exceeded); - Impl::used(getUserName(), *loaded, resource2.first, resource2.second, current_time, check_exceeded); - Impl::used(getUserName(), *loaded, resource3.first, resource3.second, current_time, check_exceeded); + Impl::used(getUserName(), *loaded, usage1.first, usage1.second, current_time, check_exceeded); + Impl::used(getUserName(), *loaded, usage2.first, usage2.second, current_time, check_exceeded); + Impl::used(getUserName(), *loaded, usage3.first, usage3.second, current_time, check_exceeded); } -void EnabledQuota::used(const std::vector> & resources, bool check_exceeded) const +void EnabledQuota::used(const std::vector> & usages, bool check_exceeded) const { auto loaded = intervals.load(); auto current_time = std::chrono::system_clock::now(); - for (const auto & resource : resources) - Impl::used(getUserName(), *loaded, resource.first, resource.second, current_time, check_exceeded); + for (const auto & usage : usages) + Impl::used(getUserName(), *loaded, usage.first, usage.second, current_time, check_exceeded); } @@ -254,10 +259,10 @@ void EnabledQuota::checkExceeded() const } -void EnabledQuota::checkExceeded(ResourceType resource_type) const +void EnabledQuota::checkExceeded(QuotaType quota_type) const { auto loaded = intervals.load(); - Impl::checkExceeded(getUserName(), *loaded, resource_type, std::chrono::system_clock::now()); + Impl::checkExceeded(getUserName(), *loaded, quota_type, std::chrono::system_clock::now()); } diff --git a/src/Access/EnabledQuota.h b/src/Access/EnabledQuota.h index 7ae107e45e3..a2d8f1653d2 100644 --- a/src/Access/EnabledQuota.h +++ b/src/Access/EnabledQuota.h @@ -37,21 +37,18 @@ public: friend bool operator >=(const Params & lhs, const Params & rhs) { return !(lhs < rhs); } }; - using ResourceType = Quota::ResourceType; - using ResourceAmount = Quota::ResourceAmount; - ~EnabledQuota(); /// Tracks resource consumption. If the quota exceeded and `check_exceeded == true`, throws an exception. - void used(ResourceType resource_type, ResourceAmount amount, bool check_exceeded = true) const; - void used(const std::pair & resource, bool check_exceeded = true) const; - void used(const std::pair & resource1, const std::pair & resource2, bool check_exceeded = true) const; - void used(const std::pair & resource1, const std::pair & resource2, const std::pair & resource3, bool check_exceeded = true) const; - void used(const std::vector> & resources, bool check_exceeded = true) const; + void used(QuotaType quota_type, QuotaValue value, bool check_exceeded = true) const; + void used(const std::pair & usage1, bool check_exceeded = true) const; + void used(const std::pair & usage1, const std::pair & usage2, bool check_exceeded = true) const; + void used(const std::pair & usage1, const std::pair & usage2, const std::pair & usage3, bool check_exceeded = true) const; + void used(const std::vector> & usages, bool check_exceeded = true) const; /// Checks if the quota exceeded. If so, throws an exception. void checkExceeded() const; - void checkExceeded(ResourceType resource_type) const; + void checkExceeded(QuotaType quota_type) const; /// Returns the information about quota consumption. std::optional getUsage() const; @@ -66,12 +63,10 @@ private: const String & getUserName() const { return params.user_name; } - static constexpr auto MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; - struct Interval { - mutable std::atomic used[MAX_RESOURCE_TYPE]; - ResourceAmount max[MAX_RESOURCE_TYPE]; + mutable std::atomic used[static_cast(QuotaType::MAX)]; + QuotaValue max[static_cast(QuotaType::MAX)]; std::chrono::seconds duration = std::chrono::seconds::zero(); bool randomize_interval = false; mutable std::atomic end_of_interval; diff --git a/src/Access/Quota.h b/src/Access/Quota.h index 5d23cf11d42..c05124d3d35 100644 --- a/src/Access/Quota.h +++ b/src/Access/Quota.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -26,40 +27,10 @@ namespace ErrorCodes */ struct Quota : public IAccessEntity { - using ResourceAmount = UInt64; - - enum ResourceType - { - QUERIES, /// Number of queries. - QUERY_SELECTS, /// Number of select queries. - QUERY_INSERTS, /// Number of inserts queries. - ERRORS, /// Number of queries with exceptions. - RESULT_ROWS, /// Number of rows returned as result. - RESULT_BYTES, /// Number of bytes returned as result. - READ_ROWS, /// Number of rows read from tables. - READ_BYTES, /// Number of bytes read from tables. - EXECUTION_TIME, /// Total amount of query execution time in nanoseconds. - - MAX_RESOURCE_TYPE - }; - - struct ResourceTypeInfo - { - const char * const raw_name = ""; - const String name; /// Lowercased with underscores, e.g. "result_rows". - const String keyword; /// Uppercased with spaces, e.g. "RESULT ROWS". - const bool output_as_float = false; - const UInt64 output_denominator = 1; - String amountToString(ResourceAmount amount) const; - ResourceAmount amountFromString(const String & str) const; - String outputWithAmount(ResourceAmount amount) const; - static const ResourceTypeInfo & get(ResourceType type); - }; - /// Amount of resources available to consume for each duration. struct Limits { - std::optional max[MAX_RESOURCE_TYPE]; + std::optional max[static_cast(QuotaType::MAX)]; std::chrono::seconds duration = std::chrono::seconds::zero(); /// Intervals can be randomized (to avoid DoS if intervals for many users end at one time). @@ -105,101 +76,6 @@ struct Quota : public IAccessEntity Type getType() const override { return TYPE; } }; - -inline String Quota::ResourceTypeInfo::amountToString(ResourceAmount amount) const -{ - if (!(amount % output_denominator)) - return std::to_string(amount / output_denominator); - else - return boost::lexical_cast(static_cast(amount) / output_denominator); -} - -inline Quota::ResourceAmount Quota::ResourceTypeInfo::amountFromString(const String & str) const -{ - if (output_denominator == 1) - return static_cast(std::strtoul(str.c_str(), nullptr, 10)); - else - return static_cast(std::strtod(str.c_str(), nullptr) * output_denominator); -} - -inline String Quota::ResourceTypeInfo::outputWithAmount(ResourceAmount amount) const -{ - String res = name; - res += " = "; - res += amountToString(amount); - return res; -} - -inline String toString(Quota::ResourceType type) -{ - return Quota::ResourceTypeInfo::get(type).raw_name; -} - -inline const Quota::ResourceTypeInfo & Quota::ResourceTypeInfo::get(ResourceType type) -{ - static constexpr auto make_info = [](const char * raw_name_, UInt64 output_denominator_) - { - String init_name = raw_name_; - boost::to_lower(init_name); - String init_keyword = raw_name_; - boost::replace_all(init_keyword, "_", " "); - bool init_output_as_float = (output_denominator_ != 1); - return ResourceTypeInfo{raw_name_, std::move(init_name), std::move(init_keyword), init_output_as_float, output_denominator_}; - }; - - switch (type) - { - case Quota::QUERIES: - { - static const auto info = make_info("QUERIES", 1); - return info; - } - case Quota::QUERY_SELECTS: - { - static const auto info = make_info("QUERY_SELECTS", 1); - return info; - } - case Quota::QUERY_INSERTS: - { - static const auto info = make_info("QUERY_INSERTS", 1); - return info; - } - case Quota::ERRORS: - { - static const auto info = make_info("ERRORS", 1); - return info; - } - case Quota::RESULT_ROWS: - { - static const auto info = make_info("RESULT_ROWS", 1); - return info; - } - case Quota::RESULT_BYTES: - { - static const auto info = make_info("RESULT_BYTES", 1); - return info; - } - case Quota::READ_ROWS: - { - static const auto info = make_info("READ_ROWS", 1); - return info; - } - case Quota::READ_BYTES: - { - static const auto info = make_info("READ_BYTES", 1); - return info; - } - case Quota::EXECUTION_TIME: - { - static const auto info = make_info("EXECUTION_TIME", 1000000000 /* execution_time is stored in nanoseconds */); - return info; - } - case Quota::MAX_RESOURCE_TYPE: break; - } - throw Exception("Unexpected resource type: " + std::to_string(static_cast(type)), ErrorCodes::LOGICAL_ERROR); -} - - inline String toString(Quota::KeyType type) { return Quota::KeyTypeInfo::get(type).raw_name; diff --git a/src/Access/QuotaCache.cpp b/src/Access/QuotaCache.cpp index e5fa9114a51..c18a5ce3d6c 100644 --- a/src/Access/QuotaCache.cpp +++ b/src/Access/QuotaCache.cpp @@ -113,7 +113,6 @@ boost::shared_ptr QuotaCache::QuotaInfo::rebuildI new_intervals->quota_key = key; auto & intervals = new_intervals->intervals; intervals.reserve(quota->all_limits.size()); - static constexpr auto MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; for (const auto & limits : quota->all_limits) { intervals.emplace_back(); @@ -124,11 +123,12 @@ boost::shared_ptr QuotaCache::QuotaInfo::rebuildI if (limits.randomize_interval) end_of_interval += randomDuration(limits.duration); interval.end_of_interval = end_of_interval.time_since_epoch(); - for (auto resource_type : collections::range(MAX_RESOURCE_TYPE)) + for (auto quota_type : collections::range(QuotaType::MAX)) { - if (limits.max[resource_type]) - interval.max[resource_type] = *limits.max[resource_type]; - interval.used[resource_type] = 0; + auto quota_type_i = static_cast(quota_type); + if (limits.max[quota_type_i]) + interval.max[quota_type_i] = *limits.max[quota_type_i]; + interval.used[quota_type_i] = 0; } } @@ -159,9 +159,10 @@ boost::shared_ptr QuotaCache::QuotaInfo::rebuildI /// Found an interval with the same duration, we need to copy its usage information to `result`. const auto & current_interval = *lower_bound; - for (auto resource_type : collections::range(MAX_RESOURCE_TYPE)) + for (auto quota_type : collections::range(QuotaType::MAX)) { - new_interval.used[resource_type].store(current_interval.used[resource_type].load()); + auto quota_type_i = static_cast(quota_type); + new_interval.used[quota_type_i].store(current_interval.used[quota_type_i].load()); new_interval.end_of_interval.store(current_interval.end_of_interval.load()); } } diff --git a/src/Access/QuotaUsage.h b/src/Access/QuotaUsage.h index 9e53c0cf7d7..a5544a06eb9 100644 --- a/src/Access/QuotaUsage.h +++ b/src/Access/QuotaUsage.h @@ -9,14 +9,10 @@ namespace DB /// The information about a quota consumption. struct QuotaUsage { - using ResourceType = Quota::ResourceType; - using ResourceAmount = Quota::ResourceAmount; - static constexpr auto MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; - struct Interval { - ResourceAmount used[MAX_RESOURCE_TYPE]; - std::optional max[MAX_RESOURCE_TYPE]; + QuotaValue used[static_cast(QuotaType::MAX)]; + std::optional max[static_cast(QuotaType::MAX)]; std::chrono::seconds duration = std::chrono::seconds::zero(); bool randomize_interval = false; std::chrono::system_clock::time_point end_of_interval; diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index 70127b22998..02447815b20 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -252,12 +252,12 @@ namespace limits.duration = duration; limits.randomize_interval = config.getBool(interval_config + ".randomize", false); - for (auto resource_type : collections::range(Quota::MAX_RESOURCE_TYPE)) + for (auto quota_type : collections::range(QuotaType::MAX)) { - const auto & type_info = Quota::ResourceTypeInfo::get(resource_type); + const auto & type_info = QuotaTypeInfo::get(quota_type); auto value = config.getString(interval_config + "." + type_info.name, "0"); if (value != "0") - limits.max[resource_type] = type_info.amountFromString(value); + limits.max[static_cast(quota_type)] = type_info.stringToValue(value); } } diff --git a/src/Client/ClientBase.cpp b/src/Client/ClientBase.cpp index 501da82da14..d274b673091 100644 --- a/src/Client/ClientBase.cpp +++ b/src/Client/ClientBase.cpp @@ -56,6 +56,8 @@ #include #include #include +#include + namespace fs = std::filesystem; using namespace std::literals; @@ -1331,6 +1333,12 @@ bool ClientBase::processQueryText(const String & text) } +String ClientBase::prompt() const +{ + return boost::replace_all_copy(prompt_by_server_display_name, "{database}", config().getString("database", "default")); +} + + void ClientBase::runInteractive() { if (config().has("query_id")) diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index c71baa235b0..bad1395e699 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -128,10 +128,7 @@ private: void initBlockOutputStream(const Block & block, ASTPtr parsed_query); void initLogsOutputStream(); - inline String prompt() const - { - return boost::replace_all_copy(prompt_by_server_display_name, "{database}", config().getString("database", "default")); - } + String prompt() const; void resetOutput(); void outputQueryInfo(bool echo_query_); diff --git a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp index 703615972c4..9cba895acb0 100644 --- a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp @@ -58,8 +58,11 @@ namespace auto & quota_limits = *it; quota_limits.randomize_interval = query_limits.randomize_interval; - for (auto resource_type : collections::range(Quota::MAX_RESOURCE_TYPE)) - quota_limits.max[resource_type] = query_limits.max[resource_type]; + for (auto quota_type : collections::range(QuotaType::MAX)) + { + auto quota_type_i = static_cast(quota_type); + quota_limits.max[quota_type_i] = query_limits.max[quota_type_i]; + } } if (override_to_roles) diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index 901025b5227..7d3f1626f81 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -159,8 +159,11 @@ namespace ASTCreateQuotaQuery::Limits create_query_limits; create_query_limits.duration = limits.duration; create_query_limits.randomize_interval = limits.randomize_interval; - for (auto resource_type : collections::range(Quota::MAX_RESOURCE_TYPE)) - create_query_limits.max[resource_type] = limits.max[resource_type]; + for (auto quota_type : collections::range(QuotaType::MAX)) + { + auto quota_type_i = static_cast(quota_type); + create_query_limits.max[quota_type_i] = limits.max[quota_type_i]; + } query->all_limits.push_back(create_query_limits); } diff --git a/src/Interpreters/executeQuery.cpp b/src/Interpreters/executeQuery.cpp index f67eac10878..2ab4167176f 100644 --- a/src/Interpreters/executeQuery.cpp +++ b/src/Interpreters/executeQuery.cpp @@ -247,7 +247,7 @@ static void onExceptionBeforeStart(const String & query_for_logging, ContextPtr { /// Exception before the query execution. if (auto quota = context->getQuota()) - quota->used(Quota::ERRORS, 1, /* check_exceeded = */ false); + quota->used(QuotaType::ERRORS, 1, /* check_exceeded = */ false); const Settings & settings = context->getSettingsRef(); @@ -607,14 +607,14 @@ static std::tuple executeQueryImpl( { if (ast->as() || ast->as()) { - quota->used(Quota::QUERY_SELECTS, 1); + quota->used(QuotaType::QUERY_SELECTS, 1); } else if (ast->as()) { - quota->used(Quota::QUERY_INSERTS, 1); + quota->used(QuotaType::QUERY_INSERTS, 1); } - quota->used(Quota::QUERIES, 1); - quota->checkExceeded(Quota::ERRORS); + quota->used(QuotaType::QUERIES, 1); + quota->checkExceeded(QuotaType::ERRORS); } } @@ -864,7 +864,7 @@ static std::tuple executeQueryImpl( quota(quota), status_info_to_query_log] () mutable { if (quota) - quota->used(Quota::ERRORS, 1, /* check_exceeded = */ false); + quota->used(QuotaType::ERRORS, 1, /* check_exceeded = */ false); elem.type = QueryLogElementType::EXCEPTION_WHILE_PROCESSING; diff --git a/src/Parsers/Access/ASTCreateQuotaQuery.cpp b/src/Parsers/Access/ASTCreateQuotaQuery.cpp index 4e4c84f9e93..aef93c7ac4b 100644 --- a/src/Parsers/Access/ASTCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ASTCreateQuotaQuery.cpp @@ -12,10 +12,6 @@ namespace { using KeyType = Quota::KeyType; using KeyTypeInfo = Quota::KeyTypeInfo; - using ResourceType = Quota::ResourceType; - using ResourceTypeInfo = Quota::ResourceTypeInfo; - using ResourceAmount = Quota::ResourceAmount; - void formatKeyType(const KeyType & key_type, const IAST::FormatSettings & settings) { @@ -64,10 +60,10 @@ namespace } - void formatLimit(ResourceType resource_type, ResourceAmount max, const IAST::FormatSettings & settings) + void formatLimit(QuotaType quota_type, QuotaValue max_value, const IAST::FormatSettings & settings) { - const auto & type_info = ResourceTypeInfo::get(resource_type); - settings.ostr << " " << type_info.name << " = " << type_info.amountToString(max); + const auto & type_info = QuotaTypeInfo::get(quota_type); + settings.ostr << " " << type_info.name << " = " << type_info.valueToString(max_value); } @@ -93,22 +89,24 @@ namespace else { bool limit_found = false; - for (auto resource_type : collections::range(Quota::MAX_RESOURCE_TYPE)) + for (auto quota_type : collections::range(QuotaType::MAX)) { - if (limits.max[resource_type]) + auto quota_type_i = static_cast(quota_type); + if (limits.max[quota_type_i]) limit_found = true; } if (limit_found) { settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " MAX" << (settings.hilite ? IAST::hilite_none : ""); bool need_comma = false; - for (auto resource_type : collections::range(Quota::MAX_RESOURCE_TYPE)) + for (auto quota_type : collections::range(QuotaType::MAX)) { - if (limits.max[resource_type]) + auto quota_type_i = static_cast(quota_type); + if (limits.max[quota_type_i]) { if (std::exchange(need_comma, true)) settings.ostr << ","; - formatLimit(resource_type, *limits.max[resource_type], settings); + formatLimit(quota_type, *limits.max[quota_type_i], settings); } } } diff --git a/src/Parsers/Access/ASTCreateQuotaQuery.h b/src/Parsers/Access/ASTCreateQuotaQuery.h index 00984d4b4c9..6fdd953c6ff 100644 --- a/src/Parsers/Access/ASTCreateQuotaQuery.h +++ b/src/Parsers/Access/ASTCreateQuotaQuery.h @@ -36,7 +36,6 @@ public: bool or_replace = false; using KeyType = Quota::KeyType; - using ResourceAmount = Quota::ResourceAmount; Strings names; String new_name; @@ -44,7 +43,7 @@ public: struct Limits { - std::optional max[Quota::MAX_RESOURCE_TYPE]; + std::optional max[static_cast(QuotaType::MAX)]; bool drop = false; std::chrono::seconds duration = std::chrono::seconds::zero(); bool randomize_interval = false; diff --git a/src/Parsers/Access/ParserCreateQuotaQuery.cpp b/src/Parsers/Access/ParserCreateQuotaQuery.cpp index 0c6e1224cce..cee9ff5f514 100644 --- a/src/Parsers/Access/ParserCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ParserCreateQuotaQuery.cpp @@ -26,10 +26,6 @@ namespace { using KeyType = Quota::KeyType; using KeyTypeInfo = Quota::KeyTypeInfo; - using ResourceType = Quota::ResourceType; - using ResourceTypeInfo = Quota::ResourceTypeInfo; - using ResourceAmount = Quota::ResourceAmount; - bool parseRenameTo(IParserBase::Pos & pos, Expected & expected, String & new_name) { @@ -81,15 +77,15 @@ namespace } - bool parseResourceType(IParserBase::Pos & pos, Expected & expected, ResourceType & resource_type) + bool parseQuotaType(IParserBase::Pos & pos, Expected & expected, QuotaType & quota_type) { return IParserBase::wrapParseImpl(pos, [&] { - for (auto rt : collections::range(Quota::MAX_RESOURCE_TYPE)) + for (auto qt : collections::range(QuotaType::MAX)) { - if (ParserKeyword{ResourceTypeInfo::get(rt).keyword.c_str()}.ignore(pos, expected)) + if (ParserKeyword{QuotaTypeInfo::get(qt).keyword.c_str()}.ignore(pos, expected)) { - resource_type = rt; + quota_type = qt; return true; } } @@ -99,11 +95,11 @@ namespace return false; String name = getIdentifierName(ast); - for (auto rt : collections::range(Quota::MAX_RESOURCE_TYPE)) + for (auto qt : collections::range(QuotaType::MAX)) { - if (ResourceTypeInfo::get(rt).name == name) + if (QuotaTypeInfo::get(qt).name == name) { - resource_type = rt; + quota_type = qt; return true; } } @@ -113,34 +109,33 @@ namespace } - bool parseMaxAmount(IParserBase::Pos & pos, Expected & expected, ResourceType resource_type, ResourceAmount & max) + bool parseMaxValue(IParserBase::Pos & pos, Expected & expected, QuotaType quota_type, QuotaValue & max_value) { ASTPtr ast; if (!ParserNumber{}.parse(pos, ast, expected)) return false; const Field & max_field = ast->as().value; - const auto & type_info = ResourceTypeInfo::get(resource_type); + const auto & type_info = QuotaTypeInfo::get(quota_type); if (type_info.output_denominator == 1) - max = applyVisitor(FieldVisitorConvertToNumber(), max_field); + max_value = applyVisitor(FieldVisitorConvertToNumber(), max_field); else - max = static_cast( - applyVisitor(FieldVisitorConvertToNumber(), max_field) * type_info.output_denominator); + max_value = static_cast(applyVisitor(FieldVisitorConvertToNumber(), max_field) * type_info.output_denominator); return true; } - bool parseLimits(IParserBase::Pos & pos, Expected & expected, std::vector> & limits) + bool parseLimits(IParserBase::Pos & pos, Expected & expected, std::vector> & limits) { - std::vector> res_limits; + std::vector> res_limits; bool max_prefix_encountered = false; auto parse_limit = [&] { max_prefix_encountered |= ParserKeyword{"MAX"}.ignore(pos, expected); - ResourceType resource_type; - if (!parseResourceType(pos, expected, resource_type)) + QuotaType quota_type; + if (!parseQuotaType(pos, expected, quota_type)) return false; if (max_prefix_encountered) @@ -153,11 +148,11 @@ namespace return false; } - ResourceAmount max; - if (!parseMaxAmount(pos, expected, resource_type, max)) + QuotaValue max_value; + if (!parseMaxValue(pos, expected, quota_type, max_value)) return false; - res_limits.emplace_back(resource_type, max); + res_limits.emplace_back(quota_type, max_value); return true; }; @@ -193,7 +188,7 @@ namespace return false; limits.duration = std::chrono::seconds(static_cast(num_intervals * interval_kind.toAvgSeconds())); - std::vector> maxs; + std::vector> new_limits; if (ParserKeyword{"NO LIMITS"}.ignore(pos, expected)) { @@ -202,10 +197,10 @@ namespace else if (ParserKeyword{"TRACKING ONLY"}.ignore(pos, expected)) { } - else if (parseLimits(pos, expected, maxs)) + else if (parseLimits(pos, expected, new_limits)) { - for (const auto & [resource_type, max] : maxs) - limits.max[resource_type] = max; + for (const auto & [quota_type, max_value] : new_limits) + limits.max[static_cast(quota_type)] = max_value; } else return false; diff --git a/src/Processors/Sources/SourceWithProgress.cpp b/src/Processors/Sources/SourceWithProgress.cpp index 0ebdd968997..9b7a5c6a762 100644 --- a/src/Processors/Sources/SourceWithProgress.cpp +++ b/src/Processors/Sources/SourceWithProgress.cpp @@ -143,7 +143,7 @@ void SourceWithProgress::progress(const Progress & value) limits.speed_limits.throttle(progress.read_rows, progress.read_bytes, total_rows, total_elapsed_microseconds); if (quota && limits.mode == LimitsMode::LIMITS_TOTAL) - quota->used({Quota::READ_ROWS, value.read_rows}, {Quota::READ_BYTES, value.read_bytes}); + quota->used({QuotaType::READ_ROWS, value.read_rows}, {QuotaType::READ_BYTES, value.read_bytes}); } ProfileEvents::increment(ProfileEvents::SelectedRows, value.read_rows); diff --git a/src/Processors/Transforms/LimitsCheckingTransform.cpp b/src/Processors/Transforms/LimitsCheckingTransform.cpp index 64b6b64ccd9..e5f74003ac3 100644 --- a/src/Processors/Transforms/LimitsCheckingTransform.cpp +++ b/src/Processors/Transforms/LimitsCheckingTransform.cpp @@ -65,9 +65,9 @@ void LimitsCheckingTransform::checkQuota(Chunk & chunk) { UInt64 total_elapsed = info.total_stopwatch.elapsedNanoseconds(); quota->used( - {Quota::RESULT_ROWS, chunk.getNumRows()}, - {Quota::RESULT_BYTES, chunk.bytes()}, - {Quota::EXECUTION_TIME, total_elapsed - prev_elapsed}); + {QuotaType::RESULT_ROWS, chunk.getNumRows()}, + {QuotaType::RESULT_BYTES, chunk.bytes()}, + {QuotaType::EXECUTION_TIME, total_elapsed - prev_elapsed}); prev_elapsed = total_elapsed; break; } diff --git a/src/QueryPipeline/Pipe.h b/src/QueryPipeline/Pipe.h index 0af02a5e662..613e92a782d 100644 --- a/src/QueryPipeline/Pipe.h +++ b/src/QueryPipeline/Pipe.h @@ -3,12 +3,12 @@ #include #include #include -#include #include namespace DB { +class EnabledQuota; struct StreamLocalLimits; class Pipe; diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 07ff0d64e70..e952deb5a87 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #include diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 36ce3f25744..accf167f5ff 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -16,6 +16,8 @@ #include #include #include +#include + namespace CurrentMetrics { diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp index c29ae873c5c..3bb592dcdcb 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeRestartingThread.cpp @@ -6,6 +6,7 @@ #include #include #include +#include namespace ProfileEvents diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 670148e9baa..56cd58c4c03 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -10,6 +10,7 @@ #include #include #include +#include namespace DB diff --git a/src/Storages/StorageExecutable.cpp b/src/Storages/StorageExecutable.cpp index 16647d0b60f..51ecfc1e884 100644 --- a/src/Storages/StorageExecutable.cpp +++ b/src/Storages/StorageExecutable.cpp @@ -21,6 +21,8 @@ #include #include +#include + namespace DB { diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 70fbdaa640d..60b576d8eac 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -60,13 +60,15 @@ #include #include - #include #include #include #include +#include +#include + #include #include #include @@ -75,8 +77,6 @@ #include #include -#include - namespace fs = std::filesystem; namespace ProfileEvents diff --git a/src/Storages/System/StorageSystemQuotaLimits.cpp b/src/Storages/System/StorageSystemQuotaLimits.cpp index 330b9935b48..c98e060a62f 100644 --- a/src/Storages/System/StorageSystemQuotaLimits.cpp +++ b/src/Storages/System/StorageSystemQuotaLimits.cpp @@ -15,27 +15,22 @@ namespace DB { -using ResourceAmount = Quota::ResourceAmount; -using ResourceType = Quota::ResourceType; -using ResourceTypeInfo = Quota::ResourceTypeInfo; -constexpr auto MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; - namespace { - void addValue(IColumn & out_column, NullMap & out_column_null_map, ResourceAmount amount, const ResourceTypeInfo & type_info) + void addValue(IColumn & out_column, NullMap & out_column_null_map, QuotaValue value, const QuotaTypeInfo & type_info) { out_column_null_map.push_back(false); if (type_info.output_as_float) - static_cast(out_column).getData().push_back(double(amount) / type_info.output_denominator); + static_cast(out_column).getData().push_back(double(value) / type_info.output_denominator); else - static_cast(out_column).getData().push_back(amount / type_info.output_denominator); + static_cast(out_column).getData().push_back(value / type_info.output_denominator); } - void addValue(IColumn & out_column, NullMap & out_column_null_map, std::optional amount, const ResourceTypeInfo & type_info) + void addValue(IColumn & out_column, NullMap & out_column_null_map, std::optional value, const QuotaTypeInfo & type_info) { - if (amount) - addValue(out_column, out_column_null_map, *amount, type_info); + if (value) + addValue(out_column, out_column_null_map, *value, type_info); else { out_column_null_map.push_back(true); @@ -53,9 +48,9 @@ NamesAndTypesList StorageSystemQuotaLimits::getNamesAndTypes() {"is_randomized_interval", std::make_shared()}, }; - for (auto resource_type : collections::range(MAX_RESOURCE_TYPE)) + for (auto quota_type : collections::range(QuotaType::MAX)) { - const auto & type_info = ResourceTypeInfo::get(resource_type); + const auto & type_info = QuotaTypeInfo::get(quota_type); String column_name = "max_" + type_info.name; DataTypePtr data_type; if (type_info.output_as_float) @@ -80,12 +75,13 @@ void StorageSystemQuotaLimits::fillData(MutableColumns & res_columns, ContextPtr auto & column_duration = assert_cast(*res_columns[column_index++]).getData(); auto & column_is_randomized_interval = assert_cast(*res_columns[column_index++]).getData(); - IColumn * column_max[MAX_RESOURCE_TYPE]; - NullMap * column_max_null_map[MAX_RESOURCE_TYPE]; - for (auto resource_type : collections::range(MAX_RESOURCE_TYPE)) + IColumn * column_max[static_cast(QuotaType::MAX)]; + NullMap * column_max_null_map[static_cast(QuotaType::MAX)]; + for (auto quota_type : collections::range(QuotaType::MAX)) { - column_max[resource_type] = &assert_cast(*res_columns[column_index]).getNestedColumn(); - column_max_null_map[resource_type] = &assert_cast(*res_columns[column_index++]).getNullMapData(); + auto quota_type_i = static_cast(quota_type); + column_max[quota_type_i] = &assert_cast(*res_columns[column_index]).getNestedColumn(); + column_max_null_map[quota_type_i] = &assert_cast(*res_columns[column_index++]).getNullMapData(); } auto add_row = [&](const String & quota_name, const Quota::Limits & limits) @@ -94,10 +90,11 @@ void StorageSystemQuotaLimits::fillData(MutableColumns & res_columns, ContextPtr column_duration.push_back(limits.duration.count()); column_is_randomized_interval.push_back(limits.randomize_interval); - for (auto resource_type : collections::range(MAX_RESOURCE_TYPE)) + for (auto quota_type : collections::range(QuotaType::MAX)) { - const auto & type_info = ResourceTypeInfo::get(resource_type); - addValue(*column_max[resource_type], *column_max_null_map[resource_type], limits.max[resource_type], type_info); + auto quota_type_i = static_cast(quota_type); + const auto & type_info = QuotaTypeInfo::get(quota_type); + addValue(*column_max[quota_type_i], *column_max_null_map[quota_type_i], limits.max[quota_type_i], type_info); } }; diff --git a/src/Storages/System/StorageSystemQuotaUsage.cpp b/src/Storages/System/StorageSystemQuotaUsage.cpp index a08f6686030..54f403803d6 100644 --- a/src/Storages/System/StorageSystemQuotaUsage.cpp +++ b/src/Storages/System/StorageSystemQuotaUsage.cpp @@ -15,27 +15,22 @@ namespace DB { -using ResourceAmount = Quota::ResourceAmount; -using ResourceType = Quota::ResourceType; -using ResourceTypeInfo = Quota::ResourceTypeInfo; -constexpr auto MAX_RESOURCE_TYPE = Quota::MAX_RESOURCE_TYPE; - namespace { - void addValue(IColumn & out_column, NullMap & out_column_null_map, ResourceAmount amount, const ResourceTypeInfo & type_info) + void addValue(IColumn & out_column, NullMap & out_column_null_map, QuotaValue value, const QuotaTypeInfo & type_info) { out_column_null_map.push_back(false); if (type_info.output_as_float) - static_cast(out_column).getData().push_back(double(amount) / type_info.output_denominator); + static_cast(out_column).getData().push_back(double(value) / type_info.output_denominator); else - static_cast(out_column).getData().push_back(amount / type_info.output_denominator); + static_cast(out_column).getData().push_back(value / type_info.output_denominator); } - void addValue(IColumn & out_column, NullMap & out_column_null_map, std::optional amount, const ResourceTypeInfo & type_info) + void addValue(IColumn & out_column, NullMap & out_column_null_map, std::optional value, const QuotaTypeInfo & type_info) { - if (amount) - addValue(out_column, out_column_null_map, *amount, type_info); + if (value) + addValue(out_column, out_column_null_map, *value, type_info); else { out_column_null_map.push_back(true); @@ -64,9 +59,9 @@ NamesAndTypesList StorageSystemQuotaUsage::getNamesAndTypesImpl(bool add_column_ names_and_types.push_back({"end_time", std::make_shared(std::make_shared())}); names_and_types.push_back({"duration", std::make_shared(std::make_shared())}); - for (auto resource_type : collections::range(MAX_RESOURCE_TYPE)) + for (auto quota_type : collections::range(QuotaType::MAX)) { - const auto & type_info = ResourceTypeInfo::get(resource_type); + const auto & type_info = QuotaTypeInfo::get(quota_type); String column_name = type_info.name; DataTypePtr data_type; if (type_info.output_as_float) @@ -113,16 +108,17 @@ void StorageSystemQuotaUsage::fillDataImpl( auto & column_duration = assert_cast(assert_cast(*res_columns[column_index]).getNestedColumn()); auto & column_duration_null_map = assert_cast(*res_columns[column_index++]).getNullMapData(); - IColumn * column_usage[MAX_RESOURCE_TYPE]; - NullMap * column_usage_null_map[MAX_RESOURCE_TYPE]; - IColumn * column_max[MAX_RESOURCE_TYPE]; - NullMap * column_max_null_map[MAX_RESOURCE_TYPE]; - for (auto resource_type : collections::range(MAX_RESOURCE_TYPE)) + IColumn * column_usage[static_cast(QuotaType::MAX)]; + NullMap * column_usage_null_map[static_cast(QuotaType::MAX)]; + IColumn * column_max[static_cast(QuotaType::MAX)]; + NullMap * column_max_null_map[static_cast(QuotaType::MAX)]; + for (auto quota_type : collections::range(QuotaType::MAX)) { - column_usage[resource_type] = &assert_cast(*res_columns[column_index]).getNestedColumn(); - column_usage_null_map[resource_type] = &assert_cast(*res_columns[column_index++]).getNullMapData(); - column_max[resource_type] = &assert_cast(*res_columns[column_index]).getNestedColumn(); - column_max_null_map[resource_type] = &assert_cast(*res_columns[column_index++]).getNullMapData(); + auto quota_type_i = static_cast(quota_type); + column_usage[quota_type_i] = &assert_cast(*res_columns[column_index]).getNestedColumn(); + column_usage_null_map[quota_type_i] = &assert_cast(*res_columns[column_index++]).getNullMapData(); + column_max[quota_type_i] = &assert_cast(*res_columns[column_index]).getNestedColumn(); + column_max_null_map[quota_type_i] = &assert_cast(*res_columns[column_index++]).getNullMapData(); } std::optional current_quota_id; @@ -148,12 +144,13 @@ void StorageSystemQuotaUsage::fillDataImpl( column_end_time_null_map.push_back(true); column_duration.insertDefault(); column_duration_null_map.push_back(true); - for (auto resource_type : collections::range(MAX_RESOURCE_TYPE)) + for (auto quota_type : collections::range(QuotaType::MAX)) { - column_usage[resource_type]->insertDefault(); - column_usage_null_map[resource_type]->push_back(true); - column_max[resource_type]->insertDefault(); - column_max_null_map[resource_type]->push_back(true); + auto quota_type_i = static_cast(quota_type); + column_usage[quota_type_i]->insertDefault(); + column_usage_null_map[quota_type_i]->push_back(true); + column_max[quota_type_i]->insertDefault(); + column_max_null_map[quota_type_i]->push_back(true); } return; } @@ -168,11 +165,12 @@ void StorageSystemQuotaUsage::fillDataImpl( column_end_time_null_map.push_back(false); column_duration_null_map.push_back(false); - for (auto resource_type : collections::range(Quota::MAX_RESOURCE_TYPE)) + for (auto quota_type : collections::range(QuotaType::MAX)) { - const auto & type_info = ResourceTypeInfo::get(resource_type); - addValue(*column_max[resource_type], *column_max_null_map[resource_type], interval->max[resource_type], type_info); - addValue(*column_usage[resource_type], *column_usage_null_map[resource_type], interval->used[resource_type], type_info); + auto quota_type_i = static_cast(quota_type); + const auto & type_info = QuotaTypeInfo::get(quota_type); + addValue(*column_max[quota_type_i], *column_max_null_map[quota_type_i], interval->max[quota_type_i], type_info); + addValue(*column_usage[quota_type_i], *column_usage_null_map[quota_type_i], interval->used[quota_type_i], type_info); } }; From 06077a29526f55db21089c5638800cbc08cc9cb2 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 18 Nov 2021 21:43:02 +0300 Subject: [PATCH 147/200] Rename Quota::KeyType -> QuotaKeyType and move it to Access/Common. --- src/Access/Common/QuotaDefs.cpp | 85 ++++++++++++++ src/Access/Common/QuotaDefs.h | 29 ++++- src/Access/EnabledQuota.h | 4 +- src/Access/Quota.h | 105 +----------------- src/Access/QuotaCache.cpp | 18 +-- src/Access/QuotaCache.h | 4 +- src/Access/QuotaUsage.h | 3 +- src/Access/UsersConfigAccessStorage.cpp | 9 +- .../Access/InterpreterCreateQuotaQuery.cpp | 1 + ...InterpreterShowCreateAccessEntityQuery.cpp | 5 +- src/Parsers/Access/ASTCreateQuotaQuery.cpp | 11 +- src/Parsers/Access/ASTCreateQuotaQuery.h | 6 +- src/Parsers/Access/ParserCreateQuotaQuery.cpp | 22 ++-- src/Storages/System/StorageSystemQuotas.cpp | 15 +-- 14 files changed, 162 insertions(+), 155 deletions(-) diff --git a/src/Access/Common/QuotaDefs.cpp b/src/Access/Common/QuotaDefs.cpp index dca5cb6fa72..5d36a0bdd01 100644 --- a/src/Access/Common/QuotaDefs.cpp +++ b/src/Access/Common/QuotaDefs.cpp @@ -1,7 +1,12 @@ #include #include + +#include + #include +#include #include +#include #include @@ -13,6 +18,12 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } + +String toString(QuotaType type) +{ + return QuotaTypeInfo::get(type).raw_name; +} + String QuotaTypeInfo::valueToString(QuotaValue value) const { if (!(value % output_denominator)) @@ -101,4 +112,78 @@ const QuotaTypeInfo & QuotaTypeInfo::get(QuotaType type) throw Exception("Unexpected quota type: " + std::to_string(static_cast(type)), ErrorCodes::LOGICAL_ERROR); } +String toString(QuotaKeyType type) +{ + return QuotaKeyTypeInfo::get(type).raw_name; +} + +const QuotaKeyTypeInfo & QuotaKeyTypeInfo::get(QuotaKeyType type) +{ + static constexpr auto make_info = [](const char * raw_name_) + { + String init_name = raw_name_; + boost::to_lower(init_name); + std::vector init_base_types; + String replaced = boost::algorithm::replace_all_copy(init_name, "_or_", "|"); + Strings tokens; + boost::algorithm::split(tokens, replaced, boost::is_any_of("|")); + if (tokens.size() > 1) + { + for (const auto & token : tokens) + { + for (auto kt : collections::range(QuotaKeyType::MAX)) + { + if (QuotaKeyTypeInfo::get(kt).name == token) + { + init_base_types.push_back(kt); + break; + } + } + } + } + return QuotaKeyTypeInfo{raw_name_, std::move(init_name), std::move(init_base_types)}; + }; + + switch (type) + { + case QuotaKeyType::NONE: + { + static const auto info = make_info("NONE"); + return info; + } + case QuotaKeyType::USER_NAME: + { + static const auto info = make_info("USER_NAME"); + return info; + } + case QuotaKeyType::IP_ADDRESS: + { + static const auto info = make_info("IP_ADDRESS"); + return info; + } + case QuotaKeyType::FORWARDED_IP_ADDRESS: + { + static const auto info = make_info("FORWARDED_IP_ADDRESS"); + return info; + } + case QuotaKeyType::CLIENT_KEY: + { + static const auto info = make_info("CLIENT_KEY"); + return info; + } + case QuotaKeyType::CLIENT_KEY_OR_USER_NAME: + { + static const auto info = make_info("CLIENT_KEY_OR_USER_NAME"); + return info; + } + case QuotaKeyType::CLIENT_KEY_OR_IP_ADDRESS: + { + static const auto info = make_info("CLIENT_KEY_OR_IP_ADDRESS"); + return info; + } + case QuotaKeyType::MAX: break; + } + throw Exception("Unexpected quota key type: " + std::to_string(static_cast(type)), ErrorCodes::LOGICAL_ERROR); +} + } diff --git a/src/Access/Common/QuotaDefs.h b/src/Access/Common/QuotaDefs.h index 6dd9d0e5934..e0c26282862 100644 --- a/src/Access/Common/QuotaDefs.h +++ b/src/Access/Common/QuotaDefs.h @@ -5,8 +5,10 @@ namespace DB { +/// We use UInt64 to count used resources. using QuotaValue = UInt64; +/// Kinds of resource what we wish to quota. enum class QuotaType { QUERIES, /// Number of queries. @@ -22,6 +24,8 @@ enum class QuotaType MAX }; +String toString(QuotaType type); + struct QuotaTypeInfo { const char * const raw_name = ""; @@ -35,6 +39,29 @@ struct QuotaTypeInfo static const QuotaTypeInfo & get(QuotaType type); }; -String toString(QuotaType type); +/// Key to share quota consumption. +/// Users with the same key share the same amount of resource. +enum class QuotaKeyType +{ + NONE, /// All users share the same quota. + USER_NAME, /// Connections with the same user name share the same quota. + IP_ADDRESS, /// Connections from the same IP share the same quota. + FORWARDED_IP_ADDRESS, /// Use X-Forwarded-For HTTP header instead of IP address. + CLIENT_KEY, /// Client should explicitly supply a key to use. + CLIENT_KEY_OR_USER_NAME, /// Same as CLIENT_KEY, but use USER_NAME if the client doesn't supply a key. + CLIENT_KEY_OR_IP_ADDRESS, /// Same as CLIENT_KEY, but use IP_ADDRESS if the client doesn't supply a key. + + MAX +}; + +String toString(QuotaKeyType type); + +struct QuotaKeyTypeInfo +{ + const char * const raw_name; + const String name; /// Lowercased with underscores, e.g. "client_key". + const std::vector base_types; /// For combined types keeps base types, e.g. for CLIENT_KEY_OR_USER_NAME it keeps [KeyType::CLIENT_KEY, KeyType::USER_NAME]. + static const QuotaKeyTypeInfo & get(QuotaKeyType type); +}; } diff --git a/src/Access/EnabledQuota.h b/src/Access/EnabledQuota.h index a2d8f1653d2..097afe861d2 100644 --- a/src/Access/EnabledQuota.h +++ b/src/Access/EnabledQuota.h @@ -1,13 +1,15 @@ #pragma once -#include +#include #include #include +#include #include #include #include #include #include +#include namespace DB diff --git a/src/Access/Quota.h b/src/Access/Quota.h index c05124d3d35..f87f9181b1e 100644 --- a/src/Access/Quota.h +++ b/src/Access/Quota.h @@ -3,19 +3,11 @@ #include #include #include -#include -#include -#include #include namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - /** Quota for resources consumption for specific interval. * Used to limit resource usage by user. @@ -44,28 +36,7 @@ struct Quota : public IAccessEntity /// Key to share quota consumption. /// Users with the same key share the same amount of resource. - enum class KeyType - { - NONE, /// All users share the same quota. - USER_NAME, /// Connections with the same user name share the same quota. - IP_ADDRESS, /// Connections from the same IP share the same quota. - FORWARDED_IP_ADDRESS, /// Use X-Forwarded-For HTTP header instead of IP address. - CLIENT_KEY, /// Client should explicitly supply a key to use. - CLIENT_KEY_OR_USER_NAME, /// Same as CLIENT_KEY, but use USER_NAME if the client doesn't supply a key. - CLIENT_KEY_OR_IP_ADDRESS, /// Same as CLIENT_KEY, but use IP_ADDRESS if the client doesn't supply a key. - - MAX - }; - - struct KeyTypeInfo - { - const char * const raw_name; - const String name; /// Lowercased with underscores, e.g. "client_key". - const std::vector base_types; /// For combined types keeps base types, e.g. for CLIENT_KEY_OR_USER_NAME it keeps [KeyType::CLIENT_KEY, KeyType::USER_NAME]. - static const KeyTypeInfo & get(KeyType type); - }; - - KeyType key_type = KeyType::NONE; + QuotaKeyType key_type = QuotaKeyType::NONE; /// Which roles or users should use this quota. RolesOrUsersSet to_roles; @@ -76,79 +47,5 @@ struct Quota : public IAccessEntity Type getType() const override { return TYPE; } }; -inline String toString(Quota::KeyType type) -{ - return Quota::KeyTypeInfo::get(type).raw_name; -} - -inline const Quota::KeyTypeInfo & Quota::KeyTypeInfo::get(KeyType type) -{ - static constexpr auto make_info = [](const char * raw_name_) - { - String init_name = raw_name_; - boost::to_lower(init_name); - std::vector init_base_types; - String replaced = boost::algorithm::replace_all_copy(init_name, "_or_", "|"); - Strings tokens; - boost::algorithm::split(tokens, replaced, boost::is_any_of("|")); - if (tokens.size() > 1) - { - for (const auto & token : tokens) - { - for (auto kt : collections::range(KeyType::MAX)) - { - if (KeyTypeInfo::get(kt).name == token) - { - init_base_types.push_back(kt); - break; - } - } - } - } - return KeyTypeInfo{raw_name_, std::move(init_name), std::move(init_base_types)}; - }; - - switch (type) - { - case KeyType::NONE: - { - static const auto info = make_info("NONE"); - return info; - } - case KeyType::USER_NAME: - { - static const auto info = make_info("USER_NAME"); - return info; - } - case KeyType::IP_ADDRESS: - { - static const auto info = make_info("IP_ADDRESS"); - return info; - } - case KeyType::FORWARDED_IP_ADDRESS: - { - static const auto info = make_info("FORWARDED_IP_ADDRESS"); - return info; - } - case KeyType::CLIENT_KEY: - { - static const auto info = make_info("CLIENT_KEY"); - return info; - } - case KeyType::CLIENT_KEY_OR_USER_NAME: - { - static const auto info = make_info("CLIENT_KEY_OR_USER_NAME"); - return info; - } - case KeyType::CLIENT_KEY_OR_IP_ADDRESS: - { - static const auto info = make_info("CLIENT_KEY_OR_IP_ADDRESS"); - return info; - } - case KeyType::MAX: break; - } - throw Exception("Unexpected quota key type: " + std::to_string(static_cast(type)), ErrorCodes::LOGICAL_ERROR); -} - using QuotaPtr = std::shared_ptr; } diff --git a/src/Access/QuotaCache.cpp b/src/Access/QuotaCache.cpp index c18a5ce3d6c..566c2409205 100644 --- a/src/Access/QuotaCache.cpp +++ b/src/Access/QuotaCache.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -44,26 +45,25 @@ void QuotaCache::QuotaInfo::setQuota(const QuotaPtr & quota_, const UUID & quota String QuotaCache::QuotaInfo::calculateKey(const EnabledQuota & enabled) const { const auto & params = enabled.params; - using KeyType = Quota::KeyType; switch (quota->key_type) { - case KeyType::NONE: + case QuotaKeyType::NONE: { return ""; } - case KeyType::USER_NAME: + case QuotaKeyType::USER_NAME: { return params.user_name; } - case KeyType::IP_ADDRESS: + case QuotaKeyType::IP_ADDRESS: { return params.client_address.toString(); } - case KeyType::FORWARDED_IP_ADDRESS: + case QuotaKeyType::FORWARDED_IP_ADDRESS: { return params.forwarded_address; } - case KeyType::CLIENT_KEY: + case QuotaKeyType::CLIENT_KEY: { if (!params.client_key.empty()) return params.client_key; @@ -71,19 +71,19 @@ String QuotaCache::QuotaInfo::calculateKey(const EnabledQuota & enabled) const "Quota " + quota->getName() + " (for user " + params.user_name + ") requires a client supplied key.", ErrorCodes::QUOTA_REQUIRES_CLIENT_KEY); } - case KeyType::CLIENT_KEY_OR_USER_NAME: + case QuotaKeyType::CLIENT_KEY_OR_USER_NAME: { if (!params.client_key.empty()) return params.client_key; return params.user_name; } - case KeyType::CLIENT_KEY_OR_IP_ADDRESS: + case QuotaKeyType::CLIENT_KEY_OR_IP_ADDRESS: { if (!params.client_key.empty()) return params.client_key; return params.client_address.toString(); } - case KeyType::MAX: break; + case QuotaKeyType::MAX: break; } throw Exception("Unexpected quota key type: " + std::to_string(static_cast(quota->key_type)), ErrorCodes::LOGICAL_ERROR); } diff --git a/src/Access/QuotaCache.h b/src/Access/QuotaCache.h index 487c7a26487..77682230370 100644 --- a/src/Access/QuotaCache.h +++ b/src/Access/QuotaCache.h @@ -11,7 +11,9 @@ namespace DB { class AccessControl; - +struct Quota; +using QuotaPtr = std::shared_ptr; +struct RolesOrUsersSet; /// Stores information how much amount of resources have been consumed and how much are left. class QuotaCache diff --git a/src/Access/QuotaUsage.h b/src/Access/QuotaUsage.h index a5544a06eb9..1d7e5258c47 100644 --- a/src/Access/QuotaUsage.h +++ b/src/Access/QuotaUsage.h @@ -1,7 +1,8 @@ #pragma once -#include +#include #include +#include namespace DB diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index 02447815b20..002517b32a0 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -223,16 +223,15 @@ namespace auto quota = std::make_shared(); quota->setName(quota_name); - using KeyType = Quota::KeyType; String quota_config = "quotas." + quota_name; if (config.has(quota_config + ".keyed_by_ip")) - quota->key_type = KeyType::IP_ADDRESS; + quota->key_type = QuotaKeyType::IP_ADDRESS; else if (config.has(quota_config + ".keyed_by_forwarded_ip")) - quota->key_type = KeyType::FORWARDED_IP_ADDRESS; + quota->key_type = QuotaKeyType::FORWARDED_IP_ADDRESS; else if (config.has(quota_config + ".keyed")) - quota->key_type = KeyType::CLIENT_KEY_OR_USER_NAME; + quota->key_type = QuotaKeyType::CLIENT_KEY_OR_USER_NAME; else - quota->key_type = KeyType::USER_NAME; + quota->key_type = QuotaKeyType::USER_NAME; Poco::Util::AbstractConfiguration::Keys interval_keys; config.keys(quota_config, interval_keys); diff --git a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp index 9cba895acb0..83a620d11c6 100644 --- a/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp +++ b/src/Interpreters/Access/InterpreterCreateQuotaQuery.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index 7d3f1626f81..dcfdc897c62 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -14,11 +14,12 @@ #include #include #include +#include #include -#include #include #include #include +#include #include #include #include @@ -149,7 +150,7 @@ namespace query->names.emplace_back(quota.getName()); query->attach = attach_mode; - if (quota.key_type != Quota::KeyType::NONE) + if (quota.key_type != QuotaKeyType::NONE) query->key_type = quota.key_type; query->all_limits.reserve(quota.all_limits.size()); diff --git a/src/Parsers/Access/ASTCreateQuotaQuery.cpp b/src/Parsers/Access/ASTCreateQuotaQuery.cpp index aef93c7ac4b..0bb6872e3af 100644 --- a/src/Parsers/Access/ASTCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ASTCreateQuotaQuery.cpp @@ -10,13 +10,10 @@ namespace DB { namespace { - using KeyType = Quota::KeyType; - using KeyTypeInfo = Quota::KeyTypeInfo; - - void formatKeyType(const KeyType & key_type, const IAST::FormatSettings & settings) + void formatKeyType(const QuotaKeyType & key_type, const IAST::FormatSettings & settings) { - const auto & type_info = KeyTypeInfo::get(key_type); - if (key_type == KeyType::NONE) + const auto & type_info = QuotaKeyTypeInfo::get(key_type); + if (key_type == QuotaKeyType::NONE) { settings.ostr << (settings.hilite ? IAST::hilite_keyword : "") << " NOT KEYED" << (settings.hilite ? IAST::hilite_none : ""); return; @@ -31,7 +28,7 @@ namespace { if (std::exchange(need_comma, true)) settings.ostr << ", "; - settings.ostr << KeyTypeInfo::get(base_type).name; + settings.ostr << QuotaKeyTypeInfo::get(base_type).name; } return; } diff --git a/src/Parsers/Access/ASTCreateQuotaQuery.h b/src/Parsers/Access/ASTCreateQuotaQuery.h index 6fdd953c6ff..154245cbfe3 100644 --- a/src/Parsers/Access/ASTCreateQuotaQuery.h +++ b/src/Parsers/Access/ASTCreateQuotaQuery.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB @@ -35,11 +35,9 @@ public: bool if_not_exists = false; bool or_replace = false; - using KeyType = Quota::KeyType; - Strings names; String new_name; - std::optional key_type; + std::optional key_type; struct Limits { diff --git a/src/Parsers/Access/ParserCreateQuotaQuery.cpp b/src/Parsers/Access/ParserCreateQuotaQuery.cpp index cee9ff5f514..a06464913b7 100644 --- a/src/Parsers/Access/ParserCreateQuotaQuery.cpp +++ b/src/Parsers/Access/ParserCreateQuotaQuery.cpp @@ -11,7 +11,10 @@ #include #include #include +#include +#include #include +#include namespace DB @@ -24,9 +27,6 @@ namespace ErrorCodes namespace { - using KeyType = Quota::KeyType; - using KeyTypeInfo = Quota::KeyTypeInfo; - bool parseRenameTo(IParserBase::Pos & pos, Expected & expected, String & new_name) { return IParserBase::wrapParseImpl(pos, [&] @@ -38,13 +38,13 @@ namespace }); } - bool parseKeyType(IParserBase::Pos & pos, Expected & expected, KeyType & key_type) + bool parseKeyType(IParserBase::Pos & pos, Expected & expected, QuotaKeyType & key_type) { return IParserBase::wrapParseImpl(pos, [&] { if (ParserKeyword{"NOT KEYED"}.ignore(pos, expected)) { - key_type = KeyType::NONE; + key_type = QuotaKeyType::NONE; return true; } @@ -59,9 +59,9 @@ namespace boost::to_lower(name); boost::replace_all(name, " ", "_"); - for (auto kt : collections::range(Quota::KeyType::MAX)) + for (auto kt : collections::range(QuotaKeyType::MAX)) { - if (KeyTypeInfo::get(kt).name == name) + if (QuotaKeyTypeInfo::get(kt).name == name) { key_type = kt; return true; @@ -69,8 +69,8 @@ namespace } String all_types_str; - for (auto kt : collections::range(Quota::KeyType::MAX)) - all_types_str += String(all_types_str.empty() ? "" : ", ") + "'" + KeyTypeInfo::get(kt).name + "'"; + for (auto kt : collections::range(QuotaKeyType::MAX)) + all_types_str += String(all_types_str.empty() ? "" : ", ") + "'" + QuotaKeyTypeInfo::get(kt).name + "'"; String msg = "Quota cannot be keyed by '" + name + "'. Expected one of the following identifiers: " + all_types_str; throw Exception(msg, ErrorCodes::SYNTAX_ERROR); }); @@ -278,7 +278,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe return false; String new_name; - std::optional key_type; + std::optional key_type; std::vector all_limits; String cluster; @@ -289,7 +289,7 @@ bool ParserCreateQuotaQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe if (!key_type) { - KeyType new_key_type; + QuotaKeyType new_key_type; if (parseKeyType(pos, expected, new_key_type)) { key_type = new_key_type; diff --git a/src/Storages/System/StorageSystemQuotas.cpp b/src/Storages/System/StorageSystemQuotas.cpp index 2294af87fed..fa262f22d2c 100644 --- a/src/Storages/System/StorageSystemQuotas.cpp +++ b/src/Storages/System/StorageSystemQuotas.cpp @@ -19,16 +19,13 @@ namespace DB { namespace { - using KeyType = Quota::KeyType; - using KeyTypeInfo = Quota::KeyTypeInfo; - DataTypeEnum8::Values getKeyTypeEnumValues() { DataTypeEnum8::Values enum_values; - for (auto key_type : collections::range(KeyType::MAX)) + for (auto key_type : collections::range(QuotaKeyType::MAX)) { - const auto & type_info = KeyTypeInfo::get(key_type); - if ((key_type != KeyType::NONE) && type_info.base_types.empty()) + const auto & type_info = QuotaKeyTypeInfo::get(key_type); + if ((key_type != QuotaKeyType::NONE) && type_info.base_types.empty()) enum_values.push_back({type_info.name, static_cast(key_type)}); } return enum_values; @@ -76,16 +73,16 @@ void StorageSystemQuotas::fillData(MutableColumns & res_columns, ContextPtr cont const UUID & id, const String & storage_name, const std::vector & all_limits, - KeyType key_type, + QuotaKeyType key_type, const RolesOrUsersSet & apply_to) { column_name.insertData(name.data(), name.length()); column_id.push_back(id.toUnderType()); column_storage.insertData(storage_name.data(), storage_name.length()); - if (key_type != KeyType::NONE) + if (key_type != QuotaKeyType::NONE) { - const auto & type_info = KeyTypeInfo::get(key_type); + const auto & type_info = QuotaKeyTypeInfo::get(key_type); for (auto base_type : type_info.base_types) column_key_types.push_back(static_cast(base_type)); if (type_info.base_types.empty()) From af56b20da7c1dd0a0e515be757b03ff3f11f120a Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Thu, 18 Nov 2021 23:54:18 +0300 Subject: [PATCH 148/200] Rename IAccessEntity::Type -> AccessEntityType and move it to Access/Common. --- src/Access/AccessEntityIO.cpp | 5 +- src/Access/AccessEntityIO.h | 5 +- src/Access/Common/AccessEntityType.cpp | 86 +++++++++++++ src/Access/Common/AccessEntityType.h | 40 ++++++ src/Access/Common/QuotaDefs.h | 2 +- src/Access/ContextAccess.h | 1 + src/Access/DiskAccessStorage.cpp | 31 +++-- src/Access/DiskAccessStorage.h | 18 +-- src/Access/IAccessEntity.h | 116 +----------------- src/Access/IAccessStorage.cpp | 72 ++++++----- src/Access/IAccessStorage.h | 43 +++---- src/Access/LDAPAccessStorage.cpp | 8 +- src/Access/LDAPAccessStorage.h | 8 +- src/Access/MemoryAccessStorage.cpp | 12 +- src/Access/MemoryAccessStorage.h | 12 +- src/Access/MultipleAccessStorage.cpp | 31 ++--- src/Access/MultipleAccessStorage.h | 12 +- src/Access/Quota.h | 4 +- src/Access/ReplicatedAccessStorage.cpp | 32 ++--- src/Access/ReplicatedAccessStorage.h | 12 +- src/Access/Role.h | 4 +- src/Access/RowPolicy.h | 4 +- src/Access/SettingsProfile.h | 4 +- src/Access/User.h | 4 +- src/Access/UsersConfigAccessStorage.cpp | 23 ++-- src/Access/UsersConfigAccessStorage.h | 8 +- .../InterpreterDropAccessEntityQuery.cpp | 16 ++- .../Access/InterpreterGrantQuery.cpp | 2 +- .../InterpreterShowAccessEntitiesQuery.cpp | 14 +-- .../Access/InterpreterShowAccessQuery.cpp | 6 +- ...InterpreterShowCreateAccessEntityQuery.cpp | 18 ++- .../Access/InterpreterShowGrantsQuery.cpp | 2 +- .../Access/ASTDropAccessEntityQuery.cpp | 6 +- src/Parsers/Access/ASTDropAccessEntityQuery.h | 6 +- .../Access/ASTShowAccessEntitiesQuery.cpp | 4 +- .../Access/ASTShowAccessEntitiesQuery.h | 6 +- .../Access/ASTShowCreateAccessEntityQuery.cpp | 5 +- .../Access/ASTShowCreateAccessEntityQuery.h | 6 +- .../Access/ParserDropAccessEntityQuery.cpp | 16 +-- .../Access/ParserShowAccessEntitiesQuery.cpp | 19 ++- .../ParserShowCreateAccessEntityQuery.cpp | 27 ++-- src/Storages/System/StorageSystemGrants.cpp | 9 +- .../System/StorageSystemRoleGrants.cpp | 10 +- .../StorageSystemSettingsProfileElements.cpp | 12 +- 44 files changed, 379 insertions(+), 402 deletions(-) create mode 100644 src/Access/Common/AccessEntityType.cpp create mode 100644 src/Access/Common/AccessEntityType.h diff --git a/src/Access/AccessEntityIO.cpp b/src/Access/AccessEntityIO.cpp index 199b3b22efc..86aed1b5ad4 100644 --- a/src/Access/AccessEntityIO.cpp +++ b/src/Access/AccessEntityIO.cpp @@ -39,9 +39,6 @@ namespace ErrorCodes extern const int INCORRECT_ACCESS_ENTITY_DEFINITION; } -using EntityType = IAccessStorage::EntityType; -using EntityTypeInfo = IAccessStorage::EntityTypeInfo; - namespace { /// Special parser for the 'ATTACH access entity' queries. @@ -80,7 +77,7 @@ String serializeAccessEntity(const IAccessEntity & entity) /// Build list of ATTACH queries. ASTs queries; queries.push_back(InterpreterShowCreateAccessEntityQuery::getAttachQuery(entity)); - if ((entity.getType() == EntityType::USER) || (entity.getType() == EntityType::ROLE)) + if ((entity.getType() == AccessEntityType::USER) || (entity.getType() == AccessEntityType::ROLE)) boost::range::push_back(queries, InterpreterShowGrantsQuery::getAttachGrantQueries(entity)); /// Serialize the list of ATTACH queries to a string. diff --git a/src/Access/AccessEntityIO.h b/src/Access/AccessEntityIO.h index 94dc027430e..aa0a3e7cf63 100644 --- a/src/Access/AccessEntityIO.h +++ b/src/Access/AccessEntityIO.h @@ -1,9 +1,12 @@ #pragma once -#include +#include +#include namespace DB { +struct IAccessEntity; +using AccessEntityPtr = std::shared_ptr; String serializeAccessEntity(const IAccessEntity & entity); diff --git a/src/Access/Common/AccessEntityType.cpp b/src/Access/Common/AccessEntityType.cpp new file mode 100644 index 00000000000..b9c618a9fc0 --- /dev/null +++ b/src/Access/Common/AccessEntityType.cpp @@ -0,0 +1,86 @@ +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int UNKNOWN_USER; + extern const int UNKNOWN_ROLE; + extern const int UNKNOWN_ROW_POLICY; + extern const int UNKNOWN_QUOTA; + extern const int THERE_IS_NO_PROFILE; + extern const int LOGICAL_ERROR; +} + + +String toString(AccessEntityType type) +{ + return AccessEntityTypeInfo::get(type).name; +} + +String AccessEntityTypeInfo::formatEntityNameWithType(const String & entity_name) const +{ + String msg = name_for_output_with_entity_name; + msg += " "; + msg += backQuote(entity_name); + return msg; +} + +const AccessEntityTypeInfo & AccessEntityTypeInfo::get(AccessEntityType type_) +{ + static constexpr auto make_info = [](const char * raw_name_, const char * plural_raw_name_, char unique_char_, int not_found_error_code_) + { + String init_names[2] = {raw_name_, plural_raw_name_}; + String init_aliases[2]; + for (size_t i = 0; i != std::size(init_names); ++i) + { + String & init_name = init_names[i]; + String & init_alias = init_aliases[i]; + boost::to_upper(init_name); + boost::replace_all(init_name, "_", " "); + if (auto underscore_pos = init_name.find_first_of(' '); underscore_pos != String::npos) + init_alias = init_name.substr(underscore_pos + 1); + } + String init_name_for_output_with_entity_name = init_names[0]; + boost::to_lower(init_name_for_output_with_entity_name); + return AccessEntityTypeInfo{raw_name_, plural_raw_name_, std::move(init_names[0]), std::move(init_aliases[0]), std::move(init_names[1]), std::move(init_aliases[1]), std::move(init_name_for_output_with_entity_name), unique_char_, not_found_error_code_}; + }; + + switch (type_) + { + case AccessEntityType::USER: + { + static const auto info = make_info("USER", "USERS", 'U', ErrorCodes::UNKNOWN_USER); + return info; + } + case AccessEntityType::ROLE: + { + static const auto info = make_info("ROLE", "ROLES", 'R', ErrorCodes::UNKNOWN_ROLE); + return info; + } + case AccessEntityType::SETTINGS_PROFILE: + { + static const auto info = make_info("SETTINGS_PROFILE", "SETTINGS_PROFILES", 'S', ErrorCodes::THERE_IS_NO_PROFILE); + return info; + } + case AccessEntityType::ROW_POLICY: + { + static const auto info = make_info("ROW_POLICY", "ROW_POLICIES", 'P', ErrorCodes::UNKNOWN_ROW_POLICY); + return info; + } + case AccessEntityType::QUOTA: + { + static const auto info = make_info("QUOTA", "QUOTAS", 'Q', ErrorCodes::UNKNOWN_QUOTA); + return info; + } + case AccessEntityType::MAX: break; + } + throw Exception("Unknown type: " + std::to_string(static_cast(type_)), ErrorCodes::LOGICAL_ERROR); +} + +} diff --git a/src/Access/Common/AccessEntityType.h b/src/Access/Common/AccessEntityType.h new file mode 100644 index 00000000000..44caeecb37a --- /dev/null +++ b/src/Access/Common/AccessEntityType.h @@ -0,0 +1,40 @@ +#pragma once + +#include + + +namespace DB +{ + +/// Represents the type of an access entity (see the IAccessEntity class). +enum class AccessEntityType +{ + USER, + ROLE, + SETTINGS_PROFILE, + ROW_POLICY, + QUOTA, + + MAX, +}; + +String toString(AccessEntityType type); + +struct AccessEntityTypeInfo +{ + const char * const raw_name; + const char * const plural_raw_name; + const String name; /// Uppercased with spaces instead of underscores, e.g. "SETTINGS PROFILE". + const String alias; /// Alias of the keyword or empty string, e.g. "PROFILE". + const String plural_name; /// Uppercased with spaces plural name, e.g. "SETTINGS PROFILES". + const String plural_alias; /// Uppercased with spaces plural name alias, e.g. "PROFILES". + const String name_for_output_with_entity_name; /// Lowercased with spaces instead of underscores, e.g. "settings profile". + const char unique_char; /// Unique character for this type. E.g. 'P' for SETTINGS_PROFILE. + const int not_found_error_code; + + String formatEntityNameWithType(const String & entity_name) const; + + static const AccessEntityTypeInfo & get(AccessEntityType type_); +}; + +} diff --git a/src/Access/Common/QuotaDefs.h b/src/Access/Common/QuotaDefs.h index e0c26282862..cfd8a07d9ff 100644 --- a/src/Access/Common/QuotaDefs.h +++ b/src/Access/Common/QuotaDefs.h @@ -60,7 +60,7 @@ struct QuotaKeyTypeInfo { const char * const raw_name; const String name; /// Lowercased with underscores, e.g. "client_key". - const std::vector base_types; /// For combined types keeps base types, e.g. for CLIENT_KEY_OR_USER_NAME it keeps [KeyType::CLIENT_KEY, KeyType::USER_NAME]. + const std::vector base_types; /// For combined types keeps base types, e.g. for CLIENT_KEY_OR_USER_NAME it keeps [KeyType::CLIENT_KEY, KeyAccessEntityType::USER_NAME]. static const QuotaKeyTypeInfo & get(QuotaKeyType type); }; diff --git a/src/Access/ContextAccess.h b/src/Access/ContextAccess.h index 05d532ae6cb..d7cc999e95f 100644 --- a/src/Access/ContextAccess.h +++ b/src/Access/ContextAccess.h @@ -8,6 +8,7 @@ #include #include #include +#include #include diff --git a/src/Access/DiskAccessStorage.cpp b/src/Access/DiskAccessStorage.cpp index 4c53ba8b632..1428e546d34 100644 --- a/src/Access/DiskAccessStorage.cpp +++ b/src/Access/DiskAccessStorage.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -26,10 +27,6 @@ namespace ErrorCodes namespace { - using EntityType = IAccessStorage::EntityType; - using EntityTypeInfo = IAccessStorage::EntityTypeInfo; - - /// Reads a file containing ATTACH queries and then parses it to build an access entity. AccessEntityPtr readEntityFile(const String & file_path) { @@ -136,9 +133,9 @@ namespace /// Calculates the path for storing a map of name of access entity to UUID for access entities of some type. - String getListFilePath(const String & directory_path, EntityType type) + String getListFilePath(const String & directory_path, AccessEntityType type) { - String file_name = EntityTypeInfo::get(type).plural_raw_name; + String file_name = AccessEntityTypeInfo::get(type).plural_raw_name; boost::to_lower(file_name); return directory_path + file_name + ".list"; } @@ -230,7 +227,7 @@ bool DiskAccessStorage::isPathEqual(const String & directory_path_) const void DiskAccessStorage::clear() { entries_by_id.clear(); - for (auto type : collections::range(EntityType::MAX)) + for (auto type : collections::range(AccessEntityType::MAX)) entries_by_name_and_type[static_cast(type)].clear(); } @@ -240,7 +237,7 @@ bool DiskAccessStorage::readLists() clear(); bool ok = true; - for (auto type : collections::range(EntityType::MAX)) + for (auto type : collections::range(AccessEntityType::MAX)) { auto & entries_by_name = entries_by_name_and_type[static_cast(type)]; auto file_path = getListFilePath(directory_path, type); @@ -313,7 +310,7 @@ bool DiskAccessStorage::writeLists() } -void DiskAccessStorage::scheduleWriteLists(EntityType type) +void DiskAccessStorage::scheduleWriteLists(AccessEntityType type) { if (failed_to_write_lists) return; /// We don't try to write list files after the first fail. @@ -399,14 +396,14 @@ bool DiskAccessStorage::rebuildLists() entries_by_name[entry.name] = &entry; } - for (auto type : collections::range(EntityType::MAX)) + for (auto type : collections::range(AccessEntityType::MAX)) types_of_lists_to_write.insert(type); return true; } -std::optional DiskAccessStorage::findImpl(EntityType type, const String & name) const +std::optional DiskAccessStorage::findImpl(AccessEntityType type, const String & name) const { std::lock_guard lock{mutex}; const auto & entries_by_name = entries_by_name_and_type[static_cast(type)]; @@ -418,7 +415,7 @@ std::optional DiskAccessStorage::findImpl(EntityType type, const String & } -std::vector DiskAccessStorage::findAllImpl(EntityType type) const +std::vector DiskAccessStorage::findAllImpl(AccessEntityType type) const { std::lock_guard lock{mutex}; const auto & entries_by_name = entries_by_name_and_type[static_cast(type)]; @@ -481,7 +478,7 @@ UUID DiskAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool repl void DiskAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, Notifications & notifications) { const String & name = new_entity->getName(); - EntityType type = new_entity->getType(); + AccessEntityType type = new_entity->getType(); if (readonly) throwReadonlyCannotInsert(type, name); @@ -535,7 +532,7 @@ void DiskAccessStorage::removeNoLock(const UUID & id, Notifications & notificati throwNotFound(id); Entry & entry = it->second; - EntityType type = entry.type; + AccessEntityType type = entry.type; if (readonly) throwReadonlyCannotRemove(type, entry.name); @@ -583,7 +580,7 @@ void DiskAccessStorage::updateNoLock(const UUID & id, const UpdateFunc & update_ const String & new_name = new_entity->getName(); const String & old_name = old_entity->getName(); - const EntityType type = entry.type; + const AccessEntityType type = entry.type; auto & entries_by_name = entries_by_name_and_type[static_cast(type)]; bool name_changed = (new_name != old_name); @@ -663,7 +660,7 @@ scope_guard DiskAccessStorage::subscribeForChangesImpl(const UUID & id, const On }; } -scope_guard DiskAccessStorage::subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const +scope_guard DiskAccessStorage::subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const { std::lock_guard lock{mutex}; auto & handlers = handlers_by_type[static_cast(type)]; @@ -690,7 +687,7 @@ bool DiskAccessStorage::hasSubscriptionImpl(const UUID & id) const return false; } -bool DiskAccessStorage::hasSubscriptionImpl(EntityType type) const +bool DiskAccessStorage::hasSubscriptionImpl(AccessEntityType type) const { std::lock_guard lock{mutex}; const auto & handlers = handlers_by_type[static_cast(type)]; diff --git a/src/Access/DiskAccessStorage.h b/src/Access/DiskAccessStorage.h index 5e3f4a1eafa..853a18590f0 100644 --- a/src/Access/DiskAccessStorage.h +++ b/src/Access/DiskAccessStorage.h @@ -27,8 +27,8 @@ public: bool isReadOnly() const { return readonly; } private: - std::optional findImpl(EntityType type, const String & name) const override; - std::vector findAllImpl(EntityType type) const override; + std::optional findImpl(AccessEntityType type, const String & name) const override; + std::vector findAllImpl(AccessEntityType type) const override; bool existsImpl(const UUID & id) const override; AccessEntityPtr readImpl(const UUID & id) const override; String readNameImpl(const UUID & id) const override; @@ -37,14 +37,14 @@ private: void removeImpl(const UUID & id) override; void updateImpl(const UUID & id, const UpdateFunc & update_func) override; scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override; - scope_guard subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const override; + scope_guard subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const override; bool hasSubscriptionImpl(const UUID & id) const override; - bool hasSubscriptionImpl(EntityType type) const override; + bool hasSubscriptionImpl(AccessEntityType type) const override; void clear(); bool readLists(); bool writeLists(); - void scheduleWriteLists(EntityType type); + void scheduleWriteLists(AccessEntityType type); bool rebuildLists(); void listsWritingThreadFunc(); @@ -63,7 +63,7 @@ private: { UUID id; String name; - EntityType type; + AccessEntityType type; mutable AccessEntityPtr entity; /// may be nullptr, if the entity hasn't been loaded yet. mutable std::list handlers_by_id; }; @@ -73,13 +73,13 @@ private: String directory_path; std::atomic readonly; std::unordered_map entries_by_id; - std::unordered_map entries_by_name_and_type[static_cast(EntityType::MAX)]; - boost::container::flat_set types_of_lists_to_write; + std::unordered_map entries_by_name_and_type[static_cast(AccessEntityType::MAX)]; + boost::container::flat_set types_of_lists_to_write; bool failed_to_write_lists = false; /// Whether writing of the list files has been failed since the recent restart of the server. ThreadFromGlobalPool lists_writing_thread; /// List files are written in a separate thread. std::condition_variable lists_writing_thread_should_exit; /// Signals `lists_writing_thread` to exit. bool lists_writing_thread_is_waiting = false; - mutable std::list handlers_by_type[static_cast(EntityType::MAX)]; + mutable std::list handlers_by_type[static_cast(AccessEntityType::MAX)]; mutable std::mutex mutex; }; } diff --git a/src/Access/IAccessEntity.h b/src/Access/IAccessEntity.h index d0fbde8da97..04faa879040 100644 --- a/src/Access/IAccessEntity.h +++ b/src/Access/IAccessEntity.h @@ -1,24 +1,13 @@ #pragma once -#include +#include #include -#include -#include +#include #include namespace DB { -namespace ErrorCodes -{ - extern const int UNKNOWN_USER; - extern const int UNKNOWN_ROLE; - extern const int UNKNOWN_ROW_POLICY; - extern const int UNKNOWN_QUOTA; - extern const int THERE_IS_NO_PROFILE; - extern const int LOGICAL_ERROR; -} - /// Access entity is a set of data which have a name and a type. Access entity control something related to the access control. /// Entities can be stored to a file or another storage, see IAccessStorage. @@ -29,41 +18,14 @@ struct IAccessEntity virtual ~IAccessEntity() = default; virtual std::shared_ptr clone() const = 0; - enum class Type - { - USER, - ROLE, - SETTINGS_PROFILE, - ROW_POLICY, - QUOTA, + virtual AccessEntityType getType() const = 0; - MAX, - }; - - virtual Type getType() const = 0; - - struct TypeInfo - { - const char * const raw_name; - const char * const plural_raw_name; - const String name; /// Uppercased with spaces instead of underscores, e.g. "SETTINGS PROFILE". - const String alias; /// Alias of the keyword or empty string, e.g. "PROFILE". - const String plural_name; /// Uppercased with spaces plural name, e.g. "SETTINGS PROFILES". - const String plural_alias; /// Uppercased with spaces plural name alias, e.g. "PROFILES". - const String name_for_output_with_entity_name; /// Lowercased with spaces instead of underscores, e.g. "settings profile". - const char unique_char; /// Unique character for this type. E.g. 'P' for SETTINGS_PROFILE. - const int not_found_error_code; - - static const TypeInfo & get(Type type_); - String outputWithEntityName(const String & entity_name) const; - }; - - const TypeInfo & getTypeInfo() const { return TypeInfo::get(getType()); } - String outputTypeAndName() const { return getTypeInfo().outputWithEntityName(getName()); } + const AccessEntityTypeInfo & getTypeInfo() const { return AccessEntityTypeInfo::get(getType()); } + String formatTypeWithName() const { return getTypeInfo().formatEntityNameWithType(getName()); } template bool isTypeOf() const { return isTypeOf(EntityClassT::TYPE); } - bool isTypeOf(Type type) const { return type == getType(); } + bool isTypeOf(AccessEntityType type) const { return type == getType(); } virtual void setName(const String & name_) { name = name_; } const String & getName() const { return name; } @@ -98,70 +60,4 @@ protected: using AccessEntityPtr = std::shared_ptr; - -inline const IAccessEntity::TypeInfo & IAccessEntity::TypeInfo::get(Type type_) -{ - static constexpr auto make_info = [](const char * raw_name_, const char * plural_raw_name_, char unique_char_, int not_found_error_code_) - { - String init_names[2] = {raw_name_, plural_raw_name_}; - String init_aliases[2]; - for (size_t i = 0; i != std::size(init_names); ++i) - { - String & init_name = init_names[i]; - String & init_alias = init_aliases[i]; - boost::to_upper(init_name); - boost::replace_all(init_name, "_", " "); - if (auto underscore_pos = init_name.find_first_of(" "); underscore_pos != String::npos) - init_alias = init_name.substr(underscore_pos + 1); - } - String init_name_for_output_with_entity_name = init_names[0]; - boost::to_lower(init_name_for_output_with_entity_name); - return TypeInfo{raw_name_, plural_raw_name_, std::move(init_names[0]), std::move(init_aliases[0]), std::move(init_names[1]), std::move(init_aliases[1]), std::move(init_name_for_output_with_entity_name), unique_char_, not_found_error_code_}; - }; - - switch (type_) - { - case Type::USER: - { - static const auto info = make_info("USER", "USERS", 'U', ErrorCodes::UNKNOWN_USER); - return info; - } - case Type::ROLE: - { - static const auto info = make_info("ROLE", "ROLES", 'R', ErrorCodes::UNKNOWN_ROLE); - return info; - } - case Type::SETTINGS_PROFILE: - { - static const auto info = make_info("SETTINGS_PROFILE", "SETTINGS_PROFILES", 'S', ErrorCodes::THERE_IS_NO_PROFILE); - return info; - } - case Type::ROW_POLICY: - { - static const auto info = make_info("ROW_POLICY", "ROW_POLICIES", 'P', ErrorCodes::UNKNOWN_ROW_POLICY); - return info; - } - case Type::QUOTA: - { - static const auto info = make_info("QUOTA", "QUOTAS", 'Q', ErrorCodes::UNKNOWN_QUOTA); - return info; - } - case Type::MAX: break; - } - throw Exception("Unknown type: " + std::to_string(static_cast(type_)), ErrorCodes::LOGICAL_ERROR); -} - -inline String IAccessEntity::TypeInfo::outputWithEntityName(const String & entity_name) const -{ - String msg = name_for_output_with_entity_name; - msg += " "; - msg += backQuote(entity_name); - return msg; -} - -inline String toString(IAccessEntity::Type type) -{ - return IAccessEntity::TypeInfo::get(type).name; -} - } diff --git a/src/Access/IAccessStorage.cpp b/src/Access/IAccessStorage.cpp index a0ad5d4ec79..51c2525d923 100644 --- a/src/Access/IAccessStorage.cpp +++ b/src/Access/IAccessStorage.cpp @@ -8,6 +8,8 @@ #include #include #include +#include +#include namespace DB @@ -26,20 +28,16 @@ namespace ErrorCodes namespace { - using EntityType = IAccessStorage::EntityType; - using EntityTypeInfo = IAccessStorage::EntityTypeInfo; - - String outputID(const UUID & id) { return "ID(" + toString(id) + ")"; } - String outputTypeAndNameOrID(const IAccessStorage & storage, const UUID & id) + String formatTypeWithNameOrID(const IAccessStorage & storage, const UUID & id) { auto entity = storage.tryRead(id); if (entity) - return entity->outputTypeAndName(); + return entity->formatTypeWithName(); return outputID(id); } @@ -132,19 +130,19 @@ namespace } -std::vector IAccessStorage::findAll(EntityType type) const +std::vector IAccessStorage::findAll(AccessEntityType type) const { return findAllImpl(type); } -std::optional IAccessStorage::find(EntityType type, const String & name) const +std::optional IAccessStorage::find(AccessEntityType type, const String & name) const { return findImpl(type, name); } -std::vector IAccessStorage::find(EntityType type, const Strings & names) const +std::vector IAccessStorage::find(AccessEntityType type, const Strings & names) const { std::vector ids; ids.reserve(names.size()); @@ -158,7 +156,7 @@ std::vector IAccessStorage::find(EntityType type, const Strings & names) c } -UUID IAccessStorage::getID(EntityType type, const String & name) const +UUID IAccessStorage::getID(AccessEntityType type, const String & name) const { auto id = findImpl(type, name); if (id) @@ -167,7 +165,7 @@ UUID IAccessStorage::getID(EntityType type, const String & name) const } -std::vector IAccessStorage::getIDs(EntityType type, const Strings & names) const +std::vector IAccessStorage::getIDs(AccessEntityType type, const Strings & names) const { std::vector ids; ids.reserve(names.size()); @@ -253,7 +251,7 @@ std::vector IAccessStorage::insert(const std::vector & mu if (tracker.errors()) { - auto get_name_function = [&](size_t i) { return multiple_entities[i]->outputTypeAndName(); }; + auto get_name_function = [&](size_t i) { return multiple_entities[i]->formatTypeWithName(); }; tracker.showErrors("Couldn't insert {failed_names}. Successfully inserted: {succeeded_names}", get_name_function); } @@ -306,7 +304,7 @@ std::vector IAccessStorage::insertOrReplace(const std::vectoroutputTypeAndName(); }; + auto get_name_function = [&](size_t i) { return multiple_entities[i]->formatTypeWithName(); }; tracker.showErrors("Couldn't insert {failed_names}. Successfully inserted: {succeeded_names}", get_name_function); } @@ -332,7 +330,7 @@ void IAccessStorage::remove(const std::vector & ids) if (tracker.errors()) { - auto get_name_function = [&](size_t i) { return outputTypeAndNameOrID(*this, ids[i]); }; + auto get_name_function = [&](size_t i) { return formatTypeWithNameOrID(*this, ids[i]); }; tracker.showErrors("Couldn't remove {failed_names}. Successfully removed: {succeeded_names}", get_name_function); } } @@ -376,7 +374,7 @@ void IAccessStorage::update(const std::vector & ids, const UpdateFunc & up if (tracker.errors()) { - auto get_name_function = [&](size_t i) { return outputTypeAndNameOrID(*this, ids[i]); }; + auto get_name_function = [&](size_t i) { return formatTypeWithNameOrID(*this, ids[i]); }; tracker.showErrors("Couldn't update {failed_names}. Successfully updated: {succeeded_names}", get_name_function); } } @@ -402,7 +400,7 @@ std::vector IAccessStorage::tryUpdate(const std::vector & ids, const } -scope_guard IAccessStorage::subscribeForChanges(EntityType type, const OnChangedHandler & handler) const +scope_guard IAccessStorage::subscribeForChanges(AccessEntityType type, const OnChangedHandler & handler) const { return subscribeForChangesImpl(type, handler); } @@ -423,7 +421,7 @@ scope_guard IAccessStorage::subscribeForChanges(const std::vector & ids, c } -bool IAccessStorage::hasSubscription(EntityType type) const +bool IAccessStorage::hasSubscription(AccessEntityType type) const { return hasSubscriptionImpl(type); } @@ -481,7 +479,7 @@ UUID IAccessStorage::loginImpl( return *id; } } - throwNotFound(EntityType::USER, credentials.getUserName()); + throwNotFound(AccessEntityType::USER, credentials.getUserName()); } @@ -542,68 +540,68 @@ void IAccessStorage::throwNotFound(const UUID & id) const } -void IAccessStorage::throwNotFound(EntityType type, const String & name) const +void IAccessStorage::throwNotFound(AccessEntityType type, const String & name) const { - int error_code = EntityTypeInfo::get(type).not_found_error_code; - throw Exception("There is no " + outputEntityTypeAndName(type, name) + " in " + getStorageName(), error_code); + int error_code = AccessEntityTypeInfo::get(type).not_found_error_code; + throw Exception("There is no " + formatEntityTypeWithName(type, name) + " in " + getStorageName(), error_code); } -void IAccessStorage::throwBadCast(const UUID & id, EntityType type, const String & name, EntityType required_type) +void IAccessStorage::throwBadCast(const UUID & id, AccessEntityType type, const String & name, AccessEntityType required_type) { throw Exception( - outputID(id) + ": " + outputEntityTypeAndName(type, name) + " expected to be of type " + toString(required_type), + outputID(id) + ": " + formatEntityTypeWithName(type, name) + " expected to be of type " + toString(required_type), ErrorCodes::LOGICAL_ERROR); } -void IAccessStorage::throwIDCollisionCannotInsert(const UUID & id, EntityType type, const String & name, EntityType existing_type, const String & existing_name) const +void IAccessStorage::throwIDCollisionCannotInsert(const UUID & id, AccessEntityType type, const String & name, AccessEntityType existing_type, const String & existing_name) const { throw Exception( - outputEntityTypeAndName(type, name) + ": cannot insert because the " + outputID(id) + " is already used by " - + outputEntityTypeAndName(existing_type, existing_name) + " in " + getStorageName(), + formatEntityTypeWithName(type, name) + ": cannot insert because the " + outputID(id) + " is already used by " + + formatEntityTypeWithName(existing_type, existing_name) + " in " + getStorageName(), ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS); } -void IAccessStorage::throwNameCollisionCannotInsert(EntityType type, const String & name) const +void IAccessStorage::throwNameCollisionCannotInsert(AccessEntityType type, const String & name) const { throw Exception( - outputEntityTypeAndName(type, name) + ": cannot insert because " + outputEntityTypeAndName(type, name) + " already exists in " + formatEntityTypeWithName(type, name) + ": cannot insert because " + formatEntityTypeWithName(type, name) + " already exists in " + getStorageName(), ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS); } -void IAccessStorage::throwNameCollisionCannotRename(EntityType type, const String & old_name, const String & new_name) const +void IAccessStorage::throwNameCollisionCannotRename(AccessEntityType type, const String & old_name, const String & new_name) const { throw Exception( - outputEntityTypeAndName(type, old_name) + ": cannot rename to " + backQuote(new_name) + " because " - + outputEntityTypeAndName(type, new_name) + " already exists in " + getStorageName(), + formatEntityTypeWithName(type, old_name) + ": cannot rename to " + backQuote(new_name) + " because " + + formatEntityTypeWithName(type, new_name) + " already exists in " + getStorageName(), ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS); } -void IAccessStorage::throwReadonlyCannotInsert(EntityType type, const String & name) const +void IAccessStorage::throwReadonlyCannotInsert(AccessEntityType type, const String & name) const { throw Exception( - "Cannot insert " + outputEntityTypeAndName(type, name) + " to " + getStorageName() + " because this storage is readonly", + "Cannot insert " + formatEntityTypeWithName(type, name) + " to " + getStorageName() + " because this storage is readonly", ErrorCodes::ACCESS_STORAGE_READONLY); } -void IAccessStorage::throwReadonlyCannotUpdate(EntityType type, const String & name) const +void IAccessStorage::throwReadonlyCannotUpdate(AccessEntityType type, const String & name) const { throw Exception( - "Cannot update " + outputEntityTypeAndName(type, name) + " in " + getStorageName() + " because this storage is readonly", + "Cannot update " + formatEntityTypeWithName(type, name) + " in " + getStorageName() + " because this storage is readonly", ErrorCodes::ACCESS_STORAGE_READONLY); } -void IAccessStorage::throwReadonlyCannotRemove(EntityType type, const String & name) const +void IAccessStorage::throwReadonlyCannotRemove(AccessEntityType type, const String & name) const { throw Exception( - "Cannot remove " + outputEntityTypeAndName(type, name) + " from " + getStorageName() + " because this storage is readonly", + "Cannot remove " + formatEntityTypeWithName(type, name) + " from " + getStorageName() + " because this storage is readonly", ErrorCodes::ACCESS_STORAGE_READONLY); } diff --git a/src/Access/IAccessStorage.h b/src/Access/IAccessStorage.h index e9df2e3c1a4..ccbb1ffe5bc 100644 --- a/src/Access/IAccessStorage.h +++ b/src/Access/IAccessStorage.h @@ -34,33 +34,30 @@ public: /// Returns a JSON with the parameters of the storage. It's up to the storage type to fill the JSON. virtual String getStorageParamsJSON() const { return "{}"; } - using EntityType = IAccessEntity::Type; - using EntityTypeInfo = IAccessEntity::TypeInfo; - /// Returns the identifiers of all the entities of a specified type contained in the storage. - std::vector findAll(EntityType type) const; + std::vector findAll(AccessEntityType type) const; template std::vector findAll() const { return findAll(EntityClassT::TYPE); } /// Searches for an entity with specified type and name. Returns std::nullopt if not found. - std::optional find(EntityType type, const String & name) const; + std::optional find(AccessEntityType type, const String & name) const; template std::optional find(const String & name) const { return find(EntityClassT::TYPE, name); } - std::vector find(EntityType type, const Strings & names) const; + std::vector find(AccessEntityType type, const Strings & names) const; template std::vector find(const Strings & names) const { return find(EntityClassT::TYPE, names); } /// Searches for an entity with specified name and type. Throws an exception if not found. - UUID getID(EntityType type, const String & name) const; + UUID getID(AccessEntityType type, const String & name) const; template UUID getID(const String & name) const { return getID(EntityClassT::TYPE, name); } - std::vector getIDs(EntityType type, const Strings & names) const; + std::vector getIDs(AccessEntityType type, const Strings & names) const; template std::vector getIDs(const Strings & names) const { return getIDs(EntityClassT::TYPE, names); } @@ -132,7 +129,7 @@ public: /// Subscribes for all changes. /// Can return nullptr if cannot subscribe (identifier not found) or if it doesn't make sense (the storage is read-only). - scope_guard subscribeForChanges(EntityType type, const OnChangedHandler & handler) const; + scope_guard subscribeForChanges(AccessEntityType type, const OnChangedHandler & handler) const; template scope_guard subscribeForChanges(OnChangedHandler handler) const { return subscribeForChanges(EntityClassT::TYPE, handler); } @@ -142,7 +139,7 @@ public: scope_guard subscribeForChanges(const UUID & id, const OnChangedHandler & handler) const; scope_guard subscribeForChanges(const std::vector & ids, const OnChangedHandler & handler) const; - bool hasSubscription(EntityType type) const; + bool hasSubscription(AccessEntityType type) const; bool hasSubscription(const UUID & id) const; /// Finds a user, check the provided credentials and returns the ID of the user if they are valid. @@ -154,8 +151,8 @@ public: UUID getIDOfLoggedUser(const String & user_name) const; protected: - virtual std::optional findImpl(EntityType type, const String & name) const = 0; - virtual std::vector findAllImpl(EntityType type) const = 0; + virtual std::optional findImpl(AccessEntityType type, const String & name) const = 0; + virtual std::vector findAllImpl(AccessEntityType type) const = 0; virtual bool existsImpl(const UUID & id) const = 0; virtual AccessEntityPtr readImpl(const UUID & id) const = 0; virtual String readNameImpl(const UUID & id) const = 0; @@ -164,9 +161,9 @@ protected: virtual void removeImpl(const UUID & id) = 0; virtual void updateImpl(const UUID & id, const UpdateFunc & update_func) = 0; virtual scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const = 0; - virtual scope_guard subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const = 0; + virtual scope_guard subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const = 0; virtual bool hasSubscriptionImpl(const UUID & id) const = 0; - virtual bool hasSubscriptionImpl(EntityType type) const = 0; + virtual bool hasSubscriptionImpl(AccessEntityType type) const = 0; virtual UUID loginImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators) const; virtual bool areCredentialsValidImpl(const User & user, const Credentials & credentials, const ExternalAuthenticators & external_authenticators) const; virtual bool isAddressAllowedImpl(const User & user, const Poco::Net::IPAddress & address) const; @@ -174,17 +171,17 @@ protected: static UUID generateRandomID(); Poco::Logger * getLogger() const; - static String outputEntityTypeAndName(EntityType type, const String & name) { return EntityTypeInfo::get(type).outputWithEntityName(name); } + static String formatEntityTypeWithName(AccessEntityType type, const String & name) { return AccessEntityTypeInfo::get(type).formatEntityNameWithType(name); } [[noreturn]] void throwNotFound(const UUID & id) const; - [[noreturn]] void throwNotFound(EntityType type, const String & name) const; - [[noreturn]] static void throwBadCast(const UUID & id, EntityType type, const String & name, EntityType required_type); + [[noreturn]] void throwNotFound(AccessEntityType type, const String & name) const; + [[noreturn]] static void throwBadCast(const UUID & id, AccessEntityType type, const String & name, AccessEntityType required_type); [[noreturn]] void throwIDCollisionCannotInsert( - const UUID & id, EntityType type, const String & name, EntityType existing_type, const String & existing_name) const; - [[noreturn]] void throwNameCollisionCannotInsert(EntityType type, const String & name) const; - [[noreturn]] void throwNameCollisionCannotRename(EntityType type, const String & old_name, const String & new_name) const; - [[noreturn]] void throwReadonlyCannotInsert(EntityType type, const String & name) const; - [[noreturn]] void throwReadonlyCannotUpdate(EntityType type, const String & name) const; - [[noreturn]] void throwReadonlyCannotRemove(EntityType type, const String & name) const; + const UUID & id, AccessEntityType type, const String & name, AccessEntityType existing_type, const String & existing_name) const; + [[noreturn]] void throwNameCollisionCannotInsert(AccessEntityType type, const String & name) const; + [[noreturn]] void throwNameCollisionCannotRename(AccessEntityType type, const String & old_name, const String & new_name) const; + [[noreturn]] void throwReadonlyCannotInsert(AccessEntityType type, const String & name) const; + [[noreturn]] void throwReadonlyCannotUpdate(AccessEntityType type, const String & name) const; + [[noreturn]] void throwReadonlyCannotRemove(AccessEntityType type, const String & name) const; [[noreturn]] static void throwAddressNotAllowed(const Poco::Net::IPAddress & address); [[noreturn]] static void throwInvalidCredentials(); [[noreturn]] static void throwCannotAuthenticate(const String & user_name); diff --git a/src/Access/LDAPAccessStorage.cpp b/src/Access/LDAPAccessStorage.cpp index 92c9b15612b..c89d8c1f953 100644 --- a/src/Access/LDAPAccessStorage.cpp +++ b/src/Access/LDAPAccessStorage.cpp @@ -412,14 +412,14 @@ String LDAPAccessStorage::getStorageParamsJSON() const } -std::optional LDAPAccessStorage::findImpl(EntityType type, const String & name) const +std::optional LDAPAccessStorage::findImpl(AccessEntityType type, const String & name) const { std::scoped_lock lock(mutex); return memory_storage.find(type, name); } -std::vector LDAPAccessStorage::findAllImpl(EntityType type) const +std::vector LDAPAccessStorage::findAllImpl(AccessEntityType type) const { std::scoped_lock lock(mutex); return memory_storage.findAll(type); @@ -482,7 +482,7 @@ scope_guard LDAPAccessStorage::subscribeForChangesImpl(const UUID & id, const On } -scope_guard LDAPAccessStorage::subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const +scope_guard LDAPAccessStorage::subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const { std::scoped_lock lock(mutex); return memory_storage.subscribeForChanges(type, handler); @@ -496,7 +496,7 @@ bool LDAPAccessStorage::hasSubscriptionImpl(const UUID & id) const } -bool LDAPAccessStorage::hasSubscriptionImpl(EntityType type) const +bool LDAPAccessStorage::hasSubscriptionImpl(AccessEntityType type) const { std::scoped_lock lock(mutex); return memory_storage.hasSubscription(type); diff --git a/src/Access/LDAPAccessStorage.h b/src/Access/LDAPAccessStorage.h index c1512117186..feb6ee4d92a 100644 --- a/src/Access/LDAPAccessStorage.h +++ b/src/Access/LDAPAccessStorage.h @@ -42,8 +42,8 @@ public: // IAccessStorage implementations. virtual String getStorageParamsJSON() const override; private: // IAccessStorage implementations. - virtual std::optional findImpl(EntityType type, const String & name) const override; - virtual std::vector findAllImpl(EntityType type) const override; + virtual std::optional findImpl(AccessEntityType type, const String & name) const override; + virtual std::vector findAllImpl(AccessEntityType type) const override; virtual bool existsImpl(const UUID & id) const override; virtual AccessEntityPtr readImpl(const UUID & id) const override; virtual String readNameImpl(const UUID & id) const override; @@ -52,9 +52,9 @@ private: // IAccessStorage implementations. virtual void removeImpl(const UUID & id) override; virtual void updateImpl(const UUID & id, const UpdateFunc & update_func) override; virtual scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override; - virtual scope_guard subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const override; + virtual scope_guard subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const override; virtual bool hasSubscriptionImpl(const UUID & id) const override; - virtual bool hasSubscriptionImpl(EntityType type) const override; + virtual bool hasSubscriptionImpl(AccessEntityType type) const override; virtual UUID loginImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators) const override; virtual UUID getIDOfLoggedUserImpl(const String & user_name) const override; diff --git a/src/Access/MemoryAccessStorage.cpp b/src/Access/MemoryAccessStorage.cpp index 68a24146107..d3c99204bd3 100644 --- a/src/Access/MemoryAccessStorage.cpp +++ b/src/Access/MemoryAccessStorage.cpp @@ -13,7 +13,7 @@ MemoryAccessStorage::MemoryAccessStorage(const String & storage_name_) } -std::optional MemoryAccessStorage::findImpl(EntityType type, const String & name) const +std::optional MemoryAccessStorage::findImpl(AccessEntityType type, const String & name) const { std::lock_guard lock{mutex}; const auto & entries_by_name = entries_by_name_and_type[static_cast(type)]; @@ -26,7 +26,7 @@ std::optional MemoryAccessStorage::findImpl(EntityType type, const String } -std::vector MemoryAccessStorage::findAllImpl(EntityType type) const +std::vector MemoryAccessStorage::findAllImpl(AccessEntityType type) const { std::lock_guard lock{mutex}; std::vector result; @@ -77,7 +77,7 @@ UUID MemoryAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool re void MemoryAccessStorage::insertNoLock(const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists, Notifications & notifications) { const String & name = new_entity->getName(); - EntityType type = new_entity->getType(); + AccessEntityType type = new_entity->getType(); /// Check that we can insert. auto it = entries_by_id.find(id); @@ -125,7 +125,7 @@ void MemoryAccessStorage::removeNoLock(const UUID & id, Notifications & notifica Entry & entry = it->second; const String & name = entry.entity->getName(); - EntityType type = entry.entity->getType(); + AccessEntityType type = entry.entity->getType(); prepareNotifications(entry, true, notifications); @@ -266,7 +266,7 @@ void MemoryAccessStorage::prepareNotifications(const Entry & entry, bool remove, } -scope_guard MemoryAccessStorage::subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const +scope_guard MemoryAccessStorage::subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const { std::lock_guard lock{mutex}; auto & handlers = handlers_by_type[static_cast(type)]; @@ -317,7 +317,7 @@ bool MemoryAccessStorage::hasSubscriptionImpl(const UUID & id) const } -bool MemoryAccessStorage::hasSubscriptionImpl(EntityType type) const +bool MemoryAccessStorage::hasSubscriptionImpl(AccessEntityType type) const { std::lock_guard lock{mutex}; const auto & handlers = handlers_by_type[static_cast(type)]; diff --git a/src/Access/MemoryAccessStorage.h b/src/Access/MemoryAccessStorage.h index 3558fcc6088..ea7b0193471 100644 --- a/src/Access/MemoryAccessStorage.h +++ b/src/Access/MemoryAccessStorage.h @@ -24,8 +24,8 @@ public: void setAll(const std::vector> & all_entities); private: - std::optional findImpl(EntityType type, const String & name) const override; - std::vector findAllImpl(EntityType type) const override; + std::optional findImpl(AccessEntityType type, const String & name) const override; + std::vector findAllImpl(AccessEntityType type) const override; bool existsImpl(const UUID & id) const override; AccessEntityPtr readImpl(const UUID & id) const override; String readNameImpl(const UUID & id) const override; @@ -34,9 +34,9 @@ private: void removeImpl(const UUID & id) override; void updateImpl(const UUID & id, const UpdateFunc & update_func) override; scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override; - scope_guard subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const override; + scope_guard subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const override; bool hasSubscriptionImpl(const UUID & id) const override; - bool hasSubscriptionImpl(EntityType type) const override; + bool hasSubscriptionImpl(AccessEntityType type) const override; struct Entry { @@ -53,7 +53,7 @@ private: mutable std::recursive_mutex mutex; std::unordered_map entries_by_id; /// We want to search entries both by ID and by the pair of name and type. - std::unordered_map entries_by_name_and_type[static_cast(EntityType::MAX)]; - mutable std::list handlers_by_type[static_cast(EntityType::MAX)]; + std::unordered_map entries_by_name_and_type[static_cast(AccessEntityType::MAX)]; + mutable std::list handlers_by_type[static_cast(AccessEntityType::MAX)]; }; } diff --git a/src/Access/MultipleAccessStorage.cpp b/src/Access/MultipleAccessStorage.cpp index bb2ef4a6df4..61bc84e8ab2 100644 --- a/src/Access/MultipleAccessStorage.cpp +++ b/src/Access/MultipleAccessStorage.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -98,7 +99,7 @@ std::shared_ptr MultipleAccessStorage::getStoragesInternal() con } -std::optional MultipleAccessStorage::findImpl(EntityType type, const String & name) const +std::optional MultipleAccessStorage::findImpl(AccessEntityType type, const String & name) const { auto storages = getStoragesInternal(); for (const auto & storage : *storages) @@ -115,7 +116,7 @@ std::optional MultipleAccessStorage::findImpl(EntityType type, const Strin } -std::vector MultipleAccessStorage::findAllImpl(EntityType type) const +std::vector MultipleAccessStorage::findAllImpl(AccessEntityType type) const { std::vector all_ids; auto storages = getStoragesInternal(); @@ -219,7 +220,7 @@ UUID MultipleAccessStorage::insertImpl(const AccessEntityPtr & entity, bool repl } if (!storage_for_insertion) - throw Exception("Not found a storage to insert " + entity->outputTypeAndName(), ErrorCodes::ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND); + throw Exception("Not found a storage to insert " + entity->formatTypeWithName(), ErrorCodes::ACCESS_STORAGE_FOR_INSERTION_NOT_FOUND); auto id = replace_if_exists ? storage_for_insertion->insertOrReplace(entity) : storage_for_insertion->insert(entity); std::lock_guard lock{mutex}; @@ -253,8 +254,8 @@ void MultipleAccessStorage::updateImpl(const UUID & id, const UpdateFunc & updat if (storage->find(new_entity->getType(), new_entity->getName())) { throw Exception( - old_entity->outputTypeAndName() + ": cannot rename to " + backQuote(new_entity->getName()) + " because " - + new_entity->outputTypeAndName() + " already exists in " + storage->getStorageName(), + old_entity->formatTypeWithName() + ": cannot rename to " + backQuote(new_entity->getName()) + " because " + + new_entity->formatTypeWithName() + " already exists in " + storage->getStorageName(), ErrorCodes::ACCESS_ENTITY_ALREADY_EXISTS); } } @@ -286,7 +287,7 @@ bool MultipleAccessStorage::hasSubscriptionImpl(const UUID & id) const } -scope_guard MultipleAccessStorage::subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const +scope_guard MultipleAccessStorage::subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const { std::unique_lock lock{mutex}; auto & handlers = handlers_by_type[static_cast(type)]; @@ -306,7 +307,7 @@ scope_guard MultipleAccessStorage::subscribeForChangesImpl(EntityType type, cons } -bool MultipleAccessStorage::hasSubscriptionImpl(EntityType type) const +bool MultipleAccessStorage::hasSubscriptionImpl(AccessEntityType type) const { std::lock_guard lock{mutex}; const auto & handlers = handlers_by_type[static_cast(type)]; @@ -321,10 +322,10 @@ void MultipleAccessStorage::updateSubscriptionsToNestedStorages(std::unique_lock { /// lock is already locked. - std::vector> added_subscriptions[static_cast(EntityType::MAX)]; + std::vector> added_subscriptions[static_cast(AccessEntityType::MAX)]; std::vector removed_subscriptions; - for (auto type : collections::range(EntityType::MAX)) + for (auto type : collections::range(AccessEntityType::MAX)) { auto & handlers = handlers_by_type[static_cast(type)]; auto & subscriptions = subscriptions_to_nested_storages[static_cast(type)]; @@ -364,7 +365,7 @@ void MultipleAccessStorage::updateSubscriptionsToNestedStorages(std::unique_lock lock.unlock(); removed_subscriptions.clear(); - for (auto type : collections::range(EntityType::MAX)) + for (auto type : collections::range(AccessEntityType::MAX)) { if (!added_subscriptions[static_cast(type)].empty()) { @@ -384,7 +385,7 @@ void MultipleAccessStorage::updateSubscriptionsToNestedStorages(std::unique_lock /// Lock the mutex again to store added subscriptions to the nested storages. lock.lock(); - for (auto type : collections::range(EntityType::MAX)) + for (auto type : collections::range(AccessEntityType::MAX)) { if (!added_subscriptions[static_cast(type)].empty()) { @@ -418,7 +419,7 @@ UUID MultipleAccessStorage::loginImpl(const Credentials & credentials, const Poc } catch (...) { - if (!storage->find(EntityType::USER, credentials.getUserName())) + if (!storage->find(AccessEntityType::USER, credentials.getUserName())) { /// The authentication failed because there no users with such name in the `storage` /// thus we can try to search in other nested storages. @@ -427,7 +428,7 @@ UUID MultipleAccessStorage::loginImpl(const Credentials & credentials, const Poc throw; } } - throwNotFound(EntityType::USER, credentials.getUserName()); + throwNotFound(AccessEntityType::USER, credentials.getUserName()); } @@ -445,7 +446,7 @@ UUID MultipleAccessStorage::getIDOfLoggedUserImpl(const String & user_name) cons } catch (...) { - if (!storage->find(EntityType::USER, user_name)) + if (!storage->find(AccessEntityType::USER, user_name)) { /// The authentication failed because there no users with such name in the `storage` /// thus we can try to search in other nested storages. @@ -454,7 +455,7 @@ UUID MultipleAccessStorage::getIDOfLoggedUserImpl(const String & user_name) cons throw; } } - throwNotFound(EntityType::USER, user_name); + throwNotFound(AccessEntityType::USER, user_name); } } diff --git a/src/Access/MultipleAccessStorage.h b/src/Access/MultipleAccessStorage.h index d1c8ec97b36..462f97d6fa9 100644 --- a/src/Access/MultipleAccessStorage.h +++ b/src/Access/MultipleAccessStorage.h @@ -35,8 +35,8 @@ public: StoragePtr getStorage(const UUID & id); protected: - std::optional findImpl(EntityType type, const String & name) const override; - std::vector findAllImpl(EntityType type) const override; + std::optional findImpl(AccessEntityType type, const String & name) const override; + std::vector findAllImpl(AccessEntityType type) const override; bool existsImpl(const UUID & id) const override; AccessEntityPtr readImpl(const UUID & id) const override; String readNameImpl(const UUID &id) const override; @@ -45,9 +45,9 @@ protected: void removeImpl(const UUID & id) override; void updateImpl(const UUID & id, const UpdateFunc & update_func) override; scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override; - scope_guard subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const override; + scope_guard subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const override; bool hasSubscriptionImpl(const UUID & id) const override; - bool hasSubscriptionImpl(EntityType type) const override; + bool hasSubscriptionImpl(AccessEntityType type) const override; UUID loginImpl(const Credentials & credentials, const Poco::Net::IPAddress & address, const ExternalAuthenticators & external_authenticators) const override; UUID getIDOfLoggedUserImpl(const String & user_name) const override; @@ -58,8 +58,8 @@ private: std::shared_ptr nested_storages; mutable LRUCache ids_cache; - mutable std::list handlers_by_type[static_cast(EntityType::MAX)]; - mutable std::unordered_map subscriptions_to_nested_storages[static_cast(EntityType::MAX)]; + mutable std::list handlers_by_type[static_cast(AccessEntityType::MAX)]; + mutable std::unordered_map subscriptions_to_nested_storages[static_cast(AccessEntityType::MAX)]; mutable std::mutex mutex; }; diff --git a/src/Access/Quota.h b/src/Access/Quota.h index f87f9181b1e..487af49d684 100644 --- a/src/Access/Quota.h +++ b/src/Access/Quota.h @@ -43,8 +43,8 @@ struct Quota : public IAccessEntity bool equal(const IAccessEntity & other) const override; std::shared_ptr clone() const override { return cloneImpl(); } - static constexpr const Type TYPE = Type::QUOTA; - Type getType() const override { return TYPE; } + static constexpr const auto TYPE = AccessEntityType::QUOTA; + AccessEntityType getType() const override { return TYPE; } }; using QuotaPtr = std::shared_ptr; diff --git a/src/Access/ReplicatedAccessStorage.cpp b/src/Access/ReplicatedAccessStorage.cpp index 7b29aab3a89..93b8a5c992a 100644 --- a/src/Access/ReplicatedAccessStorage.cpp +++ b/src/Access/ReplicatedAccessStorage.cpp @@ -94,7 +94,7 @@ static void retryOnZooKeeperUserError(size_t attempts, Func && function) UUID ReplicatedAccessStorage::insertImpl(const AccessEntityPtr & new_entity, bool replace_if_exists) { const UUID id = generateRandomID(); - const EntityTypeInfo type_info = EntityTypeInfo::get(new_entity->getType()); + const AccessEntityTypeInfo type_info = AccessEntityTypeInfo::get(new_entity->getType()); const String & name = new_entity->getName(); LOG_DEBUG(getLogger(), "Inserting entity of type {} named {} with id {}", type_info.name, name, toString(id)); @@ -113,8 +113,8 @@ void ReplicatedAccessStorage::insertZooKeeper( const zkutil::ZooKeeperPtr & zookeeper, const UUID & id, const AccessEntityPtr & new_entity, bool replace_if_exists) { const String & name = new_entity->getName(); - const EntityType type = new_entity->getType(); - const EntityTypeInfo type_info = EntityTypeInfo::get(type); + const AccessEntityType type = new_entity->getType(); + const AccessEntityTypeInfo type_info = AccessEntityTypeInfo::get(type); const String entity_uuid = toString(id); /// The entity data will be stored here, this ensures all entities have unique ids @@ -143,7 +143,7 @@ void ReplicatedAccessStorage::insertZooKeeper( String existing_entity_definition = zookeeper->get(entity_path); AccessEntityPtr existing_entity = deserializeAccessEntity(existing_entity_definition, entity_path); - EntityType existing_type = existing_entity->getType(); + AccessEntityType existing_type = existing_entity->getType(); String existing_name = existing_entity->getName(); throwIDCollisionCannotInsert(id, type, name, existing_type, existing_name); } @@ -204,7 +204,7 @@ void ReplicatedAccessStorage::removeZooKeeper(const zkutil::ZooKeeperPtr & zooke throwNotFound(id); const AccessEntityPtr entity = deserializeAccessEntity(entity_definition, entity_path); - const EntityTypeInfo type_info = EntityTypeInfo::get(entity->getType()); + const AccessEntityTypeInfo type_info = AccessEntityTypeInfo::get(entity->getType()); const String & name = entity->getName(); const String entity_name_path = zookeeper_path + "/" + type_info.unique_char + "/" + escapeForFileName(name); @@ -248,7 +248,7 @@ void ReplicatedAccessStorage::updateZooKeeper(const zkutil::ZooKeeperPtr & zooke if (!new_entity->isTypeOf(old_entity->getType())) throwBadCast(id, new_entity->getType(), new_entity->getName(), old_entity->getType()); - const EntityTypeInfo type_info = EntityTypeInfo::get(new_entity->getType()); + const AccessEntityTypeInfo type_info = AccessEntityTypeInfo::get(new_entity->getType()); Coordination::Requests ops; const String new_entity_definition = serializeAccessEntity(*new_entity); @@ -309,7 +309,7 @@ void ReplicatedAccessStorage::resetAfterError() while (refresh_queue.tryPop(id)) {} std::lock_guard lock{mutex}; - for (const auto type : collections::range(EntityType::MAX)) + for (const auto type : collections::range(AccessEntityType::MAX)) entries_by_name_and_type[static_cast(type)].clear(); entries_by_id.clear(); } @@ -334,10 +334,10 @@ void ReplicatedAccessStorage::createRootNodes(const zkutil::ZooKeeperPtr & zooke zookeeper->createAncestors(zookeeper_path); zookeeper->createIfNotExists(zookeeper_path, ""); zookeeper->createIfNotExists(zookeeper_path + "/uuid", ""); - for (const auto type : collections::range(EntityType::MAX)) + for (const auto type : collections::range(AccessEntityType::MAX)) { /// Create a znode for each type of AccessEntity - const auto type_info = EntityTypeInfo::get(type); + const auto type_info = AccessEntityTypeInfo::get(type); zookeeper->createIfNotExists(zookeeper_path + "/" + type_info.unique_char, ""); } } @@ -440,7 +440,7 @@ void ReplicatedAccessStorage::refreshEntityNoLock(const zkutil::ZooKeeperPtr & z void ReplicatedAccessStorage::setEntityNoLock(const UUID & id, const AccessEntityPtr & entity, Notifications & notifications) { LOG_DEBUG(getLogger(), "Setting id {} to entity named {}", toString(id), entity->getName()); - const EntityType type = entity->getType(); + const AccessEntityType type = entity->getType(); const String & name = entity->getName(); /// If the type+name already exists and is a different entity, remove old entity @@ -454,7 +454,7 @@ void ReplicatedAccessStorage::setEntityNoLock(const UUID & id, const AccessEntit if (auto it = entries_by_id.find(id); it != entries_by_id.end()) { const AccessEntityPtr & existing_entity = it->second.entity; - const EntityType existing_type = existing_entity->getType(); + const AccessEntityType existing_type = existing_entity->getType(); const String & existing_name = existing_entity->getName(); if (existing_type != type || existing_name != name) { @@ -482,7 +482,7 @@ void ReplicatedAccessStorage::removeEntityNoLock(const UUID & id, Notifications } const Entry & entry = it->second; - const EntityType type = entry.entity->getType(); + const AccessEntityType type = entry.entity->getType(); const String & name = entry.entity->getName(); prepareNotifications(entry, true, notifications); @@ -500,7 +500,7 @@ void ReplicatedAccessStorage::removeEntityNoLock(const UUID & id, Notifications } -std::optional ReplicatedAccessStorage::findImpl(EntityType type, const String & name) const +std::optional ReplicatedAccessStorage::findImpl(AccessEntityType type, const String & name) const { std::lock_guard lock{mutex}; const auto & entries_by_name = entries_by_name_and_type[static_cast(type)]; @@ -513,7 +513,7 @@ std::optional ReplicatedAccessStorage::findImpl(EntityType type, const Str } -std::vector ReplicatedAccessStorage::findAllImpl(EntityType type) const +std::vector ReplicatedAccessStorage::findAllImpl(AccessEntityType type) const { std::lock_guard lock{mutex}; std::vector result; @@ -560,7 +560,7 @@ void ReplicatedAccessStorage::prepareNotifications(const Entry & entry, bool rem } -scope_guard ReplicatedAccessStorage::subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const +scope_guard ReplicatedAccessStorage::subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const { std::lock_guard lock{mutex}; auto & handlers = handlers_by_type[static_cast(type)]; @@ -611,7 +611,7 @@ bool ReplicatedAccessStorage::hasSubscriptionImpl(const UUID & id) const } -bool ReplicatedAccessStorage::hasSubscriptionImpl(EntityType type) const +bool ReplicatedAccessStorage::hasSubscriptionImpl(AccessEntityType type) const { std::lock_guard lock{mutex}; const auto & handlers = handlers_by_type[static_cast(type)]; diff --git a/src/Access/ReplicatedAccessStorage.h b/src/Access/ReplicatedAccessStorage.h index 458bc0d614b..54dbfbf5b7d 100644 --- a/src/Access/ReplicatedAccessStorage.h +++ b/src/Access/ReplicatedAccessStorage.h @@ -69,8 +69,8 @@ private: mutable std::list handlers_by_id; }; - std::optional findImpl(EntityType type, const String & name) const override; - std::vector findAllImpl(EntityType type) const override; + std::optional findImpl(AccessEntityType type, const String & name) const override; + std::vector findAllImpl(AccessEntityType type) const override; bool existsImpl(const UUID & id) const override; AccessEntityPtr readImpl(const UUID & id) const override; String readNameImpl(const UUID & id) const override; @@ -78,13 +78,13 @@ private: void prepareNotifications(const Entry & entry, bool remove, Notifications & notifications) const; scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override; - scope_guard subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const override; + scope_guard subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const override; bool hasSubscriptionImpl(const UUID & id) const override; - bool hasSubscriptionImpl(EntityType type) const override; + bool hasSubscriptionImpl(AccessEntityType type) const override; mutable std::mutex mutex; std::unordered_map entries_by_id; - std::unordered_map entries_by_name_and_type[static_cast(EntityType::MAX)]; - mutable std::list handlers_by_type[static_cast(EntityType::MAX)]; + std::unordered_map entries_by_name_and_type[static_cast(AccessEntityType::MAX)]; + mutable std::list handlers_by_type[static_cast(AccessEntityType::MAX)]; }; } diff --git a/src/Access/Role.h b/src/Access/Role.h index 131bbd69195..c7f1e107d24 100644 --- a/src/Access/Role.h +++ b/src/Access/Role.h @@ -17,8 +17,8 @@ struct Role : public IAccessEntity bool equal(const IAccessEntity & other) const override; std::shared_ptr clone() const override { return cloneImpl(); } - static constexpr const Type TYPE = Type::ROLE; - Type getType() const override { return TYPE; } + static constexpr const auto TYPE = AccessEntityType::ROLE; + AccessEntityType getType() const override { return TYPE; } }; using RolePtr = std::shared_ptr; diff --git a/src/Access/RowPolicy.h b/src/Access/RowPolicy.h index 6cbf4196352..9c143aff725 100644 --- a/src/Access/RowPolicy.h +++ b/src/Access/RowPolicy.h @@ -43,8 +43,8 @@ struct RowPolicy : public IAccessEntity bool equal(const IAccessEntity & other) const override; std::shared_ptr clone() const override { return cloneImpl(); } - static constexpr const Type TYPE = Type::ROW_POLICY; - Type getType() const override { return TYPE; } + static constexpr const auto TYPE = AccessEntityType::ROW_POLICY; + AccessEntityType getType() const override { return TYPE; } /// Which roles or users should use this row policy. RolesOrUsersSet to_roles; diff --git a/src/Access/SettingsProfile.h b/src/Access/SettingsProfile.h index 210aa47c358..e554924b45e 100644 --- a/src/Access/SettingsProfile.h +++ b/src/Access/SettingsProfile.h @@ -18,8 +18,8 @@ struct SettingsProfile : public IAccessEntity bool equal(const IAccessEntity & other) const override; std::shared_ptr clone() const override { return cloneImpl(); } - static constexpr const Type TYPE = Type::SETTINGS_PROFILE; - Type getType() const override { return TYPE; } + static constexpr const auto TYPE = AccessEntityType::SETTINGS_PROFILE; + AccessEntityType getType() const override { return TYPE; } }; using SettingsProfilePtr = std::shared_ptr; diff --git a/src/Access/User.h b/src/Access/User.h index 34badd5f847..b9167d68f15 100644 --- a/src/Access/User.h +++ b/src/Access/User.h @@ -26,8 +26,8 @@ struct User : public IAccessEntity bool equal(const IAccessEntity & other) const override; std::shared_ptr clone() const override { return cloneImpl(); } - static constexpr const Type TYPE = Type::USER; - Type getType() const override { return TYPE; } + static constexpr const auto TYPE = AccessEntityType::USER; + AccessEntityType getType() const override { return TYPE; } }; using UserPtr = std::shared_ptr; diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index 002517b32a0..014eaf0179d 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -33,15 +33,12 @@ namespace ErrorCodes namespace { - using EntityType = IAccessStorage::EntityType; - using EntityTypeInfo = IAccessStorage::EntityTypeInfo; - - UUID generateID(EntityType type, const String & name) + UUID generateID(AccessEntityType type, const String & name) { Poco::MD5Engine md5; md5.update(name); char type_storage_chars[] = " USRSXML"; - type_storage_chars[0] = EntityTypeInfo::get(type).unique_char; + type_storage_chars[0] = AccessEntityTypeInfo::get(type).unique_char; md5.update(type_storage_chars, strlen(type_storage_chars)); UUID result; memcpy(&result, md5.digest().data(), md5.digestLength()); @@ -114,7 +111,7 @@ namespace { auto profile_name = config.getString(profile_name_config); SettingsProfileElement profile_element; - profile_element.parent_profile = generateID(EntityType::SETTINGS_PROFILE, profile_name); + profile_element.parent_profile = generateID(AccessEntityType::SETTINGS_PROFILE, profile_name); user->settings.push_back(std::move(profile_element)); } @@ -273,7 +270,7 @@ namespace for (const auto & user_name : user_names) { if (config.has("users." + user_name + ".quota")) - quota_to_user_ids[config.getString("users." + user_name + ".quota")].push_back(generateID(EntityType::USER, user_name)); + quota_to_user_ids[config.getString("users." + user_name + ".quota")].push_back(generateID(AccessEntityType::USER, user_name)); } Poco::Util::AbstractConfiguration::Keys quota_names; @@ -352,7 +349,7 @@ namespace auto policy = std::make_shared(); policy->setFullName(user_name, database, table_name); policy->filters[static_cast(RowPolicyFilterType::SELECT_FILTER)] = filter; - policy->to_roles.add(generateID(EntityType::USER, user_name)); + policy->to_roles.add(generateID(AccessEntityType::USER, user_name)); policies.push_back(policy); } } @@ -414,7 +411,7 @@ namespace { String parent_profile_name = config.getString(profile_config + "." + key); SettingsProfileElement profile_element; - profile_element.parent_profile = generateID(EntityType::SETTINGS_PROFILE, parent_profile_name); + profile_element.parent_profile = generateID(AccessEntityType::SETTINGS_PROFILE, parent_profile_name); profile->elements.emplace_back(std::move(profile_element)); continue; } @@ -551,13 +548,13 @@ void UsersConfigAccessStorage::startPeriodicReloading() config_reloader->start(); } -std::optional UsersConfigAccessStorage::findImpl(EntityType type, const String & name) const +std::optional UsersConfigAccessStorage::findImpl(AccessEntityType type, const String & name) const { return memory_storage.find(type, name); } -std::vector UsersConfigAccessStorage::findAllImpl(EntityType type) const +std::vector UsersConfigAccessStorage::findAllImpl(AccessEntityType type) const { return memory_storage.findAll(type); } @@ -607,7 +604,7 @@ scope_guard UsersConfigAccessStorage::subscribeForChangesImpl(const UUID & id, c } -scope_guard UsersConfigAccessStorage::subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const +scope_guard UsersConfigAccessStorage::subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const { return memory_storage.subscribeForChanges(type, handler); } @@ -619,7 +616,7 @@ bool UsersConfigAccessStorage::hasSubscriptionImpl(const UUID & id) const } -bool UsersConfigAccessStorage::hasSubscriptionImpl(EntityType type) const +bool UsersConfigAccessStorage::hasSubscriptionImpl(AccessEntityType type) const { return memory_storage.hasSubscription(type); } diff --git a/src/Access/UsersConfigAccessStorage.h b/src/Access/UsersConfigAccessStorage.h index 97164fdb233..7fb08790f77 100644 --- a/src/Access/UsersConfigAccessStorage.h +++ b/src/Access/UsersConfigAccessStorage.h @@ -43,8 +43,8 @@ public: private: void parseFromConfig(const Poco::Util::AbstractConfiguration & config); - std::optional findImpl(EntityType type, const String & name) const override; - std::vector findAllImpl(EntityType type) const override; + std::optional findImpl(AccessEntityType type, const String & name) const override; + std::vector findAllImpl(AccessEntityType type) const override; bool existsImpl(const UUID & id) const override; AccessEntityPtr readImpl(const UUID & id) const override; String readNameImpl(const UUID & id) const override; @@ -53,9 +53,9 @@ private: void removeImpl(const UUID & id) override; void updateImpl(const UUID & id, const UpdateFunc & update_func) override; scope_guard subscribeForChangesImpl(const UUID & id, const OnChangedHandler & handler) const override; - scope_guard subscribeForChangesImpl(EntityType type, const OnChangedHandler & handler) const override; + scope_guard subscribeForChangesImpl(AccessEntityType type, const OnChangedHandler & handler) const override; bool hasSubscriptionImpl(const UUID & id) const override; - bool hasSubscriptionImpl(EntityType type) const override; + bool hasSubscriptionImpl(AccessEntityType type) const override; MemoryAccessStorage memory_storage; CheckSettingNameFunction check_setting_name_function; diff --git a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp index 4cf8216931d..4d2e880561e 100644 --- a/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterDropAccessEntityQuery.cpp @@ -14,8 +14,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -using EntityType = IAccessEntity::Type; - BlockIO InterpreterDropAccessEntityQuery::execute() { @@ -36,7 +34,7 @@ BlockIO InterpreterDropAccessEntityQuery::execute() access_control.remove(access_control.getIDs(query.type, names)); }; - if (query.type == EntityType::ROW_POLICY) + if (query.type == AccessEntityType::ROW_POLICY) do_drop(query.row_policy_names->toStrings()); else do_drop(query.names); @@ -51,12 +49,12 @@ AccessRightsElements InterpreterDropAccessEntityQuery::getRequiredAccess() const AccessRightsElements res; switch (query.type) { - case EntityType::USER: res.emplace_back(AccessType::DROP_USER); return res; - case EntityType::ROLE: res.emplace_back(AccessType::DROP_ROLE); return res; - case EntityType::SETTINGS_PROFILE: res.emplace_back(AccessType::DROP_SETTINGS_PROFILE); return res; - case EntityType::ROW_POLICY: res.emplace_back(AccessType::DROP_ROW_POLICY); return res; - case EntityType::QUOTA: res.emplace_back(AccessType::DROP_QUOTA); return res; - case EntityType::MAX: break; + case AccessEntityType::USER: res.emplace_back(AccessType::DROP_USER); return res; + case AccessEntityType::ROLE: res.emplace_back(AccessType::DROP_ROLE); return res; + case AccessEntityType::SETTINGS_PROFILE: res.emplace_back(AccessType::DROP_SETTINGS_PROFILE); return res; + case AccessEntityType::ROW_POLICY: res.emplace_back(AccessType::DROP_ROW_POLICY); return res; + case AccessEntityType::QUOTA: res.emplace_back(AccessType::DROP_QUOTA); return res; + case AccessEntityType::MAX: break; } throw Exception( toString(query.type) + ": type is not supported by DROP query", ErrorCodes::NOT_IMPLEMENTED); diff --git a/src/Interpreters/Access/InterpreterGrantQuery.cpp b/src/Interpreters/Access/InterpreterGrantQuery.cpp index e17af6877be..f2b9cd58991 100644 --- a/src/Interpreters/Access/InterpreterGrantQuery.cpp +++ b/src/Interpreters/Access/InterpreterGrantQuery.cpp @@ -118,7 +118,7 @@ namespace { auto current_user = current_user_access.getUser(); if (current_user && !current_user->grantees.match(grantee_id)) - throw Exception(grantee.outputTypeAndName() + " is not allowed as grantee", ErrorCodes::ACCESS_DENIED); + throw Exception(grantee.formatTypeWithName() + " is not allowed as grantee", ErrorCodes::ACCESS_DENIED); } /// Checks if grantees are allowed for the current user, throws an exception if not. diff --git a/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp b/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp index b0fe28e1abd..acb7f521493 100644 --- a/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowAccessEntitiesQuery.cpp @@ -14,8 +14,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; } -using EntityType = IAccessEntity::Type; - InterpreterShowAccessEntitiesQuery::InterpreterShowAccessEntitiesQuery(const ASTPtr & query_ptr_, ContextMutablePtr context_) : WithMutableContext(context_), query_ptr(query_ptr_) @@ -41,7 +39,7 @@ String InterpreterShowAccessEntitiesQuery::getRewrittenQuery() const switch (query.type) { - case EntityType::ROW_POLICY: + case AccessEntityType::ROW_POLICY: { origin = "row_policies"; expr = "name"; @@ -63,7 +61,7 @@ String InterpreterShowAccessEntitiesQuery::getRewrittenQuery() const break; } - case EntityType::QUOTA: + case AccessEntityType::QUOTA: { if (query.current_quota) { @@ -78,21 +76,21 @@ String InterpreterShowAccessEntitiesQuery::getRewrittenQuery() const break; } - case EntityType::SETTINGS_PROFILE: + case AccessEntityType::SETTINGS_PROFILE: { origin = "settings_profiles"; expr = "name"; break; } - case EntityType::USER: + case AccessEntityType::USER: { origin = "users"; expr = "name"; break; } - case EntityType::ROLE: + case AccessEntityType::ROLE: { if (query.current_roles) { @@ -112,7 +110,7 @@ String InterpreterShowAccessEntitiesQuery::getRewrittenQuery() const break; } - case EntityType::MAX: + case AccessEntityType::MAX: break; } diff --git a/src/Interpreters/Access/InterpreterShowAccessQuery.cpp b/src/Interpreters/Access/InterpreterShowAccessQuery.cpp index a385f6c8d7a..26c47507ce2 100644 --- a/src/Interpreters/Access/InterpreterShowAccessQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowAccessQuery.cpp @@ -16,8 +16,6 @@ namespace DB { -using EntityType = IAccessEntity::Type; - BlockIO InterpreterShowAccessQuery::execute() { @@ -53,7 +51,7 @@ std::vector InterpreterShowAccessQuery::getEntities() const getContext()->checkAccess(AccessType::SHOW_ACCESS); std::vector entities; - for (auto type : collections::range(EntityType::MAX)) + for (auto type : collections::range(AccessEntityType::MAX)) { auto ids = access_control.findAll(type); for (const auto & id : ids) @@ -77,7 +75,7 @@ ASTs InterpreterShowAccessQuery::getCreateAndGrantQueries() const for (const auto & entity : entities) { create_queries.push_back(InterpreterShowCreateAccessEntityQuery::getCreateQuery(*entity, access_control)); - if (entity->isTypeOf(EntityType::USER) || entity->isTypeOf(EntityType::ROLE)) + if (entity->isTypeOf(AccessEntityType::USER) || entity->isTypeOf(AccessEntityType::ROLE)) boost::range::push_back(grant_queries, InterpreterShowGrantsQuery::getGrantQueries(*entity, access_control)); } diff --git a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp index dcfdc897c62..284b3cd1b48 100644 --- a/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowCreateAccessEntityQuery.cpp @@ -230,10 +230,8 @@ namespace return getCreateQueryImpl(*quota, access_control, attach_mode); if (const SettingsProfile * profile = typeid_cast(&entity)) return getCreateQueryImpl(*profile, access_control, attach_mode); - throw Exception(entity.outputTypeAndName() + ": type is not supported by SHOW CREATE query", ErrorCodes::NOT_IMPLEMENTED); + throw Exception(entity.formatTypeWithName() + ": type is not supported by SHOW CREATE query", ErrorCodes::NOT_IMPLEMENTED); } - - using EntityType = IAccessEntity::Type; } @@ -307,7 +305,7 @@ std::vector InterpreterShowCreateAccessEntityQuery::getEntities if (usage) entities.push_back(access_control.read(usage->quota_id)); } - else if (show_query.type == EntityType::ROW_POLICY) + else if (show_query.type == AccessEntityType::ROW_POLICY) { auto ids = access_control.findAll(); if (show_query.row_policy_names) @@ -379,12 +377,12 @@ AccessRightsElements InterpreterShowCreateAccessEntityQuery::getRequiredAccess() AccessRightsElements res; switch (show_query.type) { - case EntityType::USER: res.emplace_back(AccessType::SHOW_USERS); return res; - case EntityType::ROLE: res.emplace_back(AccessType::SHOW_ROLES); return res; - case EntityType::SETTINGS_PROFILE: res.emplace_back(AccessType::SHOW_SETTINGS_PROFILES); return res; - case EntityType::ROW_POLICY: res.emplace_back(AccessType::SHOW_ROW_POLICIES); return res; - case EntityType::QUOTA: res.emplace_back(AccessType::SHOW_QUOTAS); return res; - case EntityType::MAX: break; + case AccessEntityType::USER: res.emplace_back(AccessType::SHOW_USERS); return res; + case AccessEntityType::ROLE: res.emplace_back(AccessType::SHOW_ROLES); return res; + case AccessEntityType::SETTINGS_PROFILE: res.emplace_back(AccessType::SHOW_SETTINGS_PROFILES); return res; + case AccessEntityType::ROW_POLICY: res.emplace_back(AccessType::SHOW_ROW_POLICIES); return res; + case AccessEntityType::QUOTA: res.emplace_back(AccessType::SHOW_QUOTAS); return res; + case AccessEntityType::MAX: break; } throw Exception(toString(show_query.type) + ": type is not supported by SHOW CREATE query", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp b/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp index 788856dbfe0..cd98d8d4575 100644 --- a/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp +++ b/src/Interpreters/Access/InterpreterShowGrantsQuery.cpp @@ -91,7 +91,7 @@ namespace return getGrantQueriesImpl(*user, access_control, attach_mode); if (const Role * role = typeid_cast(&entity)) return getGrantQueriesImpl(*role, access_control, attach_mode); - throw Exception(entity.outputTypeAndName() + " is expected to be user or role", ErrorCodes::LOGICAL_ERROR); + throw Exception(entity.formatTypeWithName() + " is expected to be user or role", ErrorCodes::LOGICAL_ERROR); } } diff --git a/src/Parsers/Access/ASTDropAccessEntityQuery.cpp b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp index 19064ad9109..22b30d47ffa 100644 --- a/src/Parsers/Access/ASTDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ASTDropAccessEntityQuery.cpp @@ -8,8 +8,6 @@ namespace DB { namespace { - using EntityTypeInfo = IAccessEntity::TypeInfo; - void formatNames(const Strings & names, const IAST::FormatSettings & settings) { bool need_comma = false; @@ -38,11 +36,11 @@ ASTPtr ASTDropAccessEntityQuery::clone() const void ASTDropAccessEntityQuery::formatImpl(const FormatSettings & settings, FormatState &, FormatStateStacked) const { settings.ostr << (settings.hilite ? hilite_keyword : "") - << "DROP " << EntityTypeInfo::get(type).name + << "DROP " << AccessEntityTypeInfo::get(type).name << (if_exists ? " IF EXISTS" : "") << (settings.hilite ? hilite_none : ""); - if (type == EntityType::ROW_POLICY) + if (type == AccessEntityType::ROW_POLICY) { settings.ostr << " "; row_policy_names->format(settings); diff --git a/src/Parsers/Access/ASTDropAccessEntityQuery.h b/src/Parsers/Access/ASTDropAccessEntityQuery.h index 401c3d3f5f0..b1a6ca58a18 100644 --- a/src/Parsers/Access/ASTDropAccessEntityQuery.h +++ b/src/Parsers/Access/ASTDropAccessEntityQuery.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB @@ -18,9 +18,7 @@ class ASTRowPolicyNames; class ASTDropAccessEntityQuery : public IAST, public ASTQueryWithOnCluster { public: - using EntityType = IAccessEntity::Type; - - EntityType type; + AccessEntityType type; bool if_exists = false; Strings names; std::shared_ptr row_policy_names; diff --git a/src/Parsers/Access/ASTShowAccessEntitiesQuery.cpp b/src/Parsers/Access/ASTShowAccessEntitiesQuery.cpp index e2dfe031f53..f6c42171d5b 100644 --- a/src/Parsers/Access/ASTShowAccessEntitiesQuery.cpp +++ b/src/Parsers/Access/ASTShowAccessEntitiesQuery.cpp @@ -5,8 +5,6 @@ namespace DB { -using EntityTypeInfo = IAccessEntity::TypeInfo; - String ASTShowAccessEntitiesQuery::getKeyword() const { @@ -16,7 +14,7 @@ String ASTShowAccessEntitiesQuery::getKeyword() const return "CURRENT ROLES"; if (enabled_roles) return "ENABLED ROLES"; - return EntityTypeInfo::get(type).plural_name; + return AccessEntityTypeInfo::get(type).plural_name; } diff --git a/src/Parsers/Access/ASTShowAccessEntitiesQuery.h b/src/Parsers/Access/ASTShowAccessEntitiesQuery.h index 2be1e0b92f0..e633a4b506a 100644 --- a/src/Parsers/Access/ASTShowAccessEntitiesQuery.h +++ b/src/Parsers/Access/ASTShowAccessEntitiesQuery.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -16,9 +16,7 @@ namespace DB class ASTShowAccessEntitiesQuery : public ASTQueryWithOutput { public: - using EntityType = IAccessEntity::Type; - - EntityType type; + AccessEntityType type; bool all = false; bool current_quota = false; diff --git a/src/Parsers/Access/ASTShowCreateAccessEntityQuery.cpp b/src/Parsers/Access/ASTShowCreateAccessEntityQuery.cpp index db252db968d..e92af22f14f 100644 --- a/src/Parsers/Access/ASTShowCreateAccessEntityQuery.cpp +++ b/src/Parsers/Access/ASTShowCreateAccessEntityQuery.cpp @@ -8,9 +8,6 @@ namespace DB { namespace { - using EntityType = IAccessEntity::Type; - using EntityTypeInfo = IAccessEntity::TypeInfo; - void formatNames(const Strings & names, const IAST::FormatSettings & settings) { bool need_comma = false; @@ -28,7 +25,7 @@ String ASTShowCreateAccessEntityQuery::getKeyword() const { size_t total_count = (names.size()) + (row_policy_names ? row_policy_names->size() : 0) + current_user + current_quota; bool multiple = (total_count != 1) || all || !short_name.empty() || database_and_table_name; - const auto & type_info = EntityTypeInfo::get(type); + const auto & type_info = AccessEntityTypeInfo::get(type); return multiple ? type_info.plural_name : type_info.name; } diff --git a/src/Parsers/Access/ASTShowCreateAccessEntityQuery.h b/src/Parsers/Access/ASTShowCreateAccessEntityQuery.h index e20bb4f022e..27f13587033 100644 --- a/src/Parsers/Access/ASTShowCreateAccessEntityQuery.h +++ b/src/Parsers/Access/ASTShowCreateAccessEntityQuery.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB @@ -24,9 +24,7 @@ using Strings = std::vector; class ASTShowCreateAccessEntityQuery : public ASTQueryWithOutput { public: - using EntityType = IAccessEntity::Type; - - EntityType type; + AccessEntityType type; Strings names; std::shared_ptr row_policy_names; diff --git a/src/Parsers/Access/ParserDropAccessEntityQuery.cpp b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp index d91cd8280a7..1a6784d2d3c 100644 --- a/src/Parsers/Access/ParserDropAccessEntityQuery.cpp +++ b/src/Parsers/Access/ParserDropAccessEntityQuery.cpp @@ -12,15 +12,11 @@ namespace DB { namespace { - using EntityType = IAccessEntity::Type; - using EntityTypeInfo = IAccessEntity::TypeInfo; - - - bool parseEntityType(IParserBase::Pos & pos, Expected & expected, EntityType & type) + bool parseEntityType(IParserBase::Pos & pos, Expected & expected, AccessEntityType & type) { - for (auto i : collections::range(EntityType::MAX)) + for (auto i : collections::range(AccessEntityType::MAX)) { - const auto & type_info = EntityTypeInfo::get(i); + const auto & type_info = AccessEntityTypeInfo::get(i); if (ParserKeyword{type_info.name.c_str()}.ignore(pos, expected) || (!type_info.alias.empty() && ParserKeyword{type_info.alias.c_str()}.ignore(pos, expected))) { @@ -47,7 +43,7 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & if (!ParserKeyword{"DROP"}.ignore(pos, expected)) return false; - EntityType type; + AccessEntityType type; if (!parseEntityType(pos, expected, type)) return false; @@ -59,12 +55,12 @@ bool ParserDropAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & std::shared_ptr row_policy_names; String cluster; - if ((type == EntityType::USER) || (type == EntityType::ROLE)) + if ((type == AccessEntityType::USER) || (type == AccessEntityType::ROLE)) { if (!parseUserNames(pos, expected, names)) return false; } - else if (type == EntityType::ROW_POLICY) + else if (type == AccessEntityType::ROW_POLICY) { ParserRowPolicyNames parser; ASTPtr ast; diff --git a/src/Parsers/Access/ParserShowAccessEntitiesQuery.cpp b/src/Parsers/Access/ParserShowAccessEntitiesQuery.cpp index b1329735b64..3953c28c356 100644 --- a/src/Parsers/Access/ParserShowAccessEntitiesQuery.cpp +++ b/src/Parsers/Access/ParserShowAccessEntitiesQuery.cpp @@ -10,14 +10,11 @@ namespace DB { namespace { - using EntityType = IAccessEntity::Type; - using EntityTypeInfo = IAccessEntity::TypeInfo; - - bool parseEntityType(IParserBase::Pos & pos, Expected & expected, EntityType & type) + bool parseEntityType(IParserBase::Pos & pos, Expected & expected, AccessEntityType & type) { - for (auto i : collections::range(EntityType::MAX)) + for (auto i : collections::range(AccessEntityType::MAX)) { - const auto & type_info = EntityTypeInfo::get(i); + const auto & type_info = AccessEntityTypeInfo::get(i); if (ParserKeyword{type_info.plural_name.c_str()}.ignore(pos, expected) || (!type_info.plural_alias.empty() && ParserKeyword{type_info.plural_alias.c_str()}.ignore(pos, expected))) { @@ -44,7 +41,7 @@ bool ParserShowAccessEntitiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected if (!ParserKeyword{"SHOW"}.ignore(pos, expected)) return false; - EntityType type; + AccessEntityType type; bool all = false; bool current_quota = false; bool current_roles = false; @@ -56,17 +53,17 @@ bool ParserShowAccessEntitiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected } else if (ParserKeyword{"CURRENT ROLES"}.ignore(pos, expected)) { - type = EntityType::ROLE; + type = AccessEntityType::ROLE; current_roles = true; } else if (ParserKeyword{"ENABLED ROLES"}.ignore(pos, expected)) { - type = EntityType::ROLE; + type = AccessEntityType::ROLE; enabled_roles = true; } else if (ParserKeyword{"CURRENT QUOTA"}.ignore(pos, expected) || ParserKeyword{"QUOTA"}.ignore(pos, expected)) { - type = EntityType::QUOTA; + type = AccessEntityType::QUOTA; current_quota = true; } else @@ -74,7 +71,7 @@ bool ParserShowAccessEntitiesQuery::parseImpl(Pos & pos, ASTPtr & node, Expected String short_name; std::optional> database_and_table_name; - if (type == EntityType::ROW_POLICY) + if (type == AccessEntityType::ROW_POLICY) { String database, table_name; bool any_database, any_table; diff --git a/src/Parsers/Access/ParserShowCreateAccessEntityQuery.cpp b/src/Parsers/Access/ParserShowCreateAccessEntityQuery.cpp index 2df04513361..88fafcaaf22 100644 --- a/src/Parsers/Access/ParserShowCreateAccessEntityQuery.cpp +++ b/src/Parsers/Access/ParserShowCreateAccessEntityQuery.cpp @@ -20,14 +20,11 @@ namespace ErrorCodes namespace { - using EntityType = IAccessEntity::Type; - using EntityTypeInfo = IAccessEntity::TypeInfo; - - bool parseEntityType(IParserBase::Pos & pos, Expected & expected, EntityType & type, bool & plural) + bool parseEntityType(IParserBase::Pos & pos, Expected & expected, AccessEntityType & type, bool & plural) { - for (auto i : collections::range(EntityType::MAX)) + for (auto i : collections::range(AccessEntityType::MAX)) { - const auto & type_info = EntityTypeInfo::get(i); + const auto & type_info = AccessEntityTypeInfo::get(i); if (ParserKeyword{type_info.name.c_str()}.ignore(pos, expected) || (!type_info.alias.empty() && ParserKeyword{type_info.alias.c_str()}.ignore(pos, expected))) { @@ -37,9 +34,9 @@ namespace } } - for (auto i : collections::range(EntityType::MAX)) + for (auto i : collections::range(AccessEntityType::MAX)) { - const auto & type_info = EntityTypeInfo::get(i); + const auto & type_info = AccessEntityTypeInfo::get(i); if (ParserKeyword{type_info.plural_name.c_str()}.ignore(pos, expected) || (!type_info.plural_alias.empty() && ParserKeyword{type_info.plural_alias.c_str()}.ignore(pos, expected))) { @@ -68,7 +65,7 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe if (!ParserKeyword{"SHOW CREATE"}.ignore(pos, expected)) return false; - EntityType type; + AccessEntityType type; bool plural; if (!parseEntityType(pos, expected, type, plural)) return false; @@ -83,7 +80,7 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe switch (type) { - case EntityType::USER: + case AccessEntityType::USER: { if (parseCurrentUserTag(pos, expected)) current_user = true; @@ -96,7 +93,7 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe current_user = true; break; } - case EntityType::ROLE: + case AccessEntityType::ROLE: { if (parseRoleNames(pos, expected, names)) { @@ -107,7 +104,7 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe return false; break; } - case EntityType::ROW_POLICY: + case AccessEntityType::ROW_POLICY: { ASTPtr ast; String database, table_name; @@ -130,7 +127,7 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe return false; break; } - case EntityType::SETTINGS_PROFILE: + case AccessEntityType::SETTINGS_PROFILE: { if (parseIdentifiersOrStringLiterals(pos, expected, names)) { @@ -141,7 +138,7 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe return false; break; } - case EntityType::QUOTA: + case AccessEntityType::QUOTA: { if (parseIdentifiersOrStringLiterals(pos, expected, names)) { @@ -152,7 +149,7 @@ bool ParserShowCreateAccessEntityQuery::parseImpl(Pos & pos, ASTPtr & node, Expe current_quota = true; break; } - case EntityType::MAX: + case AccessEntityType::MAX: throw Exception("Type " + toString(type) + " is not implemented in SHOW CREATE query", ErrorCodes::NOT_IMPLEMENTED); } diff --git a/src/Storages/System/StorageSystemGrants.cpp b/src/Storages/System/StorageSystemGrants.cpp index d443830ee0a..f55145ccfc7 100644 --- a/src/Storages/System/StorageSystemGrants.cpp +++ b/src/Storages/System/StorageSystemGrants.cpp @@ -17,7 +17,6 @@ namespace DB { -using EntityType = IAccessEntity::Type; NamesAndTypesList StorageSystemGrants::getNamesAndTypes() { @@ -58,7 +57,7 @@ void StorageSystemGrants::fillData(MutableColumns & res_columns, ContextPtr cont auto & column_grant_option = assert_cast(*res_columns[column_index++]).getData(); auto add_row = [&](const String & grantee_name, - EntityType grantee_type, + AccessEntityType grantee_type, AccessType access_type, const String * database, const String * table, @@ -66,14 +65,14 @@ void StorageSystemGrants::fillData(MutableColumns & res_columns, ContextPtr cont bool is_partial_revoke, bool grant_option) { - if (grantee_type == EntityType::USER) + if (grantee_type == AccessEntityType::USER) { column_user_name.insertData(grantee_name.data(), grantee_name.length()); column_user_name_null_map.push_back(false); column_role_name.insertDefault(); column_role_name_null_map.push_back(true); } - else if (grantee_type == EntityType::ROLE) + else if (grantee_type == AccessEntityType::ROLE) { column_user_name.insertDefault(); column_user_name_null_map.push_back(true); @@ -123,7 +122,7 @@ void StorageSystemGrants::fillData(MutableColumns & res_columns, ContextPtr cont }; auto add_rows = [&](const String & grantee_name, - IAccessEntity::Type grantee_type, + AccessEntityType grantee_type, const AccessRightsElements & elements) { for (const auto & element : elements) diff --git a/src/Storages/System/StorageSystemRoleGrants.cpp b/src/Storages/System/StorageSystemRoleGrants.cpp index 080c73726bc..94ee28cfe83 100644 --- a/src/Storages/System/StorageSystemRoleGrants.cpp +++ b/src/Storages/System/StorageSystemRoleGrants.cpp @@ -15,8 +15,6 @@ namespace DB { -using EntityType = IAccessEntity::Type; - NamesAndTypesList StorageSystemRoleGrants::getNamesAndTypes() { @@ -48,19 +46,19 @@ void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr auto & column_admin_option = assert_cast(*res_columns[column_index++]).getData(); auto add_row = [&](const String & grantee_name, - IAccessEntity::Type grantee_type, + AccessEntityType grantee_type, const String & granted_role_name, bool is_default, bool with_admin_option) { - if (grantee_type == EntityType::USER) + if (grantee_type == AccessEntityType::USER) { column_user_name.insertData(grantee_name.data(), grantee_name.length()); column_user_name_null_map.push_back(false); column_role_name.insertDefault(); column_role_name_null_map.push_back(true); } - else if (grantee_type == EntityType::ROLE) + else if (grantee_type == AccessEntityType::ROLE) { column_user_name.insertDefault(); column_user_name_null_map.push_back(true); @@ -76,7 +74,7 @@ void StorageSystemRoleGrants::fillData(MutableColumns & res_columns, ContextPtr }; auto add_rows = [&](const String & grantee_name, - IAccessEntity::Type grantee_type, + AccessEntityType grantee_type, const GrantedRoles & granted_roles, const RolesOrUsersSet * default_roles) { diff --git a/src/Storages/System/StorageSystemSettingsProfileElements.cpp b/src/Storages/System/StorageSystemSettingsProfileElements.cpp index b2991baf9cb..8013a3f2e9e 100644 --- a/src/Storages/System/StorageSystemSettingsProfileElements.cpp +++ b/src/Storages/System/StorageSystemSettingsProfileElements.cpp @@ -16,8 +16,6 @@ namespace DB { -using EntityType = IAccessEntity::Type; - NamesAndTypesList StorageSystemSettingsProfileElements::getNamesAndTypes() { @@ -66,7 +64,7 @@ void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns auto & column_inherit_profile = assert_cast(assert_cast(*res_columns[i]).getNestedColumn()); auto & column_inherit_profile_null_map = assert_cast(*res_columns[i++]).getNullMapData(); - auto add_rows_for_single_element = [&](const String & owner_name, EntityType owner_type, const SettingsProfileElement & element, size_t & index) + auto add_rows_for_single_element = [&](const String & owner_name, AccessEntityType owner_type, const SettingsProfileElement & element, size_t & index) { size_t old_num_rows = column_profile_name.size(); size_t new_num_rows = old_num_rows + 1; @@ -133,19 +131,19 @@ void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns { switch (owner_type) { - case EntityType::SETTINGS_PROFILE: + case AccessEntityType::SETTINGS_PROFILE: { column_profile_name.insertData(owner_name.data(), owner_name.length()); column_profile_name_null_map.push_back(false); break; } - case EntityType::USER: + case AccessEntityType::USER: { column_user_name.insertData(owner_name.data(), owner_name.length()); column_user_name_null_map.push_back(false); break; } - case EntityType::ROLE: + case AccessEntityType::ROLE: { column_role_name.insertData(owner_name.data(), owner_name.length()); column_role_name_null_map.push_back(false); @@ -162,7 +160,7 @@ void StorageSystemSettingsProfileElements::fillData(MutableColumns & res_columns } }; - auto add_rows = [&](const String & owner_name, IAccessEntity::Type owner_type, const SettingsProfileElements & elements) + auto add_rows = [&](const String & owner_name, AccessEntityType owner_type, const SettingsProfileElements & elements) { size_t index = 0; for (const auto & element : elements) From 7a3721c04f82ca84038ba7cace2e9c8e3622ce89 Mon Sep 17 00:00:00 2001 From: Ali Demirci Date: Fri, 19 Nov 2021 03:56:52 +0300 Subject: [PATCH 149/200] Update client-libraries.md --- docs/en/interfaces/third-party/client-libraries.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/en/interfaces/third-party/client-libraries.md b/docs/en/interfaces/third-party/client-libraries.md index a66af2fb122..342b1c9a496 100644 --- a/docs/en/interfaces/third-party/client-libraries.md +++ b/docs/en/interfaces/third-party/client-libraries.md @@ -36,6 +36,7 @@ toc_title: Client Libraries - [clickhouse (NodeJs)](https://github.com/TimonKK/clickhouse) - [node-clickhouse](https://github.com/apla/node-clickhouse) - [nestjs-clickhouse](https://github.com/depyronick/nestjs-clickhouse) + - [clickhouse-client](https://github.com/depyronick/clickhouse-client) - Perl - [perl-DBD-ClickHouse](https://github.com/elcamlost/perl-DBD-ClickHouse) - [HTTP-ClickHouse](https://metacpan.org/release/HTTP-ClickHouse) From 4a60b5ade643d5155947dda4972d556b8d270206 Mon Sep 17 00:00:00 2001 From: Ali Demirci Date: Fri, 19 Nov 2021 03:58:12 +0300 Subject: [PATCH 150/200] Update client-libraries.md --- docs/ru/interfaces/third-party/client-libraries.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/ru/interfaces/third-party/client-libraries.md b/docs/ru/interfaces/third-party/client-libraries.md index b2896f810b7..82413062e41 100644 --- a/docs/ru/interfaces/third-party/client-libraries.md +++ b/docs/ru/interfaces/third-party/client-libraries.md @@ -31,6 +31,8 @@ toc_title: "Клиентские библиотеки от сторонних р - NodeJs - [clickhouse (NodeJs)](https://github.com/TimonKK/clickhouse) - [node-clickhouse](https://github.com/apla/node-clickhouse) + - [nestjs-clickhouse](https://github.com/depyronick/nestjs-clickhouse) + - [clickhouse-client](https://github.com/depyronick/clickhouse-client) - Perl - [perl-DBD-ClickHouse](https://github.com/elcamlost/perl-DBD-ClickHouse) - [HTTP-ClickHouse](https://metacpan.org/release/HTTP-ClickHouse) From 3c0c7fd070822bc458f4a800ab2c2c936267a231 Mon Sep 17 00:00:00 2001 From: achimbab <07c00h@gmail.com> Date: Fri, 19 Nov 2021 10:55:48 +0900 Subject: [PATCH 151/200] Update links in adopters.md --- docs/en/introduction/adopters.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 61e29ee1840..fd0f3a12b81 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -79,7 +79,7 @@ toc_title: Adopters | Ivi | Online Cinema | Analytics, Monitoring | — | — | [Article in Russian, Jan 2018](https://habr.com/en/company/ivi/blog/347408/) | | Jinshuju 金数据 | BI Analytics | Main product | — | — | [Slides in Chinese, October 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/3.%20金数据数据架构调整方案Public.pdf) | | Jitsu | Cloud Software | Data Pipeline | — | — | [Documentation](https://jitsu.com/docs/destinations-configuration/clickhouse-destination), [Hacker News](https://news.ycombinator.com/item?id=29106082) | -| kakaocorp | Internet company | — | — | — | [if(kakao)2020 conference](https://if.kakao.com/session/117) | +| kakaocorp | Internet company | — | — | — | [if(kakao)2020](https://tv.kakao.com/channel/3693125/cliplink/414129353), [if(kakao)2021](https://if.kakao.com/session/24) | | Kodiak Data | Clouds | Main product | — | — | [Slides in Engish, April 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup13/kodiak_data.pdf) | | Kontur | Software Development | Metrics | — | — | [Talk in Russian, November 2018](https://www.youtube.com/watch?v=U4u4Bd0FtrY) | | Kuaishou | Video | — | — | — | [ClickHouse Meetup, October 2018](https://clickhouse.com/blog/en/2018/clickhouse-community-meetup-in-beijing-on-october-28-2018/) | From c47ec31617fee204229be78e879a922e0f6a0345 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 19 Nov 2021 11:51:05 +0800 Subject: [PATCH 152/200] fix --- src/Storages/MergeTree/MergeTreeData.cpp | 22 ++++++++++--------- src/Storages/MergeTree/MergeTreeData.h | 2 +- .../01710_minmax_count_projection.reference | 1 + .../01710_minmax_count_projection.sql | 1 + 4 files changed, 15 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 22f582dc13a..ad9bb255845 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -4539,12 +4539,12 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( } -bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( +std::optional MergeTreeData::getQueryProcessingStageWithAggregateProjection( ContextPtr query_context, const StorageMetadataPtr & metadata_snapshot, SelectQueryInfo & query_info) const { const auto & settings = query_context->getSettingsRef(); if (!settings.allow_experimental_projection_optimization || query_info.ignore_projections || query_info.is_projection_query) - return false; + return std::nullopt; const auto & query_ptr = query_info.original_query; @@ -4552,16 +4552,16 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( { // Currently projections don't support final yet. if (select->final()) - return false; + return std::nullopt; // Currently projections don't support ARRAY JOIN yet. if (select->arrayJoinExpressionList().first) - return false; + return std::nullopt; } // Currently projections don't support sampling yet. if (settings.parallel_replicas_count > 1) - return false; + return std::nullopt; InterpreterSelectQuery select( query_ptr, @@ -4918,14 +4918,14 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( } if (!selected_candidate) - return false; + return std::nullopt; else if (min_sum_marks == 0) { /// If selected_projection indicated an empty result set. Remember it in query_info but /// don't use projection to run the query, because projection pipeline with empty result /// set will not work correctly with empty_result_for_aggregation_by_empty_set. query_info.merge_tree_empty_result = true; - return false; + return std::nullopt; } if (selected_candidate->desc->type == ProjectionDescription::Type::Aggregate) @@ -4936,8 +4936,7 @@ bool MergeTreeData::getQueryProcessingStageWithAggregateProjection( = std::make_shared(std::move(select.getQueryAnalyzer()->getSubqueriesForSets())); } - query_info.projection = std::move(*selected_candidate); - return true; + return *selected_candidate; } @@ -4949,11 +4948,14 @@ QueryProcessingStage::Enum MergeTreeData::getQueryProcessingStage( { if (to_stage >= QueryProcessingStage::Enum::WithMergeableState) { - if (getQueryProcessingStageWithAggregateProjection(query_context, metadata_snapshot, query_info)) + if (auto projection = getQueryProcessingStageWithAggregateProjection(query_context, metadata_snapshot, query_info)) { + query_info.projection = std::move(projection); if (query_info.projection->desc->type == ProjectionDescription::Type::Aggregate) return QueryProcessingStage::Enum::WithMergeableState; } + else + query_info.projection = std::nullopt; } return QueryProcessingStage::Enum::FetchColumns; diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 1b617a2ec71..50690f638ac 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -383,7 +383,7 @@ public: DataPartsVector & normal_parts, ContextPtr query_context) const; - bool getQueryProcessingStageWithAggregateProjection( + std::optional getQueryProcessingStageWithAggregateProjection( ContextPtr query_context, const StorageMetadataPtr & metadata_snapshot, SelectQueryInfo & query_info) const; QueryProcessingStage::Enum getQueryProcessingStage( diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.reference b/tests/queries/0_stateless/01710_minmax_count_projection.reference index 495a5e7b6fa..77649f536f5 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.reference +++ b/tests/queries/0_stateless/01710_minmax_count_projection.reference @@ -16,3 +16,4 @@ \N 2021-10-27 10:00:00 4 2021-10-24 10:00:00 2021-10-24 10:00:00 +0 diff --git a/tests/queries/0_stateless/01710_minmax_count_projection.sql b/tests/queries/0_stateless/01710_minmax_count_projection.sql index b3dad492bdc..713241ada72 100644 --- a/tests/queries/0_stateless/01710_minmax_count_projection.sql +++ b/tests/queries/0_stateless/01710_minmax_count_projection.sql @@ -55,5 +55,6 @@ select count() from d group by toDate(dt); SELECT pointInEllipses(min(j), NULL), max(dt), count('0.0000000007') FROM d WHERE toDate(dt) >= '2021-10-25'; SELECT min(dt) FROM d PREWHERE ceil(j) <= 0; SELECT min(dt) FROM d PREWHERE ((0.9998999834060669 AND 1023) AND 255) <= ceil(j); +SELECT count('') AND NULL FROM d PREWHERE ceil(j) <= NULL; drop table d; From 40038ec80ff32b56142cb72ecfd83ef93fc7f71c Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 19 Nov 2021 07:40:15 +0300 Subject: [PATCH 153/200] Update gcp_n2d.json --- website/benchmark/hardware/results/gcp_n2d.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/benchmark/hardware/results/gcp_n2d.json b/website/benchmark/hardware/results/gcp_n2d.json index 25c20fb5457..1ce0524f0cf 100644 --- a/website/benchmark/hardware/results/gcp_n2d.json +++ b/website/benchmark/hardware/results/gcp_n2d.json @@ -1,6 +1,6 @@ [ { - "system": "GCP n2d-16-highmem" + "system": "GCP n2d-16-highmem", "system_full": "GCP compute n2d-16-highmem, AMD EPYC 7B12, 16vCPU, 128 GiB RAM", "time": "2021-11-18 00:00:00", "kind": "cloud", From 9114bc8b9ab08bd34526eead6a128fd9ffceacac Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 19 Nov 2021 12:48:08 +0300 Subject: [PATCH 154/200] Update conftest.py --- tests/integration/conftest.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/conftest.py b/tests/integration/conftest.py index bcd47899ca0..9461f4a81c5 100644 --- a/tests/integration/conftest.py +++ b/tests/integration/conftest.py @@ -7,8 +7,10 @@ from helpers.network import _NetworkManager @pytest.fixture(autouse=True, scope="session") def cleanup_environment(): - _NetworkManager.clean_all_user_iptables_rules() try: + if int(os.environ.get("PYTEST_CLEANUP_CONTAINERS")) == 1: + logging.debug(f"Cleaning all iptables rules") + _NetworkManager.clean_all_user_iptables_rules() result = run_and_check(['docker ps | wc -l'], shell=True) if int(result) > 1: if int(os.environ.get("PYTEST_CLEANUP_CONTAINERS")) != 1: From f3d0e6d999e8159a4786f5a0ff52ab76b7cde5c7 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 19 Nov 2021 12:58:18 +0300 Subject: [PATCH 155/200] remove `partial_merge_join_optimizations` this option is redundant because optimization is controlled by partial_merge_join_left_table_buffer_bytes --- src/Core/Settings.h | 3 ++- src/Interpreters/MergeJoin.cpp | 8 ++++---- src/Interpreters/TableJoin.cpp | 1 - src/Interpreters/TableJoin.h | 2 -- 4 files changed, 6 insertions(+), 8 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4988de44f9d..7750b967854 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -331,7 +331,6 @@ class IColumn; M(OverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ M(Bool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.", IMPORTANT) \ M(JoinAlgorithm, join_algorithm, JoinAlgorithm::HASH, "Specify join algorithm: 'auto', 'hash', 'partial_merge', 'prefer_partial_merge'. 'auto' tries to change HashJoin to MergeJoin on the fly to avoid out of memory.", 0) \ - M(Bool, partial_merge_join_optimizations, true, "Enable optimizations in partial merge join", 0) \ M(UInt64, default_max_bytes_in_join, 1000000000, "Maximum size of right-side table if limit is required but max_bytes_in_join is not set.", 0) \ M(UInt64, partial_merge_join_left_table_buffer_bytes, 32000000, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread. In current version work only with 'partial_merge_join_optimizations = 1'.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ @@ -555,6 +554,8 @@ class IColumn; MAKE_OBSOLETE(M, Bool, allow_experimental_map_type, true) \ MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_temporary_directories_interval_seconds, 60) \ MAKE_OBSOLETE(M, UInt64, merge_tree_clear_old_parts_interval_seconds, 1) \ + MAKE_OBSOLETE(M, UInt64, partial_merge_join_optimizations, 0) \ + /** The section above is for obsolete settings. Do not add anything there. */ diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 48d5f5254ff..17b95259cc7 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -552,10 +552,10 @@ MergeJoin::MergeJoin(std::shared_ptr table_join_, const Block & right LOG_DEBUG(log, "Joining keys: left [{}], right [{}]", fmt::join(key_names_left, ", "), fmt::join(key_names_right, ", ")); - /// Temporary disable 'partial_merge_join_left_table_buffer_bytes' without 'partial_merge_join_optimizations' - if (table_join->enablePartialMergeJoinOptimizations()) - if (size_t max_bytes = table_join->maxBytesInLeftBuffer()) - left_blocks_buffer = std::make_shared(left_sort_description, max_bytes); + if (size_t max_bytes = table_join->maxBytesInLeftBuffer(); max_bytes > 0) + { + left_blocks_buffer = std::make_shared(left_sort_description, max_bytes); + } } /// Has to be called even if totals are empty diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index e657bf38e49..42424b81192 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -99,7 +99,6 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_) , join_use_nulls(settings.join_use_nulls) , max_joined_block_rows(settings.max_joined_block_size_rows) , join_algorithm(settings.join_algorithm) - , partial_merge_join_optimizations(settings.partial_merge_join_optimizations) , partial_merge_join_rows_in_right_blocks(settings.partial_merge_join_rows_in_right_blocks) , partial_merge_join_left_table_buffer_bytes(settings.partial_merge_join_left_table_buffer_bytes) , max_files_to_merge(settings.join_on_disk_max_files_to_merge) diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 956fed99fb8..b3e5748fb2f 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -108,7 +108,6 @@ private: const bool join_use_nulls = false; const size_t max_joined_block_rows = 0; JoinAlgorithm join_algorithm = JoinAlgorithm::AUTO; - const bool partial_merge_join_optimizations = false; const size_t partial_merge_join_rows_in_right_blocks = 0; const size_t partial_merge_join_left_table_buffer_bytes = 0; const size_t max_files_to_merge = 0; @@ -205,7 +204,6 @@ public: size_t maxBytesInLeftBuffer() const { return partial_merge_join_left_table_buffer_bytes; } size_t maxFilesToMerge() const { return max_files_to_merge; } const String & temporaryFilesCodec() const { return temporary_files_codec; } - bool enablePartialMergeJoinOptimizations() const { return partial_merge_join_optimizations; } bool needStreamWithNonJoinedRows() const; bool oneDisjunct() const; From 2e0d3efba0e22748507f204df444b72846849a8e Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 19 Nov 2021 12:59:23 +0300 Subject: [PATCH 156/200] disable `partial_merge_join_left_table_buffer_bytes` due to bug --- src/Core/Settings.h | 2 +- src/Interpreters/MergeJoin.cpp | 5 ++++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 7750b967854..b42d5708973 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -332,7 +332,7 @@ class IColumn; M(Bool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.", IMPORTANT) \ M(JoinAlgorithm, join_algorithm, JoinAlgorithm::HASH, "Specify join algorithm: 'auto', 'hash', 'partial_merge', 'prefer_partial_merge'. 'auto' tries to change HashJoin to MergeJoin on the fly to avoid out of memory.", 0) \ M(UInt64, default_max_bytes_in_join, 1000000000, "Maximum size of right-side table if limit is required but max_bytes_in_join is not set.", 0) \ - M(UInt64, partial_merge_join_left_table_buffer_bytes, 32000000, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread. In current version work only with 'partial_merge_join_optimizations = 1'.", 0) \ + M(UInt64, partial_merge_join_left_table_buffer_bytes, 0, "If not 0 group left table blocks in bigger ones for left-side table in partial merge join. It uses up to 2x of specified memory per joining thread.", 0) \ M(UInt64, partial_merge_join_rows_in_right_blocks, 65536, "Split right-hand joining data in blocks of specified size. It's a portion of data indexed by min-max values and possibly unloaded on disk.", 0) \ M(UInt64, join_on_disk_max_files_to_merge, 64, "For MergeJoin on disk set how much files it's allowed to sort simultaneously. Then this value bigger then more memory used and then less disk I/O needed. Minimum is 2.", 0) \ M(String, temporary_files_codec, "LZ4", "Set compression codec for temporary files (sort and join on disk). I.e. LZ4, NONE.", 0) \ diff --git a/src/Interpreters/MergeJoin.cpp b/src/Interpreters/MergeJoin.cpp index 17b95259cc7..6f4fef46886 100644 --- a/src/Interpreters/MergeJoin.cpp +++ b/src/Interpreters/MergeJoin.cpp @@ -554,7 +554,10 @@ MergeJoin::MergeJoin(std::shared_ptr table_join_, const Block & right if (size_t max_bytes = table_join->maxBytesInLeftBuffer(); max_bytes > 0) { - left_blocks_buffer = std::make_shared(left_sort_description, max_bytes); + /// Disabled due to https://github.com/ClickHouse/ClickHouse/issues/31009 + // left_blocks_buffer = std::make_shared(left_sort_description, max_bytes); + LOG_WARNING(log, "`partial_merge_join_left_table_buffer_bytes` is disabled in current version of ClickHouse"); + UNUSED(left_blocks_buffer); } } From dd2972b8c336e19779bbc9741db2f46ab1c4778f Mon Sep 17 00:00:00 2001 From: SuperDJY Date: Fri, 19 Nov 2021 18:25:55 +0800 Subject: [PATCH 157/200] return fake create query when executing `show create table` on system's tables. (#31391) * clarify that cannot show create table of system's tables * clarify that cannot show create table of system's tables in the document * bypass test style-check for required `database=currentDatabase()` because `show create table` do not have condition * rename supportsShowCreateTable to isSystemStorage * build fake create query for show create table of system's tables * fix tests for show create table on system's table: 1. only build fake create query when cannot get real query 2. only test rocksdb when it's enabled. * fix test for show create table system.tables * fix test for ASTCreateQuery set table and database * change to LOGICAL_ERROR when cannot get in_memory metadata Co-authored-by: tavplubix * implement common getCreateQueryFromStorage Co-authored-by: tavplubix --- docs/en/sql-reference/statements/show.md | 2 + docs/zh/sql-reference/statements/show.md | 1 + src/DataTypes/convertMySQLDataType.cpp | 9 -- src/DataTypes/convertMySQLDataType.h | 5 -- src/Databases/DatabaseOnDisk.cpp | 34 +++++++- src/Databases/DatabaseOnDisk.h | 1 + src/Databases/DatabasesCommon.cpp | 62 ++++++++++++++ src/Databases/DatabasesCommon.h | 1 + src/Databases/MySQL/DatabaseMySQL.cpp | 69 +++++---------- src/Databases/SQLite/DatabaseSQLite.cpp | 41 +-------- src/Databases/SQLite/DatabaseSQLite.h | 3 +- src/Storages/IStorage.h | 3 + src/Storages/System/IStorageSystemOneBlock.h | 2 + src/Storages/System/StorageSystemColumns.h | 2 + .../System/StorageSystemDataSkippingIndices.h | 2 + .../System/StorageSystemDetachedParts.h | 1 + src/Storages/System/StorageSystemDisks.h | 2 + src/Storages/System/StorageSystemNumbers.h | 1 + src/Storages/System/StorageSystemOne.h | 2 + src/Storages/System/StorageSystemPartsBase.h | 2 + src/Storages/System/StorageSystemReplicas.h | 2 + .../System/StorageSystemStoragePolicies.h | 2 + src/Storages/System/StorageSystemTables.h | 2 + src/Storages/System/StorageSystemZeros.h | 1 + .../02117_show_create_table_system.reference | 66 +++++++++++++++ .../02117_show_create_table_system.sql | 84 +++++++++++++++++++ .../02118_show_create_table_rocksdb.reference | 1 + .../02118_show_create_table_rocksdb.sql | 3 + 28 files changed, 304 insertions(+), 102 deletions(-) create mode 100644 tests/queries/0_stateless/02117_show_create_table_system.reference create mode 100644 tests/queries/0_stateless/02117_show_create_table_system.sql create mode 100644 tests/queries/0_stateless/02118_show_create_table_rocksdb.reference create mode 100644 tests/queries/0_stateless/02118_show_create_table_rocksdb.sql diff --git a/docs/en/sql-reference/statements/show.md b/docs/en/sql-reference/statements/show.md index e00d58dfed4..96cbee0b04d 100644 --- a/docs/en/sql-reference/statements/show.md +++ b/docs/en/sql-reference/statements/show.md @@ -13,6 +13,8 @@ SHOW CREATE [TEMPORARY] [TABLE|DICTIONARY|VIEW] [db.]table|view [INTO OUTFILE fi Returns a single `String`-type ‘statement’ column, which contains a single value – the `CREATE` query used for creating the specified object. +Note that if you use this statement to get `CREATE` query of system tables, you will get a *fake* query, which only declares table structure, but cannot be used to create table. + ## SHOW DATABASES {#show-databases} Prints a list of all databases. diff --git a/docs/zh/sql-reference/statements/show.md b/docs/zh/sql-reference/statements/show.md index 85c5744f26b..0dfd5943a0d 100644 --- a/docs/zh/sql-reference/statements/show.md +++ b/docs/zh/sql-reference/statements/show.md @@ -12,6 +12,7 @@ SHOW CREATE [TEMPORARY] [TABLE|DICTIONARY] [db.]table [INTO OUTFILE filename] [F ``` 返回单个字符串类型的 ‘statement’列,其中只包含了一个值 - 用来创建指定对象的 `CREATE` 语句。 +注意,如果使用该查询去获取系统表的 `CREATE` 语句,你得到的是一个虚构的语句,仅用来展示系统的表结构,而不能实际创建表。 ## SHOW DATABASES {#show-databases} diff --git a/src/DataTypes/convertMySQLDataType.cpp b/src/DataTypes/convertMySQLDataType.cpp index 5ad8c43d0cf..d11a2a2211a 100644 --- a/src/DataTypes/convertMySQLDataType.cpp +++ b/src/DataTypes/convertMySQLDataType.cpp @@ -20,15 +20,6 @@ namespace DB { -ASTPtr dataTypeConvertToQuery(const DataTypePtr & data_type) -{ - WhichDataType which(data_type); - - if (!which.isNullable()) - return std::make_shared(data_type->getName()); - - return makeASTFunction("Nullable", dataTypeConvertToQuery(typeid_cast(data_type.get())->getNestedType())); -} DataTypePtr convertMySQLDataType(MultiEnum type_support, const std::string & mysql_data_type, diff --git a/src/DataTypes/convertMySQLDataType.h b/src/DataTypes/convertMySQLDataType.h index f1c4a73d6f7..543119bc60e 100644 --- a/src/DataTypes/convertMySQLDataType.h +++ b/src/DataTypes/convertMySQLDataType.h @@ -9,11 +9,6 @@ namespace DB { enum class MySQLDataTypesSupport; -/// Convert data type to query. for example -/// DataTypeUInt8 -> ASTIdentifier(UInt8) -/// DataTypeNullable(DataTypeUInt8) -> ASTFunction(ASTIdentifier(UInt8)) -ASTPtr dataTypeConvertToQuery(const DataTypePtr & data_type); - /// Convert MySQL type to ClickHouse data type. DataTypePtr convertMySQLDataType(MultiEnum type_support, const std::string & mysql_data_type, bool is_nullable, bool is_unsigned, size_t length, size_t precision, size_t scale); diff --git a/src/Databases/DatabaseOnDisk.cpp b/src/Databases/DatabaseOnDisk.cpp index a1e0a825736..decf7f87473 100644 --- a/src/Databases/DatabaseOnDisk.cpp +++ b/src/Databases/DatabaseOnDisk.cpp @@ -430,7 +430,11 @@ void DatabaseOnDisk::renameTable( ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, ContextPtr, bool throw_on_error) const { ASTPtr ast; - bool has_table = tryGetTable(table_name, getContext()) != nullptr; + StoragePtr storage = tryGetTable(table_name, getContext()); + bool has_table = storage != nullptr; + bool is_system_storage = false; + if (has_table) + is_system_storage = storage->isSystemStorage(); auto table_metadata_path = getObjectMetadataPath(table_name); try { @@ -441,6 +445,8 @@ ASTPtr DatabaseOnDisk::getCreateTableQueryImpl(const String & table_name, Contex if (!has_table && e.code() == ErrorCodes::FILE_DOESNT_EXIST && throw_on_error) throw Exception{"Table " + backQuote(table_name) + " doesn't exist", ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY}; + else if (is_system_storage) + ast = getCreateQueryFromStorage(table_name, storage, throw_on_error); else if (throw_on_error) throw; } @@ -673,6 +679,32 @@ ASTPtr DatabaseOnDisk::getCreateQueryFromMetadata(const String & database_metada return ast; } +ASTPtr DatabaseOnDisk::getCreateQueryFromStorage(const String & table_name, const StoragePtr & storage, bool throw_on_error) const +{ + auto metadata_ptr = storage->getInMemoryMetadataPtr(); + if (metadata_ptr == nullptr) + { + if (throw_on_error) + throw Exception(ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY, "Cannot get metadata of {}.{}", backQuote(getDatabaseName()), backQuote(table_name)); + else + return nullptr; + } + + /// setup create table query storage info. + auto ast_engine = std::make_shared(); + ast_engine->name = storage->getName(); + auto ast_storage = std::make_shared(); + ast_storage->set(ast_storage->engine, ast_engine); + + auto create_table_query = DB::getCreateQueryFromStorage(storage, ast_storage, false, + getContext()->getSettingsRef().max_parser_depth, throw_on_error); + + create_table_query->set(create_table_query->as()->comment, + std::make_shared("SYSTEM TABLE is built on the fly.")); + + return create_table_query; +} + void DatabaseOnDisk::modifySettingsMetadata(const SettingsChanges & settings_changes, ContextPtr query_context) { std::lock_guard lock(modify_settings_mutex); diff --git a/src/Databases/DatabaseOnDisk.h b/src/Databases/DatabaseOnDisk.h index f27a332fa3e..9bc8fa3bcef 100644 --- a/src/Databases/DatabaseOnDisk.h +++ b/src/Databases/DatabaseOnDisk.h @@ -89,6 +89,7 @@ protected: bool throw_on_error) const override; ASTPtr getCreateQueryFromMetadata(const String & metadata_path, bool throw_on_error) const; + ASTPtr getCreateQueryFromStorage(const String & table_name, const StoragePtr & storage, bool throw_on_error) const; virtual void commitCreateTable(const ASTCreateQuery & query, const StoragePtr & table, const String & table_metadata_tmp_path, const String & table_metadata_path, ContextPtr query_context); diff --git a/src/Databases/DatabasesCommon.cpp b/src/Databases/DatabasesCommon.cpp index c92aac44083..efadefba7be 100644 --- a/src/Databases/DatabasesCommon.cpp +++ b/src/Databases/DatabasesCommon.cpp @@ -20,6 +20,8 @@ namespace ErrorCodes extern const int UNKNOWN_TABLE; extern const int UNKNOWN_DATABASE; extern const int NOT_IMPLEMENTED; + extern const int LOGICAL_ERROR; + extern const int CANNOT_GET_CREATE_TABLE_QUERY; } void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemoryMetadata & metadata) @@ -85,6 +87,66 @@ void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemo } +ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr & ast_storage, bool only_ordinary, uint32_t max_parser_depth, bool throw_on_error) +{ + auto table_id = storage->getStorageID(); + auto metadata_ptr = storage->getInMemoryMetadataPtr(); + if (metadata_ptr == nullptr) + { + if (throw_on_error) + throw Exception(ErrorCodes::CANNOT_GET_CREATE_TABLE_QUERY, "Cannot get metadata of {}.{}", backQuote(table_id.database_name), backQuote(table_id.table_name)); + else + return nullptr; + } + + auto create_table_query = std::make_shared(); + create_table_query->attach = false; + create_table_query->setTable(table_id.table_name); + create_table_query->setDatabase(table_id.database_name); + create_table_query->set(create_table_query->storage, ast_storage); + + /// setup create table query columns info. + { + auto ast_columns_list = std::make_shared(); + auto ast_expression_list = std::make_shared(); + NamesAndTypesList columns; + if (only_ordinary) + columns = metadata_ptr->columns.getOrdinary(); + else + columns = metadata_ptr->columns.getAll(); + for (const auto & column_name_and_type: columns) + { + const auto & ast_column_declaration = std::make_shared(); + ast_column_declaration->name = column_name_and_type.name; + /// parser typename + { + ASTPtr ast_type; + auto type_name = column_name_and_type.type->getName(); + const auto * string_end = type_name.c_str() + type_name.length(); + Expected expected; + expected.max_parsed_pos = string_end; + Tokens tokens(type_name.c_str(), string_end); + IParser::Pos pos(tokens, max_parser_depth); + ParserDataType parser; + if (!parser.parse(pos, ast_type, expected)) + { + if (throw_on_error) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot parser metadata of {}.{}", backQuote(table_id.database_name), backQuote(table_id.table_name)); + else + return nullptr; + } + ast_column_declaration->type = ast_type; + } + ast_expression_list->children.emplace_back(ast_column_declaration); + } + + ast_columns_list->set(ast_columns_list->columns, ast_expression_list); + create_table_query->set(create_table_query->columns_list, ast_columns_list); + } + return create_table_query; +} + + DatabaseWithOwnTablesBase::DatabaseWithOwnTablesBase(const String & name_, const String & logger, ContextPtr context_) : IDatabase(name_), WithContext(context_->getGlobalContext()), log(&Poco::Logger::get(logger)) { diff --git a/src/Databases/DatabasesCommon.h b/src/Databases/DatabasesCommon.h index 59a2ddc3c41..229f9941fc6 100644 --- a/src/Databases/DatabasesCommon.h +++ b/src/Databases/DatabasesCommon.h @@ -14,6 +14,7 @@ namespace DB { void applyMetadataChangesToCreateQuery(const ASTPtr & query, const StorageInMemoryMetadata & metadata); +ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr & ast_storage, bool only_ordinary, uint32_t max_parser_depth, bool throw_on_error); class Context; diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 275a1015c7b..bb590fc2752 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -113,53 +113,6 @@ StoragePtr DatabaseMySQL::tryGetTable(const String & mysql_table_name, ContextPt return StoragePtr{}; } -static ASTPtr getCreateQueryFromStorage(const StoragePtr & storage, const ASTPtr & database_engine_define) -{ - auto create_table_query = std::make_shared(); - - auto table_storage_define = database_engine_define->clone(); - create_table_query->set(create_table_query->storage, table_storage_define); - - auto columns_declare_list = std::make_shared(); - auto columns_expression_list = std::make_shared(); - - columns_declare_list->set(columns_declare_list->columns, columns_expression_list); - create_table_query->set(create_table_query->columns_list, columns_declare_list); - - { - /// init create query. - auto table_id = storage->getStorageID(); - create_table_query->setTable(table_id.table_name); - create_table_query->setDatabase(table_id.database_name); - - auto metadata_snapshot = storage->getInMemoryMetadataPtr(); - for (const auto & column_type_and_name : metadata_snapshot->getColumns().getOrdinary()) - { - const auto & column_declaration = std::make_shared(); - column_declaration->name = column_type_and_name.name; - column_declaration->type = dataTypeConvertToQuery(column_type_and_name.type); - columns_expression_list->children.emplace_back(column_declaration); - } - - ASTStorage * ast_storage = table_storage_define->as(); - ASTs storage_children = ast_storage->children; - auto storage_engine_arguments = ast_storage->engine->arguments; - - /// Add table_name to engine arguments - auto mysql_table_name = std::make_shared(table_id.table_name); - storage_engine_arguments->children.insert(storage_engine_arguments->children.begin() + 2, mysql_table_name); - - /// Unset settings - storage_children.erase( - std::remove_if(storage_children.begin(), storage_children.end(), - [&](const ASTPtr & element) { return element.get() == ast_storage->settings; }), - storage_children.end()); - ast_storage->settings = nullptr; - } - - return create_table_query; -} - ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, ContextPtr local_context, bool throw_on_error) const { std::lock_guard lock(mutex); @@ -174,7 +127,27 @@ ASTPtr DatabaseMySQL::getCreateTableQueryImpl(const String & table_name, Context return nullptr; } - return getCreateQueryFromStorage(local_tables_cache[table_name].second, database_engine_define); + auto storage = local_tables_cache[table_name].second; + auto table_storage_define = database_engine_define->clone(); + { + ASTStorage * ast_storage = table_storage_define->as(); + ASTs storage_children = ast_storage->children; + auto storage_engine_arguments = ast_storage->engine->arguments; + + /// Add table_name to engine arguments + auto mysql_table_name = std::make_shared(table_name); + storage_engine_arguments->children.insert(storage_engine_arguments->children.begin() + 2, mysql_table_name); + + /// Unset settings + storage_children.erase( + std::remove_if(storage_children.begin(), storage_children.end(), + [&](const ASTPtr & element) { return element.get() == ast_storage->settings; }), + storage_children.end()); + ast_storage->settings = nullptr; + } + auto create_table_query = DB::getCreateQueryFromStorage(storage, table_storage_define, true, + getContext()->getSettingsRef().max_parser_depth, throw_on_error); + return create_table_query; } time_t DatabaseMySQL::getObjectMetadataModificationTime(const String & table_name) const diff --git a/src/Databases/SQLite/DatabaseSQLite.cpp b/src/Databases/SQLite/DatabaseSQLite.cpp index ba53ab08407..ec15ca90b9a 100644 --- a/src/Databases/SQLite/DatabaseSQLite.cpp +++ b/src/Databases/SQLite/DatabaseSQLite.cpp @@ -180,52 +180,19 @@ ASTPtr DatabaseSQLite::getCreateTableQueryImpl(const String & table_name, Contex database_name, table_name); return nullptr; } - - auto create_table_query = std::make_shared(); auto table_storage_define = database_engine_define->clone(); - create_table_query->set(create_table_query->storage, table_storage_define); - - auto columns_declare_list = std::make_shared(); - auto columns_expression_list = std::make_shared(); - - columns_declare_list->set(columns_declare_list->columns, columns_expression_list); - create_table_query->set(create_table_query->columns_list, columns_declare_list); - - /// init create query. - auto table_id = storage->getStorageID(); - create_table_query->setTable(table_id.table_name); - create_table_query->setDatabase(table_id.database_name); - - auto metadata_snapshot = storage->getInMemoryMetadataPtr(); - for (const auto & column_type_and_name : metadata_snapshot->getColumns().getOrdinary()) - { - const auto & column_declaration = std::make_shared(); - column_declaration->name = column_type_and_name.name; - column_declaration->type = getColumnDeclaration(column_type_and_name.type); - columns_expression_list->children.emplace_back(column_declaration); - } - ASTStorage * ast_storage = table_storage_define->as(); - ASTs storage_children = ast_storage->children; auto storage_engine_arguments = ast_storage->engine->arguments; - + auto table_id = storage->getStorageID(); /// Add table_name to engine arguments storage_engine_arguments->children.insert(storage_engine_arguments->children.begin() + 1, std::make_shared(table_id.table_name)); + auto create_table_query = DB::getCreateQueryFromStorage(storage, table_storage_define, true, + getContext()->getSettingsRef().max_parser_depth, throw_on_error); + return create_table_query; } - -ASTPtr DatabaseSQLite::getColumnDeclaration(const DataTypePtr & data_type) const -{ - WhichDataType which(data_type); - - if (which.isNullable()) - return makeASTFunction("Nullable", getColumnDeclaration(typeid_cast(data_type.get())->getNestedType())); - - return std::make_shared(data_type->getName()); -} - } #endif diff --git a/src/Databases/SQLite/DatabaseSQLite.h b/src/Databases/SQLite/DatabaseSQLite.h index d0b8d582844..c8df79d0f6a 100644 --- a/src/Databases/SQLite/DatabaseSQLite.h +++ b/src/Databases/SQLite/DatabaseSQLite.h @@ -12,7 +12,7 @@ namespace DB { -class DatabaseSQLite final : public IDatabase, protected WithContext +class DatabaseSQLite final : public IDatabase, WithContext { public: using SQLitePtr = std::shared_ptr; @@ -58,7 +58,6 @@ private: StoragePtr fetchTable(const String & table_name, ContextPtr context, bool table_checked) const; - ASTPtr getColumnDeclaration(const DataTypePtr & data_type) const; }; } diff --git a/src/Storages/IStorage.h b/src/Storages/IStorage.h index fa5f2c28b06..cf7b075a204 100644 --- a/src/Storages/IStorage.h +++ b/src/Storages/IStorage.h @@ -158,6 +158,9 @@ public: /// This is true for most storages that store data on disk. virtual bool prefersLargeBlocks() const { return true; } + /// Returns true if the storage is for system, which cannot be target of SHOW CREATE TABLE. + virtual bool isSystemStorage() const { return false; } + /// Optional size information of each physical column. /// Currently it's only used by the MergeTree family for query optimizations. diff --git a/src/Storages/System/IStorageSystemOneBlock.h b/src/Storages/System/IStorageSystemOneBlock.h index 9b51bd73903..33086498730 100644 --- a/src/Storages/System/IStorageSystemOneBlock.h +++ b/src/Storages/System/IStorageSystemOneBlock.h @@ -61,6 +61,8 @@ public: return Pipe(std::make_shared(sample_block, std::move(chunk))); } + bool isSystemStorage() const override { return true; } + static NamesAndAliases getNamesAndAliases() { return {}; } }; diff --git a/src/Storages/System/StorageSystemColumns.h b/src/Storages/System/StorageSystemColumns.h index adcbf384ca7..dc184b1ae42 100644 --- a/src/Storages/System/StorageSystemColumns.h +++ b/src/Storages/System/StorageSystemColumns.h @@ -26,6 +26,8 @@ public: size_t max_block_size, unsigned num_streams) override; + bool isSystemStorage() const override { return true; } + protected: StorageSystemColumns(const StorageID & table_id_); }; diff --git a/src/Storages/System/StorageSystemDataSkippingIndices.h b/src/Storages/System/StorageSystemDataSkippingIndices.h index 72f66b025cf..d86890f5e27 100644 --- a/src/Storages/System/StorageSystemDataSkippingIndices.h +++ b/src/Storages/System/StorageSystemDataSkippingIndices.h @@ -23,6 +23,8 @@ public: size_t max_block_size, unsigned num_streams) override; + bool isSystemStorage() const override { return true; } + protected: StorageSystemDataSkippingIndices(const StorageID & table_id_); }; diff --git a/src/Storages/System/StorageSystemDetachedParts.h b/src/Storages/System/StorageSystemDetachedParts.h index 9350e758d93..ece9d495500 100644 --- a/src/Storages/System/StorageSystemDetachedParts.h +++ b/src/Storages/System/StorageSystemDetachedParts.h @@ -18,6 +18,7 @@ class StorageSystemDetachedParts final : friend struct shared_ptr_helper; public: std::string getName() const override { return "SystemDetachedParts"; } + bool isSystemStorage() const override { return true; } protected: explicit StorageSystemDetachedParts(const StorageID & table_id_); diff --git a/src/Storages/System/StorageSystemDisks.h b/src/Storages/System/StorageSystemDisks.h index d6908a295d4..2541dedd8fc 100644 --- a/src/Storages/System/StorageSystemDisks.h +++ b/src/Storages/System/StorageSystemDisks.h @@ -29,6 +29,8 @@ public: size_t max_block_size, unsigned num_streams) override; + bool isSystemStorage() const override { return true; } + protected: StorageSystemDisks(const StorageID & table_id_); }; diff --git a/src/Storages/System/StorageSystemNumbers.h b/src/Storages/System/StorageSystemNumbers.h index da16dc73ca8..32105bb055d 100644 --- a/src/Storages/System/StorageSystemNumbers.h +++ b/src/Storages/System/StorageSystemNumbers.h @@ -39,6 +39,7 @@ public: unsigned num_streams) override; bool hasEvenlyDistributedRead() const override { return true; } + bool isSystemStorage() const override { return true; } private: bool multithreaded; diff --git a/src/Storages/System/StorageSystemOne.h b/src/Storages/System/StorageSystemOne.h index 3fe0c91113c..cc1d5e05b75 100644 --- a/src/Storages/System/StorageSystemOne.h +++ b/src/Storages/System/StorageSystemOne.h @@ -30,6 +30,8 @@ public: size_t max_block_size, unsigned num_streams) override; + bool isSystemStorage() const override { return true; } + protected: explicit StorageSystemOne(const StorageID & table_id_); }; diff --git a/src/Storages/System/StorageSystemPartsBase.h b/src/Storages/System/StorageSystemPartsBase.h index aec52e6b410..87247f96b24 100644 --- a/src/Storages/System/StorageSystemPartsBase.h +++ b/src/Storages/System/StorageSystemPartsBase.h @@ -67,6 +67,8 @@ public: NamesAndTypesList getVirtuals() const override; + bool isSystemStorage() const override { return true; } + private: bool hasStateColumn(const Names & column_names, const StorageMetadataPtr & metadata_snapshot) const; diff --git a/src/Storages/System/StorageSystemReplicas.h b/src/Storages/System/StorageSystemReplicas.h index ca36eef28c1..cf457efe250 100644 --- a/src/Storages/System/StorageSystemReplicas.h +++ b/src/Storages/System/StorageSystemReplicas.h @@ -27,6 +27,8 @@ public: size_t max_block_size, unsigned num_streams) override; + bool isSystemStorage() const override { return true; } + protected: StorageSystemReplicas(const StorageID & table_id_); }; diff --git a/src/Storages/System/StorageSystemStoragePolicies.h b/src/Storages/System/StorageSystemStoragePolicies.h index ad8b507b7c0..f202299db1f 100644 --- a/src/Storages/System/StorageSystemStoragePolicies.h +++ b/src/Storages/System/StorageSystemStoragePolicies.h @@ -29,6 +29,8 @@ public: size_t max_block_size, unsigned num_streams) override; + bool isSystemStorage() const override { return true; } + protected: StorageSystemStoragePolicies(const StorageID & table_id_); }; diff --git a/src/Storages/System/StorageSystemTables.h b/src/Storages/System/StorageSystemTables.h index 6424d623830..808dc862e8d 100644 --- a/src/Storages/System/StorageSystemTables.h +++ b/src/Storages/System/StorageSystemTables.h @@ -27,6 +27,8 @@ public: size_t max_block_size, unsigned num_streams) override; + bool isSystemStorage() const override { return true; } + protected: StorageSystemTables(const StorageID & table_id_); }; diff --git a/src/Storages/System/StorageSystemZeros.h b/src/Storages/System/StorageSystemZeros.h index e40b9c3993a..f5b2bb43117 100644 --- a/src/Storages/System/StorageSystemZeros.h +++ b/src/Storages/System/StorageSystemZeros.h @@ -30,6 +30,7 @@ public: unsigned num_streams) override; bool hasEvenlyDistributedRead() const override { return true; } + bool isSystemStorage() const override { return true; } private: bool multithreaded; diff --git a/tests/queries/0_stateless/02117_show_create_table_system.reference b/tests/queries/0_stateless/02117_show_create_table_system.reference new file mode 100644 index 00000000000..2b391cd292e --- /dev/null +++ b/tests/queries/0_stateless/02117_show_create_table_system.reference @@ -0,0 +1,66 @@ +CREATE TABLE system.aggregate_function_combinators\n(\n `name` String,\n `is_internal` UInt8\n)\nENGINE = SystemAggregateFunctionCombinators()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.asynchronous_inserts\n(\n `query` String,\n `database` String,\n `table` String,\n `format` String,\n `first_update` DateTime64(6),\n `last_update` DateTime64(6),\n `total_bytes` UInt64,\n `entries.query_id` Array(String),\n `entries.bytes` Array(UInt64),\n `entries.finished` Array(UInt8),\n `entries.exception` Array(String)\n)\nENGINE = AsynchronousInserts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.asynchronous_metrics\n(\n `metric` String,\n `value` Float64\n)\nENGINE = SystemAsynchronousMetrics()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.build_options\n(\n `name` String,\n `value` String\n)\nENGINE = SystemBuildOptions()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.clusters\n(\n `cluster` String,\n `shard_num` UInt32,\n `shard_weight` UInt32,\n `replica_num` UInt32,\n `host_name` String,\n `host_address` String,\n `port` UInt16,\n `is_local` UInt8,\n `user` String,\n `default_database` String,\n `errors_count` UInt32,\n `slowdowns_count` UInt32,\n `estimated_recovery_time` UInt32\n)\nENGINE = SystemClusters()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.collations\n(\n `name` String,\n `language` Nullable(String)\n)\nENGINE = SystemTableCollations()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.columns\n(\n `database` String,\n `table` String,\n `name` String,\n `type` String,\n `position` UInt64,\n `default_kind` String,\n `default_expression` String,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `comment` String,\n `is_in_partition_key` UInt8,\n `is_in_sorting_key` UInt8,\n `is_in_primary_key` UInt8,\n `is_in_sampling_key` UInt8,\n `compression_codec` String,\n `character_octet_length` Nullable(UInt64),\n `numeric_precision` Nullable(UInt64),\n `numeric_precision_radix` Nullable(UInt64),\n `numeric_scale` Nullable(UInt64),\n `datetime_precision` Nullable(UInt64)\n)\nENGINE = SystemColumns()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.contributors\n(\n `name` String\n)\nENGINE = SystemContributors()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.current_roles\n(\n `role_name` String,\n `with_admin_option` UInt8,\n `is_default` UInt8\n)\nENGINE = SystemCurrentRoles()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.data_skipping_indices\n(\n `database` String,\n `table` String,\n `name` String,\n `type` String,\n `expr` String,\n `granularity` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks` UInt64\n)\nENGINE = SystemDataSkippingIndices()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.data_type_families\n(\n `name` String,\n `case_insensitive` UInt8,\n `alias_to` String\n)\nENGINE = SystemTableDataTypeFamilies()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.databases\n(\n `name` String,\n `engine` String,\n `data_path` String,\n `metadata_path` String,\n `uuid` UUID,\n `comment` String,\n `database` String\n)\nENGINE = SystemDatabases()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.detached_parts\n(\n `database` String,\n `table` String,\n `partition_id` Nullable(String),\n `name` String,\n `disk` String,\n `reason` Nullable(String),\n `min_block_number` Nullable(Int64),\n `max_block_number` Nullable(Int64),\n `level` Nullable(UInt32)\n)\nENGINE = SystemDetachedParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.dictionaries\n(\n `database` String,\n `name` String,\n `uuid` UUID,\n `status` Enum8(\'NOT_LOADED\' = 0, \'LOADED\' = 1, \'FAILED\' = 2, \'LOADING\' = 3, \'FAILED_AND_RELOADING\' = 4, \'LOADED_AND_RELOADING\' = 5, \'NOT_EXIST\' = 6),\n `origin` String,\n `type` String,\n `key.names` Array(String),\n `key.types` Array(String),\n `attribute.names` Array(String),\n `attribute.types` Array(String),\n `bytes_allocated` UInt64,\n `query_count` UInt64,\n `hit_rate` Float64,\n `found_rate` Float64,\n `element_count` UInt64,\n `load_factor` Float64,\n `source` String,\n `lifetime_min` UInt64,\n `lifetime_max` UInt64,\n `loading_start_time` DateTime,\n `last_successful_update_time` DateTime,\n `loading_duration` Float32,\n `last_exception` String,\n `comment` String\n)\nENGINE = SystemDictionaries()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.disks\n(\n `name` String,\n `path` String,\n `free_space` UInt64,\n `total_space` UInt64,\n `keep_free_space` UInt64,\n `type` String\n)\nENGINE = SystemDisks()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.distributed_ddl_queue\n(\n `entry` String,\n `entry_version` Nullable(UInt8),\n `initiator_host` Nullable(String),\n `initiator_port` Nullable(UInt16),\n `cluster` String,\n `query` String,\n `settings` Map(String, String),\n `query_create_time` DateTime,\n `host` Nullable(String),\n `port` Nullable(UInt16),\n `status` Nullable(Enum8(\'Inactive\' = 0, \'Active\' = 1, \'Finished\' = 2, \'Removing\' = 3, \'Unknown\' = 4)),\n `exception_code` Nullable(UInt16),\n `exception_text` Nullable(String),\n `query_finish_time` Nullable(DateTime),\n `query_duration_ms` Nullable(UInt64)\n)\nENGINE = SystemDDLWorkerQueue()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.distribution_queue\n(\n `database` String,\n `table` String,\n `data_path` String,\n `is_blocked` UInt8,\n `error_count` UInt64,\n `data_files` UInt64,\n `data_compressed_bytes` UInt64,\n `broken_data_files` UInt64,\n `broken_data_compressed_bytes` UInt64,\n `last_exception` String\n)\nENGINE = SystemDistributionQueue()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.enabled_roles\n(\n `role_name` String,\n `with_admin_option` UInt8,\n `is_current` UInt8,\n `is_default` UInt8\n)\nENGINE = SystemEnabledRoles()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.errors\n(\n `name` String,\n `code` Int32,\n `value` UInt64,\n `last_error_time` DateTime,\n `last_error_message` String,\n `last_error_trace` Array(UInt64),\n `remote` UInt8\n)\nENGINE = SystemErrors()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.events\n(\n `event` String,\n `value` UInt64,\n `description` String\n)\nENGINE = SystemEvents()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.formats\n(\n `name` String,\n `is_input` UInt8,\n `is_output` UInt8\n)\nENGINE = SystemFormats()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.functions\n(\n `name` String,\n `is_aggregate` UInt8,\n `case_insensitive` UInt8,\n `alias_to` String,\n `create_query` String,\n `origin` Enum8(\'System\' = 0, \'SQLUserDefined\' = 1, \'ExecutableUserDefined\' = 2)\n)\nENGINE = SystemFunctions()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.grants\n(\n `user_name` Nullable(String),\n `role_name` Nullable(String),\n `access_type` Enum8(\'SQLITE\' = -128, \'ODBC\' = -127, \'JDBC\' = -126, \'HDFS\' = -125, \'S3\' = -124, \'SOURCES\' = -123, \'ALL\' = -122, \'NONE\' = -121, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM\' = 116, \'dictGet\' = 117, \'addressToLine\' = 118, \'addressToSymbol\' = 119, \'demangle\' = 120, \'INTROSPECTION\' = 121, \'FILE\' = 122, \'URL\' = 123, \'REMOTE\' = 124, \'MONGO\' = 125, \'MYSQL\' = 126, \'POSTGRES\' = 127),\n `database` Nullable(String),\n `table` Nullable(String),\n `column` Nullable(String),\n `is_partial_revoke` UInt8,\n `grant_option` UInt8\n)\nENGINE = SystemGrants()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.graphite_retentions\n(\n `config_name` String,\n `regexp` String,\n `function` String,\n `age` UInt64,\n `precision` UInt64,\n `priority` UInt16,\n `is_default` UInt8,\n `Tables.database` Array(String),\n `Tables.table` Array(String)\n)\nENGINE = SystemGraphite()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.licenses\n(\n `library_name` String,\n `license_type` String,\n `license_path` String,\n `license_text` String\n)\nENGINE = SystemLicenses()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.macros\n(\n `macro` String,\n `substitution` String\n)\nENGINE = SystemMacros()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.merge_tree_settings\n(\n `name` String,\n `value` String,\n `changed` UInt8,\n `description` String,\n `type` String\n)\nENGINE = SystemMergeTreeSettings()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.merges\n(\n `database` String,\n `table` String,\n `elapsed` Float64,\n `progress` Float64,\n `num_parts` UInt64,\n `source_part_names` Array(String),\n `result_part_name` String,\n `source_part_paths` Array(String),\n `result_part_path` String,\n `partition_id` String,\n `is_mutation` UInt8,\n `total_size_bytes_compressed` UInt64,\n `total_size_marks` UInt64,\n `bytes_read_uncompressed` UInt64,\n `rows_read` UInt64,\n `bytes_written_uncompressed` UInt64,\n `rows_written` UInt64,\n `columns_written` UInt64,\n `memory_usage` UInt64,\n `thread_id` UInt64,\n `merge_type` String,\n `merge_algorithm` String\n)\nENGINE = SystemMerges()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.metrics\n(\n `metric` String,\n `value` Int64,\n `description` String\n)\nENGINE = SystemMetrics()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.models\n(\n `name` String,\n `status` Enum8(\'NOT_LOADED\' = 0, \'LOADED\' = 1, \'FAILED\' = 2, \'LOADING\' = 3, \'FAILED_AND_RELOADING\' = 4, \'LOADED_AND_RELOADING\' = 5, \'NOT_EXIST\' = 6),\n `origin` String,\n `type` String,\n `loading_start_time` DateTime,\n `loading_duration` Float32,\n `last_exception` String\n)\nENGINE = SystemModels()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.mutations\n(\n `database` String,\n `table` String,\n `mutation_id` String,\n `command` String,\n `create_time` DateTime,\n `block_numbers.partition_id` Array(String),\n `block_numbers.number` Array(Int64),\n `parts_to_do_names` Array(String),\n `parts_to_do` Int64,\n `is_done` UInt8,\n `latest_failed_part` String,\n `latest_fail_time` DateTime,\n `latest_fail_reason` String\n)\nENGINE = SystemMutations()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.numbers\n(\n `number` UInt64\n)\nENGINE = SystemNumbers()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.numbers_mt\n(\n `number` UInt64\n)\nENGINE = SystemNumbers()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.one\n(\n `dummy` UInt8\n)\nENGINE = SystemOne()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.part_moves_between_shards\n(\n `database` String,\n `table` String,\n `task_name` String,\n `task_uuid` UUID,\n `create_time` DateTime,\n `part_name` String,\n `part_uuid` UUID,\n `to_shard` String,\n `dst_part_name` String,\n `update_time` DateTime,\n `state` String,\n `rollback` UInt8,\n `num_tries` UInt32,\n `last_exception` String\n)\nENGINE = SystemShardMoves()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.parts\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `secondary_indices_compressed_bytes` UInt64,\n `secondary_indices_uncompressed_bytes` UInt64,\n `secondary_indices_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `is_frozen` UInt8,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `hash_of_all_files` String,\n `hash_of_uncompressed_files` String,\n `uncompressed_hash_of_compressed_files` String,\n `delete_ttl_info_min` DateTime,\n `delete_ttl_info_max` DateTime,\n `move_ttl_info.expression` Array(String),\n `move_ttl_info.min` Array(DateTime),\n `move_ttl_info.max` Array(DateTime),\n `default_compression_codec` String,\n `recompression_ttl_info.expression` Array(String),\n `recompression_ttl_info.min` Array(DateTime),\n `recompression_ttl_info.max` Array(DateTime),\n `group_by_ttl_info.expression` Array(String),\n `group_by_ttl_info.min` Array(DateTime),\n `group_by_ttl_info.max` Array(DateTime),\n `rows_where_ttl_info.expression` Array(String),\n `rows_where_ttl_info.min` Array(DateTime),\n `rows_where_ttl_info.max` Array(DateTime),\n `projections` Array(String),\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.parts_columns\n(\n `partition` String,\n `name` String,\n `uuid` UUID,\n `part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `column` String,\n `type` String,\n `column_position` UInt64,\n `default_kind` String,\n `default_expression` String,\n `column_bytes_on_disk` UInt64,\n `column_data_compressed_bytes` UInt64,\n `column_data_uncompressed_bytes` UInt64,\n `column_marks_bytes` UInt64,\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemPartsColumns()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.privileges\n(\n `privilege` Enum8(\'SQLITE\' = -128, \'ODBC\' = -127, \'JDBC\' = -126, \'HDFS\' = -125, \'S3\' = -124, \'SOURCES\' = -123, \'ALL\' = -122, \'NONE\' = -121, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM\' = 116, \'dictGet\' = 117, \'addressToLine\' = 118, \'addressToSymbol\' = 119, \'demangle\' = 120, \'INTROSPECTION\' = 121, \'FILE\' = 122, \'URL\' = 123, \'REMOTE\' = 124, \'MONGO\' = 125, \'MYSQL\' = 126, \'POSTGRES\' = 127),\n `aliases` Array(String),\n `level` Nullable(Enum8(\'GLOBAL\' = 0, \'DATABASE\' = 1, \'TABLE\' = 2, \'DICTIONARY\' = 3, \'VIEW\' = 4, \'COLUMN\' = 5)),\n `parent_group` Nullable(Enum8(\'SQLITE\' = -128, \'ODBC\' = -127, \'JDBC\' = -126, \'HDFS\' = -125, \'S3\' = -124, \'SOURCES\' = -123, \'ALL\' = -122, \'NONE\' = -121, \'SHOW DATABASES\' = 0, \'SHOW TABLES\' = 1, \'SHOW COLUMNS\' = 2, \'SHOW DICTIONARIES\' = 3, \'SHOW\' = 4, \'SELECT\' = 5, \'INSERT\' = 6, \'ALTER UPDATE\' = 7, \'ALTER DELETE\' = 8, \'ALTER ADD COLUMN\' = 9, \'ALTER MODIFY COLUMN\' = 10, \'ALTER DROP COLUMN\' = 11, \'ALTER COMMENT COLUMN\' = 12, \'ALTER CLEAR COLUMN\' = 13, \'ALTER RENAME COLUMN\' = 14, \'ALTER MATERIALIZE COLUMN\' = 15, \'ALTER COLUMN\' = 16, \'ALTER MODIFY COMMENT\' = 17, \'ALTER ORDER BY\' = 18, \'ALTER SAMPLE BY\' = 19, \'ALTER ADD INDEX\' = 20, \'ALTER DROP INDEX\' = 21, \'ALTER MATERIALIZE INDEX\' = 22, \'ALTER CLEAR INDEX\' = 23, \'ALTER INDEX\' = 24, \'ALTER ADD PROJECTION\' = 25, \'ALTER DROP PROJECTION\' = 26, \'ALTER MATERIALIZE PROJECTION\' = 27, \'ALTER CLEAR PROJECTION\' = 28, \'ALTER PROJECTION\' = 29, \'ALTER ADD CONSTRAINT\' = 30, \'ALTER DROP CONSTRAINT\' = 31, \'ALTER CONSTRAINT\' = 32, \'ALTER TTL\' = 33, \'ALTER MATERIALIZE TTL\' = 34, \'ALTER SETTINGS\' = 35, \'ALTER MOVE PARTITION\' = 36, \'ALTER FETCH PARTITION\' = 37, \'ALTER FREEZE PARTITION\' = 38, \'ALTER DATABASE SETTINGS\' = 39, \'ALTER TABLE\' = 40, \'ALTER DATABASE\' = 41, \'ALTER VIEW REFRESH\' = 42, \'ALTER VIEW MODIFY QUERY\' = 43, \'ALTER VIEW\' = 44, \'ALTER\' = 45, \'CREATE DATABASE\' = 46, \'CREATE TABLE\' = 47, \'CREATE VIEW\' = 48, \'CREATE DICTIONARY\' = 49, \'CREATE TEMPORARY TABLE\' = 50, \'CREATE FUNCTION\' = 51, \'CREATE\' = 52, \'DROP DATABASE\' = 53, \'DROP TABLE\' = 54, \'DROP VIEW\' = 55, \'DROP DICTIONARY\' = 56, \'DROP FUNCTION\' = 57, \'DROP\' = 58, \'TRUNCATE\' = 59, \'OPTIMIZE\' = 60, \'KILL QUERY\' = 61, \'MOVE PARTITION BETWEEN SHARDS\' = 62, \'CREATE USER\' = 63, \'ALTER USER\' = 64, \'DROP USER\' = 65, \'CREATE ROLE\' = 66, \'ALTER ROLE\' = 67, \'DROP ROLE\' = 68, \'ROLE ADMIN\' = 69, \'CREATE ROW POLICY\' = 70, \'ALTER ROW POLICY\' = 71, \'DROP ROW POLICY\' = 72, \'CREATE QUOTA\' = 73, \'ALTER QUOTA\' = 74, \'DROP QUOTA\' = 75, \'CREATE SETTINGS PROFILE\' = 76, \'ALTER SETTINGS PROFILE\' = 77, \'DROP SETTINGS PROFILE\' = 78, \'SHOW USERS\' = 79, \'SHOW ROLES\' = 80, \'SHOW ROW POLICIES\' = 81, \'SHOW QUOTAS\' = 82, \'SHOW SETTINGS PROFILES\' = 83, \'SHOW ACCESS\' = 84, \'ACCESS MANAGEMENT\' = 85, \'SYSTEM SHUTDOWN\' = 86, \'SYSTEM DROP DNS CACHE\' = 87, \'SYSTEM DROP MARK CACHE\' = 88, \'SYSTEM DROP UNCOMPRESSED CACHE\' = 89, \'SYSTEM DROP MMAP CACHE\' = 90, \'SYSTEM DROP COMPILED EXPRESSION CACHE\' = 91, \'SYSTEM DROP CACHE\' = 92, \'SYSTEM RELOAD CONFIG\' = 93, \'SYSTEM RELOAD SYMBOLS\' = 94, \'SYSTEM RELOAD DICTIONARY\' = 95, \'SYSTEM RELOAD MODEL\' = 96, \'SYSTEM RELOAD FUNCTION\' = 97, \'SYSTEM RELOAD EMBEDDED DICTIONARIES\' = 98, \'SYSTEM RELOAD\' = 99, \'SYSTEM RESTART DISK\' = 100, \'SYSTEM MERGES\' = 101, \'SYSTEM TTL MERGES\' = 102, \'SYSTEM FETCHES\' = 103, \'SYSTEM MOVES\' = 104, \'SYSTEM DISTRIBUTED SENDS\' = 105, \'SYSTEM REPLICATED SENDS\' = 106, \'SYSTEM SENDS\' = 107, \'SYSTEM REPLICATION QUEUES\' = 108, \'SYSTEM DROP REPLICA\' = 109, \'SYSTEM SYNC REPLICA\' = 110, \'SYSTEM RESTART REPLICA\' = 111, \'SYSTEM RESTORE REPLICA\' = 112, \'SYSTEM FLUSH DISTRIBUTED\' = 113, \'SYSTEM FLUSH LOGS\' = 114, \'SYSTEM FLUSH\' = 115, \'SYSTEM\' = 116, \'dictGet\' = 117, \'addressToLine\' = 118, \'addressToSymbol\' = 119, \'demangle\' = 120, \'INTROSPECTION\' = 121, \'FILE\' = 122, \'URL\' = 123, \'REMOTE\' = 124, \'MONGO\' = 125, \'MYSQL\' = 126, \'POSTGRES\' = 127))\n)\nENGINE = SystemPrivileges()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.processes\n(\n `is_initial_query` UInt8,\n `user` String,\n `query_id` String,\n `address` IPv6,\n `port` UInt16,\n `initial_user` String,\n `initial_query_id` String,\n `initial_address` IPv6,\n `initial_port` UInt16,\n `interface` UInt8,\n `os_user` String,\n `client_hostname` String,\n `client_name` String,\n `client_revision` UInt64,\n `client_version_major` UInt64,\n `client_version_minor` UInt64,\n `client_version_patch` UInt64,\n `http_method` UInt8,\n `http_user_agent` String,\n `http_referer` String,\n `forwarded_for` String,\n `quota_key` String,\n `elapsed` Float64,\n `is_cancelled` UInt8,\n `read_rows` UInt64,\n `read_bytes` UInt64,\n `total_rows_approx` UInt64,\n `written_rows` UInt64,\n `written_bytes` UInt64,\n `memory_usage` Int64,\n `peak_memory_usage` Int64,\n `query` String,\n `thread_ids` Array(UInt64),\n `ProfileEvents` Map(String, UInt64),\n `Settings` Map(String, String),\n `current_database` String,\n `ProfileEvents.Names` Array(String),\n `ProfileEvents.Values` Array(UInt64),\n `Settings.Names` Array(String),\n `Settings.Values` Array(String)\n)\nENGINE = SystemProcesses()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.projection_parts\n(\n `partition` String,\n `name` String,\n `part_type` String,\n `parent_name` String,\n `parent_uuid` UUID,\n `parent_part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `parent_marks` UInt64,\n `parent_rows` UInt64,\n `parent_bytes_on_disk` UInt64,\n `parent_data_compressed_bytes` UInt64,\n `parent_data_uncompressed_bytes` UInt64,\n `parent_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `is_frozen` UInt8,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `hash_of_all_files` String,\n `hash_of_uncompressed_files` String,\n `uncompressed_hash_of_compressed_files` String,\n `delete_ttl_info_min` DateTime,\n `delete_ttl_info_max` DateTime,\n `move_ttl_info.expression` Array(String),\n `move_ttl_info.min` Array(DateTime),\n `move_ttl_info.max` Array(DateTime),\n `default_compression_codec` String,\n `recompression_ttl_info.expression` Array(String),\n `recompression_ttl_info.min` Array(DateTime),\n `recompression_ttl_info.max` Array(DateTime),\n `group_by_ttl_info.expression` Array(String),\n `group_by_ttl_info.min` Array(DateTime),\n `group_by_ttl_info.max` Array(DateTime),\n `rows_where_ttl_info.expression` Array(String),\n `rows_where_ttl_info.min` Array(DateTime),\n `rows_where_ttl_info.max` Array(DateTime),\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemProjectionParts()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.projection_parts_columns\n(\n `partition` String,\n `name` String,\n `part_type` String,\n `parent_name` String,\n `parent_uuid` UUID,\n `parent_part_type` String,\n `active` UInt8,\n `marks` UInt64,\n `rows` UInt64,\n `bytes_on_disk` UInt64,\n `data_compressed_bytes` UInt64,\n `data_uncompressed_bytes` UInt64,\n `marks_bytes` UInt64,\n `parent_marks` UInt64,\n `parent_rows` UInt64,\n `parent_bytes_on_disk` UInt64,\n `parent_data_compressed_bytes` UInt64,\n `parent_data_uncompressed_bytes` UInt64,\n `parent_marks_bytes` UInt64,\n `modification_time` DateTime,\n `remove_time` DateTime,\n `refcount` UInt32,\n `min_date` Date,\n `max_date` Date,\n `min_time` DateTime,\n `max_time` DateTime,\n `partition_id` String,\n `min_block_number` Int64,\n `max_block_number` Int64,\n `level` UInt32,\n `data_version` UInt64,\n `primary_key_bytes_in_memory` UInt64,\n `primary_key_bytes_in_memory_allocated` UInt64,\n `database` String,\n `table` String,\n `engine` String,\n `disk_name` String,\n `path` String,\n `column` String,\n `type` String,\n `column_position` UInt64,\n `default_kind` String,\n `default_expression` String,\n `column_bytes_on_disk` UInt64,\n `column_data_compressed_bytes` UInt64,\n `column_data_uncompressed_bytes` UInt64,\n `column_marks_bytes` UInt64,\n `bytes` UInt64,\n `marks_size` UInt64\n)\nENGINE = SystemProjectionPartsColumns()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.quota_limits\n(\n `quota_name` String,\n `duration` UInt32,\n `is_randomized_interval` UInt8,\n `max_queries` Nullable(UInt64),\n `max_query_selects` Nullable(UInt64),\n `max_query_inserts` Nullable(UInt64),\n `max_errors` Nullable(UInt64),\n `max_result_rows` Nullable(UInt64),\n `max_result_bytes` Nullable(UInt64),\n `max_read_rows` Nullable(UInt64),\n `max_read_bytes` Nullable(UInt64),\n `max_execution_time` Nullable(Float64)\n)\nENGINE = SystemQuotaLimits()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.quota_usage\n(\n `quota_name` String,\n `quota_key` String,\n `start_time` Nullable(DateTime),\n `end_time` Nullable(DateTime),\n `duration` Nullable(UInt32),\n `queries` Nullable(UInt64),\n `max_queries` Nullable(UInt64),\n `query_selects` Nullable(UInt64),\n `max_query_selects` Nullable(UInt64),\n `query_inserts` Nullable(UInt64),\n `max_query_inserts` Nullable(UInt64),\n `errors` Nullable(UInt64),\n `max_errors` Nullable(UInt64),\n `result_rows` Nullable(UInt64),\n `max_result_rows` Nullable(UInt64),\n `result_bytes` Nullable(UInt64),\n `max_result_bytes` Nullable(UInt64),\n `read_rows` Nullable(UInt64),\n `max_read_rows` Nullable(UInt64),\n `read_bytes` Nullable(UInt64),\n `max_read_bytes` Nullable(UInt64),\n `execution_time` Nullable(Float64),\n `max_execution_time` Nullable(Float64)\n)\nENGINE = SystemQuotaUsage()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.quotas\n(\n `name` String,\n `id` UUID,\n `storage` String,\n `keys` Array(Enum8(\'user_name\' = 1, \'ip_address\' = 2, \'forwarded_ip_address\' = 3, \'client_key\' = 4)),\n `durations` Array(UInt32),\n `apply_to_all` UInt8,\n `apply_to_list` Array(String),\n `apply_to_except` Array(String)\n)\nENGINE = SystemQuotas()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.quotas_usage\n(\n `quota_name` String,\n `quota_key` String,\n `is_current` UInt8,\n `start_time` Nullable(DateTime),\n `end_time` Nullable(DateTime),\n `duration` Nullable(UInt32),\n `queries` Nullable(UInt64),\n `max_queries` Nullable(UInt64),\n `query_selects` Nullable(UInt64),\n `max_query_selects` Nullable(UInt64),\n `query_inserts` Nullable(UInt64),\n `max_query_inserts` Nullable(UInt64),\n `errors` Nullable(UInt64),\n `max_errors` Nullable(UInt64),\n `result_rows` Nullable(UInt64),\n `max_result_rows` Nullable(UInt64),\n `result_bytes` Nullable(UInt64),\n `max_result_bytes` Nullable(UInt64),\n `read_rows` Nullable(UInt64),\n `max_read_rows` Nullable(UInt64),\n `read_bytes` Nullable(UInt64),\n `max_read_bytes` Nullable(UInt64),\n `execution_time` Nullable(Float64),\n `max_execution_time` Nullable(Float64)\n)\nENGINE = SystemQuotasUsage()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.replicas\n(\n `database` String,\n `table` String,\n `engine` String,\n `is_leader` UInt8,\n `can_become_leader` UInt8,\n `is_readonly` UInt8,\n `is_session_expired` UInt8,\n `future_parts` UInt32,\n `parts_to_check` UInt32,\n `zookeeper_path` String,\n `replica_name` String,\n `replica_path` String,\n `columns_version` Int32,\n `queue_size` UInt32,\n `inserts_in_queue` UInt32,\n `merges_in_queue` UInt32,\n `part_mutations_in_queue` UInt32,\n `queue_oldest_time` DateTime,\n `inserts_oldest_time` DateTime,\n `merges_oldest_time` DateTime,\n `part_mutations_oldest_time` DateTime,\n `oldest_part_to_get` String,\n `oldest_part_to_merge_to` String,\n `oldest_part_to_mutate_to` String,\n `log_max_index` UInt64,\n `log_pointer` UInt64,\n `last_queue_update` DateTime,\n `absolute_delay` UInt64,\n `total_replicas` UInt8,\n `active_replicas` UInt8,\n `last_queue_update_exception` String,\n `zookeeper_exception` String,\n `replica_is_active` Map(String, UInt8)\n)\nENGINE = SystemReplicas()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.replicated_fetches\n(\n `database` String,\n `table` String,\n `elapsed` Float64,\n `progress` Float64,\n `result_part_name` String,\n `result_part_path` String,\n `partition_id` String,\n `total_size_bytes_compressed` UInt64,\n `bytes_read_compressed` UInt64,\n `source_replica_path` String,\n `source_replica_hostname` String,\n `source_replica_port` UInt16,\n `interserver_scheme` String,\n `URI` String,\n `to_detached` UInt8,\n `thread_id` UInt64\n)\nENGINE = SystemReplicatedFetches()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.replicated_merge_tree_settings\n(\n `name` String,\n `value` String,\n `changed` UInt8,\n `description` String,\n `type` String\n)\nENGINE = SystemReplicatedMergeTreeSettings()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.replication_queue\n(\n `database` String,\n `table` String,\n `replica_name` String,\n `position` UInt32,\n `node_name` String,\n `type` String,\n `create_time` DateTime,\n `required_quorum` UInt32,\n `source_replica` String,\n `new_part_name` String,\n `parts_to_merge` Array(String),\n `is_detach` UInt8,\n `is_currently_executing` UInt8,\n `num_tries` UInt32,\n `last_exception` String,\n `last_attempt_time` DateTime,\n `num_postponed` UInt32,\n `postpone_reason` String,\n `last_postpone_time` DateTime,\n `merge_type` String\n)\nENGINE = SystemReplicationQueue()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.role_grants\n(\n `user_name` Nullable(String),\n `role_name` Nullable(String),\n `granted_role_name` String,\n `granted_role_is_default` UInt8,\n `with_admin_option` UInt8\n)\nENGINE = SystemRoleGrants()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.roles\n(\n `name` String,\n `id` UUID,\n `storage` String\n)\nENGINE = SystemRoles()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.row_policies\n(\n `name` String,\n `short_name` String,\n `database` String,\n `table` String,\n `id` UUID,\n `storage` String,\n `select_filter` Nullable(String),\n `is_restrictive` UInt8,\n `apply_to_all` UInt8,\n `apply_to_list` Array(String),\n `apply_to_except` Array(String)\n)\nENGINE = SystemRowPolicies()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.settings\n(\n `name` String,\n `value` String,\n `changed` UInt8,\n `description` String,\n `min` Nullable(String),\n `max` Nullable(String),\n `readonly` UInt8,\n `type` String\n)\nENGINE = SystemSettings()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.settings_profile_elements\n(\n `profile_name` Nullable(String),\n `user_name` Nullable(String),\n `role_name` Nullable(String),\n `index` UInt64,\n `setting_name` Nullable(String),\n `value` Nullable(String),\n `min` Nullable(String),\n `max` Nullable(String),\n `readonly` Nullable(UInt8),\n `inherit_profile` Nullable(String)\n)\nENGINE = SystemSettingsProfileElements()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.settings_profiles\n(\n `name` String,\n `id` UUID,\n `storage` String,\n `num_elements` UInt64,\n `apply_to_all` UInt8,\n `apply_to_list` Array(String),\n `apply_to_except` Array(String)\n)\nENGINE = SystemSettingsProfiles()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.stack_trace\n(\n `thread_name` String,\n `thread_id` UInt64,\n `query_id` String,\n `trace` Array(UInt64)\n)\nENGINE = SystemStackTrace()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.storage_policies\n(\n `policy_name` String,\n `volume_name` String,\n `volume_priority` UInt64,\n `disks` Array(String),\n `volume_type` String,\n `max_data_part_size` UInt64,\n `move_factor` Float32,\n `prefer_not_to_merge` UInt8\n)\nENGINE = SystemStoragePolicies()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.table_engines\n(\n `name` String,\n `supports_settings` UInt8,\n `supports_skipping_indices` UInt8,\n `supports_projections` UInt8,\n `supports_sort_order` UInt8,\n `supports_ttl` UInt8,\n `supports_replication` UInt8,\n `supports_deduplication` UInt8,\n `supports_parallel_insert` UInt8\n)\nENGINE = SystemTableEngines()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.table_functions\n(\n `name` String\n)\nENGINE = SystemTableFunctions()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.tables\n(\n `database` String,\n `name` String,\n `uuid` UUID,\n `engine` String,\n `is_temporary` UInt8,\n `data_paths` Array(String),\n `metadata_path` String,\n `metadata_modification_time` DateTime,\n `dependencies_database` Array(String),\n `dependencies_table` Array(String),\n `create_table_query` String,\n `engine_full` String,\n `as_select` String,\n `partition_key` String,\n `sorting_key` String,\n `primary_key` String,\n `sampling_key` String,\n `storage_policy` String,\n `total_rows` Nullable(UInt64),\n `total_bytes` Nullable(UInt64),\n `lifetime_rows` Nullable(UInt64),\n `lifetime_bytes` Nullable(UInt64),\n `comment` String,\n `has_own_data` UInt8,\n `loading_dependencies_database` Array(String),\n `loading_dependencies_table` Array(String),\n `loading_dependent_database` Array(String),\n `loading_dependent_table` Array(String),\n `table` String\n)\nENGINE = SystemTables()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.time_zones\n(\n `time_zone` String\n)\nENGINE = SystemTimeZones()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.user_directories\n(\n `name` String,\n `type` String,\n `params` String,\n `precedence` UInt64\n)\nENGINE = SystemUserDirectories()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.users\n(\n `name` String,\n `id` UUID,\n `storage` String,\n `auth_type` Enum8(\'no_password\' = 0, \'plaintext_password\' = 1, \'sha256_password\' = 2, \'double_sha1_password\' = 3, \'ldap\' = 4, \'kerberos\' = 5),\n `auth_params` String,\n `host_ip` Array(String),\n `host_names` Array(String),\n `host_names_regexp` Array(String),\n `host_names_like` Array(String),\n `default_roles_all` UInt8,\n `default_roles_list` Array(String),\n `default_roles_except` Array(String),\n `grantees_any` UInt8,\n `grantees_list` Array(String),\n `grantees_except` Array(String),\n `default_database` String\n)\nENGINE = SystemUsers()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.warnings\n(\n `message` String\n)\nENGINE = SystemWarnings()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.zeros\n(\n `zero` UInt8\n)\nENGINE = SystemZeros()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' +CREATE TABLE system.zeros_mt\n(\n `zero` UInt8\n)\nENGINE = SystemZeros()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' diff --git a/tests/queries/0_stateless/02117_show_create_table_system.sql b/tests/queries/0_stateless/02117_show_create_table_system.sql new file mode 100644 index 00000000000..16861cc3f8e --- /dev/null +++ b/tests/queries/0_stateless/02117_show_create_table_system.sql @@ -0,0 +1,84 @@ +/* we will `use system` to bypass style check, +because `show create table` statement +cannot fit the requirement in check-sytle, which is as + +"# Queries to: +tables_with_database_column=( + system.tables + system.parts + system.detached_parts + system.parts_columns + system.columns + system.projection_parts + system.mutations +) +# should have database = currentDatabase() condition" + + */ +use system; +show create table aggregate_function_combinators; +show create table asynchronous_inserts; +show create table asynchronous_metrics; +show create table build_options; +show create table clusters; +show create table collations; +show create table columns; +show create table contributors; +show create table current_roles; +show create table data_skipping_indices; +show create table data_type_families; +show create table databases; +show create table detached_parts; +show create table dictionaries; +show create table disks; +show create table distributed_ddl_queue; +show create table distribution_queue; +show create table enabled_roles; +show create table errors; +show create table events; +show create table formats; +show create table functions; +show create table grants; +show create table graphite_retentions; +show create table licenses; +show create table macros; +show create table merge_tree_settings; +show create table merges; +show create table metrics; +show create table models; +show create table mutations; +show create table numbers; +show create table numbers_mt; +show create table one; +show create table part_moves_between_shards; +show create table parts; +show create table parts_columns; +show create table privileges; +show create table processes; +show create table projection_parts; +show create table projection_parts_columns; +show create table quota_limits; +show create table quota_usage; +show create table quotas; +show create table quotas_usage; +show create table replicas; +show create table replicated_fetches; +show create table replicated_merge_tree_settings; +show create table replication_queue; +show create table role_grants; +show create table roles; +show create table row_policies; +show create table settings; +show create table settings_profile_elements; +show create table settings_profiles; +show create table stack_trace; +show create table storage_policies; +show create table table_engines; +show create table table_functions; +show create table tables; +show create table time_zones; +show create table user_directories; +show create table users; +show create table warnings; +show create table zeros; +show create table zeros_mt; diff --git a/tests/queries/0_stateless/02118_show_create_table_rocksdb.reference b/tests/queries/0_stateless/02118_show_create_table_rocksdb.reference new file mode 100644 index 00000000000..9e487824e3e --- /dev/null +++ b/tests/queries/0_stateless/02118_show_create_table_rocksdb.reference @@ -0,0 +1 @@ +CREATE TABLE system.rocksdb\n(\n `database` String,\n `table` String,\n `name` String,\n `value` UInt64\n)\nENGINE = SystemRocksDB()\nCOMMENT \'SYSTEM TABLE is built on the fly.\' diff --git a/tests/queries/0_stateless/02118_show_create_table_rocksdb.sql b/tests/queries/0_stateless/02118_show_create_table_rocksdb.sql new file mode 100644 index 00000000000..98a64c4b756 --- /dev/null +++ b/tests/queries/0_stateless/02118_show_create_table_rocksdb.sql @@ -0,0 +1,3 @@ +-- Tags: no-fasttest +-- Tag no-fasttest: In fasttest, ENABLE_LIBRARIES=0, so rocksdb engine is not enabled by default +show create table system.rocksdb; From 7f25ac059854050d9ff3a1a3dc1212b75ab5f671 Mon Sep 17 00:00:00 2001 From: Andrey Torsunov Date: Fri, 19 Nov 2021 14:37:10 +0300 Subject: [PATCH 158/200] typo in PostgreSQL --- .../external-dictionaries/external-dicts-dict-sources.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md index f8189dcf8cf..095f059513c 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md @@ -746,7 +746,7 @@ Setting fields: !!! info "Note" The `column_family` or `where` fields cannot be used together with the `query` field. And either one of the `column_family` or `query` fields must be declared. -### PosgreSQL {#dicts-external_dicts_dict_sources-postgresql} +### PostgreSQL {#dicts-external_dicts_dict_sources-postgresql} Example of settings: From 315e793e6ce4a20f6823fd93ade0e8783a358ee5 Mon Sep 17 00:00:00 2001 From: Constantine Peresypkin Date: Fri, 19 Nov 2021 14:58:26 +0200 Subject: [PATCH 159/200] remove hardcoded CI values --- tests/ci/ast_fuzzer_check.py | 2 +- tests/ci/build_report_check.py | 8 ++++---- tests/ci/finish_check.py | 2 +- tests/ci/pr_info.py | 2 +- tests/ci/pvs_check.py | 3 ++- tests/ci/run_check.py | 2 +- tests/ci/upload_result_helper.py | 8 ++++---- 7 files changed, 14 insertions(+), 13 deletions(-) diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index f54c096705a..2aa6cacd1e0 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -102,7 +102,7 @@ if __name__ == "__main__": logging.info("Exception uploading file %s text %s", f, ex) paths[f] = '' - report_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + report_url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/actions/runs/{os.getenv('GITHUB_RUN_ID')}" if paths['runlog.log']: report_url = paths['runlog.log'] if paths['main.log']: diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 19f2d89aab6..402db7c2740 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -110,13 +110,13 @@ if __name__ == "__main__": pr_info = PRInfo(event) - branch_url = "https://github.com/ClickHouse/ClickHouse/commits/master" + branch_url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/commits/master" branch_name = "master" if pr_info.number != 0: branch_name = "PR #{}".format(pr_info.number) - branch_url = "https://github.com/ClickHouse/ClickHouse/pull/" + str(pr_info.number) - commit_url = f"https://github.com/ClickHouse/ClickHouse/commit/{pr_info.sha}" - task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID', '0')}" + branch_url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/pull/{pr_info.number}" + commit_url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/commit/{pr_info.sha}" + task_url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/actions/runs/{os.getenv('GITHUB_RUN_ID', '0')}" report = create_build_html_report( build_check_name, build_results, diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index b0c016b423c..c38b3c09448 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -33,7 +33,7 @@ if __name__ == "__main__": gh = Github(get_best_robot_token()) commit = get_commit(gh, pr_info.sha) - url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/actions/runs/{os.getenv('GITHUB_RUN_ID')}" statuses = filter_statuses(list(commit.get_statuses())) if NAME in statuses and statuses[NAME].state == "pending": commit.create_status(context=NAME, description="All checks finished", state="success", target_url=url) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 46499514027..5d5b77df7e1 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -38,7 +38,7 @@ class PRInfo: self.labels = {} if need_changed_files: commit_before = github_event['before'] - response = requests.get(f'https://api.github.com/repos/ClickHouse/ClickHouse/compare/{commit_before}...{self.sha}') + response = requests.get(f"{os.getenv('GITHUB_SERVER_URL')}/repos/{os.getenv('GITHUB_REPOSITORY')}/compare/{commit_before}...{self.sha}") response.raise_for_status() diff = response.json() diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 592f568b719..9d0e1880621 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -82,7 +82,8 @@ if __name__ == "__main__": break if not index_html: - commit.create_status(context=NAME, description='PVS report failed to build', state='failure', target_url=f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}") + commit.create_status(context=NAME, description='PVS report failed to build', state='failure', + target_url=f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/actions/runs/{os.getenv('GITHUB_RUN_ID')}") sys.exit(1) txt_report = os.path.join(temp_path, TXT_REPORT_NAME) diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 49cc809dd9e..99a99ad3063 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -112,7 +112,7 @@ if __name__ == "__main__": can_run, description = should_run_checks_for_pr(pr_info) gh = Github(get_best_robot_token()) commit = get_commit(gh, pr_info.sha) - url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/actions/runs/{os.getenv('GITHUB_RUN_ID')}" if not can_run: print("::notice ::Cannot run") commit.create_status(context=NAME, description=description, state="failure", target_url=url) diff --git a/tests/ci/upload_result_helper.py b/tests/ci/upload_result_helper.py index 5c1f1989c5d..d0705372c44 100644 --- a/tests/ci/upload_result_helper.py +++ b/tests/ci/upload_result_helper.py @@ -40,14 +40,14 @@ def upload_results(s3_client, pr_number, commit_sha, test_results, additional_fi s3_path_prefix = f"{pr_number}/{commit_sha}/" + check_name.lower().replace(' ', '_').replace('(', '_').replace(')', '_').replace(',', '_') additional_urls = process_logs(s3_client, additional_files, s3_path_prefix, test_results, with_raw_logs) - branch_url = "https://github.com/ClickHouse/ClickHouse/commits/master" + branch_url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/commits/master" branch_name = "master" if pr_number != 0: branch_name = f"PR #{pr_number}" - branch_url = f"https://github.com/ClickHouse/ClickHouse/pull/{pr_number}" - commit_url = f"https://github.com/ClickHouse/ClickHouse/commit/{commit_sha}" + branch_url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/pull/{pr_number}" + commit_url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/commit/{commit_sha}" - task_url = f"https://github.com/ClickHouse/ClickHouse/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + task_url = f"{os.getenv('GITHUB_SERVER_URL')}/{os.getenv('GITHUB_REPOSITORY')}/actions/runs/{os.getenv('GITHUB_RUN_ID')}" if additional_urls: raw_log_url = additional_urls[0] From cded91b0133ececac9bc481d61f9e3dfc0f5a07b Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 19 Nov 2021 16:51:49 +0300 Subject: [PATCH 160/200] Update verbosePrintString.h --- src/Formats/verbosePrintString.h | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Formats/verbosePrintString.h b/src/Formats/verbosePrintString.h index 02034cb8e97..26bd663d559 100644 --- a/src/Formats/verbosePrintString.h +++ b/src/Formats/verbosePrintString.h @@ -5,7 +5,6 @@ namespace DB { class WriteBuffer; -class ReadBuffer; /** Print string in double quotes and with control characters in "" form - for output diagnostic info to user. From 3070bf1e4dc32117b0695488ac53c6ae56905352 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 19 Nov 2021 16:52:31 +0300 Subject: [PATCH 161/200] Update CustomSeparatedRowOutputFormat.cpp --- src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp index 4c6b980a7c4..5c9664f2daf 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.cpp @@ -29,7 +29,7 @@ void CustomSeparatedRowOutputFormat::writeLine(const std::vector & value writeRowEndDelimiter(); } -void CustomSeparatedRowOutputFormat::doWritePrefix() +void CustomSeparatedRowOutputFormat::writePrefix() { writeString(format_settings.custom.result_before_delimiter, out); From fdd1f53d3a829331ce1fd94a914b934f76a25150 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Fri, 19 Nov 2021 16:52:48 +0300 Subject: [PATCH 162/200] Update CustomSeparatedRowOutputFormat.h --- src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h index 3ee5389d48c..274df1af330 100644 --- a/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h +++ b/src/Processors/Formats/Impl/CustomSeparatedRowOutputFormat.h @@ -23,7 +23,7 @@ private: void writeRowStartDelimiter() override; void writeRowEndDelimiter() override; void writeRowBetweenDelimiter() override; - void doWritePrefix() override; + void writePrefix() override; void writeSuffix() override; void writeLine(const std::vector & values); From 099c40d2513cfddeb806a6ac9951de17902d4e45 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 19 Nov 2021 17:14:56 +0300 Subject: [PATCH 163/200] fix tests --- src/Interpreters/ComparisonGraph.h | 2 +- .../SubstituteColumnOptimizer.cpp | 13 +++++----- src/Interpreters/TreeCNFConverter.h | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 26 +++++++++---------- .../MergeTree/MergeTreeIndexHypothesis.cpp | 1 - 5 files changed, 21 insertions(+), 23 deletions(-) diff --git a/src/Interpreters/ComparisonGraph.h b/src/Interpreters/ComparisonGraph.h index b3858081910..60cc97e68a3 100644 --- a/src/Interpreters/ComparisonGraph.h +++ b/src/Interpreters/ComparisonGraph.h @@ -116,7 +116,7 @@ private: std::vector> edges; }; - /// Recieves graph, in which each vertex corresponds to one expression. + /// Receives graph, in which each vertex corresponds to one expression. /// Then finds strongly connected components and builds graph on them. static Graph buildGraphFromAstsGraph(const Graph & asts_graph); diff --git a/src/Interpreters/SubstituteColumnOptimizer.cpp b/src/Interpreters/SubstituteColumnOptimizer.cpp index b7f5de4c558..da738d3db1e 100644 --- a/src/Interpreters/SubstituteColumnOptimizer.cpp +++ b/src/Interpreters/SubstituteColumnOptimizer.cpp @@ -90,12 +90,6 @@ struct ColumnPrice return std::tie(compressed_size, uncompressed_size) < std::tie(that.compressed_size, that.uncompressed_size); } - ColumnPrice operator+(ColumnPrice that) const - { - that += *this; - return that; - } - ColumnPrice & operator+=(const ColumnPrice & that) { compressed_size += that.compressed_size; @@ -156,7 +150,12 @@ ColumnPrice calculatePrice( { ColumnPrice result(0, 0); for (const auto & ident : identifiers) - result = result + column_prices.at(ident); + { + auto it = column_prices.find(ident); + if (it != column_prices.end()) + result += it->second; + } + return result; } diff --git a/src/Interpreters/TreeCNFConverter.h b/src/Interpreters/TreeCNFConverter.h index 52f997d83c9..22ec2969096 100644 --- a/src/Interpreters/TreeCNFConverter.h +++ b/src/Interpreters/TreeCNFConverter.h @@ -151,7 +151,7 @@ public: /// @max_growth_multipler means that it's allowed to grow size of formula only /// in that amount of times. It's needed to avoid exponential explosion of formula. /// CNF of boolean formula with N clauses can have 2^N clauses. - /// If amout of atomic formulas will be exceded nullopt will be returned. + /// If amount of atomic formulas will be exceeded nullopt will be returned. /// 0 - means unlimited. static std::optional tryConvertToCNF( const ASTPtr & query, size_t max_growth_multipler = DEFAULT_MAX_GROWTH_MULTIPLIER); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 64048824574..899c1734dee 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1061,21 +1061,21 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd } for (const auto & [granularity, index_and_condition] : merged_indices) - { - const auto & index_name = "Merged"; - LOG_DEBUG(log, "Index {} has dropped {}/{} granules.", - backQuote(index_name), - index_and_condition->granules_dropped, index_and_condition->total_granules); + { + const auto & index_name = "Merged"; + LOG_DEBUG(log, "Index {} has dropped {}/{} granules.", + backQuote(index_name), + index_and_condition->granules_dropped, index_and_condition->total_granules); - std::string description = "MERGED GRANULARITY " + std::to_string(granularity); + std::string description = "MERGED GRANULARITY " + std::to_string(granularity); - index_stats.emplace_back(ReadFromMergeTree::IndexStat{ - .type = ReadFromMergeTree::IndexType::Skip, - .name = index_name, - .description = std::move(description), - .num_parts_after = index_and_condition->total_parts - index_and_condition->parts_dropped, - .num_granules_after = index_and_condition->total_granules - index_and_condition->granules_dropped}); - } + index_stats.emplace_back(ReadFromMergeTree::IndexStat{ + .type = ReadFromMergeTree::IndexType::Skip, + .name = index_name, + .description = std::move(description), //-V1030 + .num_parts_after = index_and_condition->total_parts - index_and_condition->parts_dropped, + .num_granules_after = index_and_condition->total_granules - index_and_condition->granules_dropped}); + } return parts_with_ranges; } diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp index cc574f02737..26f534538fc 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp @@ -104,5 +104,4 @@ void hypothesisIndexValidator(const IndexDescription & index, bool /*attach*/) throw Exception("Hypothesis index needs exactly one expression", ErrorCodes::LOGICAL_ERROR); } - } From 308cc722f32d5f78412d80a2b38497f1adcdd55f Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Nov 2021 17:47:04 +0300 Subject: [PATCH 164/200] Push tests results to CI database --- tests/ci/ast_fuzzer_check.py | 14 +++ tests/ci/clickhouse_helper.py | 163 ++++++++++++++++++++++++++++ tests/ci/compatibility_check.py | 12 ++ tests/ci/docker_images_check.py | 9 ++ tests/ci/docs_check.py | 8 ++ tests/ci/fast_test_check.py | 11 ++ tests/ci/functional_test_check.py | 12 ++ tests/ci/integration_test_check.py | 12 ++ tests/ci/pr_info.py | 23 ++++ tests/ci/pvs_check.py | 9 ++ tests/ci/split_build_smoke_check.py | 10 ++ tests/ci/stopwatch.py | 16 +++ tests/ci/stress_check.py | 12 ++ tests/ci/style_check.py | 12 ++ tests/ci/unit_tests_check.py | 13 +++ 15 files changed, 336 insertions(+) create mode 100644 tests/ci/clickhouse_helper.py create mode 100644 tests/ci/stopwatch.py diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index f54c096705a..0cdce1bc7fd 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -15,6 +15,8 @@ from ci_config import build_config_to_string from build_download_helper import get_build_config_for_check, get_build_urls from docker_pull_helper import get_image_with_version from commit_status_helper import post_commit_status +from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse +from stopwatch import Stopwatch IMAGE_NAME = 'clickhouse/fuzzer' @@ -31,6 +33,9 @@ def get_commit(gh, commit_sha): if __name__ == "__main__": logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) repo_path = os.getenv("REPO_COPY", os.path.abspath("../../")) reports_path = os.getenv("REPORTS_PATH", "./reports") @@ -125,6 +130,15 @@ if __name__ == "__main__": status = 'failure' description = 'Task failed: $?=' + str(retcode) + if 'fail' in status: + test_result = [(description, 'FAIL')] + else: + test_result = [(description, 'OK')] + + ch_helper = ClickHouseHelper() + + prepared_events = prepare_tests_results_for_clickhouse(pr_info, test_result, status, stopwatch.duration_seconds, stopwatch.start_time_str, report_url, check_name) + logging.info("Result: '%s', '%s', '%s'", status, description, report_url) print(f"::notice ::Report url: {report_url}") post_commit_status(gh, pr_info.sha, check_name, description, status, report_url) diff --git a/tests/ci/clickhouse_helper.py b/tests/ci/clickhouse_helper.py new file mode 100644 index 00000000000..0b9df6cb868 --- /dev/null +++ b/tests/ci/clickhouse_helper.py @@ -0,0 +1,163 @@ +#!/usr/bin/env python3 +import time +import logging +import json + +import requests +from get_robot_token import get_parameter_from_ssm + +class ClickHouseHelper: + def __init__(self, url=None, user=None, password=None): + if url is None: + url = get_parameter_from_ssm("clickhouse-test-stat-url") + + self.url = url + self.auth = { + 'X-ClickHouse-User': user if user is not None else get_parameter_from_ssm("clickhouse-test-stat-login"), + 'X-ClickHouse-Key': password if password is not None else get_parameter_from_ssm("clickhouse-test-stat-password") + } + + def _insert_json_str_info(self, db, table, json_str): + params = { + 'database': db, + 'query': 'INSERT INTO {table} FORMAT JSONEachRow'.format(table=table), + 'date_time_input_format': 'best_effort', + 'send_logs_level': 'warning', + } + + for i in range(5): + response = requests.post(self.url, params=params, data=json_str, headers=self.auth, verify=False) + + logging.info("Response content '%s'", response.content) + + if response.ok: + break + + error = ( + "Cannot insert data into clickhouse at try " + str(i) + + ": HTTP code " + str(response.status_code) + ": '" + + str(response.text) + "'") + + if response.status_code >= 500: + # A retriable error + time.sleep(1) + continue + + logging.info("Request headers '%s', body '%s'", response.request.headers, response.request.body) + + raise Exception(error) + else: + raise Exception(error) + + def insert_event_into(self, db, table, event): + event_str = json.dumps(event) + self._insert_json_str_info(db, table, event_str) + + def insert_events_into(self, db, table, events): + jsons = [] + for event in events: + jsons.append(json.dumps(event)) + + self._insert_json_str_info(db, table, ','.join(jsons)) + + def _select_and_get_json_each_row(self, db, query): + params = { + 'database': db, + 'query': query, + 'default_format': 'JSONEachRow', + } + for i in range(5): + response = None + try: + response = requests.get(self.url, params=params, headers=self.auth, verify=False) + response.raise_for_status() + return response.text + except Exception as ex: + logging.warning("Cannot insert with exception %s", str(ex)) + if response: + logging.warning("Reponse text %s", response.text) + time.sleep(0.1 * i) + + raise Exception("Cannot insert data into clickhouse") + + def select_json_each_row(self, db, query): + text = self._select_and_get_json_each_row(db, query) + result = [] + for line in text.split('\n'): + if line: + result.append(json.loads(line)) + return result + +def prepare_tests_results_for_clickhouse( + pr_info, test_results, + check_status, check_duration, check_start_time, + report_url, check_name): + + pull_request_url = "https://github.com/ClickHouse/ClickHouse/commits/master" + base_ref = "master" + head_ref = "master" + base_repo = pr_info.repo_full_name + head_repo = pr_info.repo_full_name + if pr_info.number != 0: + pull_request_url = pr_info.pr_html_url + base_ref = pr_info.base_ref + base_repo = pr_info.base_name + head_ref = pr_info.head_ref + head_repo = pr_info.head_name + + common_properties = dict( + pull_request_number=pr_info.number, + commit_sha=pr_info.sha, + commit_url=pr_info.commit_html_url, + check_name=check_name, + check_status=check_status, + check_duration_ms=int(float(check_duration) * 1000), + check_start_time=check_start_time, + report_url=report_url, + pull_request_url=pull_request_url, + base_ref=base_ref, + base_repo=base_repo, + head_ref=head_ref, + head_repo=head_repo, + task_url=pr_info.task_url, + ) + + # Always publish a total record for all checks. For checks with individual + # tests, also publish a record per test. + result = [common_properties] + for test_result in test_results: + current_row = common_properties.copy() + test_name = test_result[0] + test_status = test_result[1] + + test_time = 0 + if len(test_result) > 2 and test_result[2]: + test_time = test_result[2] + current_row['test_duration_ms'] = int(float(test_time) * 1000) + current_row['test_name'] = test_name + current_row['test_status'] = test_status + result.append(current_row) + + return result + +def mark_flaky_tests(clickhouse_helper, check_name, test_results): + try: + query = """ + SELECT DISTINCT test_name + FROM checks + WHERE + check_start_time BETWEEN now() - INTERVAL 3 DAY AND now() + AND check_name = '{check_name}' + AND (test_status = 'FAIL' OR test_status = 'FLAKY') + AND pull_request_number = 0 + """.format(check_name=check_name) + + tests_data = clickhouse_helper.select_json_each_row('gh-data', query) + master_failed_tests = {row['test_name'] for row in tests_data} + logging.info("Found flaky tests: %s", ', '.join(master_failed_tests)) + + for test_result in test_results: + if test_result[1] == 'FAIL' and test_result[0] in master_failed_tests: + test_result[1] = 'FLAKY' + except Exception as ex: + logging.info("Exception happened during flaky tests fetch %s", ex) diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index d6a14275160..b6a8f67aa5f 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -15,6 +15,8 @@ from build_download_helper import download_builds_filter from upload_result_helper import upload_results from docker_pull_helper import get_images_with_versions from commit_status_helper import post_commit_status +from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse +from stopwatch import Stopwatch IMAGE_UBUNTU = "clickhouse/test-old-ubuntu" IMAGE_CENTOS = "clickhouse/test-old-centos" @@ -97,6 +99,9 @@ def get_run_commands(build_path, result_folder, server_log_folder, image_centos, if __name__ == "__main__": logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) repo_path = os.getenv("REPO_COPY", os.path.abspath("../../")) reports_path = os.getenv("REPORTS_PATH", "./reports") @@ -147,6 +152,13 @@ if __name__ == "__main__": s3_helper = S3Helper('https://s3.amazonaws.com') state, description, test_results, additional_logs = process_result(result_path, server_log_path) + + ch_helper = ClickHouseHelper() + mark_flaky_tests(ch_helper, CHECK_NAME, test_results) + report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_logs, CHECK_NAME) print(f"::notice ::Report url: {report_url}") post_commit_status(gh, pr_info.sha, CHECK_NAME, description, state, report_url) + + prepared_events = prepare_tests_results_for_clickhouse(pr_info, test_results, state, stopwatch.duration_seconds, stopwatch.start_time_str, report_url, CHECK_NAME) + ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events) diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 0402c66c62e..0482f05f284 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -11,6 +11,8 @@ from pr_info import PRInfo from get_robot_token import get_best_robot_token, get_parameter_from_ssm from upload_result_helper import upload_results from commit_status_helper import get_commit +from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse +from stopwatch import Stopwatch NAME = "Push to Dockerhub (actions)" @@ -152,6 +154,9 @@ def process_test_results(s3_client, test_results, s3_path_prefix): if __name__ == "__main__": logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + repo_path = os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../")) temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'docker_images_check') dockerhub_password = get_parameter_from_ssm('dockerhub_robot_password') @@ -195,6 +200,7 @@ if __name__ == "__main__": s3_path_prefix = str(pr_info.number) + "/" + pr_info.sha + "/" + NAME.lower().replace(' ', '_') status, test_results = process_test_results(s3_helper, images_processing_result, s3_path_prefix) + ch_helper = ClickHouseHelper() url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, [], NAME) with open(os.path.join(temp_path, 'changed_images.json'), 'w') as images_file: @@ -205,3 +211,6 @@ if __name__ == "__main__": gh = Github(get_best_robot_token()) commit = get_commit(gh, pr_info.sha) commit.create_status(context=NAME, description=description, state=status, target_url=url) + + prepared_events = prepare_tests_results_for_clickhouse(pr_info, test_results, status, stopwatch.duration_seconds, stopwatch.start_time_str, url, NAME) + ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events) diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index 19005d0138a..115872842a4 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -11,6 +11,8 @@ from get_robot_token import get_best_robot_token from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version from commit_status_helper import post_commit_status, get_commit +from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse +from stopwatch import Stopwatch NAME = "Docs Check (actions)" @@ -18,6 +20,8 @@ NAME = "Docs Check (actions)" if __name__ == "__main__": logging.basicConfig(level=logging.INFO) + stopwatch = Stopwatch() + temp_path = os.path.join(os.getenv("TEMP_PATH")) repo_path = os.path.join(os.getenv("REPO_COPY")) @@ -85,7 +89,11 @@ if __name__ == "__main__": lines.append(("Non zero exit code", "FAIL")) s3_helper = S3Helper('https://s3.amazonaws.com') + ch_helper = ClickHouseHelper() report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, lines, additional_files, NAME) print("::notice ::Report url: {report_url}") post_commit_status(gh, pr_info.sha, NAME, description, status, report_url) + + prepared_events = prepare_tests_results_for_clickhouse(pr_info, lines, status, stopwatch.duration_seconds, stopwatch.start_time_str, report_url, NAME) + ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events) diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 29e8284afd7..2734102be3f 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -12,6 +12,8 @@ from get_robot_token import get_best_robot_token from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version from commit_status_helper import post_commit_status +from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse +from stopwatch import Stopwatch NAME = 'Fast test (actions)' @@ -51,6 +53,9 @@ def process_results(result_folder): if __name__ == "__main__": logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) caches_path = os.getenv("CACHES_PATH", temp_path) @@ -128,6 +133,12 @@ if __name__ == "__main__": else: state, description, test_results, additional_logs = process_results(output_path) + ch_helper = ClickHouseHelper() + mark_flaky_tests(ch_helper, NAME, test_results) + report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, [run_log_path] + additional_logs, NAME, True) print("::notice ::Report url: {}".format(report_url)) post_commit_status(gh, pr_info.sha, NAME, description, state, report_url) + + prepared_events = prepare_tests_results_for_clickhouse(pr_info, test_results, state, stopwatch.duration_seconds, stopwatch.start_time_str, report_url, NAME) + ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index aa1f698d7eb..dc91ec07163 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -16,6 +16,8 @@ from build_download_helper import download_all_deb_packages from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version from commit_status_helper import post_commit_status, get_commit +from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse +from stopwatch import Stopwatch def get_image_name(check_name): @@ -96,6 +98,9 @@ def process_results(result_folder, server_log_path): if __name__ == "__main__": logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) repo_path = os.getenv("REPO_COPY", os.path.abspath("../../")) reports_path = os.getenv("REPORTS_PATH", "./reports") @@ -155,7 +160,14 @@ if __name__ == "__main__": s3_helper = S3Helper('https://s3.amazonaws.com') state, description, test_results, additional_logs = process_results(result_path, server_log_path) + + ch_helper = ClickHouseHelper() + mark_flaky_tests(ch_helper, check_name, test_results) + report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, [run_log_path] + additional_logs, check_name) print(f"::notice ::Report url: {report_url}") post_commit_status(gh, pr_info.sha, check_name, description, state, report_url) + + prepared_events = prepare_tests_results_for_clickhouse(pr_info, test_results, state, stopwatch.duration_seconds, stopwatch.start_time_str, report_url, check_name) + ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 7f24fb44e79..f6a46e72e84 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -16,6 +16,9 @@ from build_download_helper import download_all_deb_packages from upload_result_helper import upload_results from docker_pull_helper import get_images_with_versions from commit_status_helper import post_commit_status +from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse +from stopwatch import Stopwatch + DOWNLOAD_RETRIES_COUNT = 5 @@ -93,6 +96,9 @@ def process_results(result_folder): if __name__ == "__main__": logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) repo_path = os.getenv("REPO_COPY", os.path.abspath("../../")) reports_path = os.getenv("REPORTS_PATH", "./reports") @@ -148,7 +154,13 @@ if __name__ == "__main__": state, description, test_results, additional_logs = process_results(result_path) + ch_helper = ClickHouseHelper() + mark_flaky_tests(ch_helper, check_name, test_results) + s3_helper = S3Helper('https://s3.amazonaws.com') report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, [output_path_log] + additional_logs, check_name, False) print(f"::notice ::Report url: {report_url}") post_commit_status(gh, pr_info.sha, check_name, description, state, report_url) + + prepared_events = prepare_tests_results_for_clickhouse(pr_info, test_results, state, stopwatch.duration_seconds, stopwatch.start_time_str, report_url, check_name) + ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 46499514027..d3644aae461 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -17,6 +17,18 @@ class PRInfo: else: self.sha = github_event['pull_request']['head']['sha'] + repo_prefix = f"{os.getenv('GITHUB_SERVER_URL', 'https://github.com')}/{os.getenv('GITHUB_REPOSITORY', 'ClickHouse/ClickHouse')}" + self.task_url = f"{repo_prefix}/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + + self.repo_full_name = os.getenv('GITHUB_REPOSITORY', 'ClickHouse/ClickHouse') + self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" + self.pr_html_url = f"{repo_prefix}/pull/{self.number}" + + self.base_ref = github_event['pull_request']['base']['ref'] + self.base_name = github_event['pull_request']['base']['repo']['full_name'] + self.head_ref = github_event['pull_request']['head']['ref'] + self.head_name = github_event['pull_request']['base']['head']['full_name'] + self.labels = { l['name'] for l in github_event['pull_request']['labels'] } self.user_login = github_event['pull_request']['user']['login'] self.user_orgs = set([]) @@ -32,10 +44,21 @@ class PRInfo: diff = urllib.request.urlopen(diff_url) diff_object = PatchSet(diff, diff.headers.get_charsets()[0]) self.changed_files = { f.path for f in diff_object } + elif 'commits' in github_event: self.number = 0 self.sha = github_event['after'] self.labels = {} + self.repo_full_name = os.getenv('GITHUB_REPOSITORY', 'ClickHouse/ClickHouse') + repo_prefix = f"{os.getenv('GITHUB_SERVER_URL', 'https://github.com')}/{os.getenv('GITHUB_REPOSITORY', 'ClickHouse/ClickHouse')}" + self.task_url = f"{repo_prefix}/actions/runs/{os.getenv('GITHUB_RUN_ID')}" + self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" + self.pr_html_url = f"{repo_prefix}/commits/master" + self.base_ref = "master" + self.base_name = self.repo_full_name + self.head_ref = "master" + self.head_name = self.repo_full_name + if need_changed_files: commit_before = github_event['before'] response = requests.get(f'https://api.github.com/repos/ClickHouse/ClickHouse/compare/{commit_before}...{self.sha}') diff --git a/tests/ci/pvs_check.py b/tests/ci/pvs_check.py index 592f568b719..093dd4d6255 100644 --- a/tests/ci/pvs_check.py +++ b/tests/ci/pvs_check.py @@ -13,6 +13,8 @@ from pr_info import PRInfo from get_robot_token import get_best_robot_token, get_parameter_from_ssm from upload_result_helper import upload_results from commit_status_helper import get_commit +from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse +from stopwatch import Stopwatch NAME = 'PVS Studio (actions)' LICENCE_NAME = 'Free license: ClickHouse, Yandex' @@ -36,6 +38,9 @@ def _process_txt_report(path): if __name__ == "__main__": logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + repo_path = os.path.join(os.getenv("REPO_COPY", os.path.abspath("../../"))) temp_path = os.path.join(os.getenv("TEMP_PATH")) @@ -98,6 +103,10 @@ if __name__ == "__main__": print("::notice ::Report url: {}".format(report_url)) commit = get_commit(gh, pr_info.sha) commit.create_status(context=NAME, description=description, state=status, target_url=report_url) + + ch_helper = ClickHouseHelper() + prepared_events = prepare_tests_results_for_clickhouse(pr_info, test_results, status, stopwatch.duration_seconds, stopwatch.start_time_str, report_url, NAME) + ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events) except Exception as ex: print("Got an exception", ex) sys.exit(1) diff --git a/tests/ci/split_build_smoke_check.py b/tests/ci/split_build_smoke_check.py index ff7e7c9d127..28eb554d90e 100644 --- a/tests/ci/split_build_smoke_check.py +++ b/tests/ci/split_build_smoke_check.py @@ -14,6 +14,9 @@ from build_download_helper import download_shared_build from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version from commit_status_helper import post_commit_status +from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse +from stopwatch import Stopwatch + DOCKER_IMAGE = "clickhouse/split-build-smoke-test" DOWNLOAD_RETRIES_COUNT = 5 @@ -53,6 +56,9 @@ def get_run_command(build_path, result_folder, server_log_folder, docker_image): if __name__ == "__main__": logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) repo_path = os.getenv("REPO_COPY", os.path.abspath("../../")) reports_path = os.getenv("REPORTS_PATH", "./reports") @@ -102,7 +108,11 @@ if __name__ == "__main__": state, description, test_results, additional_logs = process_result(result_path, server_log_path) + ch_helper = ClickHouseHelper() s3_helper = S3Helper('https://s3.amazonaws.com') report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_logs, CHECK_NAME) print(f"::notice ::Report url: {report_url}") post_commit_status(gh, pr_info.sha, CHECK_NAME, description, state, report_url) + + prepared_events = prepare_tests_results_for_clickhouse(pr_info, test_results, state, stopwatch.duration_seconds, stopwatch.start_time_str, report_url, CHECK_NAME) + ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events) diff --git a/tests/ci/stopwatch.py b/tests/ci/stopwatch.py new file mode 100644 index 00000000000..0b145fb392d --- /dev/null +++ b/tests/ci/stopwatch.py @@ -0,0 +1,16 @@ +#!/usr/bin/env python3 + +import datetime + +class Stopwatch(): + def __init__(self): + self.start_time = datetime.datetime.utcnow() + self.start_time_str = self.start_time.strftime("%Y-%m-%d %H:%M:%S") + + @property + def duration_seconds(self): + return (datetime.datetime.utcnow() - self.start_time).total_seconds() + + @property + def start_time_str(self): + return self.start_time_str diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 870f7e1c331..4b3adfad23f 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -16,6 +16,9 @@ from build_download_helper import download_all_deb_packages from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version from commit_status_helper import post_commit_status +from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse +from stopwatch import Stopwatch + def get_run_command(build_path, result_folder, server_log_folder, image): cmd = "docker run -e S3_URL='https://clickhouse-datasets.s3.amazonaws.com' " + \ @@ -63,6 +66,8 @@ def process_results(result_folder, server_log_path, run_log_path): if __name__ == "__main__": logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) repo_path = os.getenv("REPO_COPY", os.path.abspath("../../")) reports_path = os.getenv("REPORTS_PATH", "./reports") @@ -110,9 +115,16 @@ if __name__ == "__main__": subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) + s3_helper = S3Helper('https://s3.amazonaws.com') state, description, test_results, additional_logs = process_results(result_path, server_log_path, run_log_path) + ch_helper = ClickHouseHelper() + mark_flaky_tests(ch_helper, check_name, test_results) + report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, [run_log_path] + additional_logs, check_name) print(f"::notice ::Report url: {report_url}") post_commit_status(gh, pr_info.sha, check_name, description, state, report_url) + + prepared_events = prepare_tests_results_for_clickhouse(pr_info, test_results, state, stopwatch.duration_seconds, stopwatch.start_time_str, report_url, check_name) + ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events) diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index e861668af73..8e11b295827 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -11,6 +11,9 @@ from get_robot_token import get_best_robot_token from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version from commit_status_helper import post_commit_status +from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse +from stopwatch import Stopwatch + NAME = "Style Check (actions)" @@ -45,6 +48,9 @@ def process_result(result_folder): if __name__ == "__main__": logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + repo_path = os.path.join(os.getenv("GITHUB_WORKSPACE", os.path.abspath("../../"))) temp_path = os.path.join(os.getenv("RUNNER_TEMP", os.path.abspath("./temp")), 'style_check') @@ -62,6 +68,12 @@ if __name__ == "__main__": subprocess.check_output(f"docker run -u $(id -u ${{USER}}):$(id -g ${{USER}}) --cap-add=SYS_PTRACE --volume={repo_path}:/ClickHouse --volume={temp_path}:/test_output {docker_image}", shell=True) state, description, test_results, additional_files = process_result(temp_path) + ch_helper = ClickHouseHelper() + mark_flaky_tests(ch_helper, NAME, test_results) + report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, additional_files, NAME) print("::notice ::Report url: {}".format(report_url)) post_commit_status(gh, pr_info.sha, NAME, description, state, report_url) + + prepared_events = prepare_tests_results_for_clickhouse(pr_info, test_results, state, stopwatch.duration_seconds, stopwatch.start_time_str, report_url, NAME) + ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events) diff --git a/tests/ci/unit_tests_check.py b/tests/ci/unit_tests_check.py index 863c63a41e0..21aa63e3b19 100644 --- a/tests/ci/unit_tests_check.py +++ b/tests/ci/unit_tests_check.py @@ -15,6 +15,9 @@ from build_download_helper import download_unit_tests from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version from commit_status_helper import post_commit_status +from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse +from stopwatch import Stopwatch + IMAGE_NAME = 'clickhouse/unit-test' @@ -87,6 +90,9 @@ def process_result(result_folder): if __name__ == "__main__": logging.basicConfig(level=logging.INFO) + + stopwatch = Stopwatch() + temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) repo_path = os.getenv("REPO_COPY", os.path.abspath("../../")) reports_path = os.getenv("REPORTS_PATH", "./reports") @@ -132,6 +138,13 @@ if __name__ == "__main__": s3_helper = S3Helper('https://s3.amazonaws.com') state, description, test_results, additional_logs = process_result(test_output) + + ch_helper = ClickHouseHelper() + mark_flaky_tests(ch_helper, check_name, test_results) + report_url = upload_results(s3_helper, pr_info.number, pr_info.sha, test_results, [run_log_path] + additional_logs, check_name) print(f"::notice ::Report url: {report_url}") post_commit_status(gh, pr_info.sha, check_name, description, state, report_url) + + prepared_events = prepare_tests_results_for_clickhouse(pr_info, test_results, state, stopwatch.duration_seconds, stopwatch.start_time_str, report_url, check_name) + ch_helper.insert_events_into(db="gh-data", table="checks", events=prepared_events) From f62a34e808eebefe12616447dc780923fa401012 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Nov 2021 17:56:54 +0300 Subject: [PATCH 165/200] Fix typo --- tests/ci/pr_info.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index d3644aae461..e9463404f0f 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -27,7 +27,7 @@ class PRInfo: self.base_ref = github_event['pull_request']['base']['ref'] self.base_name = github_event['pull_request']['base']['repo']['full_name'] self.head_ref = github_event['pull_request']['head']['ref'] - self.head_name = github_event['pull_request']['base']['head']['full_name'] + self.head_name = github_event['pull_request']['head']['full_name'] self.labels = { l['name'] for l in github_event['pull_request']['labels'] } self.user_login = github_event['pull_request']['user']['login'] From d69c9f951e0147f9e11f59722ca793e37e5843d0 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Nov 2021 18:09:18 +0300 Subject: [PATCH 166/200] Fix --- tests/ci/pr_info.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index e9463404f0f..ae2a188bde3 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -27,7 +27,7 @@ class PRInfo: self.base_ref = github_event['pull_request']['base']['ref'] self.base_name = github_event['pull_request']['base']['repo']['full_name'] self.head_ref = github_event['pull_request']['head']['ref'] - self.head_name = github_event['pull_request']['head']['full_name'] + self.head_name = github_event['pull_request']['head']['repo']['full_name'] self.labels = { l['name'] for l in github_event['pull_request']['labels'] } self.user_login = github_event['pull_request']['user']['login'] From 466df39bd447d340ba9d6c0c4f5a686cd25bfa21 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Nov 2021 18:24:39 +0300 Subject: [PATCH 167/200] Followup --- tests/ci/stopwatch.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/ci/stopwatch.py b/tests/ci/stopwatch.py index 0b145fb392d..b6ae8674df1 100644 --- a/tests/ci/stopwatch.py +++ b/tests/ci/stopwatch.py @@ -5,7 +5,7 @@ import datetime class Stopwatch(): def __init__(self): self.start_time = datetime.datetime.utcnow() - self.start_time_str = self.start_time.strftime("%Y-%m-%d %H:%M:%S") + self.start_time_str_value = self.start_time.strftime("%Y-%m-%d %H:%M:%S") @property def duration_seconds(self): @@ -13,4 +13,4 @@ class Stopwatch(): @property def start_time_str(self): - return self.start_time_str + return self.start_time_str_value From 9179c10d145f601d3c82757f5871e7534b82066e Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 19 Nov 2021 18:42:56 +0300 Subject: [PATCH 168/200] Fix style --- tests/ci/docs_check.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index 115872842a4..11ff68e0286 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -11,7 +11,7 @@ from get_robot_token import get_best_robot_token from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version from commit_status_helper import post_commit_status, get_commit -from clickhouse_helper import ClickHouseHelper, mark_flaky_tests, prepare_tests_results_for_clickhouse +from clickhouse_helper import ClickHouseHelper, prepare_tests_results_for_clickhouse from stopwatch import Stopwatch From 8b6bcfdc0a3aed4665180cbaa73539ea3f30c847 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 19 Nov 2021 19:17:12 +0300 Subject: [PATCH 169/200] [website] update hero button (#31508) --- website/templates/index/hero.html | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/website/templates/index/hero.html b/website/templates/index/hero.html index 45473e8c3a1..7fa7feb8459 100644 --- a/website/templates/index/hero.html +++ b/website/templates/index/hero.html @@ -12,8 +12,7 @@
- Add to Your Calendar -
18 Nov - Europe, Americas
+ Watch on YouTube
Add to Your Calendar From 8d7413b0d6052df3a2396ca4da351efeb748e6a4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 19 Nov 2021 19:51:03 +0300 Subject: [PATCH 170/200] remove strange multimap from mutations --- .../MergeTree/MergeTreeMutationEntry.cpp | 45 ++++++++++++++++--- .../MergeTree/MergeTreeMutationEntry.h | 10 +++-- src/Storages/StorageMergeTree.cpp | 23 +++++----- src/Storages/StorageMergeTree.h | 4 +- 4 files changed, 59 insertions(+), 23 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp index 2aefb3df2be..0f71742fb09 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.cpp +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -10,7 +11,39 @@ namespace DB { -MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, Int64 tmp_number) +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + +String MergeTreeMutationEntry::versionToFileName(UInt64 block_number_) +{ + assert(block_number_); + return fmt::format("mutation_{}.txt", block_number_); +} + +UInt64 MergeTreeMutationEntry::tryParseFileName(const String & file_name_) +{ + UInt64 maybe_block_number = 0; + ReadBufferFromString file_name_buf(file_name_); + if (!checkString("mutation_", file_name_buf)) + return 0; + if (!tryReadIntText(maybe_block_number, file_name_buf)) + return 0; + if (!checkString(".txt", file_name_buf)) + return 0; + assert(maybe_block_number); + return maybe_block_number; +} + +UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_) +{ + if (UInt64 maybe_block_number = tryParseFileName(file_name_)) + return maybe_block_number; + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot parse mutation version from file name, expected 'mutation_.txt', got '{}'", file_name_); +} + +MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number) : create_time(time(nullptr)) , commands(std::move(commands_)) , disk(std::move(disk_)) @@ -35,10 +68,11 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP } } -void MergeTreeMutationEntry::commit(Int64 block_number_) +void MergeTreeMutationEntry::commit(UInt64 block_number_) { + assert(block_number_); block_number = block_number_; - String new_file_name = "mutation_" + toString(block_number) + ".txt"; + String new_file_name = versionToFileName(block_number); disk->moveFile(path_prefix + file_name, path_prefix + new_file_name); is_temp = false; file_name = new_file_name; @@ -62,10 +96,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat , file_name(file_name_) , is_temp(false) { - ReadBufferFromString file_name_buf(file_name); - file_name_buf >> "mutation_" >> block_number >> ".txt"; - assertEOF(file_name_buf); - + block_number = parseFileName(file_name); auto buf = disk->readFile(path_prefix + file_name); *buf >> "format version: 1\n"; diff --git a/src/Storages/MergeTree/MergeTreeMutationEntry.h b/src/Storages/MergeTree/MergeTreeMutationEntry.h index e01ce4320b3..7554a03836e 100644 --- a/src/Storages/MergeTree/MergeTreeMutationEntry.h +++ b/src/Storages/MergeTree/MergeTreeMutationEntry.h @@ -21,7 +21,7 @@ struct MergeTreeMutationEntry String file_name; bool is_temp = false; - Int64 block_number = 0; + UInt64 block_number = 0; String latest_failed_part; MergeTreePartInfo latest_failed_part_info; @@ -29,15 +29,19 @@ struct MergeTreeMutationEntry String latest_fail_reason; /// Create a new entry and write it to a temporary file. - MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, Int64 tmp_number); + MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk, const String & path_prefix_, UInt64 tmp_number); MergeTreeMutationEntry(const MergeTreeMutationEntry &) = delete; MergeTreeMutationEntry(MergeTreeMutationEntry &&) = default; /// Commit entry and rename it to a permanent file. - void commit(Int64 block_number_); + void commit(UInt64 block_number_); void removeFile(); + static String versionToFileName(UInt64 block_number_); + static UInt64 tryParseFileName(const String & file_name_); + static UInt64 parseFileName(const String & file_name_); + /// Load an existing entry. MergeTreeMutationEntry(DiskPtr disk_, const String & path_prefix_, const String & file_name_); diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index ef98901f24a..c659c8fe324 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -411,8 +411,9 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, String version = increment.get(); entry.commit(version); mutation_file_name = entry.file_name; - auto insertion = current_mutations_by_id.emplace(mutation_file_name, std::move(entry)); - current_mutations_by_version.emplace(version, insertion.first->second); + bool inserted = current_mutations_by_version.try_emplace(version, std::move(entry)).second; + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version); LOG_INFO(log, "Added mutation: {}", mutation_file_name); } @@ -617,16 +618,18 @@ std::vector StorageMergeTree::getMutationsStatus() cons CancellationCode StorageMergeTree::killMutation(const String & mutation_id) { LOG_TRACE(log, "Killing mutation {}", mutation_id); + UInt64 mutation_version = MergeTreeMutationEntry::tryParseFileName(mutation_id); + if (!mutation_version) + return CancellationCode::NotFound; std::optional to_kill; { std::lock_guard lock(currently_processing_in_background_mutex); - auto it = current_mutations_by_id.find(mutation_id); - if (it != current_mutations_by_id.end()) + auto it = current_mutations_by_version.find(mutation_version); + if (it != current_mutations_by_version.end()) { to_kill.emplace(std::move(it->second)); - current_mutations_by_id.erase(it); - current_mutations_by_version.erase(to_kill->block_number); + current_mutations_by_version.erase(it); } } @@ -667,10 +670,11 @@ void StorageMergeTree::loadMutations() if (startsWith(it->name(), "mutation_")) { MergeTreeMutationEntry entry(disk, path, it->name()); - Int64 block_number = entry.block_number; + UInt64 block_number = entry.block_number; LOG_DEBUG(log, "Loading mutation: {} entry, commands size: {}", it->name(), entry.commands.size()); - auto insertion = current_mutations_by_id.emplace(it->name(), std::move(entry)); - current_mutations_by_version.emplace(block_number, insertion.first->second); + auto inserted = current_mutations_by_version.try_emplace(block_number, std::move(entry)).second; + if (!inserted) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number); } else if (startsWith(it->name(), "tmp_mutation_")) { @@ -1089,7 +1093,6 @@ size_t StorageMergeTree::clearOldMutations(bool truncate) for (size_t i = 0; i < to_delete_count; ++i) { mutations_to_delete.push_back(std::move(it->second)); - current_mutations_by_id.erase(mutations_to_delete.back().file_name); it = current_mutations_by_version.erase(it); } } diff --git a/src/Storages/StorageMergeTree.h b/src/Storages/StorageMergeTree.h index 2f776bd77a6..6d6ee785ad0 100644 --- a/src/Storages/StorageMergeTree.h +++ b/src/Storages/StorageMergeTree.h @@ -131,9 +131,7 @@ private: /// This set have to be used with `currently_processing_in_background_mutex`. DataParts currently_merging_mutating_parts; - - std::map current_mutations_by_id; - std::multimap current_mutations_by_version; + std::map current_mutations_by_version; std::atomic shutdown_called {false}; From 67753e5b83affecea3a35cb038fdfb2a16f77a3f Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 19 Nov 2021 20:05:45 +0300 Subject: [PATCH 171/200] Update StorageMergeTree.cpp --- src/Storages/StorageMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c659c8fe324..8e352a74d33 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -684,7 +684,7 @@ void StorageMergeTree::loadMutations() } if (!current_mutations_by_version.empty()) - increment.value = std::max(Int64(increment.value.load()), current_mutations_by_version.rbegin()->first); + increment.value = std::max(increment.value.load(), current_mutations_by_version.rbegin()->first); } std::shared_ptr StorageMergeTree::selectPartsToMerge( From ef6443f3f547f2574eecca0c2ab1174da4dc528f Mon Sep 17 00:00:00 2001 From: Alexey Date: Fri, 19 Nov 2021 19:21:50 +0000 Subject: [PATCH 172/200] Translated paragraph about NULL from ru to en. Copied corresponding example --- .../sql-reference/statements/select/where.md | 26 +++++++++++++++ .../sql-reference/statements/select/where.md | 33 ++++++++++++------- 2 files changed, 48 insertions(+), 11 deletions(-) diff --git a/docs/en/sql-reference/statements/select/where.md b/docs/en/sql-reference/statements/select/where.md index 236a5f53950..867123b768b 100644 --- a/docs/en/sql-reference/statements/select/where.md +++ b/docs/en/sql-reference/statements/select/where.md @@ -13,8 +13,12 @@ If there is a `WHERE` clause, it must contain an expression with the `UInt8` typ !!! note "Note" There is a filtering optimization called [prewhere](../../../sql-reference/statements/select/prewhere.md). +If you need to test a value for [NULL](../../../sql-reference/syntax.md#null-literal), use [IS NULL](../../operators/index.md#operator-is-null) and [IS NOT NULL](../../operators/index.md#is-not-null) operators or [isNull](../../../sql-reference/functions/functions-for-nulls.md#isnull) and [isNotNull](../../../sql-reference/functions/functions-for-nulls.md#isnotnull) functions. +Otherwise an expression with `NULL` never passes. + **Example** + To find numbers that are multiples of 3 and are greater than 10 execute the following query on the [numbers table](../../../sql-reference/table-functions/numbers.md): ``` sql @@ -30,3 +34,25 @@ Result: │ 18 │ └────────┘ ``` + +Queries with `NULL` values: + +``` sql +CREATE TABLE t_null(x Int8, y Nullable(Int8)) ENGINE=MergeTree() ORDER BY x; +INSERT INTO t_null VALUES (1, NULL), (2, 3); + +SELECT * FROM t_null WHERE y IS NULL; +SELECT * FROM t_null WHERE y != 0; +``` + +Result: + +``` text +┌─x─┬────y─┐ +│ 1 │ ᴺᵁᴸᴸ │ +└───┴──────┘ +┌─x─┬─y─┐ +│ 2 │ 3 │ +└───┴───┘ +``` + diff --git a/docs/ru/sql-reference/statements/select/where.md b/docs/ru/sql-reference/statements/select/where.md index 3cfcbb0a849..b5dd015da6a 100644 --- a/docs/ru/sql-reference/statements/select/where.md +++ b/docs/ru/sql-reference/statements/select/where.md @@ -12,17 +12,6 @@ ClickHouse использует в выражении индексы, если Если в секции необходимо проверить [NULL](../../../sql-reference/syntax.md#null-literal), то используйте операторы [IS NULL](../../operators/index.md#operator-is-null) и [IS NOT NULL](../../operators/index.md#is-not-null), а также соответствующие функции `isNull` и `isNotNull`. В противном случае выражение будет считаться всегда не выполненным. -Пример проверки на `NULL`: - -``` sql -SELECT * FROM t_null WHERE y IS NULL -``` - -``` text -┌─x─┬────y─┐ -│ 1 │ ᴺᵁᴸᴸ │ -└───┴──────┘ -``` !!! note "Примечание" Существует оптимизация фильтрации под названием [prewhere](prewhere.md). @@ -44,3 +33,25 @@ SELECT number FROM numbers(20) WHERE (number > 10) AND (number % 3 == 0); │ 18 │ └────────┘ ``` + + +Пример проверки на `NULL`: + +``` sql +CREATE TABLE t_null(x Int8, y Nullable(Int8)) ENGINE=MergeTree() ORDER BY x; +INSERT INTO t_null VALUES (1, NULL), (2, 3); + +SELECT * FROM t_null WHERE y IS NULL; +SELECT * FROM t_null WHERE y != 0; +``` + +Результат: + +``` text +┌─x─┬────y─┐ +│ 1 │ ᴺᵁᴸᴸ │ +└───┴──────┘ +┌─x─┬─y─┐ +│ 2 │ 3 │ +└───┴───┘ +``` From eb931ad7678b8b71bdd45352a5b4e1ade4dc5438 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Nov 2021 06:49:01 +0300 Subject: [PATCH 173/200] Add benchmark from @felixoid --- website/benchmark/hardware/index.html | 1 + .../results/core_i7_11800h_lenovo_p15.json | 54 +++++++++++++++++++ 2 files changed, 55 insertions(+) create mode 100644 website/benchmark/hardware/results/core_i7_11800h_lenovo_p15.json diff --git a/website/benchmark/hardware/index.html b/website/benchmark/hardware/index.html index e6a87385fa8..260a928184d 100644 --- a/website/benchmark/hardware/index.html +++ b/website/benchmark/hardware/index.html @@ -81,6 +81,7 @@ Results for Xeon Gold 6140 are from Shiv Iyer (ChistaDATA Labs).
Comparison of EBS and EFS is from Ramazan Polat.
Results for Hetzner and Scaleway are from Anthony Najjar Simon (Panelbear).
Results for GCP are from Vy Nguyen Tan.
+Results for ThinkPad P15 are from Mikhail Shiryaev.

diff --git a/website/benchmark/hardware/results/core_i7_11800h_lenovo_p15.json b/website/benchmark/hardware/results/core_i7_11800h_lenovo_p15.json new file mode 100644 index 00000000000..f42be93e1e3 --- /dev/null +++ b/website/benchmark/hardware/results/core_i7_11800h_lenovo_p15.json @@ -0,0 +1,54 @@ +[ + { + "system": "ThinkPad P15", + "system_full": "Lenovo ThinkPad P15, i7-11800H @ 2.30GHz, 16 cores, 32 GiB RAM, NVMe", + "time": "2021-11-19 00:00:00", + "kind": "laptop", + "result": + [ +[0.001, 0.001, 0.001], +[0.013, 0.008, 0.007], +[0.036, 0.025, 0.024], +[0.098, 0.046, 0.040], +[0.139, 0.102, 0.100], +[0.368, 0.279, 0.278], +[0.018, 0.015, 0.017], +[0.025, 0.008, 0.009], +[0.586, 0.527, 0.533], +[0.690, 0.579, 0.554], +[0.224, 0.171, 0.162], +[0.244, 0.201, 0.191], +[0.996, 0.854, 0.871], +[1.339, 1.199, 1.159], +[1.116, 1.073, 1.045], +[1.177, 1.060, 1.084], +[3.307, 3.236, 3.182], +[1.958, 1.789, 1.835], +[6.079, 5.883, 5.895], +[0.109, 0.051, 0.048], +[1.429, 0.898, 0.819], +[1.626, 1.023, 0.937], +[3.390, 2.296, 2.381], +[2.249, 0.997, 0.992], +[0.422, 0.260, 0.299], +[0.285, 0.230, 0.213], +[0.419, 0.256, 0.268], +[1.411, 0.777, 0.830], +[1.828, 1.305, 1.283], +[3.556, 3.725, 3.603], +[0.805, 0.643, 0.650], +[1.369, 1.049, 1.033], +[7.665, 7.623, 7.601], +[5.305, 4.513, 4.691], +[5.370, 4.686, 4.874], +[1.756, 1.492, 1.579], +[0.143, 0.132, 0.131], +[0.058, 0.057, 0.056], +[0.068, 0.048, 0.051], +[0.339, 0.298, 0.304], +[0.032, 0.022, 0.023], +[0.018, 0.011, 0.015], +[0.010, 0.002, 0.004] + ] + } +] From 334535a4c40625e0e1c898a68527b2e9eb3162b2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 21 May 2021 11:25:43 +0300 Subject: [PATCH 174/200] Better exception message when users.xml cannot be loaded due to bad password hash --- programs/server/Server.cpp | 10 ++++- src/Access/AccessControl.cpp | 3 +- src/Access/Common/AuthenticationData.cpp | 11 ++++- src/Access/UsersConfigAccessStorage.cpp | 51 ++++++++++++++++-------- 4 files changed, 56 insertions(+), 19 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 5cd295070a6..9f86e3ec808 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -888,7 +888,15 @@ if (ThreadFuzzer::instance().isEffective()) access_control.setCustomSettingsPrefixes(config().getString("custom_settings_prefixes")); /// Initialize access storages. - access_control.addStoragesFromMainConfig(config(), config_path, [&] { return global_context->getZooKeeper(); }); + try + { + access_control.addStoragesFromMainConfig(config(), config_path, [&] { return global_context->getZooKeeper(); }); + } + catch (...) + { + tryLogCurrentException(log); + throw; + } /// Reload config in SYSTEM RELOAD CONFIG query. global_context->setConfigReloadCallback([&]() diff --git a/src/Access/AccessControl.cpp b/src/Access/AccessControl.cpp index 76ae4af3e21..54ba8853b17 100644 --- a/src/Access/AccessControl.cpp +++ b/src/Access/AccessControl.cpp @@ -172,7 +172,8 @@ void AccessControl::addUsersConfigStorage(const String & storage_name_, const Po auto new_storage = std::make_shared(storage_name_, check_setting_name_function); new_storage->setConfig(users_config_); addStorage(new_storage); - LOG_DEBUG(getLogger(), "Added {} access storage '{}', path: {}", String(new_storage->getStorageType()), new_storage->getStorageName(), new_storage->getPath()); + LOG_DEBUG(getLogger(), "Added {} access storage '{}', path: {}", + String(new_storage->getStorageType()), new_storage->getStorageName(), new_storage->getPath()); } void AccessControl::addUsersConfigStorage( diff --git a/src/Access/Common/AuthenticationData.cpp b/src/Access/Common/AuthenticationData.cpp index 012e7546270..7412d7336e3 100644 --- a/src/Access/Common/AuthenticationData.cpp +++ b/src/Access/Common/AuthenticationData.cpp @@ -133,7 +133,16 @@ void AuthenticationData::setPasswordHashHex(const String & hash) { Digest digest; digest.resize(hash.size() / 2); - boost::algorithm::unhex(hash.begin(), hash.end(), digest.data()); + + try + { + boost::algorithm::unhex(hash.begin(), hash.end(), digest.data()); + } + catch (const std::exception &) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read password hash in hex, check for valid characters [0-9a-fA-F] and length"); + } + setPasswordHashBinary(digest); } diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index 014eaf0179d..a25e105b26a 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -208,8 +208,19 @@ namespace std::vector users; users.reserve(user_names.size()); + for (const auto & user_name : user_names) - users.push_back(parseUser(config, user_name)); + { + try + { + users.push_back(parseUser(config, user_name)); + } + catch (Exception & e) + { + e.addMessage(fmt::format("while parsing user '{}' in users configuration file", user_name)); + throw; + } + } return users; } @@ -517,21 +528,29 @@ void UsersConfigAccessStorage::load( const String & preprocessed_dir, const zkutil::GetZooKeeper & get_zookeeper_function) { - std::lock_guard lock{load_mutex}; - path = std::filesystem::path{users_config_path}.lexically_normal(); - config_reloader.reset(); - config_reloader = std::make_unique( - users_config_path, - include_from_path, - preprocessed_dir, - zkutil::ZooKeeperNodeCache(get_zookeeper_function), - std::make_shared(), - [&](Poco::AutoPtr new_config, bool /*initial_loading*/) - { - parseFromConfig(*new_config); - Settings::checkNoSettingNamesAtTopLevel(*new_config, users_config_path); - }, - /* already_loaded = */ false); + try + { + std::lock_guard lock{load_mutex}; + path = std::filesystem::path{users_config_path}.lexically_normal(); + config_reloader.reset(); + config_reloader = std::make_unique( + users_config_path, + include_from_path, + preprocessed_dir, + zkutil::ZooKeeperNodeCache(get_zookeeper_function), + std::make_shared(), + [&](Poco::AutoPtr new_config, bool /*initial_loading*/) + { + parseFromConfig(*new_config); + Settings::checkNoSettingNamesAtTopLevel(*new_config, users_config_path); + }, + /* already_loaded = */ false); + } + catch (Exception & e) + { + e.addMessage(fmt::format("while loading configuration file '{}'", users_config_path)); + throw; + } } void UsersConfigAccessStorage::reload() From 9ca5c960b767c6557a21bfc2a554ce63e01f1e5c Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 22 May 2021 01:16:48 +0300 Subject: [PATCH 175/200] Remove useless code --- src/Access/GSSAcceptor.cpp | 21 +-------------------- src/Access/GSSAcceptor.h | 2 +- 2 files changed, 2 insertions(+), 21 deletions(-) diff --git a/src/Access/GSSAcceptor.cpp b/src/Access/GSSAcceptor.cpp index 207baf955e0..02fa3f8e1d3 100644 --- a/src/Access/GSSAcceptor.cpp +++ b/src/Access/GSSAcceptor.cpp @@ -18,7 +18,7 @@ namespace ErrorCodes extern const int KERBEROS_ERROR; } -GSSAcceptorContext::GSSAcceptorContext(const GSSAcceptorContext::Params& params_) +GSSAcceptorContext::GSSAcceptorContext(const GSSAcceptorContext::Params & params_) : params(params_) { } @@ -50,7 +50,6 @@ std::recursive_mutex gss_global_mutex; struct PrincipalName { explicit PrincipalName(String principal); -// operator String() const; String name; std::vector instances; @@ -75,24 +74,6 @@ PrincipalName::PrincipalName(String principal) } } -/* -PrincipalName::operator String() const -{ - String principal = name; - - for (const auto & instance : instances) - { - principal += '/'; - principal += instance; - } - - principal += '@'; - principal += realm; - - return principal; -} -*/ - String bufferToString(const gss_buffer_desc & buf) { String str; diff --git a/src/Access/GSSAcceptor.h b/src/Access/GSSAcceptor.h index 4501622cd81..4b57b575455 100644 --- a/src/Access/GSSAcceptor.h +++ b/src/Access/GSSAcceptor.h @@ -30,7 +30,7 @@ public: String realm; }; - explicit GSSAcceptorContext(const Params& params_); + explicit GSSAcceptorContext(const Params & params_); virtual ~GSSAcceptorContext() override; GSSAcceptorContext(const GSSAcceptorContext &) = delete; From c70cc62b5f6c56d625baa9321c2541825ba76ac9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 19 Nov 2021 21:13:37 +0300 Subject: [PATCH 176/200] Add more checks in 02010_lc_native --- .../0_stateless/02010_lc_native.python | 32 ++++++++++++------- 1 file changed, 20 insertions(+), 12 deletions(-) diff --git a/tests/queries/0_stateless/02010_lc_native.python b/tests/queries/0_stateless/02010_lc_native.python index 6b3cad89ed6..db2a85e8b14 100755 --- a/tests/queries/0_stateless/02010_lc_native.python +++ b/tests/queries/0_stateless/02010_lc_native.python @@ -163,15 +163,22 @@ def sendEmptyBlock(s): s.sendall(ba) +def assertPacket(packet, expected): + assert(packet == expected), packet + def readHeader(s): - readVarUInt(s) # Data + packet_type = readVarUInt(s) + if packet_type == 2: # Exception + raise RuntimeError(readException(s)) + assertPacket(packet_type, 1) # Data + readStringBinary(s) # external table name # BlockInfo - readVarUInt(s) # 1 - readUInt8(s) # is_overflows - readVarUInt(s) # 2 - readUInt32(s) # bucket_num - readVarUInt(s) # 0 + assertPacket(readVarUInt(s), 1) # 1 + assertPacket(readUInt8(s), 0) # is_overflows + assertPacket(readVarUInt(s), 2) # 2 + assertPacket(readUInt32(s), 4294967295) # bucket_num + assertPacket(readVarUInt(s), 0) # 0 columns = readVarUInt(s) # rows rows = readVarUInt(s) # columns print("Rows {} Columns {}".format(rows, columns)) @@ -182,13 +189,12 @@ def readHeader(s): def readException(s): - assert(readVarUInt(s) == 2) code = readUInt32(s) name = readStringBinary(s) text = readStringBinary(s) readStringBinary(s) # trace - assert(readUInt8(s) == 0) # has_nested - print("code {}: {}".format(code, text.replace('DB::Exception:', ''))) + assertPacket(readUInt8(s), 0) # has_nested + return "code {}: {}".format(code, text.replace('DB::Exception:', '')) def insertValidLowCardinalityRow(): @@ -223,7 +229,7 @@ def insertValidLowCardinalityRow(): # Fin block sendEmptyBlock(s) - assert(readVarUInt(s) == 5) # End of stream + assertPacket(readVarUInt(s), 5) # End of stream s.close() @@ -256,7 +262,8 @@ def insertLowCardinalityRowWithIndexOverflow(): ba.extend([0] * 7 + [1]) # UInt64 index (overflow) s.sendall(ba) - readException(s) + assertPacket(readVarUInt(s), 2) + print(readException(s)) s.close() @@ -289,7 +296,8 @@ def insertLowCardinalityRowWithIncorrectDictType(): ba.extend([0] * 8) # UInt64 index (overflow) s.sendall(ba) - readException(s) + assertPacket(readVarUInt(s), 2) + print(readException(s)) s.close() From fbaa6804c288e1e206de868451c66282ed45d5e3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 19 Nov 2021 21:13:37 +0300 Subject: [PATCH 177/200] Fix 02010_lc_native flakiness (Query with id = 123456 is already running) --- tests/queries/0_stateless/02010_lc_native.python | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02010_lc_native.python b/tests/queries/0_stateless/02010_lc_native.python index db2a85e8b14..2667bba8432 100755 --- a/tests/queries/0_stateless/02010_lc_native.python +++ b/tests/queries/0_stateless/02010_lc_native.python @@ -3,10 +3,12 @@ import socket import os +import uuid CLICKHOUSE_HOST = os.environ.get('CLICKHOUSE_HOST', '127.0.0.1') CLICKHOUSE_PORT = int(os.environ.get('CLICKHOUSE_PORT_TCP', '900000')) CLICKHOUSE_DATABASE = os.environ.get('CLICKHOUSE_DATABASE', 'default') +CLICKHOUSE_QUERY_ID = uuid.uuid4().hex def writeVarUInt(x, ba): for _ in range(0, 9): @@ -111,7 +113,7 @@ def receiveHello(s): def serializeClientInfo(ba): writeStringBinary('default', ba) # initial_user - writeStringBinary('123456', ba) # initial_query_id + writeStringBinary(CLICKHOUSE_QUERY_ID, ba) # initial_query_id writeStringBinary('127.0.0.1:9000', ba) # initial_address ba.extend([0] * 8) # initial_query_start_time_microseconds ba.append(1) # TCP @@ -130,7 +132,7 @@ def serializeClientInfo(ba): def sendQuery(s, query): ba = bytearray() writeVarUInt(1, ba) # query - writeStringBinary('123456', ba) + writeStringBinary(CLICKHOUSE_QUERY_ID, ba) ba.append(1) # INITIAL_QUERY From 122b42c36a47751f9b4bdd4389131383af50eb78 Mon Sep 17 00:00:00 2001 From: Alexey Date: Sat, 20 Nov 2021 08:44:36 +0000 Subject: [PATCH 178/200] Formatting updated. Spelling fix. 2 paragraphs reordered. --- docs/en/sql-reference/statements/select/where.md | 6 ++---- docs/ru/sql-reference/statements/select/where.md | 10 ++++------ 2 files changed, 6 insertions(+), 10 deletions(-) diff --git a/docs/en/sql-reference/statements/select/where.md b/docs/en/sql-reference/statements/select/where.md index 867123b768b..348b869e2db 100644 --- a/docs/en/sql-reference/statements/select/where.md +++ b/docs/en/sql-reference/statements/select/where.md @@ -6,19 +6,18 @@ toc_title: WHERE `WHERE` clause allows to filter the data that is coming from [FROM](../../../sql-reference/statements/select/from.md) clause of `SELECT`. -If there is a `WHERE` clause, it must contain an expression with the `UInt8` type. This is usually an expression with comparison and logical operators. Rows where this expression evaluates to 0 are excluded from further transformations or result. +If there is a `WHERE` clause, it must contain an expression with the `UInt8` type. This is usually an expression with comparison and logical operators. Rows where this expression evaluates to `0` are excluded from further transformations or result. `WHERE` expression is evaluated on the ability to use indexes and partition pruning, if the underlying table engine supports that. !!! note "Note" - There is a filtering optimization called [prewhere](../../../sql-reference/statements/select/prewhere.md). + There is a filtering optimization called [PREWHERE](../../../sql-reference/statements/select/prewhere.md). If you need to test a value for [NULL](../../../sql-reference/syntax.md#null-literal), use [IS NULL](../../operators/index.md#operator-is-null) and [IS NOT NULL](../../operators/index.md#is-not-null) operators or [isNull](../../../sql-reference/functions/functions-for-nulls.md#isnull) and [isNotNull](../../../sql-reference/functions/functions-for-nulls.md#isnotnull) functions. Otherwise an expression with `NULL` never passes. **Example** - To find numbers that are multiples of 3 and are greater than 10 execute the following query on the [numbers table](../../../sql-reference/table-functions/numbers.md): ``` sql @@ -55,4 +54,3 @@ Result: │ 2 │ 3 │ └───┴───┘ ``` - diff --git a/docs/ru/sql-reference/statements/select/where.md b/docs/ru/sql-reference/statements/select/where.md index b5dd015da6a..e176780c43e 100644 --- a/docs/ru/sql-reference/statements/select/where.md +++ b/docs/ru/sql-reference/statements/select/where.md @@ -4,17 +4,16 @@ toc_title: WHERE # Секция WHERE {#select-where} -Позволяет задать выражение, которое ClickHouse использует для фильтрации данных перед всеми другими действиями в запросе кроме выражений, содержащихся в секции [PREWHERE](prewhere.md#prewhere-clause). Обычно, это выражение с логическими операторами. +Позволяет задать выражение, которое ClickHouse использует для фильтрации данных перед всеми другими действиями в запросе кроме выражений, содержащихся в секции [PREWHERE](prewhere.md#prewhere-clause). Обычно это выражение с логическими операторами. Результат выражения должен иметь тип `UInt8`. ClickHouse использует в выражении индексы, если это позволяет [движок таблицы](../../../engines/table-engines/index.md). -Если в секции необходимо проверить [NULL](../../../sql-reference/syntax.md#null-literal), то используйте операторы [IS NULL](../../operators/index.md#operator-is-null) и [IS NOT NULL](../../operators/index.md#is-not-null), а также соответствующие функции `isNull` и `isNotNull`. В противном случае выражение будет считаться всегда не выполненным. - - !!! note "Примечание" - Существует оптимизация фильтрации под названием [prewhere](prewhere.md). + Существует оптимизация фильтрации под названием [PREWHERE](prewhere.md). + +Если в секции необходимо проверить [NULL](../../../sql-reference/syntax.md#null-literal), то используйте операторы [IS NULL](../../operators/index.md#operator-is-null) и [IS NOT NULL](../../operators/index.md#is-not-null), а также соответствующие функции [isNull](../../../sql-reference/functions/functions-for-nulls.md#isnull) и [isNotNull](../../../sql-reference/functions/functions-for-nulls.md#isnotnull). В противном случае выражение будет считаться всегда не выполненным. **Пример** @@ -34,7 +33,6 @@ SELECT number FROM numbers(20) WHERE (number > 10) AND (number % 3 == 0); └────────┘ ``` - Пример проверки на `NULL`: ``` sql From 57ac39f99eca688908897d14db0d76116d4422db Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sat, 20 Nov 2021 12:10:45 +0300 Subject: [PATCH 179/200] Add more information to exceptions raised by UsersConfigAccessStorage. --- src/Access/UsersConfigAccessStorage.cpp | 100 +++++++++++++++--------- 1 file changed, 62 insertions(+), 38 deletions(-) diff --git a/src/Access/UsersConfigAccessStorage.cpp b/src/Access/UsersConfigAccessStorage.cpp index a25e105b26a..dbaf4e002b1 100644 --- a/src/Access/UsersConfigAccessStorage.cpp +++ b/src/Access/UsersConfigAccessStorage.cpp @@ -286,14 +286,25 @@ namespace Poco::Util::AbstractConfiguration::Keys quota_names; config.keys("quotas", quota_names); + std::vector quotas; quotas.reserve(quota_names.size()); + for (const auto & quota_name : quota_names) { - auto it = quota_to_user_ids.find(quota_name); - const std::vector & quota_users = (it != quota_to_user_ids.end()) ? std::move(it->second) : std::vector{}; - quotas.push_back(parseQuota(config, quota_name, quota_users)); + try + { + auto it = quota_to_user_ids.find(quota_name); + const std::vector & quota_users = (it != quota_to_user_ids.end()) ? std::move(it->second) : std::vector{}; + quotas.push_back(parseQuota(config, quota_name, quota_users)); + } + catch (Exception & e) + { + e.addMessage(fmt::format("while parsing quota '{}' in users configuration file", quota_name)); + throw; + } } + return quotas; } @@ -451,11 +462,24 @@ namespace const Poco::Util::AbstractConfiguration & config, Fn auto && check_setting_name_function) { - std::vector profiles; Poco::Util::AbstractConfiguration::Keys profile_names; config.keys("profiles", profile_names); + + std::vector profiles; + profiles.reserve(profile_names.size()); + for (const auto & profile_name : profile_names) - profiles.push_back(parseSettingsProfile(config, profile_name, check_setting_name_function)); + { + try + { + profiles.push_back(parseSettingsProfile(config, profile_name, check_setting_name_function)); + } + catch (Exception & e) + { + e.addMessage(fmt::format("while parsing profile '{}' in users configuration file", profile_name)); + throw; + } + } return profiles; } @@ -510,16 +534,24 @@ void UsersConfigAccessStorage::setConfig(const Poco::Util::AbstractConfiguration void UsersConfigAccessStorage::parseFromConfig(const Poco::Util::AbstractConfiguration & config) { - std::vector> all_entities; - for (const auto & entity : parseUsers(config)) - all_entities.emplace_back(generateID(*entity), entity); - for (const auto & entity : parseQuotas(config)) - all_entities.emplace_back(generateID(*entity), entity); - for (const auto & entity : parseRowPolicies(config)) - all_entities.emplace_back(generateID(*entity), entity); - for (const auto & entity : parseSettingsProfiles(config, check_setting_name_function)) - all_entities.emplace_back(generateID(*entity), entity); - memory_storage.setAll(all_entities); + try + { + std::vector> all_entities; + for (const auto & entity : parseUsers(config)) + all_entities.emplace_back(generateID(*entity), entity); + for (const auto & entity : parseQuotas(config)) + all_entities.emplace_back(generateID(*entity), entity); + for (const auto & entity : parseRowPolicies(config)) + all_entities.emplace_back(generateID(*entity), entity); + for (const auto & entity : parseSettingsProfiles(config, check_setting_name_function)) + all_entities.emplace_back(generateID(*entity), entity); + memory_storage.setAll(all_entities); + } + catch (Exception & e) + { + e.addMessage(fmt::format("while loading {}", path.empty() ? "configuration" : ("configuration file " + quoteString(path)))); + throw; + } } void UsersConfigAccessStorage::load( @@ -528,29 +560,21 @@ void UsersConfigAccessStorage::load( const String & preprocessed_dir, const zkutil::GetZooKeeper & get_zookeeper_function) { - try - { - std::lock_guard lock{load_mutex}; - path = std::filesystem::path{users_config_path}.lexically_normal(); - config_reloader.reset(); - config_reloader = std::make_unique( - users_config_path, - include_from_path, - preprocessed_dir, - zkutil::ZooKeeperNodeCache(get_zookeeper_function), - std::make_shared(), - [&](Poco::AutoPtr new_config, bool /*initial_loading*/) - { - parseFromConfig(*new_config); - Settings::checkNoSettingNamesAtTopLevel(*new_config, users_config_path); - }, - /* already_loaded = */ false); - } - catch (Exception & e) - { - e.addMessage(fmt::format("while loading configuration file '{}'", users_config_path)); - throw; - } + std::lock_guard lock{load_mutex}; + path = std::filesystem::path{users_config_path}.lexically_normal(); + config_reloader.reset(); + config_reloader = std::make_unique( + users_config_path, + include_from_path, + preprocessed_dir, + zkutil::ZooKeeperNodeCache(get_zookeeper_function), + std::make_shared(), + [&](Poco::AutoPtr new_config, bool /*initial_loading*/) + { + parseFromConfig(*new_config); + Settings::checkNoSettingNamesAtTopLevel(*new_config, users_config_path); + }, + /* already_loaded = */ false); } void UsersConfigAccessStorage::reload() From 71a691e9ab420f34f80133987793497b91d8f336 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 Nov 2021 14:27:34 +0300 Subject: [PATCH 180/200] Make 01114_database_atomic more stable in debug builds By reducing number of clickhouse-client invocation, since it is pretty heavy in debug builds. --- .../0_stateless/01114_database_atomic.sh | 53 +++++++++++-------- 1 file changed, 30 insertions(+), 23 deletions(-) diff --git a/tests/queries/0_stateless/01114_database_atomic.sh b/tests/queries/0_stateless/01114_database_atomic.sh index 4d77a53cf6a..ae592740551 100755 --- a/tests/queries/0_stateless/01114_database_atomic.sh +++ b/tests/queries/0_stateless/01114_database_atomic.sh @@ -6,10 +6,11 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CURDIR"/../shell_config.sh -$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_1" -$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_2" -$CLICKHOUSE_CLIENT -q "DROP DATABASE IF EXISTS test_01114_3" - +$CLICKHOUSE_CLIENT -nm -q " +DROP DATABASE IF EXISTS test_01114_1; +DROP DATABASE IF EXISTS test_01114_2; +DROP DATABASE IF EXISTS test_01114_3; +" $CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_1 ENGINE=Atomic" $CLICKHOUSE_CLIENT --default_database_engine=Atomic -q "CREATE DATABASE test_01114_2" @@ -27,17 +28,19 @@ $CLICKHOUSE_CLIENT -q "SELECT name, splitByChar('/', metadata_path)[-2] as uuid_path, ((splitByChar('/', metadata_path)[-3] as metadata) = substr(uuid_path, 1, 3)) OR metadata='metadata' FROM system.databases WHERE name LIKE 'test_01114_%'" | sed "s/$uuid_db_1/00001114-1000-4000-8000-000000000001/g" | sed "s/$uuid_db_2/00001114-1000-4000-8000-000000000002/g" -$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_1.mt_tmp (n UInt64) ENGINE=MergeTree() ORDER BY tuple()" -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_1.mt_tmp SELECT * FROM numbers(100)" -$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_3.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)" -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_3.mt SELECT * FROM numbers(20)" +$CLICKHOUSE_CLIENT -nm -q " +CREATE TABLE test_01114_1.mt_tmp (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); +INSERT INTO test_01114_1.mt_tmp SELECT * FROM numbers(100); +CREATE TABLE test_01114_3.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5); +INSERT INTO test_01114_3.mt SELECT * FROM numbers(20); -$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_1.mt_tmp TO test_01114_3.mt_tmp" # move from Atomic to Ordinary -$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_3.mt TO test_01114_1.mt" # move from Ordinary to Atomic -$CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt" -$CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_3.mt_tmp" +RENAME TABLE test_01114_1.mt_tmp TO test_01114_3.mt_tmp; /* move from Atomic to Ordinary */ +RENAME TABLE test_01114_3.mt TO test_01114_1.mt; /* move from Ordinary to Atomic */ +SELECT count() FROM test_01114_1.mt; +SELECT count() FROM test_01114_3.mt_tmp; -$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_3" +DROP DATABASE test_01114_3; +" explicit_uuid=$($CLICKHOUSE_CLIENT -q "SELECT generateUUIDv4()") $CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_2.mt UUID '$explicit_uuid' (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)" @@ -49,11 +52,13 @@ $CLICKHOUSE_CLIENT -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow $CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30)" & # 45s sleep 1 # SELECT and INSERT should start before the following RENAMEs -$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_1.mt TO test_01114_1.mt_tmp" -$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_1.mt_tmp TO test_01114_2.mt_tmp" -$CLICKHOUSE_CLIENT -q "EXCHANGE TABLES test_01114_2.mt AND test_01114_2.mt_tmp" -$CLICKHOUSE_CLIENT -q "RENAME TABLE test_01114_2.mt_tmp TO test_01114_1.mt" -$CLICKHOUSE_CLIENT -q "EXCHANGE TABLES test_01114_1.mt AND test_01114_2.mt" +$CLICKHOUSE_CLIENT -nm -q " +RENAME TABLE test_01114_1.mt TO test_01114_1.mt_tmp; +RENAME TABLE test_01114_1.mt_tmp TO test_01114_2.mt_tmp; +EXCHANGE TABLES test_01114_2.mt AND test_01114_2.mt_tmp; +RENAME TABLE test_01114_2.mt_tmp TO test_01114_1.mt; +EXCHANGE TABLES test_01114_1.mt AND test_01114_2.mt; +" # Check that nothing changed $CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt" @@ -61,16 +66,18 @@ uuid_mt1=$($CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.tables WHERE database= $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE test_01114_1.mt" | sed "s/$uuid_mt1/00001114-0000-4000-8000-000000000001/g" $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE test_01114_2.mt" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g" -$CLICKHOUSE_CLIENT -q "DROP TABLE test_01114_1.mt" --database_atomic_wait_for_drop_and_detach_synchronously=0 -$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_1.mt (s String) ENGINE=Log()" -$CLICKHOUSE_CLIENT -q "INSERT INTO test_01114_1.mt SELECT 's' || toString(number) FROM numbers(5)" -$CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt" # result: 5 +$CLICKHOUSE_CLIENT -nm -q " +DROP TABLE test_01114_1.mt SETTINGS database_atomic_wait_for_drop_and_detach_synchronously=0; +CREATE TABLE test_01114_1.mt (s String) ENGINE=Log(); +INSERT INTO test_01114_1.mt SELECT 's' || toString(number) FROM numbers(5); +SELECT count() FROM test_01114_1.mt +" # result: 5 $CLICKHOUSE_CLIENT -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt" > /dev/null & # 15s sleep 1 $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_1" --database_atomic_wait_for_drop_and_detach_synchronously=0 && echo "dropped" -wait # for INSERT +wait # for INSERT and SELECT $CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01114_2.mt" # result: 30, 435 $CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_2" --database_atomic_wait_for_drop_and_detach_synchronously=0 From edbeeaf6ec0b37e2f5009d9e2a1baea56d8b6634 Mon Sep 17 00:00:00 2001 From: Kevin Michel Date: Fri, 19 Nov 2021 13:31:52 +0100 Subject: [PATCH 181/200] Fix invalid JSON in column names If the column name contains invalid UTF-8 sequences and the output data types are all considered safe, then the output will not be sanitized and the generated JSON will be invalid. A minimal reproduction case is : `SELECT length('\x80') FORMAT JSONCompact` where we auto-generate a non-UTF-8 column name with only integer outputs, whereas : `SELECT '\x80' FORMAT JSONCompact` would be sanitized because the column type is String and will trigger UTF-8 sanitization over the entire document. --- .../Formats/Impl/JSONRowOutputFormat.cpp | 6 ++++-- .../02111_json_column_name_encoding.reference | 16 ++++++++++++++++ .../02111_json_column_name_encoding.sql | 7 +++++++ 3 files changed, 27 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02111_json_column_name_encoding.reference create mode 100644 tests/queries/0_stateless/02111_json_column_name_encoding.sql diff --git a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp index 28d774fea74..f867a0bc49b 100644 --- a/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp +++ b/src/Processors/Formats/Impl/JSONRowOutputFormat.cpp @@ -26,8 +26,10 @@ JSONRowOutputFormat::JSONRowOutputFormat( need_validate_utf8 = true; WriteBufferFromOwnString buf; - writeJSONString(fields[i].name, buf, settings); - + { + WriteBufferValidUTF8 validating_buf(buf); + writeJSONString(fields[i].name, validating_buf, settings); + } fields[i].name = buf.str(); } diff --git a/tests/queries/0_stateless/02111_json_column_name_encoding.reference b/tests/queries/0_stateless/02111_json_column_name_encoding.reference new file mode 100644 index 00000000000..dd1bf2f5982 --- /dev/null +++ b/tests/queries/0_stateless/02111_json_column_name_encoding.reference @@ -0,0 +1,16 @@ +{ + "meta": + [ + { + "name": "length('�')", + "type": "UInt64" + } + ], + + "data": + [ + ["1"] + ], + + "rows": 1 +} diff --git a/tests/queries/0_stateless/02111_json_column_name_encoding.sql b/tests/queries/0_stateless/02111_json_column_name_encoding.sql new file mode 100644 index 00000000000..69af7507295 --- /dev/null +++ b/tests/queries/0_stateless/02111_json_column_name_encoding.sql @@ -0,0 +1,7 @@ +-- Tags: no-fasttest + +SET output_format_write_statistics = 0; + +SELECT + length('\x80') + FORMAT JSONCompact; From 6aa94ae032c24b6a8be5d8522573972dbfc125c6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 Nov 2021 15:23:37 +0300 Subject: [PATCH 182/200] Fix MySQLWire format (in case of multiple writes) In case of multiple writes File() engine will set doNotWritePrefix(), and this will avoid serializations initilization, move this to do this always. Fixes: #31004 --- .../Formats/Impl/MySQLOutputFormat.cpp | 16 +++++++++------- .../02123_MySQLWire_regression.reference | 0 .../0_stateless/02123_MySQLWire_regression.sql | 6 ++++++ 3 files changed, 15 insertions(+), 7 deletions(-) create mode 100644 tests/queries/0_stateless/02123_MySQLWire_regression.reference create mode 100644 tests/queries/0_stateless/02123_MySQLWire_regression.sql diff --git a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp index ab5d5e8399b..5033176ca4b 100644 --- a/src/Processors/Formats/Impl/MySQLOutputFormat.cpp +++ b/src/Processors/Formats/Impl/MySQLOutputFormat.cpp @@ -24,6 +24,15 @@ MySQLOutputFormat::MySQLOutputFormat(WriteBuffer & out_, const Block & header_, /// But it's also possible to specify MySQLWire as output format for clickhouse-client or clickhouse-local. /// There is no `sequence_id` stored in `settings_.mysql_wire` in this case, so we create a dummy one. sequence_id = settings_.mysql_wire.sequence_id ? settings_.mysql_wire.sequence_id : &dummy_sequence_id; + + const auto & header = getPort(PortKind::Main).getHeader(); + data_types = header.getDataTypes(); + + serializations.reserve(data_types.size()); + for (const auto & type : data_types) + serializations.emplace_back(type->getDefaultSerialization()); + + packet_endpoint = MySQLProtocol::PacketEndpoint::create(out, *sequence_id); } void MySQLOutputFormat::setContext(ContextPtr context_) @@ -34,13 +43,6 @@ void MySQLOutputFormat::setContext(ContextPtr context_) void MySQLOutputFormat::writePrefix() { const auto & header = getPort(PortKind::Main).getHeader(); - data_types = header.getDataTypes(); - - serializations.reserve(data_types.size()); - for (const auto & type : data_types) - serializations.emplace_back(type->getDefaultSerialization()); - - packet_endpoint = MySQLProtocol::PacketEndpoint::create(out, *sequence_id); if (header.columns()) { diff --git a/tests/queries/0_stateless/02123_MySQLWire_regression.reference b/tests/queries/0_stateless/02123_MySQLWire_regression.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02123_MySQLWire_regression.sql b/tests/queries/0_stateless/02123_MySQLWire_regression.sql new file mode 100644 index 00000000000..504d2f2a521 --- /dev/null +++ b/tests/queries/0_stateless/02123_MySQLWire_regression.sql @@ -0,0 +1,6 @@ +DROP TABLE IF EXISTS table_MySQLWire; +CREATE TABLE table_MySQLWire (x UInt64) ENGINE = File(MySQLWire); +INSERT INTO table_MySQLWire SELECT number FROM numbers(10); +-- regression for not initializing serializations +INSERT INTO table_MySQLWire SELECT number FROM numbers(10); +DROP TABLE table_MySQLWire; From e898c6608cc47158c0f1cc08092e4d5082dab532 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 20 Nov 2021 15:58:44 +0300 Subject: [PATCH 183/200] Fix broken non-x86 builds --- src/Columns/ColumnDecimal.cpp | 2 +- src/Columns/ColumnFixedString.cpp | 2 +- src/Columns/ColumnVector.cpp | 2 +- src/Columns/ColumnsCommon.cpp | 2 +- src/Columns/ColumnsCommon.h | 8 +++----- 5 files changed, 7 insertions(+), 9 deletions(-) diff --git a/src/Columns/ColumnDecimal.cpp b/src/Columns/ColumnDecimal.cpp index 48f890edaaf..70e2b4a6d96 100644 --- a/src/Columns/ColumnDecimal.cpp +++ b/src/Columns/ColumnDecimal.cpp @@ -247,7 +247,7 @@ ColumnPtr ColumnDecimal::filter(const IColumn::Filter & filt, ssize_t result_ while (filt_pos < filt_end_aligned) { - UInt64 mask = Bytes64MaskToBits64Mask(filt_pos); + UInt64 mask = bytes64MaskToBits64Mask(filt_pos); if (0xffffffffffffffff == mask) { diff --git a/src/Columns/ColumnFixedString.cpp b/src/Columns/ColumnFixedString.cpp index cc4e7b12a04..1eb2d4d5b1f 100644 --- a/src/Columns/ColumnFixedString.cpp +++ b/src/Columns/ColumnFixedString.cpp @@ -242,7 +242,7 @@ ColumnPtr ColumnFixedString::filter(const IColumn::Filter & filt, ssize_t result while (filt_pos < filt_end_aligned) { - uint64_t mask = Bytes64MaskToBits64Mask(filt_pos); + uint64_t mask = bytes64MaskToBits64Mask(filt_pos); if (0xffffffffffffffff == mask) { diff --git a/src/Columns/ColumnVector.cpp b/src/Columns/ColumnVector.cpp index b9079857aa4..13ba522b2ac 100644 --- a/src/Columns/ColumnVector.cpp +++ b/src/Columns/ColumnVector.cpp @@ -321,7 +321,7 @@ ColumnPtr ColumnVector::filter(const IColumn::Filter & filt, ssize_t result_s while (filt_pos < filt_end_aligned) { - UInt64 mask = Bytes64MaskToBits64Mask(filt_pos); + UInt64 mask = bytes64MaskToBits64Mask(filt_pos); if (0xffffffffffffffff == mask) { diff --git a/src/Columns/ColumnsCommon.cpp b/src/Columns/ColumnsCommon.cpp index 6d16790e2f8..701b888fb25 100644 --- a/src/Columns/ColumnsCommon.cpp +++ b/src/Columns/ColumnsCommon.cpp @@ -235,7 +235,7 @@ namespace while (filt_pos < filt_end_aligned) { - uint64_t mask = Bytes64MaskToBits64Mask(filt_pos); + uint64_t mask = bytes64MaskToBits64Mask(filt_pos); if (0xffffffffffffffff == mask) { diff --git a/src/Columns/ColumnsCommon.h b/src/Columns/ColumnsCommon.h index b84b1de7d67..0c307cac291 100644 --- a/src/Columns/ColumnsCommon.h +++ b/src/Columns/ColumnsCommon.h @@ -21,7 +21,7 @@ namespace ErrorCodes } /// Transform 64-byte mask to 64-bit mask -inline UInt64 Bytes64MaskToBits64Mask(const UInt8 * bytes64) +inline UInt64 bytes64MaskToBits64Mask(const UInt8 * bytes64) { #if defined(__AVX512F__) && defined(__AVX512BW__) static const __m512i zero64 = _mm512_setzero_epi32(); @@ -46,10 +46,8 @@ inline UInt64 Bytes64MaskToBits64Mask(const UInt8 * bytes64) _mm_loadu_si128(reinterpret_cast(bytes64 + 48)), zero16))) << 48) & 0xffff000000000000); #else UInt64 res = 0; - const UInt8 * pos = bytes64; - const UInt8 * end = pos + 64; - for (; pos < end; ++pos) - res |= ((*pos == 0)<<(pos-bytes64)); + for (size_t i = 0; i < 64; ++i) + res |= static_cast(0 == bytes64[i]) << i; #endif return ~res; } From 856502fa81051a18655cae6cdd9b3f0a7aedd557 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 20 Nov 2021 20:35:41 +0300 Subject: [PATCH 184/200] log long operations in Keeper --- src/Common/ZooKeeper/ZooKeeperCommon.cpp | 69 ++++++++++++++++++------ src/Common/ZooKeeper/ZooKeeperCommon.h | 9 +++- src/Coordination/KeeperDispatcher.cpp | 3 ++ src/Server/KeeperTCPHandler.cpp | 16 ++++++ 4 files changed, 80 insertions(+), 17 deletions(-) diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.cpp b/src/Common/ZooKeeper/ZooKeeperCommon.cpp index bc8fe0fe365..6a449cf0122 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.cpp +++ b/src/Common/ZooKeeper/ZooKeeperCommon.cpp @@ -489,20 +489,20 @@ void ZooKeeperMultiResponse::writeImpl(WriteBuffer & out) const } } -ZooKeeperResponsePtr ZooKeeperHeartbeatRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperSyncRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperCreateRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperListRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperCheckRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperMultiRequest::makeResponse() const { return std::make_shared(requests); } -ZooKeeperResponsePtr ZooKeeperCloseRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperSetACLRequest::makeResponse() const { return std::make_shared(); } -ZooKeeperResponsePtr ZooKeeperGetACLRequest::makeResponse() const { return std::make_shared(); } +ZooKeeperResponsePtr ZooKeeperHeartbeatRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperSyncRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperAuthRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperCreateRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperRemoveRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperExistsRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperGetRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperSetRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperListRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperCheckRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperMultiRequest::makeResponse() const { return setTime(std::make_shared(requests)); } +ZooKeeperResponsePtr ZooKeeperCloseRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperSetACLRequest::makeResponse() const { return setTime(std::make_shared()); } +ZooKeeperResponsePtr ZooKeeperGetACLRequest::makeResponse() const { return setTime(std::make_shared()); } void ZooKeeperSessionIDRequest::writeImpl(WriteBuffer & out) const { @@ -690,6 +690,40 @@ std::shared_ptr ZooKeeperRequest::read(ReadBuffer & in) return request; } +ZooKeeperRequest::~ZooKeeperRequest() +{ + if (!request_created_time_ns) + return; + UInt64 elapsed_ns = clock_gettime_ns() - request_created_time_ns; + constexpr UInt64 max_request_time_ns = 1000000000ULL; /// 1 sec + if (max_request_time_ns < elapsed_ns) + { + LOG_TEST(&Poco::Logger::get(__PRETTY_FUNCTION__), "Processing of request xid={} took {} ms", xid, elapsed_ns / 1000000UL); + } +} + +ZooKeeperResponsePtr ZooKeeperRequest::setTime(ZooKeeperResponsePtr response) const +{ + if (request_created_time_ns) + { + response->response_created_time_ns = clock_gettime_ns(); + } + return response; +} + +ZooKeeperResponse::~ZooKeeperResponse() +{ + if (!response_created_time_ns) + return; + UInt64 elapsed_ns = clock_gettime_ns() - response_created_time_ns; + constexpr UInt64 max_request_time_ns = 1000000000ULL; /// 1 sec + if (max_request_time_ns < elapsed_ns) + { + LOG_TEST(&Poco::Logger::get(__PRETTY_FUNCTION__), "Processing of response xid={} took {} ms", xid, elapsed_ns / 1000000UL); + } +} + + ZooKeeperRequestPtr ZooKeeperRequestFactory::get(OpNum op_num) const { auto it = op_num_to_request.find(op_num); @@ -708,7 +742,12 @@ ZooKeeperRequestFactory & ZooKeeperRequestFactory::instance() template void registerZooKeeperRequest(ZooKeeperRequestFactory & factory) { - factory.registerRequest(num, [] { return std::make_shared(); }); + factory.registerRequest(num, [] + { + auto res = std::make_shared(); + res->request_created_time_ns = clock_gettime_ns(); + return res; + }); } ZooKeeperRequestFactory::ZooKeeperRequestFactory() diff --git a/src/Common/ZooKeeper/ZooKeeperCommon.h b/src/Common/ZooKeeper/ZooKeeperCommon.h index a51ca9a487e..237c98d8497 100644 --- a/src/Common/ZooKeeper/ZooKeeperCommon.h +++ b/src/Common/ZooKeeper/ZooKeeperCommon.h @@ -30,9 +30,11 @@ struct ZooKeeperResponse : virtual Response XID xid = 0; int64_t zxid = 0; + UInt64 response_created_time_ns = 0; + ZooKeeperResponse() = default; ZooKeeperResponse(const ZooKeeperResponse &) = default; - virtual ~ZooKeeperResponse() override = default; + ~ZooKeeperResponse() override; virtual void readImpl(ReadBuffer &) = 0; virtual void writeImpl(WriteBuffer &) const = 0; virtual void write(WriteBuffer & out) const; @@ -54,9 +56,11 @@ struct ZooKeeperRequest : virtual Request bool restored_from_zookeeper_log = false; + UInt64 request_created_time_ns = 0; + ZooKeeperRequest() = default; ZooKeeperRequest(const ZooKeeperRequest &) = default; - virtual ~ZooKeeperRequest() override = default; + ~ZooKeeperRequest() override; virtual OpNum getOpNum() const = 0; @@ -69,6 +73,7 @@ struct ZooKeeperRequest : virtual Request static std::shared_ptr read(ReadBuffer & in); virtual ZooKeeperResponsePtr makeResponse() const = 0; + ZooKeeperResponsePtr setTime(ZooKeeperResponsePtr response) const; virtual bool isReadRequest() const = 0; virtual void createLogElements(LogElements & elems) const; diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index c29ea4fac45..e6fe1a9834a 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -206,7 +206,10 @@ void KeeperDispatcher::setResponse(int64_t session_id, const Coordination::ZooKe /// Session was disconnected, just skip this response if (session_response_callback == session_to_response_callback.end()) + { + LOG_TEST(log, "Cannot write response xid={}, op={}, session {} disconnected", response->xid, response->getOpNum(), session_id); return; + } session_response_callback->second(response); diff --git a/src/Server/KeeperTCPHandler.cpp b/src/Server/KeeperTCPHandler.cpp index 17a13955043..afcf03b2574 100644 --- a/src/Server/KeeperTCPHandler.cpp +++ b/src/Server/KeeperTCPHandler.cpp @@ -333,6 +333,16 @@ void KeeperTCPHandler::runImpl() }; keeper_dispatcher->registerSession(session_id, response_callback); + Stopwatch logging_stopwatch; + auto log_long_operation = [&](const String & operation) + { + constexpr UInt64 operation_max_ms = 500; + auto elapsed_ms = logging_stopwatch.elapsedMilliseconds(); + if (operation_max_ms < elapsed_ms) + LOG_TEST(log, "{} for session {} took {} ms", operation, session_id, elapsed_ms); + logging_stopwatch.restart(); + }; + session_stopwatch.start(); bool close_received = false; try @@ -342,9 +352,11 @@ void KeeperTCPHandler::runImpl() using namespace std::chrono_literals; PollResult result = poll_wrapper->poll(session_timeout, in); + log_long_operation("Polling socket"); if (result.has_requests && !close_received) { auto [received_op, received_xid] = receiveRequest(); + log_long_operation("Receiving request"); if (received_op == Coordination::OpNum::Close) { @@ -370,6 +382,7 @@ void KeeperTCPHandler::runImpl() if (!responses->tryPop(response)) throw Exception(ErrorCodes::LOGICAL_ERROR, "We must have ready response, but queue is empty. It's a bug."); + log_long_operation("Waiting for response to be ready"); if (response->xid == close_xid) { @@ -378,6 +391,7 @@ void KeeperTCPHandler::runImpl() } response->write(*out); + log_long_operation("Sending response"); if (response->error == Coordination::Error::ZSESSIONEXPIRED) { LOG_DEBUG(log, "Session #{} expired because server shutting down or quorum is not alive", session_id); @@ -401,6 +415,8 @@ void KeeperTCPHandler::runImpl() } catch (const Exception & ex) { + log_long_operation("Unknown operation"); + LOG_TRACE(log, "Has {} responses in the queue", responses->size()); LOG_INFO(log, "Got exception processing session #{}: {}", session_id, getExceptionMessage(ex, true)); keeper_dispatcher->finishSession(session_id); } From 57ef950b613296c9312e773b0426def8ad7dc570 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 21 Nov 2021 01:24:28 +0300 Subject: [PATCH 185/200] Add one more hardware benchmark result --- .../hardware/results/aws_c6g_2xlarge.json | 54 +++++++++++++++++++ 1 file changed, 54 insertions(+) create mode 100644 website/benchmark/hardware/results/aws_c6g_2xlarge.json diff --git a/website/benchmark/hardware/results/aws_c6g_2xlarge.json b/website/benchmark/hardware/results/aws_c6g_2xlarge.json new file mode 100644 index 00000000000..e970e73b919 --- /dev/null +++ b/website/benchmark/hardware/results/aws_c6g_2xlarge.json @@ -0,0 +1,54 @@ +[ + { + "system": "AWS c6g.2xlarge (Graviton 2)", + "system_full": "AWS c6g.2xlarge (Graviton 2) 8 vCPU, 16 GiB RAM, EBS", + "time": "2021-11-21 00:00:00", + "kind": "cloud", + "result": + [ +[0.002, 0.001, 0.001], +[0.035, 0.022, 0.022], +[0.084, 0.045, 0.045], +[0.745, 0.056, 0.056], +[1.397, 0.126, 0.125], +[2.237, 0.498, 0.499], +[0.063, 0.052, 0.052], +[0.025, 0.023, 0.023], +[1.295, 0.544, 0.542], +[1.655, 0.656, 0.655], +[0.769, 0.317, 0.316], +[1.043, 0.362, 0.360], +[1.938, 0.861, 0.854], +[3.563, 1.131, 1.104], +[2.152, 1.036, 1.014], +[0.967, 0.917, 0.900], +[4.259, 2.678, 2.724], +[3.342, 1.741, 1.704], +[8.303, 4.812, 4.794], +[0.680, 0.079, 0.074], +[18.753, 1.239, 1.224], +[20.694, 1.362, 1.333], +[38.987, 2.967, 2.937], +[31.357, 1.420, 1.404], +[4.471, 0.421, 0.413], +[1.633, 0.360, 0.358], +[4.554, 0.423, 0.417], +[18.076, 1.133, 1.118], +[15.164, 1.762, 1.747], +[0.678, 0.656, 0.651], +[3.504, 0.829, 0.817], +[9.359, 1.104, 1.088], +[8.794, 5.886, 5.848], +[19.039, 4.025, 4.007], +[19.061, 4.015, 4.053], +[1.289, 1.194, 1.194], +[0.287, 0.217, 0.208], +[0.113, 0.094, 0.092], +[0.111, 0.084, 0.086], +[0.539, 0.447, 0.439], +[0.072, 0.033, 0.025], +[0.042, 0.019, 0.039], +[0.005, 0.011, 0.004] + ] + } +] From 7eeffc233a677216d88c9bfab59e8877e8ea5188 Mon Sep 17 00:00:00 2001 From: Natasha Murashkina Date: Sun, 21 Nov 2021 16:07:38 +0300 Subject: [PATCH 186/200] Update developer-instruction.md Added the missing --recursive flag for cloning submodules. The text says "specify the `--recursive` flag as in the example above" but the flag wasn't there in the example --- docs/en/development/developer-instruction.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/development/developer-instruction.md b/docs/en/development/developer-instruction.md index e4b51242be8..52fa307333c 100644 --- a/docs/en/development/developer-instruction.md +++ b/docs/en/development/developer-instruction.md @@ -37,7 +37,7 @@ Next, you need to download the source files onto your working machine. This is c In the command line terminal run: - git clone git@github.com:your_github_username/ClickHouse.git + git clone --recursive git@github.com:your_github_username/ClickHouse.git cd ClickHouse Note: please, substitute *your_github_username* with what is appropriate! @@ -65,7 +65,7 @@ It generally means that the SSH keys for connecting to GitHub are missing. These You can also clone the repository via https protocol: - git clone https://github.com/ClickHouse/ClickHouse.git + git clone --recursive https://github.com/ClickHouse/ClickHouse.git This, however, will not let you send your changes to the server. You can still use it temporarily and add the SSH keys later replacing the remote address of the repository with `git remote` command. From afe8d3c4240bc04aa9671aebf1a8b7d88bd77cde Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sun, 21 Nov 2021 22:14:20 +0300 Subject: [PATCH 187/200] refactor merged index conditions --- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 75 +++++++++---------- .../MergeTree/MergeTreeDataSelectExecutor.h | 1 - .../MergeTree/MergeTreeIndexHypothesis.cpp | 8 ++ .../MergeTree/MergeTreeIndexHypothesis.h | 3 + ...rgeTreeIndexHypothesisMergedCondition.cpp} | 55 +++++++------- ...MergeTreeIndexHypothesisMergedCondition.h} | 30 +++----- src/Storages/MergeTree/MergeTreeIndices.h | 34 ++++++++- .../0_stateless/helpers/pure_http_client.py | 17 +++-- 8 files changed, 126 insertions(+), 97 deletions(-) rename src/Storages/MergeTree/{MergeTreeIndexMergedCondition.cpp => MergeTreeIndexHypothesisMergedCondition.cpp} (77%) rename src/Storages/MergeTree/{MergeTreeIndexMergedCondition.h => MergeTreeIndexHypothesisMergedCondition.h} (54%) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 899c1734dee..1c420664d6e 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -783,14 +783,19 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd /// Let's start analyzing all useful indices - struct DataSkippingIndexAndCondition + struct IndexStat { - MergeTreeIndexPtr index; - MergeTreeIndexConditionPtr condition; std::atomic total_granules{0}; std::atomic granules_dropped{0}; std::atomic total_parts{0}; std::atomic parts_dropped{0}; + }; + + struct DataSkippingIndexAndCondition + { + MergeTreeIndexPtr index; + MergeTreeIndexConditionPtr condition; + IndexStat stat; DataSkippingIndexAndCondition(MergeTreeIndexPtr index_, MergeTreeIndexConditionPtr condition_) : index(index_), condition(condition_) @@ -802,15 +807,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd { std::vector indices; MergeTreeIndexMergedConditionPtr condition; - std::atomic total_granules{0}; - std::atomic granules_dropped{0}; - std::atomic total_parts{0}; - std::atomic parts_dropped{0}; - - explicit MergedDataSkippingIndexAndCondition(MergeTreeIndexMergedConditionPtr condition_) - : condition(condition_) - { - } + IndexStat stat; void addIndex(const MergeTreeIndexPtr & index) { @@ -820,7 +817,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd }; std::list useful_indices; - std::unordered_map> merged_indices; + std::map, MergedDataSkippingIndexAndCondition> merged_indices; if (use_skip_indexes) { @@ -829,15 +826,11 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd auto index_helper = MergeTreeIndexFactory::instance().get(index); if (index_helper->isMergeable()) { - if (!merged_indices.contains(index_helper->getGranularity())) - { - merged_indices.emplace( - index_helper->getGranularity(), - std::make_shared( - std::make_shared(query_info, context, index_helper->getGranularity()))); - merged_indices.at(index_helper->getGranularity())->condition->addConstraints(metadata_snapshot->getConstraints()); - } - merged_indices.at(index_helper->getGranularity())->addIndex(index_helper); + auto [it, inserted] = merged_indices.try_emplace({index_helper->index.type, index_helper->getGranularity()}); + if (inserted) + it->second.condition = index_helper->createIndexMergedCondtition(query_info, metadata_snapshot); + + it->second.addIndex(index_helper); } else { @@ -921,7 +914,7 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd if (ranges.ranges.empty()) break; - index_and_condition.total_parts.fetch_add(1, std::memory_order_relaxed); + index_and_condition.stat.total_parts.fetch_add(1, std::memory_order_relaxed); size_t total_granules = 0; size_t granules_dropped = 0; @@ -938,34 +931,34 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd uncompressed_cache.get(), log); - index_and_condition.total_granules.fetch_add(total_granules, std::memory_order_relaxed); - index_and_condition.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); + index_and_condition.stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed); + index_and_condition.stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); if (ranges.ranges.empty()) - index_and_condition.parts_dropped.fetch_add(1, std::memory_order_relaxed); + index_and_condition.stat.parts_dropped.fetch_add(1, std::memory_order_relaxed); } - for (auto & [granularity, indices_and_condition] : merged_indices) + for (auto & [_, indices_and_condition] : merged_indices) { if (ranges.ranges.empty()) break; - indices_and_condition->total_parts.fetch_add(1, std::memory_order_relaxed); + indices_and_condition.stat.total_parts.fetch_add(1, std::memory_order_relaxed); size_t total_granules = 0; size_t granules_dropped = 0; ranges.ranges = filterMarksUsingMergedIndex( - indices_and_condition->indices, indices_and_condition->condition, + indices_and_condition.indices, indices_and_condition.condition, part, ranges.ranges, settings, reader_settings, total_granules, granules_dropped, mark_cache.get(), uncompressed_cache.get(), log); - indices_and_condition->total_granules.fetch_add(total_granules, std::memory_order_relaxed); - indices_and_condition->granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); + indices_and_condition.stat.total_granules.fetch_add(total_granules, std::memory_order_relaxed); + indices_and_condition.stat.granules_dropped.fetch_add(granules_dropped, std::memory_order_relaxed); if (ranges.ranges.empty()) - indices_and_condition->parts_dropped.fetch_add(1, std::memory_order_relaxed); + indices_and_condition.stat.parts_dropped.fetch_add(1, std::memory_order_relaxed); } if (!ranges.ranges.empty()) @@ -1046,8 +1039,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd log, "Index {} has dropped {}/{} granules.", backQuote(index_name), - index_and_condition.granules_dropped, - index_and_condition.total_granules); + index_and_condition.stat.granules_dropped, + index_and_condition.stat.total_granules); std::string description = index_and_condition.index->index.type + " GRANULARITY " + std::to_string(index_and_condition.index->index.granularity); @@ -1056,25 +1049,25 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd .type = ReadFromMergeTree::IndexType::Skip, .name = index_name, .description = std::move(description), //-V1030 - .num_parts_after = index_and_condition.total_parts - index_and_condition.parts_dropped, - .num_granules_after = index_and_condition.total_granules - index_and_condition.granules_dropped}); + .num_parts_after = index_and_condition.stat.total_parts - index_and_condition.stat.parts_dropped, + .num_granules_after = index_and_condition.stat.total_granules - index_and_condition.stat.granules_dropped}); } - for (const auto & [granularity, index_and_condition] : merged_indices) + for (const auto & [type_with_granularity, index_and_condition] : merged_indices) { const auto & index_name = "Merged"; LOG_DEBUG(log, "Index {} has dropped {}/{} granules.", backQuote(index_name), - index_and_condition->granules_dropped, index_and_condition->total_granules); + index_and_condition.stat.granules_dropped, index_and_condition.stat.total_granules); - std::string description = "MERGED GRANULARITY " + std::to_string(granularity); + std::string description = "MERGED GRANULARITY " + std::to_string(type_with_granularity.second); index_stats.emplace_back(ReadFromMergeTree::IndexStat{ .type = ReadFromMergeTree::IndexType::Skip, .name = index_name, .description = std::move(description), //-V1030 - .num_parts_after = index_and_condition->total_parts - index_and_condition->parts_dropped, - .num_granules_after = index_and_condition->total_granules - index_and_condition->granules_dropped}); + .num_parts_after = index_and_condition.stat.total_parts - index_and_condition.stat.parts_dropped, + .num_granules_after = index_and_condition.stat.total_granules - index_and_condition.stat.granules_dropped}); } return parts_with_ranges; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 9664e8f6335..f19d145fc93 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp index 26f534538fc..58629b63cda 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.cpp @@ -1,4 +1,5 @@ #include +#include #include #include @@ -88,6 +89,13 @@ MergeTreeIndexConditionPtr MergeTreeIndexHypothesis::createIndexCondition( throw Exception("Not supported", ErrorCodes::LOGICAL_ERROR); } +MergeTreeIndexMergedConditionPtr MergeTreeIndexHypothesis::createIndexMergedCondtition( + const SelectQueryInfo & query_info, StorageMetadataPtr storage_metadata) const +{ + return std::make_shared( + query_info, storage_metadata->getConstraints(), index.granularity); +} + bool MergeTreeIndexHypothesis::mayBenefitFromIndexForIn(const ASTPtr &) const { return false; diff --git a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h index 4849ba2b5dc..bbdf70a052c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexHypothesis.h +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesis.h @@ -70,6 +70,9 @@ public: MergeTreeIndexConditionPtr createIndexCondition( const SelectQueryInfo & query, ContextPtr context) const override; + MergeTreeIndexMergedConditionPtr createIndexMergedCondtition( + const SelectQueryInfo & query_info, StorageMetadataPtr storage_metadata) const override; + bool mayBenefitFromIndexForIn(const ASTPtr & node) const override; size_t max_rows = 0; diff --git a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp similarity index 77% rename from src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp rename to src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp index 2de96c76933..2fa0b98bc30 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -15,13 +15,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -MergeTreeIndexMergedCondition::MergeTreeIndexMergedCondition( - const SelectQueryInfo & query_, - ContextPtr /*context_*/, - const size_t granularity_) - : granularity(granularity_) +MergeTreeIndexhypothesisMergedCondition::MergeTreeIndexhypothesisMergedCondition( + const SelectQueryInfo & query, const ConstraintsDescription & constraints, size_t granularity_) + : IMergeTreeIndexMergedCondition(granularity_) { - const auto & select = query_.query->as(); + const auto & select = query.query->as(); if (select.where() && select.prewhere()) expression_ast = makeASTFunction( @@ -35,26 +33,26 @@ MergeTreeIndexMergedCondition::MergeTreeIndexMergedCondition( expression_cnf = std::make_unique( expression_ast ? TreeCNFConverter::toCNF(expression_ast) : CNFQuery::AndGroup{}); + + addConstraints(constraints); } -void MergeTreeIndexMergedCondition::addIndex(const MergeTreeIndexPtr & index) +void MergeTreeIndexhypothesisMergedCondition::addIndex(const MergeTreeIndexPtr & index) { if (!index->isMergeable() || index->getGranularity() != granularity) - throw Exception("Index can not be merged", - ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Index {} can not be merged", index->index.type); const auto hypothesis_index = std::dynamic_pointer_cast(index); if (!hypothesis_index) - throw Exception( - "Only hypothesis index is supported here.", ErrorCodes::LOGICAL_ERROR); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Only hypothesis index is supported here"); - static const std::set relations = { - "equals", "notEquals", "less", "lessOrEquals", "greaterOrEquals", "greater"}; + static const NameSet relations = { "equals", "notEquals", "less", "lessOrEquals", "greaterOrEquals", "greater"}; // TODO: move to index hypothesis std::vector compare_hypotheses_data; std::vector hypotheses_data; const auto cnf = TreeCNFConverter::toCNF(hypothesis_index->index.expression_list_ast->children.front()).pullNotOutFunctions(); + for (const auto & group : cnf.getStatements()) { if (group.size() == 1) @@ -63,19 +61,19 @@ void MergeTreeIndexMergedCondition::addIndex(const MergeTreeIndexPtr & index) CNFQuery::AtomicFormula atomic_formula = *group.begin(); CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()}; pushNotIn(atom); - if (atom.negative) - throw Exception("negative atom", ErrorCodes::LOGICAL_ERROR); + assert(!atom.negative); - auto * func = atom.ast->as(); + const auto * func = atom.ast->as(); if (func && relations.count(func->name)) compare_hypotheses_data.push_back(atom.ast); } } + index_to_compare_atomic_hypotheses.push_back(compare_hypotheses_data); index_to_atomic_hypotheses.push_back(hypotheses_data); } -void MergeTreeIndexMergedCondition::addConstraints(const ConstraintsDescription & constraints_description) +void MergeTreeIndexhypothesisMergedCondition::addConstraints(const ConstraintsDescription & constraints_description) { auto atomic_constraints_data = constraints_description.getAtomicConstraintData(); for (const auto & atomic_formula : atomic_constraints_data) @@ -87,7 +85,7 @@ void MergeTreeIndexMergedCondition::addConstraints(const ConstraintsDescription } /// Replaces < -> <=, > -> >= and assumes that all hypotheses are true then checks if path exists -bool MergeTreeIndexMergedCondition::alwaysUnknownOrTrue() const +bool MergeTreeIndexhypothesisMergedCondition::alwaysUnknownOrTrue() const { std::vector active_atomic_formulas(atomic_constraints); for (const auto & hypothesis : index_to_compare_atomic_hypotheses) @@ -119,6 +117,7 @@ bool MergeTreeIndexMergedCondition::alwaysUnknownOrTrue() const { CNFQuery::AtomicFormula atom{atomic_formula.negative, atomic_formula.ast->clone()}; pushNotIn(atom); + const auto * func = atom.ast->as(); if (func && func->arguments->children.size() == 2) { @@ -135,7 +134,7 @@ bool MergeTreeIndexMergedCondition::alwaysUnknownOrTrue() const return useless; } -bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const +bool MergeTreeIndexhypothesisMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const { std::vector values; for (const auto & index_granule : granules) @@ -146,7 +145,7 @@ bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranu values.push_back(granule->met); } - if (const auto it = answerCache.find(values); it != std::end(answerCache)) + if (const auto it = answer_cache.find(values); it != std::end(answer_cache)) return it->second; const auto & graph = getGraph(values); @@ -177,11 +176,11 @@ bool MergeTreeIndexMergedCondition::mayBeTrueOnGranule(const MergeTreeIndexGranu always_false = true; }); - answerCache[values] = !always_false; + answer_cache[values] = !always_false; return !always_false; } -std::unique_ptr MergeTreeIndexMergedCondition::buildGraph(const std::vector & values) const +std::unique_ptr MergeTreeIndexhypothesisMergedCondition::buildGraph(const std::vector & values) const { std::vector active_atomic_formulas(atomic_constraints); for (size_t i = 0; i < values.size(); ++i) @@ -194,12 +193,12 @@ std::unique_ptr MergeTreeIndexMergedCondition::buildGraph(const } return std::make_unique(active_atomic_formulas); } -// something strange)) -const ComparisonGraph & MergeTreeIndexMergedCondition::getGraph(const std::vector & values) const + +const ComparisonGraph & MergeTreeIndexhypothesisMergedCondition::getGraph(const std::vector & values) const { - if (!graphCache.contains(values)) - graphCache[values] = buildGraph(values); - return *graphCache.at(values); + if (!graph_cache.contains(values)) + graph_cache[values] = buildGraph(values); + return *graph_cache.at(values); } } diff --git a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h similarity index 54% rename from src/Storages/MergeTree/MergeTreeIndexMergedCondition.h rename to src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h index 9a760797eb3..530e14e15cc 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMergedCondition.h +++ b/src/Storages/MergeTree/MergeTreeIndexHypothesisMergedCondition.h @@ -7,41 +7,31 @@ namespace DB { -/* - * IndexCondition checking several indexes at the same time. - * Works only for hypotheses. (will also support minmax soon). - */ -class MergeTreeIndexMergedCondition +/// MergedCondition for Indexhypothesis. +class MergeTreeIndexhypothesisMergedCondition : public IMergeTreeIndexMergedCondition { public: - MergeTreeIndexMergedCondition( - const SelectQueryInfo & query, - ContextPtr context, - const size_t granularity); + MergeTreeIndexhypothesisMergedCondition( + const SelectQueryInfo & query, const ConstraintsDescription & constraints, size_t granularity_); - void addIndex(const MergeTreeIndexPtr & index); - void addConstraints(const ConstraintsDescription & constraints_description); - - bool alwaysUnknownOrTrue() const; - bool mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const; + void addIndex(const MergeTreeIndexPtr & index) override; + bool alwaysUnknownOrTrue() const override; + bool mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const override; private: + void addConstraints(const ConstraintsDescription & constraints_description); std::unique_ptr buildGraph(const std::vector & values) const; const ComparisonGraph & getGraph(const std::vector & values) const; - const size_t granularity; ASTPtr expression_ast; std::unique_ptr expression_cnf; - mutable std::unordered_map, std::unique_ptr> graphCache; - mutable std::unordered_map, bool> answerCache; + mutable std::unordered_map, std::unique_ptr> graph_cache; + mutable std::unordered_map, bool> answer_cache; std::vector> index_to_compare_atomic_hypotheses; std::vector> index_to_atomic_hypotheses; std::vector atomic_constraints; }; -using MergeTreeIndexMergedConditionPtr = std::shared_ptr; -using MergeTreeIndexMergedConditions = std::vector; - } diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index 4fb8a70ef7a..d501b2525c2 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -89,6 +89,31 @@ public: using MergeTreeIndexConditionPtr = std::shared_ptr; using MergeTreeIndexConditions = std::vector; +struct IMergeTreeIndex; +using MergeTreeIndexPtr = std::shared_ptr; + +/// IndexCondition that checks several indexes at the same time. +class IMergeTreeIndexMergedCondition +{ +public: + explicit IMergeTreeIndexMergedCondition(size_t granularity_) + : granularity(granularity_) + { + } + + virtual ~IMergeTreeIndexMergedCondition() = default; + + virtual void addIndex(const MergeTreeIndexPtr & index) = 0; + virtual bool alwaysUnknownOrTrue() const = 0; + virtual bool mayBeTrueOnGranule(const MergeTreeIndexGranules & granules) const = 0; + +protected: + const size_t granularity; +}; + +using MergeTreeIndexMergedConditionPtr = std::shared_ptr; +using MergeTreeIndexMergedConditions = std::vector; + struct IMergeTreeIndex { @@ -130,7 +155,14 @@ struct IMergeTreeIndex virtual MergeTreeIndexAggregatorPtr createIndexAggregator() const = 0; virtual MergeTreeIndexConditionPtr createIndexCondition( - const SelectQueryInfo & query_info, ContextPtr context) const = 0; + const SelectQueryInfo & query_info, ContextPtr context) const = 0; + + virtual MergeTreeIndexMergedConditionPtr createIndexMergedCondtition( + const SelectQueryInfo & /*query_info*/, StorageMetadataPtr /*storage_metadata*/) const + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "MergedCondition is not implemented for index of type {}", index.type); + } Names getColumnsRequiredForIndexCalc() const { return index.expression->getRequiredColumns(); } diff --git a/tests/queries/0_stateless/helpers/pure_http_client.py b/tests/queries/0_stateless/helpers/pure_http_client.py index 4e18ab3a0f4..9f79c4ac529 100644 --- a/tests/queries/0_stateless/helpers/pure_http_client.py +++ b/tests/queries/0_stateless/helpers/pure_http_client.py @@ -1,6 +1,6 @@ -import os -import io -import sys +import os +import io +import sys import requests import time import pandas as pd @@ -8,6 +8,7 @@ import pandas as pd CLICKHOUSE_HOST = os.environ.get('CLICKHOUSE_HOST', '127.0.0.1') CLICKHOUSE_PORT_HTTP = os.environ.get('CLICKHOUSE_PORT_HTTP', '8123') CLICKHOUSE_SERVER_URL_STR = 'http://' + ':'.join(str(s) for s in [CLICKHOUSE_HOST, CLICKHOUSE_PORT_HTTP]) + "/" +CLICKHOUSE_DATABASE = os.environ.get('CLICKHOUSE_DATABASE', 'test') class ClickHouseClient: def __init__(self, host = CLICKHOUSE_SERVER_URL_STR): @@ -19,8 +20,12 @@ class ClickHouseClient: for i in range(NUMBER_OF_TRIES): r = requests.post( - self.host, - params = {'timeout_before_checking_execution_speed': 120, 'max_execution_time': 6000}, + self.host, + params = { + 'timeout_before_checking_execution_speed': 120, + 'max_execution_time': 6000, + 'database': CLICKHOUSE_DATABASE + }, timeout = connection_timeout, data = query) if r.status_code == 200: @@ -35,7 +40,7 @@ class ClickHouseClient: raise ValueError(r.text) def query_return_df(self, query, connection_timeout = 1500): - data = self.query(query, connection_timeout) + data = self.query(query, connection_timeout) df = pd.read_csv(io.StringIO(data), sep = '\t') return df From c034491da95909fa5a6b1a9a4f72a609fcf30d7c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sun, 21 Nov 2021 23:02:33 +0300 Subject: [PATCH 188/200] fix style check --- src/Storages/MergeTree/MergeTreeIndices.h | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/MergeTree/MergeTreeIndices.h b/src/Storages/MergeTree/MergeTreeIndices.h index d501b2525c2..8f10b2c51ba 100644 --- a/src/Storages/MergeTree/MergeTreeIndices.h +++ b/src/Storages/MergeTree/MergeTreeIndices.h @@ -18,6 +18,11 @@ constexpr auto INDEX_FILE_PREFIX = "skp_idx_"; namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + using MergeTreeIndexVersion = uint8_t; struct MergeTreeIndexFormat { From d958528d11cdc9e7f1d522d7822b038a58981a96 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 21 Nov 2021 23:15:26 +0300 Subject: [PATCH 189/200] Update build-cross-osx.md --- docs/en/development/build-cross-osx.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/development/build-cross-osx.md b/docs/en/development/build-cross-osx.md index 6f3b3a717d0..bd07e15ff03 100644 --- a/docs/en/development/build-cross-osx.md +++ b/docs/en/development/build-cross-osx.md @@ -9,14 +9,14 @@ This is for the case when you have Linux machine and want to use it to build `cl The cross-build for Mac OS X is based on the [Build instructions](../development/build.md), follow them first. -## Install Clang-8 {#install-clang-8} +## Install Clang-13 Follow the instructions from https://apt.llvm.org/ for your Ubuntu or Debian setup. For example the commands for Bionic are like: ``` bash -sudo echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-8 main" >> /etc/apt/sources.list -sudo apt-get install clang-8 +sudo echo "deb [trusted=yes] http://apt.llvm.org/bionic/ llvm-toolchain-bionic-13 main" >> /etc/apt/sources.list +sudo apt-get install clang-13 ``` ## Install Cross-Compilation Toolset {#install-cross-compilation-toolset} From fc194d8495b3c6854e580ea08aafa330d4c4c385 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 21 Nov 2021 23:15:41 +0300 Subject: [PATCH 190/200] Update build-cross-osx.md --- docs/en/development/build-cross-osx.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/build-cross-osx.md b/docs/en/development/build-cross-osx.md index bd07e15ff03..d53f18cb5ea 100644 --- a/docs/en/development/build-cross-osx.md +++ b/docs/en/development/build-cross-osx.md @@ -52,7 +52,7 @@ tar xJf MacOSX10.15.sdk.tar.xz -C build-darwin/cmake/toolchain/darwin-x86_64 --s ``` bash cd ClickHouse mkdir build-osx -CC=clang-8 CXX=clang++-8 cmake . -Bbuild-osx -DCMAKE_TOOLCHAIN_FILE=cmake/darwin/toolchain-x86_64.cmake \ +CC=clang-13 CXX=clang++-13 cmake . -Bbuild-osx -DCMAKE_TOOLCHAIN_FILE=cmake/darwin/toolchain-x86_64.cmake \ -DCMAKE_AR:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ar \ -DCMAKE_RANLIB:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ranlib \ -DLINKER_NAME=${CCTOOLS}/bin/x86_64-apple-darwin-ld From b6a01b994a37947ac9e2d66a54d362b075ac7294 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 21 Nov 2021 23:24:28 +0300 Subject: [PATCH 191/200] Update build-cross-osx.md --- docs/en/development/build-cross-osx.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/development/build-cross-osx.md b/docs/en/development/build-cross-osx.md index d53f18cb5ea..bd40a4aca1e 100644 --- a/docs/en/development/build-cross-osx.md +++ b/docs/en/development/build-cross-osx.md @@ -25,6 +25,7 @@ Let’s remember the path where we install `cctools` as ${CCTOOLS} ``` bash mkdir ${CCTOOLS} +cd ${CCTOOLS} git clone https://github.com/tpoechtrager/apple-libtapi.git cd apple-libtapi @@ -34,7 +35,7 @@ cd .. git clone https://github.com/tpoechtrager/cctools-port.git cd cctools-port/cctools -./configure --prefix=${CCTOOLS} --with-libtapi=${CCTOOLS} --target=x86_64-apple-darwin +./configure --prefix=$(readlink -f ${CCTOOLS}) --with-libtapi=$(readlink -f ${CCTOOLS}) --target=x86_64-apple-darwin make install ``` From 8e69c66c1bb1066eff05a26b5ac37d750a6cb659 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Sun, 21 Nov 2021 23:38:47 +0300 Subject: [PATCH 192/200] Update build-cross-osx.md --- docs/en/development/build-cross-osx.md | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/docs/en/development/build-cross-osx.md b/docs/en/development/build-cross-osx.md index bd40a4aca1e..c7e40013113 100644 --- a/docs/en/development/build-cross-osx.md +++ b/docs/en/development/build-cross-osx.md @@ -52,12 +52,10 @@ tar xJf MacOSX10.15.sdk.tar.xz -C build-darwin/cmake/toolchain/darwin-x86_64 --s ``` bash cd ClickHouse -mkdir build-osx -CC=clang-13 CXX=clang++-13 cmake . -Bbuild-osx -DCMAKE_TOOLCHAIN_FILE=cmake/darwin/toolchain-x86_64.cmake \ - -DCMAKE_AR:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ar \ - -DCMAKE_RANLIB:FILEPATH=${CCTOOLS}/bin/x86_64-apple-darwin-ranlib \ - -DLINKER_NAME=${CCTOOLS}/bin/x86_64-apple-darwin-ld -ninja -C build-osx +mkdir build-darwin +cd build-darwin +CC=clang-13 CXX=clang++-13 cmake -DCMAKE_AR:FILEPATH=${CCTOOLS}/bin/aarch64-apple-darwin-ar -DCMAKE_INSTALL_NAME_TOOL=${CCTOOLS}/bin/aarch64-apple-darwin-install_name_tool -DCMAKE_RANLIB:FILEPATH=${CCTOOLS}/bin/aarch64-apple-darwin-ranlib -DLINKER_NAME=${CCTOOLS}/bin/aarch64-apple-darwin-ld -DCMAKE_TOOLCHAIN_FILE=cmake/darwin/toolchain-x86_64.cmake .. +ninja ``` The resulting binary will have a Mach-O executable format and can’t be run on Linux. From 2f11ad7241eb8db60a17c2c4054ecbbfa27832f2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Nov 2021 11:33:52 +0300 Subject: [PATCH 193/200] Add workflow run for release branches --- .github/workflows/cancel.yml | 2 +- .github/workflows/release_branches.yml | 939 +++++++++++++++++++++++++ tests/ci/docker_images_check.py | 18 +- 3 files changed, 951 insertions(+), 8 deletions(-) create mode 100644 .github/workflows/release_branches.yml diff --git a/.github/workflows/cancel.yml b/.github/workflows/cancel.yml index 03e1007b841..9b96a1dbffa 100644 --- a/.github/workflows/cancel.yml +++ b/.github/workflows/cancel.yml @@ -1,7 +1,7 @@ name: Cancel on: # yamllint disable-line rule:truthy workflow_run: - workflows: ["CIGithubActions"] + workflows: ["CIGithubActions", "ReleaseCI"] types: - requested jobs: diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml new file mode 100644 index 00000000000..bfcb1b4b14d --- /dev/null +++ b/.github/workflows/release_branches.yml @@ -0,0 +1,939 @@ +name: ReleaseCI +on: # yamllint disable-line rule:truthy + pull_request: + branches: + - '21.**' + - '22.**' + - '23.**' + - '24.**' + push: + branches: + - '21.**' + - '22.**' + - '23.**' + - '24.**' +jobs: + DockerHubPush: + runs-on: [self-hosted, style-checker] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Images check + run: | + cd $GITHUB_WORKSPACE/tests/ci + python3 docker_images_check.py + - name: Upload images files to artifacts + uses: actions/upload-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/docker_images_check/changed_images.json + CompatibilityCheck: + needs: [BuilderDebRelease] + runs-on: [self-hosted, style-checker] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: CompatibilityCheck + env: + TEMP_PATH: ${{runner.temp}}/compatibility_check + REPO_COPY: ${{runner.temp}}/compatibility_check/ClickHouse + REPORTS_PATH: ${{runner.temp}}/reports_dir + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 compatibility_check.py 0 + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +######################################################################################### +#################################### ORDINARY BUILDS #################################### +######################################################################################### + BuilderDebRelease: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 0 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + + BuilderDebAsan: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 3 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + BuilderDebUBsan: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 4 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + BuilderDebTsan: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 5 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + BuilderDebMsan: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 6 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + BuilderDebDebug: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 7 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################ +##################################### BUILD REPORTER ####################################### +############################################################################################ + BuilderReport: + needs: + - BuilderDebRelease + - BuilderDebAsan + - BuilderDebTsan + - BuilderDebUBsan + - BuilderDebMsan + - BuilderDebDebug + runs-on: [self-hosted, style-checker] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Report Builder + env: + TEMP_PATH: ${{runner.temp}}/report_check + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'ClickHouse build check (actions)' + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cd $GITHUB_WORKSPACE/tests/ci + python3 build_report_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################## +########################### FUNCTIONAl STATELESS TESTS ####################################### +############################################################################################## + FunctionalStatelessTestRelease: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (release, actions)' + REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (address, actions)' + REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_tsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (thread, actions)' + REPO_COPY: ${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_ubsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (ubsan, actions)' + REPO_COPY: ${{runner.temp}}/stateless_ubsan/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_memory + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (memory, actions)' + REPO_COPY: ${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (debug, actions)' + REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################## +############################ FUNCTIONAl STATEFUL TESTS ####################################### +############################################################################################## + FunctionalStatefulTestRelease: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (release, actions)' + REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatefulTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (address, actions)' + REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatefulTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_tsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (thread, actions)' + REPO_COPY: ${{runner.temp}}/stateful_tsan/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatefulTestMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_msan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (memory, actions)' + REPO_COPY: ${{runner.temp}}/stateful_msan/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatefulTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_ubsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (ubsan, actions)' + REPO_COPY: ${{runner.temp}}/stateful_ubsan/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatefulTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (debug, actions)' + REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################## +######################################### STRESS TESTS ####################################### +############################################################################################## + StressTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + env: + TEMP_PATH: ${{runner.temp}}/stress_thread + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress test (address, actions)' + REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + StressTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + env: + TEMP_PATH: ${{runner.temp}}/stress_thread + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress test (thread, actions)' + REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + StressTestMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + env: + TEMP_PATH: ${{runner.temp}}/stress_memory + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress test (memory, actions)' + REPO_COPY: ${{runner.temp}}/stress_memory/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + StressTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + env: + TEMP_PATH: ${{runner.temp}}/stress_undefined + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress test (undefined, actions)' + REPO_COPY: ${{runner.temp}}/stress_undefined/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + StressTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + env: + TEMP_PATH: ${{runner.temp}}/stress_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress test (debug, actions)' + REPO_COPY: ${{runner.temp}}/stress_debug/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################# +############################# INTEGRATION TESTS ############################################# +############################################################################################# + IntegrationTestsAsan: + needs: [BuilderDebAsan, FunctionalStatelessTestAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + env: + TEMP_PATH: ${{runner.temp}}/integration_tests_asan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Integration tests (asan, actions)' + REPO_COPY: ${{runner.temp}}/integration_tests_asan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + IntegrationTestsTsan: + needs: [BuilderDebTsan, FunctionalStatelessTestTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + env: + TEMP_PATH: ${{runner.temp}}/integration_tests_tsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Integration tests (thread, actions)' + REPO_COPY: ${{runner.temp}}/integration_tests_tsan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + IntegrationTestsRelease: + needs: [BuilderDebRelease, FunctionalStatelessTestRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + env: + TEMP_PATH: ${{runner.temp}}/integration_tests_release + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Integration tests (release, actions)' + REPO_COPY: ${{runner.temp}}/integration_tests_release/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FinishCheck: + needs: + - DockerHubPush + - BuilderReport + - FunctionalStatelessTestDebug + - FunctionalStatelessTestRelease + - FunctionalStatelessTestAsan + - FunctionalStatelessTestTsan + - FunctionalStatelessTestMsan + - FunctionalStatelessTestUBsan + - FunctionalStatefulTestDebug + - FunctionalStatefulTestRelease + - FunctionalStatefulTestAsan + - FunctionalStatefulTestTsan + - FunctionalStatefulTestMsan + - FunctionalStatefulTestUBsan + - StressTestDebug + - StressTestAsan + - StressTestTsan + - StressTestMsan + - StressTestUBsan + - IntegrationTestsAsan + - IntegrationTestsRelease + - IntegrationTestsTsan + - CompatibilityCheck + runs-on: [self-hosted, style-checker] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Finish label + run: | + cd $GITHUB_WORKSPACE/tests/ci + python3 finish_check.py diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 0482f05f284..24d8dfe336b 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -39,13 +39,17 @@ def get_changed_docker_images(pr_info, repo_path, image_file_path): if image_description['name'].startswith('clickhouse/'): dockerhub_repo_name = 'clickhouse' - for f in files_changed: - if f.startswith(dockerfile_dir): - logging.info( - "Found changed file '%s' which affects docker image '%s' with path '%s'", - f, image_description['name'], dockerfile_dir) - changed_images.append(dockerfile_dir) - break + if 'release' in pr_info.labels: + logging.info("Release PR, will rebuild all images from branch, including %s", dockerfile_dir) + changed_images.append(dockerfile_dir) + else: + for f in files_changed: + if f.startswith(dockerfile_dir): + logging.info( + "Found changed file '%s' which affects docker image '%s' with path '%s'", + f, image_description['name'], dockerfile_dir) + changed_images.append(dockerfile_dir) + break # The order is important: dependents should go later than bases, so that # they are built with updated base versions. From d8e0b5a0a0a3383ed932b5b893dec54d839772cd Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Nov 2021 11:55:46 +0300 Subject: [PATCH 194/200] Other event for release PR --- .github/workflows/release_branches.yml | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index bfcb1b4b14d..8c4e671112f 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -1,17 +1,16 @@ name: ReleaseCI on: # yamllint disable-line rule:truthy pull_request: + types: + - synchronize + - reopened + - opened branches: - '21.**' - '22.**' - '23.**' - '24.**' - push: - branches: - - '21.**' - - '22.**' - - '23.**' - - '24.**' + - 'backport/**' jobs: DockerHubPush: runs-on: [self-hosted, style-checker] From 6f5f72738be15de34cb5f2aee5da97564691ecf3 Mon Sep 17 00:00:00 2001 From: Vitaly Baranov Date: Sun, 21 Nov 2021 18:01:28 +0300 Subject: [PATCH 195/200] Fix cleanup in integration tests. --- tests/integration/helpers/cluster.py | 32 ++++++++++++++++++---------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 6ab8b29895b..07059aa97a9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -422,20 +422,17 @@ class ClickHouseCluster: # Just in case kill unstopped containers from previous launch try: - # docker-compose names containers using the following formula: - # container_name = project_name + '_' + instance_name + '_1' - # We need to have "^/" and "$" in the "--filter name" option below to filter by exact name of the container, see - # https://stackoverflow.com/questions/48767760/how-to-make-docker-container-ls-f-name-filter-by-exact-name - filter_name = f'^/{self.project_name}_.*_1$' - if int(run_and_check(f'docker container list --all --filter name={filter_name} | wc -l', shell=True)) > 1: - logging.debug(f"Trying to kill unstopped containers for project {self.project_name}:") - unstopped_containers = run_and_check(f'docker container list --all --filter name={filter_name}', shell=True).splitlines() - logging.debug(f"Unstopped containers {unstopped_containers}") + unstopped_containers = self.get_running_containers() + if unstopped_containers: + logging.debug(f"Trying to kill unstopped containers: {unstopped_containers}") for id in unstopped_containers: run_and_check(f'docker kill {id}', shell=True, nothrow=True) run_and_check(f'docker rm {id}', shell=True, nothrow=True) - left_ids = run_and_check(f'docker container list --all --filter name={filter_name}', shell=True) - logging.debug(f"Unstopped containers killed. Left {left_ids}") + unstopped_containers = self.get_running_containers() + if unstopped_containers: + logging.debug(f"Left unstopped containers: {unstopped_containers}") + else: + logging.debug(f"Unstopped containers killed.") else: logging.debug(f"No running containers for project: {self.project_name}") except: @@ -487,6 +484,19 @@ class ClickHouseCluster: cmd += " client" return cmd + # Returns the list of currently running docker containers corresponding to this ClickHouseCluster. + def get_running_containers(self): + # docker-compose names containers using the following formula: + # container_name = project_name + '_' + instance_name + '_1' + # We need to have "^/" and "$" in the "--filter name" option below to filter by exact name of the container, see + # https://stackoverflow.com/questions/48767760/how-to-make-docker-container-ls-f-name-filter-by-exact-name + filter_name = f'^/{self.project_name}_.*_1$' + # We want the command "docker container list" to show only containers' ID and their names, separated by colon. + format = '{{.ID}}:{{.Names}}' + containers = run_and_check(f"docker container list --all --filter name='{filter_name}' --format '{format}'", shell=True) + containers = dict(line.split(':', 1) for line in containers.decode('utf8').splitlines()) + return containers + def copy_file_from_container_to_container(self, src_node, src_path, dst_node, dst_path): fname = os.path.basename(src_path) run_and_check([f"docker cp {src_node.docker_id}:{src_path} {self.instances_dir}"], shell=True) From 589bb9eb92d8a8b1fa37ba90f367ad8022de06a3 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Nov 2021 12:39:45 +0300 Subject: [PATCH 196/200] Trying to use push event --- .github/workflows/release_branches.yml | 8 +---- tests/ci/pr_info.py | 45 ++++++++++++++++++++------ 2 files changed, 37 insertions(+), 16 deletions(-) diff --git a/.github/workflows/release_branches.yml b/.github/workflows/release_branches.yml index bfcb1b4b14d..e279ae91588 100644 --- a/.github/workflows/release_branches.yml +++ b/.github/workflows/release_branches.yml @@ -1,17 +1,12 @@ name: ReleaseCI on: # yamllint disable-line rule:truthy - pull_request: - branches: - - '21.**' - - '22.**' - - '23.**' - - '24.**' push: branches: - '21.**' - '22.**' - '23.**' - '24.**' + - 'backport/**' jobs: DockerHubPush: runs-on: [self-hosted, style-checker] @@ -94,7 +89,6 @@ jobs: docker kill $(docker ps -q) ||: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH - BuilderDebAsan: needs: [DockerHubPush] runs-on: [self-hosted, builder] diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 37fc17d52d8..28207256570 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -8,6 +8,23 @@ from unidiff import PatchSet DIFF_IN_DOCUMENTATION_EXT = [".html", ".md", ".yml", ".txt", ".css", ".js", ".xml", ".ico", ".conf", ".svg", ".png", ".jpg", ".py", ".sh"] +def get_pr_for_commit(sha): + try_get_pr_url = f"https://api.github.com/repos/{os.getenv('GITHUB_REPOSITORY', 'ClickHouse/ClickHouse')}/commits/{sha}/pulls" + try: + response = requests.get(try_get_pr_url) + response.raise_for_status() + data = response.json() + if len(data) > 1: + print("Got more than one pr for commit", sha) + for pr in data: + if 'release' in {label['name'] for label in pr['labels']}: + return pr + first_pr = data[0] + return first_pr + except Exception as ex: + print("Cannot fetch PR info from commit", ex) + return None + class PRInfo: def __init__(self, github_event, need_orgs=False, need_changed_files=False): if 'pull_request' in github_event: # pull request and other similar events @@ -46,22 +63,32 @@ class PRInfo: self.changed_files = { f.path for f in diff_object } elif 'commits' in github_event: - self.number = 0 self.sha = github_event['after'] - self.labels = {} - self.repo_full_name = os.getenv('GITHUB_REPOSITORY', 'ClickHouse/ClickHouse') + pull_request = get_pr_for_commit(self.sha) repo_prefix = f"{os.getenv('GITHUB_SERVER_URL', 'https://github.com')}/{os.getenv('GITHUB_REPOSITORY', 'ClickHouse/ClickHouse')}" self.task_url = f"{repo_prefix}/actions/runs/{os.getenv('GITHUB_RUN_ID')}" self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" - self.pr_html_url = f"{repo_prefix}/commits/master" - self.base_ref = "master" - self.base_name = self.repo_full_name - self.head_ref = "master" - self.head_name = self.repo_full_name + self.repo_full_name = os.getenv('GITHUB_REPOSITORY', 'ClickHouse/ClickHouse') + if pull_request is None or pull_request['state'] == 'closed': # it's merged PR to master + self.number = 0 + self.labels = {} + self.pr_html_url = f"{repo_prefix}/commits/master" + self.base_ref = "master" + self.base_name = self.repo_full_name + self.head_ref = "master" + self.head_name = self.repo_full_name + else: + self.number = pull_request['number'] + self.labels = { l['name'] for l in pull_request['labels'] } + self.base_ref = pull_request['base']['ref'] + self.base_name = pull_request['base']['repo']['full_name'] + self.head_ref = pull_request['head']['ref'] + self.head_name = pull_request['head']['repo']['full_name'] + self.pr_html_url = pull_request['html_url'] if need_changed_files: commit_before = github_event['before'] - response = requests.get(f"{os.getenv('GITHUB_SERVER_URL')}/repos/{os.getenv('GITHUB_REPOSITORY')}/compare/{commit_before}...{self.sha}") + response = requests.get(f"https://api.github.com/repos/{os.getenv('GITHUB_REPOSITORY')}/compare/{commit_before}...{self.sha}") response.raise_for_status() diff = response.json() From 841141440baa0bd4ac9896c1d624262c7bdaea6b Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Nov 2021 12:56:13 +0300 Subject: [PATCH 197/200] Fix search for PR --- tests/ci/pr_info.py | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 28207256570..1fdc34ccdf1 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -8,7 +8,7 @@ from unidiff import PatchSet DIFF_IN_DOCUMENTATION_EXT = [".html", ".md", ".yml", ".txt", ".css", ".js", ".xml", ".ico", ".conf", ".svg", ".png", ".jpg", ".py", ".sh"] -def get_pr_for_commit(sha): +def get_pr_for_commit(sha, ref): try_get_pr_url = f"https://api.github.com/repos/{os.getenv('GITHUB_REPOSITORY', 'ClickHouse/ClickHouse')}/commits/{sha}/pulls" try: response = requests.get(try_get_pr_url) @@ -17,8 +17,11 @@ def get_pr_for_commit(sha): if len(data) > 1: print("Got more than one pr for commit", sha) for pr in data: - if 'release' in {label['name'] for label in pr['labels']}: + # refs for pushes looks like refs/head/XX + # refs for RPs looks like XX + if pr['head']['ref'] in ref: return pr + print ("Cannot find PR with required ref", ref, "returning first one") first_pr = data[0] return first_pr except Exception as ex: @@ -64,7 +67,7 @@ class PRInfo: elif 'commits' in github_event: self.sha = github_event['after'] - pull_request = get_pr_for_commit(self.sha) + pull_request = get_pr_for_commit(self.sha, github_event['ref']) repo_prefix = f"{os.getenv('GITHUB_SERVER_URL', 'https://github.com')}/{os.getenv('GITHUB_REPOSITORY', 'ClickHouse/ClickHouse')}" self.task_url = f"{repo_prefix}/actions/runs/{os.getenv('GITHUB_RUN_ID')}" self.commit_html_url = f"{repo_prefix}/commits/{self.sha}" From a0fb44a49521648ec1d4d8b37d573ab6040cf18e Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Nov 2021 14:54:02 +0300 Subject: [PATCH 198/200] Add master yml --- .github/workflows/master.yml | 1384 ++++++++++++++++++++++++++++++++++ 1 file changed, 1384 insertions(+) create mode 100644 .github/workflows/master.yml diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml new file mode 100644 index 00000000000..1f7fb1c0628 --- /dev/null +++ b/.github/workflows/master.yml @@ -0,0 +1,1384 @@ +name: MasterCI +on: # yamllint disable-line rule:truthy + push: + branches: + - 'master' +jobs: + DockerHubPush: + runs-on: [self-hosted, style-checker] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Images check + run: | + cd $GITHUB_WORKSPACE/tests/ci + python3 docker_images_check.py + - name: Upload images files to artifacts + uses: actions/upload-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/docker_images_check/changed_images.json + StyleCheck: + needs: DockerHubPush + runs-on: [self-hosted, style-checker] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/style_check + - name: Check out repository code + uses: actions/checkout@v2 + - name: Style Check + env: + TEMP_PATH: ${{ runner.temp }}/style_check + run: | + cd $GITHUB_WORKSPACE/tests/ci + python3 style_check.py + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + PVSCheck: + needs: DockerHubPush + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, func-tester] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + - name: PVS Check + env: + TEMP_PATH: ${{runner.temp}}/pvs_check + REPO_COPY: ${{runner.temp}}/pvs_check/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 pvs_check.py + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + CompatibilityCheck: + needs: [BuilderDebRelease] + runs-on: [self-hosted, style-checker] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: CompatibilityCheck + env: + TEMP_PATH: ${{runner.temp}}/compatibility_check + REPO_COPY: ${{runner.temp}}/compatibility_check/ClickHouse + REPORTS_PATH: ${{runner.temp}}/reports_dir + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 compatibility_check.py 0 + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + SplitBuildSmokeTest: + needs: [BuilderDebSplitted] + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, style-checker] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Split build check + env: + TEMP_PATH: ${{runner.temp}}/split_build_check + REPO_COPY: ${{runner.temp}}/split_build_check/ClickHouse + REPORTS_PATH: ${{runner.temp}}/reports_dir + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 split_build_smoke_check.py + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +######################################################################################### +#################################### ORDINARY BUILDS #################################### +######################################################################################### + BuilderDebRelease: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 0 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + BuilderBinRelease: + needs: [DockerHubPush, FastTest] + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 9 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + BuilderDebAsan: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 3 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + BuilderDebUBsan: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 4 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + BuilderDebTsan: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 5 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + BuilderDebMsan: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 6 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + BuilderDebDebug: + needs: [DockerHubPush] + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse build check (actions)' + BUILD_NUMBER: 7 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +########################################################################################## +##################################### SPECIAL BUILDS ##################################### +########################################################################################## + BuilderDebSplitted: + needs: [DockerHubPush, FastTest] + if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} + runs-on: [self-hosted, builder] + steps: + - name: Download changed images + uses: actions/download-artifact@v2 + with: + name: changed_images + path: ${{ runner.temp }}/images_path + - name: Check out repository code + uses: actions/checkout@v2 + with: + submodules: 'recursive' + fetch-depth: 0 # otherwise we will have no info about contributors + - name: Build + env: + TEMP_PATH: ${{runner.temp}}/build_check + IMAGES_PATH: ${{runner.temp}}/images_path + REPO_COPY: ${{runner.temp}}/build_check/ClickHouse + CACHES_PATH: ${{runner.temp}}/../ccaches + CHECK_NAME: 'ClickHouse special build check (actions)' + BUILD_NUMBER: 1 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci && python3 build_check.py "$CHECK_NAME" $BUILD_NUMBER + - name: Upload build URLs to artifacts + uses: actions/upload-artifact@v2 + with: + name: ${{ env.BUILD_NAME }} + path: ${{ runner.temp }}/build_check/${{ env.BUILD_NAME }}.json + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################ +##################################### BUILD REPORTER ####################################### +############################################################################################ + BuilderReport: + needs: + - BuilderDebRelease + - BuilderBinRelease + - BuilderDebAsan + - BuilderDebTsan + - BuilderDebUBsan + - BuilderDebMsan + - BuilderDebDebug + - BuilderDebSplitted + runs-on: [self-hosted, style-checker] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Report Builder + env: + TEMP_PATH: ${{runner.temp}}/report_check + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'ClickHouse build check (actions)' + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cd $GITHUB_WORKSPACE/tests/ci + python3 build_report_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################## +########################### FUNCTIONAl STATELESS TESTS ####################################### +############################################################################################## + FunctionalStatelessTestRelease: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (release, actions)' + REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (address, actions)' + REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_tsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (thread, actions)' + REPO_COPY: ${{runner.temp}}/stateless_tsan/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_ubsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (ubsan, actions)' + REPO_COPY: ${{runner.temp}}/stateless_ubsan/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_memory + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (memory, actions)' + REPO_COPY: ${{runner.temp}}/stateless_memory/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatelessTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateless_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateless tests (debug, actions)' + REPO_COPY: ${{runner.temp}}/stateless_debug/ClickHouse + KILL_TIMEOUT: 10800 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################## +############################ FUNCTIONAl STATEFUL TESTS ####################################### +############################################################################################## + FunctionalStatefulTestRelease: + needs: [BuilderDebRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (release, actions)' + REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatefulTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (address, actions)' + REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatefulTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_tsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (thread, actions)' + REPO_COPY: ${{runner.temp}}/stateful_tsan/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatefulTestMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_msan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (memory, actions)' + REPO_COPY: ${{runner.temp}}/stateful_msan/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatefulTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_ubsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (ubsan, actions)' + REPO_COPY: ${{runner.temp}}/stateful_ubsan/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FunctionalStatefulTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Functional test + env: + TEMP_PATH: ${{runner.temp}}/stateful_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stateful tests (debug, actions)' + REPO_COPY: ${{runner.temp}}/stateful_debug/ClickHouse + KILL_TIMEOUT: 3600 + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 functional_test_check.py "$CHECK_NAME" $KILL_TIMEOUT + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################## +######################################### STRESS TESTS ####################################### +############################################################################################## + StressTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + env: + TEMP_PATH: ${{runner.temp}}/stress_thread + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress test (address, actions)' + REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + StressTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + env: + TEMP_PATH: ${{runner.temp}}/stress_thread + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress test (thread, actions)' + REPO_COPY: ${{runner.temp}}/stress_thread/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + StressTestMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + env: + TEMP_PATH: ${{runner.temp}}/stress_memory + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress test (memory, actions)' + REPO_COPY: ${{runner.temp}}/stress_memory/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + StressTestUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + env: + TEMP_PATH: ${{runner.temp}}/stress_undefined + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress test (undefined, actions)' + REPO_COPY: ${{runner.temp}}/stress_undefined/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + StressTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Stress test + env: + TEMP_PATH: ${{runner.temp}}/stress_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Stress test (debug, actions)' + REPO_COPY: ${{runner.temp}}/stress_debug/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 stress_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################# +############################# INTEGRATION TESTS ############################################# +############################################################################################# + IntegrationTestsAsan: + needs: [BuilderDebAsan, FunctionalStatelessTestAsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + env: + TEMP_PATH: ${{runner.temp}}/integration_tests_asan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Integration tests (asan, actions)' + REPO_COPY: ${{runner.temp}}/integration_tests_asan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + IntegrationTestsTsan: + needs: [BuilderDebTsan, FunctionalStatelessTestTsan] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + env: + TEMP_PATH: ${{runner.temp}}/integration_tests_tsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Integration tests (thread, actions)' + REPO_COPY: ${{runner.temp}}/integration_tests_tsan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + IntegrationTestsRelease: + needs: [BuilderDebRelease, FunctionalStatelessTestRelease] + runs-on: [self-hosted, stress-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Integration test + env: + TEMP_PATH: ${{runner.temp}}/integration_tests_release + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Integration tests (release, actions)' + REPO_COPY: ${{runner.temp}}/integration_tests_release/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 integration_test_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################## +##################################### AST FUZZERS ############################################ +############################################################################################## + ASTFuzzerTestAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + env: + TEMP_PATH: ${{runner.temp}}/ast_fuzzer_asan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'AST fuzzer (ASan, actions)' + REPO_COPY: ${{runner.temp}}/ast_fuzzer_asan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + ASTFuzzerTestTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + env: + TEMP_PATH: ${{runner.temp}}/ast_fuzzer_tsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'AST fuzzer (TSan, actions)' + REPO_COPY: ${{runner.temp}}/ast_fuzzer_tsan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + ASTFuzzerTestUBSan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + env: + TEMP_PATH: ${{runner.temp}}/ast_fuzzer_ubsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'AST fuzzer (UBSan, actions)' + REPO_COPY: ${{runner.temp}}/ast_fuzzer_ubsan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + ASTFuzzerTestMSan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + env: + TEMP_PATH: ${{runner.temp}}/ast_fuzzer_msan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'AST fuzzer (MSan, actions)' + REPO_COPY: ${{runner.temp}}/ast_fuzzer_msan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + ASTFuzzerTestDebug: + needs: [BuilderDebDebug] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Fuzzer + env: + TEMP_PATH: ${{runner.temp}}/ast_fuzzer_debug + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'AST fuzzer (debug, actions)' + REPO_COPY: ${{runner.temp}}/ast_fuzzer_debug/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 ast_fuzzer_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH +############################################################################################# +#################################### UNIT TESTS ############################################# +############################################################################################# + UnitTestsAsan: + needs: [BuilderDebAsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + env: + TEMP_PATH: ${{runner.temp}}/unit_tests_asan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Unit tests (asan, actions)' + REPO_COPY: ${{runner.temp}}/unit_tests_asan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + UnitTestsReleaseClang: + needs: [BuilderBinRelease] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + env: + TEMP_PATH: ${{runner.temp}}/unit_tests_asan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Unit tests (release-clang, actions)' + REPO_COPY: ${{runner.temp}}/unit_tests_asan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + UnitTestsTsan: + needs: [BuilderDebTsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + env: + TEMP_PATH: ${{runner.temp}}/unit_tests_tsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Unit tests (tsan, actions)' + REPO_COPY: ${{runner.temp}}/unit_tests_tsan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + UnitTestsMsan: + needs: [BuilderDebMsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + env: + TEMP_PATH: ${{runner.temp}}/unit_tests_msan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Unit tests (msan, actions)' + REPO_COPY: ${{runner.temp}}/unit_tests_msan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + UnitTestsUBsan: + needs: [BuilderDebUBsan] + runs-on: [self-hosted, func-tester] + steps: + - name: Download json reports + uses: actions/download-artifact@v2 + with: + path: ${{runner.temp}}/reports_dir + - name: Check out repository code + uses: actions/checkout@v2 + - name: Unit test + env: + TEMP_PATH: ${{runner.temp}}/unit_tests_ubsan + REPORTS_PATH: ${{runner.temp}}/reports_dir + CHECK_NAME: 'Unit tests (msan, actions)' + REPO_COPY: ${{runner.temp}}/unit_tests_ubsan/ClickHouse + run: | + sudo rm -fr $TEMP_PATH + mkdir -p $TEMP_PATH + cp -r $GITHUB_WORKSPACE $TEMP_PATH + cd $REPO_COPY/tests/ci + python3 unit_tests_check.py "$CHECK_NAME" + - name: Cleanup + if: always() + run: | + docker kill $(docker ps -q) ||: + docker rm -f $(docker ps -a -q) ||: + sudo rm -fr $TEMP_PATH + FinishCheck: + needs: + - DockerHubPush + - BuilderReport + - FunctionalStatelessTestDebug + - FunctionalStatelessTestRelease + - FunctionalStatelessTestAsan + - FunctionalStatelessTestTsan + - FunctionalStatelessTestMsan + - FunctionalStatelessTestUBsan + - FunctionalStatefulTestDebug + - FunctionalStatefulTestRelease + - FunctionalStatefulTestAsan + - FunctionalStatefulTestTsan + - FunctionalStatefulTestMsan + - FunctionalStatefulTestUBsan + - StressTestDebug + - StressTestAsan + - StressTestTsan + - StressTestMsan + - StressTestUBsan + - IntegrationTestsAsan + - IntegrationTestsRelease + - IntegrationTestsTsan + - CompatibilityCheck + - ASTFuzzerTestDebug + - ASTFuzzerTestAsan + - ASTFuzzerTestTsan + - ASTFuzzerTestMSan + - ASTFuzzerTestUBSan + - UnitTestsAsan + - UnitTestsTsan + - UnitTestsMsan + - UnitTestsUBsan + - UnitTestsReleaseClang + - SplitBuildSmokeTest + - PVSCheck + runs-on: [self-hosted, style-checker] + steps: + - name: Check out repository code + uses: actions/checkout@v2 + - name: Finish label + run: | + cd $GITHUB_WORKSPACE/tests/ci + python3 finish_check.py From 8a7c75554a975f3abc42a8c0fae84a6b925dc252 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 22 Nov 2021 15:07:53 +0300 Subject: [PATCH 199/200] Fix master yml --- .github/workflows/master.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/master.yml b/.github/workflows/master.yml index 1f7fb1c0628..fc2e78a21e3 100644 --- a/.github/workflows/master.yml +++ b/.github/workflows/master.yml @@ -160,7 +160,7 @@ jobs: docker rm -f $(docker ps -a -q) ||: sudo rm -fr $TEMP_PATH BuilderBinRelease: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: @@ -392,7 +392,7 @@ jobs: ##################################### SPECIAL BUILDS ##################################### ########################################################################################## BuilderDebSplitted: - needs: [DockerHubPush, FastTest] + needs: [DockerHubPush] if: ${{ !contains(github.event.pull_request.labels.*.name, 'pr-documentation') && !contains(github.event.pull_request.labels.*.name, 'pr-doc-fix') }} runs-on: [self-hosted, builder] steps: From e94f5a33443da4906f11041dc389b86fe5bd876b Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Mon, 22 Nov 2021 17:17:22 +0300 Subject: [PATCH 200/200] [website] drop calendar link for today's past webinar (#31634) --- website/templates/index/hero.html | 4 ---- 1 file changed, 4 deletions(-) diff --git a/website/templates/index/hero.html b/website/templates/index/hero.html index 7fa7feb8459..8e604cad771 100644 --- a/website/templates/index/hero.html +++ b/website/templates/index/hero.html @@ -14,10 +14,6 @@ -
- Add to Your Calendar -
22 Nov - Asia, Europe
-