From c855cf705774b5e6c786858e6f6c33c6f157f822 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 2 Jun 2021 19:56:24 +0300 Subject: [PATCH 001/117] Part 1. --- src/Storages/MergeTree/KeyCondition.cpp | 198 +++++++++++++++++++----- src/Storages/MergeTree/KeyCondition.h | 16 +- 2 files changed, 171 insertions(+), 43 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 268c45c305f..38c849c728c 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -114,6 +114,130 @@ static String firstStringThatIsGreaterThanAllStringsWithPrefix(const String & pr return res; } +class KeyCondition::Tree +{ +public: + explicit Tree(const IAST * ast_) : ast(ast_) {} + explicit Tree(const ActionsDAG::Node * dag_) : dag(dag_) {} + + std::string getColumnName() const + { + if (ast) + return ast->getColumnNameWithoutAlias(); + else + getColumnNameWithoutAlias(dag); + } + + // size_t numChildren() const + // { + // if (ast) + // return ast->children.size(); + // else + // return dag->children.size(); + // } + + // Tree getChildrenAt(size_t idx) const + // { + // if (ast) + // return Tree(ast->children[idx].get()); + // else + // return Tree(dag->children[idx]); + // } + + bool isFunction() const + { + if (ast) + return typeid_cast(ast); + else + return dag->type == ActionsDAG::ActionType::FUNCTION; + } + + bool isConstant() const + { + if (ast) + return typeid_cast(ast); + else + return dag->type == ActionsDAG::ActionType::COLUMN; + } + + ColumnWithTypeAndName getConstant() const + { + if (!isConstant()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "KeyCondition::Tree node is not a constant"); + + ColumnWithTypeAndName res; + + if (ast) + { + const auto * literal = assert_cast(ast); + res.type = applyVisitor(FieldToDataType(), literal->value); + res.column = res.type->createColumnConst(0, literal->value); + + } + else + { + res.type = dag->result_type; + res.column = dag->column; + } + + return res; + } + + FunctionTree asFunction() const; + +protected: + const IAST * ast = nullptr; + const ActionsDAG::Node * dag = nullptr; +}; + +class KeyCondition::FunctionTree : public KeyCondition::Tree +{ +public: + std::string getFunctionName() const + { + if (ast) + return assert_cast(ast)->name; + else + return dag->function_base->getName(); + } + + size_t numArguments() const + { + if (ast) + { + const auto * func = assert_cast(ast); + return func->arguments ? 0 : func->arguments->size(); + } + else + return dag->children.size(); + } + + Tree getArgumentAt(size_t idx) const + { + if (ast) + return Tree(assert_cast(ast)->arguments->children[idx].get()); + else + return Tree(dag->children[idx]); + } + +private: + using Tree::Tree; + + friend class Tree; +}; + + +KeyCondition::FunctionTree KeyCondition::Tree::asFunction() const +{ + if (!isFunction()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "KeyCondition::Tree node is not a function"); + + if (ast) + return KeyCondition::FunctionTree(ast); + else + return KeyCondition::FunctionTree(dag); +} + /// A dictionary containing actions to the corresponding functions to turn them into `RPNElement` const KeyCondition::AtomMap KeyCondition::atom_map @@ -560,18 +684,19 @@ static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & return {field.columns, field.row_idx, result_idx}; } -void KeyCondition::traverseAST(const ASTPtr & node, ContextPtr context, Block & block_with_constants) +void KeyCondition::traverseAST(const Tree & node, ContextPtr context, Block & block_with_constants) { RPNElement element; - if (const auto * func = node->as()) + if (node.isFunction()) { + auto func = node.asFunction(); if (tryParseLogicalOperatorFromAST(func, element)) { - auto & args = func->arguments->children; - for (size_t i = 0, size = args.size(); i < size; ++i) + size_t num_args = func.numArguments(); + for (size_t i = 0; i < num_args; ++i) { - traverseAST(args[i], context, block_with_constants); + traverseAST(func.getArgumentAt(i), context, block_with_constants); /** The first part of the condition is for the correct support of `and` and `or` functions of arbitrary arity * - in this case `n - 1` elements are added (where `n` is the number of arguments). @@ -968,13 +1093,13 @@ private: bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( - const ASTPtr & node, + const Tree & node, ContextPtr context, size_t & out_key_column_num, DataTypePtr & out_key_res_column_type, MonotonicFunctionsChain & out_functions_chain) { - std::vector chain_not_tested_for_monotonicity; + std::vector chain_not_tested_for_monotonicity; DataTypePtr key_column_type; if (!isKeyPossiblyWrappedByMonotonicFunctionsImpl(node, out_key_column_num, key_column_type, chain_not_tested_for_monotonicity)) @@ -982,14 +1107,14 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( for (auto it = chain_not_tested_for_monotonicity.rbegin(); it != chain_not_tested_for_monotonicity.rend(); ++it) { - const auto & args = (*it)->arguments->children; - auto func_builder = FunctionFactory::instance().tryGet((*it)->name, context); + auto func = *it; + auto func_builder = FunctionFactory::instance().tryGet(func.getFunctionName(), context); if (!func_builder) return false; ColumnsWithTypeAndName arguments; ColumnWithTypeAndName const_arg; FunctionWithOptionalConstArg::Kind kind = FunctionWithOptionalConstArg::Kind::NO_CONST; - if (args.size() == 2) + if (func.numArguments() == 2) { if (const auto * arg_left = args[0]->as()) { @@ -1029,10 +1154,10 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( } bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( - const ASTPtr & node, + const Tree & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, - std::vector & out_functions_chain) + std::vector & out_functions_chain) { /** By itself, the key column can be a functional expression. for example, `intHash32(UserID)`. * Therefore, use the full name of the expression for search. @@ -1040,7 +1165,7 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( const auto & sample_block = key_expr->getSampleBlock(); // Key columns should use canonical names for index analysis - String name = node->getColumnNameWithoutAlias(); + String name = node.getColumnName(); auto it = key_columns.find(name); if (key_columns.end() != it) @@ -1050,31 +1175,30 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( return true; } - if (const auto * func = node->as()) + if (node.isFunction()) { - if (!func->arguments) - return false; + auto func = node.asFunction(); - const auto & args = func->arguments->children; - if (args.size() > 2 || args.empty()) + size_t num_args = func.numArguments(); + if (num_args > 2 || num_args == 0) return false; out_functions_chain.push_back(func); bool ret = false; - if (args.size() == 2) + if (num_args == 2) { - if (args[0]->as()) + if (func.getArgumentAt(0).isConstant()) { - ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(args[1], out_key_column_num, out_key_column_type, out_functions_chain); + ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(func.getArgumentAt(1), out_key_column_num, out_key_column_type, out_functions_chain); } - else if (args[1]->as()) + else if (func.getArgumentAt(1).isConstant()) { - ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(args[0], out_key_column_num, out_key_column_type, out_functions_chain); + ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(func.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain); } } else { - ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(args[0], out_key_column_num, out_key_column_type, out_functions_chain); + ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(func.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain); } return ret; } @@ -1099,7 +1223,7 @@ static void castValueToType(const DataTypePtr & desired_type, Field & src_value, } -bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, ContextPtr context, Block & block_with_constants, RPNElement & out) +bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Block & block_with_constants, RPNElement & out) { /** Functions < > = != <= >= in `notIn`, where one argument is a constant, and the other is one of columns of key, * or itself, wrapped in a chain of possibly-monotonic functions, @@ -1107,27 +1231,28 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, ContextPtr context, */ Field const_value; DataTypePtr const_type; - if (const auto * func = node->as()) + if (node.isFunction()) { - const ASTs & args = func->arguments->children; + auto func = node.asFunction(); + size_t num_args = func.numArguments(); DataTypePtr key_expr_type; /// Type of expression containing key column size_t key_column_num = -1; /// Number of a key column (inside key_column_names array) MonotonicFunctionsChain chain; - std::string func_name = func->name; + std::string func_name = func.getFunctionName(); if (atom_map.find(func_name) == std::end(atom_map)) return false; - if (args.size() == 1) + if (num_args == 1) { - if (!(isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain))) + if (!(isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), context, key_column_num, key_expr_type, chain))) return false; if (key_column_num == static_cast(-1)) throw Exception("`key_column_num` wasn't initialized. It is a bug.", ErrorCodes::LOGICAL_ERROR); } - else if (args.size() == 2) + else if (num_args == 2) { size_t key_arg_pos; /// Position of argument with key column (non-const argument) bool is_set_const = false; @@ -1315,25 +1440,24 @@ bool KeyCondition::tryParseAtomFromAST(const ASTPtr & node, ContextPtr context, return false; } -bool KeyCondition::tryParseLogicalOperatorFromAST(const ASTFunction * func, RPNElement & out) +bool KeyCondition::tryParseLogicalOperatorFromAST(const FunctionTree & func, RPNElement & out) { /// Functions AND, OR, NOT. /// Also a special function `indexHint` - works as if instead of calling a function there are just parentheses /// (or, the same thing - calling the function `and` from one argument). - const ASTs & args = func->arguments->children; - if (func->name == "not") + if (func.getFunctionName() == "not") { - if (args.size() != 1) + if (func.numArguments() != 1) return false; out.function = RPNElement::FUNCTION_NOT; } else { - if (func->name == "and" || func->name == "indexHint") + if (func.getFunctionName() == "and" || func.getFunctionName() == "indexHint") out.function = RPNElement::FUNCTION_AND; - else if (func->name == "or") + else if (func.getFunctionName() == "or") out.function = RPNElement::FUNCTION_OR; else return false; diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index fc28d4a93c9..8bc9d13ef3c 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -378,6 +378,10 @@ public: static const AtomMap atom_map; private: + + class Tree; + class FunctionTree; + BoolMask checkInRange( size_t used_key_size, const FieldRef * left_key, @@ -386,9 +390,9 @@ private: bool right_bounded, BoolMask initial_mask) const; - void traverseAST(const ASTPtr & node, ContextPtr context, Block & block_with_constants); - bool tryParseAtomFromAST(const ASTPtr & node, ContextPtr context, Block & block_with_constants, RPNElement & out); - static bool tryParseLogicalOperatorFromAST(const ASTFunction * func, RPNElement & out); + void traverseAST(const Tree & node, ContextPtr context, Block & block_with_constants); + bool tryParseAtomFromAST(const Tree & node, ContextPtr context, Block & block_with_constants, RPNElement & out); + static bool tryParseLogicalOperatorFromAST(const FunctionTree & func, RPNElement & out); /** Is node the key column * or expression in which column of key is wrapped by chain of functions, @@ -397,17 +401,17 @@ private: * and fills chain of possibly-monotonic functions. */ bool isKeyPossiblyWrappedByMonotonicFunctions( - const ASTPtr & node, + const Tree & node, ContextPtr context, size_t & out_key_column_num, DataTypePtr & out_key_res_column_type, MonotonicFunctionsChain & out_functions_chain); bool isKeyPossiblyWrappedByMonotonicFunctionsImpl( - const ASTPtr & node, + const Tree & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, - std::vector & out_functions_chain); + std::vector & out_functions_chain); bool canConstantBeWrappedByMonotonicFunctions( const ASTPtr & node, From 38966e3e6b8f5a65949841545cbd6813ee39869a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 Jun 2021 15:26:02 +0300 Subject: [PATCH 002/117] Part 2. --- src/Storages/MergeTree/KeyCondition.cpp | 173 ++++++++++++++++-------- src/Storages/MergeTree/KeyCondition.h | 2 +- 2 files changed, 120 insertions(+), 55 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 38c849c728c..3d96ce37c85 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -13,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -183,6 +184,99 @@ public: return res; } + bool getConstant(Block & block_with_constants, Field & out_value, DataTypePtr & out_type) + { + if (ast) + { + // Constant expr should use alias names if any + String column_name = ast->getColumnName(); + + if (const auto * lit = ast->as()) + { + /// By default block_with_constants has only one column named "_dummy". + /// If block contains only constants it's may not be preprocessed by + // ExpressionAnalyzer, so try to look up in the default column. + if (!block_with_constants.has(column_name)) + column_name = "_dummy"; + + /// Simple literal + out_value = lit->value; + out_type = block_with_constants.getByName(column_name).type; + return true; + } + else if (block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column)) + { + /// An expression which is dependent on constants only + const auto & expr_info = block_with_constants.getByName(column_name); + out_value = (*expr_info.column)[0]; + out_type = expr_info.type; + return true; + } + } + else + { + if (dag->column && isColumnConst(*dag->column)) + { + out_value = (*dag->column)[0]; + out_type = dag->result_type; + return true; + } + } + + return false; + } + + ConstSetPtr tryGetPreparedSet( + const PreparedSets & sets, + const std::vector & indexes_mapping, + const DataTypes & data_types) const + { + if (ast) + { + if (ast->as() || ast->as()) + { + auto set_it = sets.find(PreparedSetKey::forSubquery(*ast)); + if (set_it != sets.end()) + return set_it->second; + } + else + { + /// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information + /// about types in left argument of the IN operator. Instead, we manually iterate through all the sets + /// and find the one for the right arg based on the AST structure (getTreeHash), after that we check + /// that the types it was prepared with are compatible with the types of the primary key. + auto set_ast_hash = ast->getTreeHash(); + auto set_it = std::find_if( + sets.begin(), sets.end(), + [&](const auto & candidate_entry) + { + if (candidate_entry.first.ast_hash != set_ast_hash) + return false; + + for (size_t i = 0; i < indexes_mapping.size(); ++i) + if (!candidate_entry.second->areTypesEqual(indexes_mapping[i].tuple_index, data_types[i])) + return false; + + return true; + }); + if (set_it != sets.end()) + return set_it->second; + } + } + else + { + if (dag->column) + { + const auto * col_set = typeid_cast(dag->column.get()); + auto set = col_set->getData(); + if (set->isCreated()) + return set; + } + } + + return nullptr; + } + FunctionTree asFunction() const; protected: @@ -929,18 +1023,18 @@ bool KeyCondition::canConstantBeWrappedByFunctions( } bool KeyCondition::tryPrepareSetIndex( - const ASTs & args, + const FunctionTree & func, ContextPtr context, RPNElement & out, size_t & out_key_column_num) { - const ASTPtr & left_arg = args[0]; + const auto & left_arg = func.getArgumentAt(0); out_key_column_num = 0; std::vector indexes_mapping; DataTypes data_types; - auto get_key_tuple_position_mapping = [&](const ASTPtr & node, size_t tuple_index) + auto get_key_tuple_position_mapping = [&](const Tree & node, size_t tuple_index) { MergeTreeSetIndex::KeyTuplePositionMapping index_mapping; index_mapping.tuple_index = tuple_index; @@ -956,13 +1050,17 @@ bool KeyCondition::tryPrepareSetIndex( }; size_t left_args_count = 1; - const auto * left_arg_tuple = left_arg->as(); - if (left_arg_tuple && left_arg_tuple->name == "tuple") + if (left_arg.isFunction()) { - const auto & tuple_elements = left_arg_tuple->arguments->children; - left_args_count = tuple_elements.size(); - for (size_t i = 0; i < left_args_count; ++i) - get_key_tuple_position_mapping(tuple_elements[i], i); + /// Note: in case of ActionsDAG, tuple may be a constant. + /// In this case, there is no keys in tuple. So, we don't have to check it. + auto left_arg_tuple = left_arg.asFunction(); + if (left_arg_tuple.getFunctionName() == "tuple") + { + left_args_count = left_arg_tuple.numArguments(); + for (size_t i = 0; i < left_args_count; ++i) + get_key_tuple_position_mapping(left_arg_tuple.getArgumentAt(i), i); + } } else get_key_tuple_position_mapping(left_arg, 0); @@ -970,42 +1068,11 @@ bool KeyCondition::tryPrepareSetIndex( if (indexes_mapping.empty()) return false; - const ASTPtr & right_arg = args[1]; + const auto right_arg = func.getArgumentAt(1); - SetPtr prepared_set; - if (right_arg->as() || right_arg->as()) - { - auto set_it = prepared_sets.find(PreparedSetKey::forSubquery(*right_arg)); - if (set_it == prepared_sets.end()) - return false; - - prepared_set = set_it->second; - } - else - { - /// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information - /// about types in left argument of the IN operator. Instead, we manually iterate through all the sets - /// and find the one for the right arg based on the AST structure (getTreeHash), after that we check - /// that the types it was prepared with are compatible with the types of the primary key. - auto set_ast_hash = right_arg->getTreeHash(); - auto set_it = std::find_if( - prepared_sets.begin(), prepared_sets.end(), - [&](const auto & candidate_entry) - { - if (candidate_entry.first.ast_hash != set_ast_hash) - return false; - - for (size_t i = 0; i < indexes_mapping.size(); ++i) - if (!candidate_entry.second->areTypesEqual(indexes_mapping[i].tuple_index, data_types[i])) - return false; - - return true; - }); - if (set_it == prepared_sets.end()) - return false; - - prepared_set = set_it->second; - } + auto prepared_set = right_arg.tryGetPreparedSet(prepared_sets, indexes_mapping, data_types); + if (!prepared_set) + return false; /// The index can be prepared if the elements of the set were saved in advance. if (!prepared_set->hasExplicitSetElements()) @@ -1107,28 +1174,26 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( for (auto it = chain_not_tested_for_monotonicity.rbegin(); it != chain_not_tested_for_monotonicity.rend(); ++it) { - auto func = *it; - auto func_builder = FunctionFactory::instance().tryGet(func.getFunctionName(), context); + auto function = *it; + auto func_builder = FunctionFactory::instance().tryGet(function.getFunctionName(), context); if (!func_builder) return false; ColumnsWithTypeAndName arguments; ColumnWithTypeAndName const_arg; FunctionWithOptionalConstArg::Kind kind = FunctionWithOptionalConstArg::Kind::NO_CONST; - if (func.numArguments() == 2) + if (function.numArguments() == 2) { - if (const auto * arg_left = args[0]->as()) + if (function.getArgumentAt(0).isConstant()) { - auto left_arg_type = applyVisitor(FieldToDataType(), arg_left->value); - const_arg = { left_arg_type->createColumnConst(0, arg_left->value), left_arg_type, "" }; + const_arg = function.getArgumentAt(0).getConstant(); arguments.push_back(const_arg); arguments.push_back({ nullptr, key_column_type, "" }); kind = FunctionWithOptionalConstArg::Kind::LEFT_CONST; } - else if (const auto * arg_right = args[1]->as()) + else if (function.getArgumentAt(1).isConstant()) { arguments.push_back({ nullptr, key_column_type, "" }); - auto right_arg_type = applyVisitor(FieldToDataType(), arg_right->value); - const_arg = { right_arg_type->createColumnConst(0, arg_right->value), right_arg_type, "" }; + const_arg = function.getArgumentAt(1).getConstant(); arguments.push_back(const_arg); kind = FunctionWithOptionalConstArg::Kind::RIGHT_CONST; } @@ -1277,7 +1342,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl if (functionIsInOrGlobalInOperator(func_name)) { - if (tryPrepareSetIndex(args, context, out, key_column_num)) + if (tryPrepareSetIndex(func, context, out, key_column_num)) { key_arg_pos = 0; is_set_const = true; diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 8bc9d13ef3c..f34c2a950f8 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -433,7 +433,7 @@ private: /// that will filter values (possibly tuples) by the content of 'prepared_set', /// do it and return true. bool tryPrepareSetIndex( - const ASTs & args, + const FunctionTree & func, ContextPtr context, RPNElement & out, size_t & out_key_column_num); From dee032c89975be004c036c9240847ae4722cdb9d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 3 Jun 2021 15:27:38 +0300 Subject: [PATCH 003/117] Part 2. --- src/Storages/MergeTree/KeyCondition.cpp | 35 ++++--------------------- 1 file changed, 5 insertions(+), 30 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 3d96ce37c85..40529296aaf 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -666,32 +666,7 @@ bool KeyCondition::addCondition(const String & column, const Range & range) */ bool KeyCondition::getConstant(const ASTPtr & expr, Block & block_with_constants, Field & out_value, DataTypePtr & out_type) { - // Constant expr should use alias names if any - String column_name = expr->getColumnName(); - - if (const auto * lit = expr->as()) - { - /// By default block_with_constants has only one column named "_dummy". - /// If block contains only constants it's may not be preprocessed by - // ExpressionAnalyzer, so try to look up in the default column. - if (!block_with_constants.has(column_name)) - column_name = "_dummy"; - - /// Simple literal - out_value = lit->value; - out_type = block_with_constants.getByName(column_name).type; - return true; - } - else if (block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column)) - { - /// An expression which is dependent on constants only - const auto & expr_info = block_with_constants.getByName(column_name); - out_value = (*expr_info.column)[0]; - out_type = expr_info.type; - return true; - } - else - return false; + return Tree(expr.get()).getConstant(block_with_constants, out_value, out_type); } @@ -1350,22 +1325,22 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl else return false; } - else if (getConstant(args[1], block_with_constants, const_value, const_type)) + else if (func.getArgumentAt(1).getConstant(block_with_constants, const_value, const_type)) { - if (isKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain)) + if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), context, key_column_num, key_expr_type, chain)) { key_arg_pos = 0; } else if ( !strict_condition - && canConstantBeWrappedByMonotonicFunctions(args[0], key_column_num, key_expr_type, const_value, const_type)) + && canConstantBeWrappedByMonotonicFunctions(func.getArgumentAt(0), key_column_num, key_expr_type, const_value, const_type)) { key_arg_pos = 0; is_constant_transformed = true; } else if ( single_point && func_name == "equals" && !strict_condition - && canConstantBeWrappedByFunctions(args[0], key_column_num, key_expr_type, const_value, const_type)) + && canConstantBeWrappedByFunctions(func.getArgumentAt(0), key_column_num, key_expr_type, const_value, const_type)) { key_arg_pos = 0; is_constant_transformed = true; From eef6c73030f317a77dec089b6caadbed7b8b7b03 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 21 Jun 2021 19:17:05 +0300 Subject: [PATCH 004/117] Use DAG in KeyCondition --- .../QueryPlan/ReadFromMergeTree.cpp | 2 +- src/Storages/MergeTree/KeyCondition.cpp | 275 +++++++++++++++--- src/Storages/MergeTree/KeyCondition.h | 20 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 +- .../MergeTree/MergeTreeIndexMinMax.cpp | 2 +- src/Storages/MergeTree/PartitionPruner.h | 3 +- 6 files changed, 257 insertions(+), 49 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index fd5de98b4c0..c9204f41ce4 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -788,7 +788,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre // Build and check if primary key is used when necessary const auto & primary_key = metadata_snapshot->getPrimaryKey(); Names primary_key_columns = primary_key.column_names; - KeyCondition key_condition(query_info, context, primary_key_columns, primary_key.expression); + KeyCondition key_condition(query_info.query, query_info.syntax_analyzer_result, query_info.sets, context, primary_key_columns, primary_key.expression); if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue()) { diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 82ca403154f..16807d0361c 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -116,6 +116,54 @@ static String firstStringThatIsGreaterThanAllStringsWithPrefix(const String & pr return res; } +static void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & out, bool legacy = false) +{ + switch (node.type) + { + case (ActionsDAG::ActionType::INPUT): + writeString(node.result_name, out); + break; + case (ActionsDAG::ActionType::COLUMN): + writeString(node.result_name, out); + break; + case (ActionsDAG::ActionType::ALIAS): + appendColumnNameWithoutAlias(*node.children.front(), out, legacy); + break; + case (ActionsDAG::ActionType::ARRAY_JOIN): + writeCString("arrayJoin(", out); + appendColumnNameWithoutAlias(*node.children.front(), out, legacy); + writeChar(')', out); + break; + case (ActionsDAG::ActionType::FUNCTION): + { + auto name = node.function_base->getName(); + if (legacy && name == "modulo") + writeCString("moduleLegacy", out); + else + writeString(name, out); + + writeChar('(', out); + bool first = true; + for (const auto * arg : node.children) + { + if (!first) + writeCString(", ", out); + first = false; + + appendColumnNameWithoutAlias(*arg, out, legacy); + } + writeChar(')', out); + } + } +} + +static std::string getColumnNameWithoutAlias(const ActionsDAG::Node & node, bool legacy = false) +{ + WriteBufferFromOwnString out; + appendColumnNameWithoutAlias(node, out, legacy); + return std::move(out.str()); +} + class KeyCondition::Tree { public: @@ -127,24 +175,20 @@ public: if (ast) return ast->getColumnNameWithoutAlias(); else - getColumnNameWithoutAlias(dag); + return getColumnNameWithoutAlias(*dag); } - // size_t numChildren() const - // { - // if (ast) - // return ast->children.size(); - // else - // return dag->children.size(); - // } - - // Tree getChildrenAt(size_t idx) const - // { - // if (ast) - // return Tree(ast->children[idx].get()); - // else - // return Tree(dag->children[idx]); - // } + std::string getColumnNameLegacy() const + { + if (ast) + { + auto adjusted_ast = ast->clone(); + KeyDescription::moduloToModuloLegacyRecursive(adjusted_ast); + return adjusted_ast->getColumnNameWithoutAlias(); + } + else + return getColumnNameWithoutAlias(*dag, true); + } bool isFunction() const { @@ -185,7 +229,7 @@ public: return res; } - bool getConstant(Block & block_with_constants, Field & out_value, DataTypePtr & out_type) + bool getConstant(const Block & block_with_constants, Field & out_value, DataTypePtr & out_type) const { if (ast) { @@ -584,6 +628,123 @@ ASTPtr cloneASTWithInversionPushDown(const ASTPtr node, const bool need_inversio return need_inversion ? makeASTFunction("not", cloned_node) : cloned_node; } +static const ActionsDAG::Node & cloneASTWithInversionPushDown( + const ActionsDAG::Node & node, + ActionsDAG & inverted_dag, + std::unordered_map to_inverted, + const ContextPtr & context, + const bool need_inversion) +{ + { + auto it = to_inverted.find(&node); + if (it != to_inverted.end()) + return *it->second; + } + + const ActionsDAG::Node * res = nullptr; + + switch (node.type) + { + case (ActionsDAG::ActionType::INPUT): + /// Should be already added + throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot clone input in cloneASTWithInversionPushDown"); + case (ActionsDAG::ActionType::COLUMN): + { + res = &inverted_dag.addInput({node.column, node.result_type, node.result_name}); + break; + } + case (ActionsDAG::ActionType::ALIAS): + { + /// Ignore aliases + const auto & alias = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, need_inversion); + to_inverted[&node] = &alias; + return alias; + } + case (ActionsDAG::ActionType::ARRAY_JOIN): + { + const auto & arg = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, false); + res = &inverted_dag.addArrayJoin(arg, node.result_name); + break; + } + case (ActionsDAG::ActionType::FUNCTION): + { + auto name = node.function_base->getName(); + if (name == "not") + { + const auto & arg = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, !need_inversion); + to_inverted[&node] = &arg; + return arg; + } + + if (isLogicalOperator(name)) + { + ActionsDAG::NodeRawConstPtrs children(node.children); + + for (auto & arg : children) + arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, need_inversion); + + FunctionOverloadResolverPtr function_builder; + if (name == "indexHint") + function_builder = node.function_builder; + else if (name == "and") + function_builder = FunctionFactory::instance().get("or", context); + else if (name == "or") + function_builder = FunctionFactory::instance().get("and", context); + + assert(function_builder); + + const auto & func = inverted_dag.addFunction(function_builder, children, ""); + to_inverted[&node] = &func; + return func; + } + + ActionsDAG::NodeRawConstPtrs children(node.children); + + for (auto & arg : children) + arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, false); + + auto it = inverse_relations.find(name); + if (it != inverse_relations.end()) + { + const auto & func_name = need_inversion ? it->second : it->first; + auto function_builder = FunctionFactory::instance().get(func_name, context); + const auto & func = inverted_dag.addFunction(function_builder, children, ""); + to_inverted[&node] = &func; + return func; + } + + res = &inverted_dag.addFunction(node.function_builder, children, ""); + } + } + + if (need_inversion) + res = &inverted_dag.addFunction(FunctionFactory::instance().get("not", context), {res}, ""); + + to_inverted[&node] = res; + return *res; +} + +static ActionsDAGPtr cloneASTWithInversionPushDown(const ActionsDAG & dag, const ContextPtr & context) +{ + auto res = std::make_shared(dag.getRequiredColumns()); + + std::unordered_map to_inverted; + const auto & res_inputs = res->getInputs(); + auto it = res_inputs.begin(); + for (const auto * input : dag.getInputs()) + { + to_inverted[input] = *it; + ++it; + } + + ActionsDAG::NodeRawConstPtrs index; + index.reserve(dag.getIndex().size()); + for (const auto * node : dag.getIndex()) + index.push_back(&cloneASTWithInversionPushDown(*node, *res, to_inverted, context, false)); + + return res; +} + inline bool Range::equals(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateEquals(), lhs, rhs); } inline bool Range::less(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateLess(), lhs, rhs); } @@ -617,7 +778,9 @@ static NameSet getAllSubexpressionNames(const ExpressionActions & key_expr) } KeyCondition::KeyCondition( - const SelectQueryInfo & query_info, + const ASTPtr & query, + TreeRewriterResultPtr syntax_analyzer_result, + PreparedSets prepared_sets_, ContextPtr context, const Names & key_column_names, const ExpressionActionsPtr & key_expr_, @@ -625,7 +788,7 @@ KeyCondition::KeyCondition( bool strict_) : key_expr(key_expr_) , key_subexpr_names(getAllSubexpressionNames(*key_expr)) - , prepared_sets(query_info.sets) + , prepared_sets(std::move(prepared_sets_)) , single_point(single_point_) , strict(strict_) { @@ -639,9 +802,9 @@ KeyCondition::KeyCondition( /** Evaluation of expressions that depend only on constants. * For the index to be used, if it is written, for example `WHERE Date = toDate(now())`. */ - Block block_with_constants = getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context); + Block block_with_constants = getBlockWithConstants(query, syntax_analyzer_result, context); - const ASTSelectQuery & select = query_info.query->as(); + const ASTSelectQuery & select = query->as(); if (select.where() || select.prewhere()) { ASTPtr filter_query; @@ -657,7 +820,43 @@ KeyCondition::KeyCondition( * To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's * are pushed down and applied (when possible) to leaf nodes. */ - traverseAST(cloneASTWithInversionPushDown(filter_query), context, block_with_constants); + auto ast = cloneASTWithInversionPushDown(filter_query); + traverseAST(Tree(ast.get()), context, block_with_constants); + } + else + { + rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN); + } +} + +KeyCondition::KeyCondition( + const ActionsDAG & dag, + PreparedSets prepared_sets_, + ContextPtr context, + const Names & key_column_names, + const ExpressionActionsPtr & key_expr_, + bool single_point_, + bool strict_) + : key_expr(key_expr_) + , key_subexpr_names(getAllSubexpressionNames(*key_expr)) + , prepared_sets(std::move(prepared_sets_)) + , single_point(single_point_) + , strict(strict_) +{ + for (size_t i = 0, size = key_column_names.size(); i < size; ++i) + { + std::string name = key_column_names[i]; + if (!key_columns.count(name)) + key_columns[name] = i; + } + + auto inverted_dag = cloneASTWithInversionPushDown(dag, context); + + if (!inverted_dag->getIndex().empty()) + { + Block empty; + for (const auto * node : inverted_dag->getIndex()) + traverseAST(Tree(node), context, empty); } else { @@ -797,13 +996,13 @@ void KeyCondition::traverseAST(const Tree & node, ContextPtr context, Block & bl } bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( - const ASTPtr & node, + const Tree & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type) { - String expr_name = node->getColumnNameWithoutAlias(); + String expr_name = node.getColumnName(); if (key_subexpr_names.count(expr_name) == 0) return false; @@ -904,9 +1103,9 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( /// Looking for possible transformation of `column = constant` into `partition_expr = function(constant)` bool KeyCondition::canConstantBeWrappedByFunctions( - const ASTPtr & ast, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type) + const Tree & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type) { - String expr_name = ast->getColumnNameWithoutAlias(); + String expr_name = node.getColumnName(); if (key_subexpr_names.count(expr_name) == 0) { @@ -919,9 +1118,7 @@ bool KeyCondition::canConstantBeWrappedByFunctions( /// The case `f(modulo(...))` for totally monotonic `f ` is consedered to be rare. /// /// Note: for negative values, we can filter more partitions then needed. - auto adjusted_ast = ast->clone(); - KeyDescription::moduloToModuloLegacyRecursive(adjusted_ast); - expr_name = adjusted_ast->getColumnName(); + expr_name = node.getColumnNameLegacy(); if (key_subexpr_names.count(expr_name) == 0) return false; @@ -933,14 +1130,14 @@ bool KeyCondition::canConstantBeWrappedByFunctions( if (out_value.isNull()) return false; - for (const auto & node : key_expr->getNodes()) + for (const auto & dag_node : key_expr->getNodes()) { - auto it = key_columns.find(node.result_name); + auto it = key_columns.find(dag_node.result_name); if (it != key_columns.end()) { std::stack chain; - const auto * cur_node = &node; + const auto * cur_node = &dag_node; bool is_valid_chain = true; while (is_valid_chain) @@ -1280,7 +1477,7 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( } -static void castValueToType(const DataTypePtr & desired_type, Field & src_value, const DataTypePtr & src_type, const ASTPtr & node) +static void castValueToType(const DataTypePtr & desired_type, Field & src_value, const DataTypePtr & src_type, const KeyCondition::Tree & node) { try { @@ -1290,7 +1487,7 @@ static void castValueToType(const DataTypePtr & desired_type, Field & src_value, { throw Exception("Key expression contains comparison between inconvertible types: " + desired_type->getName() + " and " + src_type->getName() + - " inside " + queryToString(node), + " inside " + node.getColumnName(), ErrorCodes::BAD_TYPE_OF_FIELD); } } @@ -1381,22 +1578,22 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl else return false; } - else if (getConstant(args[0], block_with_constants, const_value, const_type)) + else if (func.getArgumentAt(0).getConstant(block_with_constants, const_value, const_type)) { - if (isKeyPossiblyWrappedByMonotonicFunctions(args[1], context, key_column_num, key_expr_type, chain)) + if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(1), context, key_column_num, key_expr_type, chain)) { key_arg_pos = 1; } else if ( !strict_condition - && canConstantBeWrappedByMonotonicFunctions(args[1], key_column_num, key_expr_type, const_value, const_type)) + && canConstantBeWrappedByMonotonicFunctions(func.getArgumentAt(1), key_column_num, key_expr_type, const_value, const_type)) { key_arg_pos = 1; is_constant_transformed = true; } else if ( single_point && func_name == "equals" && !strict_condition - && canConstantBeWrappedByFunctions(args[1], key_column_num, key_expr_type, const_value, const_type)) + && canConstantBeWrappedByFunctions(func.getArgumentAt(1), key_column_num, key_expr_type, const_value, const_type)) { key_arg_pos = 0; is_constant_transformed = true; @@ -1490,7 +1687,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl return atom_it->second(out, const_value); } - else if (getConstant(node, block_with_constants, const_value, const_type)) + else if (node.getConstant(block_with_constants, const_value, const_type)) { /// For cases where it says, for example, `WHERE 0 AND something` diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index 6e1af69fd37..f0ffbf9e6bf 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -228,7 +228,18 @@ class KeyCondition public: /// Does not take into account the SAMPLE section. all_columns - the set of all columns of the table. KeyCondition( - const SelectQueryInfo & query_info, + const ASTPtr & query, + TreeRewriterResultPtr syntax_analyzer_result, + PreparedSets prepared_sets_, + ContextPtr context, + const Names & key_column_names, + const ExpressionActionsPtr & key_expr, + bool single_point_ = false, + bool strict_ = false); + + KeyCondition( + const ActionsDAG & dag, + PreparedSets prepared_sets_, ContextPtr context, const Names & key_column_names, const ExpressionActionsPtr & key_expr, @@ -377,11 +388,10 @@ private: public: static const AtomMap atom_map; -private: - class Tree; class FunctionTree; +private: BoolMask checkInRange( size_t used_key_size, const FieldRef * left_key, @@ -414,14 +424,14 @@ private: std::vector & out_functions_chain); bool canConstantBeWrappedByMonotonicFunctions( - const ASTPtr & node, + const Tree & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type); bool canConstantBeWrappedByFunctions( - const ASTPtr & ast, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type); + const Tree & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type); /// If it's possible to make an RPNElement /// that will filter values (possibly tuples) by the content of 'prepared_set', diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index ae3b533918d..695b7509a59 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -677,7 +677,7 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition( minmax_columns_types = data.getMinMaxColumnsTypes(partition_key); minmax_idx_condition.emplace( - query_info, context, minmax_columns_names, data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context))); + query_info.query, query_info.syntax_analyzer_result, query_info.sets, context, minmax_columns_names, data.getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(context))); partition_pruner.emplace(metadata_snapshot, query_info, context, false /* strict */); if (settings.force_index_by_date && (minmax_idx_condition->alwaysUnknownOrTrue() && partition_pruner->isUseless())) @@ -1116,7 +1116,7 @@ size_t MergeTreeDataSelectExecutor::estimateNumMarksToRead( const auto & primary_key = metadata_snapshot->getPrimaryKey(); Names primary_key_columns = primary_key.column_names; - KeyCondition key_condition(query_info, context, primary_key_columns, primary_key.expression); + KeyCondition key_condition(query_info.query, query_info.syntax_analyzer_result, query_info.sets, context, primary_key_columns, primary_key.expression); if (key_condition.alwaysUnknownOrTrue()) { diff --git a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp index 099d561cf80..40ccb48a72e 100644 --- a/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexMinMax.cpp @@ -140,7 +140,7 @@ MergeTreeIndexConditionMinMax::MergeTreeIndexConditionMinMax( const SelectQueryInfo & query, ContextPtr context) : index_data_types(index.data_types) - , condition(query, context, index.column_names, index.expression) + , condition(query.query, query.syntax_analyzer_result, query.sets, context, index.column_names, index.expression) { } diff --git a/src/Storages/MergeTree/PartitionPruner.h b/src/Storages/MergeTree/PartitionPruner.h index fbed0e6ab99..3af52fd9a38 100644 --- a/src/Storages/MergeTree/PartitionPruner.h +++ b/src/Storages/MergeTree/PartitionPruner.h @@ -27,7 +27,8 @@ public: PartitionPruner(const StorageMetadataPtr & metadata, const SelectQueryInfo & query_info, ContextPtr context, bool strict) : partition_key(MergeTreePartition::adjustPartitionKey(metadata, context)) , partition_condition( - query_info, context, partition_key.column_names, partition_key.expression, true /* single_point */, strict) + query_info.query, query_info.syntax_analyzer_result, query_info.sets, + context, partition_key.column_names, partition_key.expression, true /* single_point */, strict) , useless(strict ? partition_condition.anyUnknownOrAlwaysTrue() : partition_condition.alwaysUnknownOrTrue()) { } From 68176f064bdbc21356e5d5485752e394b1e32e2f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 21 Jun 2021 20:28:15 +0300 Subject: [PATCH 005/117] Fix some tests. --- src/Storages/MergeTree/KeyCondition.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 16807d0361c..9bc01375c3e 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -345,7 +345,7 @@ public: if (ast) { const auto * func = assert_cast(ast); - return func->arguments ? 0 : func->arguments->size(); + return func->arguments ? func->arguments->children.size() : 0; } else return dag->children.size(); @@ -1266,6 +1266,8 @@ bool KeyCondition::tryPrepareSetIndex( for (size_t i = 0; i < left_args_count; ++i) get_key_tuple_position_mapping(left_arg_tuple.getArgumentAt(i), i); } + else + get_key_tuple_position_mapping(left_arg, 0); } else get_key_tuple_position_mapping(left_arg, 0); From d15d16fee0956f55a106199fc3028dc8106576ff Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 22 Jun 2021 10:26:45 +0300 Subject: [PATCH 006/117] Fix build. --- src/Storages/MergeTree/KeyCondition.cpp | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 9bc01375c3e..05b5399a84f 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -120,9 +120,7 @@ static void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuf { switch (node.type) { - case (ActionsDAG::ActionType::INPUT): - writeString(node.result_name, out); - break; + case (ActionsDAG::ActionType::INPUT): [[fallthrough]]; case (ActionsDAG::ActionType::COLUMN): writeString(node.result_name, out); break; @@ -167,8 +165,8 @@ static std::string getColumnNameWithoutAlias(const ActionsDAG::Node & node, bool class KeyCondition::Tree { public: - explicit Tree(const IAST * ast_) : ast(ast_) {} - explicit Tree(const ActionsDAG::Node * dag_) : dag(dag_) {} + explicit Tree(const IAST * ast_) : ast(ast_) { assert(ast); } + explicit Tree(const ActionsDAG::Node * dag_) : dag(dag_) { assert(dag); } std::string getColumnName() const { From 21e39e10ea93dcd7624a1ae122fa6c7f793bda9e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 22 Jun 2021 13:28:56 +0300 Subject: [PATCH 007/117] Update KeyCondition constructor --- src/Interpreters/ActionsDAG.cpp | 3 ++ src/Interpreters/ActionsDAG.h | 6 ++++ src/Storages/MergeTree/KeyCondition.cpp | 42 ++++++++++++++----------- src/Storages/MergeTree/KeyCondition.h | 3 +- 4 files changed, 34 insertions(+), 20 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 9fa48f6ceab..8b1c5afa014 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -161,6 +161,9 @@ const ActionsDAG::Node & ActionsDAG::addArrayJoin(const Node & child, std::strin if (!array_type) throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH); + if (result_name.empty()) + result_name = "arrayJoin(" + child.result_name + ")"; + Node node; node.type = ActionType::ARRAY_JOIN; node.result_type = array_type->getNestedType(); diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 9bea1d1c040..e06662af725 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -279,4 +279,10 @@ private: static ActionsDAGPtr cloneActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs); }; +/// This is an ugly way to bypass impossibility to forward declare ActionDAG::Node. +struct ActionDAGNodes +{ + ActionsDAG::NodeRawConstPtrs nodes; +}; + } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 05b5399a84f..fd9e846cf16 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -644,11 +644,14 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( switch (node.type) { case (ActionsDAG::ActionType::INPUT): - /// Should be already added - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot clone input in cloneASTWithInversionPushDown"); + { + /// Note: inputs order is not important here. Will match columns by names. + res = &inverted_dag.addInput({node.column, node.result_type, node.result_name}); + break; + } case (ActionsDAG::ActionType::COLUMN): { - res = &inverted_dag.addInput({node.column, node.result_type, node.result_name}); + res = &inverted_dag.addColumn({node.column, node.result_type, node.result_name}); break; } case (ActionsDAG::ActionType::ALIAS): @@ -661,7 +664,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( case (ActionsDAG::ActionType::ARRAY_JOIN): { const auto & arg = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, false); - res = &inverted_dag.addArrayJoin(arg, node.result_name); + res = &inverted_dag.addArrayJoin(arg, ""); break; } case (ActionsDAG::ActionType::FUNCTION): @@ -691,6 +694,8 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( assert(function_builder); + /// We match columns by name, so it is important to fill name correctly. + /// So, use empty string to make it automatically. const auto & func = inverted_dag.addFunction(function_builder, children, ""); to_inverted[&node] = &func; return func; @@ -722,23 +727,22 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( return *res; } -static ActionsDAGPtr cloneASTWithInversionPushDown(const ActionsDAG & dag, const ContextPtr & context) +static ActionsDAGPtr cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context) { - auto res = std::make_shared(dag.getRequiredColumns()); + auto res = std::make_shared(); std::unordered_map to_inverted; - const auto & res_inputs = res->getInputs(); - auto it = res_inputs.begin(); - for (const auto * input : dag.getInputs()) + + for (auto & node : nodes) + node = &cloneASTWithInversionPushDown(*node, *res, to_inverted, context, false); + + if (nodes.size() > 1) { - to_inverted[input] = *it; - ++it; + auto function_builder = FunctionFactory::instance().get("and", context); + nodes = {&res->addFunction(function_builder, std::move(nodes), "")}; } - ActionsDAG::NodeRawConstPtrs index; - index.reserve(dag.getIndex().size()); - for (const auto * node : dag.getIndex()) - index.push_back(&cloneASTWithInversionPushDown(*node, *res, to_inverted, context, false)); + res->getIndex().swap(nodes); return res; } @@ -828,7 +832,7 @@ KeyCondition::KeyCondition( } KeyCondition::KeyCondition( - const ActionsDAG & dag, + ActionDAGNodes dag_nodes, PreparedSets prepared_sets_, ContextPtr context, const Names & key_column_names, @@ -848,10 +852,10 @@ KeyCondition::KeyCondition( key_columns[name] = i; } - auto inverted_dag = cloneASTWithInversionPushDown(dag, context); - - if (!inverted_dag->getIndex().empty()) + if (!dag_nodes.nodes.empty()) { + auto inverted_dag = cloneASTWithInversionPushDown(std::move(dag_nodes.nodes), context); + Block empty; for (const auto * node : inverted_dag->getIndex()) traverseAST(Tree(node), context, empty); diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index f0ffbf9e6bf..8c54d9bb751 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -18,6 +18,7 @@ class IFunction; using FunctionBasePtr = std::shared_ptr; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; +struct ActionDAGNodes; /** A field, that can be stored in two representations: * - A standalone field. @@ -238,7 +239,7 @@ public: bool strict_ = false); KeyCondition( - const ActionsDAG & dag, + ActionDAGNodes dag_nodes, PreparedSets prepared_sets_, ContextPtr context, const Names & key_column_names, From 47f130d39c7f654960386efadbaa43854ebac5a4 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 22 Jun 2021 16:54:00 +0300 Subject: [PATCH 008/117] Try use expression for KeyCondition from query plan. --- src/Core/Settings.h | 1 + .../QueryPlan/Optimizations/Optimizations.h | 2 + .../optimizePrimaryKeyCondition.cpp | 53 +++++++++++++++++++ src/Processors/QueryPlan/QueryPlan.cpp | 1 + .../QueryPlan/ReadFromMergeTree.cpp | 32 +++++++++-- src/Processors/QueryPlan/ReadFromMergeTree.h | 9 ++++ src/Storages/MergeTree/KeyCondition.cpp | 16 ++++-- 7 files changed, 104 insertions(+), 10 deletions(-) create mode 100644 src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 84e7500b064..a99f06904fd 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -473,6 +473,7 @@ class IColumn; M(Bool, query_plan_enable_optimizations, true, "Apply optimizations to query plan", 0) \ M(UInt64, query_plan_max_optimizations_to_apply, 10000, "Limit the total number of optimizations applied to query plan. If zero, ignored. If limit reached, throw exception", 0) \ M(Bool, query_plan_filter_push_down, true, "Allow to push down filter by predicate query plan step", 0) \ + M(Bool, query_plan_optimize_primary_key, true, "Analyze primary key using query plan (instead of AST)", 0) \ \ M(UInt64, limit, 0, "Limit on read rows from the most 'end' result for select query, default 0 means no limit length", 0) \ M(UInt64, offset, 0, "Offset on read rows from the most 'end' result for select query", 0) \ diff --git a/src/Processors/QueryPlan/Optimizations/Optimizations.h b/src/Processors/QueryPlan/Optimizations/Optimizations.h index 10bc6293537..9808f8c1850 100644 --- a/src/Processors/QueryPlan/Optimizations/Optimizations.h +++ b/src/Processors/QueryPlan/Optimizations/Optimizations.h @@ -12,6 +12,8 @@ namespace QueryPlanOptimizations /// This is the main function which optimizes the whole QueryPlan tree. void optimizeTree(const QueryPlanOptimizationSettings & settings, QueryPlan::Node & root, QueryPlan::Nodes & nodes); +void optimizePrimaryKeyCondition(QueryPlan::Node & root); + /// Optimization is a function applied to QueryPlan::Node. /// It can read and update subtree of specified node. /// It return the number of updated layers of subtree if some change happened. diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp new file mode 100644 index 00000000000..5724e50f893 --- /dev/null +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -0,0 +1,53 @@ +#include +#include +#include +#include +#include +#include + +namespace DB::QueryPlanOptimizations +{ + +void optimizePrimaryKeyCondition(QueryPlan::Node & root) +{ + struct Frame + { + QueryPlan::Node * node = nullptr; + size_t next_child = 0; + }; + + std::stack stack; + stack.push({.node = &root}); + + while (!stack.empty()) + { + auto & frame = stack.top(); + + /// Traverse all children first. + if (frame.next_child < frame.node->children.size()) + { + stack.push({.node = frame.node->children[frame.next_child]}); + + ++frame.next_child; + continue; + } + + if (auto * filter_step = typeid_cast(frame.node->step.get())) + { + auto * child = frame.node->children.at(0); + if (typeid_cast(child->step.get())) + { + auto * child_child = child->children.at(0); + if (auto * read_from_merge_tree = typeid_cast(child_child->step.get())) + read_from_merge_tree->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); + } + else if (auto * read_from_merge_tree = typeid_cast(child->step.get())) + read_from_merge_tree->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); + + } + + stack.pop(); + } +} + +} diff --git a/src/Processors/QueryPlan/QueryPlan.cpp b/src/Processors/QueryPlan/QueryPlan.cpp index 44c5c48975c..c13b866d078 100644 --- a/src/Processors/QueryPlan/QueryPlan.cpp +++ b/src/Processors/QueryPlan/QueryPlan.cpp @@ -432,6 +432,7 @@ void QueryPlan::explainPipeline(WriteBuffer & buffer, const ExplainPipelineOptio void QueryPlan::optimize(const QueryPlanOptimizationSettings & optimization_settings) { QueryPlanOptimizations::optimizeTree(optimization_settings, *root, nodes); + QueryPlanOptimizations::optimizePrimaryKeyCondition(*root); } } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index c9204f41ce4..dc0eed5bb73 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -788,16 +788,38 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre // Build and check if primary key is used when necessary const auto & primary_key = metadata_snapshot->getPrimaryKey(); Names primary_key_columns = primary_key.column_names; - KeyCondition key_condition(query_info.query, query_info.syntax_analyzer_result, query_info.sets, context, primary_key_columns, primary_key.expression); + std::optional key_condition; - if (settings.force_primary_key && key_condition.alwaysUnknownOrTrue()) + if (settings.query_plan_optimize_primary_key) + { + ActionDAGNodes nodes; + if (prewhere_info) + { + const auto & node = prewhere_info->prewhere_actions->getActionsDAG().findInIndex(prewhere_info->prewhere_column_name); + nodes.nodes.push_back(&node); + } + + if (added_filter) + { + const auto & node = added_filter->findInIndex(added_filter_column_name); + nodes.nodes.push_back(&node); + } + + key_condition.emplace(std::move(nodes), query_info.sets, context, primary_key_columns, primary_key.expression); + } + else + { + key_condition.emplace(query_info.query, query_info.syntax_analyzer_result, query_info.sets, context, primary_key_columns, primary_key.expression); + } + + if (settings.force_primary_key && key_condition->alwaysUnknownOrTrue()) { throw Exception( ErrorCodes::INDEX_NOT_USED, "Primary key ({}) is not used and setting 'force_primary_key' is set.", fmt::join(primary_key_columns, ", ")); } - LOG_DEBUG(log, "Key condition: {}", key_condition.toString()); + LOG_DEBUG(log, "Key condition: {}", key_condition->toString()); const auto & select = query_info.query->as(); @@ -806,7 +828,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre max_block_numbers_to_read.get(), log, result.index_stats); result.sampling = MergeTreeDataSelectExecutor::getSampling( - select, metadata_snapshot->getColumns().getAllPhysical(), parts, key_condition, + select, metadata_snapshot->getColumns().getAllPhysical(), parts, *key_condition, data, metadata_snapshot, context, sample_factor_column_queried, log); if (result.sampling.read_nothing) @@ -823,7 +845,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre metadata_snapshot, query_info, context, - key_condition, + *key_condition, reader_settings, log, requested_num_streams, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 6e1efffdb02..259cb32d82a 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -80,6 +80,12 @@ public: void describeActions(JSONBuilder::JSONMap & map) const override; void describeIndexes(JSONBuilder::JSONMap & map) const override; + void addFilter(ActionsDAGPtr expression, std::string column_name) + { + added_filter = std::move(expression); + added_filter_column_name = std::move(column_name); + } + private: const MergeTreeReaderSettings reader_settings; @@ -91,6 +97,9 @@ private: SelectQueryInfo query_info; PrewhereInfoPtr prewhere_info; + ActionsDAGPtr added_filter; + std::string added_filter_column_name; + StorageMetadataPtr metadata_snapshot; StorageMetadataPtr metadata_snapshot_base; diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index fd9e846cf16..60cf6b82abd 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -201,7 +201,7 @@ public: if (ast) return typeid_cast(ast); else - return dag->type == ActionsDAG::ActionType::COLUMN; + return dag->column && isColumnConst(*dag->column); } ColumnWithTypeAndName getConstant() const @@ -310,10 +310,16 @@ public: { if (dag->column) { - const auto * col_set = typeid_cast(dag->column.get()); - auto set = col_set->getData(); - if (set->isCreated()) - return set; + const IColumn * col = dag->column.get(); + if (const auto * col_const = typeid_cast(col)) + col = &col_const->getDataColumn(); + + if (const auto * col_set = typeid_cast(col)) + { + auto set = col_set->getData(); + if (set->isCreated()) + return set; + } } } From f5f57781b718542a84a81b5b386614fccd325dd6 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 22 Jun 2021 17:45:22 +0300 Subject: [PATCH 009/117] Fix build after merge. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- src/Storages/MergeTree/KeyCondition.cpp | 4 ++++ src/Storages/MergeTree/KeyCondition.h | 1 + 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index dc0eed5bb73..84520b714f7 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -805,7 +805,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre nodes.nodes.push_back(&node); } - key_condition.emplace(std::move(nodes), query_info.sets, context, primary_key_columns, primary_key.expression); + key_condition.emplace(std::move(nodes), query_info.syntax_analyzer_result, query_info.sets, context, primary_key_columns, primary_key.expression); } else { diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index ac60b32246c..996c6bc01ee 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -842,6 +842,7 @@ KeyCondition::KeyCondition( KeyCondition::KeyCondition( ActionDAGNodes dag_nodes, + TreeRewriterResultPtr syntax_analyzer_result, PreparedSets prepared_sets_, ContextPtr context, const Names & key_column_names, @@ -861,6 +862,9 @@ KeyCondition::KeyCondition( key_columns[name] = i; } + for (const auto & [name, _] : syntax_analyzer_result->array_join_result_to_source) + array_joined_columns.insert(name); + if (!dag_nodes.nodes.empty()) { auto inverted_dag = cloneASTWithInversionPushDown(std::move(dag_nodes.nodes), context); diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index c01e32cf2da..e3ac56b2e3b 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -240,6 +240,7 @@ public: KeyCondition( ActionDAGNodes dag_nodes, + TreeRewriterResultPtr syntax_analyzer_result, PreparedSets prepared_sets_, ContextPtr context, const Names & key_column_names, From a45290bfb363f7c6b2902417999bcd29652f2c6e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 22 Jun 2021 18:52:14 +0300 Subject: [PATCH 010/117] Fix some tests. --- src/Storages/MergeTree/KeyCondition.cpp | 4 +++- tests/queries/0_stateless/00160_merge_and_index_in_in.sql | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 996c6bc01ee..07298593abb 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -683,7 +683,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( return arg; } - if (isLogicalOperator(name)) + if (isLogicalOperator(name) && need_inversion) { ActionsDAG::NodeRawConstPtrs children(node.children); @@ -869,6 +869,8 @@ KeyCondition::KeyCondition( { auto inverted_dag = cloneASTWithInversionPushDown(std::move(dag_nodes.nodes), context); + // std::cerr << "========== inverted dag: " << inverted_dag->dumpDAG() << std::endl; + Block empty; for (const auto * node : inverted_dag->getIndex()) traverseAST(Tree(node), context, empty); diff --git a/tests/queries/0_stateless/00160_merge_and_index_in_in.sql b/tests/queries/0_stateless/00160_merge_and_index_in_in.sql index bdab3f7640d..3ed829c4d59 100644 --- a/tests/queries/0_stateless/00160_merge_and_index_in_in.sql +++ b/tests/queries/0_stateless/00160_merge_and_index_in_in.sql @@ -9,6 +9,7 @@ SET max_block_size = 1000000; INSERT INTO mt_00160 (x) SELECT number AS x FROM system.numbers LIMIT 100000; SELECT *, b FROM mt_00160 WHERE x IN (12345, 67890) AND NOT ignore(blockSize() < 10 AS b) ORDER BY x; +SET query_plan_optimize_primary_key = 0; -- Need separate query plan step for merge SELECT *, b FROM merge_00160 WHERE x IN (12345, 67890) AND NOT ignore(blockSize() < 10 AS b) ORDER BY x; DROP TABLE merge_00160; From c22f856d3620ad8e5d2b082da767ea34cead7879 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 23 Jun 2021 15:19:22 +0300 Subject: [PATCH 011/117] Fix indexHint --- src/CMakeLists.txt | 4 +- src/Functions/IFunctionAdaptors.h | 2 + src/Functions/indexHint.cpp | 54 +-------------- src/Functions/indexHint.h | 67 +++++++++++++++++++ src/Interpreters/ActionsVisitor.cpp | 38 ++++++++++- src/Processors/ya.make | 1 + src/Storages/MergeTree/KeyCondition.cpp | 28 ++++++-- .../0_stateless/01739_index_hint.reference | 4 +- .../queries/0_stateless/01739_index_hint.sql | 2 +- .../01783_merge_engine_join_key_condition.sql | 1 + 10 files changed, 138 insertions(+), 63 deletions(-) create mode 100644 src/Functions/indexHint.h diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 88a6113b8fa..791085eee92 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -107,8 +107,8 @@ endif() list (APPEND clickhouse_common_io_sources ${CONFIG_BUILD}) list (APPEND clickhouse_common_io_headers ${CONFIG_VERSION} ${CONFIG_COMMON}) -list (APPEND dbms_sources Functions/IFunction.cpp Functions/FunctionFactory.cpp Functions/FunctionHelpers.cpp Functions/extractTimeZoneFromFunctionArguments.cpp Functions/replicate.cpp Functions/FunctionsLogical.cpp) -list (APPEND dbms_headers Functions/IFunction.h Functions/FunctionFactory.h Functions/FunctionHelpers.h Functions/extractTimeZoneFromFunctionArguments.h Functions/replicate.h Functions/FunctionsLogical.h) +list (APPEND dbms_sources Functions/IFunction.cpp Functions/FunctionFactory.cpp Functions/FunctionHelpers.cpp Functions/extractTimeZoneFromFunctionArguments.cpp Functions/replicate.cpp Functions/FunctionsLogical.cpp Functions/indexHint.cpp) +list (APPEND dbms_headers Functions/IFunction.h Functions/FunctionFactory.h Functions/FunctionHelpers.h Functions/extractTimeZoneFromFunctionArguments.h Functions/replicate.h Functions/FunctionsLogical.h Functions/indexHint.h) list (APPEND dbms_sources AggregateFunctions/IAggregateFunction.cpp diff --git a/src/Functions/IFunctionAdaptors.h b/src/Functions/IFunctionAdaptors.h index 6a865af0dd3..e032f2ca90d 100644 --- a/src/Functions/IFunctionAdaptors.h +++ b/src/Functions/IFunctionAdaptors.h @@ -133,6 +133,8 @@ public: void getLambdaArgumentTypesImpl(DataTypes & arguments) const override { function->getLambdaArgumentTypes(arguments); } + const IFunction * getFunction() const { return function.get(); } + private: std::shared_ptr function; }; diff --git a/src/Functions/indexHint.cpp b/src/Functions/indexHint.cpp index f3c856c38ce..8637a6762b9 100644 --- a/src/Functions/indexHint.cpp +++ b/src/Functions/indexHint.cpp @@ -1,4 +1,4 @@ -#include +#include #include #include @@ -6,58 +6,6 @@ namespace DB { - -/** The `indexHint` function takes any number of any arguments and always returns one. - * - * This function has a special meaning (see ExpressionAnalyzer, KeyCondition) - * - the expressions inside it are not evaluated; - * - but when analyzing the index (selecting ranges for reading), this function is treated the same way, - * as if instead of using it the expression itself would be. - * - * Example: WHERE something AND indexHint(CounterID = 34) - * - do not read or calculate CounterID = 34, but select ranges in which the CounterID = 34 expression can be true. - * - * The function can be used for debugging purposes, as well as for (hidden from the user) query conversions. - */ -class FunctionIndexHint : public IFunction -{ -public: - static constexpr auto name = "indexHint"; - static FunctionPtr create(ContextPtr) - { - return std::make_shared(); - } - - bool isVariadic() const override - { - return true; - } - size_t getNumberOfArguments() const override - { - return 0; - } - - bool useDefaultImplementationForNulls() const override { return false; } - - bool isSuitableForConstantFolding() const override { return false; } - - String getName() const override - { - return name; - } - DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override - { - return std::make_shared(); - } - - ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override - { - return DataTypeUInt8().createColumnConst(input_rows_count, 1u); - } - -}; - - void registerFunctionIndexHint(FunctionFactory & factory) { factory.registerFunction(); diff --git a/src/Functions/indexHint.h b/src/Functions/indexHint.h new file mode 100644 index 00000000000..db838df7763 --- /dev/null +++ b/src/Functions/indexHint.h @@ -0,0 +1,67 @@ +#include +#include +#include + + +namespace DB +{ + +class ActionsDAG; +using ActionsDAGPtr = std::shared_ptr; + +/** The `indexHint` function takes any number of any arguments and always returns one. + * + * This function has a special meaning (see ExpressionAnalyzer, KeyCondition) + * - the expressions inside it are not evaluated; + * - but when analyzing the index (selecting ranges for reading), this function is treated the same way, + * as if instead of using it the expression itself would be. + * + * Example: WHERE something AND indexHint(CounterID = 34) + * - do not read or calculate CounterID = 34, but select ranges in which the CounterID = 34 expression can be true. + * + * The function can be used for debugging purposes, as well as for (hidden from the user) query conversions. + */ +class FunctionIndexHint : public IFunction +{ +public: + static constexpr auto name = "indexHint"; + static FunctionPtr create(ContextPtr) + { + return std::make_shared(); + } + + bool isVariadic() const override + { + return true; + } + size_t getNumberOfArguments() const override + { + return 0; + } + + bool useDefaultImplementationForNulls() const override { return false; } + + bool isSuitableForConstantFolding() const override { return false; } + + String getName() const override + { + return name; + } + DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override + { + return std::make_shared(); + } + + ColumnPtr executeImpl(const ColumnsWithTypeAndName &, const DataTypePtr &, size_t input_rows_count) const override + { + return DataTypeUInt8().createColumnConst(input_rows_count, 1u); + } + + void setActions(ActionsDAGPtr actions_) { actions = std::move(actions_); } + const ActionsDAGPtr & getActions() const { return actions; } + +private: + ActionsDAGPtr actions; +}; + +} diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index 20f54e9b66e..b74ee4ba02a 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -3,6 +3,7 @@ #include #include +#include #include @@ -818,8 +819,43 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & /// A special function `indexHint`. Everything that is inside it is not calculated if (node.name == "indexHint") { + if (data.only_consts) + return; + + /// Here we create a separate DAG for indexHint condition. + /// It will be used only for index analysis. + Data index_hint_data( + data.getContext(), + data.set_size_limit, + data.subquery_depth, + data.source_columns, + std::make_shared(data.source_columns), + data.prepared_sets, + data.subqueries_for_sets, + data.no_subqueries, + data.no_makeset, + data.only_consts, + /*create_source_for_in*/ false); + + NamesWithAliases args; + + if (node.arguments) + { + for (const auto & arg : node.arguments->children) + { + visit(arg, index_hint_data); + args.push_back({arg->getColumnNameWithoutAlias(), {}}); + } + } + + auto dag = index_hint_data.getActions(); + dag->project(args); + + auto index_hint = std::make_shared(); + index_hint->setActions(std::move(dag)); + // Arguments are removed. We add function instead of constant column to avoid constant folding. - data.addFunction(FunctionFactory::instance().get("indexHint", data.getContext()), {}, column_name); + data.addFunction(std::make_unique(index_hint), {}, column_name); return; } diff --git a/src/Processors/ya.make b/src/Processors/ya.make index 86a40685d1f..f86577bde99 100644 --- a/src/Processors/ya.make +++ b/src/Processors/ya.make @@ -119,6 +119,7 @@ SRCS( QueryPlan/Optimizations/liftUpArrayJoin.cpp QueryPlan/Optimizations/limitPushDown.cpp QueryPlan/Optimizations/mergeExpressions.cpp + QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp QueryPlan/Optimizations/optimizeTree.cpp QueryPlan/Optimizations/splitFilter.cpp QueryPlan/PartialSortingStep.cpp diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 07298593abb..9bc26b8ba9b 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include #include @@ -683,7 +684,27 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( return arg; } - if (isLogicalOperator(name) && need_inversion) + if (name == "indexHint") + { + ActionsDAG::NodeRawConstPtrs children; + if (const auto * adaptor = typeid_cast(node.function_builder.get())) + { + if (const auto * index_hint = typeid_cast(adaptor->getFunction())) + { + const auto & index_hint_dag = index_hint->getActions(); + children = index_hint_dag->getIndex(); + + for (auto & arg : children) + arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, need_inversion); + } + } + + const auto & func = inverted_dag.addFunction(node.function_builder, children, ""); + to_inverted[&node] = &func; + return func; + } + + if (need_inversion && (name == "and" || name == "or")) { ActionsDAG::NodeRawConstPtrs children(node.children); @@ -691,9 +712,8 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( arg = &cloneASTWithInversionPushDown(*arg, inverted_dag, to_inverted, context, need_inversion); FunctionOverloadResolverPtr function_builder; - if (name == "indexHint") - function_builder = node.function_builder; - else if (name == "and") + + if (name == "and") function_builder = FunctionFactory::instance().get("or", context); else if (name == "or") function_builder = FunctionFactory::instance().get("and", context); diff --git a/tests/queries/0_stateless/01739_index_hint.reference b/tests/queries/0_stateless/01739_index_hint.reference index 6aa40c5d302..e0877a10007 100644 --- a/tests/queries/0_stateless/01739_index_hint.reference +++ b/tests/queries/0_stateless/01739_index_hint.reference @@ -30,6 +30,6 @@ SELECT count() FROM XXXX WHERE indexHint(t = 42); drop table if exists XXXX; create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=8192; insert into XXXX select number*60, 0 from numbers(100000); -SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)); -100000 +SELECT count() >= 1 FROM XXXX WHERE indexHint(t = toDateTime(0)); +1 drop table XXXX; diff --git a/tests/queries/0_stateless/01739_index_hint.sql b/tests/queries/0_stateless/01739_index_hint.sql index 28395c2dc1d..99cd7d001a4 100644 --- a/tests/queries/0_stateless/01739_index_hint.sql +++ b/tests/queries/0_stateless/01739_index_hint.sql @@ -30,6 +30,6 @@ create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings inde insert into XXXX select number*60, 0 from numbers(100000); -SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)); +SELECT count() >= 1 FROM XXXX WHERE indexHint(t = toDateTime(0)); drop table XXXX; diff --git a/tests/queries/0_stateless/01783_merge_engine_join_key_condition.sql b/tests/queries/0_stateless/01783_merge_engine_join_key_condition.sql index 372c1bd3572..606597850ab 100644 --- a/tests/queries/0_stateless/01783_merge_engine_join_key_condition.sql +++ b/tests/queries/0_stateless/01783_merge_engine_join_key_condition.sql @@ -11,6 +11,7 @@ CREATE TABLE t2 (Id Int32, Val Int32, X Int32) Engine=Memory; INSERT INTO t2 values (4, 3, 4); SET force_primary_key = 1, force_index_by_date=1; +SET query_plan_optimize_primary_key = 0; SELECT * FROM foo_merge WHERE Val = 3 AND Id = 3; SELECT count(), X FROM foo_merge JOIN t2 USING Val WHERE Val = 3 AND Id = 3 AND t2.X == 4 GROUP BY X; From 4186bc29fdb474e1dc534371bf2afbb7f0862534 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 29 Jun 2021 20:01:22 +0300 Subject: [PATCH 012/117] Fix build. --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 6e453fd97e9..06dfc6aa22f 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -796,7 +796,7 @@ ReadFromMergeTree::AnalysisResult ReadFromMergeTree::selectRangesToRead(MergeTre ActionDAGNodes nodes; if (prewhere_info) { - const auto & node = prewhere_info->prewhere_actions->getActionsDAG().findInIndex(prewhere_info->prewhere_column_name); + const auto & node = prewhere_info->prewhere_actions->findInIndex(prewhere_info->prewhere_column_name); nodes.nodes.push_back(&node); } From 8a94a10b83fb51f472c16481c0bc003e4cb04652 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 14 Jul 2021 12:08:30 +0300 Subject: [PATCH 013/117] Fix style. --- src/Functions/indexHint.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/indexHint.h b/src/Functions/indexHint.h index db838df7763..0dc44e613ef 100644 --- a/src/Functions/indexHint.h +++ b/src/Functions/indexHint.h @@ -1,3 +1,4 @@ +#pragma once #include #include #include From d274b05fac7957067bf7ff37e4120c2b8ea2c4fd Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 7 Jul 2022 20:19:15 +0000 Subject: [PATCH 014/117] improvements --- docker/packager/packager | 2 +- tests/integration/helpers/cluster.py | 175 ++++++++++-------- tests/integration/pytest.ini | 2 +- tests/integration/runner | 1 + .../configs/users.d/network.xml | 3 - .../test_allowed_client_hosts/test.py | 29 +-- .../test_allowed_url_from_config/test.py | 15 ++ .../test_dictionaries_redis/test.py | 38 ++-- .../test.py | 33 ++-- tests/integration/test_merge_tree_s3/test.py | 15 +- .../test_merge_tree_s3_restore/test.py | 24 +-- .../test.py | 17 +- .../test.py | 2 +- tests/integration/test_storage_s3/test.py | 27 +-- 14 files changed, 214 insertions(+), 169 deletions(-) diff --git a/docker/packager/packager b/docker/packager/packager index 7c0f046b76c..98c8e49385a 100755 --- a/docker/packager/packager +++ b/docker/packager/packager @@ -62,7 +62,7 @@ def pre_build(repo_path: str, env_variables: List[str]): f"git -C {repo_path} fetch --no-recurse-submodules " "--no-tags origin master:master" ) - logging.info("Getting master branch for performance artifact: ''%s'", cmd) + logging.info("Getting master branch for performance artifact: '%s'", cmd) subprocess.check_call(cmd, shell=True) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 5983c886680..f5c546ff264 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -237,14 +237,22 @@ def enable_consistent_hash_plugin(rabbitmq_id): return p.returncode == 0 -def get_instances_dir(): - if ( - "INTEGRATION_TESTS_RUN_ID" in os.environ - and os.environ["INTEGRATION_TESTS_RUN_ID"] - ): - return "_instances_" + shlex.quote(os.environ["INTEGRATION_TESTS_RUN_ID"]) - else: - return "_instances" +def get_instances_dir(name): + instances_dir_name = "_instances" + + worker_name = os.environ.get("PYTEST_XDIST_WORKER", "") + run_id = os.environ.get("INTEGRATION_TESTS_RUN_ID", "") + + if worker_name: + instances_dir_name += "_" + worker_name + + if name: + instances_dir_name += "_" + name + + if run_id: + instances_dir_name += "_" + shlex.quote(run_id) + + return instances_dir_name class ClickHouseCluster: @@ -270,6 +278,7 @@ class ClickHouseCluster: zookeeper_keyfile=None, zookeeper_certfile=None, ): + logging.debug(f"INIT CALLED") for param in list(os.environ.keys()): logging.debug("ENV %40s %s" % (param, os.environ[param])) self.base_path = base_path @@ -306,19 +315,8 @@ class ClickHouseCluster: ) # docker-compose removes everything non-alphanumeric from project names so we do it too. self.project_name = re.sub(r"[^a-z0-9]", "", project_name.lower()) - instances_dir_name = "_instances" - if self.name: - instances_dir_name += "_" + self.name - - if ( - "INTEGRATION_TESTS_RUN_ID" in os.environ - and os.environ["INTEGRATION_TESTS_RUN_ID"] - ): - instances_dir_name += "_" + shlex.quote( - os.environ["INTEGRATION_TESTS_RUN_ID"] - ) - - self.instances_dir = p.join(self.base_dir, instances_dir_name) + self.instances_dir_name = get_instances_dir(self.name) + self.instances_dir = p.join(self.base_dir, self.instances_dir_name) self.docker_logs_path = p.join(self.instances_dir, "docker.log") self.env_file = p.join(self.instances_dir, DEFAULT_ENV_NAME) self.env_variables = {} @@ -536,8 +534,37 @@ class ClickHouseCluster: self.is_up = False self.env = os.environ.copy() logging.debug(f"CLUSTER INIT base_config_dir:{self.base_config_dir}") + if p.exists(self.instances_dir): + shutil.rmtree(self.instances_dir, ignore_errors=True) + logging.debug(f"Removed :{self.instances_dir}") + os.mkdir(self.instances_dir) + + + def print_all_docker_pieces(self): + res_networks = subprocess.check_output( + f"docker network ls --filter name='{self.project_name}*'", + shell=True, + universal_newlines=True, + ) + logging.debug(f"Docker networks for project {self.project_name} are {res_networks}") + res_containers = subprocess.check_output( + f"docker container ls -a --filter name='{self.project_name}*'", + shell=True, + universal_newlines=True, + ) + logging.debug(f"Docker containers for project {self.project_name} are {res_containers}") + res_volumes = subprocess.check_output( + f"docker volume ls --filter name='{self.project_name}*'", + shell=True, + universal_newlines=True, + ) + logging.debug(f"Docker volumes for project {self.project_name} are {res_volumes}") + def cleanup(self): + logging.debug('Cleanup called') + self.print_all_docker_pieces() + if ( os.environ and "DISABLE_CLEANUP" in os.environ @@ -549,12 +576,16 @@ class ClickHouseCluster: # Just in case kill unstopped containers from previous launch try: unstopped_containers = self.get_running_containers() - if unstopped_containers: + logging.debug(f"Unstopped containers: {unstopped_containers}") + if len(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) + try: + run_and_check(f"docker kill {id}", shell=True, nothrow=True) + except: + pass run_and_check(f"docker rm {id}", shell=True, nothrow=True) unstopped_containers = self.get_running_containers() if unstopped_containers: @@ -563,26 +594,33 @@ class ClickHouseCluster: logging.debug(f"Unstopped containers killed.") else: logging.debug(f"No running containers for project: {self.project_name}") + except Exception as ex: + logging.debug(f"Got exception removing containers {str(ex)}") + + # # Just in case remove unused networks + try: + logging.debug("Trying to prune unused networks...") + + list_networks = subprocess.check_output( + f"docker network ls -q --filter name='{self.project_name}'", + shell=True, + universal_newlines=True, + ).splitlines() + if list_networks: + logging.debug(f"Trying to remove networks: {list_networks}") + subprocess.check_call(f"docker network rm {' '.join(list_networks)}", shell=True) + logging.debug(f"Networks removed: {list_networks}") except: pass - # # Just in case remove unused networks - # try: - # logging.debug("Trying to prune unused networks...") - - # run_and_check(['docker', 'network', 'prune', '-f']) - # logging.debug("Networks pruned") - # except: - # pass - # Remove unused images - # try: - # logging.debug("Trying to prune unused images...") + try: + logging.debug("Trying to prune unused images...") - # run_and_check(['docker', 'image', 'prune', '-f']) - # logging.debug("Images pruned") - # except: - # pass + run_and_check(['docker', 'image', 'prune', '-f']) + logging.debug("Images pruned") + except: + pass # Remove unused volumes try: @@ -626,7 +664,7 @@ class ClickHouseCluster: shell=True, ) containers = dict( - line.split(":", 1) for line in containers.decode("utf8").splitlines() + line.split(":", 1) for line in containers.splitlines() ) return containers @@ -1767,7 +1805,7 @@ class ClickHouseCluster: errors += [str(ex)] time.sleep(0.5) - run_and_check(["docker-compose", "ps", "--services", "--all"]) + run_and_check(["docker", "ps", "--all"]) logging.error("Can't connect to MySQL:{}".format(errors)) raise Exception("Cannot wait MySQL container") @@ -1789,7 +1827,7 @@ class ClickHouseCluster: logging.debug("Can't connect to MySQL 8 " + str(ex)) time.sleep(0.5) - run_and_check(["docker-compose", "ps", "--services", "--all"]) + run_and_check(["docker", "ps", "--all"]) raise Exception("Cannot wait MySQL 8 container") def wait_mysql_cluster_to_start(self, timeout=180): @@ -1814,7 +1852,7 @@ class ClickHouseCluster: errors += [str(ex)] time.sleep(0.5) - run_and_check(["docker-compose", "ps", "--services", "--all"]) + run_and_check(["docker", "ps", "--all"]) logging.error("Can't connect to MySQL:{}".format(errors)) raise Exception("Cannot wait MySQL container") @@ -2087,7 +2125,7 @@ class ClickHouseCluster: logging.debug("Can't connect to MeiliSearch " + str(ex)) time.sleep(1) - def wait_minio_to_start(self, timeout=180, secure=False): + def wait_minio_to_start(self, timeout=10, secure=False): self.minio_ip = self.get_instance_ip(self.minio_host) self.minio_redirect_ip = self.get_instance_ip(self.minio_redirect_host) @@ -2129,6 +2167,14 @@ class ClickHouseCluster: logging.debug("Can't connect to Minio: %s", str(ex)) time.sleep(1) + try: + with open(os.path.join(self.minio_dir, "docker.log"), "w+") as f: + subprocess.check_call( # STYLE_CHECK_ALLOW_SUBPROCESS_CHECK_CALL + self.base_minio_cmd + ["logs"], stdout=f + ) + except Exception as e: + logging.debug("Unable to get logs from docker.") + raise Exception("Can't wait Minio to start") def wait_azurite_to_start(self, timeout=180): @@ -2199,15 +2245,13 @@ class ClickHouseCluster: raise Exception("Can't wait Cassandra to start") - def start(self, destroy_dirs=True): + def start(self): pytest_xdist_logging_to_separate_files.setup() logging.info("Running tests in {}".format(self.base_path)) - logging.debug( - "Cluster start called. is_up={}, destroy_dirs={}".format( - self.is_up, destroy_dirs - ) - ) + logging.debug(f"Cluster start called. is_up={self.is_up}") + self.print_all_docker_pieces() + if self.is_up: return @@ -2217,23 +2261,9 @@ class ClickHouseCluster: logging.warning("Cleanup failed:{e}") try: - # clickhouse_pull_cmd = self.base_cmd + ['pull'] - # print(f"Pulling images for {self.base_cmd}") - # retry_exception(10, 5, subprocess_check_call, Exception, clickhouse_pull_cmd) - - if destroy_dirs and p.exists(self.instances_dir): - logging.debug(f"Removing instances dir {self.instances_dir}") - shutil.rmtree(self.instances_dir) - for instance in list(self.instances.values()): - logging.debug( - ( - "Setup directory for instance: {} destroy_dirs: {}".format( - instance.name, destroy_dirs - ) - ) - ) - instance.create_dir(destroy_dir=destroy_dirs) + logging.debug(f"Setup directory for instance: {instance.name}") + instance.create_dir() _create_env_file(os.path.join(self.env_file), self.env_variables) self.docker_client = docker.DockerClient( @@ -2627,13 +2657,9 @@ class ClickHouseCluster: def pause_container(self, instance_name): subprocess_check_call(self.base_cmd + ["pause", instance_name]) - # subprocess_check_call(self.base_cmd + ['kill', '-s SIGSTOP', instance_name]) - def unpause_container(self, instance_name): subprocess_check_call(self.base_cmd + ["unpause", instance_name]) - # subprocess_check_call(self.base_cmd + ['kill', '-s SIGCONT', instance_name]) - def open_bash_shell(self, instance_name): os.system(" ".join(self.base_cmd + ["exec", instance_name, "/bin/bash"])) @@ -3687,14 +3713,9 @@ class ClickHouseInstance: ["bash", "-c", f"sed -i 's/{replace}/{replacement}/g' {path_to_config}"] ) - def create_dir(self, destroy_dir=True): + def create_dir(self): """Create the instance directory and all the needed files there.""" - if destroy_dir: - self.destroy_dir() - elif p.exists(self.path): - return - os.makedirs(self.path) instance_config_dir = p.abspath(p.join(self.path, "configs")) @@ -3953,10 +3974,6 @@ class ClickHouseInstance: ) ) - def destroy_dir(self): - if p.exists(self.path): - shutil.rmtree(self.path) - def wait_for_path_exists(self, path, seconds): while seconds > 0: seconds -= 1 diff --git a/tests/integration/pytest.ini b/tests/integration/pytest.ini index 2a57ea5a229..772c96f7361 100644 --- a/tests/integration/pytest.ini +++ b/tests/integration/pytest.ini @@ -1,5 +1,5 @@ [pytest] -python_files = test*.py +python_files = test_*/test*.py norecursedirs = _instances* timeout = 900 junit_duration_report = call diff --git a/tests/integration/runner b/tests/integration/runner index cd07875ad1d..f7d9387f72b 100755 --- a/tests/integration/runner +++ b/tests/integration/runner @@ -383,6 +383,7 @@ if __name__ == "__main__": --volume=/run:/run/host:ro \ {dockerd_internal_volume} -e DOCKER_CLIENT_TIMEOUT=300 -e COMPOSE_HTTP_TIMEOUT=600 \ -e XTABLES_LOCKFILE=/run/host/xtables.lock \ + -e PYTHONUNBUFFERED=1 \ {env_tags} {env_cleanup} -e PYTEST_OPTS='{parallel} {opts} {tests_list} -vvv' {img} {command}".format( net=net, tty=tty, diff --git a/tests/integration/test_allowed_client_hosts/configs/users.d/network.xml b/tests/integration/test_allowed_client_hosts/configs/users.d/network.xml index 1207e2703de..6c55d61481a 100644 --- a/tests/integration/test_allowed_client_hosts/configs/users.d/network.xml +++ b/tests/integration/test_allowed_client_hosts/configs/users.d/network.xml @@ -26,9 +26,6 @@ 127.0.0.1 clientA1.com clientA3.com - clientB\d+\.ru - clientC\d+\.ru$ - ^clientD\d+\.ru$ diff --git a/tests/integration/test_allowed_client_hosts/test.py b/tests/integration/test_allowed_client_hosts/test.py index db2ba464b38..dda3439be14 100644 --- a/tests/integration/test_allowed_client_hosts/test.py +++ b/tests/integration/test_allowed_client_hosts/test.py @@ -1,4 +1,5 @@ import pytest +import logging from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) @@ -7,16 +8,6 @@ server = cluster.add_instance("server", user_configs=["configs/users.d/network.x clientA1 = cluster.add_instance("clientA1", hostname="clientA1.com") clientA2 = cluster.add_instance("clientA2", hostname="clientA2.com") clientA3 = cluster.add_instance("clientA3", hostname="clientA3.com") -clientB1 = cluster.add_instance("clientB1", hostname="clientB001.ru") -clientB2 = cluster.add_instance("clientB2", hostname="clientB002.ru") -clientB3 = cluster.add_instance("clientB3", hostname="xxx.clientB003.rutracker.com") -clientC1 = cluster.add_instance("clientC1", hostname="clientC01.ru") -clientC2 = cluster.add_instance("clientC2", hostname="xxx.clientC02.ru") -clientC3 = cluster.add_instance("clientC3", hostname="xxx.clientC03.rutracker.com") -clientD1 = cluster.add_instance("clientD1", hostname="clientD0001.ru") -clientD2 = cluster.add_instance("clientD2", hostname="xxx.clientD0002.ru") -clientD3 = cluster.add_instance("clientD3", hostname="clientD0003.ru") - def check_clickhouse_is_ok(client_node, server_node): assert ( @@ -29,6 +20,13 @@ def check_clickhouse_is_ok(client_node, server_node): def query_from_one_node_to_another(client_node, server_node, query): check_clickhouse_is_ok(client_node, server_node) + res1 = client_node.exec_in_container(["ip", "address", "show"]) + res2 = client_node.exec_in_container(["host", "clientA1.com"]) + res3 = client_node.exec_in_container(["host", "clientA2.com"]) + res4 = client_node.exec_in_container(["host", "clientA3.com"]) + + logging.debug(f"IP: {res1}, A1 {res2}, A2 {res3}, A3 {res4}") + return client_node.exec_in_container( [ "bash", @@ -55,6 +53,13 @@ def setup_nodes(): ) query(server, "INSERT INTO test_allowed_client_hosts VALUES (5)") + s = query(server, "SELECT fqdn(), hostName()") + a1 = query(clientA1, "SELECT fqdn(), hostName()") + a2 = query(clientA2, "SELECT fqdn(), hostName()") + a3 = query(clientA3, "SELECT fqdn(), hostName()") + + logging.debug(f"s:{s}, a1:{a1}, a2:{a2}, a3:{a3}") + yield cluster finally: @@ -63,7 +68,6 @@ def setup_nodes(): def test_allowed_host(): expected_to_pass = [clientA1, clientA3] - expected_to_fail = [clientA2] # Reverse DNS lookup currently isn't working as expected in this test. # For example, it gives something like "vitbartestallowedclienthosts_clientB1_1.vitbartestallowedclienthosts_default" instead of "clientB001.ru". @@ -79,6 +83,9 @@ def test_allowed_host(): == "5\n" ) +def test_denied_host(): + expected_to_fail = [clientA2] + for client_node in expected_to_fail: with pytest.raises(Exception, match=r"default: Authentication failed"): query_from_one_node_to_another( diff --git a/tests/integration/test_allowed_url_from_config/test.py b/tests/integration/test_allowed_url_from_config/test.py index da9d4404c82..3106cf12702 100644 --- a/tests/integration/test_allowed_url_from_config/test.py +++ b/tests/integration/test_allowed_url_from_config/test.py @@ -49,6 +49,8 @@ def test_config_with_hosts(start_cluster): assert "not allowed" in node1.query_and_get_error( "CREATE TABLE table_test_1_4 (word String) Engine=URL('https://yandex2.ru', CSV)" ) + node1.query("DROP TABLE table_test_1_1") + node1.query("DROP TABLE table_test_1_2") def test_config_with_only_primary_hosts(start_cluster): @@ -86,6 +88,11 @@ def test_config_with_only_primary_hosts(start_cluster): "CREATE TABLE table_test_2_6 (word String) Engine=URL('https://yandex2.ru', CSV)" ) + node2.query("DROP TABLE table_test_2_1") + node2.query("DROP TABLE table_test_2_2") + node2.query("DROP TABLE table_test_2_3") + node2.query("DROP TABLE table_test_2_4") + def test_config_with_only_regexp_hosts(start_cluster): assert ( @@ -106,6 +113,8 @@ def test_config_with_only_regexp_hosts(start_cluster): assert "not allowed" in node3.query_and_get_error( "CREATE TABLE table_test_3_4 (word String) Engine=URL('https://yandex2.ru', CSV)" ) + node3.query("DROP TABLE table_test_3_1") + node3.query("DROP TABLE table_test_3_2") def test_config_without_allowed_hosts_section(start_cluster): @@ -139,6 +148,11 @@ def test_config_without_allowed_hosts_section(start_cluster): ) == "" ) + node4.query("DROP TABLE table_test_4_1") + node4.query("DROP TABLE table_test_4_2") + node4.query("DROP TABLE table_test_4_3") + node4.query("DROP TABLE table_test_4_4") + node4.query("DROP TABLE table_test_4_5") def test_config_without_allowed_hosts(start_cluster): @@ -267,6 +281,7 @@ def test_redirect(start_cluster): assert "not allowed" in node7.query_and_get_error( "SET max_http_get_redirects=1; SELECT * from table_test_7_1" ) + node7.query("DROP TABLE table_test_7_1") def test_HDFS(start_cluster): diff --git a/tests/integration/test_dictionaries_redis/test.py b/tests/integration/test_dictionaries_redis/test.py index bc8170ab08d..e7b5fcb7b80 100644 --- a/tests/integration/test_dictionaries_redis/test.py +++ b/tests/integration/test_dictionaries_redis/test.py @@ -1,14 +1,14 @@ import os import shutil import pytest +import logging from helpers.cluster import ClickHouseCluster from helpers.dictionary import Field, Row, Dictionary, DictionaryStructure, Layout from helpers.external_sources import SourceRedis -cluster = None +cluster = ClickHouseCluster(__file__) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) dict_configs_path = os.path.join(SCRIPT_DIR, "configs/dictionaries") -node = None KEY_FIELDS = { "simple": [Field("KeyField", "UInt64", is_key=True, default_value_for_get=9999999)], @@ -70,8 +70,6 @@ DICTIONARIES = [] def get_dict(source, layout, fields, suffix_name=""): - global dict_configs_path - structure = DictionaryStructure(layout, fields) dict_name = source.name + "_" + layout.name + "_" + suffix_name dict_path = os.path.join(dict_configs_path, dict_name + ".xml") @@ -82,13 +80,9 @@ def get_dict(source, layout, fields, suffix_name=""): return dictionary -def setup_module(module): +def generate_dict_configs(): global DICTIONARIES global cluster - global node - global dict_configs_path - - cluster = ClickHouseCluster(__file__) if os.path.exists(dict_configs_path): shutil.rmtree(dict_configs_path) @@ -126,9 +120,7 @@ def setup_module(module): for source in sources: for layout in LAYOUTS: if not source.compatible_with_layout(layout): - print( - "Source", source.name, "incompatible with layout", layout.name - ) + logging.debug(f"Source {source.name} incompatible with layout {layout.name}") continue fields = KEY_FIELDS[layout.layout_type] + [field] @@ -137,7 +129,9 @@ def setup_module(module): main_configs = [] dictionaries = [] for fname in os.listdir(dict_configs_path): - dictionaries.append(os.path.join(dict_configs_path, fname)) + path = os.path.join(dict_configs_path, fname) + logging.debug(f"Found dictionary {path}") + dictionaries.append(path) node = cluster.add_instance( "node", main_configs=main_configs, dictionaries=dictionaries, with_redis=True @@ -147,13 +141,15 @@ def setup_module(module): @pytest.fixture(scope="module", autouse=True) def started_cluster(): try: + generate_dict_configs() + cluster.start() assert len(FIELDS) == len(VALUES) for dicts in DICTIONARIES: for dictionary in dicts: - print("Preparing", dictionary.name) + logging.debug(f"Preparing {dictionary.name}") dictionary.prepare_source(cluster) - print("Prepared") + logging.debug(f"Prepared {dictionary.name}") yield cluster @@ -161,14 +157,19 @@ def started_cluster(): cluster.shutdown() -@pytest.mark.parametrize("id", list(range(len(FIELDS)))) +def get_entity_id(entity): + return FIELDS[entity].name + + +@pytest.mark.parametrize("id", list(range(len(FIELDS))), ids=get_entity_id) def test_redis_dictionaries(started_cluster, id): - print("id:", id) + logging.debug(f"Run test with id: {id}") dicts = DICTIONARIES[id] values = VALUES[id] field = FIELDS[id] + node = started_cluster.instances["node"] node.query("system reload dictionaries") for dct in dicts: @@ -193,10 +194,9 @@ def test_redis_dictionaries(started_cluster, id): for query in dct.get_select_get_or_default_queries(field, row): queries_with_answers.append((query, field.default_value_for_get)) - node.query("system reload dictionary {}".format(dct.name)) + node.query(f"system reload dictionary {dct.name}") for query, answer in queries_with_answers: - print(query) assert node.query(query) == str(answer) + "\n" # Checks, that dictionaries can be reloaded. diff --git a/tests/integration/test_merge_tree_azure_blob_storage/test.py b/tests/integration/test_merge_tree_azure_blob_storage/test.py index bc549210b39..668432d4a69 100644 --- a/tests/integration/test_merge_tree_azure_blob_storage/test.py +++ b/tests/integration/test_merge_tree_azure_blob_storage/test.py @@ -3,16 +3,11 @@ import time import os import pytest -from helpers.cluster import ClickHouseCluster, get_instances_dir +from helpers.cluster import ClickHouseCluster from helpers.utility import generate_values, replace_config, SafeThread SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -CONFIG_PATH = os.path.join( - SCRIPT_DIR, - "./{}/node/configs/config.d/storage_conf.xml".format(get_instances_dir()), -) - NODE_NAME = "node" TABLE_NAME = "blob_storage_table" AZURE_BLOB_STORAGE_DISK = "blob_storage_disk" @@ -51,7 +46,7 @@ def azure_query(node, query, try_num=3): return node.query(query) except Exception as ex: retriable_errors = [ - "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response", + "DB::Exception: Azure::Core::Http::TransportException: Connection was closed by the server while trying to read a response" ] retry = False for error in retriable_errors: @@ -160,13 +155,7 @@ def test_inserts_selects(cluster): ) -@pytest.mark.parametrize( - "merge_vertical", - [ - (True), - (False), - ], -) +@pytest.mark.parametrize("merge_vertical", [(True), (False)]) def test_insert_same_partition_and_merge(cluster, merge_vertical): settings = {} if merge_vertical: @@ -498,6 +487,12 @@ def test_freeze_unfreeze(cluster): def test_apply_new_settings(cluster): node = cluster.instances[NODE_NAME] create_table(node, TABLE_NAME) + config_path = os.path.join( + SCRIPT_DIR, + "./{}/node/configs/config.d/storage_conf.xml".format( + cluster.instances_dir_name + ), + ) azure_query( node, f"INSERT INTO {TABLE_NAME} VALUES {generate_values('2020-01-03', 4096)}" @@ -505,7 +500,7 @@ def test_apply_new_settings(cluster): # Force multi-part upload mode. replace_config( - CONFIG_PATH, + config_path, "33554432", "4096", ) @@ -522,10 +517,16 @@ def test_apply_new_settings(cluster): def test_restart_during_load(cluster): node = cluster.instances[NODE_NAME] create_table(node, TABLE_NAME) + config_path = os.path.join( + SCRIPT_DIR, + "./{}/node/configs/config.d/storage_conf.xml".format( + cluster.instances_dir_name + ), + ) # Force multi-part upload mode. replace_config( - CONFIG_PATH, "false", "" + config_path, "false", "" ) azure_query( diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index f5a9bf153b7..4c0ea40f637 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -3,15 +3,11 @@ import time import os import pytest -from helpers.cluster import ClickHouseCluster, get_instances_dir +from helpers.cluster import ClickHouseCluster from helpers.utility import generate_values, replace_config, SafeThread SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -CONFIG_PATH = os.path.join( - SCRIPT_DIR, - "./{}/node/configs/config.d/storage_conf.xml".format(get_instances_dir()), -) @pytest.fixture(scope="module") @@ -562,6 +558,13 @@ def test_s3_disk_apply_new_settings(cluster, node_name): node = cluster.instances[node_name] create_table(node, "s3_test") + config_path = os.path.join( + SCRIPT_DIR, + "./{}/node/configs/config.d/storage_conf.xml".format( + cluster.instances_dir_name + ), + ) + def get_s3_requests(): node.query("SYSTEM FLUSH LOGS") return int( @@ -578,7 +581,7 @@ def test_s3_disk_apply_new_settings(cluster, node_name): # Force multi-part upload mode. replace_config( - CONFIG_PATH, + config_path, "33554432", "0", ) diff --git a/tests/integration/test_merge_tree_s3_restore/test.py b/tests/integration/test_merge_tree_s3_restore/test.py index f4acc4ac91e..0652c31951d 100644 --- a/tests/integration/test_merge_tree_s3_restore/test.py +++ b/tests/integration/test_merge_tree_s3_restore/test.py @@ -5,28 +5,22 @@ import string import time import pytest -from helpers.cluster import ClickHouseCluster, get_instances_dir +from helpers.cluster import ClickHouseCluster SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -NOT_RESTORABLE_CONFIG_PATH = os.path.join( - SCRIPT_DIR, - "./{}/node_not_restorable/configs/config.d/storage_conf_not_restorable.xml".format( - get_instances_dir() - ), -) COMMON_CONFIGS = [ "configs/config.d/bg_processing_pool_conf.xml", "configs/config.d/clusters.xml", ] -def replace_config(old, new): - config = open(NOT_RESTORABLE_CONFIG_PATH, "r") +def replace_config(path, old, new): + config = open(path, "r") config_lines = config.readlines() config.close() config_lines = [line.replace(old, new) for line in config_lines] - config = open(NOT_RESTORABLE_CONFIG_PATH, "w") + config = open(path, "w") config.writelines(config_lines) config.close() @@ -507,6 +501,12 @@ def test_restore_mutations(cluster, db_atomic): def test_migrate_to_restorable_schema(cluster): db_atomic = True node = cluster.instances["node_not_restorable"] + config_path = os.path.join( + SCRIPT_DIR, + "./{}/node_not_restorable/configs/config.d/storage_conf_not_restorable.xml".format( + cluster.instances_dir_name + ), + ) create_table(node, "test", db_atomic=db_atomic) uuid = get_table_uuid(node, db_atomic, "test") @@ -525,7 +525,9 @@ def test_migrate_to_restorable_schema(cluster): ) replace_config( - "false", "true" + config_path, + "false", + "true", ) node.restart_clickhouse() diff --git a/tests/integration/test_reload_max_table_size_to_drop/test.py b/tests/integration/test_reload_max_table_size_to_drop/test.py index da7dba12fa0..90e60e5cfa4 100644 --- a/tests/integration/test_reload_max_table_size_to_drop/test.py +++ b/tests/integration/test_reload_max_table_size_to_drop/test.py @@ -2,18 +2,13 @@ import os import time import pytest -from helpers.cluster import ClickHouseCluster, get_instances_dir +from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node = cluster.add_instance("node", main_configs=["configs/max_table_size_to_drop.xml"]) SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -CONFIG_PATH = os.path.join( - SCRIPT_DIR, - "./{}/node/configs/config.d/max_table_size_to_drop.xml".format(get_instances_dir()), -) - @pytest.fixture(scope="module") def start_cluster(): @@ -29,6 +24,12 @@ def start_cluster(): def test_reload_max_table_size_to_drop(start_cluster): node.query("INSERT INTO test VALUES (now(), 0)") + config_path = os.path.join( + SCRIPT_DIR, + "./{}/node/configs/config.d/max_table_size_to_drop.xml".format( + start_cluster.instances_dir_name + ), + ) time.sleep(5) # wait for data part commit @@ -37,14 +38,14 @@ def test_reload_max_table_size_to_drop(start_cluster): assert out == "" assert err != "" - config = open(CONFIG_PATH, "r") + config = open(config_path, "r") config_lines = config.readlines() config.close() config_lines = [ line.replace("1", "1000000") for line in config_lines ] - config = open(CONFIG_PATH, "w") + config = open(config_path, "w") config.writelines(config_lines) config.close() diff --git a/tests/integration/test_replicated_merge_tree_s3_restore/test.py b/tests/integration/test_replicated_merge_tree_s3_restore/test.py index d743dedbdde..fc13c8a1184 100644 --- a/tests/integration/test_replicated_merge_tree_s3_restore/test.py +++ b/tests/integration/test_replicated_merge_tree_s3_restore/test.py @@ -5,7 +5,7 @@ import string import time import pytest -from helpers.cluster import ClickHouseCluster, get_instances_dir +from helpers.cluster import ClickHouseCluster COMMON_CONFIGS = ["configs/config.d/clusters.xml"] diff --git a/tests/integration/test_storage_s3/test.py b/tests/integration/test_storage_s3/test.py index 5dd09ddd362..7d22cb24a5c 100644 --- a/tests/integration/test_storage_s3/test.py +++ b/tests/integration/test_storage_s3/test.py @@ -9,7 +9,7 @@ import time import helpers.client import pytest -from helpers.cluster import ClickHouseCluster, ClickHouseInstance, get_instances_dir +from helpers.cluster import ClickHouseCluster, ClickHouseInstance from helpers.network import PartitionManager from helpers.test_tools import exec_query_with_retry @@ -17,11 +17,6 @@ MINIO_INTERNAL_PORT = 9001 SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) -CONFIG_PATH = os.path.join( - SCRIPT_DIR, "./{}/dummy/configs/config.d/defaultS3.xml".format(get_instances_dir()) -) - - # Creates S3 bucket for tests and allows anonymous read-write access to it. def prepare_s3_bucket(started_cluster): # Allows read-write access for bucket without authorization. @@ -724,17 +719,24 @@ def run_s3_mocks(started_cluster): logging.info("S3 mocks started") -def replace_config(old, new): - config = open(CONFIG_PATH, "r") +def replace_config(path, old, new): + config = open(path, "r") config_lines = config.readlines() config.close() config_lines = [line.replace(old, new) for line in config_lines] - config = open(CONFIG_PATH, "w") + config = open(path, "w") config.writelines(config_lines) config.close() def test_custom_auth_headers(started_cluster): + config_path = os.path.join( + SCRIPT_DIR, + "./{}/dummy/configs/config.d/defaultS3.xml".format( + started_cluster.instances_dir_name + ), + ) + table_format = "column1 UInt32, column2 UInt32, column3 UInt32" filename = "test.csv" get_query = "select * from s3('http://resolver:8080/{bucket}/{file}', 'CSV', '{table_format}')".format( @@ -758,6 +760,7 @@ def test_custom_auth_headers(started_cluster): assert run_query(instance, "SELECT * FROM test") == "1\t2\t3\n" replace_config( + config_path, "
Authorization: Bearer TOKEN", "
Authorization: Bearer INVALID_TOKEN", ) @@ -765,6 +768,7 @@ def test_custom_auth_headers(started_cluster): ret, err = instance.query_and_get_answer_with_error("SELECT * FROM test") assert ret == "" and err != "" replace_config( + config_path, "
Authorization: Bearer INVALID_TOKEN", "
Authorization: Bearer TOKEN", ) @@ -805,10 +809,7 @@ def test_infinite_redirect(started_cluster): @pytest.mark.parametrize( "extension,method", - [ - pytest.param("bin", "gzip", id="bin"), - pytest.param("gz", "auto", id="gz"), - ], + [pytest.param("bin", "gzip", id="bin"), pytest.param("gz", "auto", id="gz")], ) def test_storage_s3_get_gzip(started_cluster, extension, method): bucket = started_cluster.minio_bucket From e898f65cc523b8b06b1e903ea113818419ca8b68 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Thu, 7 Jul 2022 20:42:41 +0000 Subject: [PATCH 015/117] black --- tests/integration/helpers/cluster.py | 28 ++++++++++--------- .../test_allowed_client_hosts/test.py | 2 ++ .../test_dictionaries_redis/test.py | 4 ++- 3 files changed, 20 insertions(+), 14 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index f5c546ff264..32a4e663975 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -539,30 +539,34 @@ class ClickHouseCluster: logging.debug(f"Removed :{self.instances_dir}") os.mkdir(self.instances_dir) - def print_all_docker_pieces(self): res_networks = subprocess.check_output( f"docker network ls --filter name='{self.project_name}*'", shell=True, universal_newlines=True, ) - logging.debug(f"Docker networks for project {self.project_name} are {res_networks}") + logging.debug( + f"Docker networks for project {self.project_name} are {res_networks}" + ) res_containers = subprocess.check_output( f"docker container ls -a --filter name='{self.project_name}*'", shell=True, universal_newlines=True, ) - logging.debug(f"Docker containers for project {self.project_name} are {res_containers}") + logging.debug( + f"Docker containers for project {self.project_name} are {res_containers}" + ) res_volumes = subprocess.check_output( f"docker volume ls --filter name='{self.project_name}*'", shell=True, universal_newlines=True, ) - logging.debug(f"Docker volumes for project {self.project_name} are {res_volumes}") - + logging.debug( + f"Docker volumes for project {self.project_name} are {res_volumes}" + ) def cleanup(self): - logging.debug('Cleanup called') + logging.debug("Cleanup called") self.print_all_docker_pieces() if ( @@ -608,7 +612,7 @@ class ClickHouseCluster: ).splitlines() if list_networks: logging.debug(f"Trying to remove networks: {list_networks}") - subprocess.check_call(f"docker network rm {' '.join(list_networks)}", shell=True) + run_and_check(f"docker network rm {' '.join(list_networks)}") logging.debug(f"Networks removed: {list_networks}") except: pass @@ -617,7 +621,7 @@ class ClickHouseCluster: try: logging.debug("Trying to prune unused images...") - run_and_check(['docker', 'image', 'prune', '-f']) + run_and_check(["docker", "image", "prune", "-f"]) logging.debug("Images pruned") except: pass @@ -663,9 +667,7 @@ class ClickHouseCluster: f"docker container list --all --filter name='{filter_name}' --format '{format}'", shell=True, ) - containers = dict( - line.split(":", 1) for line in containers.splitlines() - ) + containers = dict(line.split(":", 1) for line in containers.splitlines()) return containers def copy_file_from_container_to_container( @@ -3551,14 +3553,14 @@ class ClickHouseInstance: "bash", "-c", "echo 'ATTACH DATABASE system ENGINE=Ordinary' > /var/lib/clickhouse/metadata/system.sql", - ], + ] ) self.exec_in_container( [ "bash", "-c", "echo 'ATTACH DATABASE system ENGINE=Ordinary' > /var/lib/clickhouse/metadata/default.sql", - ], + ] ) self.exec_in_container( ["bash", "-c", "{} --daemon".format(self.clickhouse_start_command)], diff --git a/tests/integration/test_allowed_client_hosts/test.py b/tests/integration/test_allowed_client_hosts/test.py index dda3439be14..1fc9d0432cb 100644 --- a/tests/integration/test_allowed_client_hosts/test.py +++ b/tests/integration/test_allowed_client_hosts/test.py @@ -9,6 +9,7 @@ clientA1 = cluster.add_instance("clientA1", hostname="clientA1.com") clientA2 = cluster.add_instance("clientA2", hostname="clientA2.com") clientA3 = cluster.add_instance("clientA3", hostname="clientA3.com") + def check_clickhouse_is_ok(client_node, server_node): assert ( client_node.exec_in_container( @@ -83,6 +84,7 @@ def test_allowed_host(): == "5\n" ) + def test_denied_host(): expected_to_fail = [clientA2] diff --git a/tests/integration/test_dictionaries_redis/test.py b/tests/integration/test_dictionaries_redis/test.py index e7b5fcb7b80..c2dc73db782 100644 --- a/tests/integration/test_dictionaries_redis/test.py +++ b/tests/integration/test_dictionaries_redis/test.py @@ -120,7 +120,9 @@ def generate_dict_configs(): for source in sources: for layout in LAYOUTS: if not source.compatible_with_layout(layout): - logging.debug(f"Source {source.name} incompatible with layout {layout.name}") + logging.debug( + f"Source {source.name} incompatible with layout {layout.name}" + ) continue fields = KEY_FIELDS[layout.layout_type] + [field] From 8ef2d87adfe95f6eb02ddfe445086e183904b7b9 Mon Sep 17 00:00:00 2001 From: Ilya Yatsishin <2159081+qoega@users.noreply.github.com> Date: Mon, 11 Jul 2022 15:09:42 +0200 Subject: [PATCH 016/117] Update tests/integration/helpers/cluster.py --- tests/integration/helpers/cluster.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 32a4e663975..4c6304632e7 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -2127,7 +2127,7 @@ class ClickHouseCluster: logging.debug("Can't connect to MeiliSearch " + str(ex)) time.sleep(1) - def wait_minio_to_start(self, timeout=10, secure=False): + def wait_minio_to_start(self, timeout=180, secure=False): self.minio_ip = self.get_instance_ip(self.minio_host) self.minio_redirect_ip = self.get_instance_ip(self.minio_redirect_host) From 36e34d8cc610e3afc25b5f5bc79e3775ee93be75 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Jul 2022 14:53:23 +0000 Subject: [PATCH 017/117] Respect remote_url_allow_hosts in relevant dictionary sources. --- src/Dictionaries/CassandraDictionarySource.h | 1 + src/Dictionaries/ClickHouseDictionarySource.cpp | 3 +++ src/Dictionaries/HTTPDictionarySource.cpp | 6 ++++-- src/Dictionaries/MongoDBDictionarySource.cpp | 5 ++++- src/Dictionaries/MySQLDictionarySource.cpp | 9 +++++++++ src/Dictionaries/PostgreSQLDictionarySource.cpp | 10 +++++++++- 6 files changed, 30 insertions(+), 4 deletions(-) diff --git a/src/Dictionaries/CassandraDictionarySource.h b/src/Dictionaries/CassandraDictionarySource.h index e73383aa75c..2591b33c638 100644 --- a/src/Dictionaries/CassandraDictionarySource.h +++ b/src/Dictionaries/CassandraDictionarySource.h @@ -8,6 +8,7 @@ #include "IDictionarySource.h" #include "ExternalQueryBuilder.h" #include +#include #include #include diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 1a3f3f0edc4..b616979ba2e 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -284,6 +284,9 @@ void registerDictionarySourceClickHouse(DictionarySourceFactory & factory) else { context = Context::createCopy(global_context); + + if (created_from_ddl) + context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); } context->applySettingsChanges(readSettingsFromDictionaryConfig(config, config_prefix)); diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index d45213fa7f8..95a891e18ce 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -32,8 +32,7 @@ HTTPDictionarySource::HTTPDictionarySource( const Configuration & configuration_, const Poco::Net::HTTPBasicCredentials & credentials_, Block & sample_block_, - ContextPtr context_, - bool created_from_ddl) + ContextPtr context_) : log(&Poco::Logger::get("HTTPDictionarySource")) , update_time(std::chrono::system_clock::from_time_t(0)) , dict_struct(dict_struct_) @@ -303,6 +302,9 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) auto context = copyContextAndApplySettingsFromDictionaryConfig(global_context, config, config_prefix); + if (created_from_ddl) + context->getRemoteHostFilter().checkURL(Poco::URI(configuration.url)); + return std::make_unique(dict_struct, configuration, credentials, sample_block, context, created_from_ddl); }; factory.registerSource("http", create_table_source); diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index 6a1bfceaec7..1ede0ec5045 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -20,7 +20,7 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) Block & sample_block, ContextPtr context, const std::string & /* default_database */, - bool /* created_from_ddl */) + bool created_from_ddl) { const auto config_prefix = root_config_prefix + ".mongodb"; ExternalDataSourceConfiguration configuration; @@ -39,6 +39,9 @@ void registerDictionarySourceMongoDB(DictionarySourceFactory & factory) configuration.database = config.getString(config_prefix + ".db", ""); } + if (created_from_ddl) + context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); + return std::make_unique(dict_struct, config.getString(config_prefix + ".uri", ""), configuration.host, diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index a5807f58f8a..6fb4dffc5e7 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -78,6 +78,9 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) : std::nullopt; if (named_collection) { + if (created_from_ddl) + global_context->getRemoteHostFilter().checkHostAndPort(configuration.host, toString(configuration.port)); + mysql_settings.applyChanges(named_collection->settings_changes); configuration.set(named_collection->configuration); configuration.addresses = {std::make_pair(configuration.host, configuration.port)}; @@ -90,6 +93,12 @@ void registerDictionarySourceMysql(DictionarySourceFactory & factory) } else { + if (created_from_ddl) + { + for (auto & address : configuration.addresses) + global_context->getRemoteHostFilter().checkHostAndPort(address.first, toString(address.second)); + } + configuration.database = config.getString(settings_config_prefix + ".db", ""); configuration.table = config.getString(settings_config_prefix + ".table", ""); pool = std::make_shared(mysqlxx::PoolFactory::instance().get(config, settings_config_prefix)); diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 42884278e7d..bec8283e724 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -185,13 +185,21 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) Block & sample_block, ContextPtr context, const std::string & /* default_database */, - bool /* created_from_ddl */) -> DictionarySourcePtr + bool created_from_ddl) -> DictionarySourcePtr { #if USE_LIBPQXX const auto settings_config_prefix = config_prefix + ".postgresql"; auto has_config_key = [](const String & key) { return dictionary_allowed_keys.contains(key) || key.starts_with("replica"); }; auto configuration = getExternalDataSourceConfigurationByPriority(config, settings_config_prefix, context, has_config_key); const auto & settings = context->getSettingsRef(); + + if (created_from_ddl) + { + for (const auto & replicas : configuration.replicas_configurations) + for (const auto & replica : replicas.second) + context->getRemoteHostFilter().checkHostAndPort(replica.host, toString(replica.port)); + } + auto pool = std::make_shared( configuration.replicas_configurations, settings.postgresql_connection_pool_size, From 937a9e9d9fecd06520ab89d5b59bffe6c716d72b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Jul 2022 15:05:49 +0000 Subject: [PATCH 018/117] Fixing build/ --- src/Dictionaries/HTTPDictionarySource.cpp | 3 --- 1 file changed, 3 deletions(-) diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 95a891e18ce..265c8f7cb99 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -41,9 +41,6 @@ HTTPDictionarySource::HTTPDictionarySource( , context(context_) , timeouts(ConnectionTimeouts::getHTTPTimeouts(context)) { - if (created_from_ddl) - context->getRemoteHostFilter().checkURL(Poco::URI(configuration.url)); - credentials.setUsername(credentials_.getUsername()); credentials.setPassword(credentials_.getPassword()); } From 5005700eee75ed791cf21479d470c088dfb6b72e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 13 Jul 2022 15:33:18 +0000 Subject: [PATCH 019/117] Fixing build --- src/Dictionaries/HTTPDictionarySource.cpp | 2 +- src/Dictionaries/HTTPDictionarySource.h | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 265c8f7cb99..39e034a071f 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -302,7 +302,7 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) if (created_from_ddl) context->getRemoteHostFilter().checkURL(Poco::URI(configuration.url)); - return std::make_unique(dict_struct, configuration, credentials, sample_block, context, created_from_ddl); + return std::make_unique(dict_struct, configuration, credentials, sample_block, context); }; factory.registerSource("http", create_table_source); } diff --git a/src/Dictionaries/HTTPDictionarySource.h b/src/Dictionaries/HTTPDictionarySource.h index 71351cd9987..86e3836f2dc 100644 --- a/src/Dictionaries/HTTPDictionarySource.h +++ b/src/Dictionaries/HTTPDictionarySource.h @@ -37,8 +37,7 @@ public: const Configuration & configuration, const Poco::Net::HTTPBasicCredentials & credentials_, Block & sample_block_, - ContextPtr context_, - bool created_from_ddl); + ContextPtr context_); HTTPDictionarySource(const HTTPDictionarySource & other); HTTPDictionarySource & operator=(const HTTPDictionarySource &) = delete; From 772c009f2fbe7e8991407d8f7f8dad0b6652dcaf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 14 Jul 2022 18:44:43 +0000 Subject: [PATCH 020/117] Fixing build. --- src/Dictionaries/PostgreSQLDictionarySource.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index bec8283e724..9153b9cb05d 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -185,7 +185,7 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) Block & sample_block, ContextPtr context, const std::string & /* default_database */, - bool created_from_ddl) -> DictionarySourcePtr + [[maybe_unused]] bool created_from_ddl) -> DictionarySourcePtr { #if USE_LIBPQXX const auto settings_config_prefix = config_prefix + ".postgresql"; From 9eb7553fce3270472ba15c537ec470f3dae6257e Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 15 Jul 2022 18:03:24 +0200 Subject: [PATCH 021/117] Update tests/integration/helpers/cluster.py --- tests/integration/helpers/cluster.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 4c6304632e7..99b06e6ddb9 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -278,7 +278,6 @@ class ClickHouseCluster: zookeeper_keyfile=None, zookeeper_certfile=None, ): - logging.debug(f"INIT CALLED") for param in list(os.environ.keys()): logging.debug("ENV %40s %s" % (param, os.environ[param])) self.base_path = base_path From e5b0c85f8c3827ee874c23c50d005d54402b663e Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 18 Jul 2022 09:30:43 +0000 Subject: [PATCH 022/117] improve test_rabbitmq_drop_mv: it was flaky and waited 900 second timeout --- .../integration/test_storage_rabbitmq/test.py | 21 +++++++++++++++++-- 1 file changed, 19 insertions(+), 2 deletions(-) diff --git a/tests/integration/test_storage_rabbitmq/test.py b/tests/integration/test_storage_rabbitmq/test.py index 18b1e9d974b..f5d216a8b92 100644 --- a/tests/integration/test_storage_rabbitmq/test.py +++ b/tests/integration/test_storage_rabbitmq/test.py @@ -2590,9 +2590,17 @@ def test_rabbitmq_drop_mv(rabbitmq_cluster): rabbitmq_exchange_name = 'mv', rabbitmq_format = 'JSONEachRow', rabbitmq_queue_base = 'drop_mv'; + """ + ) + instance.query( + """ CREATE TABLE test.view (key UInt64, value UInt64) ENGINE = MergeTree() ORDER BY key; + """ + ) + instance.query( + """ CREATE MATERIALIZED VIEW test.consumer TO test.view AS SELECT * FROM test.rabbitmq; """ @@ -2611,6 +2619,14 @@ def test_rabbitmq_drop_mv(rabbitmq_cluster): exchange="mv", routing_key="", body=json.dumps({"key": i, "value": i}) ) + start = time.time() + while time.time() - start < 30: + res = instance.query("SELECT COUNT(*) FROM test.view") + if "20" == res: + break + else: + logging.debug(f"Number of rows in test.view: {res}") + instance.query("DROP VIEW test.consumer") for i in range(20, 40): channel.basic_publish( @@ -2643,7 +2659,8 @@ def test_rabbitmq_drop_mv(rabbitmq_cluster): connection.close() count = 0 - while True: + start = time.time() + while time.time() - start < 30: count = int(instance.query("SELECT count() FROM test.rabbitmq")) if count: break @@ -2685,7 +2702,7 @@ def test_rabbitmq_random_detach(rabbitmq_cluster): channel = connection.channel() messages = [] - for i in range(messages_num): + for j in range(messages_num): messages.append(json.dumps({"key": i[0], "value": i[0]})) i[0] += 1 mes_id = str(i) From d243b5c7854b05a80b310c7d113b3512456fd1e0 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 18 Jul 2022 09:32:19 +0000 Subject: [PATCH 023/117] remove unnecessary try-except --- tests/integration/helpers/cluster.py | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index e43fb0dd79d..c739583c587 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -586,10 +586,7 @@ class ClickHouseCluster: f"Trying to kill unstopped containers: {unstopped_containers}" ) for id in unstopped_containers: - try: - run_and_check(f"docker kill {id}", shell=True, nothrow=True) - except: - pass + run_and_check(f"docker kill {id}", shell=True, nothrow=True) run_and_check(f"docker rm {id}", shell=True, nothrow=True) unstopped_containers = self.get_running_containers() if unstopped_containers: From 8eb628f7fac80c4731388a1d64e69d76d98e1eb4 Mon Sep 17 00:00:00 2001 From: kssenii Date: Mon, 18 Jul 2022 15:25:21 +0200 Subject: [PATCH 024/117] Update azure --- contrib/azure | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure b/contrib/azure index ac4b763d4ca..9030dea3088 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit ac4b763d4ca40122275f1497cbdc5451337461d9 +Subproject commit 9030dea30881e561b06dc80e0453b9b2a593cdbe From dd1349e2d4496a2771050d2c38f49f40fc5eb07d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 19 Jul 2022 16:20:59 +0000 Subject: [PATCH 025/117] Fix test. --- .../test_dictionaries_ddl/configs/allow_remote_node.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_dictionaries_ddl/configs/allow_remote_node.xml b/tests/integration/test_dictionaries_ddl/configs/allow_remote_node.xml index 031e2257ba0..259ce83c03f 100644 --- a/tests/integration/test_dictionaries_ddl/configs/allow_remote_node.xml +++ b/tests/integration/test_dictionaries_ddl/configs/allow_remote_node.xml @@ -1,5 +1,6 @@ node1 + node4 From a00bcd389fb923491c5df37b18f4220ee3e2cd12 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 19 Jul 2022 19:33:14 +0200 Subject: [PATCH 026/117] Fixing build --- src/Storages/MergeTree/KeyCondition.cpp | 198 ++++++------------------ 1 file changed, 47 insertions(+), 151 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index e62968453e4..e590d98fce2 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -241,6 +241,11 @@ public: /// Simple literal out_value = lit->value; out_type = block_with_constants.getByName(column_name).type; + + /// If constant is not Null, we can assume it's type is not Nullable as well. + if (!out_value.isNull()) + out_type = removeNullable(out_type); + return true; } else if (block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column)) @@ -249,6 +254,10 @@ public: const auto & expr_info = block_with_constants.getByName(column_name); out_value = (*expr_info.column)[0]; out_type = expr_info.type; + + if (!out_value.isNull()) + out_type = removeNullable(out_type); + return true; } } @@ -258,6 +267,10 @@ public: { out_value = (*dag->column)[0]; out_type = dag->result_type; + + if (!out_value.isNull()) + out_type = removeNullable(out_type); + return true; } } @@ -1067,156 +1080,34 @@ void KeyCondition::traverseAST(const Tree & node, ContextPtr context, Block & bl rpn.emplace_back(std::move(element)); } -bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( - const Tree & node, +/** The key functional expression constraint may be inferred from a plain column in the expression. + * For example, if the key contains `toStartOfHour(Timestamp)` and query contains `WHERE Timestamp >= now()`, + * it can be assumed that if `toStartOfHour()` is monotonic on [now(), inf), the `toStartOfHour(Timestamp) >= toStartOfHour(now())` + * condition also holds, so the index may be used to select only parts satisfying this condition. + * + * To check the assumption, we'd need to assert that the inverse function to this transformation is also monotonic, however the + * inversion isn't exported (or even viable for not strictly monotonic functions such as `toStartOfHour()`). + * Instead, we can qualify only functions that do not transform the range (for example rounding), + * which while not strictly monotonic, are monotonic everywhere on the input range. + */ +bool KeyCondition::transformConstantWithValidFunctions( + const String & expr_name, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, - DataTypePtr & out_type) + DataTypePtr & out_type, + std::function always_monotonic) const { - String expr_name = node.getColumnName(); - - if (array_joined_columns.count(expr_name)) - return false; - - if (key_subexpr_names.count(expr_name) == 0) - return false; - - /// TODO Nullable index is not yet landed. - if (out_value.isNull()) - return false; - const auto & sample_block = key_expr->getSampleBlock(); - /** The key functional expression constraint may be inferred from a plain column in the expression. - * For example, if the key contains `toStartOfHour(Timestamp)` and query contains `WHERE Timestamp >= now()`, - * it can be assumed that if `toStartOfHour()` is monotonic on [now(), inf), the `toStartOfHour(Timestamp) >= toStartOfHour(now())` - * condition also holds, so the index may be used to select only parts satisfying this condition. - * - * To check the assumption, we'd need to assert that the inverse function to this transformation is also monotonic, however the - * inversion isn't exported (or even viable for not strictly monotonic functions such as `toStartOfHour()`). - * Instead, we can qualify only functions that do not transform the range (for example rounding), - * which while not strictly monotonic, are monotonic everywhere on the input range. - */ - for (const auto & dag_node : key_expr->getNodes()) + for (const auto & node : key_expr->getNodes()) { - auto it = key_columns.find(dag_node.result_name); + auto it = key_columns.find(node.result_name); if (it != key_columns.end()) { std::stack chain; - const auto * cur_node = &dag_node; - bool is_valid_chain = true; - - while (is_valid_chain) - { - if (cur_node->result_name == expr_name) - break; - - chain.push(cur_node); - - if (cur_node->type == ActionsDAG::ActionType::FUNCTION && cur_node->children.size() == 1) - { - const auto * next_node = cur_node->children.front(); - - if (!cur_node->function_base->hasInformationAboutMonotonicity()) - is_valid_chain = false; - else - { - /// Range is irrelevant in this case. - auto monotonicity = cur_node->function_base->getMonotonicityForRange( - *next_node->result_type, Field(), Field()); - if (!monotonicity.is_always_monotonic) - is_valid_chain = false; - } - - cur_node = next_node; - } - else if (cur_node->type == ActionsDAG::ActionType::ALIAS) - cur_node = cur_node->children.front(); - else - is_valid_chain = false; - } - - if (is_valid_chain && !chain.empty()) - { - /// Here we cast constant to the input type. - /// It is not clear, why this works in general. - /// I can imagine the case when expression like `column < const` is legal, - /// but `type(column)` and `type(const)` are of different types, - /// and const cannot be casted to column type. - /// (There could be `superType(type(column), type(const))` which is used for comparison). - /// - /// However, looks like this case newer happenes (I could not find such). - /// Let's assume that any two comparable types are castable to each other. - auto const_type = cur_node->result_type; - auto const_column = out_type->createColumnConst(1, out_value); - auto const_value = (*castColumn({const_column, out_type, ""}, const_type))[0]; - - while (!chain.empty()) - { - const auto * func = chain.top(); - chain.pop(); - - if (func->type != ActionsDAG::ActionType::FUNCTION) - continue; - - std::tie(const_value, const_type) = - applyFunctionForFieldOfUnknownType(func->function_base, const_type, const_value); - } - - out_key_column_num = it->second; - out_key_column_type = sample_block.getByName(it->first).type; - out_value = const_value; - out_type = const_type; - return true; - } - } - } - - return false; -} - -/// Looking for possible transformation of `column = constant` into `partition_expr = function(constant)` -bool KeyCondition::canConstantBeWrappedByFunctions( - const Tree & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type) -{ - String expr_name = node.getColumnName(); - - if (array_joined_columns.count(expr_name)) - return false; - - if (key_subexpr_names.count(expr_name) == 0) - { - /// Let's check another one case. - /// If our storage was created with moduloLegacy in partition key, - /// We can assume that `modulo(...) = const` is the same as `moduloLegacy(...) = const`. - /// Replace modulo to moduloLegacy in AST and check if we also have such a column. - /// - /// We do not check this in canConstantBeWrappedByMonotonicFunctions. - /// The case `f(modulo(...))` for totally monotonic `f ` is consedered to be rare. - /// - /// Note: for negative values, we can filter more partitions then needed. - expr_name = node.getColumnNameLegacy(); - - if (key_subexpr_names.count(expr_name) == 0) - return false; - } - - const auto & sample_block = key_expr->getSampleBlock(); - - /// TODO Nullable index is not yet landed. - if (out_value.isNull()) - return false; - - for (const auto & dag_node : key_expr->getNodes()) - { - auto it = key_columns.find(dag_node.result_name); - if (it != key_columns.end()) - { - std::stack chain; - - const auto * cur_node = &dag_node; + const auto * cur_node = &node; bool is_valid_chain = true; while (is_valid_chain) @@ -1255,12 +1146,18 @@ bool KeyCondition::canConstantBeWrappedByFunctions( if (is_valid_chain) { + /// Here we cast constant to the input type. + /// It is not clear, why this works in general. + /// I can imagine the case when expression like `column < const` is legal, + /// but `type(column)` and `type(const)` are of different types, + /// and const cannot be casted to column type. + /// (There could be `superType(type(column), type(const))` which is used for comparison). + /// + /// However, looks like this case newer happenes (I could not find such). + /// Let's assume that any two comparable types are castable to each other. auto const_type = cur_node->result_type; auto const_column = out_type->createColumnConst(1, out_value); - auto const_value = (*castColumnAccurateOrNull({const_column, out_type, ""}, const_type))[0]; - - if (const_value.isNull()) - return false; + auto const_value = (*castColumn({const_column, out_type, ""}, const_type))[0]; while (!chain.empty()) { @@ -1304,17 +1201,18 @@ bool KeyCondition::canConstantBeWrappedByFunctions( } } } + return false; } bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( - const ASTPtr & node, + const Tree & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type) { - String expr_name = node->getColumnNameWithoutAlias(); + String expr_name = node.getColumnName(); if (array_joined_columns.contains(expr_name)) return false; @@ -1343,9 +1241,9 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( /// Looking for possible transformation of `column = constant` into `partition_expr = function(constant)` bool KeyCondition::canConstantBeWrappedByFunctions( - const ASTPtr & ast, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type) + const Tree & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type) { - String expr_name = ast->getColumnNameWithoutAlias(); + String expr_name = node.getColumnName(); if (array_joined_columns.contains(expr_name)) return false; @@ -1358,12 +1256,10 @@ bool KeyCondition::canConstantBeWrappedByFunctions( /// Replace modulo to moduloLegacy in AST and check if we also have such a column. /// /// We do not check this in canConstantBeWrappedByMonotonicFunctions. - /// The case `f(modulo(...))` for totally monotonic `f ` is consedered to be rare. + /// The case `f(modulo(...))` for totally monotonic `f ` is considered to be rare. /// /// Note: for negative values, we can filter more partitions then needed. - auto adjusted_ast = ast->clone(); - KeyDescription::moduloToModuloLegacyRecursive(adjusted_ast); - expr_name = adjusted_ast->getColumnName(); + expr_name = node.getColumnNameLegacy(); if (!key_subexpr_names.contains(expr_name)) return false; From f570cde815d1fb55f83b3c1d5819b57b8baec3ee Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 19 Jul 2022 20:19:57 +0000 Subject: [PATCH 027/117] Fixing build. --- src/Functions/indexHint.h | 2 ++ src/Interpreters/ActionsVisitor.cpp | 3 ++- .../Optimizations/optimizePrimaryKeyCondition.cpp | 9 +-------- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 6 ++++++ src/Processors/QueryPlan/ReadFromMergeTree.h | 3 +++ src/Storages/Hive/StorageHive.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeData.cpp | 2 +- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 3 +++ 8 files changed, 20 insertions(+), 12 deletions(-) diff --git a/src/Functions/indexHint.h b/src/Functions/indexHint.h index 0dc44e613ef..3b71c7a5585 100644 --- a/src/Functions/indexHint.h +++ b/src/Functions/indexHint.h @@ -44,6 +44,8 @@ public: bool isSuitableForConstantFolding() const override { return false; } + bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return false; } + String getName() const override { return name; diff --git a/src/Interpreters/ActionsVisitor.cpp b/src/Interpreters/ActionsVisitor.cpp index b78c4a1e48f..e31fea977ae 100644 --- a/src/Interpreters/ActionsVisitor.cpp +++ b/src/Interpreters/ActionsVisitor.cpp @@ -951,7 +951,8 @@ void ActionsMatcher::visit(const ASTFunction & node, const ASTPtr & ast, Data & data.no_subqueries, data.no_makeset, data.only_consts, - /*create_source_for_in*/ false); + /*create_source_for_in*/ false, + data.aggregation_keys_info); NamesWithAliases args; diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index 5724e50f893..f9abe662006 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include @@ -35,13 +34,7 @@ void optimizePrimaryKeyCondition(QueryPlan::Node & root) if (auto * filter_step = typeid_cast(frame.node->step.get())) { auto * child = frame.node->children.at(0); - if (typeid_cast(child->step.get())) - { - auto * child_child = child->children.at(0); - if (auto * read_from_merge_tree = typeid_cast(child_child->step.get())) - read_from_merge_tree->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); - } - else if (auto * read_from_merge_tree = typeid_cast(child->step.get())) + if (auto * read_from_merge_tree = typeid_cast(child->step.get())) read_from_merge_tree->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 36acc235dae..3306178ec11 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -831,6 +831,9 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge { return selectRangesToRead( std::move(parts), + prewhere_info, + added_filter, + added_filter_column_name, storage_snapshot->metadata, storage_snapshot->getMetadataForQuery(), query_info, @@ -845,6 +848,9 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead(Merge MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( MergeTreeData::DataPartsVector parts, + const PrewhereInfoPtr & prewhere_info, + const ActionsDAGPtr & added_filter, + const std::string & added_filter_column_name, const StorageMetadataPtr & metadata_snapshot_base, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 5231f2c0e05..9257bbdd119 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -124,6 +124,9 @@ public: static MergeTreeDataSelectAnalysisResultPtr selectRangesToRead( MergeTreeData::DataPartsVector parts, + const PrewhereInfoPtr & prewhere_info, + const ActionsDAGPtr & added_filter, + const std::string & added_filter_column_name, const StorageMetadataPtr & metadata_snapshot_base, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, diff --git a/src/Storages/Hive/StorageHive.cpp b/src/Storages/Hive/StorageHive.cpp index ddd9f526091..2ddca5e9ba6 100644 --- a/src/Storages/Hive/StorageHive.cpp +++ b/src/Storages/Hive/StorageHive.cpp @@ -623,7 +623,7 @@ HiveFiles StorageHive::collectHiveFilesFromPartition( for (size_t i = 0; i < partition_names.size(); ++i) ranges.emplace_back(fields[i]); - const KeyCondition partition_key_condition(query_info, getContext(), partition_names, partition_minmax_idx_expr); + const KeyCondition partition_key_condition(query_info.query, query_info.syntax_analyzer_result, query_info.sets, getContext(), partition_names, partition_minmax_idx_expr); if (!partition_key_condition.checkInHyperrectangle(ranges, partition_types).can_be_true) return {}; } @@ -691,7 +691,7 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded( if (prune_level >= PruneLevel::File) { - const KeyCondition hivefile_key_condition(query_info, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); + const KeyCondition hivefile_key_condition(query_info.query, query_info.syntax_analyzer_result, query_info.sets, getContext(), hivefile_name_types.getNames(), hivefile_minmax_idx_expr); if (hive_file->useFileMinMaxIndex()) { /// Load file level minmax index and apply diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 3b39100b3de..c2f29ed4c3c 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5240,7 +5240,7 @@ Block MergeTreeData::getMinMaxCountProjectionBlock( minmax_columns_types = getMinMaxColumnsTypes(partition_key); minmax_idx_condition.emplace( - query_info, + query_info.query, query_info.syntax_analyzer_result, query_info.sets, query_context, minmax_columns_names, getMinMaxExpr(partition_key, ExpressionActionsSettings::fromContext(query_context))); diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 1a55844ad94..9381d6f5264 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1267,6 +1267,9 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar return ReadFromMergeTree::selectRangesToRead( std::move(parts), + query_info.prewhere_info, + nullptr, + "", metadata_snapshot_base, metadata_snapshot, query_info, From e0d2c8fb37e2c89990c332739fa1266d3ff9ee84 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 21 Jul 2022 14:47:19 +0000 Subject: [PATCH 028/117] fix json type with sparse columns --- src/Columns/ColumnConst.h | 5 - src/Columns/ColumnTuple.cpp | 11 --- src/Columns/ColumnTuple.h | 1 - src/Columns/IColumn.cpp | 5 - src/Columns/IColumn.h | 5 - src/DataTypes/DataTypeFactory.cpp | 1 - src/DataTypes/DataTypeFactory.h | 2 +- src/DataTypes/DataTypeTuple.cpp | 24 ++++- src/DataTypes/DataTypeTuple.h | 2 + src/DataTypes/IDataType.cpp | 11 ++- src/DataTypes/IDataType.h | 4 +- .../Serializations/SerializationInfo.cpp | 26 ++++- .../Serializations/SerializationInfo.h | 5 + .../Serializations/SerializationInfoTuple.cpp | 41 +++++--- .../Serializations/SerializationInfoTuple.h | 12 ++- src/Formats/NativeWriter.cpp | 4 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 96 ++++++++++--------- src/Storages/MergeTree/IMergeTreeDataPart.h | 11 ++- src/Storages/MergeTree/IMergeTreeReader.cpp | 42 +++----- src/Storages/MergeTree/IMergeTreeReader.h | 12 +-- .../MergeTree/MergeTreeBlockReadUtils.cpp | 2 +- .../MergeTree/MergeTreeReaderCompact.cpp | 9 +- .../MergeTree/MergeTreeReaderInMemory.cpp | 4 +- .../MergeTree/MergeTreeReaderWide.cpp | 6 +- .../01825_type_json_sparse.reference | 13 +++ .../0_stateless/01825_type_json_sparse.sql | 59 ++++++++++++ 26 files changed, 264 insertions(+), 149 deletions(-) create mode 100644 tests/queries/0_stateless/01825_type_json_sparse.reference create mode 100644 tests/queries/0_stateless/01825_type_json_sparse.sql diff --git a/src/Columns/ColumnConst.h b/src/Columns/ColumnConst.h index fdee16a1dbb..21cfaf7f64c 100644 --- a/src/Columns/ColumnConst.h +++ b/src/Columns/ColumnConst.h @@ -263,11 +263,6 @@ public: } } - SerializationInfoPtr getSerializationInfo() const override - { - return data->getSerializationInfo(); - } - bool isNullable() const override { return isColumnNullable(*data); } bool onlyNull() const override { return data->isNullAt(0); } bool isNumeric() const override { return data->isNumeric(); } diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 7c8d73edd16..6763c11bb9a 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -561,15 +561,4 @@ void ColumnTuple::getIndicesOfNonDefaultRows(Offsets & indices, size_t from, siz return getIndicesOfNonDefaultRowsImpl(indices, from, limit); } -SerializationInfoPtr ColumnTuple::getSerializationInfo() const -{ - MutableSerializationInfos infos; - infos.reserve(columns.size()); - - for (const auto & column : columns) - infos.push_back(const_pointer_cast(column->getSerializationInfo())); - - return std::make_shared(std::move(infos), SerializationInfo::Settings{}); -} - } diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index e6797c2582c..b1de8df74a9 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -102,7 +102,6 @@ public: ColumnPtr compress() const override; double getRatioOfDefaultRows(double sample_ratio) const override; void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override; - SerializationInfoPtr getSerializationInfo() const override; size_t tupleSize() const { return columns.size(); } diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index e7caee3b23d..2158adb86a8 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -64,11 +64,6 @@ ColumnPtr IColumn::createWithOffsets(const Offsets & offsets, const Field & defa return res; } -SerializationInfoPtr IColumn::getSerializationInfo() const -{ - return std::make_shared(ISerialization::getKind(*this), SerializationInfo::Settings{}); -} - bool isColumnNullable(const IColumn & column) { return checkColumn(column); diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index a99d4172e5b..8b693015674 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -35,9 +35,6 @@ class ColumnGathererStream; class Field; class WeakHash32; -class SerializationInfo; -using SerializationInfoPtr = std::shared_ptr; - /* * Represents a set of equal ranges in previous column to perform sorting in current column. * Used in sorting by tuples. @@ -445,8 +442,6 @@ public: /// Used to create full column from sparse. [[nodiscard]] virtual Ptr createWithOffsets(const Offsets & offsets, const Field & default_field, size_t total_rows, size_t shift) const; - [[nodiscard]] virtual SerializationInfoPtr getSerializationInfo() const; - /// Compress column in memory to some representation that allows to decompress it back. /// Return itself if compression is not applicable for this column type. [[nodiscard]] virtual Ptr compress() const diff --git a/src/DataTypes/DataTypeFactory.cpp b/src/DataTypes/DataTypeFactory.cpp index dd08656dccc..9f60210f8d6 100644 --- a/src/DataTypes/DataTypeFactory.cpp +++ b/src/DataTypes/DataTypeFactory.cpp @@ -119,7 +119,6 @@ void DataTypeFactory::registerDataType(const String & family_name, Value creator throw Exception("DataTypeFactory: the data type family name '" + family_name + "' is not unique", ErrorCodes::LOGICAL_ERROR); - if (case_sensitiveness == CaseInsensitive && !case_insensitive_data_types.emplace(family_name_lowercase, creator).second) throw Exception("DataTypeFactory: the case insensitive data type family name '" + family_name + "' is not unique", diff --git a/src/DataTypes/DataTypeFactory.h b/src/DataTypes/DataTypeFactory.h index 704d8926bf0..f452f6167c7 100644 --- a/src/DataTypes/DataTypeFactory.h +++ b/src/DataTypes/DataTypeFactory.h @@ -25,7 +25,7 @@ class DataTypeFactory final : private boost::noncopyable, public IFactoryWithAli private: using SimpleCreator = std::function; using DataTypesDictionary = std::unordered_map; - using CreatorWithCustom = std::function(const ASTPtr & parameters)>; + using CreatorWithCustom = std::function(const ASTPtr & parameters)>; using SimpleCreatorWithCustom = std::function()>; public: diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 558b13927c1..768d88b57df 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -257,6 +258,7 @@ size_t DataTypeTuple::getSizeOfValueInMemory() const SerializationPtr DataTypeTuple::doGetDefaultSerialization() const { SerializationTuple::ElementSerializations serializations(elems.size()); + for (size_t i = 0; i < elems.size(); ++i) { String elem_name = have_explicit_names ? names[i] : toString(i + 1); @@ -289,7 +291,27 @@ MutableSerializationInfoPtr DataTypeTuple::createSerializationInfo(const Seriali for (const auto & elem : elems) infos.push_back(elem->createSerializationInfo(settings)); - return std::make_shared(std::move(infos), settings); + return std::make_shared(std::move(infos), names, settings); +} + +SerializationInfoPtr DataTypeTuple::getSerializationInfo(const IColumn & column) const +{ + if (const auto * column_const = checkAndGetColumn(&column)) + return getSerializationInfo(column_const->getDataColumn()); + + MutableSerializationInfos infos; + infos.reserve(elems.size()); + + const auto & column_tuple = assert_cast(column); + assert(elems.size() == column_tuple.getColumns().size()); + + for (size_t i = 0; i < elems.size(); ++i) + { + auto element_info = elems[i]->getSerializationInfo(column_tuple.getColumn(i)); + infos.push_back(const_pointer_cast(element_info)); + } + + return std::make_shared(std::move(infos), names, SerializationInfo::Settings{}); } diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 009a2284a0a..1ce6bd9e966 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -22,6 +22,7 @@ private: DataTypes elems; Strings names; bool have_explicit_names; + public: static constexpr bool is_parametric = true; @@ -54,6 +55,7 @@ public: SerializationPtr doGetDefaultSerialization() const override; SerializationPtr getSerialization(const SerializationInfo & info) const override; MutableSerializationInfoPtr createSerializationInfo(const SerializationInfo::Settings & settings) const override; + SerializationInfoPtr getSerializationInfo(const IColumn & column) const override; const DataTypePtr & getElement(size_t i) const { return elems[i]; } const DataTypes & getElements() const { return elems; } diff --git a/src/DataTypes/IDataType.cpp b/src/DataTypes/IDataType.cpp index f2bb878a533..cb3bab5c653 100644 --- a/src/DataTypes/IDataType.cpp +++ b/src/DataTypes/IDataType.cpp @@ -179,12 +179,19 @@ void IDataType::setCustomization(DataTypeCustomDescPtr custom_desc_) const custom_serialization = std::move(custom_desc_->serialization); } -MutableSerializationInfoPtr IDataType::createSerializationInfo( - const SerializationInfo::Settings & settings) const +MutableSerializationInfoPtr IDataType::createSerializationInfo(const SerializationInfo::Settings & settings) const { return std::make_shared(ISerialization::Kind::DEFAULT, settings); } +SerializationInfoPtr IDataType::getSerializationInfo(const IColumn & column) const +{ + if (const auto * column_const = checkAndGetColumn(&column)) + return getSerializationInfo(column_const->getDataColumn()); + + return std::make_shared(ISerialization::getKind(column), SerializationInfo::Settings{}); +} + SerializationPtr IDataType::getDefaultSerialization() const { if (custom_serialization) diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index fce8906abe5..fc6ba2a2fc3 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -101,8 +101,8 @@ public: Names getSubcolumnNames() const; - virtual MutableSerializationInfoPtr createSerializationInfo( - const SerializationInfo::Settings & settings) const; + virtual MutableSerializationInfoPtr createSerializationInfo(const SerializationInfo::Settings & settings) const; + virtual SerializationInfoPtr getSerializationInfo(const IColumn & column) const; /// TODO: support more types. virtual bool supportsSparseSerialization() const { return !haveSubtypes(); } diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index a0dc20b6479..543b4a75f0a 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -1,9 +1,9 @@ #include -#include #include #include #include #include +#include #include #include @@ -47,12 +47,25 @@ void SerializationInfo::Data::add(const Data & other) num_defaults += other.num_defaults; } +void SerializationInfo::Data::addDefaults(size_t length) +{ + num_rows += length; + num_defaults += length; +} + SerializationInfo::SerializationInfo(ISerialization::Kind kind_, const Settings & settings_) : settings(settings_) , kind(kind_) { } +SerializationInfo::SerializationInfo(ISerialization::Kind kind_, const Settings & settings_, const Data & data_) + : settings(settings_) + , kind(kind_) + , data(data_) +{ +} + void SerializationInfo::add(const IColumn & column) { data.add(column); @@ -67,6 +80,13 @@ void SerializationInfo::add(const SerializationInfo & other) kind = chooseKind(data, settings); } +void SerializationInfo::addDefaults(size_t length) +{ + data.addDefaults(length); + if (settings.choose_kind) + kind = chooseKind(data, settings); +} + void SerializationInfo::replaceData(const SerializationInfo & other) { data = other.data; @@ -74,9 +94,7 @@ void SerializationInfo::replaceData(const SerializationInfo & other) MutableSerializationInfoPtr SerializationInfo::clone() const { - auto res = std::make_shared(kind, settings); - res->data = data; - return res; + return std::make_shared(kind, settings, data); } void SerializationInfo::serialializeKindBinary(WriteBuffer & out) const diff --git a/src/DataTypes/Serializations/SerializationInfo.h b/src/DataTypes/Serializations/SerializationInfo.h index d83fc16f2f6..a4a5685253f 100644 --- a/src/DataTypes/Serializations/SerializationInfo.h +++ b/src/DataTypes/Serializations/SerializationInfo.h @@ -34,6 +34,7 @@ public: void add(const IColumn & column); void add(const Data & other); + void addDefaults(size_t length); }; struct Settings @@ -45,6 +46,7 @@ public: }; SerializationInfo(ISerialization::Kind kind_, const Settings & settings_); + SerializationInfo(ISerialization::Kind kind_, const Settings & settings_, const Data & data_); virtual ~SerializationInfo() = default; @@ -52,7 +54,9 @@ public: virtual void add(const IColumn & column); virtual void add(const SerializationInfo & other); + virtual void addDefaults(size_t length); virtual void replaceData(const SerializationInfo & other); + virtual std::shared_ptr clone() const; virtual void serialializeKindBinary(WriteBuffer & out) const; @@ -61,6 +65,7 @@ public: virtual Poco::JSON::Object toJSON() const; virtual void fromJSON(const Poco::JSON::Object & object); + void setKind(ISerialization::Kind kind_) { kind = kind_; } const Settings & getSettings() const { return settings; } const Data & getData() const { return data; } ISerialization::Kind getKind() const { return kind; } diff --git a/src/DataTypes/Serializations/SerializationInfoTuple.cpp b/src/DataTypes/Serializations/SerializationInfoTuple.cpp index 803302f9642..b4683c7d0a0 100644 --- a/src/DataTypes/Serializations/SerializationInfoTuple.cpp +++ b/src/DataTypes/Serializations/SerializationInfoTuple.cpp @@ -10,13 +10,18 @@ namespace ErrorCodes { extern const int CORRUPTED_DATA; extern const int THERE_IS_NO_COLUMN; + extern const int LOGICAL_ERROR; } SerializationInfoTuple::SerializationInfoTuple( - MutableSerializationInfos elems_, const Settings & settings_) + MutableSerializationInfos elems_, Names names_, const Settings & settings_) : SerializationInfo(ISerialization::Kind::DEFAULT, settings_) , elems(std::move(elems_)) + , names(std::move(names_)) { + assert(names.size() == elems.size()); + for (size_t i = 0; i < names.size(); ++i) + name_to_elem[names[i]] = elems[i]; } bool SerializationInfoTuple::hasCustomSerialization() const @@ -40,22 +45,34 @@ void SerializationInfoTuple::add(const SerializationInfo & other) { SerializationInfo::add(other); - const auto & info_tuple = assert_cast(other); - assert(elems.size() == info_tuple.elems.size()); + const auto & other_info = assert_cast(other); + for (const auto & [name, elem] : name_to_elem) + { + auto it = other_info.name_to_elem.find(name); + if (it != other_info.name_to_elem.end()) + elem->add(*it->second); + else + elem->addDefaults(other_info.getData().num_rows); + } +} - for (size_t i = 0; i < elems.size(); ++i) - elems[i]->add(*info_tuple.elems[i]); +void SerializationInfoTuple::addDefaults(size_t length) +{ + for (const auto & elem : elems) + elem->addDefaults(length); } void SerializationInfoTuple::replaceData(const SerializationInfo & other) { SerializationInfo::add(other); - const auto & info_tuple = assert_cast(other); - assert(elems.size() == info_tuple.elems.size()); - - for (size_t i = 0; i < elems.size(); ++i) - elems[i]->replaceData(*info_tuple.elems[i]); + const auto & other_info = assert_cast(other); + for (const auto & [name, elem] : name_to_elem) + { + auto it = other_info.name_to_elem.find(name); + if (it != other_info.name_to_elem.end()) + elem->replaceData(*it->second); + } } MutableSerializationInfoPtr SerializationInfoTuple::clone() const @@ -65,7 +82,7 @@ MutableSerializationInfoPtr SerializationInfoTuple::clone() const for (const auto & elem : elems) elems_cloned.push_back(elem->clone()); - return std::make_shared(std::move(elems_cloned), settings); + return std::make_shared(std::move(elems_cloned), names, settings); } void SerializationInfoTuple::serialializeKindBinary(WriteBuffer & out) const @@ -99,7 +116,7 @@ void SerializationInfoTuple::fromJSON(const Poco::JSON::Object & object) if (!object.has("subcolumns")) throw Exception(ErrorCodes::CORRUPTED_DATA, - "Missed field '{}' in SerializationInfo of columns SerializationInfoTuple"); + "Missed field 'subcolumns' in SerializationInfo of columns SerializationInfoTuple"); auto subcolumns = object.getArray("subcolumns"); if (elems.size() != subcolumns->size()) diff --git a/src/DataTypes/Serializations/SerializationInfoTuple.h b/src/DataTypes/Serializations/SerializationInfoTuple.h index d196f80393e..b01c629d2ff 100644 --- a/src/DataTypes/Serializations/SerializationInfoTuple.h +++ b/src/DataTypes/Serializations/SerializationInfoTuple.h @@ -1,4 +1,5 @@ #pragma once +#include #include namespace DB @@ -7,25 +8,32 @@ namespace DB class SerializationInfoTuple : public SerializationInfo { public: - SerializationInfoTuple(MutableSerializationInfos elems_, const Settings & settings_); + SerializationInfoTuple(MutableSerializationInfos elems_, Names names_, const Settings & settings_); bool hasCustomSerialization() const override; + void add(const IColumn & column) override; void add(const SerializationInfo & other) override; + void addDefaults(size_t length) override; void replaceData(const SerializationInfo & other) override; MutableSerializationInfoPtr clone() const override; + void serialializeKindBinary(WriteBuffer & out) const override; void deserializeFromKindsBinary(ReadBuffer & in) override; Poco::JSON::Object toJSON() const override; void fromJSON(const Poco::JSON::Object & object) override; - MutableSerializationInfoPtr getElementInfo(size_t i) const { return elems[i]; } + const MutableSerializationInfoPtr & getElementInfo(size_t i) const { return elems[i]; } ISerialization::Kind getElementKind(size_t i) const { return elems[i]->getKind(); } private: MutableSerializationInfos elems; + Names names; + + using NameToElem = std::unordered_map; + NameToElem name_to_elem; }; } diff --git a/src/Formats/NativeWriter.cpp b/src/Formats/NativeWriter.cpp index 77692eec6b6..9d4cfb68d56 100644 --- a/src/Formats/NativeWriter.cpp +++ b/src/Formats/NativeWriter.cpp @@ -103,7 +103,7 @@ void NativeWriter::write(const Block & block) mark.offset_in_decompressed_block = ostr_concrete->getRemainingBytes(); } - ColumnWithTypeAndName column = block.safeGetByPosition(i); + auto column = block.safeGetByPosition(i); /// Send data to old clients without low cardinality type. if (remove_low_cardinality || (client_revision && client_revision < DBMS_MIN_REVISION_WITH_LOW_CARDINALITY_TYPE)) @@ -145,7 +145,7 @@ void NativeWriter::write(const Block & block) SerializationPtr serialization; if (client_revision >= DBMS_MIN_REVISION_WITH_CUSTOM_SERIALIZATION) { - auto info = column.column->getSerializationInfo(); + auto info = column.type->getSerializationInfo(*column.column); serialization = column.type->getSerialization(*info); bool has_custom = info->hasCustomSerialization(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 60941108f00..d4d3844de6c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -15,6 +15,8 @@ #include #include #include +#include +#include #include #include #include @@ -445,6 +447,18 @@ void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns) for (const auto & column : columns) column_name_to_position.emplace(column.name, pos++); + + columns_description = ColumnsDescription(columns); +} + +NameAndTypePair IMergeTreeDataPart::getColumn(const String & column_name) const +{ + return columns_description.getColumnOrSubcolumn(GetColumnsOptions::AllPhysical, column_name); +} + +std::optional IMergeTreeDataPart::tryGetColumn(const String & column_name) const +{ + return columns_description.tryGetColumnOrSubcolumn(GetColumnsOptions::AllPhysical, column_name); } void IMergeTreeDataPart::setSerializationInfos(const SerializationInfoByName & new_infos) @@ -454,10 +468,15 @@ void IMergeTreeDataPart::setSerializationInfos(const SerializationInfoByName & n SerializationPtr IMergeTreeDataPart::getSerialization(const NameAndTypePair & column) const { - auto it = serialization_infos.find(column.getNameInStorage()); - return it == serialization_infos.end() - ? IDataType::getSerialization(column) - : IDataType::getSerialization(column, *it->second); + auto column_in_part = tryGetColumn(column.name); + if (!column_in_part) + return IDataType::getSerialization(column); + + auto it = serialization_infos.find(column_in_part->getNameInStorage()); + if (it == serialization_infos.end()) + return IDataType::getSerialization(*column_in_part); + + return IDataType::getSerialization(*column_in_part, *it->second); } void IMergeTreeDataPart::removeIfNeeded() @@ -564,37 +583,38 @@ size_t IMergeTreeDataPart::getFileSizeOrZero(const String & file_name) const return checksum->second.file_size; } -String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize( - const StorageSnapshotPtr & storage_snapshot, bool with_subcolumns) const +String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize(bool with_subcolumns) const { - auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects(); - if (with_subcolumns) - options.withSubcolumns(); + auto find_column_with_minimum_size = [&](const auto & columns_list) + { + std::optional minimum_size_column; + UInt64 minimum_size = std::numeric_limits::max(); - auto storage_columns = storage_snapshot->getColumns(options); - MergeTreeData::AlterConversions alter_conversions; - if (!parent_part) - alter_conversions = storage.getAlterConversionsForPart(shared_from_this()); + for (const auto & column : columns_list) + { + if (!hasColumnFiles(column)) + continue; + + const auto size = getColumnSize(column.name).data_compressed; + if (size < minimum_size) + { + minimum_size = size; + minimum_size_column = column.name; + } + } + + return minimum_size_column; + }; std::optional minimum_size_column; - UInt64 minimum_size = std::numeric_limits::max(); - - for (const auto & column : storage_columns) + if (with_subcolumns) { - auto column_name = column.name; - auto column_type = column.type; - if (alter_conversions.isColumnRenamed(column.name)) - column_name = alter_conversions.getColumnOldName(column.name); - - if (!hasColumnFiles(column)) - continue; - - const auto size = getColumnSize(column_name).data_compressed; - if (size < minimum_size) - { - minimum_size = size; - minimum_size_column = column_name; - } + auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withSubcolumns(); + minimum_size_column = find_column_with_minimum_size(columns_description.get(options)); + } + else + { + minimum_size_column = find_column_with_minimum_size(columns); } if (!minimum_size_column) @@ -603,22 +623,6 @@ String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize( return *minimum_size_column; } -// String IMergeTreeDataPart::getFullPath() const -// { -// if (relative_path.empty()) -// throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); - -// return fs::path(storage.getFullPathOnDisk(volume->getDisk())) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; -// } - -// String IMergeTreeDataPart::getRelativePath() const -// { -// if (relative_path.empty()) -// throw Exception("Part relative_path cannot be empty. It's bug.", ErrorCodes::LOGICAL_ERROR); - -// return fs::path(storage.relative_data_path) / (parent_part ? parent_part->relative_path : "") / relative_path / ""; -// } - void IMergeTreeDataPart::loadColumnsChecksumsIndexes(bool require_columns_checksums, bool check_consistency) { assertOnDisk(); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 7f3c41ce4c2..613fda59c84 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -14,6 +14,7 @@ #include #include #include +#include #include #include #include @@ -136,6 +137,9 @@ public: const NamesAndTypesList & getColumns() const { return columns; } + NameAndTypePair getColumn(const String & name) const; + std::optional tryGetColumn(const String & column_name) const; + void setSerializationInfos(const SerializationInfoByName & new_infos); const SerializationInfoByName & getSerializationInfos() const { return serialization_infos; } @@ -167,8 +171,7 @@ public: /// Returns the name of a column with minimum compressed size (as returned by getColumnSize()). /// If no checksums are present returns the name of the first physically existing column. - String getColumnNameWithMinimumCompressedSize( - const StorageSnapshotPtr & storage_snapshot, bool with_subcolumns) const; + String getColumnNameWithMinimumCompressedSize(bool with_subcolumns) const; bool contains(const IMergeTreeDataPart & other) const { return info.contains(other.info); } @@ -521,6 +524,10 @@ private: /// Map from name of column to its serialization info. SerializationInfoByName serialization_infos; + /// Columns description for more convinient access + /// to columns by name and getting subcolumns. + ColumnsDescription columns_description; + /// Reads part unique identifier (if exists) from uuid.txt void loadUUID(); diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index b8aeb8e6a5a..08796636f89 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -18,6 +18,7 @@ namespace namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int THERE_IS_NO_COLUMN; } @@ -33,7 +34,6 @@ IMergeTreeReader::IMergeTreeReader( : data_part(data_part_) , avg_value_size_hints(avg_value_size_hints_) , columns(columns_) - , part_columns(data_part->getColumns()) , uncompressed_cache(uncompressed_cache_) , mark_cache(mark_cache_) , settings(settings_) @@ -47,11 +47,7 @@ IMergeTreeReader::IMergeTreeReader( /// For wide parts convert plain arrays of Nested to subcolumns /// to allow to use shared offset column from cache. columns = Nested::convertToSubcolumns(columns); - part_columns = Nested::collect(part_columns); } - - for (const auto & column_from_part : part_columns) - columns_from_part[column_from_part.name] = &column_from_part.type; } IMergeTreeReader::~IMergeTreeReader() = default; @@ -124,37 +120,25 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns } } -NameAndTypePair IMergeTreeReader::getColumnFromPart(const NameAndTypePair & required_column) const +String IMergeTreeReader::getColumnNameInPart(const NameAndTypePair & required_column) const { auto name_in_storage = required_column.getNameInStorage(); - - ColumnsFromPart::ConstLookupResult it; if (alter_conversions.isColumnRenamed(name_in_storage)) { - String old_name = alter_conversions.getColumnOldName(name_in_storage); - it = columns_from_part.find(old_name); - } - else - { - it = columns_from_part.find(name_in_storage); + name_in_storage = alter_conversions.getColumnOldName(name_in_storage); + return Nested::concatenateName(name_in_storage, required_column.getSubcolumnName()); } - if (it == columns_from_part.end()) - return required_column; + return required_column.name; +} - const DataTypePtr & type = *it->getMapped(); - if (required_column.isSubcolumn()) - { - auto subcolumn_name = required_column.getSubcolumnName(); - auto subcolumn_type = type->tryGetSubcolumnType(subcolumn_name); +NameAndTypePair IMergeTreeReader::getColumnInPart(const NameAndTypePair & required_column) const +{ + auto column_in_part = data_part->tryGetColumn(getColumnNameInPart(required_column)); + if (column_in_part) + return *column_in_part; - if (!subcolumn_type) - return required_column; - - return {String(it->getKey()), subcolumn_name, type, subcolumn_type}; - } - - return {String(it->getKey()), type}; + return required_column; } void IMergeTreeReader::performRequiredConversions(Columns & res_columns) const @@ -183,7 +167,7 @@ void IMergeTreeReader::performRequiredConversions(Columns & res_columns) const if (res_columns[pos] == nullptr) continue; - copy_block.insert({res_columns[pos], getColumnFromPart(*name_and_type).type, name_and_type->name}); + copy_block.insert({res_columns[pos], getColumnInPart(*name_and_type).type, name_and_type->name}); } DB::performRequiredConversions(copy_block, columns, storage.getContext()); diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index b13db9c3255..78cc96ccd27 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -63,8 +63,10 @@ public: MergeTreeData::DataPartPtr data_part; protected: - /// Returns actual column type in part, which can differ from table metadata. - NameAndTypePair getColumnFromPart(const NameAndTypePair & required_column) const; + /// Returns actual column name in part, which can differ from table metadata. + String getColumnNameInPart(const NameAndTypePair & required_column) const; + /// Returns actual column name and type in part, which can differ from table metadata. + NameAndTypePair getColumnInPart(const NameAndTypePair & required_column) const; void checkNumberOfColumns(size_t num_columns_to_read) const; @@ -75,7 +77,6 @@ protected: /// Columns that are read. NamesAndTypesList columns; - NamesAndTypesList part_columns; UncompressedCache * uncompressed_cache; MarkCache * mark_cache; @@ -92,11 +93,6 @@ protected: private: /// Alter conversions, which must be applied on fly if required MergeTreeData::AlterConversions alter_conversions; - - /// Actual data type of columns in part - - using ColumnsFromPart = HashMapWithSavedHash; - ColumnsFromPart columns_from_part; }; } diff --git a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp index 50f4c34f004..b71044b8373 100644 --- a/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp +++ b/src/Storages/MergeTree/MergeTreeBlockReadUtils.cpp @@ -122,7 +122,7 @@ NameSet injectRequiredColumns( */ if (!have_at_least_one_physical_column) { - const auto minimum_size_column_name = part->getColumnNameWithMinimumCompressedSize(storage_snapshot, with_subcolumns); + const auto minimum_size_column_name = part->getColumnNameWithMinimumCompressedSize(with_subcolumns); columns.push_back(minimum_size_column_name); /// correctly report added column injected_columns.insert(columns.back()); diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 8e2e1d02836..5cb37b0515a 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -54,14 +54,14 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( { if (name_and_type->isSubcolumn()) { - auto storage_column_from_part = getColumnFromPart( + auto storage_column_from_part = getColumnInPart( {name_and_type->getNameInStorage(), name_and_type->getTypeInStorage()}); if (!storage_column_from_part.type->tryGetSubcolumnType(name_and_type->getSubcolumnName())) continue; } - auto column_from_part = getColumnFromPart(*name_and_type); + auto column_from_part = getColumnInPart(*name_and_type); auto position = data_part->getColumnPosition(column_from_part.getNameInStorage()); if (!position && typeid_cast(column_from_part.type.get())) @@ -153,7 +153,7 @@ size_t MergeTreeReaderCompact::readRows( if (!column_positions[i]) continue; - auto column_from_part = getColumnFromPart(*column_it); + auto column_from_part = getColumnInPart(*column_it); if (res_columns[i] == nullptr) { auto serialization = data_part->getSerialization(column_from_part); @@ -168,10 +168,11 @@ size_t MergeTreeReaderCompact::readRows( auto name_and_type = columns.begin(); for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) { - auto column_from_part = getColumnFromPart(*name_and_type); if (!res_columns[pos]) continue; + auto column_from_part = getColumnInPart(*name_and_type); + try { auto & column = res_columns[pos]; diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index c1b0067dbb0..d8e0b6e4d6c 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -34,7 +34,7 @@ MergeTreeReaderInMemory::MergeTreeReaderInMemory( { for (const auto & name_and_type : columns) { - auto [name, type] = getColumnFromPart(name_and_type); + auto [name, type] = getColumnInPart(name_and_type); /// If array of Nested column is missing in part, /// we have to read its offsets if they exist. @@ -67,7 +67,7 @@ size_t MergeTreeReaderInMemory::readRows( auto column_it = columns.begin(); for (size_t i = 0; i < num_columns; ++i, ++column_it) { - auto name_type = getColumnFromPart(*column_it); + auto name_type = getColumnInPart(*column_it); /// Copy offsets, if array of Nested column is missing in part. auto offsets_it = positions_for_offsets.find(name_type.name); diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index 07fe187332e..e098e76d4ee 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -49,7 +49,7 @@ MergeTreeReaderWide::MergeTreeReaderWide( { for (const NameAndTypePair & column : columns) { - auto column_from_part = getColumnFromPart(column); + auto column_from_part = getColumnInPart(column); addStreams(column_from_part, profile_callback_, clock_type_); } } @@ -83,7 +83,7 @@ size_t MergeTreeReaderWide::readRows( auto name_and_type = columns.begin(); for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) { - auto column_from_part = getColumnFromPart(*name_and_type); + auto column_from_part = getColumnInPart(*name_and_type); try { auto & cache = caches[column_from_part.getNameInStorage()]; @@ -102,7 +102,7 @@ size_t MergeTreeReaderWide::readRows( for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) { - auto column_from_part = getColumnFromPart(*name_and_type); + auto column_from_part = getColumnInPart(*name_and_type); const auto & [name, type] = column_from_part; /// The column is already present in the block so we will append the values to the end. diff --git a/tests/queries/0_stateless/01825_type_json_sparse.reference b/tests/queries/0_stateless/01825_type_json_sparse.reference new file mode 100644 index 00000000000..aaaa683d7d5 --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_sparse.reference @@ -0,0 +1,13 @@ +k1 Default 2 +k2.k3 Default 1 +============= +k1 Default 1 +k2.k3 Sparse 1 +============= +k1 Default 1 +k2.k3 Sparse 1 +============= +k1 Default 1 +k2.k3 Sparse 1 +1 1 4 +2 400000 0 diff --git a/tests/queries/0_stateless/01825_type_json_sparse.sql b/tests/queries/0_stateless/01825_type_json_sparse.sql new file mode 100644 index 00000000000..2e7520f3e7a --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_sparse.sql @@ -0,0 +1,59 @@ +-- Tags: no-fasttest + +DROP TABLE IF EXISTS t_json_sparse; + +SET allow_experimental_object_type = 1; + +CREATE TABLE t_json_sparse (data JSON) +ENGINE = MergeTree ORDER BY tuple() +SETTINGS ratio_of_defaults_for_sparse_serialization = 0.1, +min_bytes_for_wide_part = 0; + +SYSTEM STOP MERGES t_json_sparse; + +INSERT INTO t_json_sparse VALUES ('{"k1": 1, "k2": {"k3": 4}}'); +INSERT INTO t_json_sparse SELECT '{"k1": 2}' FROM numbers(200000); + +SELECT subcolumns.names, subcolumns.serializations, count() FROM system.parts_columns +ARRAY JOIN subcolumns +WHERE database = currentDatabase() + AND table = 't_json_sparse' AND column = 'data' AND active +GROUP BY subcolumns.names, subcolumns.serializations; + +SELECT '============='; + +SYSTEM START MERGES t_json_sparse; +OPTIMIZE TABLE t_json_sparse FINAL; + +SELECT subcolumns.names, subcolumns.serializations, count() FROM system.parts_columns +ARRAY JOIN subcolumns +WHERE database = currentDatabase() + AND table = 't_json_sparse' AND column = 'data' AND active +GROUP BY subcolumns.names, subcolumns.serializations; + +SELECT '============='; + +DETACH TABLE t_json_sparse; +ATTACH TABLE t_json_sparse; + +SELECT subcolumns.names, subcolumns.serializations, count() FROM system.parts_columns +ARRAY JOIN subcolumns +WHERE database = currentDatabase() + AND table = 't_json_sparse' AND column = 'data' AND active +GROUP BY subcolumns.names, subcolumns.serializations; + +INSERT INTO t_json_sparse SELECT '{"k1": 2}' FROM numbers(200000); + +SELECT '============='; + +OPTIMIZE TABLE t_json_sparse FINAL; + +SELECT subcolumns.names, subcolumns.serializations, count() FROM system.parts_columns +ARRAY JOIN subcolumns +WHERE database = currentDatabase() + AND table = 't_json_sparse' AND column = 'data' AND active +GROUP BY subcolumns.names, subcolumns.serializations; + +SELECT data.k1, count(), sum(data.k2.k3) FROM t_json_sparse GROUP BY data.k1; + +DROP TABLE t_json_sparse; From 10719fd61d36b9d991764727869fbd994553aa51 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 21 Jul 2022 15:06:55 +0000 Subject: [PATCH 029/117] fix style check --- src/DataTypes/Serializations/SerializationInfoTuple.cpp | 1 - src/Storages/MergeTree/IMergeTreeDataPart.h | 2 +- src/Storages/MergeTree/IMergeTreeReader.cpp | 1 - 3 files changed, 1 insertion(+), 3 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationInfoTuple.cpp b/src/DataTypes/Serializations/SerializationInfoTuple.cpp index b4683c7d0a0..d0fa5572a48 100644 --- a/src/DataTypes/Serializations/SerializationInfoTuple.cpp +++ b/src/DataTypes/Serializations/SerializationInfoTuple.cpp @@ -10,7 +10,6 @@ namespace ErrorCodes { extern const int CORRUPTED_DATA; extern const int THERE_IS_NO_COLUMN; - extern const int LOGICAL_ERROR; } SerializationInfoTuple::SerializationInfoTuple( diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index 613fda59c84..b6e8b75147d 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -524,7 +524,7 @@ private: /// Map from name of column to its serialization info. SerializationInfoByName serialization_infos; - /// Columns description for more convinient access + /// Columns description for more convenient access /// to columns by name and getting subcolumns. ColumnsDescription columns_description; diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 08796636f89..896fc805aa5 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -18,7 +18,6 @@ namespace namespace ErrorCodes { extern const int LOGICAL_ERROR; - extern const int THERE_IS_NO_COLUMN; } From e98d8a23ca5896b917cad05ca1fa2f670dabe15a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 20 Jul 2022 20:39:26 +0000 Subject: [PATCH 030/117] Fixing test. --- .../01834_alias_columns_laziness_filimonov.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.reference b/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.reference index 5290ffc5dbe..91c37a277a1 100644 --- a/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.reference +++ b/tests/queries/0_stateless/01834_alias_columns_laziness_filimonov.reference @@ -1,2 +1,2 @@ -SleepFunctionCalls: 4 (increment) -SleepFunctionMicroseconds: 400000 (increment) +SleepFunctionCalls: 3 (increment) +SleepFunctionMicroseconds: 300000 (increment) From 53175e43f37612b13b26337eca60fc9575206edf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 22 Jul 2022 18:56:01 +0000 Subject: [PATCH 031/117] Better support for projections. --- src/Common/mmap.cpp | 26 +++++++++++++++++++ src/Storages/MergeTree/MergeTreeData.cpp | 19 ++++++++++++++ .../MergeTree/MergeTreeDataSelectExecutor.cpp | 6 +++-- .../MergeTree/MergeTreeDataSelectExecutor.h | 2 ++ 4 files changed, 51 insertions(+), 2 deletions(-) create mode 100644 src/Common/mmap.cpp diff --git a/src/Common/mmap.cpp b/src/Common/mmap.cpp new file mode 100644 index 00000000000..15c42ea6636 --- /dev/null +++ b/src/Common/mmap.cpp @@ -0,0 +1,26 @@ +#if defined(OS_LINUX) +#include + + +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wredundant-decls" +extern "C" +{ + void * mmap(void * /*addr*/, size_t /*length*/, int /*prot*/, int /*flags*/, int /*fd*/, off_t /*offset*/); + int munmap(void * /*addr*/, size_t /*length*/); +} +#pragma GCC diagnostic pop + +template +inline void ignore(T x __attribute__((unused))) +{ +} + +static void dummyFunctionForInterposing() __attribute__((used)); +static void dummyFunctionForInterposing() +{ + /// Suppression for PVS-Studio and clang-tidy. + ignore(mmap(nullptr, 0, 0, 0, 0, 0)); // -V575 NOLINT + ignore(munmap(nullptr, 0)); // -V575 NOLINT +} +#endif diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index c5c3b82e775..04a9580df21 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5095,6 +5095,8 @@ static void selectBestProjection( const MergeTreeDataSelectExecutor & reader, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info, + const ActionsDAGPtr & added_filter, + const std::string & added_filter_column_name, const Names & required_columns, ProjectionCandidate & candidate, ContextPtr query_context, @@ -5125,6 +5127,8 @@ static void selectBestProjection( storage_snapshot->metadata, candidate.desc->metadata, query_info, + added_filter, + added_filter_column_name, query_context, settings.max_threads, max_added_blocks); @@ -5147,6 +5151,8 @@ static void selectBestProjection( storage_snapshot->metadata, storage_snapshot->metadata, query_info, // TODO syntax_analysis_result set in index + added_filter, + added_filter_column_name, query_context, settings.max_threads, max_added_blocks); @@ -5470,6 +5476,9 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg query_info.sets = std::move(select.getQueryAnalyzer()->getPreparedSets()); query_info.subquery_for_sets = std::move(select.getQueryAnalyzer()->getSubqueriesForSets()); + query_info.prewhere_info = analysis_result.prewhere_info; + const auto & before_where = analysis_result.before_where; + const auto & where_column_name = analysis_result.where_column_name; bool can_use_aggregate_projection = true; /// If the first stage of the query pipeline is more complex than Aggregating - Expression - Filter - ReadFromStorage, @@ -5739,6 +5748,8 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg metadata_snapshot, metadata_snapshot, query_info, + before_where, + where_column_name, query_context, settings.max_threads, max_added_blocks); @@ -5770,6 +5781,8 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg metadata_snapshot, metadata_snapshot, query_info, + before_where, + where_column_name, query_context, settings.max_threads, max_added_blocks); @@ -5795,6 +5808,8 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg reader, storage_snapshot, query_info, + before_where, + where_column_name, analysis_result.required_columns, candidate, query_context, @@ -5815,6 +5830,8 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg reader, storage_snapshot, query_info, + before_where, + where_column_name, analysis_result.required_columns, candidate, query_context, @@ -5844,6 +5861,8 @@ std::optional MergeTreeData::getQueryProcessingStageWithAgg selected_candidate->aggregate_descriptions = select.getQueryAnalyzer()->aggregates(); } + /// Just in case, reset prewhere info calculated from projection. + query_info.prewhere_info.reset(); return *selected_candidate; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 909fb6af82f..553e9a4fab8 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1249,6 +1249,8 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar const StorageMetadataPtr & metadata_snapshot_base, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, + const ActionsDAGPtr & added_filter, + const std::string & added_filter_column_name, ContextPtr context, unsigned num_streams, std::shared_ptr max_block_numbers_to_read) const @@ -1269,8 +1271,8 @@ MergeTreeDataSelectAnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar return ReadFromMergeTree::selectRangesToRead( std::move(parts), query_info.prewhere_info, - nullptr, - "", + added_filter, + added_filter_column_name, metadata_snapshot_base, metadata_snapshot, query_info, diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index e01e624a9b7..899cf1f2862 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -60,6 +60,8 @@ public: const StorageMetadataPtr & metadata_snapshot_base, const StorageMetadataPtr & metadata_snapshot, const SelectQueryInfo & query_info, + const ActionsDAGPtr & added_filter, + const std::string & added_filter_column_name, ContextPtr context, unsigned num_streams, std::shared_ptr max_block_numbers_to_read = nullptr) const; From dd65a36c5949dc803fb5c661c5ee7eb57ba6e70b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 22 Jul 2022 19:10:10 +0000 Subject: [PATCH 032/117] Fix another one test. --- src/Storages/MergeTree/KeyCondition.cpp | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index e590d98fce2..158165d9022 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1146,18 +1146,12 @@ bool KeyCondition::transformConstantWithValidFunctions( if (is_valid_chain) { - /// Here we cast constant to the input type. - /// It is not clear, why this works in general. - /// I can imagine the case when expression like `column < const` is legal, - /// but `type(column)` and `type(const)` are of different types, - /// and const cannot be casted to column type. - /// (There could be `superType(type(column), type(const))` which is used for comparison). - /// - /// However, looks like this case newer happenes (I could not find such). - /// Let's assume that any two comparable types are castable to each other. auto const_type = cur_node->result_type; auto const_column = out_type->createColumnConst(1, out_value); - auto const_value = (*castColumn({const_column, out_type, ""}, const_type))[0]; + auto const_value = (*castColumnAccurateOrNull({const_column, out_type, ""}, const_type))[0]; + + if (const_value.isNull()) + return false; while (!chain.empty()) { From 3d03b2714b7f97942d18c069bba4c03828ea3983 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 22 Jul 2022 23:05:11 +0000 Subject: [PATCH 033/117] try to fix Nested --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 7 ++++++- src/Storages/MergeTree/IMergeTreeReader.cpp | 6 ------ src/Storages/MergeTree/MergeTreeDataPartWide.cpp | 6 ++++-- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index d4d3844de6c..45e5d9d92de 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -448,7 +448,12 @@ void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns) for (const auto & column : columns) column_name_to_position.emplace(column.name, pos++); - columns_description = ColumnsDescription(columns); + /// For wide parts convert plain arrays to Nested for + /// more convinient managing of shared offsets column. + if (part_type == Type::Wide) + columns_description = ColumnsDescription(Nested::collect(columns)); + else + columns_description = ColumnsDescription(columns); } NameAndTypePair IMergeTreeDataPart::getColumn(const String & column_name) const diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 2ae087deffa..a5ef2fa6ac8 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -42,12 +42,6 @@ IMergeTreeReader::IMergeTreeReader( , all_mark_ranges(all_mark_ranges_) , alter_conversions(storage.getAlterConversionsForPart(data_part)) { - if (isWidePart(data_part)) - { - /// For wide parts convert plain arrays of Nested to subcolumns - /// to allow to use shared offset column from cache. - columns = Nested::convertToSubcolumns(columns); - } } IMergeTreeReader::~IMergeTreeReader() = default; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 7fe68420310..3e5576b6bdf 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -49,7 +49,8 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( { auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( - ptr, columns_to_read, metadata_snapshot, uncompressed_cache, + ptr, Nested::convertToSubcolumns(columns_to_read), + metadata_snapshot, uncompressed_cache, mark_cache, mark_ranges, reader_settings, avg_value_size_hints, profile_callback); } @@ -64,7 +65,8 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartWide::getWriter( const MergeTreeIndexGranularity & computed_index_granularity) const { return std::make_unique( - shared_from_this(), data_part_storage_builder, columns_list, metadata_snapshot, indices_to_recalc, + shared_from_this(), data_part_storage_builder, + Nested::convertToSubcolumns(columns_list), metadata_snapshot, indices_to_recalc, index_granularity_info.marks_file_extension, default_codec_, writer_settings, computed_index_granularity); } From 1ce8e714d3dd53ff801a249894a2f6ceaa2cd6c8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 25 Jul 2022 12:30:01 +0000 Subject: [PATCH 034/117] Fixing build. --- src/Common/mmap.cpp | 26 ------------------------- src/Storages/MergeTree/KeyCondition.cpp | 4 ++-- 2 files changed, 2 insertions(+), 28 deletions(-) delete mode 100644 src/Common/mmap.cpp diff --git a/src/Common/mmap.cpp b/src/Common/mmap.cpp deleted file mode 100644 index 15c42ea6636..00000000000 --- a/src/Common/mmap.cpp +++ /dev/null @@ -1,26 +0,0 @@ -#if defined(OS_LINUX) -#include - - -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wredundant-decls" -extern "C" -{ - void * mmap(void * /*addr*/, size_t /*length*/, int /*prot*/, int /*flags*/, int /*fd*/, off_t /*offset*/); - int munmap(void * /*addr*/, size_t /*length*/); -} -#pragma GCC diagnostic pop - -template -inline void ignore(T x __attribute__((unused))) -{ -} - -static void dummyFunctionForInterposing() __attribute__((used)); -static void dummyFunctionForInterposing() -{ - /// Suppression for PVS-Studio and clang-tidy. - ignore(mmap(nullptr, 0, 0, 0, 0, 0)); // -V575 NOLINT - ignore(munmap(nullptr, 0)); // -V575 NOLINT -} -#endif diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 158165d9022..d79861e2916 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -925,8 +925,8 @@ KeyCondition::KeyCondition( { for (size_t i = 0, size = key_column_names.size(); i < size; ++i) { - std::string name = key_column_names[i]; - if (!key_columns.count(name)) + const auto & name = key_column_names[i]; + if (!key_columns.contains(name)) key_columns[name] = i; } From 633b628e158459437a2d85bf2ab166cfdc402689 Mon Sep 17 00:00:00 2001 From: Yatsishin Ilya <2159081+qoega@users.noreply.github.com> Date: Mon, 25 Jul 2022 12:35:34 +0000 Subject: [PATCH 035/117] fix after conflict --- tests/integration/helpers/cluster.py | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/tests/integration/helpers/cluster.py b/tests/integration/helpers/cluster.py index 1e2c56995f7..0efc20e77a8 100644 --- a/tests/integration/helpers/cluster.py +++ b/tests/integration/helpers/cluster.py @@ -267,16 +267,6 @@ def extract_test_name(base_path): return name -def get_instances_dir(): - if ( - "INTEGRATION_TESTS_RUN_ID" in os.environ - and os.environ["INTEGRATION_TESTS_RUN_ID"] - ): - return "_instances_" + shlex.quote(os.environ["INTEGRATION_TESTS_RUN_ID"]) - else: - return "_instances" - - class ClickHouseCluster: """ClickHouse cluster with several instances and (possibly) ZooKeeper. From 06f3b31179c37ec0d3656dc26963b434d61cf5e5 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 25 Jul 2022 13:10:01 +0000 Subject: [PATCH 036/117] Roll back test for index hint. --- tests/queries/0_stateless/01739_index_hint.reference | 4 ++-- tests/queries/0_stateless/01739_index_hint.sql | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01739_index_hint.reference b/tests/queries/0_stateless/01739_index_hint.reference index af22b4a9676..71dfab29154 100644 --- a/tests/queries/0_stateless/01739_index_hint.reference +++ b/tests/queries/0_stateless/01739_index_hint.reference @@ -30,6 +30,6 @@ SELECT sum(t) FROM XXXX WHERE indexHint(t = 42); drop table if exists XXXX; create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings index_granularity=8192; insert into XXXX select number*60, 0 from numbers(100000); -SELECT count() >= 1 FROM XXXX WHERE indexHint(t = toDateTime(0)); -1 +SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)); +100000 drop table XXXX; diff --git a/tests/queries/0_stateless/01739_index_hint.sql b/tests/queries/0_stateless/01739_index_hint.sql index 63e7011327c..30dfa43d334 100644 --- a/tests/queries/0_stateless/01739_index_hint.sql +++ b/tests/queries/0_stateless/01739_index_hint.sql @@ -30,6 +30,6 @@ create table XXXX (t Int64, f Float64) Engine=MergeTree order by t settings inde insert into XXXX select number*60, 0 from numbers(100000); -SELECT count() >= 1 FROM XXXX WHERE indexHint(t = toDateTime(0)); +SELECT count() FROM XXXX WHERE indexHint(t = toDateTime(0)); drop table XXXX; From be9c7ed52c5c2f7eba85cc3d37287d2808a0d214 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 25 Jul 2022 19:41:43 +0000 Subject: [PATCH 037/117] Add ReadFromMerge step. --- src/QueryPipeline/QueryPipelineBuilder.cpp | 7 + src/QueryPipeline/QueryPipelineBuilder.h | 3 + src/Storages/StorageMerge.cpp | 162 +++++++++------------ src/Storages/StorageMerge.h | 43 ++++-- 4 files changed, 115 insertions(+), 100 deletions(-) diff --git a/src/QueryPipeline/QueryPipelineBuilder.cpp b/src/QueryPipeline/QueryPipelineBuilder.cpp index 340b85efae9..82907d883bc 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.cpp +++ b/src/QueryPipeline/QueryPipelineBuilder.cpp @@ -23,6 +23,7 @@ #include #include #include "Core/SortDescription.h" +#include #include #include #include @@ -195,6 +196,12 @@ void QueryPipelineBuilder::resize(size_t num_streams, bool force, bool strict) pipe.resize(num_streams, force, strict); } +void QueryPipelineBuilder::narrow(size_t size) +{ + checkInitializedAndNotCompleted(); + narrowPipe(pipe, size); +} + void QueryPipelineBuilder::addTotalsHavingTransform(ProcessorPtr transform) { checkInitializedAndNotCompleted(); diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index 2d9b8028627..a48828be8b9 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -94,6 +94,9 @@ public: /// Changes the number of output ports if needed. Adds ResizeTransform. void resize(size_t num_streams, bool force = false, bool strict = false); + /// Concat some ports to have no more then size outputs. + void narrow(size_t size); + /// Unite several pipelines together. Result pipeline would have common_header structure. /// If collector is used, it will collect only newly-added processors, but not processors from pipelines. static QueryPipelineBuilder unitePipelines( diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 0afc7a0df7e..500e35d062a 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -216,7 +216,7 @@ QueryProcessingStage::Enum StorageMerge::getQueryProcessingStage( } -SelectQueryInfo StorageMerge::getModifiedQueryInfo( +SelectQueryInfo getModifiedQueryInfo( const SelectQueryInfo & query_info, ContextPtr modified_context, const StorageID & current_storage_id, bool is_merge_engine) { SelectQueryInfo modified_query_info = query_info; @@ -248,7 +248,18 @@ void StorageMerge::read( const size_t max_block_size, unsigned num_streams) { - Pipes pipes; + /** Just in case, turn off optimization "transfer to PREWHERE", + * since there is no certainty that it works when one of table is MergeTree and other is not. + */ + auto modified_context = Context::createCopy(context); + modified_context->setSetting("optimize_move_to_prewhere", false); + + query_plan.addInterpreterContext(modified_context); +} + +void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) +{ + std::vector> pipelines; bool has_database_virtual_column = false; bool has_table_virtual_column = false; @@ -257,60 +268,39 @@ void StorageMerge::read( for (const auto & column_name : column_names) { - if (column_name == "_database" && isVirtualColumn(column_name, storage_snapshot->metadata)) + if (column_name == "_database" && storage_merge->isVirtualColumn(column_name, merge_storage_snapshot->metadata)) has_database_virtual_column = true; - else if (column_name == "_table" && isVirtualColumn(column_name, storage_snapshot->metadata)) + else if (column_name == "_table" && storage_merge->isVirtualColumn(column_name, merge_storage_snapshot->metadata)) has_table_virtual_column = true; else real_column_names.push_back(column_name); } - /** Just in case, turn off optimization "transfer to PREWHERE", - * since there is no certainty that it works when one of table is MergeTree and other is not. - */ - auto modified_context = Context::createCopy(local_context); - modified_context->setSetting("optimize_move_to_prewhere", false); - /// What will be result structure depending on query processed stage in source tables? - Block header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, local_context, processed_stage); + Block header = getHeaderForProcessingStage(column_names, merge_storage_snapshot, query_info, context, common_processed_stage); /** First we make list of selected tables to find out its size. * This is necessary to correctly pass the recommended number of threads to each table. */ StorageListWithLocks selected_tables - = getSelectedTables(local_context, query_info.query, has_database_virtual_column, has_table_virtual_column); - - query_plan.addInterpreterContext(modified_context); - - QueryPlanResourceHolder resources; + = storage_merge->getSelectedTables(context, query_info.query, has_database_virtual_column, has_table_virtual_column); if (selected_tables.empty()) { - auto modified_query_info = getModifiedQueryInfo(query_info, modified_context, getStorageID(), false); - /// FIXME: do we support sampling in this case? - auto pipe = createSources( - resources, - {}, - modified_query_info, - processed_stage, - max_block_size, - header, - {}, - {}, - real_column_names, - modified_context, - 0, - has_database_virtual_column, - has_table_virtual_column); + pipeline = InterpreterSelectQuery( + query_info.query, context, + Pipe(std::make_shared(header)), + SelectQueryOptions(common_processed_stage).analyze()).buildQueryPipeline(); - IStorage::readFromPipe(query_plan, std::move(pipe), column_names, storage_snapshot, query_info, local_context, getName()); return; } + QueryPlanResourceHolder resources; + size_t tables_count = selected_tables.size(); Float64 num_streams_multiplier - = std::min(static_cast(tables_count), std::max(1U, static_cast(local_context->getSettingsRef().max_streams_multiplier_for_merge_tables))); - num_streams *= num_streams_multiplier; + = std::min(static_cast(tables_count), std::max(1U, static_cast(context->getSettingsRef().max_streams_multiplier_for_merge_tables))); + size_t num_streams = requested_num_streams * num_streams_multiplier; size_t remaining_streams = num_streams; InputOrderInfoPtr input_sorting_info; @@ -320,7 +310,7 @@ void StorageMerge::read( { auto storage_ptr = std::get<1>(*it); auto storage_metadata_snapshot = storage_ptr->getInMemoryMetadataPtr(); - auto current_info = query_info.order_optimizer->getInputOrder(storage_metadata_snapshot, local_context); + auto current_info = query_info.order_optimizer->getInputOrder(storage_metadata_snapshot, context); if (it == selected_tables.begin()) input_sorting_info = current_info; else if (!current_info || (input_sorting_info && *current_info != *input_sorting_info)) @@ -333,7 +323,7 @@ void StorageMerge::read( query_info.input_order_info = input_sorting_info; } - auto sample_block = getInMemoryMetadataPtr()->getSampleBlock(); + auto sample_block = merge_metadata_for_reading->getSampleBlock(); for (const auto & table : selected_tables) { @@ -351,14 +341,14 @@ void StorageMerge::read( Aliases aliases; auto storage_metadata_snapshot = storage->getInMemoryMetadataPtr(); auto storage_columns = storage_metadata_snapshot->getColumns(); - auto nested_storage_snaphsot = storage->getStorageSnapshot(storage_metadata_snapshot, local_context); + auto nested_storage_snaphsot = storage->getStorageSnapshot(storage_metadata_snapshot, context); - auto modified_query_info = getModifiedQueryInfo(query_info, modified_context, storage->getStorageID(), storage->as()); - auto syntax_result = TreeRewriter(local_context).analyzeSelect( + auto modified_query_info = getModifiedQueryInfo(query_info, context, storage->getStorageID(), storage->as()); + auto syntax_result = TreeRewriter(context).analyzeSelect( modified_query_info.query, TreeRewriterResult({}, storage, nested_storage_snaphsot)); Names column_names_as_aliases; - bool with_aliases = processed_stage == QueryProcessingStage::FetchColumns && !storage_columns.getAliases().empty(); + bool with_aliases = common_processed_stage == QueryProcessingStage::FetchColumns && !storage_columns.getAliases().empty(); if (with_aliases) { ASTPtr required_columns_expr_list = std::make_shared(); @@ -373,11 +363,11 @@ void StorageMerge::read( { column_expr = column_default->expression->clone(); replaceAliasColumnsInQuery(column_expr, storage_metadata_snapshot->getColumns(), - syntax_result->array_join_result_to_source, local_context); + syntax_result->array_join_result_to_source, context); auto column_description = storage_columns.get(column); column_expr = addTypeConversionToAST(std::move(column_expr), column_description.type->getName(), - storage_metadata_snapshot->getColumns().getAll(), local_context); + storage_metadata_snapshot->getColumns().getAll(), context); column_expr = setAlias(column_expr, column); auto type = sample_block.getByName(column).type; @@ -389,54 +379,51 @@ void StorageMerge::read( required_columns_expr_list->children.emplace_back(std::move(column_expr)); } - syntax_result = TreeRewriter(local_context).analyze( - required_columns_expr_list, storage_columns.getAllPhysical(), storage, storage->getStorageSnapshot(storage_metadata_snapshot, local_context)); + syntax_result = TreeRewriter(context).analyze( + required_columns_expr_list, storage_columns.getAllPhysical(), storage, storage->getStorageSnapshot(storage_metadata_snapshot, context)); - auto alias_actions = ExpressionAnalyzer(required_columns_expr_list, syntax_result, local_context).getActionsDAG(true); + auto alias_actions = ExpressionAnalyzer(required_columns_expr_list, syntax_result, context).getActionsDAG(true); column_names_as_aliases = alias_actions->getRequiredColumns().getNames(); if (column_names_as_aliases.empty()) column_names_as_aliases.push_back(ExpressionActions::getSmallestColumn(storage_metadata_snapshot->getColumns().getAllPhysical())); } - auto source_pipe = createSources( - resources, + auto source_pipeline = createSources( nested_storage_snaphsot, modified_query_info, - processed_stage, - max_block_size, + common_processed_stage, + required_max_block_size, header, aliases, table, column_names_as_aliases.empty() ? real_column_names : column_names_as_aliases, - modified_context, + context, current_streams, has_database_virtual_column, has_table_virtual_column); - if (!source_pipe.empty()) + if (!source_pipeline->initialized()) { - query_plan.addStorageHolder(std::get<1>(table)); - query_plan.addTableLock(std::get<2>(table)); + resources.storage_holders.push_back(std::get<1>(table)); + resources.table_locks.push_back(std::get<2>(table)); - pipes.emplace_back(std::move(source_pipe)); + pipelines.emplace_back(std::move(source_pipeline)); } } - auto pipe = Pipe::unitePipes(std::move(pipes)); + pipeline = QueryPipelineBuilder::unitePipelines(std::move(pipelines)); - if (!pipe.empty() && !query_info.input_order_info) + if (!query_info.input_order_info) // It's possible to have many tables read from merge, resize(num_streams) might open too many files at the same time. // Using narrowPipe instead. But in case of reading in order of primary key, we cannot do it, // because narrowPipe doesn't preserve order. - narrowPipe(pipe, num_streams); + pipeline.narrow(num_streams); - IStorage::readFromPipe(query_plan, std::move(pipe), column_names, storage_snapshot, query_info, local_context, getName()); - query_plan.addResources(std::move(resources)); + pipeline.addResources(std::move(resources)); } -Pipe StorageMerge::createSources( - QueryPlanResourceHolder & resources, +QueryPipelineBuilderPtr ReadFromMerge::createSources( const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & modified_query_info, const QueryProcessingStage::Enum & processed_stage, @@ -454,18 +441,14 @@ Pipe StorageMerge::createSources( const auto & [database_name, storage, _, table_name] = storage_with_lock; auto & modified_select = modified_query_info.query->as(); - Pipe pipe; + QueryPipelineBuilderPtr builder; if (!storage) { - auto builder = InterpreterSelectQuery( + return std::make_unique(InterpreterSelectQuery( modified_query_info.query, modified_context, Pipe(std::make_shared(header)), - SelectQueryOptions(processed_stage).analyze()).buildQueryPipeline(); - - pipe = QueryPipelineBuilder::getPipe(std::move(builder), resources); - - return pipe; + SelectQueryOptions(processed_stage).analyze()).buildQueryPipeline()); } if (!modified_select.final() && storage->needRewriteQueryWithFinal(real_column_names)) @@ -497,11 +480,9 @@ Pipe StorageMerge::createSources( if (!plan.isInitialized()) return {}; - auto builder = plan.buildQueryPipeline( + return plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(modified_context), BuildQueryPipelineSettings::fromContext(modified_context)); - - pipe = QueryPipelineBuilder::getPipe(std::move(*builder), resources); } else if (processed_stage > storage_stage) { @@ -515,28 +496,27 @@ Pipe StorageMerge::createSources( InterpreterSelectQuery interpreter{ modified_query_info.query, modified_context, SelectQueryOptions(processed_stage).ignoreProjections()}; - - pipe = QueryPipelineBuilder::getPipe(interpreter.buildQueryPipeline(), resources); + builder = std::make_unique(interpreter.buildQueryPipeline()); /** Materialization is needed, since from distributed storage the constants come materialized. * If you do not do this, different types (Const and non-Const) columns will be produced in different threads, * And this is not allowed, since all code is based on the assumption that in the block stream all types are the same. */ - pipe.addSimpleTransform([](const Block & stream_header) { return std::make_shared(stream_header); }); + builder->addSimpleTransform([](const Block & stream_header) { return std::make_shared(stream_header); }); } - if (!pipe.empty()) + if (builder->initialized()) { - if (concat_streams && pipe.numOutputPorts() > 1) + if (concat_streams && builder->getNumStreams() > 1) { // It's possible to have many tables read from merge, resize(1) might open too many files at the same time. // Using concat instead. - pipe.addTransform(std::make_shared(pipe.getHeader(), pipe.numOutputPorts())); + builder->addTransform(std::make_shared(builder->getHeader(), builder->getNumStreams())); } /// Add virtual columns if we don't already have them. - Block pipe_header = pipe.getHeader(); + Block pipe_header = builder->getHeader(); if (has_database_virtual_column && !pipe_header.has("_database")) { @@ -550,7 +530,7 @@ Pipe StorageMerge::createSources( std::move(adding_column_dag), ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); - pipe.addSimpleTransform([&](const Block & stream_header) + builder->addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, adding_column_actions); }); @@ -568,7 +548,7 @@ Pipe StorageMerge::createSources( std::move(adding_column_dag), ExpressionActionsSettings::fromContext(modified_context, CompileExpressions::yes)); - pipe.addSimpleTransform([&](const Block & stream_header) + builder->addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, adding_column_actions); }); @@ -576,10 +556,10 @@ Pipe StorageMerge::createSources( /// Subordinary tables could have different but convertible types, like numeric types of different width. /// We must return streams with structure equals to structure of Merge table. - convertingSourceStream(header, storage_snapshot->metadata, aliases, modified_context, modified_query_info.query, pipe, processed_stage); + convertingSourceStream(header, storage_snapshot->metadata, aliases, modified_context, modified_query_info.query, *builder, processed_stage); } - return pipe; + return builder; } StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( @@ -747,19 +727,19 @@ void StorageMerge::alter( setInMemoryMetadata(storage_metadata); } -void StorageMerge::convertingSourceStream( +void ReadFromMerge::convertingSourceStream( const Block & header, const StorageMetadataPtr & metadata_snapshot, const Aliases & aliases, ContextPtr local_context, ASTPtr & query, - Pipe & pipe, + QueryPipelineBuilder & builder, QueryProcessingStage::Enum processed_stage) { - Block before_block_header = pipe.getHeader(); + Block before_block_header = builder.getHeader(); auto storage_sample_block = metadata_snapshot->getSampleBlock(); - auto pipe_columns = pipe.getHeader().getNamesAndTypesList(); + auto pipe_columns = builder.getHeader().getNamesAndTypesList(); for (const auto & alias : aliases) { @@ -772,21 +752,21 @@ void StorageMerge::convertingSourceStream( auto actions_dag = expression_analyzer.getActionsDAG(true, false); auto actions = std::make_shared(actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - pipe.addSimpleTransform([&](const Block & stream_header) + builder.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); } { - auto convert_actions_dag = ActionsDAG::makeConvertingActions(pipe.getHeader().getColumnsWithTypeAndName(), + auto convert_actions_dag = ActionsDAG::makeConvertingActions(builder.getHeader().getColumnsWithTypeAndName(), header.getColumnsWithTypeAndName(), ActionsDAG::MatchColumnsMode::Name); auto actions = std::make_shared( convert_actions_dag, ExpressionActionsSettings::fromContext(local_context, CompileExpressions::yes)); - pipe.addSimpleTransform([&](const Block & stream_header) + builder.addSimpleTransform([&](const Block & stream_header) { return std::make_shared(stream_header, actions); }); @@ -809,7 +789,7 @@ void StorageMerge::convertingSourceStream( if (!header_column.type->equals(*before_column.type.get())) { NamesAndTypesList source_columns = metadata_snapshot->getSampleBlock().getNamesAndTypesList(); - auto virtual_column = *getVirtuals().tryGetByName("_table"); + auto virtual_column = *storage_merge->getVirtuals().tryGetByName("_table"); source_columns.emplace_back(NameAndTypePair{virtual_column.name, virtual_column.type}); auto syntax_result = TreeRewriter(local_context).analyze(where_expression, source_columns); ExpressionActionsPtr actions = ExpressionAnalyzer{where_expression, syntax_result, local_context}.getActions(false, false); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index f6dae239e79..440aa01890d 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -1,7 +1,9 @@ #pragma once #include +#include #include +#include namespace DB @@ -105,7 +107,36 @@ private: NamesAndTypesList getVirtuals() const override; ColumnSizeByName getColumnSizes() const override; -protected: + ColumnsDescription getColumnsDescriptionFromSourceTables() const; + + friend class ReadFromMerge; +}; + +class ReadFromMerge final : public ISourceStep +{ +public: + static constexpr auto name = "ReadFromMerge"; + String getName() const override { return name; } + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + + using StorageWithLockAndName = std::tuple; + using StorageListWithLocks = std::list; + using DatabaseTablesIterators = std::vector; + +private: + const size_t requested_num_streams; + const size_t required_max_block_size; + + Names column_names; + std::shared_ptr storage_merge; + QueryProcessingStage::Enum common_processed_stage; + + SelectQueryInfo query_info; + StorageSnapshotPtr merge_storage_snapshot; + StorageMetadataPtr merge_metadata_for_reading; + ContextMutablePtr context; + struct AliasData { String name; @@ -115,8 +146,7 @@ protected: using Aliases = std::vector; - Pipe createSources( - QueryPlanResourceHolder & resources, + QueryPipelineBuilderPtr createSources( const StorageSnapshotPtr & storage_snapshot, SelectQueryInfo & query_info, const QueryProcessingStage::Enum & processed_stage, @@ -134,12 +164,7 @@ protected: void convertingSourceStream( const Block & header, const StorageMetadataPtr & metadata_snapshot, const Aliases & aliases, ContextPtr context, ASTPtr & query, - Pipe & pipe, QueryProcessingStage::Enum processed_stage); - - static SelectQueryInfo getModifiedQueryInfo( - const SelectQueryInfo & query_info, ContextPtr modified_context, const StorageID & current_storage_id, bool is_merge_engine); - - ColumnsDescription getColumnsDescriptionFromSourceTables() const; + QueryPipelineBuilder & builder, QueryProcessingStage::Enum processed_stage); }; } From c5110bb1640ea22bae24c481d721c70c07f92ee8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 26 Jul 2022 14:43:05 +0000 Subject: [PATCH 038/117] Add a query plan step for StorageMerge --- src/Storages/StorageMerge.cpp | 69 ++++++++++++++++++++++++++++------- src/Storages/StorageMerge.h | 21 ++++++++--- 2 files changed, 71 insertions(+), 19 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 500e35d062a..286499a4e78 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -35,6 +36,8 @@ #include +using std::operator""sv; + namespace DB { @@ -251,10 +254,49 @@ void StorageMerge::read( /** Just in case, turn off optimization "transfer to PREWHERE", * since there is no certainty that it works when one of table is MergeTree and other is not. */ - auto modified_context = Context::createCopy(context); + auto modified_context = Context::createCopy(local_context); modified_context->setSetting("optimize_move_to_prewhere", false); query_plan.addInterpreterContext(modified_context); + + /// What will be result structure depending on query processed stage in source tables? + Block common_header = getHeaderForProcessingStage(column_names, storage_snapshot, query_info, local_context, processed_stage); + + auto step = std::make_unique( + common_header, + column_names, + max_block_size, + num_streams, + shared_from_this(), + storage_snapshot, + query_info, + std::move(modified_context), + processed_stage); + + query_plan.addStep(std::move(step)); +} + +ReadFromMerge::ReadFromMerge( + Block common_header_, + Names column_names_, + size_t max_block_size, + size_t num_streams, + StoragePtr storage, + StorageSnapshotPtr storage_snapshot, + const SelectQueryInfo & query_info_, + ContextMutablePtr context_, + QueryProcessingStage::Enum processed_stage) + : ISourceStep(DataStream{.header = common_header_ /*addVirtualColumns(common_header_, column_names_, *storage, *storage_snapshot)*/}) + , required_max_block_size(max_block_size) + , requested_num_streams(num_streams) + , common_header(std::move(common_header_)) + , column_names(std::move(column_names_)) + , storage_merge(std::move(storage)) + , merge_storage_snapshot(std::move(storage_snapshot)) + , query_info(query_info_) + , context(std::move(context_)) + , common_processed_stage(processed_stage) +{ } void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) @@ -276,22 +318,15 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu real_column_names.push_back(column_name); } - /// What will be result structure depending on query processed stage in source tables? - Block header = getHeaderForProcessingStage(column_names, merge_storage_snapshot, query_info, context, common_processed_stage); - /** First we make list of selected tables to find out its size. * This is necessary to correctly pass the recommended number of threads to each table. */ StorageListWithLocks selected_tables - = storage_merge->getSelectedTables(context, query_info.query, has_database_virtual_column, has_table_virtual_column); + = storage_merge->as().getSelectedTables(context, query_info.query, has_database_virtual_column, has_table_virtual_column); if (selected_tables.empty()) { - pipeline = InterpreterSelectQuery( - query_info.query, context, - Pipe(std::make_shared(header)), - SelectQueryOptions(common_processed_stage).analyze()).buildQueryPipeline(); - + pipeline.init(Pipe(std::make_shared(output_stream->header))); return; } @@ -323,7 +358,7 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu query_info.input_order_info = input_sorting_info; } - auto sample_block = merge_metadata_for_reading->getSampleBlock(); + auto sample_block = merge_storage_snapshot->getMetadataForQuery()->getSampleBlock(); for (const auto & table : selected_tables) { @@ -394,7 +429,7 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu modified_query_info, common_processed_stage, required_max_block_size, - header, + common_header, aliases, table, column_names_as_aliases.empty() ? real_column_names : column_names_as_aliases, @@ -403,7 +438,7 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu has_database_virtual_column, has_table_virtual_column); - if (!source_pipeline->initialized()) + if (source_pipeline->initialized()) { resources.storage_holders.push_back(std::get<1>(table)); resources.table_locks.push_back(std::get<2>(table)); @@ -412,6 +447,12 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu } } + if (pipelines.empty()) + { + pipeline.init(Pipe(std::make_shared(output_stream->header))); + return; + } + pipeline = QueryPipelineBuilder::unitePipelines(std::move(pipelines)); if (!query_info.input_order_info) @@ -480,7 +521,7 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( if (!plan.isInitialized()) return {}; - return plan.buildQueryPipeline( + builder = plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(modified_context), BuildQueryPipelineSettings::fromContext(modified_context)); } diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 440aa01890d..243c20a0406 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -118,6 +118,17 @@ public: static constexpr auto name = "ReadFromMerge"; String getName() const override { return name; } + ReadFromMerge( + Block common_header_, + Names column_names_, + size_t max_block_size, + size_t num_streams, + StoragePtr storage, + StorageSnapshotPtr storage_snapshot, + const SelectQueryInfo & query_info_, + ContextMutablePtr context_, + QueryProcessingStage::Enum processed_stage); + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; using StorageWithLockAndName = std::tuple; @@ -125,17 +136,17 @@ public: using DatabaseTablesIterators = std::vector; private: - const size_t requested_num_streams; const size_t required_max_block_size; + const size_t requested_num_streams; + const Block common_header; Names column_names; - std::shared_ptr storage_merge; - QueryProcessingStage::Enum common_processed_stage; + StoragePtr storage_merge; + StorageSnapshotPtr merge_storage_snapshot; SelectQueryInfo query_info; - StorageSnapshotPtr merge_storage_snapshot; - StorageMetadataPtr merge_metadata_for_reading; ContextMutablePtr context; + QueryProcessingStage::Enum common_processed_stage; struct AliasData { From d7cc3831ea94e84386bf115e21f6cd780b978569 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 26 Jul 2022 15:01:39 +0000 Subject: [PATCH 039/117] Push predicate over StorageMerge. --- .../Optimizations/optimizePrimaryKeyCondition.cpp | 3 +++ src/Processors/QueryPlan/QueryPlan.h | 2 ++ src/Storages/StorageMerge.cpp | 4 ++++ src/Storages/StorageMerge.h | 9 +++++++++ .../queries/0_stateless/00160_merge_and_index_in_in.sql | 1 - .../01783_merge_engine_join_key_condition.sql | 1 - 6 files changed, 18 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index f9abe662006..e559c23bbaf 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include @@ -37,6 +38,8 @@ void optimizePrimaryKeyCondition(QueryPlan::Node & root) if (auto * read_from_merge_tree = typeid_cast(child->step.get())) read_from_merge_tree->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); + if (auto * read_from_merge = typeid_cast(child->step.get())) + read_from_merge->addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); } stack.pop(); diff --git a/src/Processors/QueryPlan/QueryPlan.h b/src/Processors/QueryPlan/QueryPlan.h index ce12ce7beda..e9b08c20280 100644 --- a/src/Processors/QueryPlan/QueryPlan.h +++ b/src/Processors/QueryPlan/QueryPlan.h @@ -103,6 +103,8 @@ public: std::vector children = {}; }; + const Node * getRootNode() const { return root; } + using Nodes = std::list; private: diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 286499a4e78..5f7cd776d11 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -23,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -521,6 +522,9 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( if (!plan.isInitialized()) return {}; + if (auto * read_from_merge_tree = typeid_cast(plan.getRootNode()->step.get())) + read_from_merge_tree->addFilter(added_filter, added_filter_column_name); + builder = plan.buildQueryPipeline( QueryPlanOptimizationSettings::fromContext(modified_context), BuildQueryPipelineSettings::fromContext(modified_context)); diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 243c20a0406..74f5628bb24 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -131,6 +131,12 @@ public: void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override; + void addFilter(ActionsDAGPtr expression, std::string column_name) + { + added_filter = std::move(expression); + added_filter_column_name = std::move(column_name); + } + using StorageWithLockAndName = std::tuple; using StorageListWithLocks = std::list; using DatabaseTablesIterators = std::vector; @@ -148,6 +154,9 @@ private: ContextMutablePtr context; QueryProcessingStage::Enum common_processed_stage; + ActionsDAGPtr added_filter; + std::string added_filter_column_name; + struct AliasData { String name; diff --git a/tests/queries/0_stateless/00160_merge_and_index_in_in.sql b/tests/queries/0_stateless/00160_merge_and_index_in_in.sql index 3ed829c4d59..bdab3f7640d 100644 --- a/tests/queries/0_stateless/00160_merge_and_index_in_in.sql +++ b/tests/queries/0_stateless/00160_merge_and_index_in_in.sql @@ -9,7 +9,6 @@ SET max_block_size = 1000000; INSERT INTO mt_00160 (x) SELECT number AS x FROM system.numbers LIMIT 100000; SELECT *, b FROM mt_00160 WHERE x IN (12345, 67890) AND NOT ignore(blockSize() < 10 AS b) ORDER BY x; -SET query_plan_optimize_primary_key = 0; -- Need separate query plan step for merge SELECT *, b FROM merge_00160 WHERE x IN (12345, 67890) AND NOT ignore(blockSize() < 10 AS b) ORDER BY x; DROP TABLE merge_00160; diff --git a/tests/queries/0_stateless/01783_merge_engine_join_key_condition.sql b/tests/queries/0_stateless/01783_merge_engine_join_key_condition.sql index 606597850ab..372c1bd3572 100644 --- a/tests/queries/0_stateless/01783_merge_engine_join_key_condition.sql +++ b/tests/queries/0_stateless/01783_merge_engine_join_key_condition.sql @@ -11,7 +11,6 @@ CREATE TABLE t2 (Id Int32, Val Int32, X Int32) Engine=Memory; INSERT INTO t2 values (4, 3, 4); SET force_primary_key = 1, force_index_by_date=1; -SET query_plan_optimize_primary_key = 0; SELECT * FROM foo_merge WHERE Val = 3 AND Id = 3; SELECT count(), X FROM foo_merge JOIN t2 USING Val WHERE Val = 3 AND Id = 3 AND t2.X == 4 GROUP BY X; From 9321ca34cf317193cc872403e04c9ebfc9dc3e32 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 26 Jul 2022 17:31:56 +0000 Subject: [PATCH 040/117] try to fix Nested --- .../Serializations/ISerialization.cpp | 6 +++++- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 7 +------ src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + src/Storages/MergeTree/IMergeTreeReader.h | 6 ++---- .../MergeTree/MergeTreeDataPartWide.cpp | 4 ++-- src/Storages/MergeTree/MergeTreeReaderWide.cpp | 18 ++++++++++++++++-- src/Storages/MergeTree/MergeTreeReaderWide.h | 2 ++ 7 files changed, 29 insertions(+), 15 deletions(-) diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index 7df4a956c1a..d4acb877e87 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -224,7 +224,11 @@ String ISerialization::getSubcolumnNameForStream(const SubstreamPath & path, siz void ISerialization::addToSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path, ColumnPtr column) { - if (cache && !path.empty()) + if (!cache || path.empty()) + return; + + auto subcolumn_name = getSubcolumnNameForStream(path); + if (!subcolumn_name.empty()) cache->emplace(getSubcolumnNameForStream(path), column); } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 45e5d9d92de..d4d3844de6c 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -448,12 +448,7 @@ void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns) for (const auto & column : columns) column_name_to_position.emplace(column.name, pos++); - /// For wide parts convert plain arrays to Nested for - /// more convinient managing of shared offsets column. - if (part_type == Type::Wide) - columns_description = ColumnsDescription(Nested::collect(columns)); - else - columns_description = ColumnsDescription(columns); + columns_description = ColumnsDescription(columns); } NameAndTypePair IMergeTreeDataPart::getColumn(const String & column_name) const diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index b6e8b75147d..bf47ced2a77 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -136,6 +136,7 @@ public: void setColumns(const NamesAndTypesList & new_columns); const NamesAndTypesList & getColumns() const { return columns; } + const ColumnsDescription & getColumnsDescription() const { return columns_description; } NameAndTypePair getColumn(const String & name) const; std::optional tryGetColumn(const String & column_name) const; diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 78cc96ccd27..f844946fe22 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -55,16 +55,14 @@ public: const NamesAndTypesList & getColumns() const { return columns; } size_t numColumnsInResult() const { return columns.size(); } - size_t getFirstMarkToRead() const - { - return all_mark_ranges.front().begin; - } + size_t getFirstMarkToRead() const { return all_mark_ranges.front().begin; } MergeTreeData::DataPartPtr data_part; protected: /// Returns actual column name in part, which can differ from table metadata. String getColumnNameInPart(const NameAndTypePair & required_column) const; + /// Returns actual column name and type in part, which can differ from table metadata. NameAndTypePair getColumnInPart(const NameAndTypePair & required_column) const; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index 3e5576b6bdf..b70d67de3b4 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -49,7 +49,7 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartWide::getReader( { auto ptr = std::static_pointer_cast(shared_from_this()); return std::make_unique( - ptr, Nested::convertToSubcolumns(columns_to_read), + ptr, columns_to_read, metadata_snapshot, uncompressed_cache, mark_cache, mark_ranges, reader_settings, avg_value_size_hints, profile_callback); @@ -66,7 +66,7 @@ IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartWide::getWriter( { return std::make_unique( shared_from_this(), data_part_storage_builder, - Nested::convertToSubcolumns(columns_list), metadata_snapshot, indices_to_recalc, + columns_list, metadata_snapshot, indices_to_recalc, index_granularity_info.marks_file_extension, default_codec_, writer_settings, computed_index_granularity); } diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index e098e76d4ee..aed9a906a5c 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -60,6 +61,19 @@ MergeTreeReaderWide::MergeTreeReaderWide( } } +String MergeTreeReaderWide::getNameForSubstreamCache(const NameAndTypePair & column) const +{ + if (!column.isSubcolumn() && isArray(column.type)) + { + auto split = Nested::splitName(column.name); + const auto & part_columns = data_part->getColumnsDescription(); + + if (!split.second.empty() && part_columns.hasNested(split.first)) + return split.first; + } + + return column.getNameInStorage(); +} size_t MergeTreeReaderWide::readRows( size_t from_mark, size_t current_task_last_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) @@ -86,7 +100,7 @@ size_t MergeTreeReaderWide::readRows( auto column_from_part = getColumnInPart(*name_and_type); try { - auto & cache = caches[column_from_part.getNameInStorage()]; + auto & cache = caches[getNameForSubstreamCache(column_from_part)]; prefetch(column_from_part, from_mark, continue_reading, current_task_last_mark, cache, prefetched_streams); } catch (Exception & e) @@ -117,7 +131,7 @@ size_t MergeTreeReaderWide::readRows( try { size_t column_size_before_reading = column->size(); - auto & cache = caches[column_from_part.getNameInStorage()]; + auto & cache = caches[getNameForSubstreamCache(column_from_part)]; readData( column_from_part, column, from_mark, continue_reading, current_task_last_mark, diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index 7bb1ccfd173..e382f3f1dde 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -38,6 +38,8 @@ public: private: FileStreams streams; + String getNameForSubstreamCache(const NameAndTypePair & column) const; + void addStreams(const NameAndTypePair & name_and_type, const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type); From 1adb021df6c45507a9fba68adbd2024d7eb9b63f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 26 Jul 2022 18:42:27 +0000 Subject: [PATCH 041/117] Fixing tests. --- src/Storages/StorageMerge.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 5f7cd776d11..69666e2de35 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -439,7 +439,7 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu has_database_virtual_column, has_table_virtual_column); - if (source_pipeline->initialized()) + if (source_pipeline && source_pipeline->initialized()) { resources.storage_holders.push_back(std::get<1>(table)); resources.table_locks.push_back(std::get<2>(table)); From f9a8a4cd3291715026141a5fa157b95309f30360 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 27 Jul 2022 12:00:55 +0000 Subject: [PATCH 042/117] Fixing read-in-order for Merge --- src/Storages/StorageMerge.cpp | 89 ++++++++++++++++++++++------------- src/Storages/StorageMerge.h | 16 ++++--- 2 files changed, 65 insertions(+), 40 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 69666e2de35..3fc7ca36569 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -258,6 +258,44 @@ void StorageMerge::read( auto modified_context = Context::createCopy(local_context); modified_context->setSetting("optimize_move_to_prewhere", false); + bool has_database_virtual_column = false; + bool has_table_virtual_column = false; + Names real_column_names; + real_column_names.reserve(column_names.size()); + + for (const auto & column_name : column_names) + { + if (column_name == "_database" && isVirtualColumn(column_name, storage_snapshot->metadata)) + has_database_virtual_column = true; + else if (column_name == "_table" && isVirtualColumn(column_name, storage_snapshot->metadata)) + has_table_virtual_column = true; + else + real_column_names.push_back(column_name); + } + + StorageListWithLocks selected_tables + = getSelectedTables(modified_context, query_info.query, has_database_virtual_column, has_table_virtual_column); + + InputOrderInfoPtr input_sorting_info; + if (query_info.order_optimizer) + { + for (auto it = selected_tables.begin(); it != selected_tables.end(); ++it) + { + auto storage_ptr = std::get<1>(*it); + auto storage_metadata_snapshot = storage_ptr->getInMemoryMetadataPtr(); + auto current_info = query_info.order_optimizer->getInputOrder(storage_metadata_snapshot, modified_context); + if (it == selected_tables.begin()) + input_sorting_info = current_info; + else if (!current_info || (input_sorting_info && *current_info != *input_sorting_info)) + input_sorting_info.reset(); + + if (!input_sorting_info) + break; + } + + query_info.input_order_info = input_sorting_info; + } + query_plan.addInterpreterContext(modified_context); /// What will be result structure depending on query processed stage in source tables? @@ -265,7 +303,10 @@ void StorageMerge::read( auto step = std::make_unique( common_header, - column_names, + std::move(selected_tables), + real_column_names, + has_database_virtual_column, + has_table_virtual_column, max_block_size, num_streams, shared_from_this(), @@ -279,7 +320,10 @@ void StorageMerge::read( ReadFromMerge::ReadFromMerge( Block common_header_, + StorageListWithLocks selected_tables_, Names column_names_, + bool has_database_virtual_column_, + bool has_table_virtual_column_, size_t max_block_size, size_t num_streams, StoragePtr storage, @@ -287,11 +331,14 @@ ReadFromMerge::ReadFromMerge( const SelectQueryInfo & query_info_, ContextMutablePtr context_, QueryProcessingStage::Enum processed_stage) - : ISourceStep(DataStream{.header = common_header_ /*addVirtualColumns(common_header_, column_names_, *storage, *storage_snapshot)*/}) + : ISourceStep(DataStream{.header = common_header_}) , required_max_block_size(max_block_size) , requested_num_streams(num_streams) , common_header(std::move(common_header_)) + , selected_tables(std::move(selected_tables_)) , column_names(std::move(column_names_)) + , has_database_virtual_column(has_database_virtual_column_) + , has_table_virtual_column(has_table_virtual_column_) , storage_merge(std::move(storage)) , merge_storage_snapshot(std::move(storage_snapshot)) , query_info(query_info_) @@ -302,37 +349,12 @@ ReadFromMerge::ReadFromMerge( void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { - std::vector> pipelines; - - bool has_database_virtual_column = false; - bool has_table_virtual_column = false; - Names real_column_names; - real_column_names.reserve(column_names.size()); - - for (const auto & column_name : column_names) - { - if (column_name == "_database" && storage_merge->isVirtualColumn(column_name, merge_storage_snapshot->metadata)) - has_database_virtual_column = true; - else if (column_name == "_table" && storage_merge->isVirtualColumn(column_name, merge_storage_snapshot->metadata)) - has_table_virtual_column = true; - else - real_column_names.push_back(column_name); - } - - /** First we make list of selected tables to find out its size. - * This is necessary to correctly pass the recommended number of threads to each table. - */ - StorageListWithLocks selected_tables - = storage_merge->as().getSelectedTables(context, query_info.query, has_database_virtual_column, has_table_virtual_column); - if (selected_tables.empty()) { pipeline.init(Pipe(std::make_shared(output_stream->header))); return; } - QueryPlanResourceHolder resources; - size_t tables_count = selected_tables.size(); Float64 num_streams_multiplier = std::min(static_cast(tables_count), std::max(1U, static_cast(context->getSettingsRef().max_streams_multiplier_for_merge_tables))); @@ -361,6 +383,9 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu auto sample_block = merge_storage_snapshot->getMetadataForQuery()->getSampleBlock(); + std::vector> pipelines; + QueryPlanResourceHolder resources; + for (const auto & table : selected_tables) { size_t current_need_streams = tables_count >= num_streams ? 1 : (num_streams / tables_count); @@ -390,7 +415,7 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu ASTPtr required_columns_expr_list = std::make_shared(); ASTPtr column_expr; - for (const auto & column : real_column_names) + for (const auto & column : column_names) { const auto column_default = storage_columns.getDefault(column); bool is_alias = column_default && column_default->kind == ColumnDefaultKind::Alias; @@ -433,11 +458,9 @@ void ReadFromMerge::initializePipeline(QueryPipelineBuilder & pipeline, const Bu common_header, aliases, table, - column_names_as_aliases.empty() ? real_column_names : column_names_as_aliases, + column_names_as_aliases.empty() ? column_names : column_names_as_aliases, context, - current_streams, - has_database_virtual_column, - has_table_virtual_column); + current_streams); if (source_pipeline && source_pipeline->initialized()) { @@ -476,8 +499,6 @@ QueryPipelineBuilderPtr ReadFromMerge::createSources( Names & real_column_names, ContextMutablePtr modified_context, size_t streams_num, - bool has_database_virtual_column, - bool has_table_virtual_column, bool concat_streams) { const auto & [database_name, storage, _, table_name] = storage_with_lock; diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index 74f5628bb24..d2f94ac6b88 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -118,9 +118,16 @@ public: static constexpr auto name = "ReadFromMerge"; String getName() const override { return name; } + using StorageWithLockAndName = std::tuple; + using StorageListWithLocks = std::list; + using DatabaseTablesIterators = std::vector; + ReadFromMerge( Block common_header_, + StorageListWithLocks selected_tables_, Names column_names_, + bool has_database_virtual_column_, + bool has_table_virtual_column_, size_t max_block_size, size_t num_streams, StoragePtr storage, @@ -137,16 +144,15 @@ public: added_filter_column_name = std::move(column_name); } - using StorageWithLockAndName = std::tuple; - using StorageListWithLocks = std::list; - using DatabaseTablesIterators = std::vector; - private: const size_t required_max_block_size; const size_t requested_num_streams; const Block common_header; + StorageListWithLocks selected_tables; Names column_names; + bool has_database_virtual_column; + bool has_table_virtual_column; StoragePtr storage_merge; StorageSnapshotPtr merge_storage_snapshot; @@ -177,8 +183,6 @@ private: Names & real_column_names, ContextMutablePtr modified_context, size_t streams_num, - bool has_database_virtual_column, - bool has_table_virtual_column, bool concat_streams = false); void convertingSourceStream( From 4cf03538921f002d4dcc137cbfda11bdba4d2bab Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 27 Jul 2022 14:05:16 +0000 Subject: [PATCH 043/117] try to fix Nested --- .../Serializations/ISerialization.cpp | 9 +- src/Storages/MergeTree/DataPartsExchange.cpp | 4 +- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 91 ++++++++++--------- src/Storages/MergeTree/IMergeTreeDataPart.h | 10 +- src/Storages/MergeTree/IMergeTreeReader.cpp | 55 +++++++---- src/Storages/MergeTree/IMergeTreeReader.h | 18 +++- .../MergeTree/IMergedBlockOutputStream.cpp | 14 +-- src/Storages/MergeTree/MergeTask.cpp | 3 +- .../MergeTree/MergeTreeDataPartInMemory.cpp | 4 +- .../MergeTree/MergeTreeDataPartWide.cpp | 18 ++-- .../MergeTreeDataPartWriterCompact.cpp | 4 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 18 ++-- .../MergeTree/MergeTreeDataWriter.cpp | 6 +- .../MergeTree/MergeTreeReaderCompact.cpp | 51 ++++------- .../MergeTree/MergeTreeReaderInMemory.cpp | 21 ++--- .../MergeTree/MergeTreeReaderWide.cpp | 68 +++++--------- src/Storages/MergeTree/MergeTreeReaderWide.h | 12 ++- .../MergeTree/MergeTreeWriteAheadLog.cpp | 2 +- .../MergeTree/MergedBlockOutputStream.cpp | 3 +- .../MergedColumnOnlyOutputStream.cpp | 3 +- src/Storages/MergeTree/MutateTask.cpp | 30 +++--- src/Storages/StorageReplicatedMergeTree.cpp | 2 +- .../System/StorageSystemPartsColumns.cpp | 2 +- 23 files changed, 216 insertions(+), 232 deletions(-) diff --git a/src/DataTypes/Serializations/ISerialization.cpp b/src/DataTypes/Serializations/ISerialization.cpp index d4acb877e87..e6e6fdba5dc 100644 --- a/src/DataTypes/Serializations/ISerialization.cpp +++ b/src/DataTypes/Serializations/ISerialization.cpp @@ -227,9 +227,7 @@ void ISerialization::addToSubstreamsCache(SubstreamsCache * cache, const Substre if (!cache || path.empty()) return; - auto subcolumn_name = getSubcolumnNameForStream(path); - if (!subcolumn_name.empty()) - cache->emplace(getSubcolumnNameForStream(path), column); + cache->emplace(getSubcolumnNameForStream(path), column); } ColumnPtr ISerialization::getFromSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path) @@ -238,10 +236,7 @@ ColumnPtr ISerialization::getFromSubstreamsCache(SubstreamsCache * cache, const return nullptr; auto it = cache->find(getSubcolumnNameForStream(path)); - if (it == cache->end()) - return nullptr; - - return it->second; + return it == cache->end() ? nullptr : it->second; } bool ISerialization::isSpecialCompressionAllowed(const SubstreamPath & path) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index 3609a65bc71..4bf44689ba2 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -646,7 +646,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( std::make_shared(data, projection_name, new_part_info, projection_part_storage, new_data_part.get()); new_projection_part->is_temp = false; - new_projection_part->setColumns(block.getNamesAndTypesList()); + new_projection_part->setColumns(block.getNamesAndTypesList(), {}); MergeTreePartition partition{}; new_projection_part->partition = std::move(partition); new_projection_part->minmax_idx = std::make_shared(); @@ -676,7 +676,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( new_data_part->uuid = part_uuid; new_data_part->is_temp = true; - new_data_part->setColumns(block.getNamesAndTypesList()); + new_data_part->setColumns(block.getNamesAndTypesList(), {}); new_data_part->minmax_idx->update(block, data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); new_data_part->partition.create(metadata_snapshot, block, 0, context); diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index d4d3844de6c..30ec9bbcdd4 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -64,6 +64,7 @@ namespace ErrorCodes extern const int BAD_SIZE_OF_FILE_IN_DATA_PART; extern const int BAD_TTL_FILE; extern const int NOT_IMPLEMENTED; + extern const int NO_SUCH_COLUMN_IN_TABLE; } void IMergeTreeDataPart::MinMaxIndex::load(const MergeTreeData & data, const PartMetadataManagerPtr & manager) @@ -437,17 +438,33 @@ std::pair IMergeTreeDataPart::getMinMaxTime() const } -void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns) +void IMergeTreeDataPart::setColumns(const NamesAndTypesList & new_columns, const SerializationInfoByName & new_infos) { columns = new_columns; + serialization_infos = new_infos; column_name_to_position.clear(); column_name_to_position.reserve(new_columns.size()); size_t pos = 0; - for (const auto & column : columns) column_name_to_position.emplace(column.name, pos++); + for (const auto & column : columns) + { + auto it = serialization_infos.find(column.name); + auto serialization = it == serialization_infos.end() + ? IDataType::getSerialization(column) + : IDataType::getSerialization(column, *it->second); + + serializations.emplace(column.name, serialization); + + IDataType::forEachSubcolumn([&](const auto &, const auto & subname, const auto & subdata) + { + auto full_name = Nested::concatenateName(column.name, subname); + serializations.emplace(full_name, subdata.serialization); + }, {serialization, nullptr, nullptr, nullptr}); + } + columns_description = ColumnsDescription(columns); } @@ -461,22 +478,20 @@ std::optional IMergeTreeDataPart::tryGetColumn(const String & c return columns_description.tryGetColumnOrSubcolumn(GetColumnsOptions::AllPhysical, column_name); } -void IMergeTreeDataPart::setSerializationInfos(const SerializationInfoByName & new_infos) +SerializationPtr IMergeTreeDataPart::getSerialization(const String & column_name) const { - serialization_infos = new_infos; + auto serialization = tryGetSerialization(column_name); + if (!serialization) + throw Exception(ErrorCodes::NO_SUCH_COLUMN_IN_TABLE, + "There is no column or subcolumn {} in part {}", column_name, name); + + return serialization; } -SerializationPtr IMergeTreeDataPart::getSerialization(const NameAndTypePair & column) const +SerializationPtr IMergeTreeDataPart::tryGetSerialization(const String & column_name) const { - auto column_in_part = tryGetColumn(column.name); - if (!column_in_part) - return IDataType::getSerialization(column); - - auto it = serialization_infos.find(column_in_part->getNameInStorage()); - if (it == serialization_infos.end()) - return IDataType::getSerialization(*column_in_part); - - return IDataType::getSerialization(*column_in_part, *it->second); + auto it = serializations.find(column_name); + return it == serializations.end() ? nullptr : it->second; } void IMergeTreeDataPart::removeIfNeeded() @@ -585,36 +600,23 @@ size_t IMergeTreeDataPart::getFileSizeOrZero(const String & file_name) const String IMergeTreeDataPart::getColumnNameWithMinimumCompressedSize(bool with_subcolumns) const { - auto find_column_with_minimum_size = [&](const auto & columns_list) - { - std::optional minimum_size_column; - UInt64 minimum_size = std::numeric_limits::max(); - - for (const auto & column : columns_list) - { - if (!hasColumnFiles(column)) - continue; - - const auto size = getColumnSize(column.name).data_compressed; - if (size < minimum_size) - { - minimum_size = size; - minimum_size_column = column.name; - } - } - - return minimum_size_column; - }; + auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withSubcolumns(with_subcolumns); + auto columns_list = columns_description.get(options); std::optional minimum_size_column; - if (with_subcolumns) + UInt64 minimum_size = std::numeric_limits::max(); + + for (const auto & column : columns_list) { - auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withSubcolumns(); - minimum_size_column = find_column_with_minimum_size(columns_description.get(options)); - } - else - { - minimum_size_column = find_column_with_minimum_size(columns); + if (!hasColumnFiles(column)) + continue; + + const auto size = getColumnSize(column.name).data_compressed; + if (size < minimum_size) + { + minimum_size = size; + minimum_size_column = column.name; + } } if (!minimum_size_column) @@ -868,7 +870,7 @@ CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const if (column_size.data_compressed != 0 && !storage_columns.hasCompressionCodec(part_column.name)) { String path_to_data_file; - getSerialization(part_column)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + getSerialization(part_column.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { if (path_to_data_file.empty()) { @@ -1066,7 +1068,7 @@ void IMergeTreeDataPart::loadRowsCount() for (const NameAndTypePair & column : columns) { - ColumnPtr column_col = column.type->createColumn(*getSerialization(column)); + ColumnPtr column_col = column.type->createColumn(*getSerialization(column.name)); if (!column_col->isFixedAndContiguous() || column_col->lowCardinality()) continue; @@ -1208,8 +1210,7 @@ void IMergeTreeDataPart::loadColumns(bool require) infos.readJSON(*in); } - setColumns(loaded_columns); - setSerializationInfos(infos); + setColumns(loaded_columns, infos); } void IMergeTreeDataPart::assertHasVersionMetadata(MergeTreeTransaction * txn) const diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index bf47ced2a77..cbef544f816 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -133,19 +133,17 @@ public: String getTypeName() const { return getType().toString(); } - void setColumns(const NamesAndTypesList & new_columns); + void setColumns(const NamesAndTypesList & new_columns, const SerializationInfoByName & new_infos); const NamesAndTypesList & getColumns() const { return columns; } - const ColumnsDescription & getColumnsDescription() const { return columns_description; } NameAndTypePair getColumn(const String & name) const; std::optional tryGetColumn(const String & column_name) const; - void setSerializationInfos(const SerializationInfoByName & new_infos); - const SerializationInfoByName & getSerializationInfos() const { return serialization_infos; } - SerializationPtr getSerialization(const NameAndTypePair & column) const; + SerializationPtr getSerialization(const String & column_name) const; + SerializationPtr tryGetSerialization(const String & column_name) const; /// Throws an exception if part is not stored in on-disk format. void assertOnDisk() const; @@ -525,6 +523,8 @@ private: /// Map from name of column to its serialization info. SerializationInfoByName serialization_infos; + SerializationByName serializations; + /// Columns description for more convenient access /// to columns by name and getting subcolumns. ColumnsDescription columns_description; diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index a5ef2fa6ac8..30969939622 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -33,7 +33,6 @@ IMergeTreeReader::IMergeTreeReader( const ValueSizeMap & avg_value_size_hints_) : data_part(data_part_) , avg_value_size_hints(avg_value_size_hints_) - , columns(columns_) , uncompressed_cache(uncompressed_cache_) , mark_cache(mark_cache_) , settings(settings_) @@ -41,12 +40,19 @@ IMergeTreeReader::IMergeTreeReader( , metadata_snapshot(metadata_snapshot_) , all_mark_ranges(all_mark_ranges_) , alter_conversions(storage.getAlterConversionsForPart(data_part)) + , requested_columns(isWidePart(data_part) ? Nested::convertToSubcolumns(columns_) : columns_) + , part_columns(isWidePart(data_part) ? Nested::collect(data_part->getColumns()) : data_part->getColumns()) { + columns_to_read.reserve(requested_columns.size()); + serializations.reserve(requested_columns.size()); + + for (const auto & column : requested_columns) + { + columns_to_read.emplace_back(getColumnInPart(column)); + serializations.emplace_back(getSerializationInPart(column)); + } } -IMergeTreeReader::~IMergeTreeReader() = default; - - const IMergeTreeReader::ValueSizeMap & IMergeTreeReader::getAvgValueSizeHints() const { return avg_value_size_hints; @@ -56,7 +62,7 @@ void IMergeTreeReader::fillMissingColumns(Columns & res_columns, bool & should_e { try { - DB::fillMissingColumns(res_columns, num_rows, columns, metadata_snapshot); + DB::fillMissingColumns(res_columns, num_rows, requested_columns, metadata_snapshot); should_evaluate_missing_defaults = std::any_of( res_columns.begin(), res_columns.end(), [](const auto & column) { return column == nullptr; }); } @@ -72,7 +78,7 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns { try { - size_t num_columns = columns.size(); + size_t num_columns = requested_columns.size(); if (res_columns.size() != num_columns) throw Exception("invalid number of columns passed to MergeTreeReader::fillMissingColumns. " @@ -81,7 +87,7 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns /// Convert columns list to block. /// TODO: rewrite with columns interface. It will be possible after changes in ExpressionActions. - auto name_and_type = columns.begin(); + auto name_and_type = requested_columns.begin(); for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) { if (res_columns[pos] == nullptr) @@ -91,7 +97,7 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns } auto dag = DB::evaluateMissingDefaults( - additional_columns, columns, metadata_snapshot->getColumns(), storage.getContext()); + additional_columns, requested_columns, metadata_snapshot->getColumns(), storage.getContext()); if (dag) { dag->addMaterializingOutputActions(); @@ -102,7 +108,7 @@ void IMergeTreeReader::evaluateMissingDefaults(Block additional_columns, Columns } /// Move columns from block. - name_and_type = columns.begin(); + name_and_type = requested_columns.begin(); for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) res_columns[pos] = std::move(additional_columns.getByName(name_and_type->name).column); } @@ -128,18 +134,33 @@ String IMergeTreeReader::getColumnNameInPart(const NameAndTypePair & required_co NameAndTypePair IMergeTreeReader::getColumnInPart(const NameAndTypePair & required_column) const { - auto column_in_part = data_part->tryGetColumn(getColumnNameInPart(required_column)); + auto name_in_part = getColumnNameInPart(required_column); + auto column_in_part = part_columns.tryGetColumnOrSubcolumn(GetColumnsOptions::AllPhysical, name_in_part); if (column_in_part) return *column_in_part; return required_column; } +SerializationPtr IMergeTreeReader::getSerializationInPart(const NameAndTypePair & required_column) const +{ + auto name_in_part = getColumnNameInPart(required_column); + auto column_in_part = part_columns.tryGetColumnOrSubcolumn(GetColumnsOptions::AllPhysical, name_in_part); + if (!column_in_part) + return IDataType::getSerialization(required_column); + + const auto & infos = data_part->getSerializationInfos(); + if (auto it = infos.find(column_in_part->getNameInStorage()); it != infos.end()) + return IDataType::getSerialization(*column_in_part, *it->second); + + return IDataType::getSerialization(*column_in_part); +} + void IMergeTreeReader::performRequiredConversions(Columns & res_columns) const { try { - size_t num_columns = columns.size(); + size_t num_columns = requested_columns.size(); if (res_columns.size() != num_columns) { @@ -154,7 +175,7 @@ void IMergeTreeReader::performRequiredConversions(Columns & res_columns) const } Block copy_block; - auto name_and_type = columns.begin(); + auto name_and_type = requested_columns.begin(); for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) { @@ -164,14 +185,12 @@ void IMergeTreeReader::performRequiredConversions(Columns & res_columns) const copy_block.insert({res_columns[pos], getColumnInPart(*name_and_type).type, name_and_type->name}); } - DB::performRequiredConversions(copy_block, columns, storage.getContext()); + DB::performRequiredConversions(copy_block, requested_columns, storage.getContext()); /// Move columns from block. - name_and_type = columns.begin(); + name_and_type = requested_columns.begin(); for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) - { res_columns[pos] = std::move(copy_block.getByName(name_and_type->name).column); - } } catch (Exception & e) { @@ -199,9 +218,9 @@ IMergeTreeReader::ColumnPosition IMergeTreeReader::findColumnForOffsets(const St void IMergeTreeReader::checkNumberOfColumns(size_t num_columns_to_read) const { - if (num_columns_to_read != columns.size()) + if (num_columns_to_read != requested_columns.size()) throw Exception("invalid number of columns passed to MergeTreeReader::readRows. " - "Expected " + toString(columns.size()) + ", " + "Expected " + toString(requested_columns.size()) + ", " "got " + toString(num_columns_to_read), ErrorCodes::LOGICAL_ERROR); } diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index f844946fe22..0d6c0e607cd 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -37,7 +37,7 @@ public: virtual bool canReadIncompleteGranules() const = 0; - virtual ~IMergeTreeReader(); + virtual ~IMergeTreeReader() = default; const ValueSizeMap & getAvgValueSizeHints() const; @@ -52,8 +52,8 @@ public: /// try to perform conversions of columns. void performRequiredConversions(Columns & res_columns) const; - const NamesAndTypesList & getColumns() const { return columns; } - size_t numColumnsInResult() const { return columns.size(); } + const NamesAndTypesList & getColumns() const { return requested_columns; } + size_t numColumnsInResult() const { return requested_columns.size(); } size_t getFirstMarkToRead() const { return all_mark_ranges.front().begin; } @@ -65,6 +65,8 @@ protected: /// Returns actual column name and type in part, which can differ from table metadata. NameAndTypePair getColumnInPart(const NameAndTypePair & required_column) const; + /// Returns actual serialization in part, which can differ from table metadata. + SerializationPtr getSerializationInPart(const NameAndTypePair & required_column) const; void checkNumberOfColumns(size_t num_columns_to_read) const; @@ -73,8 +75,8 @@ protected: /// Stores states for IDataType::deserializeBinaryBulk DeserializeBinaryBulkStateMap deserialize_binary_bulk_state_map; - /// Columns that are read. - NamesAndTypesList columns; + NamesAndTypes columns_to_read; + Serializations serializations; UncompressedCache * uncompressed_cache; MarkCache * mark_cache; @@ -91,6 +93,12 @@ protected: private: /// Alter conversions, which must be applied on fly if required MergeTreeData::AlterConversions alter_conversions; + + /// Columns that are requested to read. + NamesAndTypesList requested_columns; + + /// Actual columns description in part. + ColumnsDescription part_columns; }; } diff --git a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp index 48cf720ad67..31c6a635b18 100644 --- a/src/Storages/MergeTree/IMergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/IMergedBlockOutputStream.cpp @@ -48,10 +48,10 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( std::map stream_counts; for (const auto & column : columns) { - data_part->getSerialization(column)->enumerateStreams( + data_part->getSerialization(column.name)->enumerateStreams( [&](const ISerialization::SubstreamPath & substream_path) { - ++stream_counts[ISerialization::getFileNameForStream(column, substream_path)]; + ++stream_counts[ISerialization::getFileNameForStream(column.name, substream_path)]; }); } @@ -59,13 +59,13 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( const String mrk_extension = data_part->getMarksFileExtension(); for (const auto & column_name : empty_columns) { - auto column_with_type = columns.tryGetByName(column_name); - if (!column_with_type) - continue; + auto serialization = data_part->tryGetSerialization(column_name); + if (!serialization) + continue; ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path) { - String stream_name = ISerialization::getFileNameForStream(*column_with_type, substream_path); + String stream_name = ISerialization::getFileNameForStream(column_name, substream_path); /// Delete files if they are no longer shared with another column. if (--stream_counts[stream_name] == 0) { @@ -74,7 +74,7 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart( } }; - data_part->getSerialization(*column_with_type)->enumerateStreams(callback); + serialization->enumerateStreams(callback); serialization_infos.erase(column_name); } diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index dc468174dfa..2c21e5de2b0 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -199,8 +199,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() infos.add(part->getSerializationInfos()); } - global_ctx->new_data_part->setColumns(global_ctx->storage_columns); - global_ctx->new_data_part->setSerializationInfos(infos); + global_ctx->new_data_part->setColumns(global_ctx->storage_columns, infos); const auto & local_part_min_ttl = global_ctx->new_data_part->ttl_infos.part_min_ttl; if (local_part_min_ttl && local_part_min_ttl <= global_ctx->time_of_merge) diff --git a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp index b22356a38ed..1c5006f4211 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartInMemory.cpp @@ -78,7 +78,7 @@ DataPartStoragePtr MergeTreeDataPartInMemory::flushToDisk(const String & new_rel auto new_data_part = storage.createPart(name, new_type, info, new_data_part_storage); new_data_part->uuid = uuid; - new_data_part->setColumns(columns); + new_data_part->setColumns(columns, {}); new_data_part->partition.value = partition.value; new_data_part->minmax_idx = minmax_idx; @@ -118,7 +118,7 @@ DataPartStoragePtr MergeTreeDataPartInMemory::flushToDisk(const String & new_rel auto projection_data_part = storage.createPart(projection_name, projection_type, projection_info, projection_part_storage_builder->getStorage(), parent_part); projection_data_part->is_temp = false; // clean up will be done on parent part - projection_data_part->setColumns(projection->getColumns()); + projection_data_part->setColumns(projection->getColumns(), {}); projection_part_storage_builder->createDirectories(); const auto & desc = projections.get(name); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp index b70d67de3b4..c7b6ff0c4dd 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWide.cpp @@ -81,7 +81,7 @@ ColumnSize MergeTreeDataPartWide::getColumnSizeImpl( if (checksums.empty()) return size; - getSerialization(column)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + getSerialization(column.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { String file_name = ISerialization::getFileNameForStream(column, substream_path); @@ -169,9 +169,9 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const { if (require_part_metadata) { - for (const NameAndTypePair & name_type : columns) + for (const auto & name_type : columns) { - getSerialization(name_type)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + getSerialization(name_type.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { String file_name = ISerialization::getFileNameForStream(name_type, substream_path); String mrk_file_name = file_name + index_granularity_info.marks_file_extension; @@ -196,9 +196,9 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const { /// Check that all marks are nonempty and have the same size. std::optional marks_size; - for (const NameAndTypePair & name_type : columns) + for (const auto & name_type : columns) { - getSerialization(name_type)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + getSerialization(name_type.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { auto file_path = ISerialization::getFileNameForStream(name_type, substream_path) + index_granularity_info.marks_file_extension; @@ -237,7 +237,7 @@ bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const }; bool res = true; - getSerialization(column)->enumerateStreams([&](const auto & substream_path) + getSerialization(column.name)->enumerateStreams([&](const auto & substream_path) { String file_name = ISerialization::getFileNameForStream(column, substream_path); if (!check_stream_exists(file_name)) @@ -250,7 +250,7 @@ bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const String MergeTreeDataPartWide::getFileNameForColumn(const NameAndTypePair & column) const { String filename; - getSerialization(column)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) + getSerialization(column.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) { if (filename.empty()) filename = ISerialization::getFileNameForStream(column, substream_path); @@ -261,7 +261,7 @@ String MergeTreeDataPartWide::getFileNameForColumn(const NameAndTypePair & colum void MergeTreeDataPartWide::calculateEachColumnSizes(ColumnSizeByName & each_columns_size, ColumnSize & total_size) const { std::unordered_set processed_substreams; - for (const NameAndTypePair & column : columns) + for (const auto & column : columns) { ColumnSize size = getColumnSizeImpl(column, &processed_substreams); each_columns_size[column.name] = size; @@ -272,7 +272,7 @@ void MergeTreeDataPartWide::calculateEachColumnSizes(ColumnSizeByName & each_col if (rows_count != 0 && column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes() - && getSerialization(column)->getKind() == ISerialization::Kind::DEFAULT) + && getSerialization(column.name)->getKind() == ISerialization::Kind::DEFAULT) { size_t rows_in_column = size.data_uncompressed / column.type->getSizeOfValueInMemory(); if (rows_in_column != rows_count) diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index d181a15d08f..771248b99c6 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -67,7 +67,7 @@ void MergeTreeDataPartWriterCompact::addStreams(const NameAndTypePair & column, }; ISerialization::SubstreamPath path; - data_part->getSerialization(column)->enumerateStreams(path, callback, column.type); + data_part->getSerialization(column.name)->enumerateStreams(path, callback, column.type); } namespace @@ -208,7 +208,7 @@ void MergeTreeDataPartWriterCompact::writeDataBlock(const Block & block, const G writeIntBinary(static_cast(0), marks); writeColumnSingleGranule( - block.getByName(name_and_type->name), data_part->getSerialization(*name_and_type), + block.getByName(name_and_type->name), data_part->getSerialization(name_and_type->name), stream_getter, granule.start_row, granule.rows_to_write); /// Each type always have at least one substream diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index e3925940553..712c3f74bdd 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -121,7 +121,7 @@ void MergeTreeDataPartWriterWide::addStreams( }; ISerialization::SubstreamPath path; - data_part->getSerialization(column)->enumerateStreams(path, callback, column.type); + data_part->getSerialization(column.name)->enumerateStreams(path, callback, column.type); } @@ -216,7 +216,7 @@ void MergeTreeDataPartWriterWide::write(const Block & block, const IColumn::Perm { auto & column = block_to_write.getByName(it->name); - if (data_part->getSerialization(*it)->getKind() != ISerialization::Kind::SPARSE) + if (data_part->getSerialization(it->name)->getKind() != ISerialization::Kind::SPARSE) column.column = recursiveRemoveSparse(column.column); if (permutation) @@ -278,7 +278,7 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn( ISerialization::SubstreamPath & path) { StreamsWithMarks result; - data_part->getSerialization(column)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) + data_part->getSerialization(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; @@ -313,7 +313,7 @@ void MergeTreeDataPartWriterWide::writeSingleGranule( ISerialization::SerializeBinaryBulkSettings & serialize_settings, const Granule & granule) { - const auto & serialization = data_part->getSerialization(name_and_type); + const auto & serialization = data_part->getSerialization(name_and_type.name); serialization->serializeBinaryBulkWithMultipleStreams(column, granule.start_row, granule.rows_to_write, serialize_settings, serialization_state); /// So that instead of the marks pointing to the end of the compressed block, there were marks pointing to the beginning of the next one. @@ -343,7 +343,7 @@ void MergeTreeDataPartWriterWide::writeColumn( const auto & [name, type] = name_and_type; auto [it, inserted] = serialization_states.emplace(name, nullptr); - auto serialization = data_part->getSerialization(name_and_type); + auto serialization = data_part->getSerialization(name_and_type.name); if (inserted) { @@ -405,7 +405,7 @@ void MergeTreeDataPartWriterWide::writeColumn( void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePair & name_type) { const auto & [name, type] = name_type; - const auto & serialization = data_part->getSerialization(name_type); + const auto & serialization = data_part->getSerialization(name_type.name); if (!type->isValueRepresentedByNumber() || type->haveSubtypes() || serialization->getKind() != ISerialization::Kind::DEFAULT) throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot validate column of non fixed type {}", type->getName()); @@ -549,7 +549,7 @@ void MergeTreeDataPartWriterWide::fillDataChecksums(IMergeTreeDataPart::Checksum if (!serialization_states.empty()) { serialize_settings.getter = createStreamGetter(*it, written_offset_columns ? *written_offset_columns : offset_columns); - data_part->getSerialization(*it)->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[it->name]); + data_part->getSerialization(it->name)->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[it->name]); } if (write_final_mark) @@ -583,7 +583,7 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(bool sync) { if (column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes() - && data_part->getSerialization(column)->getKind() == ISerialization::Kind::DEFAULT) + && data_part->getSerialization(columnn.name)->getKind() == ISerialization::Kind::DEFAULT) { validateColumnOfFixedSize(column); } @@ -623,7 +623,7 @@ void MergeTreeDataPartWriterWide::writeFinalMark( { writeSingleMark(column, offset_columns, 0, path); /// Memoize information about offsets - data_part->getSerialization(column)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) + data_part->getSerialization(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) { bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; if (is_offsets) diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 89042e25a0e..957958e0a50 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -395,8 +395,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart( SerializationInfoByName infos(columns, settings); infos.add(block); - new_data_part->setColumns(columns); - new_data_part->setSerializationInfos(infos); + new_data_part->setColumns(columns, infos); new_data_part->rows_count = block.rows(); new_data_part->partition = std::move(partition); new_data_part->minmax_idx = std::move(minmax_idx); @@ -523,8 +522,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( SerializationInfoByName infos(columns, settings); infos.add(block); - new_data_part->setColumns(columns); - new_data_part->setSerializationInfos(infos); + new_data_part->setColumns(columns, infos); if (new_data_part->isStoredOnDisk()) { diff --git a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp index 5cb37b0515a..3c05804c3f5 100644 --- a/src/Storages/MergeTree/MergeTreeReaderCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderCompact.cpp @@ -45,30 +45,30 @@ MergeTreeReaderCompact::MergeTreeReaderCompact( { try { - size_t columns_num = columns.size(); + size_t columns_num = columns_to_read.size(); column_positions.resize(columns_num); read_only_offsets.resize(columns_num); - auto name_and_type = columns.begin(); - for (size_t i = 0; i < columns_num; ++i, ++name_and_type) + + for (size_t i = 0; i < columns_num; ++i) { - if (name_and_type->isSubcolumn()) + const auto & column_to_read = columns_to_read[i]; + + if (column_to_read.isSubcolumn()) { auto storage_column_from_part = getColumnInPart( - {name_and_type->getNameInStorage(), name_and_type->getTypeInStorage()}); + {column_to_read.getNameInStorage(), column_to_read.getTypeInStorage()}); - if (!storage_column_from_part.type->tryGetSubcolumnType(name_and_type->getSubcolumnName())) + if (!storage_column_from_part.type->tryGetSubcolumnType(column_to_read.getSubcolumnName())) continue; } - auto column_from_part = getColumnInPart(*name_and_type); - - auto position = data_part->getColumnPosition(column_from_part.getNameInStorage()); - if (!position && typeid_cast(column_from_part.type.get())) + auto position = data_part->getColumnPosition(column_to_read.getNameInStorage()); + if (!position && typeid_cast(column_to_read.type.get())) { /// If array of Nested column is missing in part, /// we have to read its offsets if they exist. - position = findColumnForOffsets(column_from_part.name); + position = findColumnForOffsets(column_to_read.name); read_only_offsets[i] = (position != std::nullopt); } @@ -143,42 +143,31 @@ size_t MergeTreeReaderCompact::readRows( from_mark = next_mark; size_t read_rows = 0; - size_t num_columns = columns.size(); + size_t num_columns = columns_to_read.size(); checkNumberOfColumns(num_columns); MutableColumns mutable_columns(num_columns); - auto column_it = columns.begin(); - for (size_t i = 0; i < num_columns; ++i, ++column_it) + for (size_t i = 0; i < num_columns; ++i) { - if (!column_positions[i]) - continue; - - auto column_from_part = getColumnInPart(*column_it); - if (res_columns[i] == nullptr) - { - auto serialization = data_part->getSerialization(column_from_part); - res_columns[i] = column_from_part.type->createColumn(*serialization); - } + if (column_positions[i] && res_columns[i] == nullptr) + res_columns[i] = columns_to_read[i].type->createColumn(*serializations[i]); } while (read_rows < max_rows_to_read) { size_t rows_to_read = data_part->index_granularity.getMarkRows(from_mark); - auto name_and_type = columns.begin(); - for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) + for (size_t pos = 0; pos < num_columns; ++pos) { if (!res_columns[pos]) continue; - auto column_from_part = getColumnInPart(*name_and_type); - try { auto & column = res_columns[pos]; size_t column_size_before_reading = column->size(); - readData(column_from_part, column, from_mark, current_task_last_mark, *column_positions[pos], rows_to_read, read_only_offsets[pos]); + readData(columns_to_read[pos], column, from_mark, current_task_last_mark, *column_positions[pos], rows_to_read, read_only_offsets[pos]); size_t read_rows_in_column = column->size() - column_size_before_reading; if (read_rows_in_column != rows_to_read) @@ -192,7 +181,7 @@ size_t MergeTreeReaderCompact::readRows( storage.reportBrokenPart(data_part); /// Better diagnostics. - e.addMessage("(while reading column " + column_from_part.name + ")"); + e.addMessage("(while reading column " + columns_to_read[pos].name + ")"); throw; } catch (...) @@ -240,7 +229,7 @@ void MergeTreeReaderCompact::readData( const auto & type_in_storage = name_and_type.getTypeInStorage(); const auto & name_in_storage = name_and_type.getNameInStorage(); - auto serialization = data_part->getSerialization(NameAndTypePair{name_in_storage, type_in_storage}); + auto serialization = getSerializationInPart({name_in_storage, type_in_storage}); ColumnPtr temp_column = type_in_storage->createColumn(*serialization); serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state); @@ -256,7 +245,7 @@ void MergeTreeReaderCompact::readData( } else { - auto serialization = data_part->getSerialization(name_and_type); + auto serialization = getSerializationInPart(name_and_type); serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state); serialization->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state, nullptr); } diff --git a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp index d8e0b6e4d6c..766c28c99b9 100644 --- a/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderInMemory.cpp @@ -32,10 +32,8 @@ MergeTreeReaderInMemory::MergeTreeReaderInMemory( {}) , part_in_memory(std::move(data_part_)) { - for (const auto & name_and_type : columns) + for (const auto & [name, type] : columns_to_read) { - auto [name, type] = getColumnInPart(name_and_type); - /// If array of Nested column is missing in part, /// we have to read its offsets if they exist. if (!part_in_memory->block.has(name) && typeid_cast(type.get())) @@ -64,20 +62,19 @@ size_t MergeTreeReaderInMemory::readRows( + toString(total_rows_read) + ". Rows in part: " + toString(part_rows), ErrorCodes::CANNOT_READ_ALL_DATA); size_t rows_to_read = std::min(max_rows_to_read, part_rows - total_rows_read); - auto column_it = columns.begin(); - for (size_t i = 0; i < num_columns; ++i, ++column_it) + for (size_t i = 0; i < num_columns; ++i) { - auto name_type = getColumnInPart(*column_it); + const auto & column_to_read = columns_to_read[i]; /// Copy offsets, if array of Nested column is missing in part. - auto offsets_it = positions_for_offsets.find(name_type.name); - if (offsets_it != positions_for_offsets.end() && !name_type.isSubcolumn()) + auto offsets_it = positions_for_offsets.find(column_to_read.name); + if (offsets_it != positions_for_offsets.end() && !column_to_read.isSubcolumn()) { const auto & source_offsets = assert_cast( *part_in_memory->block.getByPosition(offsets_it->second).column).getOffsets(); if (res_columns[i] == nullptr) - res_columns[i] = name_type.type->createColumn(); + res_columns[i] = column_to_read.type->createColumn(); auto mutable_column = res_columns[i]->assumeMutable(); auto & res_offstes = assert_cast(*mutable_column).getOffsets(); @@ -87,9 +84,9 @@ size_t MergeTreeReaderInMemory::readRows( res_columns[i] = std::move(mutable_column); } - else if (part_in_memory->hasColumnFiles(name_type)) + else if (part_in_memory->hasColumnFiles(column_to_read)) { - auto block_column = getColumnFromBlock(part_in_memory->block, name_type); + auto block_column = getColumnFromBlock(part_in_memory->block, column_to_read); if (rows_to_read == part_rows) { res_columns[i] = block_column; @@ -97,7 +94,7 @@ size_t MergeTreeReaderInMemory::readRows( else { if (res_columns[i] == nullptr) - res_columns[i] = name_type.type->createColumn(); + res_columns[i] = column_to_read.type->createColumn(); auto mutable_column = res_columns[i]->assumeMutable(); mutable_column->insertRangeFrom(*block_column, total_rows_read, rows_to_read); diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.cpp b/src/Storages/MergeTree/MergeTreeReaderWide.cpp index aed9a906a5c..1274017b865 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.cpp +++ b/src/Storages/MergeTree/MergeTreeReaderWide.cpp @@ -48,11 +48,8 @@ MergeTreeReaderWide::MergeTreeReaderWide( { try { - for (const NameAndTypePair & column : columns) - { - auto column_from_part = getColumnInPart(column); - addStreams(column_from_part, profile_callback_, clock_type_); - } + for (size_t i = 0; i < columns_to_read.size(); ++i) + addStreams(columns_to_read[i], serializations[i], profile_callback_, clock_type_); } catch (...) { @@ -61,20 +58,6 @@ MergeTreeReaderWide::MergeTreeReaderWide( } } -String MergeTreeReaderWide::getNameForSubstreamCache(const NameAndTypePair & column) const -{ - if (!column.isSubcolumn() && isArray(column.type)) - { - auto split = Nested::splitName(column.name); - const auto & part_columns = data_part->getColumnsDescription(); - - if (!split.second.empty() && part_columns.hasNested(split.first)) - return split.first; - } - - return column.getNameInStorage(); -} - size_t MergeTreeReaderWide::readRows( size_t from_mark, size_t current_task_last_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns) { @@ -94,47 +77,40 @@ size_t MergeTreeReaderWide::readRows( { /// Request reading of data in advance, /// so if reading can be asynchronous, it will also be performed in parallel for all columns. - auto name_and_type = columns.begin(); - for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) + for (size_t pos = 0; pos < num_columns; ++pos) { - auto column_from_part = getColumnInPart(*name_and_type); try { - auto & cache = caches[getNameForSubstreamCache(column_from_part)]; - prefetch(column_from_part, from_mark, continue_reading, current_task_last_mark, cache, prefetched_streams); + auto & cache = caches[columns_to_read[pos].getNameInStorage()]; + prefetch(columns_to_read[pos], serializations[pos], from_mark, continue_reading, current_task_last_mark, cache, prefetched_streams); } catch (Exception & e) { /// Better diagnostics. - e.addMessage("(while reading column " + column_from_part.name + ")"); + e.addMessage("(while reading column " + columns_to_read[pos].name + ")"); throw; } } } - auto name_and_type = columns.begin(); - - for (size_t pos = 0; pos < num_columns; ++pos, ++name_and_type) + for (size_t pos = 0; pos < num_columns; ++pos) { - auto column_from_part = getColumnInPart(*name_and_type); - const auto & [name, type] = column_from_part; + const auto & column_to_read = columns_to_read[pos]; /// The column is already present in the block so we will append the values to the end. bool append = res_columns[pos] != nullptr; if (!append) - { - auto serialization = data_part->getSerialization(column_from_part); - res_columns[pos] = type->createColumn(*serialization); - } + res_columns[pos] = column_to_read.type->createColumn(*serializations[pos]); auto & column = res_columns[pos]; try { size_t column_size_before_reading = column->size(); - auto & cache = caches[getNameForSubstreamCache(column_from_part)]; + auto & cache = caches[column_to_read.getNameInStorage()]; readData( - column_from_part, column, from_mark, continue_reading, current_task_last_mark, + column_to_read, serializations[pos], column, + from_mark, continue_reading, current_task_last_mark, max_rows_to_read, cache, /* was_prefetched =*/ !prefetched_streams.empty()); /// For elements of Nested, column_size_before_reading may be greater than column size @@ -145,7 +121,7 @@ size_t MergeTreeReaderWide::readRows( catch (Exception & e) { /// Better diagnostics. - e.addMessage("(while reading column " + name + ")"); + e.addMessage("(while reading column " + column_to_read.name + ")"); throw; } @@ -178,8 +154,11 @@ size_t MergeTreeReaderWide::readRows( return read_rows; } -void MergeTreeReaderWide::addStreams(const NameAndTypePair & name_and_type, - const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type) +void MergeTreeReaderWide::addStreams( + const NameAndTypePair & name_and_type, + const SerializationPtr & serialization, + const ReadBufferFromFileBase::ProfileCallback & profile_callback, + clockid_t clock_type) { ISerialization::StreamCallback callback = [&] (const ISerialization::SubstreamPath & substream_path) { @@ -206,7 +185,7 @@ void MergeTreeReaderWide::addStreams(const NameAndTypePair & name_and_type, profile_callback, clock_type, is_lc_dict)); }; - data_part->getSerialization(name_and_type)->enumerateStreams(callback); + serialization->enumerateStreams(callback); } @@ -260,13 +239,13 @@ void MergeTreeReaderWide::deserializePrefix( void MergeTreeReaderWide::prefetch( const NameAndTypePair & name_and_type, + const SerializationPtr & serialization, size_t from_mark, bool continue_reading, size_t current_task_last_mark, ISerialization::SubstreamsCache & cache, std::unordered_set & prefetched_streams) { - auto serialization = data_part->getSerialization(name_and_type); deserializePrefix(serialization, name_and_type, current_task_last_mark, cache); serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) @@ -286,7 +265,7 @@ void MergeTreeReaderWide::prefetch( void MergeTreeReaderWide::readData( - const NameAndTypePair & name_and_type, ColumnPtr & column, + const NameAndTypePair & name_and_type, const SerializationPtr & serialization, ColumnPtr & column, size_t from_mark, bool continue_reading, size_t current_task_last_mark, size_t max_rows_to_read, ISerialization::SubstreamsCache & cache, bool was_prefetched) { @@ -294,9 +273,6 @@ void MergeTreeReaderWide::readData( ISerialization::DeserializeBinaryBulkSettings deserialize_settings; deserialize_settings.avg_value_size_hint = avg_value_size_hint; - const auto & name = name_and_type.name; - auto serialization = data_part->getSerialization(name_and_type); - deserializePrefix(serialization, name_and_type, current_task_last_mark, cache); deserialize_settings.getter = [&](const ISerialization::SubstreamPath & substream_path) @@ -308,7 +284,7 @@ void MergeTreeReaderWide::readData( seek_to_mark, current_task_last_mark, cache); }; deserialize_settings.continuous_reading = continue_reading; - auto & deserialize_state = deserialize_binary_bulk_state_map[name]; + auto & deserialize_state = deserialize_binary_bulk_state_map[name_and_type.name]; serialization->deserializeBinaryBulkWithMultipleStreams(column, max_rows_to_read, deserialize_settings, deserialize_state, &cache); IDataType::updateAvgValueSizeHint(*column, avg_value_size_hint); diff --git a/src/Storages/MergeTree/MergeTreeReaderWide.h b/src/Storages/MergeTree/MergeTreeReaderWide.h index e382f3f1dde..2137695b6d7 100644 --- a/src/Storages/MergeTree/MergeTreeReaderWide.h +++ b/src/Storages/MergeTree/MergeTreeReaderWide.h @@ -38,19 +38,21 @@ public: private: FileStreams streams; - String getNameForSubstreamCache(const NameAndTypePair & column) const; - - void addStreams(const NameAndTypePair & name_and_type, - const ReadBufferFromFileBase::ProfileCallback & profile_callback, clockid_t clock_type); + void addStreams( + const NameAndTypePair & name_and_type, + const SerializationPtr & serialization, + const ReadBufferFromFileBase::ProfileCallback & profile_callback, + clockid_t clock_type); void readData( - const NameAndTypePair & name_and_type, ColumnPtr & column, + const NameAndTypePair & name_and_type, const SerializationPtr & serialization, ColumnPtr & column, size_t from_mark, bool continue_reading, size_t current_task_last_mark, size_t max_rows_to_read, ISerialization::SubstreamsCache & cache, bool was_prefetched); /// Make next readData more simple by calling 'prefetch' of all related ReadBuffers (column streams). void prefetch( const NameAndTypePair & name_and_type, + const SerializationPtr & serialization, size_t from_mark, bool continue_reading, size_t current_task_last_mark, diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 9ed8fe0ad14..9b79f89ff98 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -212,7 +212,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore(const Stor part->minmax_idx->update(block, storage.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); part->partition.create(metadata_snapshot, block, 0, context); - part->setColumns(block.getNamesAndTypesList()); + part->setColumns(block.getNamesAndTypesList(), {}); if (metadata_snapshot->hasSortingKey()) metadata_snapshot->getSortingKey().expression->execute(block); diff --git a/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/src/Storages/MergeTree/MergedBlockOutputStream.cpp index f7da1eb2585..6e5d16b8c7b 100644 --- a/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -162,8 +162,7 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync( serialization_infos.replaceData(new_serialization_infos); files_to_remove_after_sync = removeEmptyColumnsFromPart(new_part, part_columns, serialization_infos, checksums); - new_part->setColumns(part_columns); - new_part->setSerializationInfos(serialization_infos); + new_part->setColumns(part_columns, serialization_infos); } auto finalizer = std::make_unique(*writer, new_part, data_part_storage_builder, files_to_remove_after_sync, sync); diff --git a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index deab5e748c7..21e01223e2c 100644 --- a/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -87,8 +87,7 @@ MergedColumnOnlyOutputStream::fillChecksums( all_checksums.files.erase(removed_file); } - new_part->setColumns(columns); - new_part->setSerializationInfos(serialization_infos); + new_part->setColumns(columns, serialization_infos); return checksums; } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 3a5aa2f8860..ea8fce19991 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -441,7 +441,8 @@ NameSet collectFilesToSkip( files_to_skip.insert(stream_name + mrk_extension); }; - source_part->getSerialization({entry.name, entry.type})->enumerateStreams(callback); + if (auto serialization = source_part->tryGetSerialization(entry.name)) + serialization->enumerateStreams(callback); } for (const auto & index : indices_to_recalc) { @@ -466,11 +467,14 @@ static NameToNameVector collectFilesForRenames( std::map stream_counts; for (const auto & column : source_part->getColumns()) { - source_part->getSerialization(column)->enumerateStreams( - [&](const ISerialization::SubstreamPath & substream_path) - { - ++stream_counts[ISerialization::getFileNameForStream(column, substream_path)]; - }); + if (auto serialization = source_part->tryGetSerialization(column.name)) + { + serialization->enumerateStreams( + [&](const ISerialization::SubstreamPath & substream_path) + { + ++stream_counts[ISerialization::getFileNameForStream(column, substream_path)]; + }); + } } NameToNameVector rename_vector; @@ -508,9 +512,9 @@ static NameToNameVector collectFilesForRenames( } }; - auto column = source_part->getColumns().tryGetByName(command.column_name); - if (column) - source_part->getSerialization(*column)->enumerateStreams(callback); + + if (auto serialization = source_part->tryGetSerialization(command.column_name)) + serialization->enumerateStreams(callback); } else if (command.type == MutationCommand::Type::RENAME_COLUMN) { @@ -530,9 +534,8 @@ static NameToNameVector collectFilesForRenames( } }; - auto column = source_part->getColumns().tryGetByName(command.column_name); - if (column) - source_part->getSerialization(*column)->enumerateStreams(callback); + if (auto serialization = source_part->tryGetSerialization(command.column_name)) + serialization->enumerateStreams(callback); } } @@ -1494,8 +1497,7 @@ bool MutateTask::prepare() ctx->source_part, ctx->updated_header, ctx->storage_columns, ctx->source_part->getSerializationInfos(), ctx->commands_for_part); - ctx->new_data_part->setColumns(new_columns); - ctx->new_data_part->setSerializationInfos(new_infos); + ctx->new_data_part->setColumns(new_columns, new_infos); ctx->new_data_part->partition.assign(ctx->source_part->partition); /// Don't change granularity type while mutating subset of columns diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 219093e8d75..eff9fce2e27 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -7862,7 +7862,7 @@ bool StorageReplicatedMergeTree::createEmptyPartInsteadOfLost(zkutil::ZooKeeperP if (settings->assign_part_uuids) new_data_part->uuid = UUIDHelpers::generateV4(); - new_data_part->setColumns(columns); + new_data_part->setColumns(columns, {}); new_data_part->rows_count = block.rows(); { diff --git a/src/Storages/System/StorageSystemPartsColumns.cpp b/src/Storages/System/StorageSystemPartsColumns.cpp index cebcfc492bf..292d8087c10 100644 --- a/src/Storages/System/StorageSystemPartsColumns.cpp +++ b/src/Storages/System/StorageSystemPartsColumns.cpp @@ -227,7 +227,7 @@ void StorageSystemPartsColumns::processNextStorage( if (columns_mask[src_index++]) columns[res_index++]->insert(column_size.marks); - auto serialization = part->getSerialization(column); + auto serialization = part->getSerialization(column.name); if (columns_mask[src_index++]) columns[res_index++]->insert(ISerialization::kindToString(serialization->getKind())); From 56039c87806a44d65d73bcd2d0338352561680f4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 27 Jul 2022 15:35:09 +0000 Subject: [PATCH 044/117] fix alter of LowCardinality --- .../Serializations/SerializationInfo.cpp | 9 +- src/Storages/MergeTree/IMergeTreeDataPart.h | 1 + src/Storages/MergeTree/MutateTask.cpp | 102 +++++++++--------- 3 files changed, 56 insertions(+), 56 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationInfo.cpp b/src/DataTypes/Serializations/SerializationInfo.cpp index 543b4a75f0a..1b845533aaf 100644 --- a/src/DataTypes/Serializations/SerializationInfo.cpp +++ b/src/DataTypes/Serializations/SerializationInfo.cpp @@ -239,13 +239,8 @@ void SerializationInfoByName::readJSON(ReadBuffer & in) "Missed field '{}' in SerializationInfo of columns", KEY_NAME); auto name = elem_object->getValue(KEY_NAME); - auto it = find(name); - - if (it == end()) - throw Exception(ErrorCodes::CORRUPTED_DATA, - "There is no column {} in serialization infos", name); - - it->second->fromJSON(*elem_object); + if (auto it = find(name); it != end()) + it->second->fromJSON(*elem_object); } } } diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index cbef544f816..bf4eba1c354 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -136,6 +136,7 @@ public: void setColumns(const NamesAndTypesList & new_columns, const SerializationInfoByName & new_infos); const NamesAndTypesList & getColumns() const { return columns; } + const ColumnsDescription & getColumnsDescription() const { return columns_description; } NameAndTypePair getColumn(const String & name) const; std::optional tryGetColumn(const String & column_name) const; diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index ea8fce19991..5b54c1e6ae7 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -57,7 +57,7 @@ static void splitMutationCommands( MutationCommands & for_interpreter, MutationCommands & for_file_renames) { - ColumnsDescription part_columns(part->getColumns()); + auto part_columns = part->getColumnsDescription(); if (!isWidePart(part)) { @@ -139,18 +139,11 @@ static void splitMutationCommands( else if (part_columns.has(command.column_name)) { if (command.type == MutationCommand::Type::READ_COLUMN) - { for_interpreter.push_back(command); - } else if (command.type == MutationCommand::Type::RENAME_COLUMN) - { part_columns.rename(command.column_name, command.rename_to); - for_file_renames.push_back(command); - } - else - { - for_file_renames.push_back(command); - } + + for_file_renames.push_back(command); } } } @@ -436,7 +429,7 @@ NameSet collectFilesToSkip( { ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path) { - String stream_name = ISerialization::getFileNameForStream({entry.name, entry.type}, substream_path); + String stream_name = ISerialization::getFileNameForStream(entry.name, substream_path); files_to_skip.insert(stream_name + ".bin"); files_to_skip.insert(stream_name + mrk_extension); }; @@ -461,23 +454,14 @@ NameSet collectFilesToSkip( /// from filesystem and in-memory checksums. Ordered result is important, /// because we can apply renames that affects each other: x -> z, y -> x. static NameToNameVector collectFilesForRenames( - MergeTreeData::DataPartPtr source_part, const MutationCommands & commands_for_removes, const String & mrk_extension) + MergeTreeData::DataPartPtr source_part, + MergeTreeData::DataPartPtr new_part, + const MutationCommands & commands_for_removes, + const String & mrk_extension) { - /// Collect counts for shared streams of different columns. As an example, Nested columns have shared stream with array sizes. - std::map stream_counts; - for (const auto & column : source_part->getColumns()) - { - if (auto serialization = source_part->tryGetSerialization(column.name)) - { - serialization->enumerateStreams( - [&](const ISerialization::SubstreamPath & substream_path) - { - ++stream_counts[ISerialization::getFileNameForStream(column, substream_path)]; - }); - } - } - NameToNameVector rename_vector; + NameSet renamed_streams; + /// Remove old data for (const auto & command : commands_for_removes) { @@ -499,23 +483,6 @@ static NameToNameVector collectFilesForRenames( if (source_part->checksums.has(command.column_name + ".proj")) rename_vector.emplace_back(command.column_name + ".proj", ""); } - else if (command.type == MutationCommand::Type::DROP_COLUMN) - { - ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path) - { - String stream_name = ISerialization::getFileNameForStream({command.column_name, command.data_type}, substream_path); - /// Delete files if they are no longer shared with another column. - if (--stream_counts[stream_name] == 0) - { - rename_vector.emplace_back(stream_name + ".bin", ""); - rename_vector.emplace_back(stream_name + mrk_extension, ""); - } - }; - - - if (auto serialization = source_part->tryGetSerialization(command.column_name)) - serialization->enumerateStreams(callback); - } else if (command.type == MutationCommand::Type::RENAME_COLUMN) { String escaped_name_from = escapeForFileName(command.column_name); @@ -523,12 +490,12 @@ static NameToNameVector collectFilesForRenames( ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path) { - String stream_from = ISerialization::getFileNameForStream({command.column_name, command.data_type}, substream_path); - + String stream_from = ISerialization::getFileNameForStream(command.column_name, substream_path); String stream_to = boost::replace_first_copy(stream_from, escaped_name_from, escaped_name_to); if (stream_from != stream_to) { + renamed_streams.insert(stream_from); rename_vector.emplace_back(stream_from + ".bin", stream_to + ".bin"); rename_vector.emplace_back(stream_from + mrk_extension, stream_to + mrk_extension); } @@ -539,6 +506,41 @@ static NameToNameVector collectFilesForRenames( } } + auto collect_all_stream_names = [&](const auto & data_part) + { + NameSet res; + for (const auto & column : data_part->getColumns()) + { + if (auto serialization = data_part->tryGetSerialization(column.name)) + { + serialization->enumerateStreams( + [&](const ISerialization::SubstreamPath & substream_path) + { + res.insert(ISerialization::getFileNameForStream(column.name, substream_path)); + }); + } + } + + return res; + }; + + /// Remove files for streams that exists in source part, + /// but were removed in new_part by DROP COLUMN + /// or MODIFY COLUMN from type with higher number of streams + /// (e.g. LowCardinality -> String). + + auto old_streams = collect_all_stream_names(source_part); + auto new_streams = collect_all_stream_names(new_part); + + for (const auto & old_stream : old_streams) + { + if (!new_streams.contains(old_stream) && !renamed_streams.contains(old_stream)) + { + rename_vector.emplace_back(old_stream + ".bin", ""); + rename_vector.emplace_back(old_stream + mrk_extension, ""); + } + } + return rename_vector; } @@ -552,9 +554,6 @@ void finalizeMutatedPart( const CompressionCodecPtr & codec, ContextPtr context) { - //auto disk = new_data_part->volume->getDisk(); - //auto part_path = fs::path(new_data_part->getRelativePath()); - if (new_data_part->uuid != UUIDHelpers::Nil) { auto out = data_part_storage_builder->writeFile(IMergeTreeDataPart::UUID_FILE_NAME, 4096, context->getWriteSettings()); @@ -1535,7 +1534,12 @@ bool MutateTask::prepare() ctx->indices_to_recalc, ctx->mrk_extension, ctx->projections_to_recalc); - ctx->files_to_rename = MutationHelpers::collectFilesForRenames(ctx->source_part, ctx->for_file_renames, ctx->mrk_extension); + + ctx->files_to_rename = MutationHelpers::collectFilesForRenames( + ctx->source_part, + ctx->new_data_part, + ctx->for_file_renames, + ctx->mrk_extension); if (ctx->indices_to_recalc.empty() && ctx->projections_to_recalc.empty() && From 51c4c9c957bff6afe2119ec7a44441fc501f81ac Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 27 Jul 2022 15:45:23 +0000 Subject: [PATCH 045/117] Fixing build. --- src/Storages/StorageMerge.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 3fc7ca36569..666717e50a0 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -37,8 +37,6 @@ #include -using std::operator""sv; - namespace DB { From 6206e468a495490c39e76a5dec39058083b346b8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 27 Jul 2022 17:02:48 +0000 Subject: [PATCH 046/117] Ignor materialize function for ActionsDAG in PK analysis. --- src/Storages/MergeTree/KeyCondition.cpp | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index d79861e2916..5cd193eb1c9 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -732,6 +732,14 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( return arg; } + if (name == "materialize") + { + /// Ignore materialize + const auto & arg = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, need_inversion); + to_inverted[&node] = &arg; + return arg; + } + if (name == "indexHint") { ActionsDAG::NodeRawConstPtrs children; From 6919ae7c919fdb985fb14d190bb1fc21e3d4816e Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 28 Jul 2022 12:24:16 +0000 Subject: [PATCH 047/117] Fixing a test with indexHint --- src/Interpreters/RequiredSourceColumnsData.h | 1 + src/Interpreters/RequiredSourceColumnsVisitor.cpp | 9 ++++++--- src/Interpreters/TreeRewriter.cpp | 9 +++++---- src/Interpreters/TreeRewriter.h | 2 +- 4 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Interpreters/RequiredSourceColumnsData.h b/src/Interpreters/RequiredSourceColumnsData.h index 501f6961efa..dd4e2dc3d68 100644 --- a/src/Interpreters/RequiredSourceColumnsData.h +++ b/src/Interpreters/RequiredSourceColumnsData.h @@ -36,6 +36,7 @@ struct RequiredSourceColumnsData bool has_table_join = false; bool has_array_join = false; + bool visit_index_hint = false; bool addColumnAliasIfAny(const IAST & ast); void addColumnIdentifier(const ASTIdentifier & node); diff --git a/src/Interpreters/RequiredSourceColumnsVisitor.cpp b/src/Interpreters/RequiredSourceColumnsVisitor.cpp index f4305692eb6..fe0ee6f97ac 100644 --- a/src/Interpreters/RequiredSourceColumnsVisitor.cpp +++ b/src/Interpreters/RequiredSourceColumnsVisitor.cpp @@ -52,10 +52,8 @@ bool RequiredSourceColumnsMatcher::needChildVisit(const ASTPtr & node, const AST if (const auto * f = node->as()) { - /// "indexHint" is a special function for index analysis. - /// Everything that is inside it is not calculated. See KeyCondition /// "lambda" visit children itself. - if (f->name == "indexHint" || f->name == "lambda") + if (f->name == "lambda") return false; } @@ -73,6 +71,11 @@ void RequiredSourceColumnsMatcher::visit(const ASTPtr & ast, Data & data) } if (auto * t = ast->as()) { + /// "indexHint" is a special function for index analysis. + /// Everything that is inside it is not calculated. See KeyCondition + if (!data.visit_index_hint && t->name == "indexHint") + return; + data.addColumnAliasIfAny(*ast); visit(*t, ast, data); return; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index b389c3eb705..53d015f79a8 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -963,12 +963,13 @@ void TreeRewriterResult::collectSourceColumns(bool add_special) /// Calculate which columns are required to execute the expression. /// Then, delete all other columns from the list of available columns. /// After execution, columns will only contain the list of columns needed to read from the table. -void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select) +void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint) { /// We calculate required_source_columns with source_columns modifications and swap them on exit required_source_columns = source_columns; RequiredSourceColumnsVisitor::Data columns_context; + columns_context.visit_index_hint = visit_index_hint; RequiredSourceColumnsVisitor(columns_context).visit(query); NameSet source_column_names; @@ -1285,7 +1286,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( result.aggregates = getAggregates(query, *select_query); result.window_function_asts = getWindowFunctions(query, *select_query); result.expressions_with_window_function = getExpressionsWithWindowFunctions(query); - result.collectUsedColumns(query, true); + result.collectUsedColumns(query, true, settings.query_plan_optimize_primary_key); result.required_source_columns_before_expanding_alias_columns = result.required_source_columns.getNames(); /// rewrite filters for select query, must go after getArrayJoinedColumns @@ -1309,7 +1310,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( result.aggregates = getAggregates(query, *select_query); result.window_function_asts = getWindowFunctions(query, *select_query); result.expressions_with_window_function = getExpressionsWithWindowFunctions(query); - result.collectUsedColumns(query, true); + result.collectUsedColumns(query, true, settings.query_plan_optimize_primary_key); } } @@ -1375,7 +1376,7 @@ TreeRewriterResultPtr TreeRewriter::analyze( else assertNoAggregates(query, "in wrong place"); - result.collectUsedColumns(query, false); + result.collectUsedColumns(query, false, settings.query_plan_optimize_primary_key); return std::make_shared(result); } diff --git a/src/Interpreters/TreeRewriter.h b/src/Interpreters/TreeRewriter.h index b84756260a8..16ff7f8b6c3 100644 --- a/src/Interpreters/TreeRewriter.h +++ b/src/Interpreters/TreeRewriter.h @@ -88,7 +88,7 @@ struct TreeRewriterResult bool add_special = true); void collectSourceColumns(bool add_special); - void collectUsedColumns(const ASTPtr & query, bool is_select); + void collectUsedColumns(const ASTPtr & query, bool is_select, bool visit_index_hint); Names requiredSourceColumns() const { return required_source_columns.getNames(); } const Names & requiredSourceColumnsForAccessCheck() const { return required_source_columns_before_expanding_alias_columns; } NameSet getArrayJoinSourceNameSet() const; From 04fd72cdef67c72cf79dad618daf4ca4d7b16b29 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 28 Jul 2022 13:10:43 +0000 Subject: [PATCH 048/117] fix alter column rename --- src/Storages/MergeTree/IMergeTreeDataPart.h | 3 +- src/Storages/MergeTree/IMergeTreeReader.cpp | 2 + src/Storages/MergeTree/IMergeTreeReader.h | 3 + .../MergeTree/MergeTreeDataPartWriterWide.cpp | 2 +- src/Storages/MergeTree/MutateTask.cpp | 91 ++++++++++++------- 5 files changed, 66 insertions(+), 35 deletions(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.h b/src/Storages/MergeTree/IMergeTreeDataPart.h index bf4eba1c354..010a97625d1 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.h +++ b/src/Storages/MergeTree/IMergeTreeDataPart.h @@ -524,9 +524,10 @@ private: /// Map from name of column to its serialization info. SerializationInfoByName serialization_infos; + /// Serializations for every columns and subcolumns by their names. SerializationByName serializations; - /// Columns description for more convenient access + /// Columns description for more convenient access /// to columns by name and getting subcolumns. ColumnsDescription columns_description; diff --git a/src/Storages/MergeTree/IMergeTreeReader.cpp b/src/Storages/MergeTree/IMergeTreeReader.cpp index 30969939622..8c861248580 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.cpp +++ b/src/Storages/MergeTree/IMergeTreeReader.cpp @@ -40,6 +40,8 @@ IMergeTreeReader::IMergeTreeReader( , metadata_snapshot(metadata_snapshot_) , all_mark_ranges(all_mark_ranges_) , alter_conversions(storage.getAlterConversionsForPart(data_part)) + /// For wide parts convert plain arrays of Nested to subcolumns + /// to allow to use shared offset column from cache. , requested_columns(isWidePart(data_part) ? Nested::convertToSubcolumns(columns_) : columns_) , part_columns(isWidePart(data_part) ? Nested::collect(data_part->getColumns()) : data_part->getColumns()) { diff --git a/src/Storages/MergeTree/IMergeTreeReader.h b/src/Storages/MergeTree/IMergeTreeReader.h index 0d6c0e607cd..453563522a5 100644 --- a/src/Storages/MergeTree/IMergeTreeReader.h +++ b/src/Storages/MergeTree/IMergeTreeReader.h @@ -75,7 +75,10 @@ protected: /// Stores states for IDataType::deserializeBinaryBulk DeserializeBinaryBulkStateMap deserialize_binary_bulk_state_map; + /// Actual column names and types of columns in part, + /// which may differ from table metadata. NamesAndTypes columns_to_read; + /// Actual serialization of columns in part. Serializations serializations; UncompressedCache * uncompressed_cache; diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 712c3f74bdd..3d4aa0a7707 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -583,7 +583,7 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(bool sync) { if (column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes() - && data_part->getSerialization(columnn.name)->getKind() == ISerialization::Kind::DEFAULT) + && data_part->getSerialization(column.name)->getKind() == ISerialization::Kind::DEFAULT) { validateColumnOfFixedSize(column); } diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 5b54c1e6ae7..9b41c7bc623 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -459,8 +459,21 @@ static NameToNameVector collectFilesForRenames( const MutationCommands & commands_for_removes, const String & mrk_extension) { + /// Collect counts for shared streams of different columns. As an example, Nested columns have shared stream with array sizes. + std::unordered_map stream_counts; + for (const auto & column : source_part->getColumns()) + { + if (auto serialization = source_part->tryGetSerialization(column.name)) + { + serialization->enumerateStreams( + [&](const ISerialization::SubstreamPath & substream_path) + { + ++stream_counts[ISerialization::getFileNameForStream(column, substream_path)]; + }); + } + } + NameToNameVector rename_vector; - NameSet renamed_streams; /// Remove old data for (const auto & command : commands_for_removes) @@ -483,6 +496,22 @@ static NameToNameVector collectFilesForRenames( if (source_part->checksums.has(command.column_name + ".proj")) rename_vector.emplace_back(command.column_name + ".proj", ""); } + else if (command.type == MutationCommand::Type::DROP_COLUMN) + { + ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path) + { + String stream_name = ISerialization::getFileNameForStream({command.column_name, command.data_type}, substream_path); + /// Delete files if they are no longer shared with another column. + if (--stream_counts[stream_name] == 0) + { + rename_vector.emplace_back(stream_name + ".bin", ""); + rename_vector.emplace_back(stream_name + mrk_extension, ""); + } + }; + + if (auto serialization = source_part->tryGetSerialization(command.column_name)) + serialization->enumerateStreams(callback); + } else if (command.type == MutationCommand::Type::RENAME_COLUMN) { String escaped_name_from = escapeForFileName(command.column_name); @@ -495,7 +524,6 @@ static NameToNameVector collectFilesForRenames( if (stream_from != stream_to) { - renamed_streams.insert(stream_from); rename_vector.emplace_back(stream_from + ".bin", stream_to + ".bin"); rename_vector.emplace_back(stream_from + mrk_extension, stream_to + mrk_extension); } @@ -504,41 +532,38 @@ static NameToNameVector collectFilesForRenames( if (auto serialization = source_part->tryGetSerialization(command.column_name)) serialization->enumerateStreams(callback); } - } - - auto collect_all_stream_names = [&](const auto & data_part) - { - NameSet res; - for (const auto & column : data_part->getColumns()) + else if (command.type == MutationCommand::Type::READ_COLUMN) { - if (auto serialization = data_part->tryGetSerialization(column.name)) + /// Remove files for streams that exist in source_part, + /// but were removed in new_part by MODIFY COLUMN from + /// type with higher number of streams (e.g. LowCardinality -> String). + + auto collect_stream_names = [&](const auto & data_part) { - serialization->enumerateStreams( - [&](const ISerialization::SubstreamPath & substream_path) - { - res.insert(ISerialization::getFileNameForStream(column.name, substream_path)); - }); + NameSet res; + if (auto serialization = data_part->tryGetSerialization(command.column_name)) + { + serialization->enumerateStreams( + [&](const ISerialization::SubstreamPath & substream_path) + { + res.insert(ISerialization::getFileNameForStream(command.column_name, substream_path)); + }); + } + return res; + }; + + auto old_streams = collect_stream_names(source_part); + auto new_streams = collect_stream_names(new_part); + + for (const auto & old_stream : old_streams) + { + if (!new_streams.contains(old_stream)) + { + rename_vector.emplace_back(old_stream + ".bin", ""); + rename_vector.emplace_back(old_stream + mrk_extension, ""); + } } } - - return res; - }; - - /// Remove files for streams that exists in source part, - /// but were removed in new_part by DROP COLUMN - /// or MODIFY COLUMN from type with higher number of streams - /// (e.g. LowCardinality -> String). - - auto old_streams = collect_all_stream_names(source_part); - auto new_streams = collect_all_stream_names(new_part); - - for (const auto & old_stream : old_streams) - { - if (!new_streams.contains(old_stream) && !renamed_streams.contains(old_stream)) - { - rename_vector.emplace_back(old_stream + ".bin", ""); - rename_vector.emplace_back(old_stream + mrk_extension, ""); - } } return rename_vector; From d9c585ecf17618c81a73a19cb59dd78394d4b594 Mon Sep 17 00:00:00 2001 From: Dale Mcdiarmid Date: Fri, 29 Jul 2022 12:25:03 +0100 Subject: [PATCH 049/117] Escape credentials for diagnostics tool --- programs/diagnostics/internal/platform/database/native.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/programs/diagnostics/internal/platform/database/native.go b/programs/diagnostics/internal/platform/database/native.go index e512a634fbf..45b9af0349e 100644 --- a/programs/diagnostics/internal/platform/database/native.go +++ b/programs/diagnostics/internal/platform/database/native.go @@ -3,6 +3,7 @@ package database import ( "database/sql" "fmt" + "net/url" "strings" "github.com/ClickHouse/ClickHouse/programs/diagnostics/internal/platform/data" @@ -17,7 +18,7 @@ type ClickhouseNativeClient struct { func NewNativeClient(host string, port uint16, username string, password string) (*ClickhouseNativeClient, error) { // debug output ?debug=true - connection, err := sql.Open("clickhouse", fmt.Sprintf("clickhouse://%s:%s@%s:%d/", username, password, host, port)) + connection, err := sql.Open("clickhouse", fmt.Sprintf("clickhouse://%s:%s@%s:%d/", url.QueryEscape(username), url.QueryEscape(password), host, port)) if err != nil { return &ClickhouseNativeClient{}, err } From bb551566e7c29430339a7d8fef5e4a2ee670c4c4 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 29 Jul 2022 11:41:53 +0000 Subject: [PATCH 050/117] fix build --- src/Storages/MergeTree/IMergeTreeDataPart.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/IMergeTreeDataPart.cpp b/src/Storages/MergeTree/IMergeTreeDataPart.cpp index 30ec9bbcdd4..16ffc15d292 100644 --- a/src/Storages/MergeTree/IMergeTreeDataPart.cpp +++ b/src/Storages/MergeTree/IMergeTreeDataPart.cpp @@ -1016,7 +1016,7 @@ void IMergeTreeDataPart::loadRowsCount() /// Most trivial types if (column.type->isValueRepresentedByNumber() && !column.type->haveSubtypes() - && getSerialization(column)->getKind() == ISerialization::Kind::DEFAULT) + && getSerialization(column.name)->getKind() == ISerialization::Kind::DEFAULT) { auto size = getColumnSize(column.name); From ae7fd5bf937e350af37a25c92e1c6c3960989117 Mon Sep 17 00:00:00 2001 From: Anton Kozlov Date: Wed, 27 Jul 2022 16:57:56 +0000 Subject: [PATCH 051/117] Default database resolution in distributed reads --- src/Server/TCPHandler.cpp | 7 +++ .../__init__.py | 0 .../configs/remote_servers.xml | 18 +++++++ .../configs/users.xml | 24 +++++++++ .../test_distributed_default_database/test.py | 51 +++++++++++++++++++ 5 files changed, 100 insertions(+) create mode 100644 tests/integration/test_distributed_default_database/__init__.py create mode 100644 tests/integration/test_distributed_default_database/configs/remote_servers.xml create mode 100644 tests/integration/test_distributed_default_database/configs/users.xml create mode 100644 tests/integration/test_distributed_default_database/test.py diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 05565063893..ed46abd9c36 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -760,6 +760,13 @@ void TCPHandler::processTablesStatusRequest() request.read(*in, client_tcp_protocol_version); ContextPtr context_to_resolve_table_names = (session && session->sessionContext()) ? session->sessionContext() : server.context(); + if (is_interserver_mode && !default_database.empty()) + { + // in interserver mode, the session doesn't exist. + ContextMutablePtr interserver_context = Context::createCopy(context_to_resolve_table_names); + interserver_context->setCurrentDatabase(default_database); + context_to_resolve_table_names = interserver_context; + } TablesStatusResponse response; for (const QualifiedTableName & table_name: request.tables) diff --git a/tests/integration/test_distributed_default_database/__init__.py b/tests/integration/test_distributed_default_database/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_distributed_default_database/configs/remote_servers.xml b/tests/integration/test_distributed_default_database/configs/remote_servers.xml new file mode 100644 index 00000000000..58641d93fc6 --- /dev/null +++ b/tests/integration/test_distributed_default_database/configs/remote_servers.xml @@ -0,0 +1,18 @@ + + + + foo + + node1 + 9000 + r0 + + + node2 + 9000 + r1 + + + + + diff --git a/tests/integration/test_distributed_default_database/configs/users.xml b/tests/integration/test_distributed_default_database/configs/users.xml new file mode 100644 index 00000000000..a49b1363433 --- /dev/null +++ b/tests/integration/test_distributed_default_database/configs/users.xml @@ -0,0 +1,24 @@ + + + + + false + + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/tests/integration/test_distributed_default_database/test.py b/tests/integration/test_distributed_default_database/test.py new file mode 100644 index 00000000000..ef69533416b --- /dev/null +++ b/tests/integration/test_distributed_default_database/test.py @@ -0,0 +1,51 @@ +""" +This test is similar to test_cross_replication, except +in this test we write into per-node tables and read from the distributed table. + +The default database in the distributed table definition is left empty on purpose to test +default database deduction. +""" +import pytest + +from helpers.client import QueryRuntimeException +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV + +from contextlib import contextmanager + + +def bootstrap(cluster): + for i, node in enumerate(list(cluster.instances.values())): + node.query(f"CREATE DATABASE IF NOT EXISTS r{i}") + node.query(f"CREATE TABLE r{i}.test_data(v UInt64) ENGINE = Memory()") + node.query(f"INSERT INTO r{i}.test_data SELECT * FROM numbers(10)") + node.query( + f"""CREATE TABLE default.test AS r{i}.test_data ENGINE = Distributed(secure, '', test_data, rand())""" + ) + + +@contextmanager +def start_cluster(): + cluster_disabled = ClickHouseCluster(__file__) + cluster_disabled.add_instance( + "node1", + main_configs=["configs/remote_servers.xml"], + user_configs=["configs/users.xml"], + ) + cluster_disabled.add_instance( + "node2", + main_configs=["configs/remote_servers.xml"], + user_configs=["configs/users.xml"], + ) + try: + cluster_disabled.start() + bootstrap(cluster_disabled) + yield cluster_disabled + finally: + cluster_disabled.shutdown() + + +def test_query(): + with start_cluster() as cluster: + node1 = cluster.instances["node1"] + assert TSV(node1.query("SELECT count() FROM default.test")) == TSV("20") From f5f6c746487dadaa321ce30aa455c4694c843a4a Mon Sep 17 00:00:00 2001 From: kssenii Date: Fri, 29 Jul 2022 22:00:21 +0300 Subject: [PATCH 052/117] One more update --- contrib/azure | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure b/contrib/azure index 9030dea3088..ef75afc075f 160000 --- a/contrib/azure +++ b/contrib/azure @@ -1 +1 @@ -Subproject commit 9030dea30881e561b06dc80e0453b9b2a593cdbe +Subproject commit ef75afc075fc71fbcd8fe28dcda3794ae265fd1c From cc73b531168f869b19000c189d832a6e0f008537 Mon Sep 17 00:00:00 2001 From: HarryLeeIBM Date: Fri, 29 Jul 2022 13:21:50 -0700 Subject: [PATCH 053/117] Fix Endian issue in SipHash for s390x --- base/base/unaligned.h | 32 ++++++++++++++++++++++++++++++++ src/Common/SipHash.h | 6 +++--- 2 files changed, 35 insertions(+), 3 deletions(-) diff --git a/base/base/unaligned.h b/base/base/unaligned.h index 013aa5274e9..7fff0ddb20c 100644 --- a/base/base/unaligned.h +++ b/base/base/unaligned.h @@ -2,7 +2,39 @@ #include #include +#include +inline void reverseMemcpy(void *dst, const void *src, int length) +{ + uint8_t *d = reinterpret_cast(dst); + const uint8_t *s = reinterpret_cast(src); + + d += length; + while (length--) + *--d = *s++; +} + +template +inline T unalignedLoadLE(const void * address) +{ + T res {}; + if constexpr (std::endian::native == std::endian::little) + memcpy(&res, address, sizeof(res)); + else + reverseMemcpy(&res, address, sizeof(res)); + return res; +} + +template +inline void unalignedStoreLE(void * address, + const typename std::enable_if::type & src) +{ + static_assert(std::is_trivially_copyable_v); + if constexpr (std::endian::native == std::endian::little) + memcpy(address, &src, sizeof(src)); + else + reverseMemcpy(address, &src, sizeof(src)); +} template inline T unalignedLoad(const void * address) diff --git a/src/Common/SipHash.h b/src/Common/SipHash.h index 38188a022f8..1f26cba31fb 100644 --- a/src/Common/SipHash.h +++ b/src/Common/SipHash.h @@ -111,7 +111,7 @@ public: while (data + 8 <= end) { - current_word = unalignedLoad(data); + current_word = unalignedLoadLE(data); v3 ^= current_word; SIPROUND; @@ -157,8 +157,8 @@ public: void get128(char * out) { finalize(); - unalignedStore(out, v0 ^ v1); - unalignedStore(out + 8, v2 ^ v3); + unalignedStoreLE(out, v0 ^ v1); + unalignedStoreLE(out + 8, v2 ^ v3); } template From 7cdad883278f9764b6888ec2d7d7dc5d7967f75a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sat, 30 Jul 2022 11:34:17 +0000 Subject: [PATCH 054/117] Create snapshot on shutdown --- src/Coordination/KeeperContext.h | 3 ++- src/Coordination/KeeperServer.cpp | 9 ++++++++- src/Coordination/KeeperServer.h | 2 ++ src/Coordination/KeeperStateMachine.cpp | 8 +++++++- 4 files changed, 19 insertions(+), 3 deletions(-) diff --git a/src/Coordination/KeeperContext.h b/src/Coordination/KeeperContext.h index 84ec65cecde..64fa8cea6ec 100644 --- a/src/Coordination/KeeperContext.h +++ b/src/Coordination/KeeperContext.h @@ -8,7 +8,8 @@ struct KeeperContext enum class Phase : uint8_t { INIT, - RUNNING + RUNNING, + SHUTDOWN }; Phase server_state{Phase::INIT}; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 587ab9c8f66..20ce7e42afc 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -107,8 +107,9 @@ KeeperServer::KeeperServer( : server_id(configuration_and_settings_->server_id) , coordination_settings(configuration_and_settings_->coordination_settings) , log(&Poco::Logger::get("KeeperServer")) - , is_recovering(config.has("keeper_server.force_recovery") && config.getBool("keeper_server.force_recovery")) + , is_recovering(config.getBool("keeper_server.force_recovery", false)) , keeper_context{std::make_shared()} + , create_snapshot_on_exit(config.getBool("keeper_server.create_snapshot_on_exit", true)) { if (coordination_settings->quorum_reads) LOG_WARNING(log, "Quorum reads enabled, Keeper will work slower."); @@ -367,6 +368,12 @@ void KeeperServer::shutdownRaftServer() } raft_instance->shutdown(); + + keeper_context->server_state = KeeperContext::Phase::SHUTDOWN; + + if (create_snapshot_on_exit) + raft_instance->create_snapshot(); + raft_instance.reset(); if (asio_listener) diff --git a/src/Coordination/KeeperServer.h b/src/Coordination/KeeperServer.h index 74dd05631f0..1fb3e579214 100644 --- a/src/Coordination/KeeperServer.h +++ b/src/Coordination/KeeperServer.h @@ -64,6 +64,8 @@ private: std::shared_ptr keeper_context; + const bool create_snapshot_on_exit; + public: KeeperServer( const KeeperConfigurationAndSettingsPtr & settings_, diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index f43a3dbb319..a55acaf9b91 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -383,7 +383,13 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res }; - LOG_DEBUG(log, "In memory snapshot {} created, queueing task to flash to disk", s.get_last_log_idx()); + if (keeper_context->server_state == KeeperContext::Phase::SHUTDOWN) + { + snapshot_task.create_snapshot(std::move(snapshot_task.snapshot)); + return; + } + + LOG_DEBUG(log, "In memory snapshot {} created, queueing task to flush to disk", s.get_last_log_idx()); /// Flush snapshot to disk in a separate thread. if (!snapshots_queue.push(std::move(snapshot_task))) LOG_WARNING(log, "Cannot push snapshot task into queue"); From 16a98d8eeff0820e653f37b5e6d488c2184508f3 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sun, 31 Jul 2022 09:11:46 +0000 Subject: [PATCH 055/117] Add test for creating snapshot on exit --- .../configs/enable_keeper1.xml | 28 +++++++++++ .../configs/enable_keeper2.xml | 28 +++++++++++ .../test_keeper_snapshot_on_exit/test.py | 50 +++++++++++++++++++ 3 files changed, 106 insertions(+) create mode 100644 tests/integration/test_keeper_snapshot_on_exit/configs/enable_keeper1.xml create mode 100644 tests/integration/test_keeper_snapshot_on_exit/configs/enable_keeper2.xml create mode 100644 tests/integration/test_keeper_snapshot_on_exit/test.py diff --git a/tests/integration/test_keeper_snapshot_on_exit/configs/enable_keeper1.xml b/tests/integration/test_keeper_snapshot_on_exit/configs/enable_keeper1.xml new file mode 100644 index 00000000000..e4d5ae9aa82 --- /dev/null +++ b/tests/integration/test_keeper_snapshot_on_exit/configs/enable_keeper1.xml @@ -0,0 +1,28 @@ + + + 9181 + 1 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + true + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + + + + diff --git a/tests/integration/test_keeper_snapshot_on_exit/configs/enable_keeper2.xml b/tests/integration/test_keeper_snapshot_on_exit/configs/enable_keeper2.xml new file mode 100644 index 00000000000..216e6905a56 --- /dev/null +++ b/tests/integration/test_keeper_snapshot_on_exit/configs/enable_keeper2.xml @@ -0,0 +1,28 @@ + + + 9181 + 2 + /var/lib/clickhouse/coordination/log + /var/lib/clickhouse/coordination/snapshots + false + + + 5000 + 10000 + trace + + + + + 1 + node1 + 9234 + + + 2 + node2 + 9234 + + + + diff --git a/tests/integration/test_keeper_snapshot_on_exit/test.py b/tests/integration/test_keeper_snapshot_on_exit/test.py new file mode 100644 index 00000000000..ad373011136 --- /dev/null +++ b/tests/integration/test_keeper_snapshot_on_exit/test.py @@ -0,0 +1,50 @@ +import pytest +from helpers.cluster import ClickHouseCluster +import os +from kazoo.client import KazooClient + +cluster = ClickHouseCluster(__file__) +CONFIG_DIR = os.path.join(os.path.dirname(os.path.realpath(__file__)), "configs") + +node1 = cluster.add_instance( + "node1", main_configs=["configs/enable_keeper1.xml"], stay_alive=True +) + +node2 = cluster.add_instance( + "node2", main_configs=["configs/enable_keeper2.xml"], stay_alive=True +) + +def get_fake_zk(node, timeout=30.0): + _fake_zk_instance = KazooClient( + hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout + ) + _fake_zk_instance.start() + return _fake_zk_instance + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + yield cluster + + finally: + cluster.shutdown() + +def test_snapshot_on_exit(started_cluster): + zk_conn = get_fake_zk(node1); + + zk_conn.create("/some_path", b"some_data") + + node1.stop_clickhouse() + assert node1.contains_in_log("Created persistent snapshot") + + node1.start_clickhouse() + assert node1.contains_in_log("Loaded snapshot") + + node2.stop_clickhouse() + assert not node2.contains_in_log("Created persistent snapshot") + + node2.start_clickhouse() + assert node2.contains_in_log("No existing snapshots") From 0b1554c2e76a439918c535bc278848c206649350 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Sun, 31 Jul 2022 09:24:53 +0000 Subject: [PATCH 056/117] Automatic style fix --- tests/integration/test_keeper_snapshot_on_exit/test.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_keeper_snapshot_on_exit/test.py b/tests/integration/test_keeper_snapshot_on_exit/test.py index ad373011136..1ca5888ab4d 100644 --- a/tests/integration/test_keeper_snapshot_on_exit/test.py +++ b/tests/integration/test_keeper_snapshot_on_exit/test.py @@ -14,6 +14,7 @@ node2 = cluster.add_instance( "node2", main_configs=["configs/enable_keeper2.xml"], stay_alive=True ) + def get_fake_zk(node, timeout=30.0): _fake_zk_instance = KazooClient( hosts=cluster.get_instance_ip(node.name) + ":9181", timeout=timeout @@ -32,8 +33,9 @@ def started_cluster(): finally: cluster.shutdown() + def test_snapshot_on_exit(started_cluster): - zk_conn = get_fake_zk(node1); + zk_conn = get_fake_zk(node1) zk_conn.create("/some_path", b"some_data") From 687f0fabfc162e1f61815930a5aebcf7a6428048 Mon Sep 17 00:00:00 2001 From: kssenii Date: Sun, 31 Jul 2022 20:16:15 +0300 Subject: [PATCH 057/117] Move some dependencies --- src/CMakeLists.txt | 3 +- src/Disks/DiskWebServer.cpp | 15 +- .../IO/ReadBufferFromAzureBlobStorage.cpp | 2 +- .../IO/ReadBufferFromAzureBlobStorage.h | 0 src/Disks/IO/ReadBufferFromRemoteFSGather.cpp | 142 ++++------------- src/Disks/IO/ReadBufferFromRemoteFSGather.h | 144 +++--------------- src/Disks/IO/ReadBufferFromWebServer.cpp | 2 +- src/Disks/IO/ReadBufferFromWebServer.h | 6 +- .../IO/WriteBufferFromAzureBlobStorage.cpp | 2 +- .../IO/WriteBufferFromAzureBlobStorage.h | 0 .../IO/WriteIndirectBufferFromRemoteFS.cpp | 6 - .../AzureBlobStorage/AzureObjectStorage.cpp | 25 ++- .../AzureBlobStorage/AzureObjectStorage.h | 3 + .../ObjectStorages/HDFS/HDFSObjectStorage.cpp | 14 +- .../ObjectStorages/S3/S3ObjectStorage.cpp | 24 ++- 15 files changed, 124 insertions(+), 264 deletions(-) rename src/{ => Disks}/IO/ReadBufferFromAzureBlobStorage.cpp (98%) rename src/{ => Disks}/IO/ReadBufferFromAzureBlobStorage.h (100%) rename src/{ => Disks}/IO/WriteBufferFromAzureBlobStorage.cpp (97%) rename src/{ => Disks}/IO/WriteBufferFromAzureBlobStorage.h (100%) diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 49acf55892a..26f3f44e110 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -488,7 +488,7 @@ if (TARGET ch_contrib::aws_s3) endif() if (TARGET ch_contrib::azure_sdk) - target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::azure_sdk) + dbms_target_link_libraries (PRIVATE ch_contrib::azure_sdk) endif() if (TARGET ch_contrib::s2) @@ -611,4 +611,3 @@ if (ENABLE_TESTS) add_check(unit_tests_dbms) endif () - diff --git a/src/Disks/DiskWebServer.cpp b/src/Disks/DiskWebServer.cpp index b6cda8288d7..74e96cb5e30 100644 --- a/src/Disks/DiskWebServer.cpp +++ b/src/Disks/DiskWebServer.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -172,7 +173,19 @@ std::unique_ptr DiskWebServer::readFile(const String & p StoredObjects objects; objects.emplace_back(remote_path, iter->second.size); - auto web_impl = std::make_unique(url, objects, getContext(), read_settings); + auto read_buffer_creator = + [this, read_settings] + (const std::string & path_, size_t read_until_position) -> std::shared_ptr + { + return std::make_shared( + fs::path(url) / path_, + getContext(), + read_settings, + /* use_external_buffer */true, + read_until_position); + }; + + auto web_impl = std::make_unique(std::move(read_buffer_creator), objects, read_settings); if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) { diff --git a/src/IO/ReadBufferFromAzureBlobStorage.cpp b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp similarity index 98% rename from src/IO/ReadBufferFromAzureBlobStorage.cpp rename to src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp index 3e6581cd786..069d0aed312 100644 --- a/src/IO/ReadBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/ReadBufferFromAzureBlobStorage.cpp @@ -2,7 +2,7 @@ #if USE_AZURE_BLOB_STORAGE -#include +#include #include #include #include diff --git a/src/IO/ReadBufferFromAzureBlobStorage.h b/src/Disks/IO/ReadBufferFromAzureBlobStorage.h similarity index 100% rename from src/IO/ReadBufferFromAzureBlobStorage.h rename to src/Disks/IO/ReadBufferFromAzureBlobStorage.h diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp index 14614871185..3ac4ea07945 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.cpp @@ -1,28 +1,13 @@ #include "ReadBufferFromRemoteFSGather.h" #include -#include - -#if USE_AWS_S3 -#include -#endif - -#if USE_AZURE_BLOB_STORAGE -#include -#endif - -#if USE_HDFS -#include -#endif #include #include -#include #include #include #include - -namespace fs = std::filesystem; +#include namespace DB @@ -33,97 +18,12 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(const String & path, size_t file_size) -{ - if (!current_file_path.empty() && !with_cache && enable_cache_log) - { - appendFilesystemCacheLog(); - } - - current_file_path = path; - current_file_size = file_size; - total_bytes_read_from_current_file = 0; - - return createImplementationBufferImpl(path, file_size); -} - -#if USE_AWS_S3 -SeekableReadBufferPtr ReadBufferFromS3Gather::createImplementationBufferImpl(const String & path, size_t file_size) -{ - auto remote_file_reader_creator = [=, this]() - { - return std::make_unique( - client_ptr, - bucket, - path, - version_id, - max_single_read_retries, - settings, - /* use_external_buffer */true, - /* offset */0, - read_until_position, - /* restricted_seek */true); - }; - - if (with_cache) - { - return std::make_shared( - path, - settings.remote_fs_cache, - remote_file_reader_creator, - settings, - query_id, - read_until_position ? read_until_position : file_size); - } - - return remote_file_reader_creator(); -} -#endif - - -#if USE_AZURE_BLOB_STORAGE -SeekableReadBufferPtr ReadBufferFromAzureBlobStorageGather::createImplementationBufferImpl(const String & path, size_t /* file_size */) -{ - return std::make_unique( - blob_container_client, - path, - settings, - max_single_read_retries, - max_single_download_retries, - /* use_external_buffer */true, - read_until_position); -} -#endif - - -SeekableReadBufferPtr ReadBufferFromWebServerGather::createImplementationBufferImpl(const String & path, size_t /* file_size */) -{ - return std::make_unique( - fs::path(uri) / path, - context, - settings, - /* use_external_buffer */true, - read_until_position); -} - - -#if USE_HDFS -SeekableReadBufferPtr ReadBufferFromHDFSGather::createImplementationBufferImpl(const String & path, size_t /* file_size */) -{ - size_t begin_of_path = path.find('/', path.find("//") + 2); - auto hdfs_path = path.substr(begin_of_path); - auto hdfs_uri = path.substr(0, begin_of_path); - LOG_TEST(log, "HDFS uri: {}, path: {}", hdfs_path, hdfs_uri); - - return std::make_unique(hdfs_uri, hdfs_path, config, settings); -} -#endif - - ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( + ReadBufferCreator && read_buffer_creator_, const StoredObjects & blobs_to_read_, const ReadSettings & settings_) : ReadBuffer(nullptr, 0) + , read_buffer_creator(std::move(read_buffer_creator_)) , blobs_to_read(blobs_to_read_) , settings(settings_) , query_id(CurrentThread::isInitialized() && CurrentThread::get().getQueryContext() != nullptr ? CurrentThread::getQueryId() : "") @@ -138,6 +38,33 @@ ReadBufferFromRemoteFSGather::ReadBufferFromRemoteFSGather( && (!IFileCache::isReadOnly() || settings.read_from_filesystem_cache_if_exists_otherwise_bypass_cache); } +SeekableReadBufferPtr ReadBufferFromRemoteFSGather::createImplementationBuffer(const String & path, size_t file_size) +{ + if (!current_file_path.empty() && !with_cache && enable_cache_log) + { + appendFilesystemCacheLog(); + } + + current_file_path = path; + current_file_size = file_size; + total_bytes_read_from_current_file = 0; + + size_t current_read_until_position = read_until_position ? read_until_position : file_size; + auto current_read_buffer_creator = [path, current_read_until_position, this]() { return read_buffer_creator(path, current_read_until_position); }; + + if (with_cache) + { + return std::make_shared( + path, + settings.remote_fs_cache, + current_read_buffer_creator, + settings, + query_id, + current_read_until_position); + } + + return current_read_buffer_creator(); +} void ReadBufferFromRemoteFSGather::appendFilesystemCacheLog() { @@ -156,7 +83,6 @@ void ReadBufferFromRemoteFSGather::appendFilesystemCacheLog() cache_log->add(elem); } - IAsynchronousReader::Result ReadBufferFromRemoteFSGather::readInto(char * data, size_t size, size_t offset, size_t ignore) { /** @@ -178,7 +104,6 @@ IAsynchronousReader::Result ReadBufferFromRemoteFSGather::readInto(char * data, return {0, 0}; } - void ReadBufferFromRemoteFSGather::initialize() { /// One clickhouse file can be split into multiple files in remote fs. @@ -206,7 +131,6 @@ void ReadBufferFromRemoteFSGather::initialize() current_buf = nullptr; } - bool ReadBufferFromRemoteFSGather::nextImpl() { /// Find first available buffer that fits to given offset. @@ -232,7 +156,6 @@ bool ReadBufferFromRemoteFSGather::nextImpl() return readImpl(); } - bool ReadBufferFromRemoteFSGather::moveToNextBuffer() { /// If there is no available buffers - nothing to read. @@ -247,7 +170,6 @@ bool ReadBufferFromRemoteFSGather::moveToNextBuffer() return true; } - bool ReadBufferFromRemoteFSGather::readImpl() { swap(*current_buf); @@ -294,13 +216,11 @@ bool ReadBufferFromRemoteFSGather::readImpl() return result; } - size_t ReadBufferFromRemoteFSGather::getFileOffsetOfBufferEnd() const { return file_offset_of_buffer_end; } - void ReadBufferFromRemoteFSGather::setReadUntilPosition(size_t position) { if (position != read_until_position) @@ -310,7 +230,6 @@ void ReadBufferFromRemoteFSGather::setReadUntilPosition(size_t position) } } - void ReadBufferFromRemoteFSGather::reset() { current_buf.reset(); @@ -321,7 +240,6 @@ String ReadBufferFromRemoteFSGather::getFileName() const return current_file_path; } - size_t ReadBufferFromRemoteFSGather::getFileSize() const { size_t size = 0; diff --git a/src/Disks/IO/ReadBufferFromRemoteFSGather.h b/src/Disks/IO/ReadBufferFromRemoteFSGather.h index e065c714e04..4ed61501281 100644 --- a/src/Disks/IO/ReadBufferFromRemoteFSGather.h +++ b/src/Disks/IO/ReadBufferFromRemoteFSGather.h @@ -6,12 +6,6 @@ #include #include -#if USE_AZURE_BLOB_STORAGE -#include -#endif - -namespace Aws { namespace S3 { class S3Client; } } - namespace Poco { class Logger; } namespace DB @@ -21,12 +15,15 @@ namespace DB * Remote disk might need to split one clickhouse file into multiple files in remote fs. * This class works like a proxy to allow transition from one file into multiple. */ -class ReadBufferFromRemoteFSGather : public ReadBuffer +class ReadBufferFromRemoteFSGather final : public ReadBuffer { friend class ReadIndirectBufferFromRemoteFS; public: + using ReadBufferCreator = std::function(const std::string & path, size_t read_until_position)>; + ReadBufferFromRemoteFSGather( + ReadBufferCreator && read_buffer_creator_, const StoredObjects & blobs_to_read_, const ReadSettings & settings_); @@ -50,8 +47,20 @@ public: size_t getImplementationBufferOffset() const; -protected: - virtual SeekableReadBufferPtr createImplementationBufferImpl(const String & path, size_t file_size) = 0; +private: + SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t file_size); + + bool nextImpl() override; + + void initialize(); + + bool readImpl(); + + bool moveToNextBuffer(); + + void appendFilesystemCacheLog(); + + ReadBufferCreator read_buffer_creator; StoredObjects blobs_to_read; @@ -68,19 +77,6 @@ protected: Poco::Logger * log; -private: - SeekableReadBufferPtr createImplementationBuffer(const String & path, size_t file_size); - - bool nextImpl() override; - - void initialize(); - - bool readImpl(); - - bool moveToNextBuffer(); - - void appendFilesystemCacheLog(); - SeekableReadBufferPtr current_buf; size_t current_buf_idx = 0; @@ -99,108 +95,4 @@ private: bool enable_cache_log = false; }; - -#if USE_AWS_S3 -/// Reads data from S3 using stored paths in metadata. -class ReadBufferFromS3Gather final : public ReadBufferFromRemoteFSGather -{ -public: - ReadBufferFromS3Gather( - std::shared_ptr client_ptr_, - const String & bucket_, - const String & version_id_, - const StoredObjects & blobs_to_read_, - size_t max_single_read_retries_, - const ReadSettings & settings_) - : ReadBufferFromRemoteFSGather(blobs_to_read_, settings_) - , client_ptr(std::move(client_ptr_)) - , bucket(bucket_) - , version_id(version_id_) - , max_single_read_retries(max_single_read_retries_) - { - } - - SeekableReadBufferPtr createImplementationBufferImpl(const String & path, size_t file_size) override; - -private: - std::shared_ptr client_ptr; - String bucket; - String version_id; - UInt64 max_single_read_retries; -}; -#endif - - -#if USE_AZURE_BLOB_STORAGE -/// Reads data from AzureBlob Storage using paths stored in metadata. -class ReadBufferFromAzureBlobStorageGather final : public ReadBufferFromRemoteFSGather -{ -public: - ReadBufferFromAzureBlobStorageGather( - std::shared_ptr blob_container_client_, - const StoredObjects & blobs_to_read_, - size_t max_single_read_retries_, - size_t max_single_download_retries_, - const ReadSettings & settings_) - : ReadBufferFromRemoteFSGather(blobs_to_read_, settings_) - , blob_container_client(blob_container_client_) - , max_single_read_retries(max_single_read_retries_) - , max_single_download_retries(max_single_download_retries_) - { - } - - SeekableReadBufferPtr createImplementationBufferImpl(const String & path, size_t file_size) override; - -private: - std::shared_ptr blob_container_client; - size_t max_single_read_retries; - size_t max_single_download_retries; -}; -#endif - - -class ReadBufferFromWebServerGather final : public ReadBufferFromRemoteFSGather -{ -public: - ReadBufferFromWebServerGather( - const String & uri_, - const StoredObjects & blobs_to_read_, - ContextPtr context_, - const ReadSettings & settings_) - : ReadBufferFromRemoteFSGather(blobs_to_read_, settings_) - , uri(uri_) - , context(context_) - { - } - - SeekableReadBufferPtr createImplementationBufferImpl(const String & path, size_t file_size) override; - -private: - String uri; - ContextPtr context; -}; - - -#if USE_HDFS -/// Reads data from HDFS using stored paths in metadata. -class ReadBufferFromHDFSGather final : public ReadBufferFromRemoteFSGather -{ -public: - ReadBufferFromHDFSGather( - const Poco::Util::AbstractConfiguration & config_, - const StoredObjects & blobs_to_read_, - const ReadSettings & settings_) - : ReadBufferFromRemoteFSGather(blobs_to_read_, settings_) - , config(config_) - { - } - - SeekableReadBufferPtr createImplementationBufferImpl(const String & path, size_t file_size) override; - -private: - const Poco::Util::AbstractConfiguration & config; -}; - -#endif - } diff --git a/src/Disks/IO/ReadBufferFromWebServer.cpp b/src/Disks/IO/ReadBufferFromWebServer.cpp index b9614601a72..5e9d4f63807 100644 --- a/src/Disks/IO/ReadBufferFromWebServer.cpp +++ b/src/Disks/IO/ReadBufferFromWebServer.cpp @@ -27,7 +27,7 @@ ReadBufferFromWebServer::ReadBufferFromWebServer( const ReadSettings & settings_, bool use_external_buffer_, size_t read_until_position_) - : SeekableReadBuffer(nullptr, 0) + : ReadBufferFromFileBase(settings_.remote_fs_buffer_size, nullptr, 0) , log(&Poco::Logger::get("ReadBufferFromWebServer")) , context(context_) , url(url_) diff --git a/src/Disks/IO/ReadBufferFromWebServer.h b/src/Disks/IO/ReadBufferFromWebServer.h index 1e4219d53ee..99a9df9ebcb 100644 --- a/src/Disks/IO/ReadBufferFromWebServer.h +++ b/src/Disks/IO/ReadBufferFromWebServer.h @@ -1,6 +1,6 @@ #pragma once -#include +#include #include #include #include @@ -15,7 +15,7 @@ namespace DB * * Usage: ReadIndirectBufferFromRemoteFS -> SeekAvoidingReadBuffer -> ReadBufferFromWebServer -> ReadWriteBufferFromHTTP. */ -class ReadBufferFromWebServer : public SeekableReadBuffer +class ReadBufferFromWebServer : public ReadBufferFromFileBase { public: explicit ReadBufferFromWebServer( @@ -33,6 +33,8 @@ public: size_t getFileOffsetOfBufferEnd() const override { return offset; } + String getFileName() const override { return url; } + private: std::unique_ptr initialize(); diff --git a/src/IO/WriteBufferFromAzureBlobStorage.cpp b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp similarity index 97% rename from src/IO/WriteBufferFromAzureBlobStorage.cpp rename to src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp index bc7b505cd91..8130d742ee5 100644 --- a/src/IO/WriteBufferFromAzureBlobStorage.cpp +++ b/src/Disks/IO/WriteBufferFromAzureBlobStorage.cpp @@ -2,7 +2,7 @@ #if USE_AZURE_BLOB_STORAGE -#include +#include #include #include #include diff --git a/src/IO/WriteBufferFromAzureBlobStorage.h b/src/Disks/IO/WriteBufferFromAzureBlobStorage.h similarity index 100% rename from src/IO/WriteBufferFromAzureBlobStorage.h rename to src/Disks/IO/WriteBufferFromAzureBlobStorage.h diff --git a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp b/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp index bf99934dd73..a909e8f109c 100644 --- a/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp +++ b/src/Disks/IO/WriteIndirectBufferFromRemoteFS.cpp @@ -1,11 +1,5 @@ #include "WriteIndirectBufferFromRemoteFS.h" -#include -#include -#include -#include - - namespace DB { diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index 37fbc0fb05c..1726f4eeed7 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -3,8 +3,8 @@ #if USE_AZURE_BLOB_STORAGE #include -#include -#include +#include +#include #include #include @@ -79,11 +79,24 @@ std::unique_ptr AzureObjectStorage::readObjects( /// NOL { ReadSettings disk_read_settings = patchSettings(read_settings); auto settings_ptr = settings.get(); - auto reader_impl = std::make_unique( - client.get(), + + auto read_buffer_creator = + [this, settings_ptr, disk_read_settings] + (const std::string & path, size_t read_until_position) -> std::shared_ptr + { + return std::make_unique( + client.get(), + path, + disk_read_settings, + settings_ptr->max_single_read_retries, + settings_ptr->max_single_download_retries, + /* use_external_buffer */true, + read_until_position); + }; + + auto reader_impl = std::make_unique( + std::move(read_buffer_creator), objects, - settings_ptr->max_single_read_retries, - settings_ptr->max_single_download_retries, disk_read_settings); if (disk_read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 34b3d86b355..8564e2dcd36 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -12,6 +12,9 @@ #include #include +#if USE_AZURE_BLOB_STORAGE +#include +#endif namespace DB { diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index 4ffbf5b2ceb..78bbed21e39 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -61,7 +61,19 @@ std::unique_ptr HDFSObjectStorage::readObjects( /// NOLI std::optional, std::optional) const { - auto hdfs_impl = std::make_unique(config, objects, patchSettings(read_settings)); + auto disk_read_settings = patchSettings(read_settings); + auto read_buffer_creator = + [this, disk_read_settings] + (const std::string & path, size_t /* read_until_position */) -> std::shared_ptr + { + size_t begin_of_path = path.find('/', path.find("//") + 2); + auto hdfs_path = path.substr(begin_of_path); + auto hdfs_uri = path.substr(0, begin_of_path); + + return std::make_unique(hdfs_uri, hdfs_path, config, disk_read_settings); + }; + + auto hdfs_impl = std::make_unique(std::move(read_buffer_creator), objects, disk_read_settings); auto buf = std::make_unique(std::move(hdfs_impl)); return std::make_unique(std::move(buf), settings->min_bytes_for_seek); } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index d36bf655c02..25dafac4120 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -144,12 +144,26 @@ std::unique_ptr S3ObjectStorage::readObjects( /// NOLINT auto settings_ptr = s3_settings.get(); - auto s3_impl = std::make_unique( - client.get(), - bucket, - version_id, + auto read_buffer_creator = + [this, settings_ptr, disk_read_settings] + (const std::string & path, size_t read_until_position) -> std::shared_ptr + { + return std::make_shared( + client.get(), + bucket, + path, + version_id, + settings_ptr->s3_settings.max_single_read_retries, + disk_read_settings, + /* use_external_buffer */true, + /* offset */0, + read_until_position, + /* restricted_seek */true); + }; + + auto s3_impl = std::make_unique( + std::move(read_buffer_creator), objects, - settings_ptr->s3_settings.max_single_read_retries, disk_read_settings); if (read_settings.remote_fs_method == RemoteFSReadMethod::threadpool) From 3731a415e6af99858b46557a69c49a8704bb8017 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sun, 31 Jul 2022 20:07:33 -0400 Subject: [PATCH 058/117] run resulting executable with execvp --- .../decompressor.cpp | 81 ++++++------------- 1 file changed, 26 insertions(+), 55 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 679dc144f13..1e19d47d47d 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -295,45 +295,30 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress return 0; } -/// Copy particular part of command and update shift -void fill(char * dest, char * source, size_t length, size_t& shift) +int do_system(const char * cmd) { - memcpy(dest + shift, source, length); - shift += length; -} - -/// Set command to `mv filename.decompressed filename && filename agrs...` -void fillCommand(char command[], int argc, char * argv[], size_t length) -{ - memset(command, '\0', 3 + strlen(argv[0]) + 14 + strlen(argv[0]) + 4 + strlen(argv[0]) + length + argc); - - /// position in command - size_t shift = 0; - - /// Support variables to create command - char mv[] = "mv "; - char decompressed[] = ".decompressed "; - char add_command[] = " && "; - char space[] = " "; - - fill(command, mv, 3, shift); - fill(command, argv[0], strlen(argv[0]), shift); - fill(command, decompressed, 14, shift); - fill(command, argv[0], strlen(argv[0]), shift); - fill(command, add_command, 4, shift); - fill(command, argv[0], strlen(argv[0]), shift); - fill(command, space, 1, shift); - - /// forward all arguments - for (int i = 1; i < argc; ++i) + int ret = system(cmd); + if (ret == -1) { - fill(command, argv[i], strlen(argv[i]), shift); - if (i != argc - 1) - fill(command, space, 1, shift); + perror(nullptr); + return 1; } + + if (WIFEXITED(ret) && WEXITSTATUS(ret)) + { + std::cerr << "Command [" << cmd << "] exited with code " << WEXITSTATUS(ret) << std::endl; + return 1; + } + else if (WIFSIGNALED(ret)) + { + std::cerr << "Command [" << cmd << "] killed by signal " << WTERMSIG(ret) << std::endl; + return 1; + } + + return 0; } -int main(int argc, char* argv[]) +int main(int/* argc*/, char* argv[]) { char file_path[strlen(argv[0]) + 1]; memset(file_path, 0, sizeof(file_path)); @@ -371,12 +356,6 @@ int main(int argc, char* argv[]) if (0 != close(input_fd)) perror(nullptr); - /// According to documentation `mv` will rename file if it - /// doesn't move to other directory. - /// Sometimes `rename` doesn't exist by default and - /// `rename.ul` is set instead. It will lead to errors - /// that can be easily avoided with help of `mv` - if (!have_compressed_analoge) { printf("No target executable - decompression only was performed.\n"); @@ -387,22 +366,14 @@ int main(int argc, char* argv[]) } else { - /// move decompressed file instead of this binary and apply command - char bash[] = "sh"; - char executable[] = "-c"; + const char * const cmd_fmt = "mv %s %s.delete && cp %s.decompressed %s && rm %s.delete %s.decompressed"; + int cmd_len = snprintf(nullptr, 0, cmd_fmt, argv[0], argv[0], argv[0], argv[0], argv[0], argv[0]); + char command[cmd_len + 1]; + snprintf(command, cmd_len + 1, cmd_fmt, argv[0], argv[0], argv[0], argv[0], argv[0], argv[0]); + if (do_system(command)) + return 1; - /// length of forwarded args - size_t length = 0; - for (int i = 1; i < argc; ++i) - length += strlen(argv[i]); - - /// mv filename.decompressed filename && filename agrs... - char command[3 + strlen(argv[0]) + 14 + strlen(argv[0]) + 4 + strlen(argv[0]) + length + argc]; - fillCommand(command, argc, argv, length); - - /// replace file and call executable - char * newargv[] = { bash, executable, command, nullptr }; - execvp(bash, newargv); + execvp(argv[0], argv); /// This part of code will be reached only if error happened perror(nullptr); From c534bdd42ff8fa95fa5937e41db88030bc432ec3 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 1 Aug 2022 01:13:50 -0400 Subject: [PATCH 059/117] clang tidy treats call to system as security issue - replace with fork/exec --- .../decompressor.cpp | 32 ++++++++++++++----- 1 file changed, 24 insertions(+), 8 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 1e19d47d47d..00dd784dabf 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -297,21 +297,37 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress int do_system(const char * cmd) { - int ret = system(cmd); - if (ret == -1) - { - perror(nullptr); + pid_t pid = fork(); + if (pid == -1) return 1; + + if (pid == 0) + { + execlp( "sh" , "sh", "-c", cmd, NULL); + + perror(nullptr); + exit(127); } - if (WIFEXITED(ret) && WEXITSTATUS(ret)) + int ret = 0; + int status = 0; + while ((ret = waitpid(pid, &status, 0)) == -1) { - std::cerr << "Command [" << cmd << "] exited with code " << WEXITSTATUS(ret) << std::endl; + if ( errno != EINTR) + { + perror(nullptr); + return 1; + } + } + + if (WIFEXITED(status) && WEXITSTATUS(status)) + { + std::cerr << "Command [" << cmd << "] exited with code " << WEXITSTATUS(status) << std::endl; return 1; } - else if (WIFSIGNALED(ret)) + else if (WIFSIGNALED(status)) { - std::cerr << "Command [" << cmd << "] killed by signal " << WTERMSIG(ret) << std::endl; + std::cerr << "Command [" << cmd << "] killed by signal " << WTERMSIG(status) << std::endl; return 1; } From 7a2d969850862beeddb35f776d60cb4523b55974 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 1 Aug 2022 01:20:40 -0400 Subject: [PATCH 060/117] fix clang tidy unused return --- utils/self-extracting-executable/decompressor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 00dd784dabf..6563aa49541 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -385,7 +385,7 @@ int main(int/* argc*/, char* argv[]) const char * const cmd_fmt = "mv %s %s.delete && cp %s.decompressed %s && rm %s.delete %s.decompressed"; int cmd_len = snprintf(nullptr, 0, cmd_fmt, argv[0], argv[0], argv[0], argv[0], argv[0], argv[0]); char command[cmd_len + 1]; - snprintf(command, cmd_len + 1, cmd_fmt, argv[0], argv[0], argv[0], argv[0], argv[0], argv[0]); + cmd_len = snprintf(command, cmd_len + 1, cmd_fmt, argv[0], argv[0], argv[0], argv[0], argv[0], argv[0]); if (do_system(command)) return 1; From 3aeb5250367f37e0b6a3152dd50a8d5164145f70 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 1 Aug 2022 01:45:02 -0400 Subject: [PATCH 061/117] style fix --- utils/self-extracting-executable/decompressor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 6563aa49541..9967a9f1904 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -303,7 +303,7 @@ int do_system(const char * cmd) if (pid == 0) { - execlp( "sh" , "sh", "-c", cmd, NULL); + execlp("sh" , "sh", "-c", cmd, NULL); perror(nullptr); exit(127); @@ -313,7 +313,7 @@ int do_system(const char * cmd) int status = 0; while ((ret = waitpid(pid, &status, 0)) == -1) { - if ( errno != EINTR) + if (errno != EINTR) { perror(nullptr); return 1; From c79893f4ef192b85293cb8007360397de517b77d Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 1 Aug 2022 10:58:08 -0400 Subject: [PATCH 062/117] remove call to sh, use random names suffix for temporaries --- .../decompressor.cpp | 103 ++++++++++-------- 1 file changed, 57 insertions(+), 46 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 9967a9f1904..63877a4ad83 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -12,6 +12,7 @@ #include #include #include +#include #if (defined(OS_DARWIN) || defined(OS_FREEBSD)) && defined(__GNUC__) # include @@ -27,6 +28,9 @@ #include "types.h" +char decompressed_suffix[7] = {0}; +uint64_t decompressed_umask = 0; + /// decompress part int doDecompress(char * input, char * output, off_t & in_offset, off_t & out_offset, off_t input_size, off_t output_size, ZSTD_DCtx* dctx) @@ -221,7 +225,7 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress files_pointer += sizeof(FileData); size_t file_name_len = - (strcmp(input + files_pointer, name) ? le64toh(file_info.name_length) : le64toh(file_info.name_length) + 13); + (strcmp(input + files_pointer, name) ? le64toh(file_info.name_length) : le64toh(file_info.name_length) + 13 + 7); size_t file_path_len = path ? strlen(path) + 1 + file_name_len : file_name_len; @@ -236,7 +240,16 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress files_pointer += le64toh(file_info.name_length); if (file_name_len != le64toh(file_info.name_length)) { - strcat(file_name, ".decompressed"); + strcat(file_name, ".decompressed.XXXXXX"); + int fd = mkstemp(file_name); + if(fd == -1) + { + perror(nullptr); + return 1; + } + close(fd); + strncpy(decompressed_suffix, file_name + strlen(file_name) - 6, 6); + decompressed_umask = le64toh(file_info.umask); have_compressed_analoge = true; } @@ -295,44 +308,6 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress return 0; } -int do_system(const char * cmd) -{ - pid_t pid = fork(); - if (pid == -1) - return 1; - - if (pid == 0) - { - execlp("sh" , "sh", "-c", cmd, NULL); - - perror(nullptr); - exit(127); - } - - int ret = 0; - int status = 0; - while ((ret = waitpid(pid, &status, 0)) == -1) - { - if (errno != EINTR) - { - perror(nullptr); - return 1; - } - } - - if (WIFEXITED(status) && WEXITSTATUS(status)) - { - std::cerr << "Command [" << cmd << "] exited with code " << WEXITSTATUS(status) << std::endl; - return 1; - } - else if (WIFSIGNALED(status)) - { - std::cerr << "Command [" << cmd << "] killed by signal " << WTERMSIG(status) << std::endl; - return 1; - } - - return 0; -} int main(int/* argc*/, char* argv[]) { @@ -382,14 +357,50 @@ int main(int/* argc*/, char* argv[]) } else { - const char * const cmd_fmt = "mv %s %s.delete && cp %s.decompressed %s && rm %s.delete %s.decompressed"; - int cmd_len = snprintf(nullptr, 0, cmd_fmt, argv[0], argv[0], argv[0], argv[0], argv[0], argv[0]); - char command[cmd_len + 1]; - cmd_len = snprintf(command, cmd_len + 1, cmd_fmt, argv[0], argv[0], argv[0], argv[0], argv[0], argv[0]); - if (do_system(command)) + const char * const delete_name_fmt = "%s.delete.XXXXXX"; + int delete_name_len = snprintf(nullptr, 0, delete_name_fmt, argv[0]); + char delete_name[delete_name_len + 1]; + delete_name_len = snprintf(delete_name, delete_name_len + 1, delete_name_fmt, argv[0]); + int fd = mkstemp(delete_name); + if(fd == -1) + { + perror(nullptr); return 1; + } + close(fd); - execvp(argv[0], argv); + if(rename(argv[0], delete_name)) + { + perror(nullptr); + return 1; + } + + const char * const decompressed_name_fmt = "%s.decompressed.%s"; + int decompressed_name_len = snprintf(nullptr, 0, decompressed_name_fmt, argv[0], decompressed_suffix); + char decompressed_name[decompressed_name_len + 1]; + decompressed_name_len = snprintf(decompressed_name, decompressed_name_len + 1, decompressed_name_fmt, argv[0], decompressed_suffix); + + std::error_code ec; + std::filesystem::copy_file(static_cast(decompressed_name), argv[0], ec); + if (ec) + { + std::cerr << ec.message() << std::endl; + return 1; + } + + if (chmod(argv[0], decompressed_umask)) + { + perror(nullptr); + return 1; + } + + if (unlink(delete_name) || unlink(decompressed_name)) + { + perror(nullptr); + return 1; + } + + execv(argv[0], argv); /// This part of code will be reached only if error happened perror(nullptr); From 2a074288f02b232cdaf9e7d9553ee37f8c1b77ca Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 1 Aug 2022 11:06:54 -0400 Subject: [PATCH 063/117] style fix --- utils/self-extracting-executable/decompressor.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 63877a4ad83..ac9981c7130 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -242,7 +242,7 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress { strcat(file_name, ".decompressed.XXXXXX"); int fd = mkstemp(file_name); - if(fd == -1) + if (fd == -1) { perror(nullptr); return 1; @@ -362,14 +362,14 @@ int main(int/* argc*/, char* argv[]) char delete_name[delete_name_len + 1]; delete_name_len = snprintf(delete_name, delete_name_len + 1, delete_name_fmt, argv[0]); int fd = mkstemp(delete_name); - if(fd == -1) + if (fd == -1) { perror(nullptr); return 1; } close(fd); - if(rename(argv[0], delete_name)) + if (rename(argv[0], delete_name)) { perror(nullptr); return 1; @@ -387,7 +387,7 @@ int main(int/* argc*/, char* argv[]) std::cerr << ec.message() << std::endl; return 1; } - + if (chmod(argv[0], decompressed_umask)) { perror(nullptr); From afcf76e8991d6cfe3e6f7d1604a6a6f96a2db126 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Mon, 1 Aug 2022 18:27:17 +0200 Subject: [PATCH 064/117] Add cloudflare DNS as a fallback --- tests/ci/worker/init_runner.sh | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index 74ad4be2547..dddf3bd4174 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -17,6 +17,20 @@ export RUNNER_URL="https://github.com/ClickHouse" INSTANCE_ID=$(ec2metadata --instance-id) export INSTANCE_ID +# Add cloudflare DNS as a fallback +# Get default gateway interface +IFACE=$(ip -j route l | jq '.[]|select(.dst == "default").dev' -r) +# `Link 2 (eth0): 172.31.0.2` +ETH_DNS=$(resolvectl dns "$IFACE") || : +CLOUDFRONT_NS=1.1.1.1 +if [[ "$ETH_DNS" ]] && [[ "${ETH_DNS#*: }" != *"$CLOUDFRONT_NS"* ]]; then + # Cut the leading legend + ETH_DNS=${ETH_DNS#*: } + # shellcheck disable=SC2206 + new_dns=(${ETH_DNS} "$CLOUDFRONT_NS") + resolvectl dns "$IFACE" "${new_dns[@]}" +fi + # combine labels RUNNER_TYPE=$(/usr/local/bin/aws ec2 describe-tags --filters "Name=resource-id,Values=$INSTANCE_ID" --query "Tags[?Key=='github:runner-type'].Value" --output text) LABELS="self-hosted,Linux,$(uname -m),$RUNNER_TYPE" From 8820774fe35f63dc34032d01fdeebaa728dc6106 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 1 Aug 2022 13:39:26 -0400 Subject: [PATCH 065/117] do not rename original file - just remove --- .../self-extracting-executable/decompressor.cpp | 16 ++-------------- 1 file changed, 2 insertions(+), 14 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index ac9981c7130..312734d3865 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -357,19 +357,7 @@ int main(int/* argc*/, char* argv[]) } else { - const char * const delete_name_fmt = "%s.delete.XXXXXX"; - int delete_name_len = snprintf(nullptr, 0, delete_name_fmt, argv[0]); - char delete_name[delete_name_len + 1]; - delete_name_len = snprintf(delete_name, delete_name_len + 1, delete_name_fmt, argv[0]); - int fd = mkstemp(delete_name); - if (fd == -1) - { - perror(nullptr); - return 1; - } - close(fd); - - if (rename(argv[0], delete_name)) + if (unlink(argv[0])) { perror(nullptr); return 1; @@ -394,7 +382,7 @@ int main(int/* argc*/, char* argv[]) return 1; } - if (unlink(delete_name) || unlink(decompressed_name)) + if (unlink(decompressed_name)) { perror(nullptr); return 1; From e65cef79adc79cdf61ab4ccd43f20aa5584f6e50 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Mon, 1 Aug 2022 14:32:32 -0400 Subject: [PATCH 066/117] some refactoring --- utils/self-extracting-executable/decompressor.cpp | 14 ++++---------- 1 file changed, 4 insertions(+), 10 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 312734d3865..5615216e262 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -347,22 +347,16 @@ int main(int/* argc*/, char* argv[]) if (0 != close(input_fd)) perror(nullptr); - if (!have_compressed_analoge) + if (unlink(argv[0])) { - printf("No target executable - decompression only was performed.\n"); - /// remove file - execlp("rm", "rm", argv[0], NULL); perror(nullptr); return 1; } + + if (!have_compressed_analoge) + printf("No target executable - decompression only was performed.\n"); else { - if (unlink(argv[0])) - { - perror(nullptr); - return 1; - } - const char * const decompressed_name_fmt = "%s.decompressed.%s"; int decompressed_name_len = snprintf(nullptr, 0, decompressed_name_fmt, argv[0], decompressed_suffix); char decompressed_name[decompressed_name_len + 1]; From 488ae322957ffec5a75540eaa86d918bd529e18e Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 2 Aug 2022 03:23:05 +0300 Subject: [PATCH 067/117] Update init_runner.sh --- tests/ci/worker/init_runner.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index dddf3bd4174..20e65d5ac04 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -19,7 +19,7 @@ export INSTANCE_ID # Add cloudflare DNS as a fallback # Get default gateway interface -IFACE=$(ip -j route l | jq '.[]|select(.dst == "default").dev' -r) +IFACE=$(ip --json route list | jq '.[]|select(.dst == "default").dev' --raw-output) # `Link 2 (eth0): 172.31.0.2` ETH_DNS=$(resolvectl dns "$IFACE") || : CLOUDFRONT_NS=1.1.1.1 From b275aae95d970670c21b89f21ff6295fb7db5d8a Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 2 Aug 2022 03:23:53 +0300 Subject: [PATCH 068/117] Update init_runner.sh --- tests/ci/worker/init_runner.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/ci/worker/init_runner.sh b/tests/ci/worker/init_runner.sh index 20e65d5ac04..3dfd1761e88 100644 --- a/tests/ci/worker/init_runner.sh +++ b/tests/ci/worker/init_runner.sh @@ -22,12 +22,12 @@ export INSTANCE_ID IFACE=$(ip --json route list | jq '.[]|select(.dst == "default").dev' --raw-output) # `Link 2 (eth0): 172.31.0.2` ETH_DNS=$(resolvectl dns "$IFACE") || : -CLOUDFRONT_NS=1.1.1.1 -if [[ "$ETH_DNS" ]] && [[ "${ETH_DNS#*: }" != *"$CLOUDFRONT_NS"* ]]; then +CLOUDFLARE_NS=1.1.1.1 +if [[ "$ETH_DNS" ]] && [[ "${ETH_DNS#*: }" != *"$CLOUDFLARE_NS"* ]]; then # Cut the leading legend ETH_DNS=${ETH_DNS#*: } # shellcheck disable=SC2206 - new_dns=(${ETH_DNS} "$CLOUDFRONT_NS") + new_dns=(${ETH_DNS} "$CLOUDFLARE_NS") resolvectl dns "$IFACE" "${new_dns[@]}" fi From 26ef951fdea1d7a71515ba475f3cc59c9bda9129 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 2 Aug 2022 12:04:26 +0200 Subject: [PATCH 069/117] Add DNS trick to the dockerhub proxy too --- tests/ci/worker/dockerhub_proxy_template.sh | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/tests/ci/worker/dockerhub_proxy_template.sh b/tests/ci/worker/dockerhub_proxy_template.sh index add15158d94..38f2bc6486a 100644 --- a/tests/ci/worker/dockerhub_proxy_template.sh +++ b/tests/ci/worker/dockerhub_proxy_template.sh @@ -1,6 +1,20 @@ #!/usr/bin/env bash set -xeuo pipefail +# Add cloudflare DNS as a fallback +# Get default gateway interface +IFACE=$(ip --json route list | jq '.[]|select(.dst == "default").dev' --raw-output) +# `Link 2 (eth0): 172.31.0.2` +ETH_DNS=$(resolvectl dns "$IFACE") || : +CLOUDFLARE_NS=1.1.1.1 +if [[ "$ETH_DNS" ]] && [[ "${ETH_DNS#*: }" != *"$CLOUDFLARE_NS"* ]]; then + # Cut the leading legend + ETH_DNS=${ETH_DNS#*: } + # shellcheck disable=SC2206 + new_dns=(${ETH_DNS} "$CLOUDFLARE_NS") + resolvectl dns "$IFACE" "${new_dns[@]}" +fi + mkdir /home/ubuntu/registrystorage sed -i 's/preserve_hostname: false/preserve_hostname: true/g' /etc/cloud/cloud.cfg From 57e31cadfae61d3504018431dbb4b3fbc0ff9a37 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 3 Aug 2022 09:08:51 +0300 Subject: [PATCH 070/117] Update unaligned.h --- base/base/unaligned.h | 21 ++++++++++++++------- 1 file changed, 14 insertions(+), 7 deletions(-) diff --git a/base/base/unaligned.h b/base/base/unaligned.h index 7fff0ddb20c..c469972b4b6 100644 --- a/base/base/unaligned.h +++ b/base/base/unaligned.h @@ -4,14 +4,20 @@ #include #include -inline void reverseMemcpy(void *dst, const void *src, int length) -{ - uint8_t *d = reinterpret_cast(dst); - const uint8_t *s = reinterpret_cast(src); - d += length; - while (length--) - *--d = *s++; +inline void reverseMemcpy(void * dst, const void * src, size_t size) +{ + uint8_t * uint_dst = reinterpret_cast(dst); + const uint8_t * uint_src = reinterpret_cast(src); + + uint_dst += length; + while (length) + { + --uint_dst; + *uint_dst = *uint_src; + ++uint_src; + --length; + } } template @@ -25,6 +31,7 @@ inline T unalignedLoadLE(const void * address) return res; } + template inline void unalignedStoreLE(void * address, const typename std::enable_if::type & src) From 5a5d028154d9204cdd3bb3f7b33ca3b161731253 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> Date: Wed, 3 Aug 2022 03:34:51 -0400 Subject: [PATCH 071/117] remove global variables --- utils/self-extracting-executable/decompressor.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 5615216e262..45da2189713 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -28,9 +28,6 @@ #include "types.h" -char decompressed_suffix[7] = {0}; -uint64_t decompressed_umask = 0; - /// decompress part int doDecompress(char * input, char * output, off_t & in_offset, off_t & out_offset, off_t input_size, off_t output_size, ZSTD_DCtx* dctx) @@ -166,7 +163,7 @@ int decompress(char * input, char * output, off_t start, off_t end, size_t max_n /// Read data about files and decomrpess them. -int decompressFiles(int input_fd, char * path, char * name, bool & have_compressed_analoge) +int decompressFiles(int input_fd, char * path, char * name, bool & have_compressed_analoge, char * decompressed_suffix, uint64_t * decompressed_umask) { /// Read data about output file. /// Compressed data will replace data in file @@ -249,7 +246,7 @@ int decompressFiles(int input_fd, char * path, char * name, bool & have_compress } close(fd); strncpy(decompressed_suffix, file_name + strlen(file_name) - 6, 6); - decompressed_umask = le64toh(file_info.umask); + *decompressed_umask = le64toh(file_info.umask); have_compressed_analoge = true; } @@ -334,9 +331,11 @@ int main(int/* argc*/, char* argv[]) } bool have_compressed_analoge = false; + char decompressed_suffix[7] = {0}; + uint64_t decompressed_umask = 0; /// Decompress all files - if (0 != decompressFiles(input_fd, path, name, have_compressed_analoge)) + if (0 != decompressFiles(input_fd, path, name, have_compressed_analoge, decompressed_suffix, &decompressed_umask)) { printf("Error happened during decompression.\n"); if (0 != close(input_fd)) From 724d3d0359fc6a9f2f87d8b38e1617e19a05c0b7 Mon Sep 17 00:00:00 2001 From: SkyhotQin Date: Wed, 3 Aug 2022 17:09:10 +0800 Subject: [PATCH 072/117] Update dbms-naming.md delete the `h` and some spaces --- docs/zh/faq/general/dbms-naming.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/zh/faq/general/dbms-naming.md b/docs/zh/faq/general/dbms-naming.md index 72b96886924..6725524d58a 100644 --- a/docs/zh/faq/general/dbms-naming.md +++ b/docs/zh/faq/general/dbms-naming.md @@ -10,7 +10,7 @@ sidebar_position: 10 这个由两部分组成的意思有两个结果: -- 唯一正确的写“Click**H** house”的方式是用大写H。 +- 唯一正确的写“Click**H**ouse”的方式是用大写H。 - 如果需要缩写,请使用“**CH**”。由于一些历史原因,缩写CK在中国也很流行,主要是因为中文中最早的一个关于ClickHouse的演讲使用了这种形式。 !!! info “有趣的事实” From 08474cf869078120e5e32e83b3f3eb8cca676de2 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 3 Aug 2022 11:43:47 +0200 Subject: [PATCH 073/117] Update tweak on version part update --- tests/ci/version_helper.py | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/tests/ci/version_helper.py b/tests/ci/version_helper.py index 08ee052cf1a..de98b8431de 100755 --- a/tests/ci/version_helper.py +++ b/tests/ci/version_helper.py @@ -65,14 +65,20 @@ class ClickHouseVersion: return method() def major_update(self) -> "ClickHouseVersion": + if self._git is not None: + self._git.update() return ClickHouseVersion(self.major + 1, 1, 1, self.revision + 1, self._git) def minor_update(self) -> "ClickHouseVersion": + if self._git is not None: + self._git.update() return ClickHouseVersion( self.major, self.minor + 1, 1, self.revision + 1, self._git ) def patch_update(self) -> "ClickHouseVersion": + if self._git is not None: + self._git.update() return ClickHouseVersion( self.major, self.minor, self.patch + 1, self.revision, self._git ) From da49a0cc0206f50a62e823bc8b4b68680a230b41 Mon Sep 17 00:00:00 2001 From: kssenii Date: Wed, 3 Aug 2022 12:32:55 +0200 Subject: [PATCH 074/117] Fix build --- contrib/azure-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure-cmake/CMakeLists.txt b/contrib/azure-cmake/CMakeLists.txt index 19f2940cbf0..367131daf0b 100644 --- a/contrib/azure-cmake/CMakeLists.txt +++ b/contrib/azure-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ option (ENABLE_AZURE_BLOB_STORAGE "Enable Azure blob storage" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_AZURE_BLOB_STORAGE) +if (NOT ENABLE_AZURE_BLOB_STORAGE OR BUILD_STANDALONE_KEEPER) message(STATUS "Not using Azure blob storage") return() endif() From 62a05dc10df37725b58e9e222c633addc801bacb Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 3 Aug 2022 13:40:31 +0200 Subject: [PATCH 075/117] Add instructions for github-cli installation --- tests/ci/release.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index b07deffa1fb..f982c7c6f53 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -96,10 +96,17 @@ class Release: def check_prerequisites(self): """ - Check tooling installed in the system + Check tooling installed in the system, `git` is checked by Git() init """ - self.run("gh auth status") - self.run("git status") + try: + self.run("gh auth status") + except subprocess.SubprocessError: + logging.error( + "The github-cli either not installed or not setup, please follow " + "the instructions on https://github.com/cli/cli#installation and " + "https://cli.github.com/manual/" + ) + raise def do(self, check_dirty: bool, check_branch: bool, with_release_branch: bool): self.check_prerequisites() From 71e1b1916c2489f8a486d7b33463b8f4e9f242ab Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 3 Aug 2022 11:43:30 +0000 Subject: [PATCH 076/117] Randomize snapshot on exit in tests --- tests/config/config.d/keeper_port.xml | 2 ++ tests/config/install.sh | 4 ++++ 2 files changed, 6 insertions(+) diff --git a/tests/config/config.d/keeper_port.xml b/tests/config/config.d/keeper_port.xml index 8cea9044dd0..cffd325e968 100644 --- a/tests/config/config.d/keeper_port.xml +++ b/tests/config/config.d/keeper_port.xml @@ -3,6 +3,8 @@ 9181 1 + true + 10000 100000 diff --git a/tests/config/install.sh b/tests/config/install.sh index a9e66ebb633..46009ce671e 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -80,6 +80,10 @@ ln -sf $SRC_PATH/dhparam.pem $DEST_SERVER_PATH/ ln -sf --backup=simple --suffix=_original.xml \ $SRC_PATH/config.d/query_masking_rules.xml $DEST_SERVER_PATH/config.d/ +# We randomize creating the snapshot on exit for Keeper to test out using older snapshots +create_snapshot_on_exit=$(($RANDOM % 2)) +cat $DEST_SERVER_PATH/config.d/keeper_port.xml | sed "s|true|$create_snapshot_on_exit|" > $DEST_SERVER_PATH/config.d/keeper_port.xml + if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then ln -sf $SRC_PATH/config.d/polymorphic_parts.xml $DEST_SERVER_PATH/config.d/ fi From 05a74850b31c29bea08d71f9a1604ec0fa615e38 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 3 Aug 2022 13:57:20 +0200 Subject: [PATCH 077/117] Update tests/config/install.sh Co-authored-by: Alexander Tokmakov --- tests/config/install.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/config/install.sh b/tests/config/install.sh index 46009ce671e..478601620e1 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -82,7 +82,7 @@ ln -sf --backup=simple --suffix=_original.xml \ # We randomize creating the snapshot on exit for Keeper to test out using older snapshots create_snapshot_on_exit=$(($RANDOM % 2)) -cat $DEST_SERVER_PATH/config.d/keeper_port.xml | sed "s|true|$create_snapshot_on_exit|" > $DEST_SERVER_PATH/config.d/keeper_port.xml +sed --follow-symlinks -i "s|true|$create_snapshot_on_exit|" $DEST_SERVER_PATH/config.d/keeper_port.xml if [[ -n "$USE_POLYMORPHIC_PARTS" ]] && [[ "$USE_POLYMORPHIC_PARTS" -eq 1 ]]; then ln -sf $SRC_PATH/config.d/polymorphic_parts.xml $DEST_SERVER_PATH/config.d/ From 56ca93ca3f5325bd5712e9589d6ee84492b6103c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 3 Aug 2022 14:14:57 +0000 Subject: [PATCH 078/117] Block memory tracker in Keeper during commit --- contrib/NuRaft | 2 +- src/Coordination/KeeperServer.cpp | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index 2ef198694e1..d73e12adf75 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit 2ef198694e10c86175ee6ead389346d199060437 +Subproject commit d73e12adf7557a0ebca7f7ecde68c064dee22fa0 diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 7238c86cc50..a7c9b0836f1 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -20,6 +20,7 @@ #include #include #include +#include #include #include @@ -173,6 +174,12 @@ struct KeeperServer::KeeperRaftServer : public nuraft::raft_server reconfigure(new_config); } + void commit_in_bg() override + { + MemoryTrackerBlockerInThread blocker; + nuraft::raft_server::commit_in_bg(); + } + using nuraft::raft_server::raft_server; // peers are initially marked as responding because at least one cycle From 5da32fafb3aed6417c10156ad67174924afeee45 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 3 Aug 2022 15:08:38 +0000 Subject: [PATCH 079/117] fix ALTER MODIFY COLUMN from nested --- src/Storages/MergeTree/MutateTask.cpp | 83 ++++++++++--------- .../02377_modify_column_from_lc.reference | 1 + .../02377_modify_column_from_lc.sql | 43 ++++++++++ .../02377_modify_column_from_nested.reference | 6 ++ .../02377_modify_column_from_nested.sql | 21 +++++ 5 files changed, 115 insertions(+), 39 deletions(-) create mode 100644 tests/queries/0_stateless/02377_modify_column_from_lc.reference create mode 100644 tests/queries/0_stateless/02377_modify_column_from_lc.sql create mode 100644 tests/queries/0_stateless/02377_modify_column_from_nested.reference create mode 100644 tests/queries/0_stateless/02377_modify_column_from_nested.sql diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index ab441fd8b77..4a68c9917fa 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -434,11 +434,34 @@ std::set getProjectionsToRecalculate( return projections_to_recalc; } +static std::unordered_map getStreamCounts( + const MergeTreeDataPartPtr & data_part, const Names & column_names) +{ + std::unordered_map stream_counts; + + for (const auto & column_name : column_names) + { + if (auto serialization = data_part->getSerialization(column_name)) + { + auto callback = [&](const ISerialization::SubstreamPath & substream_path) + { + auto stream_name = ISerialization::getFileNameForStream(column_name, substream_path); + ++stream_counts[stream_name]; + }; + + serialization->enumerateStreams(callback); + } + } + + return stream_counts; +} + /// Files, that we don't need to remove and don't need to hardlink, for example columns.txt and checksums.txt. /// Because we will generate new versions of them after we perform mutation. -NameSet collectFilesToSkip( +static NameSet collectFilesToSkip( const MergeTreeDataPartPtr & source_part, + const MergeTreeDataPartPtr & new_part, const Block & updated_header, const std::set & indices_to_recalc, const String & mrk_extension, @@ -446,24 +469,31 @@ NameSet collectFilesToSkip( { NameSet files_to_skip = source_part->getFileNamesWithoutChecksums(); + auto new_stream_counts = getStreamCounts(new_part, new_part->getColumns().getNames()); + auto source_updated_stream_counts = getStreamCounts(source_part, updated_header.getNames()); + auto new_updated_stream_counts = getStreamCounts(new_part, updated_header.getNames()); + /// Skip updated files - for (const auto & entry : updated_header) + for (const auto & [stream_name, _] : source_updated_stream_counts) { - ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path) + /// If we read shared stream and do not write it + /// (e.g. while ALTER MODIFY COLUMN from array of Nested type to String), + /// we need to hardlink its files, because they will be lost otherwise. + bool need_hardlink = new_updated_stream_counts[stream_name] == 0 && new_stream_counts[stream_name] != 0; + + if (!need_hardlink) { - String stream_name = ISerialization::getFileNameForStream(entry.name, substream_path); files_to_skip.insert(stream_name + ".bin"); files_to_skip.insert(stream_name + mrk_extension); - }; - - if (auto serialization = source_part->tryGetSerialization(entry.name)) - serialization->enumerateStreams(callback); + } } + for (const auto & index : indices_to_recalc) { files_to_skip.insert(index->getFileName() + ".idx"); files_to_skip.insert(index->getFileName() + mrk_extension); } + for (const auto & projection : projections_to_recalc) files_to_skip.insert(projection->getDirectoryName()); @@ -482,19 +512,7 @@ static NameToNameVector collectFilesForRenames( const String & mrk_extension) { /// Collect counts for shared streams of different columns. As an example, Nested columns have shared stream with array sizes. - std::unordered_map stream_counts; - for (const auto & column : source_part->getColumns()) - { - if (auto serialization = source_part->tryGetSerialization(column.name)) - { - serialization->enumerateStreams( - [&](const ISerialization::SubstreamPath & substream_path) - { - ++stream_counts[ISerialization::getFileNameForStream(column, substream_path)]; - }); - } - } - + auto stream_counts = getStreamCounts(source_part, source_part->getColumns().getNames()); NameToNameVector rename_vector; /// Remove old data @@ -560,26 +578,12 @@ static NameToNameVector collectFilesForRenames( /// but were removed in new_part by MODIFY COLUMN from /// type with higher number of streams (e.g. LowCardinality -> String). - auto collect_stream_names = [&](const auto & data_part) - { - NameSet res; - if (auto serialization = data_part->tryGetSerialization(command.column_name)) - { - serialization->enumerateStreams( - [&](const ISerialization::SubstreamPath & substream_path) - { - res.insert(ISerialization::getFileNameForStream(command.column_name, substream_path)); - }); - } - return res; - }; + auto old_streams = getStreamCounts(source_part, source_part->getColumns().getNames()); + auto new_streams = getStreamCounts(new_part, source_part->getColumns().getNames()); - auto old_streams = collect_stream_names(source_part); - auto new_streams = collect_stream_names(new_part); - - for (const auto & old_stream : old_streams) + for (const auto & [old_stream, _] : old_streams) { - if (!new_streams.contains(old_stream)) + if (!new_streams.contains(old_stream) && --stream_counts[old_stream] == 0) { rename_vector.emplace_back(old_stream + ".bin", ""); rename_vector.emplace_back(old_stream + mrk_extension, ""); @@ -1580,6 +1584,7 @@ bool MutateTask::prepare() ctx->files_to_skip = MutationHelpers::collectFilesToSkip( ctx->source_part, + ctx->new_data_part, ctx->updated_header, ctx->indices_to_recalc, ctx->mrk_extension, diff --git a/tests/queries/0_stateless/02377_modify_column_from_lc.reference b/tests/queries/0_stateless/02377_modify_column_from_lc.reference new file mode 100644 index 00000000000..556d825db42 --- /dev/null +++ b/tests/queries/0_stateless/02377_modify_column_from_lc.reference @@ -0,0 +1 @@ +2 1 diff --git a/tests/queries/0_stateless/02377_modify_column_from_lc.sql b/tests/queries/0_stateless/02377_modify_column_from_lc.sql new file mode 100644 index 00000000000..a578e7cb03a --- /dev/null +++ b/tests/queries/0_stateless/02377_modify_column_from_lc.sql @@ -0,0 +1,43 @@ +DROP TABLE IF EXISTS t_modify_from_lc_1; +DROP TABLE IF EXISTS t_modify_from_lc_2; + +SET allow_suspicious_low_cardinality_types = 1; + +CREATE TABLE t_modify_from_lc_1 +( + id UInt64, + a LowCardinality(UInt32) CODEC(NONE) +) +ENGINE = MergeTree ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0; + +CREATE TABLE t_modify_from_lc_2 +( + id UInt64, + a LowCardinality(UInt32) CODEC(NONE) +) +ENGINE = MergeTree ORDER BY tuple() +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO t_modify_from_lc_1 SELECT number, number FROM numbers(100000); +INSERT INTO t_modify_from_lc_2 SELECT number, number FROM numbers(100000); + +OPTIMIZE TABLE t_modify_from_lc_1 FINAL; +OPTIMIZE TABLE t_modify_from_lc_2 FINAL; + +ALTER TABLE t_modify_from_lc_1 MODIFY COLUMN a UInt32; + +-- Check that dictionary of LowCardinality is actually +-- dropped and total size on disk is reduced. +WITH groupArray((table, bytes))::Map(String, UInt64) AS stats +SELECT + length(stats), stats['t_modify_from_lc_1'] < stats['t_modify_from_lc_2'] +FROM +( + SELECT table, sum(bytes_on_disk) AS bytes FROM system.parts + WHERE database = currentDatabase() AND table LIKE 't_modify_from_lc%' AND active + GROUP BY table +); + +DROP TABLE IF EXISTS t_modify_from_lc_1; +DROP TABLE IF EXISTS t_modify_from_lc_2; diff --git a/tests/queries/0_stateless/02377_modify_column_from_nested.reference b/tests/queries/0_stateless/02377_modify_column_from_nested.reference new file mode 100644 index 00000000000..0c66773ab43 --- /dev/null +++ b/tests/queries/0_stateless/02377_modify_column_from_nested.reference @@ -0,0 +1,6 @@ +1 [2] ['aa'] Array(String) +2 [44,55] ['bb','cc'] Array(String) +1 [2] [\'aa\'] String +2 [44,55] [\'bb\',\'cc\'] String +1 [2] [\'aa\'] String +2 [44,55] [\'bb\',\'cc\'] String diff --git a/tests/queries/0_stateless/02377_modify_column_from_nested.sql b/tests/queries/0_stateless/02377_modify_column_from_nested.sql new file mode 100644 index 00000000000..8270cce6278 --- /dev/null +++ b/tests/queries/0_stateless/02377_modify_column_from_nested.sql @@ -0,0 +1,21 @@ +DROP TABLE IF EXISTS t_nested_modify; + +CREATE TABLE t_nested_modify (id UInt64, `n.a` Array(UInt32), `n.b` Array(String)) +ENGINE = MergeTree ORDER BY id +SETTINGS min_bytes_for_wide_part = 0; + +INSERT INTO t_nested_modify VALUES (1, [2], ['aa']); +INSERT INTO t_nested_modify VALUES (2, [44, 55], ['bb', 'cc']); + +SELECT id, `n.a`, `n.b`, toTypeName(`n.b`) FROM t_nested_modify ORDER BY id; + +ALTER TABLE t_nested_modify MODIFY COLUMN `n.b` String; + +SELECT id, `n.a`, `n.b`, toTypeName(`n.b`) FROM t_nested_modify ORDER BY id; + +DETACH TABLE t_nested_modify; +ATTACH TABLE t_nested_modify; + +SELECT id, `n.a`, `n.b`, toTypeName(`n.b`) FROM t_nested_modify ORDER BY id; + +DROP TABLE t_nested_modify; From 53530a5fa4b4e34f11cc8cd42bd095172ce39ee6 Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Wed, 3 Aug 2022 13:11:11 -0400 Subject: [PATCH 080/117] deadcode/clang-tidy fight solution --- utils/self-extracting-executable/decompressor.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/utils/self-extracting-executable/decompressor.cpp b/utils/self-extracting-executable/decompressor.cpp index 5615216e262..8fc9092932e 100644 --- a/utils/self-extracting-executable/decompressor.cpp +++ b/utils/self-extracting-executable/decompressor.cpp @@ -360,7 +360,7 @@ int main(int/* argc*/, char* argv[]) const char * const decompressed_name_fmt = "%s.decompressed.%s"; int decompressed_name_len = snprintf(nullptr, 0, decompressed_name_fmt, argv[0], decompressed_suffix); char decompressed_name[decompressed_name_len + 1]; - decompressed_name_len = snprintf(decompressed_name, decompressed_name_len + 1, decompressed_name_fmt, argv[0], decompressed_suffix); + (void)snprintf(decompressed_name, decompressed_name_len + 1, decompressed_name_fmt, argv[0], decompressed_suffix); std::error_code ec; std::filesystem::copy_file(static_cast(decompressed_name), argv[0], ec); From 04f6a850706f935c71f5d4d913175d333a59c070 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Wed, 3 Aug 2022 17:34:46 +0000 Subject: [PATCH 081/117] fix mutations --- src/Storages/MergeTree/MutateTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MutateTask.cpp b/src/Storages/MergeTree/MutateTask.cpp index 4a68c9917fa..a72b6af9c65 100644 --- a/src/Storages/MergeTree/MutateTask.cpp +++ b/src/Storages/MergeTree/MutateTask.cpp @@ -441,7 +441,7 @@ static std::unordered_map getStreamCounts( for (const auto & column_name : column_names) { - if (auto serialization = data_part->getSerialization(column_name)) + if (auto serialization = data_part->tryGetSerialization(column_name)) { auto callback = [&](const ISerialization::SubstreamPath & substream_path) { From b8ffaabdd8b849ba3a947363990e3b356cfbb7e7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 3 Aug 2022 17:43:14 +0200 Subject: [PATCH 082/117] add chmod and stat to IDisk --- src/Disks/DiskDecorator.h | 6 ++++++ src/Disks/DiskLocal.cpp | 20 +++++++++++++++++++ src/Disks/DiskLocal.h | 6 ++++++ src/Disks/FakeDiskTransaction.h | 5 +++++ src/Disks/IDisk.h | 6 ++++++ src/Disks/IDiskTransaction.h | 3 +++ .../ObjectStorages/DiskObjectStorage.cpp | 12 +++++++++++ src/Disks/ObjectStorages/DiskObjectStorage.h | 6 ++++++ .../DiskObjectStorageTransaction.cpp | 9 +++++++++ .../DiskObjectStorageTransaction.h | 1 + .../FakeMetadataStorageFromDisk.h | 10 ++++++++++ src/Disks/ObjectStorages/IMetadataStorage.h | 8 ++++++++ .../MetadataStorageFromDisk.cpp | 5 +++++ .../ObjectStorages/MetadataStorageFromDisk.h | 10 ++++++++++ ...taStorageFromDiskTransactionOperations.cpp | 18 +++++++++++++++++ ...dataStorageFromDiskTransactionOperations.h | 15 ++++++++++++++ 16 files changed, 140 insertions(+) diff --git a/src/Disks/DiskDecorator.h b/src/Disks/DiskDecorator.h index 1eac37e13f3..0110154ff78 100644 --- a/src/Disks/DiskDecorator.h +++ b/src/Disks/DiskDecorator.h @@ -99,6 +99,12 @@ public: void syncRevision(UInt64 revision) override { delegate->syncRevision(revision); } UInt64 getRevision() const override { return delegate->getRevision(); } + bool supportsStat() const override { return delegate->supportsStat(); } + struct stat stat(const String & path) const override { return delegate->stat(path); } + + bool supportsChmod() const override { return delegate->supportsChmod(); } + void chmod(const String & path, mode_t mode) override { delegate->chmod(path, mode); } + protected: Executor & getExecutor() override; diff --git a/src/Disks/DiskLocal.cpp b/src/Disks/DiskLocal.cpp index e793f4dfb5a..b224d202541 100644 --- a/src/Disks/DiskLocal.cpp +++ b/src/Disks/DiskLocal.cpp @@ -11,6 +11,8 @@ #include #include +#include +#include #include #include @@ -39,6 +41,7 @@ namespace ErrorCodes extern const int CANNOT_UNLINK; extern const int CANNOT_RMDIR; extern const int BAD_ARGUMENTS; + extern const int CANNOT_STAT; } std::mutex DiskLocal::reservation_mutex; @@ -671,6 +674,23 @@ bool DiskLocal::setup() return true; } +struct stat DiskLocal::stat(const String & path) const +{ + struct stat st; + auto full_path = fs::path(disk_path) / path; + if (::stat(full_path.string().c_str(), &st) == 0) + return st; + DB::throwFromErrnoWithPath("Cannot stat file: " + path, path, DB::ErrorCodes::CANNOT_STAT); +} + +void DiskLocal::chmod(const String & path, mode_t mode) +{ + auto full_path = fs::path(disk_path) / path; + if (::chmod(full_path.string().c_str(), mode) == 0) + return; + DB::throwFromErrnoWithPath("Cannot chmod file: " + path, path, DB::ErrorCodes::PATH_ACCESS_DENIED); +} + void registerDiskLocal(DiskFactory & factory) { auto creator = [](const String & name, diff --git a/src/Disks/DiskLocal.h b/src/Disks/DiskLocal.h index 0db9d8d467c..2919063c979 100644 --- a/src/Disks/DiskLocal.h +++ b/src/Disks/DiskLocal.h @@ -122,6 +122,12 @@ public: bool canRead() const noexcept; bool canWrite() const noexcept; + bool supportsStat() const override { return true; } + struct stat stat(const String & path) const override; + + bool supportsChmod() const override { return true; } + void chmod(const String & path, mode_t mode) override; + private: std::optional tryReserve(UInt64 bytes); diff --git a/src/Disks/FakeDiskTransaction.h b/src/Disks/FakeDiskTransaction.h index 6d61ac752f2..e80b45a94ec 100644 --- a/src/Disks/FakeDiskTransaction.h +++ b/src/Disks/FakeDiskTransaction.h @@ -112,6 +112,11 @@ public: disk.setLastModified(path, timestamp); } + void chmod(const String & path, mode_t mode) override + { + disk.chmod(path, mode); + } + void setReadOnly(const std::string & path) override { disk.setReadOnly(path); diff --git a/src/Disks/IDisk.h b/src/Disks/IDisk.h index 2337fa00af5..816db57b016 100644 --- a/src/Disks/IDisk.h +++ b/src/Disks/IDisk.h @@ -351,6 +351,12 @@ public: getType()); } + virtual bool supportsStat() const { return false; } + virtual struct stat stat(const String & /*path*/) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Disk does not support stat"); } + + virtual bool supportsChmod() const { return false; } + virtual void chmod(const String & /*path*/, mode_t /*mode*/) { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Disk does not support chmod"); } + protected: friend class DiskDecorator; diff --git a/src/Disks/IDiskTransaction.h b/src/Disks/IDiskTransaction.h index 74fbe8919fe..572d86dcfdb 100644 --- a/src/Disks/IDiskTransaction.h +++ b/src/Disks/IDiskTransaction.h @@ -103,6 +103,9 @@ public: /// Set last modified time to file or directory at `path`. virtual void setLastModified(const std::string & path, const Poco::Timestamp & timestamp) = 0; + /// Just chmod. + virtual void chmod(const String & path, mode_t mode) = 0; + /// Set file at `path` as read-only. virtual void setReadOnly(const std::string & path) = 0; diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.cpp b/src/Disks/ObjectStorages/DiskObjectStorage.cpp index 0b7d16bd895..c534e59c710 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorage.cpp @@ -367,6 +367,18 @@ time_t DiskObjectStorage::getLastChanged(const String & path) const return metadata_storage->getLastChanged(path); } +struct stat DiskObjectStorage::stat(const String & path) const +{ + return metadata_storage->stat(path); +} + +void DiskObjectStorage::chmod(const String & path, mode_t mode) +{ + auto transaction = createObjectStorageTransaction(); + transaction->chmod(path, mode); + transaction->commit(); +} + void DiskObjectStorage::shutdown() { LOG_INFO(log, "Shutting down disk {}", name); diff --git a/src/Disks/ObjectStorages/DiskObjectStorage.h b/src/Disks/ObjectStorages/DiskObjectStorage.h index 9494f421e6a..ad9f744f92e 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorage.h +++ b/src/Disks/ObjectStorages/DiskObjectStorage.h @@ -168,6 +168,12 @@ public: bool supportsCache() const override; + bool supportsStat() const override { return metadata_storage->supportsStat(); } + struct stat stat(const String & path) const override; + + bool supportsChmod() const override { return metadata_storage->supportsChmod(); } + void chmod(const String & path, mode_t mode) override; + private: /// Create actual disk object storage transaction for operations diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp index 6e807747478..7fc3532ed4a 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.cpp @@ -613,6 +613,15 @@ void DiskObjectStorageTransaction::setLastModified(const std::string & path, con })); } +void DiskObjectStorageTransaction::chmod(const String & path, mode_t mode) +{ + operations_to_execute.emplace_back( + std::make_unique(object_storage, metadata_storage, [path, mode](MetadataTransactionPtr tx) + { + tx->chmod(path, mode); + })); +} + void DiskObjectStorageTransaction::createFile(const std::string & path) { operations_to_execute.emplace_back( diff --git a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h index ceed79a23b7..9c42203b613 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h +++ b/src/Disks/ObjectStorages/DiskObjectStorageTransaction.h @@ -109,6 +109,7 @@ public: void removeSharedFiles(const RemoveBatchRequest & files, bool keep_all_batch_data, const NameSet & file_names_remove_metadata_only) override; void setLastModified(const std::string & path, const Poco::Timestamp & timestamp) override; + void chmod(const String & path, mode_t mode) override; void setReadOnly(const std::string & path) override; void createHardLink(const std::string & src_path, const std::string & dst_path) override; }; diff --git a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h index 6d5ae12a157..b1c8340ef1b 100644 --- a/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/FakeMetadataStorageFromDisk.h @@ -42,6 +42,12 @@ public: time_t getLastChanged(const std::string & path) const override; + bool supportsChmod() const override { return disk->supportsChmod(); } + + bool supportsStat() const override { return disk->supportsStat(); } + + struct stat stat(const String & path) const override { return disk->stat(path); } + std::vector listDirectory(const std::string & path) const override; DirectoryIteratorPtr iterateDirectory(const std::string & path) const override; @@ -89,6 +95,10 @@ public: void setLastModified(const std::string & path, const Poco::Timestamp & timestamp) override; + bool supportsChmod() const override { return disk->supportsChmod(); } + + void chmod(const String & path, mode_t mode) override { disk->chmod(path, mode); } + void setReadOnly(const std::string & path) override; void unlinkFile(const std::string & path) override; diff --git a/src/Disks/ObjectStorages/IMetadataStorage.h b/src/Disks/ObjectStorages/IMetadataStorage.h index a941f1ca514..948bfba2615 100644 --- a/src/Disks/ObjectStorages/IMetadataStorage.h +++ b/src/Disks/ObjectStorages/IMetadataStorage.h @@ -37,6 +37,9 @@ public: virtual void setLastModified(const std::string & path, const Poco::Timestamp & timestamp) = 0; + virtual bool supportsChmod() const = 0; + virtual void chmod(const String & path, mode_t mode) = 0; + virtual void setReadOnly(const std::string & path) = 0; virtual void unlinkFile(const std::string & path) = 0; @@ -107,6 +110,11 @@ public: virtual time_t getLastChanged(const std::string & path) const = 0; + virtual bool supportsChmod() const = 0; + + virtual bool supportsStat() const = 0; + virtual struct stat stat(const String & path) const = 0; + virtual std::vector listDirectory(const std::string & path) const = 0; virtual DirectoryIteratorPtr iterateDirectory(const std::string & path) const = 0; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp index 489772647d1..cdd0fdc8457 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.cpp @@ -250,6 +250,11 @@ void MetadataStorageFromDiskTransaction::setLastModified(const std::string & pat addOperation(std::make_unique(path, timestamp, *metadata_storage.getDisk())); } +void MetadataStorageFromDiskTransaction::chmod(const String & path, mode_t mode) +{ + addOperation(std::make_unique(path, mode, *metadata_storage.getDisk())); +} + void MetadataStorageFromDiskTransaction::unlinkFile(const std::string & path) { addOperation(std::make_unique(path, *metadata_storage.getDisk())); diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h index 104e9d54bff..8c1fb6edd14 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDisk.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDisk.h @@ -39,6 +39,12 @@ public: time_t getLastChanged(const std::string & path) const override; + bool supportsChmod() const override { return disk->supportsChmod(); } + + bool supportsStat() const override { return disk->supportsStat(); } + + struct stat stat(const String & path) const override { return disk->stat(path); } + std::vector listDirectory(const std::string & path) const override; DirectoryIteratorPtr iterateDirectory(const std::string & path) const override; @@ -94,6 +100,10 @@ public: void setLastModified(const std::string & path, const Poco::Timestamp & timestamp) override; + bool supportsChmod() const override { return metadata_storage.supportsChmod(); } + + void chmod(const String & path, mode_t mode) override; + void setReadOnly(const std::string & path) override; void unlinkFile(const std::string & path) override; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp index 72da240cf8a..4a3f76cb2aa 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.cpp @@ -36,6 +36,24 @@ void SetLastModifiedOperation::undo() disk.setLastModified(path, old_timestamp); } +ChmodOperation::ChmodOperation(const std::string & path_, mode_t mode_, IDisk & disk_) + : path(path_) + , mode(mode_) + , disk(disk_) +{ +} + +void ChmodOperation::execute(std::unique_lock &) +{ + old_mode = disk.stat(path).st_mode; + disk.chmod(path, mode); +} + +void ChmodOperation::undo() +{ + disk.chmod(path, old_mode); +} + UnlinkFileOperation::UnlinkFileOperation(const std::string & path_, IDisk & disk_) : path(path_) , disk(disk_) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h index 5f8e772ebc7..0bce6141301 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromDiskTransactionOperations.h @@ -37,6 +37,21 @@ private: IDisk & disk; }; +struct ChmodOperation final : public IMetadataOperation +{ + ChmodOperation(const std::string & path_, mode_t mode_, IDisk & disk_); + + void execute(std::unique_lock & metadata_lock) override; + + void undo() override; + +private: + std::string path; + mode_t mode; + mode_t old_mode; + IDisk & disk; +}; + struct UnlinkFileOperation final : public IMetadataOperation { From a269cf29e4ebfebd9034997de2b877be85e9d8ce Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 3 Aug 2022 19:49:16 +0200 Subject: [PATCH 083/117] remove unused dirs from store on all disks --- src/Interpreters/DatabaseCatalog.cpp | 199 +++++++++---------- src/Interpreters/DatabaseCatalog.h | 4 +- tests/integration/test_merge_tree_s3/test.py | 25 +++ 3 files changed, 125 insertions(+), 103 deletions(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index 02d3e5eac32..a92a82ea821 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -19,10 +20,6 @@ #include #include -#include -#include -#include - #include "config_core.h" #if USE_MYSQL @@ -894,7 +891,7 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr create->setTable(table_id.table_name); try { - table = createTableFromAST(*create, table_id.getDatabaseName(), data_path, getContext(), false).second; + table = createTableFromAST(*create, table_id.getDatabaseName(), data_path, getContext(), true).second; table->is_dropped = true; } catch (...) @@ -979,7 +976,6 @@ void DatabaseCatalog::dropTableDataTask() if (table.table_id) { - try { dropTableFinally(table); @@ -1019,13 +1015,15 @@ void DatabaseCatalog::dropTableFinally(const TableMarkedAsDropped & table) table.table->drop(); } - /// Even if table is not loaded, try remove its data from disk. - /// TODO remove data from all volumes - fs::path data_path = fs::path(getContext()->getPath()) / "store" / getPathForUUID(table.table_id.uuid); - if (fs::exists(data_path)) + /// Even if table is not loaded, try remove its data from disks. + for (const auto & [disk_name, disk] : getContext()->getDisksMap()) { - LOG_INFO(log, "Removing data directory {} of dropped table {}", data_path.string(), table.table_id.getNameForLogs()); - fs::remove_all(data_path); + String data_path = "store/" + getPathForUUID(table.table_id.uuid); + if (!disk->exists(data_path)) + continue; + + LOG_INFO(log, "Removing data directory {} of dropped table {} from disk {}", data_path, table.table_id.getNameForLogs(), disk_name); + disk->removeRecursive(data_path); } LOG_INFO(log, "Removing metadata {} of dropped table {}", table.metadata_path, table.table_id.getNameForLogs()); @@ -1169,121 +1167,118 @@ void DatabaseCatalog::updateLoadingDependencies(const StorageID & table_id, Tabl void DatabaseCatalog::cleanupStoreDirectoryTask() { - fs::path store_path = fs::path(getContext()->getPath()) / "store"; - size_t affected_dirs = 0; - for (const auto & prefix_dir : fs::directory_iterator{store_path}) + for (const auto & [disk_name, disk] : getContext()->getDisksMap()) { - String prefix = prefix_dir.path().filename(); - bool expected_prefix_dir = prefix_dir.is_directory() && - prefix.size() == 3 && - isHexDigit(prefix[0]) && - isHexDigit(prefix[1]) && - isHexDigit(prefix[2]); - - if (!expected_prefix_dir) - { - LOG_WARNING(log, "Found invalid directory {}, will try to remove it", prefix_dir.path().string()); - affected_dirs += maybeRemoveDirectory(prefix_dir.path()); + if (!disk->supportsStat() || !disk->supportsChmod()) continue; - } - for (const auto & uuid_dir : fs::directory_iterator{prefix_dir.path()}) + size_t affected_dirs = 0; + for (auto it = disk->iterateDirectory("store"); it->isValid(); it->next()) { - String uuid_str = uuid_dir.path().filename(); - UUID uuid; - bool parsed = tryParse(uuid, uuid_str); + String prefix = it->name(); + bool expected_prefix_dir = disk->isDirectory(it->path()) && prefix.size() == 3 && isHexDigit(prefix[0]) && isHexDigit(prefix[1]) + && isHexDigit(prefix[2]); - bool expected_dir = uuid_dir.is_directory() && - parsed && - uuid != UUIDHelpers::Nil && - uuid_str.starts_with(prefix); - - if (!expected_dir) + if (!expected_prefix_dir) { - LOG_WARNING(log, "Found invalid directory {}, will try to remove it", uuid_dir.path().string()); - affected_dirs += maybeRemoveDirectory(uuid_dir.path()); + LOG_WARNING(log, "Found invalid directory {} on disk {}, will try to remove it", it->path(), disk_name); + affected_dirs += maybeRemoveDirectory(disk_name, disk, it->path()); continue; } - /// Order is important - if (!hasUUIDMapping(uuid)) + for (auto jt = disk->iterateDirectory(it->path()); jt->isValid(); jt->next()) { - /// We load uuids even for detached and permanently detached tables, - /// so it looks safe enough to remove directory if we don't have uuid mapping for it. - /// No table or database using this directory should concurrently appear, - /// because creation of new table would fail with "directory already exists". - affected_dirs += maybeRemoveDirectory(uuid_dir.path()); + String uuid_str = jt->name(); + UUID uuid; + bool parsed = tryParse(uuid, uuid_str); + + bool expected_dir = disk->isDirectory(jt->path()) && parsed && uuid != UUIDHelpers::Nil && uuid_str.starts_with(prefix); + + if (!expected_dir) + { + LOG_WARNING(log, "Found invalid directory {} on disk {}, will try to remove it", jt->path(), disk_name); + affected_dirs += maybeRemoveDirectory(disk_name, disk, jt->path()); + continue; + } + + /// Order is important + if (!hasUUIDMapping(uuid)) + { + /// We load uuids even for detached and permanently detached tables, + /// so it looks safe enough to remove directory if we don't have uuid mapping for it. + /// No table or database using this directory should concurrently appear, + /// because creation of new table would fail with "directory already exists". + affected_dirs += maybeRemoveDirectory(disk_name, disk, jt->path()); + } } } - } - if (affected_dirs) - LOG_INFO(log, "Cleaned up {} directories from store/", affected_dirs); + if (affected_dirs) + LOG_INFO(log, "Cleaned up {} directories from store/ on disk {}", affected_dirs, disk_name); + } (*cleanup_task)->scheduleAfter(unused_dir_cleanup_period_sec * 1000); } -bool DatabaseCatalog::maybeRemoveDirectory(const fs::path & unused_dir) +bool DatabaseCatalog::maybeRemoveDirectory(const String & disk_name, const DiskPtr & disk, const String & unused_dir) { /// "Safe" automatic removal of some directory. /// At first we do not remove anything and only revoke all access right. /// And remove only if nobody noticed it after, for example, one month. - struct stat st; - if (stat(unused_dir.string().c_str(), &st)) + try { - LOG_ERROR(log, "Failed to stat {}, errno: {}", unused_dir.string(), errno); + struct stat st = disk->stat(unused_dir); + + if (st.st_uid != geteuid()) + { + /// Directory is not owned by clickhouse, it's weird, let's ignore it (chmod will likely fail anyway). + LOG_WARNING(log, "Found directory {} with unexpected owner (uid={}) on disk {}", unused_dir, st.st_uid, disk_name); + return false; + } + + time_t max_modification_time = std::max(st.st_atime, std::max(st.st_mtime, st.st_ctime)); + time_t current_time = time(nullptr); + if (st.st_mode & (S_IRWXU | S_IRWXG | S_IRWXO)) + { + if (current_time <= max_modification_time + unused_dir_hide_timeout_sec) + return false; + + LOG_INFO(log, "Removing access rights for unused directory {} from disk {} (will remove it when timeout exceed)", unused_dir, disk_name); + + /// Explicitly update modification time just in case + + disk->setLastModified(unused_dir, Poco::Timestamp::fromEpochTime(current_time)); + + /// Remove all access right + disk->chmod(unused_dir, 0); + + return true; + } + else + { + if (!unused_dir_rm_timeout_sec) + return false; + + if (current_time <= max_modification_time + unused_dir_rm_timeout_sec) + return false; + + LOG_INFO(log, "Removing unused directory {} from disk {}", unused_dir, disk_name); + + /// We have to set these access rights to make recursive removal work + disk->chmod(unused_dir, S_IRWXU); + + disk->removeRecursive(unused_dir); + + return true; + } + } + catch (...) + { + tryLogCurrentException(log, fmt::format("Failed to remove unused directory {} from disk {} ({})", + unused_dir, disk->getName(), disk->getPath())); return false; } - - if (st.st_uid != geteuid()) - { - /// Directory is not owned by clickhouse, it's weird, let's ignore it (chmod will likely fail anyway). - LOG_WARNING(log, "Found directory {} with unexpected owner (uid={})", unused_dir.string(), st.st_uid); - return false; - } - - time_t max_modification_time = std::max(st.st_atime, std::max(st.st_mtime, st.st_ctime)); - time_t current_time = time(nullptr); - if (st.st_mode & (S_IRWXU | S_IRWXG | S_IRWXO)) - { - if (current_time <= max_modification_time + unused_dir_hide_timeout_sec) - return false; - - LOG_INFO(log, "Removing access rights for unused directory {} (will remove it when timeout exceed)", unused_dir.string()); - - /// Explicitly update modification time just in case - - struct utimbuf tb; - tb.actime = current_time; - tb.modtime = current_time; - if (utime(unused_dir.string().c_str(), &tb) != 0) - LOG_ERROR(log, "Failed to utime {}, errno: {}", unused_dir.string(), errno); - - /// Remove all access right - if (chmod(unused_dir.string().c_str(), 0)) - LOG_ERROR(log, "Failed to chmod {}, errno: {}", unused_dir.string(), errno); - - return true; - } - else - { - if (!unused_dir_rm_timeout_sec) - return false; - - if (current_time <= max_modification_time + unused_dir_rm_timeout_sec) - return false; - - LOG_INFO(log, "Removing unused directory {}", unused_dir.string()); - - /// We have to set these access rights to make recursive removal work - if (chmod(unused_dir.string().c_str(), S_IRWXU)) - LOG_ERROR(log, "Failed to chmod {}, errno: {}", unused_dir.string(), errno); - - fs::remove_all(unused_dir); - - return true; - } } static void maybeUnlockUUID(UUID uuid) diff --git a/src/Interpreters/DatabaseCatalog.h b/src/Interpreters/DatabaseCatalog.h index d82ad56eadd..0b3daefb258 100644 --- a/src/Interpreters/DatabaseCatalog.h +++ b/src/Interpreters/DatabaseCatalog.h @@ -31,10 +31,12 @@ class IDatabase; class Exception; class ColumnsDescription; struct ConstraintsDescription; +class IDisk; using DatabasePtr = std::shared_ptr; using DatabaseAndTable = std::pair; using Databases = std::map>; +using DiskPtr = std::shared_ptr; /// Table -> set of table-views that make SELECT from it. using ViewDependencies = std::map>; @@ -271,7 +273,7 @@ private: void dropTableFinally(const TableMarkedAsDropped & table); void cleanupStoreDirectoryTask(); - bool maybeRemoveDirectory(const fs::path & unused_dir); + bool maybeRemoveDirectory(const String & disk_name, const DiskPtr & disk, const String & unused_dir); static constexpr size_t reschedule_time_ms = 100; static constexpr time_t drop_error_cooldown_sec = 5; diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index 3ce2a08ae74..ab380d31494 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -24,6 +24,7 @@ def cluster(): "configs/config.d/storage_conf.xml", "configs/config.d/bg_processing_pool_conf.xml", ], + stay_alive=True, with_minio=True, ) @@ -709,3 +710,27 @@ def test_cache_with_full_disk_space(cluster, node_name): "Insert into cache is skipped due to insufficient disk space" ) node.query("DROP TABLE IF EXISTS s3_test NO DELAY") + + +@pytest.mark.parametrize("node_name", ["node"]) +def test_store_cleanup_disk_s3(cluster, node_name): + node = cluster.instances[node_name] + node.query("DROP TABLE IF EXISTS store_cleanup SYNC") + node.query( + "CREATE TABLE store_cleanup UUID '00000000-1000-4000-8000-000000000001' (n UInt64) Engine=MergeTree() ORDER BY n SETTINGS storage_policy='s3';" + ) + node.query("INSERT INTO store_cleanup SELECT 1") + + node.stop_clickhouse(kill=True) + path_to_data = "/var/lib/clickhouse/" + node.exec_in_container(["rm", f"{path_to_data}/metadata/default/store_cleanup.sql"]) + node.start_clickhouse() + + node.wait_for_log_line( + "Removing unused directory", timeout=90, look_behind_lines=1000 + ) + node.wait_for_log_line("directories from store") + node.query( + "CREATE TABLE store_cleanup UUID '00000000-1000-4000-8000-000000000001' (n UInt64) Engine=MergeTree() ORDER BY n SETTINGS storage_policy='s3';" + ) + node.query("INSERT INTO store_cleanup SELECT 1") From ff26492830551a46ea6c3903baaf3d221a31ed6a Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 4 Aug 2022 00:24:43 +0200 Subject: [PATCH 084/117] Prevent spoiling rollback_stack --- tests/ci/release.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/release.py b/tests/ci/release.py index f982c7c6f53..0e2bdae176d 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -185,7 +185,7 @@ class Release: def log_rollback(self): if self._rollback_stack: - rollback = self._rollback_stack + rollback = self._rollback_stack.copy() rollback.reverse() logging.info( "To rollback the action run the following commands:\n %s", From 8919fd6e58aa18fd0f783b51458bced9a643ce2b Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Thu, 4 Aug 2022 00:40:32 +0200 Subject: [PATCH 085/117] Add handful notes to a post-release logging --- tests/ci/release.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/tests/ci/release.py b/tests/ci/release.py index 0e2bdae176d..c7d54bd77fd 100755 --- a/tests/ci/release.py +++ b/tests/ci/release.py @@ -148,6 +148,7 @@ class Release: with self.stable(): logging.info("Stable part of the releasing is done") + self.log_post_workflows() self.log_rollback() def check_no_tags_after(self): @@ -192,6 +193,15 @@ class Release: "\n ".join(rollback), ) + def log_post_workflows(self): + logging.info( + "To verify all actions are running good visit the following links:\n %s", + "\n ".join( + f"https://github.com/{self.repo}/actions/workflows/{action}.yml" + for action in ("release", "tags_stable") + ), + ) + @contextmanager def create_release_branch(self): self.check_no_tags_after() From 6cc08afb72165e295d59f98acd49a0c10c725d17 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 4 Aug 2022 01:58:08 +0300 Subject: [PATCH 086/117] Update unaligned.h --- base/base/unaligned.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/base/base/unaligned.h b/base/base/unaligned.h index c469972b4b6..842c3b2cdde 100644 --- a/base/base/unaligned.h +++ b/base/base/unaligned.h @@ -10,13 +10,13 @@ inline void reverseMemcpy(void * dst, const void * src, size_t size) uint8_t * uint_dst = reinterpret_cast(dst); const uint8_t * uint_src = reinterpret_cast(src); - uint_dst += length; + uint_dst += size; while (length) { --uint_dst; *uint_dst = *uint_src; ++uint_src; - --length; + --size; } } From 848a349e42c0512ae24a25c58dbb853e4fb8286e Mon Sep 17 00:00:00 2001 From: pzhdfy <982092332@qq.com> Date: Thu, 4 Aug 2022 11:37:27 +0800 Subject: [PATCH 087/117] fix potential netlink leak when init fail in TaskStatsInfoGetter --- src/Common/TaskStatsInfoGetter.cpp | 45 ++++++++++++++++++------------ 1 file changed, 27 insertions(+), 18 deletions(-) diff --git a/src/Common/TaskStatsInfoGetter.cpp b/src/Common/TaskStatsInfoGetter.cpp index 304ccc84765..9f9c0c2b4fa 100644 --- a/src/Common/TaskStatsInfoGetter.cpp +++ b/src/Common/TaskStatsInfoGetter.cpp @@ -238,27 +238,36 @@ TaskStatsInfoGetter::TaskStatsInfoGetter() if (netlink_socket_fd < 0) throwFromErrno("Can't create PF_NETLINK socket", ErrorCodes::NETLINK_ERROR); - /// On some containerized environments, operation on Netlink socket could hang forever. - /// We set reasonably small timeout to overcome this issue. - - struct timeval tv; - tv.tv_sec = 0; - tv.tv_usec = 50000; - - if (0 != ::setsockopt(netlink_socket_fd, SOL_SOCKET, SO_RCVTIMEO, reinterpret_cast(&tv), sizeof(tv))) - throwFromErrno("Can't set timeout on PF_NETLINK socket", ErrorCodes::NETLINK_ERROR); - - union + try { - ::sockaddr_nl addr{}; - ::sockaddr sockaddr; - }; - addr.nl_family = AF_NETLINK; + /// On some containerized environments, operation on Netlink socket could hang forever. + /// We set reasonably small timeout to overcome this issue. - if (::bind(netlink_socket_fd, &sockaddr, sizeof(addr)) < 0) - throwFromErrno("Can't bind PF_NETLINK socket", ErrorCodes::NETLINK_ERROR); + struct timeval tv; + tv.tv_sec = 0; + tv.tv_usec = 50000; - taskstats_family_id = getFamilyId(netlink_socket_fd); + if (0 != ::setsockopt(netlink_socket_fd, SOL_SOCKET, SO_RCVTIMEO, reinterpret_cast(&tv), sizeof(tv))) + throwFromErrno("Can't set timeout on PF_NETLINK socket", ErrorCodes::NETLINK_ERROR); + + union + { + ::sockaddr_nl addr{}; + ::sockaddr sockaddr; + }; + addr.nl_family = AF_NETLINK; + + if (::bind(netlink_socket_fd, &sockaddr, sizeof(addr)) < 0) + throwFromErrno("Can't bind PF_NETLINK socket", ErrorCodes::NETLINK_ERROR); + + taskstats_family_id = getFamilyId(netlink_socket_fd); + } + catch (...) + { + if (netlink_socket_fd >= 0) + close(netlink_socket_fd); + throw; + } } From 24ee9449860f6afae6b6856f66245e3c2144dc6b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 4 Aug 2022 07:34:36 +0300 Subject: [PATCH 088/117] Update TaskStatsInfoGetter.cpp --- src/Common/TaskStatsInfoGetter.cpp | 38 +++++++++++++++--------------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/src/Common/TaskStatsInfoGetter.cpp b/src/Common/TaskStatsInfoGetter.cpp index 9f9c0c2b4fa..b81da2f3fe2 100644 --- a/src/Common/TaskStatsInfoGetter.cpp +++ b/src/Common/TaskStatsInfoGetter.cpp @@ -240,33 +240,33 @@ TaskStatsInfoGetter::TaskStatsInfoGetter() try { - /// On some containerized environments, operation on Netlink socket could hang forever. - /// We set reasonably small timeout to overcome this issue. + /// On some containerized environments, operation on Netlink socket could hang forever. + /// We set reasonably small timeout to overcome this issue. - struct timeval tv; - tv.tv_sec = 0; - tv.tv_usec = 50000; + struct timeval tv; + tv.tv_sec = 0; + tv.tv_usec = 50000; - if (0 != ::setsockopt(netlink_socket_fd, SOL_SOCKET, SO_RCVTIMEO, reinterpret_cast(&tv), sizeof(tv))) - throwFromErrno("Can't set timeout on PF_NETLINK socket", ErrorCodes::NETLINK_ERROR); + if (0 != ::setsockopt(netlink_socket_fd, SOL_SOCKET, SO_RCVTIMEO, reinterpret_cast(&tv), sizeof(tv))) + throwFromErrno("Can't set timeout on PF_NETLINK socket", ErrorCodes::NETLINK_ERROR); - union - { - ::sockaddr_nl addr{}; - ::sockaddr sockaddr; - }; - addr.nl_family = AF_NETLINK; + union + { + ::sockaddr_nl addr{}; + ::sockaddr sockaddr; + }; + addr.nl_family = AF_NETLINK; - if (::bind(netlink_socket_fd, &sockaddr, sizeof(addr)) < 0) - throwFromErrno("Can't bind PF_NETLINK socket", ErrorCodes::NETLINK_ERROR); + if (::bind(netlink_socket_fd, &sockaddr, sizeof(addr)) < 0) + throwFromErrno("Can't bind PF_NETLINK socket", ErrorCodes::NETLINK_ERROR); - taskstats_family_id = getFamilyId(netlink_socket_fd); + taskstats_family_id = getFamilyId(netlink_socket_fd); } catch (...) { - if (netlink_socket_fd >= 0) - close(netlink_socket_fd); - throw; + if (netlink_socket_fd >= 0) + close(netlink_socket_fd); + throw; } } From dd96aee8d261606a3cd1dab12a6bfe8edac50ed0 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Thu, 4 Aug 2022 08:18:45 +0300 Subject: [PATCH 089/117] Update unaligned.h --- base/base/unaligned.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/base/base/unaligned.h b/base/base/unaligned.h index 842c3b2cdde..fcaaa38f2fe 100644 --- a/base/base/unaligned.h +++ b/base/base/unaligned.h @@ -11,7 +11,7 @@ inline void reverseMemcpy(void * dst, const void * src, size_t size) const uint8_t * uint_src = reinterpret_cast(src); uint_dst += size; - while (length) + while (size) { --uint_dst; *uint_dst = *uint_src; From 8094f33078c92a7aed27377b34226e2add19a56a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 4 Aug 2022 06:48:33 +0000 Subject: [PATCH 090/117] Add init file for test --- tests/integration/test_keeper_snapshot_on_exit/__init__.py | 1 + 1 file changed, 1 insertion(+) create mode 100644 tests/integration/test_keeper_snapshot_on_exit/__init__.py diff --git a/tests/integration/test_keeper_snapshot_on_exit/__init__.py b/tests/integration/test_keeper_snapshot_on_exit/__init__.py new file mode 100644 index 00000000000..e5a0d9b4834 --- /dev/null +++ b/tests/integration/test_keeper_snapshot_on_exit/__init__.py @@ -0,0 +1 @@ +#!/usr/bin/env python3 From fa98338ce158e4a714a7440853cf22985db4b036 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 4 Aug 2022 07:13:42 +0000 Subject: [PATCH 091/117] Add comment for memory tracker blocking --- src/Coordination/KeeperServer.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index a7c9b0836f1..6e017e36919 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -176,6 +176,11 @@ struct KeeperServer::KeeperRaftServer : public nuraft::raft_server void commit_in_bg() override { + // For NuRaft, if any commit fails (uncaught exception) the whole server aborts as a safety + // This includes failed allocation which can produce an unknown state for the storage, + // making it impossible to handle correctly. + // We block the memory tracker for all the commit operations (including KeeperStateMachine::commit) + // assuming that the allocations are small MemoryTrackerBlockerInThread blocker; nuraft::raft_server::commit_in_bg(); } From 278921be3b57d6a2ce74af03bad44a6e346398f0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 4 Aug 2022 07:25:28 +0000 Subject: [PATCH 092/117] Fix style --- src/Coordination/KeeperServer.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 6e017e36919..82a7a54401d 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -179,7 +179,7 @@ struct KeeperServer::KeeperRaftServer : public nuraft::raft_server // For NuRaft, if any commit fails (uncaught exception) the whole server aborts as a safety // This includes failed allocation which can produce an unknown state for the storage, // making it impossible to handle correctly. - // We block the memory tracker for all the commit operations (including KeeperStateMachine::commit) + // We block the memory tracker for all the commit operations (including KeeperStateMachine::commit) // assuming that the allocations are small MemoryTrackerBlockerInThread blocker; nuraft::raft_server::commit_in_bg(); From 92882fe3374f8dabdd8443ca19306b2b622435df Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Thu, 4 Aug 2022 11:08:59 +0200 Subject: [PATCH 093/117] Update CMakeLists.txt --- contrib/azure-cmake/CMakeLists.txt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/azure-cmake/CMakeLists.txt b/contrib/azure-cmake/CMakeLists.txt index 367131daf0b..1e2a4c97824 100644 --- a/contrib/azure-cmake/CMakeLists.txt +++ b/contrib/azure-cmake/CMakeLists.txt @@ -1,6 +1,6 @@ option (ENABLE_AZURE_BLOB_STORAGE "Enable Azure blob storage" ${ENABLE_LIBRARIES}) -if (NOT ENABLE_AZURE_BLOB_STORAGE OR BUILD_STANDALONE_KEEPER) +if (NOT ENABLE_AZURE_BLOB_STORAGE OR BUILD_STANDALONE_KEEPER OR OS_FREEBSD) message(STATUS "Not using Azure blob storage") return() endif() From 1133425624ba6312c8e9fed1544d10dec0d52142 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 4 Aug 2022 12:16:38 +0300 Subject: [PATCH 094/117] Update storage_conf.xml --- .../test_merge_tree_s3/configs/config.d/storage_conf.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml index e414ae5a259..c021a2163d3 100644 --- a/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml +++ b/tests/integration/test_merge_tree_s3/configs/config.d/storage_conf.xml @@ -90,4 +90,8 @@ 0 + + 0 + 60 + 1 From 0d892b1cac4f12558b4204f81c68fbd8b9b6907f Mon Sep 17 00:00:00 2001 From: Constantine Peresypkin Date: Thu, 4 Aug 2022 13:23:45 +0200 Subject: [PATCH 095/117] fix broken NFS mkdir introduced in #36341 --- docker/server/entrypoint.sh | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index edda079e7e2..fae11cd267e 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -57,7 +57,15 @@ do # check if variable not empty [ -z "$dir" ] && continue # ensure directories exist - if ! mkdir -p "$dir"; then + if [ "$DO_CHOWN" = "1" ]; then + mkdir="mkdir" + else + # if DO_CHOWN=0 it means that the system does not map root user to "admin" permissions + # it mainly happens on NFS mounts where root==nobody for security reasons + # thus mkdir MUST run with user id/gid and not from nobody that has zero permissions + mkdir="/usr/bin/clickhouse su "${USER}:${GROUP}" mkdir" + fi + if ! $mkdir -p "$dir"; then echo "Couldn't create necessary directory: $dir" exit 1 fi From 7caf4c210e6ead19415bdc1a404b7df43ce0b36f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 4 Aug 2022 14:06:19 +0200 Subject: [PATCH 096/117] Update local queue to see current mutation entry --- src/Storages/StorageReplicatedMergeTree.cpp | 9 ++++++--- src/Storages/StorageReplicatedMergeTree.h | 4 ++-- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 1ec14f643d4..8bfe025002e 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -501,11 +501,14 @@ bool StorageReplicatedMergeTree::checkFixedGranularityInZookeeper() void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( - const Strings & replicas, const String & mutation_id) const + const Strings & replicas, const String & mutation_id) { if (replicas.empty()) return; + /// We need to make sure that local queue has been synced from zookeeper and contains the entry for the + /// current mutation. We use local mutation status for checkig for errors. + queue.updateMutations(getZooKeeper()); std::set inactive_replicas; for (const String & replica : replicas) @@ -5958,7 +5961,7 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, Conte waitMutation(mutation_entry.znode_name, query_context->getSettingsRef().mutations_sync); } -void StorageReplicatedMergeTree::waitMutation(const String & znode_name, size_t mutations_sync) const +void StorageReplicatedMergeTree::waitMutation(const String & znode_name, size_t mutations_sync) { if (!mutations_sync) return; @@ -5974,7 +5977,7 @@ void StorageReplicatedMergeTree::waitMutation(const String & znode_name, size_t { if (*it == replica_name) { - std::iter_swap(it, replicas.rbegin()); + std::iter_swap(it, replicas.begin()); break; } } diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index c35e2d5cf5c..d7c9e047f0b 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -148,7 +148,7 @@ public: void alter(const AlterCommands & commands, ContextPtr query_context, AlterLockHolder & table_lock_holder) override; void mutate(const MutationCommands & commands, ContextPtr context) override; - void waitMutation(const String & znode_name, size_t mutations_sync) const; + void waitMutation(const String & znode_name, size_t mutations_sync); std::vector getMutationsStatus() const override; CancellationCode killMutation(const String & mutation_id) override; @@ -796,7 +796,7 @@ private: /// Wait for timeout seconds mutation is finished on replicas void waitMutationToFinishOnReplicas( - const Strings & replicas, const String & mutation_id) const; + const Strings & replicas, const String & mutation_id); MutationCommands getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const override; From 0a7d2e7b8ab4e45e18b1cf95eca008109f7dcbae Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 4 Aug 2022 13:03:05 +0000 Subject: [PATCH 097/117] Use LockMemoryExceptionThread --- src/Coordination/KeeperServer.cpp | 4 ++-- src/Coordination/KeeperStorage.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index 82a7a54401d..24551f27ec2 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -20,7 +20,7 @@ #include #include #include -#include +#include #include #include @@ -181,7 +181,7 @@ struct KeeperServer::KeeperRaftServer : public nuraft::raft_server // making it impossible to handle correctly. // We block the memory tracker for all the commit operations (including KeeperStateMachine::commit) // assuming that the allocations are small - MemoryTrackerBlockerInThread blocker; + LockMemoryExceptionInThread blocker{VariableContext::Global}; nuraft::raft_server::commit_in_bg(); } diff --git a/src/Coordination/KeeperStorage.cpp b/src/Coordination/KeeperStorage.cpp index bbc647fd951..693a1b16f0d 100644 --- a/src/Coordination/KeeperStorage.cpp +++ b/src/Coordination/KeeperStorage.cpp @@ -14,7 +14,7 @@ #include #include #include -#include +#include #include #include #include @@ -2127,7 +2127,7 @@ void KeeperStorage::rollbackRequest(int64_t rollback_zxid, bool allow_missing) // if an exception occurs during rollback, the best option is to terminate because we can end up in an inconsistent state // we block memory tracking so we can avoid terminating if we're rollbacking because of memory limit - MemoryTrackerBlockerInThread temporarily_ignore_any_memory_limits; + LockMemoryExceptionInThread blocker{VariableContext::Global}; try { uncommitted_transactions.pop_back(); From 10d7259c2b21c583abaa824507a1238eb5f26fb9 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 4 Aug 2022 13:12:24 +0000 Subject: [PATCH 098/117] Add log for snapshot on exit --- src/Coordination/KeeperStateMachine.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Coordination/KeeperStateMachine.cpp b/src/Coordination/KeeperStateMachine.cpp index a55acaf9b91..a568bb88302 100644 --- a/src/Coordination/KeeperStateMachine.cpp +++ b/src/Coordination/KeeperStateMachine.cpp @@ -385,6 +385,7 @@ void KeeperStateMachine::create_snapshot(nuraft::snapshot & s, nuraft::async_res if (keeper_context->server_state == KeeperContext::Phase::SHUTDOWN) { + LOG_INFO(log, "Creating a snapshot during shutdown because 'create_snapshot_on_exit' is enabled."); snapshot_task.create_snapshot(std::move(snapshot_task.snapshot)); return; } From f63e4ba261b4ba3ddbb03f3ff591507eb6590d1a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 4 Aug 2022 13:14:57 +0000 Subject: [PATCH 099/117] Update NuRaft --- contrib/NuRaft | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/NuRaft b/contrib/NuRaft index d73e12adf75..1b0af760b35 160000 --- a/contrib/NuRaft +++ b/contrib/NuRaft @@ -1 +1 @@ -Subproject commit d73e12adf7557a0ebca7f7ecde68c064dee22fa0 +Subproject commit 1b0af760b3506b8e35b50cb7df098cbad5064ff2 From 6a5171c82956ab79d5f20ffaca7cad27e4bb651b Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 4 Aug 2022 17:10:13 +0300 Subject: [PATCH 100/117] Update DatabaseCatalog.cpp --- src/Interpreters/DatabaseCatalog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/DatabaseCatalog.cpp b/src/Interpreters/DatabaseCatalog.cpp index a92a82ea821..e0887423459 100644 --- a/src/Interpreters/DatabaseCatalog.cpp +++ b/src/Interpreters/DatabaseCatalog.cpp @@ -891,7 +891,7 @@ void DatabaseCatalog::enqueueDroppedTableCleanup(StorageID table_id, StoragePtr create->setTable(table_id.table_name); try { - table = createTableFromAST(*create, table_id.getDatabaseName(), data_path, getContext(), true).second; + table = createTableFromAST(*create, table_id.getDatabaseName(), data_path, getContext(), /* force_restore */ true).second; table->is_dropped = true; } catch (...) From 60599197b2bfac4beedc5a9f99ee4fb4c34d5d00 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 4 Aug 2022 15:23:10 +0000 Subject: [PATCH 101/117] Review fixes. --- src/QueryPipeline/QueryPipelineBuilder.h | 2 ++ src/Storages/MergeTree/KeyCondition.cpp | 12 ++++++------ 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/QueryPipeline/QueryPipelineBuilder.h b/src/QueryPipeline/QueryPipelineBuilder.h index a48828be8b9..346b47a1d1d 100644 --- a/src/QueryPipeline/QueryPipelineBuilder.h +++ b/src/QueryPipeline/QueryPipelineBuilder.h @@ -95,6 +95,8 @@ public: void resize(size_t num_streams, bool force = false, bool strict = false); /// Concat some ports to have no more then size outputs. + /// This method is needed for Merge table engine in case of reading from many tables. + /// It prevents opening too many files at the same time. void narrow(size_t size); /// Unite several pipelines together. Result pipeline would have common_header structure. diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 0fda619d1b5..a44ecd512ea 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -223,7 +223,7 @@ public: return res; } - bool getConstant(const Block & block_with_constants, Field & out_value, DataTypePtr & out_type) const + bool tryGetConstant(const Block & block_with_constants, Field & out_value, DataTypePtr & out_type) const { if (ast) { @@ -719,7 +719,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown( case (ActionsDAG::ActionType::ARRAY_JOIN): { const auto & arg = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, false); - res = &inverted_dag.addArrayJoin(arg, ""); + res = &inverted_dag.addArrayJoin(arg, {}); break; } case (ActionsDAG::ActionType::FUNCTION): @@ -971,7 +971,7 @@ bool KeyCondition::addCondition(const String & column, const Range & range) */ bool KeyCondition::getConstant(const ASTPtr & expr, Block & block_with_constants, Field & out_value, DataTypePtr & out_type) { - return Tree(expr.get()).getConstant(block_with_constants, out_value, out_type); + return Tree(expr.get()).tryGetConstant(block_with_constants, out_value, out_type); } @@ -1612,7 +1612,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl else return false; } - else if (func.getArgumentAt(1).getConstant(block_with_constants, const_value, const_type)) + else if (func.getArgumentAt(1).tryGetConstant(block_with_constants, const_value, const_type)) { if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), context, key_column_num, key_expr_type, chain)) { @@ -1635,7 +1635,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl else return false; } - else if (func.getArgumentAt(0).getConstant(block_with_constants, const_value, const_type)) + else if (func.getArgumentAt(0).tryGetConstant(block_with_constants, const_value, const_type)) { if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(1), context, key_column_num, key_expr_type, chain)) { @@ -1762,7 +1762,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl return atom_it->second(out, const_value); } - else if (node.getConstant(block_with_constants, const_value, const_type)) + else if (node.tryGetConstant(block_with_constants, const_value, const_type)) { /// For cases where it says, for example, `WHERE 0 AND something` From 20cf4e8d227aab2d790461ca71a12cc090491145 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 4 Aug 2022 18:04:06 +0200 Subject: [PATCH 102/117] slightly more readable if conditions --- src/Server/TCPHandler.cpp | 22 +++++++++++++++------- 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index ed46abd9c36..6e45969bd34 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -759,13 +759,21 @@ void TCPHandler::processTablesStatusRequest() TablesStatusRequest request; request.read(*in, client_tcp_protocol_version); - ContextPtr context_to_resolve_table_names = (session && session->sessionContext()) ? session->sessionContext() : server.context(); - if (is_interserver_mode && !default_database.empty()) + ContextPtr context_to_resolve_table_names; + if (is_interserver_mode) { - // in interserver mode, the session doesn't exist. - ContextMutablePtr interserver_context = Context::createCopy(context_to_resolve_table_names); - interserver_context->setCurrentDatabase(default_database); - context_to_resolve_table_names = interserver_context; + /// In interserver mode session context does not exists, because authentication is done for each query. + /// We also cannot create query context earlier, because it cannot be created before authentication, + /// but query is not received yet. So we have to do this trick. + ContextMutablePtr fake_interserver_context = Context::createCopy(server.context()); + if (!default_database.empty()) + fake_interserver_context->setCurrentDatabase(default_database); + context_to_resolve_table_names = fake_interserver_context; + } + else + { + assert(session); + context_to_resolve_table_names = session->sessionContext(); } TablesStatusResponse response; @@ -1361,7 +1369,7 @@ void TCPHandler::receiveQuery() query_context = session->makeQueryContext(std::move(client_info)); /// Sets the default database if it wasn't set earlier for the session context. - if (!default_database.empty() && !session->sessionContext()) + if (is_interserver_mode && !default_database.empty()) query_context->setCurrentDatabase(default_database); if (state.part_uuids_to_ignore) From 110b60e5b70e59e74f0d32c3a2135a99d65fa274 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 4 Aug 2022 19:48:55 +0200 Subject: [PATCH 103/117] fix tests --- tests/integration/test_merge_tree_s3/test.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/integration/test_merge_tree_s3/test.py b/tests/integration/test_merge_tree_s3/test.py index ab380d31494..8a73cafd30f 100644 --- a/tests/integration/test_merge_tree_s3/test.py +++ b/tests/integration/test_merge_tree_s3/test.py @@ -715,15 +715,15 @@ def test_cache_with_full_disk_space(cluster, node_name): @pytest.mark.parametrize("node_name", ["node"]) def test_store_cleanup_disk_s3(cluster, node_name): node = cluster.instances[node_name] - node.query("DROP TABLE IF EXISTS store_cleanup SYNC") + node.query("DROP TABLE IF EXISTS s3_test SYNC") node.query( - "CREATE TABLE store_cleanup UUID '00000000-1000-4000-8000-000000000001' (n UInt64) Engine=MergeTree() ORDER BY n SETTINGS storage_policy='s3';" + "CREATE TABLE s3_test UUID '00000000-1000-4000-8000-000000000001' (n UInt64) Engine=MergeTree() ORDER BY n SETTINGS storage_policy='s3';" ) - node.query("INSERT INTO store_cleanup SELECT 1") + node.query("INSERT INTO s3_test SELECT 1") node.stop_clickhouse(kill=True) path_to_data = "/var/lib/clickhouse/" - node.exec_in_container(["rm", f"{path_to_data}/metadata/default/store_cleanup.sql"]) + node.exec_in_container(["rm", f"{path_to_data}/metadata/default/s3_test.sql"]) node.start_clickhouse() node.wait_for_log_line( @@ -731,6 +731,6 @@ def test_store_cleanup_disk_s3(cluster, node_name): ) node.wait_for_log_line("directories from store") node.query( - "CREATE TABLE store_cleanup UUID '00000000-1000-4000-8000-000000000001' (n UInt64) Engine=MergeTree() ORDER BY n SETTINGS storage_policy='s3';" + "CREATE TABLE s3_test UUID '00000000-1000-4000-8000-000000000001' (n UInt64) Engine=MergeTree() ORDER BY n SETTINGS storage_policy='s3';" ) - node.query("INSERT INTO store_cleanup SELECT 1") + node.query("INSERT INTO s3_test SELECT 1") From f5e358ecc57d6d545200b9cb09b9c2ed4af0c1fb Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Thu, 4 Aug 2022 20:26:39 +0200 Subject: [PATCH 104/117] Make sure that the current replica is the first on the list --- src/Storages/StorageReplicatedMergeTree.cpp | 19 +++++++++++++------ src/Storages/StorageReplicatedMergeTree.h | 4 ++-- 2 files changed, 15 insertions(+), 8 deletions(-) diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8bfe025002e..6a3f52f4e23 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -501,17 +501,24 @@ bool StorageReplicatedMergeTree::checkFixedGranularityInZookeeper() void StorageReplicatedMergeTree::waitMutationToFinishOnReplicas( - const Strings & replicas, const String & mutation_id) + const Strings & replicas, const String & mutation_id) const { if (replicas.empty()) return; - /// We need to make sure that local queue has been synced from zookeeper and contains the entry for the - /// current mutation. We use local mutation status for checkig for errors. - queue.updateMutations(getZooKeeper()); + /// Current replica must always be present in the list as the first element because we use local mutation status + /// to check for mutation errors. So if it is not there, just add it. + const Strings * all_required_replicas = &replicas; + Strings extended_list_of_replicas; + if (replicas.front() != replica_name) + { + extended_list_of_replicas.push_back(replica_name); + extended_list_of_replicas.insert(extended_list_of_replicas.end(), replicas.begin(), replicas.end()); + all_required_replicas = &extended_list_of_replicas; + } std::set inactive_replicas; - for (const String & replica : replicas) + for (const String & replica : *all_required_replicas) { LOG_DEBUG(log, "Waiting for {} to apply mutation {}", replica, mutation_id); zkutil::EventPtr wait_event = std::make_shared(); @@ -5961,7 +5968,7 @@ void StorageReplicatedMergeTree::mutate(const MutationCommands & commands, Conte waitMutation(mutation_entry.znode_name, query_context->getSettingsRef().mutations_sync); } -void StorageReplicatedMergeTree::waitMutation(const String & znode_name, size_t mutations_sync) +void StorageReplicatedMergeTree::waitMutation(const String & znode_name, size_t mutations_sync) const { if (!mutations_sync) return; diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index d7c9e047f0b..c35e2d5cf5c 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -148,7 +148,7 @@ public: void alter(const AlterCommands & commands, ContextPtr query_context, AlterLockHolder & table_lock_holder) override; void mutate(const MutationCommands & commands, ContextPtr context) override; - void waitMutation(const String & znode_name, size_t mutations_sync); + void waitMutation(const String & znode_name, size_t mutations_sync) const; std::vector getMutationsStatus() const override; CancellationCode killMutation(const String & mutation_id) override; @@ -796,7 +796,7 @@ private: /// Wait for timeout seconds mutation is finished on replicas void waitMutationToFinishOnReplicas( - const Strings & replicas, const String & mutation_id); + const Strings & replicas, const String & mutation_id) const; MutationCommands getFirstAlterMutationCommandsForPart(const DataPartPtr & part) const override; From 6a3a9dcf964f59428eeff8eed250dae629c67797 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 5 Aug 2022 05:29:26 +0800 Subject: [PATCH 105/117] Fix building aggregate projections when external aggregation is on (#39671) --- src/Storages/ProjectionsDescription.cpp | 4 +++- .../01710_projection_aggregation_in_order.sql | 4 ++++ ...10_projection_external_aggregate.reference | 0 .../01710_projection_external_aggregate.sql | 24 +++++++++++++++++++ 4 files changed, 31 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01710_projection_external_aggregate.reference create mode 100644 tests/queries/0_stateless/01710_projection_external_aggregate.sql diff --git a/src/Storages/ProjectionsDescription.cpp b/src/Storages/ProjectionsDescription.cpp index 0f24aecf72a..969577fdf3f 100644 --- a/src/Storages/ProjectionsDescription.cpp +++ b/src/Storages/ProjectionsDescription.cpp @@ -286,7 +286,9 @@ Block ProjectionDescription::calculate(const Block & block, ContextPtr context) : QueryProcessingStage::WithMergeableState}) .buildQueryPipeline(); builder.resize(1); - builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), block.bytes())); + // Generate aggregated blocks with rows less or equal than the original block. + // There should be only one output block after this transformation. + builder.addTransform(std::make_shared(builder.getHeader(), block.rows(), 0)); auto pipeline = QueryPipelineBuilder::getPipeline(std::move(builder)); PullingPipelineExecutor executor(pipeline); diff --git a/tests/queries/0_stateless/01710_projection_aggregation_in_order.sql b/tests/queries/0_stateless/01710_projection_aggregation_in_order.sql index 06f05e36237..31d32da0ed3 100644 --- a/tests/queries/0_stateless/01710_projection_aggregation_in_order.sql +++ b/tests/queries/0_stateless/01710_projection_aggregation_in_order.sql @@ -31,6 +31,8 @@ SET allow_experimental_projection_optimization=1, optimize_aggregation_in_order= WITH toStartOfHour(ts) AS a SELECT sum(value) v FROM normal WHERE ts > '2021-12-06 22:00:00' GROUP BY a ORDER BY v LIMIT 5; WITH toStartOfHour(ts) AS a SELECT sum(value) v FROM normal WHERE ts > '2021-12-06 22:00:00' GROUP BY toStartOfHour(ts), a ORDER BY v LIMIT 5; +DROP TABLE normal; + DROP TABLE IF EXISTS agg; CREATE TABLE agg @@ -60,3 +62,5 @@ SET allow_experimental_projection_optimization=1, optimize_aggregation_in_order= WITH toStartOfHour(ts) AS a SELECT sum(value) v FROM agg WHERE ts > '2021-12-06 22:00:00' GROUP BY a ORDER BY v LIMIT 5; WITH toStartOfHour(ts) AS a SELECT sum(value) v FROM agg WHERE ts > '2021-12-06 22:00:00' GROUP BY toStartOfHour(ts), a ORDER BY v LIMIT 5; + +DROP TABLE agg; diff --git a/tests/queries/0_stateless/01710_projection_external_aggregate.reference b/tests/queries/0_stateless/01710_projection_external_aggregate.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01710_projection_external_aggregate.sql b/tests/queries/0_stateless/01710_projection_external_aggregate.sql new file mode 100644 index 00000000000..8c9c1522af8 --- /dev/null +++ b/tests/queries/0_stateless/01710_projection_external_aggregate.sql @@ -0,0 +1,24 @@ +DROP TABLE IF EXISTS agg; + +CREATE TABLE agg +( + `key` UInt32, + `ts` DateTime, + `value` UInt32, + PROJECTION aaaa + ( + SELECT + ts, + key, + sum(value) + GROUP BY ts, key + ) +) +ENGINE = MergeTree +ORDER BY (key, ts); + +SET max_bytes_before_external_group_by=1; + +INSERT INTO agg SELECT 1, toDateTime('2021-12-06 00:00:00') + number, number FROM numbers(100000); + +DROP TABLE agg; From a25b85809717dea097cd67d39e5395fc10aee3f6 Mon Sep 17 00:00:00 2001 From: santrancisco Date: Fri, 5 Aug 2022 09:51:12 +1000 Subject: [PATCH 106/117] Move username and password from URL parameter to Basic Authentication header --- programs/server/play.html | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/programs/server/play.html b/programs/server/play.html index c7ea5e4ef12..f678da6f353 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -505,8 +505,6 @@ (server_address.indexOf('?') >= 0 ? '&' : '?') + /// Ask server to allow cross-domain requests. 'add_http_cors_header=1' + - '&user=' + encodeURIComponent(user) + - '&password=' + encodeURIComponent(password) + '&default_format=JSONCompact' + /// Safety settings to prevent results that browser cannot display. '&max_result_rows=1000&max_result_bytes=10000000&result_overflow_mode=break'; @@ -514,6 +512,7 @@ const xhr = new XMLHttpRequest; xhr.open('POST', url, true); + xhr.setRequestHeader("Authorization", "Basic " + btoa(user+":"+password)); xhr.onreadystatechange = function() { From 5363e2cf47d9e8f9c6f092194ab304758f694847 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 5 Aug 2022 05:13:46 +0300 Subject: [PATCH 107/117] Remove cache flush from the Docs Check --- docs/tools/release.sh | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/docs/tools/release.sh b/docs/tools/release.sh index b55841f9da2..1d344457bf1 100755 --- a/docs/tools/release.sh +++ b/docs/tools/release.sh @@ -36,15 +36,4 @@ then # Push to GitHub rewriting the existing contents. # Sometimes it does not work with error message "! [remote rejected] master -> master (cannot lock ref 'refs/heads/master': is at 42a0f6b6b6c7be56a469441b4bf29685c1cebac3 but expected 520e9b02c0d4678a2a5f41d2f561e6532fb98cc1)" for _ in {1..10}; do git push --force origin master && break; sleep 5; done - - # Turn off logging. - set +x - - if [[ -n "${CLOUDFLARE_TOKEN}" ]] - then - sleep 1m - # https://api.cloudflare.com/#zone-purge-files-by-cache-tags,-host-or-prefix - POST_DATA='{"hosts":["clickhouse.com"]}' - curl -X POST "https://api.cloudflare.com/client/v4/zones/4fc6fb1d46e87851605aa7fa69ca6fe0/purge_cache" -H "Authorization: Bearer ${CLOUDFLARE_TOKEN}" -H "Content-Type:application/json" --data "${POST_DATA}" - fi fi From 3f4baa765890b7008f31215cdd270da1f57367b5 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Thu, 4 Aug 2022 23:49:19 -0300 Subject: [PATCH 108/117] Update mergetree.md --- docs/en/engines/table-engines/mergetree-family/mergetree.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/en/engines/table-engines/mergetree-family/mergetree.md b/docs/en/engines/table-engines/mergetree-family/mergetree.md index 2ca07276e63..9339ec2454f 100644 --- a/docs/en/engines/table-engines/mergetree-family/mergetree.md +++ b/docs/en/engines/table-engines/mergetree-family/mergetree.md @@ -878,8 +878,6 @@ User can assign new big parts to different disks of a [JBOD](https://en.wikipedi `MergeTree` family table engines can store data to [S3](https://aws.amazon.com/s3/) using a disk with type `s3`. -This feature is under development and not ready for production. There are known drawbacks such as very low performance. - Configuration markup: ``` xml From c7f0598a6ff4a42847438964c28ab8a21ac1feed Mon Sep 17 00:00:00 2001 From: santrancisco Date: Fri, 5 Aug 2022 13:10:45 +1000 Subject: [PATCH 109/117] Fall back to url parameters if file is opened locally --- programs/server/play.html | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/programs/server/play.html b/programs/server/play.html index f678da6f353..ab04722609c 100644 --- a/programs/server/play.html +++ b/programs/server/play.html @@ -501,7 +501,8 @@ const server_address = document.getElementById('url').value; - const url = server_address + + + var url = server_address + (server_address.indexOf('?') >= 0 ? '&' : '?') + /// Ask server to allow cross-domain requests. 'add_http_cors_header=1' + @@ -509,11 +510,19 @@ /// Safety settings to prevent results that browser cannot display. '&max_result_rows=1000&max_result_bytes=10000000&result_overflow_mode=break'; + // If play.html is opened locally, append username and password to the URL parameter to avoid CORS issue. + if (document.location.href.startsWith("file://")) { + url += '&user=' + encodeURIComponent(user) + + '&password=' + encodeURIComponent(password) + } + const xhr = new XMLHttpRequest; xhr.open('POST', url, true); - xhr.setRequestHeader("Authorization", "Basic " + btoa(user+":"+password)); - + // If play.html is open normally, use Basic auth to prevent username and password being exposed in URL parameters + if (!document.location.href.startsWith("file://")) { + xhr.setRequestHeader("Authorization", "Basic " + btoa(user+":"+password)); + } xhr.onreadystatechange = function() { if (posted_request_num != request_num) { From 67a6b32a8985854667a0850abfbdd9bd7feda4eb Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Fri, 5 Aug 2022 07:12:50 +0300 Subject: [PATCH 110/117] Update entrypoint.sh --- docker/server/entrypoint.sh | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docker/server/entrypoint.sh b/docker/server/entrypoint.sh index fae11cd267e..89dd501bf38 100755 --- a/docker/server/entrypoint.sh +++ b/docker/server/entrypoint.sh @@ -58,12 +58,12 @@ do [ -z "$dir" ] && continue # ensure directories exist if [ "$DO_CHOWN" = "1" ]; then - mkdir="mkdir" + mkdir="mkdir" else - # if DO_CHOWN=0 it means that the system does not map root user to "admin" permissions - # it mainly happens on NFS mounts where root==nobody for security reasons - # thus mkdir MUST run with user id/gid and not from nobody that has zero permissions - mkdir="/usr/bin/clickhouse su "${USER}:${GROUP}" mkdir" + # if DO_CHOWN=0 it means that the system does not map root user to "admin" permissions + # it mainly happens on NFS mounts where root==nobody for security reasons + # thus mkdir MUST run with user id/gid and not from nobody that has zero permissions + mkdir="/usr/bin/clickhouse su "${USER}:${GROUP}" mkdir" fi if ! $mkdir -p "$dir"; then echo "Couldn't create necessary directory: $dir" From 4bf7a2ca3a3bc8f0093c619e8a23d26b61d6dc41 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 5 Aug 2022 12:05:50 +0200 Subject: [PATCH 111/117] fix warning --- src/Storages/MergeTree/MergeTreeData.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 4fc8f77b5b6..6b4ec3a6dee 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -5009,7 +5009,13 @@ MergeTreeData::DataPartsVector MergeTreeData::Transaction::commit(MergeTreeData: DataPartsVector covered_parts = data.getActivePartsToReplace(part->info, part->name, covering_part, *owing_parts_lock); if (covering_part) { - LOG_WARNING(data.log, "Tried to commit obsolete part {} covered by {}", part->name, covering_part->getNameWithState()); + /// It's totally fine for zero-level parts, because of possible race condition between ReplicatedMergeTreeSink and + /// background queue execution (new part is added to ZK before this function is called, + /// so other replica may produce covering part and replication queue may download covering part). + if (part->info.level) + LOG_WARNING(data.log, "Tried to commit obsolete part {} covered by {}", part->name, covering_part->getNameWithState()); + else + LOG_INFO(data.log, "Tried to commit obsolete part {} covered by {}", part->name, covering_part->getNameWithState()); part->remove_time.store(0, std::memory_order_relaxed); /// The part will be removed without waiting for old_parts_lifetime seconds. data.modifyPartState(part, DataPartState::Outdated); From 4efa847a1f08dc71d5062dc6f3e36e4aecb9ddf2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 20 Jul 2022 22:29:46 +0300 Subject: [PATCH 112/117] Fix LOGICAL_ERROR on race between DROP and INSERT with materialized views In case of parallel INSERT (max_insert_threads > 1) it is possible for VIEW to be DROP/DETACH'ed while building pipeline for various paralell streams, and in this case the header will not match since when you have VIEW you will got empty header and non-empty header otherwise. And this leads to LOGICAL_ERROR later, while checking that output headers are the same (in QueryPipelineBuilder::addChains() -> Pipe::addChains()). However it also makes the pipeline different for various parallel streams, and it looks like it is better to fail in this case, so instead of always returning empty header from buildChainImpl() explicit check had been added. Note, that I wasn't able to reproduce the issue with the added test, but CI may have more "luck" (although I've verified it manually). Fixes: #35902 Cc: @KochetovNicolai Signed-off-by: Azat Khuzhin --- src/Interpreters/InterpreterInsertQuery.cpp | 10 +++ .../02380_insert_mv_race.reference | 0 .../0_stateless/02380_insert_mv_race.sh | 66 +++++++++++++++++++ 3 files changed, 76 insertions(+) create mode 100644 tests/queries/0_stateless/02380_insert_mv_race.reference create mode 100755 tests/queries/0_stateless/02380_insert_mv_race.sh diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 7b6066575ae..493d8df57ff 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -44,6 +44,7 @@ namespace ErrorCodes extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int ILLEGAL_COLUMN; extern const int DUPLICATE_COLUMN; + extern const int TABLE_IS_DROPPED; } InterpreterInsertQuery::InterpreterInsertQuery( @@ -424,6 +425,15 @@ BlockIO InterpreterInsertQuery::execute() for (size_t i = 0; i < out_streams_size; ++i) { auto out = buildChainImpl(table, metadata_snapshot, query_sample_block, nullptr, nullptr); + if (!out_chains.empty()) + { + if (out.getProcessors().size() != out_chains.back().getProcessors().size()) + { + throw Exception(ErrorCodes::TABLE_IS_DROPPED, + "Some VIEW is gone in between ({} vs {} processors, on {} parallel stream)", + out.getProcessors().size(), out_chains.back().getProcessors().size(), i); + } + } out_chains.emplace_back(std::move(out)); } } diff --git a/tests/queries/0_stateless/02380_insert_mv_race.reference b/tests/queries/0_stateless/02380_insert_mv_race.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02380_insert_mv_race.sh b/tests/queries/0_stateless/02380_insert_mv_race.sh new file mode 100755 index 00000000000..ba002832715 --- /dev/null +++ b/tests/queries/0_stateless/02380_insert_mv_race.sh @@ -0,0 +1,66 @@ +#!/usr/bin/env bash +# Tags: long, race + +# Regression test for INSERT into table with MV attached, +# to avoid possible errors if some table will disappears, +# in case of multiple streams was used (i.e. max_insert_threads>1) + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +function bootstrap() +{ + $CLICKHOUSE_CLIENT -nm -q " + DROP TABLE IF EXISTS null; + CREATE TABLE null (key Int) ENGINE = Null; + + DROP TABLE IF EXISTS mv; + CREATE MATERIALIZED VIEW mv ENGINE = Null() AS SELECT * FROM null; + " +} + +function insert_thread() +{ + local opts=( + --max_insert_threads 100 + --max_threads 100 + ) + local patterns=( + -e UNKNOWN_TABLE + -e TABLE_IS_DROPPED + ) + + while :; do + $CLICKHOUSE_CLIENT "${opts[@]}" -q "INSERT INTO null SELECT * FROM numbers_mt(1e6)" |& { + grep -F "DB::Exception: " | grep -v -F "${patterns[@]}" + } + done +} +export -f insert_thread + +function drop_thread() +{ + local opts=( + --database_atomic_wait_for_drop_and_detach_synchronously 1 + ) + + while :; do + $CLICKHOUSE_CLIENT -nm "${opts[@]}" -q "DETACH TABLE mv" + sleep 0.01 + $CLICKHOUSE_CLIENT -nm "${opts[@]}" -q "ATTACH TABLE mv" + done +} +export -f drop_thread + +function main() +{ + local test_timeout=1m + + bootstrap + timeout "$test_timeout" bash -c insert_thread & + timeout "$test_timeout" bash -c drop_thread & + + wait +} +main "$@" From cf342326755973e1f0f54d1c9346fa3d8f2ff71d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 1 Aug 2022 11:48:36 +0000 Subject: [PATCH 113/117] Output header is now empty for every MV chain. Instead of checking that number of processors different for different threads, simply always return empty header from buildChainImpl(), by adding explicit conversion. v2: ignore UNKNOWN_TABLE errors in test --- src/Interpreters/InterpreterInsertQuery.cpp | 10 --- .../Transforms/buildPushingToViewsChain.cpp | 8 +++ .../0_stateless/02380_insert_mv_race.sh | 68 ++++--------------- 3 files changed, 22 insertions(+), 64 deletions(-) diff --git a/src/Interpreters/InterpreterInsertQuery.cpp b/src/Interpreters/InterpreterInsertQuery.cpp index 493d8df57ff..7b6066575ae 100644 --- a/src/Interpreters/InterpreterInsertQuery.cpp +++ b/src/Interpreters/InterpreterInsertQuery.cpp @@ -44,7 +44,6 @@ namespace ErrorCodes extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int ILLEGAL_COLUMN; extern const int DUPLICATE_COLUMN; - extern const int TABLE_IS_DROPPED; } InterpreterInsertQuery::InterpreterInsertQuery( @@ -425,15 +424,6 @@ BlockIO InterpreterInsertQuery::execute() for (size_t i = 0; i < out_streams_size; ++i) { auto out = buildChainImpl(table, metadata_snapshot, query_sample_block, nullptr, nullptr); - if (!out_chains.empty()) - { - if (out.getProcessors().size() != out_chains.back().getProcessors().size()) - { - throw Exception(ErrorCodes::TABLE_IS_DROPPED, - "Some VIEW is gone in between ({} vs {} processors, on {} parallel stream)", - out.getProcessors().size(), out_chains.back().getProcessors().size(), i); - } - } out_chains.emplace_back(std::move(out)); } } diff --git a/src/Processors/Transforms/buildPushingToViewsChain.cpp b/src/Processors/Transforms/buildPushingToViewsChain.cpp index a8890f5bccb..d71d6901cee 100644 --- a/src/Processors/Transforms/buildPushingToViewsChain.cpp +++ b/src/Processors/Transforms/buildPushingToViewsChain.cpp @@ -410,6 +410,14 @@ Chain buildPushingToViewsChain( if (result_chain.empty()) result_chain.addSink(std::make_shared(storage_header)); + if (result_chain.getOutputHeader().columns() != 0) + { + /// Convert result header to empty block. + auto dag = ActionsDAG::makeConvertingActions(result_chain.getOutputHeader().getColumnsWithTypeAndName(), {}, ActionsDAG::MatchColumnsMode::Name); + auto actions = std::make_shared(std::move(dag)); + result_chain.addSink(std::make_shared(result_chain.getOutputHeader(), std::move(actions))); + } + return result_chain; } diff --git a/tests/queries/0_stateless/02380_insert_mv_race.sh b/tests/queries/0_stateless/02380_insert_mv_race.sh index ba002832715..d66e7b62d89 100755 --- a/tests/queries/0_stateless/02380_insert_mv_race.sh +++ b/tests/queries/0_stateless/02380_insert_mv_race.sh @@ -9,58 +9,18 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -function bootstrap() -{ - $CLICKHOUSE_CLIENT -nm -q " - DROP TABLE IF EXISTS null; - CREATE TABLE null (key Int) ENGINE = Null; +$CLICKHOUSE_CLIENT -nm -q " + DROP TABLE IF EXISTS null; + CREATE TABLE null (key Int) ENGINE = Null; + DROP TABLE IF EXISTS mv; + CREATE MATERIALIZED VIEW mv ENGINE = Null() AS SELECT * FROM null; +" - DROP TABLE IF EXISTS mv; - CREATE MATERIALIZED VIEW mv ENGINE = Null() AS SELECT * FROM null; - " -} - -function insert_thread() -{ - local opts=( - --max_insert_threads 100 - --max_threads 100 - ) - local patterns=( - -e UNKNOWN_TABLE - -e TABLE_IS_DROPPED - ) - - while :; do - $CLICKHOUSE_CLIENT "${opts[@]}" -q "INSERT INTO null SELECT * FROM numbers_mt(1e6)" |& { - grep -F "DB::Exception: " | grep -v -F "${patterns[@]}" - } - done -} -export -f insert_thread - -function drop_thread() -{ - local opts=( - --database_atomic_wait_for_drop_and_detach_synchronously 1 - ) - - while :; do - $CLICKHOUSE_CLIENT -nm "${opts[@]}" -q "DETACH TABLE mv" - sleep 0.01 - $CLICKHOUSE_CLIENT -nm "${opts[@]}" -q "ATTACH TABLE mv" - done -} -export -f drop_thread - -function main() -{ - local test_timeout=1m - - bootstrap - timeout "$test_timeout" bash -c insert_thread & - timeout "$test_timeout" bash -c drop_thread & - - wait -} -main "$@" +$CLICKHOUSE_CLIENT -q "INSERT INTO null SELECT * FROM numbers_mt(1000) settings max_threads=1000, max_insert_threads=1000, max_block_size=1" |& { + # To avoid handling stacktrace here, get only first line (-m1) + # this should be OK, since you cannot have multiple exceptions from the client anyway. + grep -m1 -F 'DB::Exception:' | grep -F -v -e 'UNKNOWN_TABLE' +} & +sleep 0.05 +$CLICKHOUSE_CLIENT -q "DETACH TABLE mv" +wait From e59f217f8de701e9529dcb4c88e98fd34ae1ad0c Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Fri, 5 Aug 2022 13:33:27 +0200 Subject: [PATCH 114/117] add test logs on connection close --- src/Server/TCPHandler.cpp | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/src/Server/TCPHandler.cpp b/src/Server/TCPHandler.cpp index 05565063893..16599c378e7 100644 --- a/src/Server/TCPHandler.cpp +++ b/src/Server/TCPHandler.cpp @@ -190,7 +190,10 @@ void TCPHandler::runImpl() /// If we need to shut down, or client disconnects. if (!tcp_server.isOpen() || server.isCancelled() || in->eof()) + { + LOG_TEST(log, "Closing connection (open: {}, cancelled: {}, eof: {})", tcp_server.isOpen(), server.isCancelled(), in->eof()); break; + } Stopwatch watch; state.reset(); @@ -406,6 +409,8 @@ void TCPHandler::runImpl() if (e.code() == ErrorCodes::UNKNOWN_PACKET_FROM_CLIENT) throw; + LOG_TEST(log, "Going to close connection due to exception: {}", e.message()); + /// If there is UNEXPECTED_PACKET_FROM_CLIENT emulate network_error /// to break the loop, but do not throw to send the exception to /// the client. @@ -435,7 +440,7 @@ void TCPHandler::runImpl() // Server should die on std logic errors in debug, like with assert() // or ErrorCodes::LOGICAL_ERROR. This helps catch these errors in // tests. -#ifndef NDEBUG +#ifdef ABORT_ON_LOGICAL_ERROR catch (const std::logic_error & e) { state.io.onException(); @@ -554,14 +559,14 @@ bool TCPHandler::readDataNext() Stopwatch watch(CLOCK_MONOTONIC_COARSE); /// Poll interval should not be greater than receive_timeout - constexpr UInt64 min_timeout_ms = 5000; // 5 ms - UInt64 timeout_ms = std::max(min_timeout_ms, std::min(poll_interval * 1000000, static_cast(receive_timeout.totalMicroseconds()))); + constexpr UInt64 min_timeout_us = 5000; // 5 ms + UInt64 timeout_us = std::max(min_timeout_us, std::min(poll_interval * 1000000, static_cast(receive_timeout.totalMicroseconds()))); bool read_ok = false; /// We are waiting for a packet from the client. Thus, every `POLL_INTERVAL` seconds check whether we need to shut down. while (true) { - if (static_cast(*in).poll(timeout_ms)) + if (static_cast(*in).poll(timeout_us)) { /// If client disconnected. if (in->eof()) From c3608de37eab41fb96ed6ceddb34b20be53c795a Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 5 Aug 2022 14:50:39 +0200 Subject: [PATCH 115/117] fix flaky test 02360_send_logs_level_colors --- tests/queries/0_stateless/02360_send_logs_level_colors.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02360_send_logs_level_colors.sh b/tests/queries/0_stateless/02360_send_logs_level_colors.sh index 4e5ce057702..0585e779815 100755 --- a/tests/queries/0_stateless/02360_send_logs_level_colors.sh +++ b/tests/queries/0_stateless/02360_send_logs_level_colors.sh @@ -13,7 +13,7 @@ function run() command=$1 expect << EOF log_user 0 -set timeout 3 +set timeout 60 match_max 100000 spawn bash -c "$command" From 340ff4f1adbbe3c2f32e3eabe82b203bc5cb5faf Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Fri, 5 Aug 2022 14:51:18 +0200 Subject: [PATCH 116/117] impl (#39908) --- ...th_external_aggregation_memory_usage.reference | 1 - ...ted_with_external_aggregation_memory_usage.sql | 15 +++++++-------- 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.reference b/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.reference index d00491fd7e5..e69de29bb2d 100644 --- a/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.reference +++ b/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.reference @@ -1 +0,0 @@ -1 diff --git a/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql b/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql index 2601355ff46..c1cb2280381 100644 --- a/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql +++ b/tests/queries/0_stateless/02354_distributed_with_external_aggregation_memory_usage.sql @@ -9,15 +9,14 @@ set max_bytes_before_external_group_by = '2G', aggregation_memory_efficient_merge_threads = 16, distributed_aggregation_memory_efficient = 1, prefer_localhost_replica = 1, - group_by_two_level_threshold = 100000; + group_by_two_level_threshold = 100000, + group_by_two_level_threshold_bytes = 1000000, + max_block_size = 65505; +-- whole aggregation state of local aggregation uncompressed is 5.8G +-- it is hard to provide an accurate estimation for memory usage, so 4G is just the actual value taken from the logs + delta select a, b, c, sum(a) as s from remote('127.0.0.{1,2}', currentDatabase(), t_2354_dist_with_external_aggr) group by a, b, c -format Null; - -system flush logs; - -select memory_usage < 4 * 1024 * 1024 * 1024 -- whole aggregation state of local aggregation uncompressed is 5.8G -from system.query_log -where event_time >= now() - interval '15 minute' and type = 'QueryFinish' and is_initial_query and query like '%t_2354_dist_with_external_aggr%group_by%' and current_database = currentDatabase(); +format Null +settings max_memory_usage = '4G'; From 1a52fa183da6a27574c095583507e09ef7bf06af Mon Sep 17 00:00:00 2001 From: Yakov Olkhovskiy Date: Sat, 30 Jul 2022 01:07:22 -0400 Subject: [PATCH 117/117] rename Simple Check to Mergeable Check, refactor processing --- tests/ci/ast_fuzzer_check.py | 2 +- tests/ci/build_report_check.py | 20 ++++---- tests/ci/ci_config.py | 8 +++ tests/ci/codebrowser_check.py | 2 +- tests/ci/commit_status_helper.py | 62 ++++++++++++++++++------ tests/ci/compatibility_check.py | 2 +- tests/ci/docker_images_check.py | 2 +- tests/ci/docker_manifests_merge.py | 2 +- tests/ci/docker_server.py | 2 +- tests/ci/docs_check.py | 2 +- tests/ci/docs_release.py | 2 +- tests/ci/fast_test_check.py | 8 +-- tests/ci/finish_check.py | 2 +- tests/ci/functional_test_check.py | 2 +- tests/ci/get_robot_token.py | 2 +- tests/ci/integration_test_check.py | 2 +- tests/ci/keeper_jepsen_check.py | 2 +- tests/ci/performance_comparison_check.py | 2 +- tests/ci/pr_info.py | 2 +- tests/ci/run_check.py | 6 +-- tests/ci/split_build_smoke_check.py | 2 +- tests/ci/stress_check.py | 2 +- tests/ci/style_check.py | 6 ++- tests/ci/unit_tests_check.py | 2 +- 24 files changed, 96 insertions(+), 50 deletions(-) diff --git a/tests/ci/ast_fuzzer_check.py b/tests/ci/ast_fuzzer_check.py index 9ccae89b403..918e27a4e11 100644 --- a/tests/ci/ast_fuzzer_check.py +++ b/tests/ci/ast_fuzzer_check.py @@ -58,7 +58,7 @@ if __name__ == "__main__": pr_info = PRInfo() - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) rerun_helper = RerunHelper(gh, pr_info, check_name) if rerun_helper.is_already_finished_by_status(): diff --git a/tests/ci/build_report_check.py b/tests/ci/build_report_check.py index 4bb7a619b9f..c9fc6776827 100644 --- a/tests/ci/build_report_check.py +++ b/tests/ci/build_report_check.py @@ -4,6 +4,7 @@ import json import logging import os import sys +import atexit from typing import Dict, List, Tuple from github import Github @@ -21,7 +22,7 @@ from get_robot_token import get_best_robot_token from pr_info import PRInfo from commit_status_helper import ( get_commit, - fail_simple_check, + update_mergeable_check, ) from ci_config import CI_CONFIG from rerun_helper import RerunHelper @@ -154,16 +155,19 @@ def main(): needs_data = json.load(file_handler) required_builds = len(needs_data) - # A report might be empty in case of `do not test` label, for example. - # We should still be able to merge such PRs. - all_skipped = needs_data is not None and all( + if needs_data is not None and all( i["result"] == "skipped" for i in needs_data.values() - ) + ): + logging.info("All builds are skipped, exiting") + sys.exit(0) logging.info("The next builds are required: %s", ", ".join(needs_data)) - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) pr_info = PRInfo() + + atexit.register(update_mergeable_check, gh, pr_info, build_check_name) + rerun_helper = RerunHelper(gh, pr_info, build_check_name) if rerun_helper.is_already_finished_by_status(): logging.info("Check is already finished according to github status, exiting") @@ -237,8 +241,6 @@ def main(): total_groups = len(build_results) logging.info("Totally got %s artifact groups", total_groups) if total_groups == 0: - if not all_skipped: - fail_simple_check(gh, pr_info, f"{build_check_name} failed") logging.error("No success builds, failing check") sys.exit(1) @@ -308,8 +310,6 @@ def main(): ) if summary_status == "error": - if not all_skipped: - fail_simple_check(gh, pr_info, f"{build_check_name} failed") sys.exit(1) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 10db5d05ad4..252f2581792 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -345,3 +345,11 @@ CI_CONFIG = { }, }, } # type: dict + +# checks required by Mergeable Check +REQUIRED_CHECKS = [ + "Fast test", + "Style Check", + "ClickHouse build check", + "ClickHouse special build check", +] diff --git a/tests/ci/codebrowser_check.py b/tests/ci/codebrowser_check.py index 6e7f98bd82a..230a778c598 100644 --- a/tests/ci/codebrowser_check.py +++ b/tests/ci/codebrowser_check.py @@ -35,7 +35,7 @@ if __name__ == "__main__": temp_path = os.getenv("TEMP_PATH", os.path.abspath(".")) - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) if not os.path.exists(temp_path): os.makedirs(temp_path) diff --git a/tests/ci/commit_status_helper.py b/tests/ci/commit_status_helper.py index 83b6203c050..8b9d28502c1 100644 --- a/tests/ci/commit_status_helper.py +++ b/tests/ci/commit_status_helper.py @@ -4,12 +4,13 @@ import csv import os import time from typing import Optional +import logging -from ci_config import CI_CONFIG +from ci_config import CI_CONFIG, REQUIRED_CHECKS from env_helper import GITHUB_REPOSITORY, GITHUB_RUN_URL from github import Github from github.Commit import Commit -from pr_info import SKIP_SIMPLE_CHECK_LABEL +from pr_info import SKIP_MERGEABLE_CHECK_LABEL RETRY = 5 @@ -82,26 +83,59 @@ def post_labels(gh, pr_info, labels_names): pull_request.add_to_labels(label) -def fail_simple_check(gh, pr_info, description): - if SKIP_SIMPLE_CHECK_LABEL in pr_info.labels: - return - commit = get_commit(gh, pr_info.sha) +def fail_mergeable_check(commit, description): commit.create_status( - context="Simple Check", + context="Mergeable Check", description=description, state="failure", target_url=GITHUB_RUN_URL, ) -def create_simple_check(gh, pr_info): - commit = get_commit(gh, pr_info.sha) - for status in commit.get_statuses(): - if "Simple Check" in status.context: - return +def reset_mergeable_check(commit, description=""): commit.create_status( - context="Simple Check", - description="Skipped", + context="Mergeable Check", + description=description, state="success", target_url=GITHUB_RUN_URL, ) + + +def update_mergeable_check(gh, pr_info, check_name): + if SKIP_MERGEABLE_CHECK_LABEL in pr_info.labels: + return + + logging.info("Update Mergeable Check by %s", check_name) + + commit = get_commit(gh, pr_info.sha) + checks = { + check.context: check.state + for check in filter( + lambda check: (check.context in REQUIRED_CHECKS), + # get_statuses() returns generator, which cannot be reversed - we need comprehension + # pylint: disable=unnecessary-comprehension + reversed([status for status in commit.get_statuses()]), + ) + } + + success = [] + fail = [] + for name, state in checks.items(): + if state == "success": + success.append(name) + else: + fail.append(name) + + if fail: + description = "failed: " + ", ".join(fail) + if success: + description += "; succeeded: " + ", ".join(success) + if len(description) > 140: + description = description[:137] + "..." + fail_mergeable_check(commit, description) + return + + description = ", ".join(success) + if len(description) > 140: + description = description[:137] + "..." + reset_mergeable_check(commit, description) diff --git a/tests/ci/compatibility_check.py b/tests/ci/compatibility_check.py index 71a959a064c..2a1b9716189 100644 --- a/tests/ci/compatibility_check.py +++ b/tests/ci/compatibility_check.py @@ -119,7 +119,7 @@ if __name__ == "__main__": pr_info = PRInfo() - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) rerun_helper = RerunHelper(gh, pr_info, CHECK_NAME) if rerun_helper.is_already_finished_by_status(): diff --git a/tests/ci/docker_images_check.py b/tests/ci/docker_images_check.py index 5742bc6c22e..8b838defa8b 100644 --- a/tests/ci/docker_images_check.py +++ b/tests/ci/docker_images_check.py @@ -477,7 +477,7 @@ def main(): if not args.reports: return - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) post_commit_status(gh, pr_info.sha, NAME, description, status, url) prepared_events = prepare_tests_results_for_clickhouse( diff --git a/tests/ci/docker_manifests_merge.py b/tests/ci/docker_manifests_merge.py index aa13bbea2fb..00ab0b9e77f 100644 --- a/tests/ci/docker_manifests_merge.py +++ b/tests/ci/docker_manifests_merge.py @@ -221,7 +221,7 @@ def main(): if len(description) >= 140: description = description[:136] + "..." - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) post_commit_status(gh, pr_info.sha, NAME, description, status, url) prepared_events = prepare_tests_results_for_clickhouse( diff --git a/tests/ci/docker_server.py b/tests/ci/docker_server.py index 710c18a56cb..09a75206442 100644 --- a/tests/ci/docker_server.py +++ b/tests/ci/docker_server.py @@ -351,7 +351,7 @@ def main(): if len(description) >= 140: description = description[:136] + "..." - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) post_commit_status(gh, pr_info.sha, NAME, description, status, url) prepared_events = prepare_tests_results_for_clickhouse( diff --git a/tests/ci/docs_check.py b/tests/ci/docs_check.py index d6131535ef8..cf4fd8da692 100644 --- a/tests/ci/docs_check.py +++ b/tests/ci/docs_check.py @@ -47,7 +47,7 @@ if __name__ == "__main__": pr_info = PRInfo(need_changed_files=True) - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) rerun_helper = RerunHelper(gh, pr_info, NAME) if rerun_helper.is_already_finished_by_status(): diff --git a/tests/ci/docs_release.py b/tests/ci/docs_release.py index 4a9686ec99f..35203486fae 100644 --- a/tests/ci/docs_release.py +++ b/tests/ci/docs_release.py @@ -39,7 +39,7 @@ if __name__ == "__main__": temp_path = TEMP_PATH repo_path = REPO_COPY - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) pr_info = PRInfo() rerun_helper = RerunHelper(gh, pr_info, NAME) if rerun_helper.is_already_finished_by_status(): diff --git a/tests/ci/fast_test_check.py b/tests/ci/fast_test_check.py index 2768eccd53a..9852175ca92 100644 --- a/tests/ci/fast_test_check.py +++ b/tests/ci/fast_test_check.py @@ -5,6 +5,7 @@ import subprocess import os import csv import sys +import atexit from github import Github @@ -16,7 +17,7 @@ from upload_result_helper import upload_results from docker_pull_helper import get_image_with_version from commit_status_helper import ( post_commit_status, - fail_simple_check, + update_mergeable_check, ) from clickhouse_helper import ( ClickHouseHelper, @@ -93,7 +94,9 @@ if __name__ == "__main__": pr_info = PRInfo() - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) + + atexit.register(update_mergeable_check, gh, pr_info, NAME) rerun_helper = RerunHelper(gh, pr_info, NAME) if rerun_helper.is_already_finished_by_status(): @@ -222,5 +225,4 @@ if __name__ == "__main__": if FORCE_TESTS_LABEL in pr_info.labels and state != "error": print(f"'{FORCE_TESTS_LABEL}' enabled, will report success") else: - fail_simple_check(gh, pr_info, f"{NAME} failed") sys.exit(1) diff --git a/tests/ci/finish_check.py b/tests/ci/finish_check.py index 0697f52abed..a0b7f14ecfb 100644 --- a/tests/ci/finish_check.py +++ b/tests/ci/finish_check.py @@ -30,7 +30,7 @@ if __name__ == "__main__": logging.basicConfig(level=logging.INFO) pr_info = PRInfo(need_orgs=True) - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) commit = get_commit(gh, pr_info.sha) url = GITHUB_RUN_URL diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index b73e6f9d708..e5a05b3e816 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -205,7 +205,7 @@ if __name__ == "__main__": flaky_check = "flaky" in check_name.lower() run_changed_tests = flaky_check or validate_bugix_check - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) pr_info = PRInfo(need_changed_files=run_changed_tests) diff --git a/tests/ci/get_robot_token.py b/tests/ci/get_robot_token.py index cb79d9ae01a..4fb8cb8f49f 100644 --- a/tests/ci/get_robot_token.py +++ b/tests/ci/get_robot_token.py @@ -15,7 +15,7 @@ def get_best_robot_token(token_prefix_env_name="github_robot_token_", total_toke for i in range(1, total_tokens + 1): token_name = token_prefix_env_name + str(i) token = get_parameter_from_ssm(token_name, True, client) - gh = Github(token) + gh = Github(token, per_page=100) rest, _ = gh.rate_limiting tokens[token] = rest diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 1c53247c072..565864d576c 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -180,7 +180,7 @@ if __name__ == "__main__": logging.info("Skipping '%s' (no pr-bugfix)", check_name) sys.exit(0) - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) rerun_helper = RerunHelper(gh, pr_info, check_name_with_group) if rerun_helper.is_already_finished_by_status(): diff --git a/tests/ci/keeper_jepsen_check.py b/tests/ci/keeper_jepsen_check.py index 3c2f72f73d1..88ccf8e8828 100644 --- a/tests/ci/keeper_jepsen_check.py +++ b/tests/ci/keeper_jepsen_check.py @@ -159,7 +159,7 @@ if __name__ == "__main__": logging.info("Not jepsen test label in labels list, skipping") sys.exit(0) - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) rerun_helper = RerunHelper(gh, pr_info, CHECK_NAME) if rerun_helper.is_already_finished_by_status(): diff --git a/tests/ci/performance_comparison_check.py b/tests/ci/performance_comparison_check.py index baf2593130a..57a52dcaa6a 100644 --- a/tests/ci/performance_comparison_check.py +++ b/tests/ci/performance_comparison_check.py @@ -80,7 +80,7 @@ if __name__ == "__main__": with open(GITHUB_EVENT_PATH, "r", encoding="utf-8") as event_file: event = json.load(event_file) - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) pr_info = PRInfo(event) commit = get_commit(gh, pr_info.sha) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 86b3081f98c..2acd0e4c811 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -15,7 +15,7 @@ from env_helper import ( ) FORCE_TESTS_LABEL = "force tests" -SKIP_SIMPLE_CHECK_LABEL = "skip simple check" +SKIP_MERGEABLE_CHECK_LABEL = "skip mergeable check" DIFF_IN_DOCUMENTATION_EXT = [ ".html", diff --git a/tests/ci/run_check.py b/tests/ci/run_check.py index 2777fe55fd7..5e6542f6e4c 100644 --- a/tests/ci/run_check.py +++ b/tests/ci/run_check.py @@ -10,7 +10,7 @@ from commit_status_helper import ( get_commit, post_labels, remove_labels, - create_simple_check, + reset_mergeable_check, ) from env_helper import GITHUB_RUN_URL, GITHUB_REPOSITORY, GITHUB_SERVER_URL from get_robot_token import get_best_robot_token @@ -196,7 +196,7 @@ if __name__ == "__main__": pr_info = PRInfo(need_orgs=True, pr_event_from_api=True, need_changed_files=True) can_run, description, labels_state = should_run_checks_for_pr(pr_info) - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) commit = get_commit(gh, pr_info.sha) description_error, category = check_pr_description(pr_info) @@ -228,7 +228,7 @@ if __name__ == "__main__": if pr_labels_to_remove: remove_labels(gh, pr_info, pr_labels_to_remove) - create_simple_check(gh, pr_info) + reset_mergeable_check(commit, "skipped") if description_error: print( diff --git a/tests/ci/split_build_smoke_check.py b/tests/ci/split_build_smoke_check.py index 9237df23a26..87a528d2761 100644 --- a/tests/ci/split_build_smoke_check.py +++ b/tests/ci/split_build_smoke_check.py @@ -76,7 +76,7 @@ if __name__ == "__main__": pr_info = PRInfo() - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) rerun_helper = RerunHelper(gh, pr_info, CHECK_NAME) if rerun_helper.is_already_finished_by_status(): diff --git a/tests/ci/stress_check.py b/tests/ci/stress_check.py index 6bd4c580687..e63f66e2e50 100644 --- a/tests/ci/stress_check.py +++ b/tests/ci/stress_check.py @@ -109,7 +109,7 @@ if __name__ == "__main__": pr_info = PRInfo() - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) rerun_helper = RerunHelper(gh, pr_info, check_name) if rerun_helper.is_already_finished_by_status(): diff --git a/tests/ci/style_check.py b/tests/ci/style_check.py index 7ba0dc2a297..66837ccb84e 100644 --- a/tests/ci/style_check.py +++ b/tests/ci/style_check.py @@ -5,6 +5,7 @@ import logging import os import subprocess import sys +import atexit from clickhouse_helper import ( @@ -12,7 +13,7 @@ from clickhouse_helper import ( mark_flaky_tests, prepare_tests_results_for_clickhouse, ) -from commit_status_helper import fail_simple_check, post_commit_status +from commit_status_helper import post_commit_status, update_mergeable_check from docker_pull_helper import get_image_with_version from env_helper import GITHUB_WORKSPACE, RUNNER_TEMP from get_robot_token import get_best_robot_token @@ -150,6 +151,8 @@ if __name__ == "__main__": gh = GitHub(get_best_robot_token()) + atexit.register(update_mergeable_check, gh, pr_info, NAME) + rerun_helper = RerunHelper(gh, pr_info, NAME) if rerun_helper.is_already_finished_by_status(): logging.info("Check is already finished according to github status, exiting") @@ -202,5 +205,4 @@ if __name__ == "__main__": ch_helper.insert_events_into(db="default", table="checks", events=prepared_events) if state in ["error", "failure"]: - fail_simple_check(gh, pr_info, f"{NAME} failed") sys.exit(1) diff --git a/tests/ci/unit_tests_check.py b/tests/ci/unit_tests_check.py index c2329fab955..4441709cb7b 100644 --- a/tests/ci/unit_tests_check.py +++ b/tests/ci/unit_tests_check.py @@ -114,7 +114,7 @@ if __name__ == "__main__": pr_info = PRInfo() - gh = Github(get_best_robot_token()) + gh = Github(get_best_robot_token(), per_page=100) rerun_helper = RerunHelper(gh, pr_info, check_name) if rerun_helper.is_already_finished_by_status():