From 93b661ad5a39bd94e1d85c4b971ee32111bfde7b Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 3 Mar 2021 16:36:20 +0800 Subject: [PATCH 01/60] partition id pruning --- src/Functions/partitionID.cpp | 75 +++++++++++++++++ .../registerFunctionsMiscellaneous.cpp | 2 + src/Functions/ya.make | 1 + src/Interpreters/ExpressionAnalyzer.cpp | 6 +- src/Interpreters/ExpressionAnalyzer.h | 26 +++--- src/Interpreters/TreeRewriter.cpp | 5 +- src/Storages/MergeTree/MergeTreeData.cpp | 33 ++++++++ src/Storages/MergeTree/MergeTreeData.h | 3 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 59 +++++++------ .../MergeTree/MergeTreeDataSelectExecutor.h | 6 ++ src/Storages/StorageMergeTree.cpp | 14 +--- src/Storages/StorageReplicatedMergeTree.cpp | 14 +--- src/Storages/VirtualColumnUtils.cpp | 82 +++++++++++++++---- src/Storages/VirtualColumnUtils.h | 7 +- .../01748_partition_id_pruning.reference | 7 ++ .../01748_partition_id_pruning.sql | 19 +++++ 16 files changed, 278 insertions(+), 81 deletions(-) create mode 100644 src/Functions/partitionID.cpp create mode 100644 tests/queries/0_stateless/01748_partition_id_pruning.reference create mode 100644 tests/queries/0_stateless/01748_partition_id_pruning.sql diff --git a/src/Functions/partitionID.cpp b/src/Functions/partitionID.cpp new file mode 100644 index 00000000000..9f5a3d09b86 --- /dev/null +++ b/src/Functions/partitionID.cpp @@ -0,0 +1,75 @@ +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + + +/** partitionID(x, y, ...) is a function that computes partition ids of arguments. + */ +class FunctionPartitionID : public IFunction +{ +public: + static constexpr auto name = "partitionID"; + + static FunctionPtr create(const Context &) { return std::make_shared(); } + + String getName() const override { return name; } + + bool isVariadic() const override { return true; } + + size_t getNumberOfArguments() const override { return 0; } + + bool isInjective(const ColumnsWithTypeAndName & /*sample_columns*/) const override { return true; } + + bool useDefaultImplementationForNulls() const override { return true; } + bool useDefaultImplementationForConstants() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.empty()) + throw Exception("Function " + getName() + " requires at least one argument.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + return std::make_shared(); + } + + virtual ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override + { + size_t size = arguments.size(); + Columns columns; + Block sample_block; + for (const auto & argument : arguments) + { + sample_block.insert(argument); + columns.push_back(argument.column); + } + + auto result_column = ColumnString::create(); + for (size_t j = 0; j < input_rows_count; ++j) + { + Row row(size); + for (size_t i = 0; i < size; ++i) + columns[i]->get(j, row[i]); + MergeTreePartition partition(std::move(row)); + result_column->insert(partition.getID(sample_block)); + } + return result_column; + } +}; + +void registerFunctionPartitionID(FunctionFactory & factory) +{ + factory.registerFunction(); +} + +} diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 592f0d6774d..3557b0fb865 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -70,6 +70,7 @@ void registerFunctionTcpPort(FunctionFactory &); void registerFunctionByteSize(FunctionFactory &); void registerFunctionFile(FunctionFactory & factory); void registerFunctionConnectionID(FunctionFactory & factory); +void registerFunctionPartitionID(FunctionFactory & factory); #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -140,6 +141,7 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionByteSize(factory); registerFunctionFile(factory); registerFunctionConnectionID(factory); + registerFunctionPartitionID(factory); #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/src/Functions/ya.make b/src/Functions/ya.make index 7a4deae4d04..01857423dc1 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -373,6 +373,7 @@ SRCS( now.cpp now64.cpp nullIf.cpp + partitionID.cpp pi.cpp plus.cpp pointInEllipses.cpp diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 2e2c3354d4c..92388a46872 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -305,7 +305,7 @@ void ExpressionAnalyzer::initGlobalSubqueriesAndExternalTables(bool do_global) } -void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name) +void ExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options) { auto set_key = PreparedSetKey::forSubquery(*subquery_or_table_name); @@ -341,7 +341,7 @@ void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr prepared_sets[set_key] = std::move(set); } -SetPtr SelectQueryExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name) +SetPtr ExpressionAnalyzer::isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name) { const auto * table = subquery_or_table_name->as(); if (!table) @@ -387,7 +387,7 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) if (arg->as() || arg->as()) { if (settings.use_index_for_in_with_subqueries) - tryMakeSetForIndexFromSubquery(arg); + tryMakeSetForIndexFromSubquery(arg, query_options); } else { diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index dc7afb183fc..441274200bd 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -127,6 +127,19 @@ public: void makeWindowDescriptions(ActionsDAGPtr actions); + /** + * Create Set from a subquery or a table expression in the query. The created set is suitable for using the index. + * The set will not be created if its size hits the limit. + */ + void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name, const SelectQueryOptions & query_options = {}); + + /** + * Checks if subquery is not a plain StorageSet. + * Because while making set we will read data from StorageSet which is not allowed. + * Returns valid SetPtr from StorageSet if the latter is used after IN or nullptr otherwise. + */ + SetPtr isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name); + protected: ExpressionAnalyzer( const ASTPtr & query_, @@ -297,19 +310,6 @@ private: NameSet required_result_columns; SelectQueryOptions query_options; - /** - * Create Set from a subquery or a table expression in the query. The created set is suitable for using the index. - * The set will not be created if its size hits the limit. - */ - void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name); - - /** - * Checks if subquery is not a plain StorageSet. - * Because while making set we will read data from StorageSet which is not allowed. - * Returns valid SetPtr from StorageSet if the latter is used after IN or nullptr otherwise. - */ - SetPtr isPlainStorageSetInSubquery(const ASTPtr & subquery_or_table_name); - /// Create Set-s that we make from IN section to use index on them. void makeSetsForIndex(const ASTPtr & node); diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index bcfdf6869c3..5c9ba8fae57 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -656,7 +656,10 @@ void TreeRewriterResult::collectUsedColumns(const ASTPtr & query, bool is_select const auto & partition_desc = metadata_snapshot->getPartitionKey(); if (partition_desc.expression) { - const auto & partition_source_columns = partition_desc.expression->getRequiredColumns(); + auto partition_source_columns = partition_desc.expression->getRequiredColumns(); + partition_source_columns.push_back("_part"); + partition_source_columns.push_back("_partition_id"); + partition_source_columns.push_back("_part_uuid"); optimize_trivial_count = true; for (const auto & required_column : required) { diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 0c22d5fbc0f..49e129e5fac 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -42,6 +42,7 @@ #include #include #include +#include #include #include #include @@ -675,6 +676,38 @@ void MergeTreeData::MergingParams::check(const StorageInMemoryMetadata & metadat } +std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( + const SelectQueryInfo & query_info, const Context & context, const DataPartsVector & parts) const +{ + auto metadata_snapshot = getInMemoryMetadataPtr(); + Block part_block = MergeTreeDataSelectExecutor::getSampleBlockWithVirtualPartColumns(); + ASTPtr expression_ast; + bool valid = VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, part_block, expression_ast); + PartitionPruner partition_pruner(metadata_snapshot->getPartitionKey(), query_info, context, true /* strict */); + if (partition_pruner.isUseless() && !valid) + return {}; + + std::unordered_set part_values; + if (valid) + { + MergeTreeDataSelectExecutor::fillBlockWithVirtualPartColumns(parts, part_block); + VirtualColumnUtils::filterBlockWithQuery(query_info.query, part_block, context, expression_ast); + part_values = VirtualColumnUtils::extractSingleValueFromBlock(part_block, "_part"); + if (part_values.empty()) + return 0; + } + // At this point, empty `part_values` means all parts. + + size_t res = 0; + for (const auto & part : parts) + { + if ((part_values.empty() || part_values.find(part->name) != part_values.end()) && !partition_pruner.canBePruned(part)) + res += part->rows_count; + } + return res; +} + + String MergeTreeData::MergingParams::getModeName() const { switch (mode) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 15059ab47e5..77df4ac5026 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -825,6 +825,9 @@ protected: return {begin, end}; } + std::optional totalRowsByPartitionPredicateImpl( + const SelectQueryInfo & query_info, const Context & context, const DataPartsVector & parts) const; + static decltype(auto) getStateModifier(DataPartState state) { return [state] (const DataPartPtr & part) { part->setState(state); }; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index b1f3f524beb..917c0bcec81 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -71,28 +71,30 @@ MergeTreeDataSelectExecutor::MergeTreeDataSelectExecutor(const MergeTreeData & d } -/// Construct a block consisting only of possible values of virtual columns -static Block getBlockWithVirtualPartColumns(const MergeTreeData::DataPartsVector & parts, bool with_uuid) +Block MergeTreeDataSelectExecutor::getSampleBlockWithVirtualPartColumns() { - auto part_column = ColumnString::create(); - auto part_uuid_column = ColumnUUID::create(); + return Block(std::initializer_list{ + ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "_part"), + ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "_partition_id"), + ColumnWithTypeAndName(ColumnUUID::create(), std::make_shared(), "_part_uuid")}); +} + +void MergeTreeDataSelectExecutor::fillBlockWithVirtualPartColumns(const MergeTreeData::DataPartsVector & parts, Block & block) +{ + MutableColumns columns = block.mutateColumns(); + + auto & part_column = columns[0]; + auto & partition_id_column = columns[1]; + auto & part_uuid_column = columns[2]; for (const auto & part : parts) { part_column->insert(part->name); - if (with_uuid) - part_uuid_column->insert(part->uuid); + partition_id_column->insert(part->info.partition_id); + part_uuid_column->insert(part->uuid); } - if (with_uuid) - { - return Block(std::initializer_list{ - ColumnWithTypeAndName(std::move(part_column), std::make_shared(), "_part"), - ColumnWithTypeAndName(std::move(part_uuid_column), std::make_shared(), "_part_uuid"), - }); - } - - return Block{ColumnWithTypeAndName(std::move(part_column), std::make_shared(), "_part")}; + block.setColumns(std::move(columns)); } @@ -176,8 +178,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( Names real_column_names; size_t total_parts = parts.size(); - bool part_column_queried = false; - bool part_uuid_column_queried = false; bool sample_factor_column_queried = false; Float64 used_sample_factor = 1; @@ -186,7 +186,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( { if (name == "_part") { - part_column_queried = true; virt_column_names.push_back(name); } else if (name == "_part_index") @@ -199,7 +198,6 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( } else if (name == "_part_uuid") { - part_uuid_column_queried = true; virt_column_names.push_back(name); } else if (name == "_sample_factor") @@ -219,12 +217,21 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( if (real_column_names.empty()) real_column_names.push_back(ExpressionActions::getSmallestColumn(available_real_columns)); - /// If `_part` or `_part_uuid` virtual columns are requested, we try to filter out data by them. - Block virtual_columns_block = getBlockWithVirtualPartColumns(parts, part_uuid_column_queried); - if (part_column_queried || part_uuid_column_queried) - VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, context); + std::unordered_set part_values; + ASTPtr expression_ast; + Block virtual_columns_block = getSampleBlockWithVirtualPartColumns(); + VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, virtual_columns_block, expression_ast); - auto part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); + // If there is still something left, fill the virtual block and do the filtering. + if (expression_ast) + { + fillBlockWithVirtualPartColumns(parts, virtual_columns_block); + VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, context, expression_ast); + part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); + if (part_values.empty()) + return {}; + } + // At this point, empty `part_values` means all parts. metadata_snapshot->check(real_column_names, data.getVirtuals(), data.getStorageID()); @@ -1899,7 +1906,7 @@ void MergeTreeDataSelectExecutor::selectPartsToRead( for (const auto & part : prev_parts) { - if (part_values.find(part->name) == part_values.end()) + if (!part_values.empty() && part_values.find(part->name) == part_values.end()) continue; if (part->isEmpty()) @@ -1950,7 +1957,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter( for (const auto & part : prev_parts) { - if (part_values.find(part->name) == part_values.end()) + if (!part_values.empty() && part_values.find(part->name) == part_values.end()) continue; if (part->isEmpty()) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h index 634719639ad..0702605a539 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.h @@ -44,6 +44,12 @@ public: unsigned num_streams, const PartitionIdToMaxBlock * max_block_numbers_to_read = nullptr) const; + /// Construct a sample block consisting only of possible virtual columns for part pruning. + static Block getSampleBlockWithVirtualPartColumns(); + + /// Fill in values of possible virtual columns for part pruning. + static void fillBlockWithVirtualPartColumns(const MergeTreeData::DataPartsVector & parts, Block & block); + private: const MergeTreeData & data; diff --git a/src/Storages/StorageMergeTree.cpp b/src/Storages/StorageMergeTree.cpp index c8f44c78e6e..ce81849708b 100644 --- a/src/Storages/StorageMergeTree.cpp +++ b/src/Storages/StorageMergeTree.cpp @@ -208,18 +208,8 @@ std::optional StorageMergeTree::totalRows(const Settings &) const std::optional StorageMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, const Context & context) const { - auto metadata_snapshot = getInMemoryMetadataPtr(); - PartitionPruner partition_pruner(metadata_snapshot->getPartitionKey(), query_info, context, true /* strict */); - if (partition_pruner.isUseless()) - return {}; - size_t res = 0; - auto lock = lockParts(); - for (const auto & part : getDataPartsStateRange(DataPartState::Committed)) - { - if (!partition_pruner.canBePruned(part)) - res += part->rows_count; - } - return res; + auto parts = getDataPartsVector({DataPartState::Committed}); + return totalRowsByPartitionPredicateImpl(query_info, context, parts); } std::optional StorageMergeTree::totalBytes(const Settings &) const diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 68f3b6d80d1..9f18d9ead21 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -3890,17 +3890,9 @@ std::optional StorageReplicatedMergeTree::totalRows(const Settings & set std::optional StorageReplicatedMergeTree::totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, const Context & context) const { - auto metadata_snapshot = getInMemoryMetadataPtr(); - PartitionPruner partition_pruner(metadata_snapshot->getPartitionKey(), query_info, context, true /* strict */); - if (partition_pruner.isUseless()) - return {}; - size_t res = 0; - foreachCommittedParts([&](auto & part) - { - if (!partition_pruner.canBePruned(part)) - res += part->rows_count; - }, context.getSettingsRef().select_sequential_consistency); - return res; + DataPartsVector parts; + foreachCommittedParts([&](auto & part) { parts.push_back(part); }, context.getSettingsRef().select_sequential_consistency); + return totalRowsByPartitionPredicateImpl(query_info, context, parts); } std::optional StorageReplicatedMergeTree::totalBytes(const Settings & settings) const diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 6b99dc25e37..ba35bde6d51 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -5,12 +5,14 @@ #include #include #include +#include #include #include #include #include #include +#include #include #include @@ -30,29 +32,52 @@ namespace /// Verifying that the function depends only on the specified columns bool isValidFunction(const ASTPtr & expression, const NameSet & columns) { - for (const auto & child : expression->children) - if (!isValidFunction(child, columns)) - return false; - - if (auto opt_name = IdentifierSemantic::getColumnName(expression)) - return columns.count(*opt_name); + const auto * function = expression->as(); + if (function) + { + if (functionIsInOrGlobalInOperator(function->name)) + { + // Second argument of IN can be a scalar subquery + if (!isValidFunction(function->arguments->children[0], columns)) + return false; + } + else + { + if (function->arguments) + { + for (const auto & child : function->arguments->children) + if (!isValidFunction(child, columns)) + return false; + } + } + } + else + { + if (auto opt_name = IdentifierSemantic::getColumnName(expression)) + return columns.count(*opt_name); + } return true; } /// Extract all subfunctions of the main conjunction, but depending only on the specified columns -void extractFunctions(const ASTPtr & expression, const NameSet & columns, std::vector & result) +bool extractFunctions(const ASTPtr & expression, const NameSet & columns, std::vector & result) { const auto * function = expression->as(); - if (function && function->name == "and") + if (function && (function->name == "and" || function->name == "indexHint")) { + bool ret = true; for (const auto & child : function->arguments->children) - extractFunctions(child, columns, result); + ret &= extractFunctions(child, columns, result); + return ret; } else if (isValidFunction(expression, columns)) { result.push_back(expression->clone()); + return true; } + else + return false; } /// Construct a conjunction from given functions @@ -65,6 +90,25 @@ ASTPtr buildWhereExpression(const ASTs & functions) return makeASTFunction("and", functions); } +void buildSets(const ASTPtr & expression, ExpressionAnalyzer & analyzer) +{ + const auto * func = expression->as(); + if (func && functionIsInOrGlobalInOperator(func->name)) + { + const IAST & args = *func->arguments; + const ASTPtr & arg = args.children.at(1); + if (arg->as() || arg->as()) + { + analyzer.tryMakeSetForIndexFromSubquery(arg); + } + } + else + { + for (const auto & child : expression->children) + buildSets(child, analyzer); + } +} + } namespace VirtualColumnUtils @@ -96,11 +140,12 @@ void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & va } } -void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context) +bool prepareFilterBlockWithQuery(const ASTPtr & query, const Block & block, ASTPtr & expression_ast) { + bool ret = true; const auto & select = query->as(); if (!select.where() && !select.prewhere()) - return; + return ret; NameSet columns; for (const auto & it : block.getNamesAndTypesList()) @@ -109,17 +154,26 @@ void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & c /// We will create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns. std::vector functions; if (select.where()) - extractFunctions(select.where(), columns, functions); + ret &= extractFunctions(select.where(), columns, functions); if (select.prewhere()) - extractFunctions(select.prewhere(), columns, functions); + ret &= extractFunctions(select.prewhere(), columns, functions); + + expression_ast = buildWhereExpression(functions); + return ret; +} + +void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context, ASTPtr expression_ast) +{ + if (!expression_ast) + prepareFilterBlockWithQuery(query, block, expression_ast); - ASTPtr expression_ast = buildWhereExpression(functions); if (!expression_ast) return; /// Let's analyze and calculate the expression. auto syntax_result = TreeRewriter(context).analyze(expression_ast, block.getNamesAndTypesList()); ExpressionAnalyzer analyzer(expression_ast, syntax_result, context); + buildSets(expression_ast, analyzer); ExpressionActionsPtr actions = analyzer.getActions(false); Block block_with_filter = block; diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 445a996ab87..326738f4259 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -24,9 +24,14 @@ namespace VirtualColumnUtils /// - `WITH toUInt16(9000) as _port`. void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & value, const String & func = ""); +/// Prepare `expression_ast` to filter block. Returns true if `expression_ast` is not trimmed, that is, +/// the sample block provides all needed columns, else return false. +bool prepareFilterBlockWithQuery(const ASTPtr & query, const Block & sample_block, ASTPtr & expression_ast); + /// Leave in the block only the rows that fit under the WHERE clause and the PREWHERE clause of the query. /// Only elements of the outer conjunction are considered, depending only on the columns present in the block. -void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context); +/// If `expression_ast` is passed, use it to filter block. +void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context, ASTPtr expression_ast = {}); /// Extract from the input stream a set of `name` column values template diff --git a/tests/queries/0_stateless/01748_partition_id_pruning.reference b/tests/queries/0_stateless/01748_partition_id_pruning.reference new file mode 100644 index 00000000000..192e33c03c9 --- /dev/null +++ b/tests/queries/0_stateless/01748_partition_id_pruning.reference @@ -0,0 +1,7 @@ +1 1 +1 2 +1 3 +1 1 +1 2 +1 3 +3 diff --git a/tests/queries/0_stateless/01748_partition_id_pruning.sql b/tests/queries/0_stateless/01748_partition_id_pruning.sql new file mode 100644 index 00000000000..cca366c5168 --- /dev/null +++ b/tests/queries/0_stateless/01748_partition_id_pruning.sql @@ -0,0 +1,19 @@ +drop table if exists x; + +create table x (i int, j int) engine MergeTree partition by i order by j settings index_granularity = 1; + +insert into x values (1, 1), (1, 2), (1, 3), (2, 4), (2, 5), (2, 6); + +set max_rows_to_read = 3; + +select * from x where _partition_id = partitionID(1); + +set max_rows_to_read = 4; -- one row for subquery + +select * from x where _partition_id in (select partitionID(number + 1) from numbers(1)); + +-- trivial count optimization test +set max_rows_to_read = 1; -- one row for subquery +select count() from x where _partition_id in (select partitionID(number + 1) from numbers(1)); + +drop table x; From 2f8f4e96973a48b1900e08cee6b0094381713542 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 3 Mar 2021 22:26:55 +0800 Subject: [PATCH 02/60] Fix tests --- src/Storages/VirtualColumnUtils.cpp | 4 ++++ tests/queries/0_stateless/01651_bugs_from_15889.reference | 1 + tests/queries/0_stateless/01651_bugs_from_15889.sql | 2 +- 3 files changed, 6 insertions(+), 1 deletion(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index ba35bde6d51..196455c70f0 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -41,6 +41,10 @@ bool isValidFunction(const ASTPtr & expression, const NameSet & columns) if (!isValidFunction(function->arguments->children[0], columns)) return false; } + else if (function->name == "ignore") + { + return false; + } else { if (function->arguments) diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.reference b/tests/queries/0_stateless/01651_bugs_from_15889.reference index 77ac542d4fb..28271a697e2 100644 --- a/tests/queries/0_stateless/01651_bugs_from_15889.reference +++ b/tests/queries/0_stateless/01651_bugs_from_15889.reference @@ -1,2 +1,3 @@ 0 +0 diff --git a/tests/queries/0_stateless/01651_bugs_from_15889.sql b/tests/queries/0_stateless/01651_bugs_from_15889.sql index 1fbf669a1b8..97da4d78ab6 100644 --- a/tests/queries/0_stateless/01651_bugs_from_15889.sql +++ b/tests/queries/0_stateless/01651_bugs_from_15889.sql @@ -8,7 +8,7 @@ INSERT INTO xp SELECT '2020-01-01', number, '' FROM numbers(100000); CREATE TABLE xp_d AS xp ENGINE = Distributed(test_shard_localhost, currentDatabase(), xp); -SELECT count(7 = (SELECT number FROM numbers(0) ORDER BY number ASC NULLS FIRST LIMIT 7)) FROM xp_d PREWHERE toYYYYMM(A) GLOBAL IN (SELECT NULL = (SELECT number FROM numbers(1) ORDER BY number DESC NULLS LAST LIMIT 1), toYYYYMM(min(A)) FROM xp_d) WHERE B > NULL; -- { serverError 20 } +SELECT count(7 = (SELECT number FROM numbers(0) ORDER BY number ASC NULLS FIRST LIMIT 7)) FROM xp_d PREWHERE toYYYYMM(A) GLOBAL IN (SELECT NULL = (SELECT number FROM numbers(1) ORDER BY number DESC NULLS LAST LIMIT 1), toYYYYMM(min(A)) FROM xp_d) WHERE B > NULL; -- B > NULL is evaluated to 0 and this works SELECT count() FROM xp_d WHERE A GLOBAL IN (SELECT NULL); -- { serverError 53 } From 4c9b0f666ca165d51dd7c9f395b5fc1da8fb9485 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 3 Mar 2021 23:32:37 +0800 Subject: [PATCH 03/60] Fix error --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 917c0bcec81..0d4c4c4aa9a 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -229,7 +229,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, context, expression_ast); part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); if (part_values.empty()) - return {}; + return std::make_unique(); } // At this point, empty `part_values` means all parts. @@ -380,7 +380,7 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( { LOG_DEBUG(log, "Will use no data on this replica because parallel replicas processing has been requested" " (the setting 'max_parallel_replicas') but the table does not support sampling and this replica is not the first."); - return {}; + return std::make_unique(); } bool use_sampling = relative_sample_size > 0 || (settings.parallel_replicas_count > 1 && data.supportsSampling()); From 9205fad8c762644d374936dbba39a3730118dcea Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 4 Mar 2021 13:59:57 +0800 Subject: [PATCH 04/60] Better --- .../ExecuteScalarSubqueriesVisitor.cpp | 2 +- src/Interpreters/ExpressionAnalyzer.cpp | 4 +- src/Interpreters/ExpressionAnalyzer.h | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 15 +++-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 18 +++--- src/Storages/StorageMerge.cpp | 9 +-- src/Storages/StorageMerge.h | 2 +- src/Storages/System/StorageSystemColumns.cpp | 4 +- .../System/StorageSystemDistributionQueue.cpp | 2 +- .../System/StorageSystemMutations.cpp | 2 +- .../System/StorageSystemPartsBase.cpp | 4 +- src/Storages/System/StorageSystemReplicas.cpp | 2 +- .../System/StorageSystemReplicationQueue.cpp | 2 +- src/Storages/System/StorageSystemTables.cpp | 6 +- src/Storages/VirtualColumnUtils.cpp | 59 +++++++------------ src/Storages/VirtualColumnUtils.h | 7 ++- ...read_distribution_and_max_rows_to_read.sql | 5 +- 17 files changed, 68 insertions(+), 77 deletions(-) diff --git a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp index 7ee7bb1f301..94b3a5218b4 100644 --- a/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp +++ b/src/Interpreters/ExecuteScalarSubqueriesVisitor.cpp @@ -201,7 +201,7 @@ void ExecuteScalarSubqueriesMatcher::visit(const ASTSubquery & subquery, ASTPtr void ExecuteScalarSubqueriesMatcher::visit(const ASTFunction & func, ASTPtr & ast, Data & data) { /// Don't descend into subqueries in arguments of IN operator. - /// But if an argument is not subquery, than deeper may be scalar subqueries and we need to descend in them. + /// But if an argument is not subquery, then deeper may be scalar subqueries and we need to descend in them. std::vector out; if (checkFunctionIsInOrGlobalInOperator(func)) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 92388a46872..50dd6ede2a1 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1324,9 +1324,9 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool proje } -ExpressionActionsPtr ExpressionAnalyzer::getConstActions() +ExpressionActionsPtr ExpressionAnalyzer::getConstActions(const NamesAndTypesList & constant_inputs) { - auto actions = std::make_shared(NamesAndTypesList()); + auto actions = std::make_shared(constant_inputs); getRootActions(query, true, actions, true); return std::make_shared(actions); diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index 441274200bd..c44507cc4e2 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -110,7 +110,7 @@ public: /// Actions that can be performed on an empty block: adding constants and applying functions that depend only on constants. /// Does not execute subqueries. - ExpressionActionsPtr getConstActions(); + ExpressionActionsPtr getConstActions(const NamesAndTypesList & constant_inputs = {}); /** Sets that require a subquery to be create. * Only the sets needed to perform actions returned from already executed `append*` or `getActions`. diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 49e129e5fac..945fb5639c1 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -680,19 +680,22 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( const SelectQueryInfo & query_info, const Context & context, const DataPartsVector & parts) const { auto metadata_snapshot = getInMemoryMetadataPtr(); - Block part_block = MergeTreeDataSelectExecutor::getSampleBlockWithVirtualPartColumns(); ASTPtr expression_ast; - bool valid = VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, part_block, expression_ast); + Block virtual_columns_block = MergeTreeDataSelectExecutor::getSampleBlockWithVirtualPartColumns(); + + // Generate valid expressions for filtering + bool valid = VirtualColumnUtils::prepareFilterBlockWithQuery(query_info, context, virtual_columns_block, expression_ast); + PartitionPruner partition_pruner(metadata_snapshot->getPartitionKey(), query_info, context, true /* strict */); if (partition_pruner.isUseless() && !valid) return {}; std::unordered_set part_values; - if (valid) + if (valid && expression_ast) { - MergeTreeDataSelectExecutor::fillBlockWithVirtualPartColumns(parts, part_block); - VirtualColumnUtils::filterBlockWithQuery(query_info.query, part_block, context, expression_ast); - part_values = VirtualColumnUtils::extractSingleValueFromBlock(part_block, "_part"); + MergeTreeDataSelectExecutor::fillBlockWithVirtualPartColumns(parts, virtual_columns_block); + VirtualColumnUtils::filterBlockWithQuery(query_info, virtual_columns_block, context, expression_ast); + part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); if (part_values.empty()) return 0; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 0d4c4c4aa9a..59bf32fc14a 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -39,6 +39,7 @@ #include #include #include +#include namespace ProfileEvents { @@ -74,14 +75,15 @@ MergeTreeDataSelectExecutor::MergeTreeDataSelectExecutor(const MergeTreeData & d Block MergeTreeDataSelectExecutor::getSampleBlockWithVirtualPartColumns() { return Block(std::initializer_list{ - ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "_part"), - ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "_partition_id"), - ColumnWithTypeAndName(ColumnUUID::create(), std::make_shared(), "_part_uuid")}); + ColumnWithTypeAndName(DataTypeString().createColumnConstWithDefaultValue(1), std::make_shared(), "_part"), + ColumnWithTypeAndName(DataTypeString().createColumnConstWithDefaultValue(1), std::make_shared(), "_partition_id"), + ColumnWithTypeAndName(DataTypeUUID().createColumnConstWithDefaultValue(1), std::make_shared(), "_part_uuid")}); } void MergeTreeDataSelectExecutor::fillBlockWithVirtualPartColumns(const MergeTreeData::DataPartsVector & parts, Block & block) { - MutableColumns columns = block.mutateColumns(); + materializeBlockInplace(block); + MutableColumns columns = block.cloneEmptyColumns(); auto & part_column = columns[0]; auto & partition_id_column = columns[1]; @@ -219,14 +221,16 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( std::unordered_set part_values; ASTPtr expression_ast; - Block virtual_columns_block = getSampleBlockWithVirtualPartColumns(); - VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, virtual_columns_block, expression_ast); + auto virtual_columns_block = getSampleBlockWithVirtualPartColumns(); + + // Generate valid expressions for filtering + VirtualColumnUtils::prepareFilterBlockWithQuery(query_info, context, virtual_columns_block, expression_ast); // If there is still something left, fill the virtual block and do the filtering. if (expression_ast) { fillBlockWithVirtualPartColumns(parts, virtual_columns_block); - VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, context, expression_ast); + VirtualColumnUtils::filterBlockWithQuery(query_info, virtual_columns_block, context, expression_ast); part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); if (part_values.empty()) return std::make_unique(); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 46be91ba258..54f87d4e459 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -219,8 +219,8 @@ Pipe StorageMerge::read( /** 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( - query_info.query, has_table_virtual_column, context.getCurrentQueryId(), context.getSettingsRef()); + StorageListWithLocks selected_tables + = getSelectedTables(query_info, has_table_virtual_column, context.getCurrentQueryId(), context.getSettingsRef()); if (selected_tables.empty()) /// FIXME: do we support sampling in this case? @@ -407,8 +407,9 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables(const String StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( - const ASTPtr & query, bool has_virtual_column, const String & query_id, const Settings & settings) const + const SelectQueryInfo & query_info, bool has_virtual_column, const String & query_id, const Settings & settings) const { + const ASTPtr & query = query_info.query; StorageListWithLocks selected_tables; DatabaseTablesIteratorPtr iterator = getDatabaseIterator(global_context); @@ -436,7 +437,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( if (has_virtual_column) { Block virtual_columns_block = Block{ColumnWithTypeAndName(std::move(virtual_column), std::make_shared(), "_table")}; - VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, global_context); + VirtualColumnUtils::filterBlockWithQuery(query_info, virtual_columns_block, global_context); auto values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_table"); /// Remove unused tables from the list diff --git a/src/Storages/StorageMerge.h b/src/Storages/StorageMerge.h index eaffd34a379..ea8667aa186 100644 --- a/src/Storages/StorageMerge.h +++ b/src/Storages/StorageMerge.h @@ -59,7 +59,7 @@ private: StorageListWithLocks getSelectedTables(const String & query_id, const Settings & settings) const; StorageMerge::StorageListWithLocks getSelectedTables( - const ASTPtr & query, bool has_virtual_column, const String & query_id, const Settings & settings) const; + const SelectQueryInfo & query_info, bool has_virtual_column, const String & query_id, const Settings & settings) const; template StoragePtr getFirstTable(F && predicate) const; diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 6726d502071..75d90ffe1f8 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -299,7 +299,7 @@ Pipe StorageSystemColumns::read( block_to_filter.insert(ColumnWithTypeAndName(std::move(database_column_mut), std::make_shared(), "database")); /// Filter block with `database` column. - VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); + VirtualColumnUtils::filterBlockWithQuery(query_info, block_to_filter, context); if (!block_to_filter.rows()) { @@ -345,7 +345,7 @@ Pipe StorageSystemColumns::read( } /// Filter block with `database` and `table` columns. - VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); + VirtualColumnUtils::filterBlockWithQuery(query_info, block_to_filter, context); if (!block_to_filter.rows()) { diff --git a/src/Storages/System/StorageSystemDistributionQueue.cpp b/src/Storages/System/StorageSystemDistributionQueue.cpp index db649e7e1ba..58b468ab7f6 100644 --- a/src/Storages/System/StorageSystemDistributionQueue.cpp +++ b/src/Storages/System/StorageSystemDistributionQueue.cpp @@ -155,7 +155,7 @@ void StorageSystemDistributionQueue::fillData(MutableColumns & res_columns, cons { col_table_to_filter, std::make_shared(), "table" }, }; - VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); + VirtualColumnUtils::filterBlockWithQuery(query_info, filtered_block, context); if (!filtered_block.rows()) return; diff --git a/src/Storages/System/StorageSystemMutations.cpp b/src/Storages/System/StorageSystemMutations.cpp index f66f57ef5d1..50498f49c81 100644 --- a/src/Storages/System/StorageSystemMutations.cpp +++ b/src/Storages/System/StorageSystemMutations.cpp @@ -89,7 +89,7 @@ void StorageSystemMutations::fillData(MutableColumns & res_columns, const Contex { col_table, std::make_shared(), "table" }, }; - VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); + VirtualColumnUtils::filterBlockWithQuery(query_info, filtered_block, context); if (!filtered_block.rows()) return; diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index 39cc651e147..e88cf8b2865 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -93,7 +93,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const std::move(database_column_mut), std::make_shared(), "database")); /// Filter block_to_filter with column 'database'. - VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); + VirtualColumnUtils::filterBlockWithQuery(query_info, block_to_filter, context); rows = block_to_filter.rows(); /// Block contains new columns, update database_column. @@ -162,7 +162,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const if (rows) { /// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'. - VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); + VirtualColumnUtils::filterBlockWithQuery(query_info, block_to_filter, context); rows = block_to_filter.rows(); } diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index 0af67ab6986..b66e578ea1a 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -135,7 +135,7 @@ Pipe StorageSystemReplicas::read( { col_engine, std::make_shared(), "engine" }, }; - VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); + VirtualColumnUtils::filterBlockWithQuery(query_info, filtered_block, context); if (!filtered_block.rows()) return {}; diff --git a/src/Storages/System/StorageSystemReplicationQueue.cpp b/src/Storages/System/StorageSystemReplicationQueue.cpp index 9cd5e8b8ff3..549b93b5ebc 100644 --- a/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -98,7 +98,7 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const { col_table_to_filter, std::make_shared(), "table" }, }; - VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); + VirtualColumnUtils::filterBlockWithQuery(query_info, filtered_block, context); if (!filtered_block.rows()) return; diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index 132ed234323..af25d299c2e 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -62,7 +62,7 @@ StorageSystemTables::StorageSystemTables(const StorageID & table_id_) } -static ColumnPtr getFilteredDatabases(const ASTPtr & query, const Context & context) +static ColumnPtr getFilteredDatabases(const SelectQueryInfo & query_info, const Context & context) { MutableColumnPtr column = ColumnString::create(); @@ -76,7 +76,7 @@ static ColumnPtr getFilteredDatabases(const ASTPtr & query, const Context & cont } Block block { ColumnWithTypeAndName(std::move(column), std::make_shared(), "database") }; - VirtualColumnUtils::filterBlockWithQuery(query, block, context); + VirtualColumnUtils::filterBlockWithQuery(query_info, block, context); return block.getByPosition(0).column; } @@ -524,7 +524,7 @@ Pipe StorageSystemTables::read( } } - ColumnPtr filtered_databases_column = getFilteredDatabases(query_info.query, context); + ColumnPtr filtered_databases_column = getFilteredDatabases(query_info, context); return Pipe(std::make_shared( std::move(columns_mask), std::move(res_block), max_block_size, std::move(filtered_databases_column), context)); diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 196455c70f0..af817c52805 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -30,52 +30,33 @@ namespace { /// Verifying that the function depends only on the specified columns -bool isValidFunction(const ASTPtr & expression, const NameSet & columns) +bool isValidFunction(const ASTPtr & expression, const Block & block) { const auto * function = expression->as(); - if (function) + if (function && functionIsInOrGlobalInOperator(function->name)) { - if (functionIsInOrGlobalInOperator(function->name)) - { - // Second argument of IN can be a scalar subquery - if (!isValidFunction(function->arguments->children[0], columns)) - return false; - } - else if (function->name == "ignore") - { - return false; - } - else - { - if (function->arguments) - { - for (const auto & child : function->arguments->children) - if (!isValidFunction(child, columns)) - return false; - } - } + // Second argument of IN can be a scalar subquery + return isValidFunction(function->arguments->children[0], block); } else { - if (auto opt_name = IdentifierSemantic::getColumnName(expression)) - return columns.count(*opt_name); + auto column_name = expression->getColumnName(); + return block.has(column_name) && isColumnConst(*block.getByName(column_name).column); } - - return true; } /// Extract all subfunctions of the main conjunction, but depending only on the specified columns -bool extractFunctions(const ASTPtr & expression, const NameSet & columns, std::vector & result) +bool extractFunctions(const ASTPtr & expression, const Block & block, std::vector & result) { const auto * function = expression->as(); if (function && (function->name == "and" || function->name == "indexHint")) { bool ret = true; for (const auto & child : function->arguments->children) - ret &= extractFunctions(child, columns, result); + ret &= extractFunctions(child, block, result); return ret; } - else if (isValidFunction(expression, columns)) + else if (isValidFunction(expression, block)) { result.push_back(expression->clone()); return true; @@ -124,7 +105,6 @@ void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & va if (!select.with()) select.setExpression(ASTSelectQuery::Expression::WITH, std::make_shared()); - if (func.empty()) { auto literal = std::make_shared(value); @@ -144,37 +124,38 @@ void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & va } } -bool prepareFilterBlockWithQuery(const ASTPtr & query, const Block & block, ASTPtr & expression_ast) +bool prepareFilterBlockWithQuery(const SelectQueryInfo & query_info, const Context & context, Block block, ASTPtr & expression_ast) { bool ret = true; - const auto & select = query->as(); + const auto & select = query_info.query->as(); if (!select.where() && !select.prewhere()) return ret; - NameSet columns; - for (const auto & it : block.getNamesAndTypesList()) - columns.insert(it.name); + // Prepare a block with valid expressions + ExpressionAnalyzer(query_info.query, query_info.syntax_analyzer_result, context) + .getConstActions(block.getNamesAndTypesList()) + ->execute(block); /// We will create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns. std::vector functions; if (select.where()) - ret &= extractFunctions(select.where(), columns, functions); + ret &= extractFunctions(select.where(), block, functions); if (select.prewhere()) - ret &= extractFunctions(select.prewhere(), columns, functions); + ret &= extractFunctions(select.prewhere(), block, functions); expression_ast = buildWhereExpression(functions); return ret; } -void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context, ASTPtr expression_ast) +void filterBlockWithQuery(const SelectQueryInfo & query_info, Block & block, const Context & context, ASTPtr expression_ast) { if (!expression_ast) - prepareFilterBlockWithQuery(query, block, expression_ast); + prepareFilterBlockWithQuery(query_info, context, block, expression_ast); if (!expression_ast) return; - /// Let's analyze and calculate the expression. + /// Let's analyze and calculate the prepared expression. auto syntax_result = TreeRewriter(context).analyze(expression_ast, block.getNamesAndTypesList()); ExpressionAnalyzer analyzer(expression_ast, syntax_result, context); buildSets(expression_ast, analyzer); diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 326738f4259..7c414aa0984 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -4,6 +4,7 @@ #include #include +#include namespace DB @@ -25,13 +26,13 @@ namespace VirtualColumnUtils void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & value, const String & func = ""); /// Prepare `expression_ast` to filter block. Returns true if `expression_ast` is not trimmed, that is, -/// the sample block provides all needed columns, else return false. -bool prepareFilterBlockWithQuery(const ASTPtr & query, const Block & sample_block, ASTPtr & expression_ast); +/// `block` provides all needed columns for `expression_ast`, else return false. +bool prepareFilterBlockWithQuery(const SelectQueryInfo & query_info, const Context & context, Block block, ASTPtr & expression_ast); /// Leave in the block only the rows that fit under the WHERE clause and the PREWHERE clause of the query. /// Only elements of the outer conjunction are considered, depending only on the columns present in the block. /// If `expression_ast` is passed, use it to filter block. -void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context, ASTPtr expression_ast = {}); +void filterBlockWithQuery(const SelectQueryInfo & query_info, Block & block, const Context & context, ASTPtr expression_ast = {}); /// Extract from the input stream a set of `name` column values template diff --git a/tests/queries/0_stateless/00971_merge_tree_uniform_read_distribution_and_max_rows_to_read.sql b/tests/queries/0_stateless/00971_merge_tree_uniform_read_distribution_and_max_rows_to_read.sql index d31989d65bd..5abb1af620a 100644 --- a/tests/queries/0_stateless/00971_merge_tree_uniform_read_distribution_and_max_rows_to_read.sql +++ b/tests/queries/0_stateless/00971_merge_tree_uniform_read_distribution_and_max_rows_to_read.sql @@ -10,7 +10,8 @@ SELECT count() FROM merge_tree; SET max_rows_to_read = 900000; -SELECT count() FROM merge_tree WHERE not ignore(); -- { serverError 158 } -SELECT count() FROM merge_tree WHERE not ignore(); -- { serverError 158 } +-- constant ignore will be pruned by part pruner. ignore(*) is used. +SELECT count() FROM merge_tree WHERE not ignore(*); -- { serverError 158 } +SELECT count() FROM merge_tree WHERE not ignore(*); -- { serverError 158 } DROP TABLE merge_tree; From fac832227cd08c7e12170c4ca60fbc33a2b5b7e0 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 4 Mar 2021 16:53:31 +0800 Subject: [PATCH 05/60] Fix again --- src/Interpreters/ExpressionAnalyzer.cpp | 2 +- src/Interpreters/ExpressionAnalyzer.h | 2 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 9 ++++----- src/Storages/VirtualColumnUtils.cpp | 13 +++++++++++-- 4 files changed, 17 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 50dd6ede2a1..f931ecf8c21 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -1324,7 +1324,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool proje } -ExpressionActionsPtr ExpressionAnalyzer::getConstActions(const NamesAndTypesList & constant_inputs) +ExpressionActionsPtr ExpressionAnalyzer::getConstActions(const ColumnsWithTypeAndName & constant_inputs) { auto actions = std::make_shared(constant_inputs); diff --git a/src/Interpreters/ExpressionAnalyzer.h b/src/Interpreters/ExpressionAnalyzer.h index c44507cc4e2..69b6d75de35 100644 --- a/src/Interpreters/ExpressionAnalyzer.h +++ b/src/Interpreters/ExpressionAnalyzer.h @@ -110,7 +110,7 @@ public: /// Actions that can be performed on an empty block: adding constants and applying functions that depend only on constants. /// Does not execute subqueries. - ExpressionActionsPtr getConstActions(const NamesAndTypesList & constant_inputs = {}); + ExpressionActionsPtr getConstActions(const ColumnsWithTypeAndName & constant_inputs = {}); /** Sets that require a subquery to be create. * Only the sets needed to perform actions returned from already executed `append*` or `getActions`. diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 59bf32fc14a..2b26c59ee08 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -75,15 +75,14 @@ MergeTreeDataSelectExecutor::MergeTreeDataSelectExecutor(const MergeTreeData & d Block MergeTreeDataSelectExecutor::getSampleBlockWithVirtualPartColumns() { return Block(std::initializer_list{ - ColumnWithTypeAndName(DataTypeString().createColumnConstWithDefaultValue(1), std::make_shared(), "_part"), - ColumnWithTypeAndName(DataTypeString().createColumnConstWithDefaultValue(1), std::make_shared(), "_partition_id"), - ColumnWithTypeAndName(DataTypeUUID().createColumnConstWithDefaultValue(1), std::make_shared(), "_part_uuid")}); + ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "_part"), + ColumnWithTypeAndName(ColumnString::create(), std::make_shared(), "_partition_id"), + ColumnWithTypeAndName(ColumnUUID::create(), std::make_shared(), "_part_uuid")}); } void MergeTreeDataSelectExecutor::fillBlockWithVirtualPartColumns(const MergeTreeData::DataPartsVector & parts, Block & block) { - materializeBlockInplace(block); - MutableColumns columns = block.cloneEmptyColumns(); + MutableColumns columns = block.mutateColumns(); auto & part_column = columns[0]; auto & partition_id_column = columns[1]; diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index af817c52805..c69fa59d93a 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -131,9 +131,18 @@ bool prepareFilterBlockWithQuery(const SelectQueryInfo & query_info, const Conte if (!select.where() && !select.prewhere()) return ret; + ASTPtr condition_ast; + if (select.prewhere() && select.where()) + condition_ast = makeASTFunction("and", select.prewhere()->clone(), select.where()->clone()); + else + condition_ast = select.prewhere() ? select.prewhere()->clone() : select.where()->clone(); + // Prepare a block with valid expressions - ExpressionAnalyzer(query_info.query, query_info.syntax_analyzer_result, context) - .getConstActions(block.getNamesAndTypesList()) + for (size_t i = 0; i < block.columns(); ++i) + block.getByPosition(i).column = block.getByPosition(i).type->createColumnConstWithDefaultValue(1); + + ExpressionAnalyzer(condition_ast, query_info.syntax_analyzer_result, context) + .getConstActions(block.getColumnsWithTypeAndName()) ->execute(block); /// We will create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns. From 909cb3c24324e923a371255e01cbba6f0730b854 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 4 Mar 2021 22:27:07 +0800 Subject: [PATCH 06/60] Fix again.... --- src/Storages/MergeTree/MergeTreeData.cpp | 4 ++-- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 4 ++-- src/Storages/StorageMerge.cpp | 2 +- src/Storages/System/StorageSystemColumns.cpp | 4 ++-- .../System/StorageSystemDistributionQueue.cpp | 2 +- .../System/StorageSystemMutations.cpp | 2 +- .../System/StorageSystemPartsBase.cpp | 4 ++-- src/Storages/System/StorageSystemReplicas.cpp | 2 +- .../System/StorageSystemReplicationQueue.cpp | 2 +- src/Storages/System/StorageSystemTables.cpp | 2 +- src/Storages/VirtualColumnUtils.cpp | 21 ++++++++++++------- src/Storages/VirtualColumnUtils.h | 4 ++-- 12 files changed, 30 insertions(+), 23 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index 945fb5639c1..e7b83eae85d 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -684,7 +684,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( Block virtual_columns_block = MergeTreeDataSelectExecutor::getSampleBlockWithVirtualPartColumns(); // Generate valid expressions for filtering - bool valid = VirtualColumnUtils::prepareFilterBlockWithQuery(query_info, context, virtual_columns_block, expression_ast); + bool valid = VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, context, virtual_columns_block, expression_ast); PartitionPruner partition_pruner(metadata_snapshot->getPartitionKey(), query_info, context, true /* strict */); if (partition_pruner.isUseless() && !valid) @@ -694,7 +694,7 @@ std::optional MergeTreeData::totalRowsByPartitionPredicateImpl( if (valid && expression_ast) { MergeTreeDataSelectExecutor::fillBlockWithVirtualPartColumns(parts, virtual_columns_block); - VirtualColumnUtils::filterBlockWithQuery(query_info, virtual_columns_block, context, expression_ast); + VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, context, expression_ast); part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); if (part_values.empty()) return 0; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 2b26c59ee08..6588729e25e 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -223,13 +223,13 @@ QueryPlanPtr MergeTreeDataSelectExecutor::readFromParts( auto virtual_columns_block = getSampleBlockWithVirtualPartColumns(); // Generate valid expressions for filtering - VirtualColumnUtils::prepareFilterBlockWithQuery(query_info, context, virtual_columns_block, expression_ast); + VirtualColumnUtils::prepareFilterBlockWithQuery(query_info.query, context, virtual_columns_block, expression_ast); // If there is still something left, fill the virtual block and do the filtering. if (expression_ast) { fillBlockWithVirtualPartColumns(parts, virtual_columns_block); - VirtualColumnUtils::filterBlockWithQuery(query_info, virtual_columns_block, context, expression_ast); + VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, context, expression_ast); part_values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_part"); if (part_values.empty()) return std::make_unique(); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index 54f87d4e459..6ac8d02e54d 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -437,7 +437,7 @@ StorageMerge::StorageListWithLocks StorageMerge::getSelectedTables( if (has_virtual_column) { Block virtual_columns_block = Block{ColumnWithTypeAndName(std::move(virtual_column), std::make_shared(), "_table")}; - VirtualColumnUtils::filterBlockWithQuery(query_info, virtual_columns_block, global_context); + VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, global_context); auto values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_table"); /// Remove unused tables from the list diff --git a/src/Storages/System/StorageSystemColumns.cpp b/src/Storages/System/StorageSystemColumns.cpp index 75d90ffe1f8..6726d502071 100644 --- a/src/Storages/System/StorageSystemColumns.cpp +++ b/src/Storages/System/StorageSystemColumns.cpp @@ -299,7 +299,7 @@ Pipe StorageSystemColumns::read( block_to_filter.insert(ColumnWithTypeAndName(std::move(database_column_mut), std::make_shared(), "database")); /// Filter block with `database` column. - VirtualColumnUtils::filterBlockWithQuery(query_info, block_to_filter, context); + VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); if (!block_to_filter.rows()) { @@ -345,7 +345,7 @@ Pipe StorageSystemColumns::read( } /// Filter block with `database` and `table` columns. - VirtualColumnUtils::filterBlockWithQuery(query_info, block_to_filter, context); + VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); if (!block_to_filter.rows()) { diff --git a/src/Storages/System/StorageSystemDistributionQueue.cpp b/src/Storages/System/StorageSystemDistributionQueue.cpp index 58b468ab7f6..db649e7e1ba 100644 --- a/src/Storages/System/StorageSystemDistributionQueue.cpp +++ b/src/Storages/System/StorageSystemDistributionQueue.cpp @@ -155,7 +155,7 @@ void StorageSystemDistributionQueue::fillData(MutableColumns & res_columns, cons { col_table_to_filter, std::make_shared(), "table" }, }; - VirtualColumnUtils::filterBlockWithQuery(query_info, filtered_block, context); + VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); if (!filtered_block.rows()) return; diff --git a/src/Storages/System/StorageSystemMutations.cpp b/src/Storages/System/StorageSystemMutations.cpp index 50498f49c81..f66f57ef5d1 100644 --- a/src/Storages/System/StorageSystemMutations.cpp +++ b/src/Storages/System/StorageSystemMutations.cpp @@ -89,7 +89,7 @@ void StorageSystemMutations::fillData(MutableColumns & res_columns, const Contex { col_table, std::make_shared(), "table" }, }; - VirtualColumnUtils::filterBlockWithQuery(query_info, filtered_block, context); + VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); if (!filtered_block.rows()) return; diff --git a/src/Storages/System/StorageSystemPartsBase.cpp b/src/Storages/System/StorageSystemPartsBase.cpp index e88cf8b2865..39cc651e147 100644 --- a/src/Storages/System/StorageSystemPartsBase.cpp +++ b/src/Storages/System/StorageSystemPartsBase.cpp @@ -93,7 +93,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const std::move(database_column_mut), std::make_shared(), "database")); /// Filter block_to_filter with column 'database'. - VirtualColumnUtils::filterBlockWithQuery(query_info, block_to_filter, context); + VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); rows = block_to_filter.rows(); /// Block contains new columns, update database_column. @@ -162,7 +162,7 @@ StoragesInfoStream::StoragesInfoStream(const SelectQueryInfo & query_info, const if (rows) { /// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'. - VirtualColumnUtils::filterBlockWithQuery(query_info, block_to_filter, context); + VirtualColumnUtils::filterBlockWithQuery(query_info.query, block_to_filter, context); rows = block_to_filter.rows(); } diff --git a/src/Storages/System/StorageSystemReplicas.cpp b/src/Storages/System/StorageSystemReplicas.cpp index b66e578ea1a..0af67ab6986 100644 --- a/src/Storages/System/StorageSystemReplicas.cpp +++ b/src/Storages/System/StorageSystemReplicas.cpp @@ -135,7 +135,7 @@ Pipe StorageSystemReplicas::read( { col_engine, std::make_shared(), "engine" }, }; - VirtualColumnUtils::filterBlockWithQuery(query_info, filtered_block, context); + VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); if (!filtered_block.rows()) return {}; diff --git a/src/Storages/System/StorageSystemReplicationQueue.cpp b/src/Storages/System/StorageSystemReplicationQueue.cpp index 549b93b5ebc..9cd5e8b8ff3 100644 --- a/src/Storages/System/StorageSystemReplicationQueue.cpp +++ b/src/Storages/System/StorageSystemReplicationQueue.cpp @@ -98,7 +98,7 @@ void StorageSystemReplicationQueue::fillData(MutableColumns & res_columns, const { col_table_to_filter, std::make_shared(), "table" }, }; - VirtualColumnUtils::filterBlockWithQuery(query_info, filtered_block, context); + VirtualColumnUtils::filterBlockWithQuery(query_info.query, filtered_block, context); if (!filtered_block.rows()) return; diff --git a/src/Storages/System/StorageSystemTables.cpp b/src/Storages/System/StorageSystemTables.cpp index af25d299c2e..2b1788f991d 100644 --- a/src/Storages/System/StorageSystemTables.cpp +++ b/src/Storages/System/StorageSystemTables.cpp @@ -76,7 +76,7 @@ static ColumnPtr getFilteredDatabases(const SelectQueryInfo & query_info, const } Block block { ColumnWithTypeAndName(std::move(column), std::make_shared(), "database") }; - VirtualColumnUtils::filterBlockWithQuery(query_info, block, context); + VirtualColumnUtils::filterBlockWithQuery(query_info.query, block, context); return block.getByPosition(0).column; } diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index c69fa59d93a..537ef91f4f5 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -21,6 +21,7 @@ #include #include #include +#include namespace DB @@ -124,10 +125,10 @@ void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & va } } -bool prepareFilterBlockWithQuery(const SelectQueryInfo & query_info, const Context & context, Block block, ASTPtr & expression_ast) +bool prepareFilterBlockWithQuery(const ASTPtr & query, const Context & context, Block block, ASTPtr & expression_ast) { bool ret = true; - const auto & select = query_info.query->as(); + const auto & select = query->as(); if (!select.where() && !select.prewhere()) return ret; @@ -141,9 +142,15 @@ bool prepareFilterBlockWithQuery(const SelectQueryInfo & query_info, const Conte for (size_t i = 0; i < block.columns(); ++i) block.getByPosition(i).column = block.getByPosition(i).type->createColumnConstWithDefaultValue(1); - ExpressionAnalyzer(condition_ast, query_info.syntax_analyzer_result, context) - .getConstActions(block.getColumnsWithTypeAndName()) - ->execute(block); + auto actions = std::make_shared(block.getColumnsWithTypeAndName()); + PreparedSets prepared_sets; + SubqueriesForSets subqueries_for_sets; + ActionsVisitor::Data visitor_data( + context, SizeLimits{}, 1, {}, std::move(actions), prepared_sets, subqueries_for_sets, true, true, true, false); + ActionsVisitor(visitor_data).visit(condition_ast); + actions = visitor_data.getActions(); + auto expression_actions = std::make_shared(actions); + expression_actions->execute(block); /// We will create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns. std::vector functions; @@ -156,10 +163,10 @@ bool prepareFilterBlockWithQuery(const SelectQueryInfo & query_info, const Conte return ret; } -void filterBlockWithQuery(const SelectQueryInfo & query_info, Block & block, const Context & context, ASTPtr expression_ast) +void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context, ASTPtr expression_ast) { if (!expression_ast) - prepareFilterBlockWithQuery(query_info, context, block, expression_ast); + prepareFilterBlockWithQuery(query, context, block, expression_ast); if (!expression_ast) return; diff --git a/src/Storages/VirtualColumnUtils.h b/src/Storages/VirtualColumnUtils.h index 7c414aa0984..78e3d62472e 100644 --- a/src/Storages/VirtualColumnUtils.h +++ b/src/Storages/VirtualColumnUtils.h @@ -27,12 +27,12 @@ void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & va /// Prepare `expression_ast` to filter block. Returns true if `expression_ast` is not trimmed, that is, /// `block` provides all needed columns for `expression_ast`, else return false. -bool prepareFilterBlockWithQuery(const SelectQueryInfo & query_info, const Context & context, Block block, ASTPtr & expression_ast); +bool prepareFilterBlockWithQuery(const ASTPtr & query, const Context & context, Block block, ASTPtr & expression_ast); /// Leave in the block only the rows that fit under the WHERE clause and the PREWHERE clause of the query. /// Only elements of the outer conjunction are considered, depending only on the columns present in the block. /// If `expression_ast` is passed, use it to filter block. -void filterBlockWithQuery(const SelectQueryInfo & query_info, Block & block, const Context & context, ASTPtr expression_ast = {}); +void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context, ASTPtr expression_ast = {}); /// Extract from the input stream a set of `name` column values template From 091894f8cac17fffe5dfdd7aa1f088aaf0347439 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 8 Mar 2021 11:09:06 +0800 Subject: [PATCH 07/60] Add more comments --- src/Storages/VirtualColumnUtils.cpp | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 537ef91f4f5..4e2c9577089 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -127,10 +127,10 @@ void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & va bool prepareFilterBlockWithQuery(const ASTPtr & query, const Context & context, Block block, ASTPtr & expression_ast) { - bool ret = true; + bool unmodified = true; const auto & select = query->as(); if (!select.where() && !select.prewhere()) - return ret; + return unmodified; ASTPtr condition_ast; if (select.prewhere() && select.where()) @@ -138,10 +138,11 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, const Context & context, else condition_ast = select.prewhere() ? select.prewhere()->clone() : select.where()->clone(); - // Prepare a block with valid expressions + // Prepare a constant block with valid expressions for (size_t i = 0; i < block.columns(); ++i) block.getByPosition(i).column = block.getByPosition(i).type->createColumnConstWithDefaultValue(1); + // Collect all expression columns in expression_ast. Constant expressions will have constant columns. auto actions = std::make_shared(block.getColumnsWithTypeAndName()); PreparedSets prepared_sets; SubqueriesForSets subqueries_for_sets; @@ -152,15 +153,15 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, const Context & context, auto expression_actions = std::make_shared(actions); expression_actions->execute(block); - /// We will create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns. + /// Create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns. std::vector functions; if (select.where()) - ret &= extractFunctions(select.where(), block, functions); + unmodified &= extractFunctions(select.where(), block, functions); if (select.prewhere()) - ret &= extractFunctions(select.prewhere(), block, functions); + unmodified &= extractFunctions(select.prewhere(), block, functions); expression_ast = buildWhereExpression(functions); - return ret; + return unmodified; } void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context, ASTPtr expression_ast) From b936619fa963a128633b960b9b1b7134f2ed9307 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Tue, 9 Mar 2021 20:09:35 +0800 Subject: [PATCH 08/60] Rename --- .../{connectionID.cpp => connectionId.cpp} | 16 ++++++++-------- .../{partitionID.cpp => partitionId.cpp} | 12 ++++++------ src/Functions/registerFunctionsMiscellaneous.cpp | 8 ++++---- .../0_stateless/01748_partition_id_pruning.sql | 6 +++--- 4 files changed, 21 insertions(+), 21 deletions(-) rename src/Functions/{connectionID.cpp => connectionId.cpp} (64%) rename src/Functions/{partitionID.cpp => partitionId.cpp} (86%) diff --git a/src/Functions/connectionID.cpp b/src/Functions/connectionId.cpp similarity index 64% rename from src/Functions/connectionID.cpp rename to src/Functions/connectionId.cpp index 8e9c81aed6c..a2587afdb1b 100644 --- a/src/Functions/connectionID.cpp +++ b/src/Functions/connectionId.cpp @@ -7,15 +7,15 @@ namespace DB { -/// Get the connection ID. It's used for MySQL handler only. -class FunctionConnectionID : public IFunction +/// Get the connection Id. It's used for MySQL handler only. +class FunctionConnectionId : public IFunction { public: - static constexpr auto name = "connectionID"; + static constexpr auto name = "connectionId"; - explicit FunctionConnectionID(const Context & context_) : context(context_) {} + explicit FunctionConnectionId(const Context & context_) : context(context_) {} - static FunctionPtr create(const Context & context) { return std::make_shared(context); } + static FunctionPtr create(const Context & context) { return std::make_shared(context); } String getName() const override { return name; } @@ -32,10 +32,10 @@ private: const Context & context; }; -void registerFunctionConnectionID(FunctionFactory & factory) +void registerFunctionConnectionId(FunctionFactory & factory) { - factory.registerFunction(); - factory.registerAlias("connection_id", "connectionID"); + factory.registerFunction(); + factory.registerAlias("connection_id", "connectionId"); } } diff --git a/src/Functions/partitionID.cpp b/src/Functions/partitionId.cpp similarity index 86% rename from src/Functions/partitionID.cpp rename to src/Functions/partitionId.cpp index 9f5a3d09b86..b6d9d1bf4e6 100644 --- a/src/Functions/partitionID.cpp +++ b/src/Functions/partitionId.cpp @@ -15,14 +15,14 @@ namespace ErrorCodes } -/** partitionID(x, y, ...) is a function that computes partition ids of arguments. +/** partitionId(x, y, ...) is a function that computes partition ids of arguments. */ -class FunctionPartitionID : public IFunction +class FunctionPartitionId : public IFunction { public: - static constexpr auto name = "partitionID"; + static constexpr auto name = "partitionId"; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(const Context &) { return std::make_shared(); } String getName() const override { return name; } @@ -67,9 +67,9 @@ public: } }; -void registerFunctionPartitionID(FunctionFactory & factory) +void registerFunctionPartitionId(FunctionFactory & factory) { - factory.registerFunction(); + factory.registerFunction(); } } diff --git a/src/Functions/registerFunctionsMiscellaneous.cpp b/src/Functions/registerFunctionsMiscellaneous.cpp index 3557b0fb865..f2c35420ab6 100644 --- a/src/Functions/registerFunctionsMiscellaneous.cpp +++ b/src/Functions/registerFunctionsMiscellaneous.cpp @@ -69,8 +69,8 @@ void registerFunctionErrorCodeToName(FunctionFactory &); void registerFunctionTcpPort(FunctionFactory &); void registerFunctionByteSize(FunctionFactory &); void registerFunctionFile(FunctionFactory & factory); -void registerFunctionConnectionID(FunctionFactory & factory); -void registerFunctionPartitionID(FunctionFactory & factory); +void registerFunctionConnectionId(FunctionFactory & factory); +void registerFunctionPartitionId(FunctionFactory & factory); #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -140,8 +140,8 @@ void registerFunctionsMiscellaneous(FunctionFactory & factory) registerFunctionTcpPort(factory); registerFunctionByteSize(factory); registerFunctionFile(factory); - registerFunctionConnectionID(factory); - registerFunctionPartitionID(factory); + registerFunctionConnectionId(factory); + registerFunctionPartitionId(factory); #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/tests/queries/0_stateless/01748_partition_id_pruning.sql b/tests/queries/0_stateless/01748_partition_id_pruning.sql index cca366c5168..0a3f7d2713c 100644 --- a/tests/queries/0_stateless/01748_partition_id_pruning.sql +++ b/tests/queries/0_stateless/01748_partition_id_pruning.sql @@ -6,14 +6,14 @@ insert into x values (1, 1), (1, 2), (1, 3), (2, 4), (2, 5), (2, 6); set max_rows_to_read = 3; -select * from x where _partition_id = partitionID(1); +select * from x where _partition_id = partitionId(1); set max_rows_to_read = 4; -- one row for subquery -select * from x where _partition_id in (select partitionID(number + 1) from numbers(1)); +select * from x where _partition_id in (select partitionId(number + 1) from numbers(1)); -- trivial count optimization test set max_rows_to_read = 1; -- one row for subquery -select count() from x where _partition_id in (select partitionID(number + 1) from numbers(1)); +select count() from x where _partition_id in (select partitionId(number + 1) from numbers(1)); drop table x; From 02604185f288bee8635350a38f7515b22f2a5dc8 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 10 Mar 2021 02:48:46 +0800 Subject: [PATCH 09/60] Correctly check constant expr --- src/Storages/VirtualColumnUtils.cpp | 45 +++++++++++++++-------------- 1 file changed, 24 insertions(+), 21 deletions(-) diff --git a/src/Storages/VirtualColumnUtils.cpp b/src/Storages/VirtualColumnUtils.cpp index 4e2c9577089..0002d7c9c28 100644 --- a/src/Storages/VirtualColumnUtils.cpp +++ b/src/Storages/VirtualColumnUtils.cpp @@ -31,33 +31,30 @@ namespace { /// Verifying that the function depends only on the specified columns -bool isValidFunction(const ASTPtr & expression, const Block & block) +bool isValidFunction(const ASTPtr & expression, const std::function & is_constant) { const auto * function = expression->as(); if (function && functionIsInOrGlobalInOperator(function->name)) { // Second argument of IN can be a scalar subquery - return isValidFunction(function->arguments->children[0], block); + return isValidFunction(function->arguments->children[0], is_constant); } else - { - auto column_name = expression->getColumnName(); - return block.has(column_name) && isColumnConst(*block.getByName(column_name).column); - } + return is_constant(expression); } /// Extract all subfunctions of the main conjunction, but depending only on the specified columns -bool extractFunctions(const ASTPtr & expression, const Block & block, std::vector & result) +bool extractFunctions(const ASTPtr & expression, const std::function & is_constant, std::vector & result) { const auto * function = expression->as(); if (function && (function->name == "and" || function->name == "indexHint")) { bool ret = true; for (const auto & child : function->arguments->children) - ret &= extractFunctions(child, block, result); + ret &= extractFunctions(child, is_constant, result); return ret; } - else if (isValidFunction(expression, block)) + else if (isValidFunction(expression, is_constant)) { result.push_back(expression->clone()); return true; @@ -142,23 +139,29 @@ bool prepareFilterBlockWithQuery(const ASTPtr & query, const Context & context, for (size_t i = 0; i < block.columns(); ++i) block.getByPosition(i).column = block.getByPosition(i).type->createColumnConstWithDefaultValue(1); - // Collect all expression columns in expression_ast. Constant expressions will have constant columns. - auto actions = std::make_shared(block.getColumnsWithTypeAndName()); - PreparedSets prepared_sets; - SubqueriesForSets subqueries_for_sets; - ActionsVisitor::Data visitor_data( - context, SizeLimits{}, 1, {}, std::move(actions), prepared_sets, subqueries_for_sets, true, true, true, false); - ActionsVisitor(visitor_data).visit(condition_ast); - actions = visitor_data.getActions(); - auto expression_actions = std::make_shared(actions); - expression_actions->execute(block); + // Provide input columns as constant columns to check if an expression is constant. + std::function is_constant = [&block, &context](const ASTPtr & node) + { + auto actions = std::make_shared(block.getColumnsWithTypeAndName()); + PreparedSets prepared_sets; + SubqueriesForSets subqueries_for_sets; + ActionsVisitor::Data visitor_data( + context, SizeLimits{}, 1, {}, std::move(actions), prepared_sets, subqueries_for_sets, true, true, true, false); + ActionsVisitor(visitor_data).visit(node); + actions = visitor_data.getActions(); + auto expression_actions = std::make_shared(actions); + auto block_with_constants = block; + expression_actions->execute(block_with_constants); + auto column_name = node->getColumnName(); + return block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column); + }; /// Create an expression that evaluates the expressions in WHERE and PREWHERE, depending only on the existing columns. std::vector functions; if (select.where()) - unmodified &= extractFunctions(select.where(), block, functions); + unmodified &= extractFunctions(select.where(), is_constant, functions); if (select.prewhere()) - unmodified &= extractFunctions(select.prewhere(), block, functions); + unmodified &= extractFunctions(select.prewhere(), is_constant, functions); expression_ast = buildWhereExpression(functions); return unmodified; From 8e00ff78987f41f5fcd767af067cdbbea6517355 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 13 Mar 2021 04:25:13 +0300 Subject: [PATCH 10/60] try run two nodes with Replicated db and NuKeeper --- docker/test/stateless/run.sh | 49 ++++++++++++- tests/clickhouse-test | 7 +- tests/config/config.d/database_replicated.xml | 69 +++++++++++++++++++ tests/config/install.sh | 3 + 4 files changed, 124 insertions(+), 4 deletions(-) create mode 100644 tests/config/config.d/database_replicated.xml diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 3119ae27c59..0d67df8226c 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -11,6 +11,20 @@ dpkg -i package_folder/clickhouse-test_*.deb # install test configs /usr/share/clickhouse-test/config/install.sh +#cp /use_test_keeper.xml /etc/clickhouse-server/config.d/zookeeper.xml +#cp /enable_test_keeper1.xml /etc/clickhouse-server/config.d/test_keeper_port.xml +#cp /clusters2.xml /etc/clickhouse-server/config.d/ + +#mkdir /etc/clickhouse-server2 +#chown clickhouse /etc/clickhouse-server2 +#chgrp clickhouse /etc/clickhouse-server2 +#sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server2 +#rm /etc/clickhouse-server2/config.d/macros.xml +#sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|r1|r2|" > /etc/clickhouse-server2/config.d/macros.xml + +#cat /usr/bin/clickhouse-test | sed "s| ENGINE=Replicated('/test/clickhouse/db/{}', 's1', 'r1')| ON CLUSTER test_cluster_database_replicated ENGINE=Replicated('/test/clickhouse/db/{}', '{{shard}}', '{{replica}}')|" > /usr/bin/clickhouse-test-tmp +#mv /usr/bin/clickhouse-test-tmp /usr/bin/clickhouse-test +#chmod a+x /usr/bin/clickhouse-test # For flaky check we also enable thread fuzzer if [ "$NUM_TRIES" -gt "1" ]; then @@ -34,11 +48,33 @@ if [ "$NUM_TRIES" -gt "1" ]; then # simpliest way to forward env variables to server sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server/config.xml --daemon - sleep 5 else - service clickhouse-server start && sleep 5 + service clickhouse-server start fi +if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + # There is a bug in config reloading, so we cannot override macros using --macros.replica r2 + # And we have to copy configs... + mkdir /etc/clickhouse-server2 + chown clickhouse /etc/clickhouse-server2 + chgrp clickhouse /etc/clickhouse-server2 + sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server2 + rm /etc/clickhouse-server2/config.d/macros.xml + sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|r1|r2|" > /etc/clickhouse-server2/config.d/macros.xml + + sudo mkdir /var/lib/clickhouse2 + sudo chmod a=rwx /var/lib/clickhouse2 + sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server2/config.xml --daemon \ + -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ + --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ + --mysql_port 19004 \ + --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 \ + --macros.replica r2 # It doesn't work :( +fi + +sleep 5 + if grep -q -- "--use-skip-list" /usr/bin/clickhouse-test; then SKIP_LIST_OPT="--use-skip-list" fi @@ -68,6 +104,10 @@ function run_tests() | tee -a test_output/test_result.txt } +#clickhouse-client --port 9000 -q "SELECT * FROM system.macros" +#clickhouse-client --port 19000 -q "SELECT * FROM system.macros" +#clickhouse-client --port 19000 -q "SELECT 2" + export -f run_tests timeout "$MAX_RUN_TIME" bash -c run_tests ||: @@ -81,3 +121,8 @@ if [[ -n "$WITH_COVERAGE" ]] && [[ "$WITH_COVERAGE" -eq 1 ]]; then fi tar -chf /test_output/text_log_dump.tar /var/lib/clickhouse/data/system/text_log ||: tar -chf /test_output/query_log_dump.tar /var/lib/clickhouse/data/system/query_log ||: + +if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + pigz < /var/log/clickhouse-server/clickhouse-server2.log > /test_output/clickhouse-server2.log.gz ||: + mv /var/log/clickhouse-server/stderr2.log /test_output/ ||: +fi diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 198c8bbe0c9..212503a3e71 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -107,7 +107,7 @@ def remove_control_characters(s): def get_db_engine(args, database_name): if args.replicated_database: - return " ENGINE=Replicated('/test/clickhouse/db/{}', 's1', 'r1')".format(database_name) + return " ON CLUSTER test_cluster_database_replicated ENGINE=Replicated('/test/clickhouse/db/{}', 's1', 'r1')".format(database_name) if args.db_engine: return " ENGINE=" + args.db_engine return "" # Will use default engine @@ -172,7 +172,10 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std clickhouse_proc_create = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE, universal_newlines=True) seconds_left = max(args.timeout - (datetime.now() - start_time).total_seconds(), 10) try: - clickhouse_proc_create.communicate(("DROP DATABASE " + database), timeout=seconds_left) + drop_database_query = "DROP DATABASE " + database + if args.replicated_database: + drop_database_query += " ON CLUSTER test_cluster_database_replicated" + clickhouse_proc_create.communicate((drop_database_query), timeout=seconds_left) except TimeoutExpired: # kill test process because it can also hung if proc.returncode is None: diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml new file mode 100644 index 00000000000..7392968b415 --- /dev/null +++ b/tests/config/config.d/database_replicated.xml @@ -0,0 +1,69 @@ + + + + localhost + 9181 + + + localhost + 19181 + + + localhost + 29181 + + + + + 9181 + 1 + + + 5000 + 10000 + trace + + + + + 1 + localhost + 44444 + true + 3 + + + 2 + localhost + 44445 + true + true + 2 + + + 3 + localhost + 44446 + true + true + 1 + + + + + + + + + localhost + 9000 + + + localhost + 19000 + + + + + + diff --git a/tests/config/install.sh b/tests/config/install.sh index 1fca2b11e04..d0f978a3315 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -65,6 +65,9 @@ if [[ -n "$USE_DATABASE_ORDINARY" ]] && [[ "$USE_DATABASE_ORDINARY" -eq 1 ]]; th fi if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then ln -sf $SRC_PATH/users.d/database_replicated.xml $DEST_SERVER_PATH/users.d/ + ln -sf $SRC_PATH/config.d/database_replicated.xml $DEST_SERVER_PATH/config.d/ + rm /etc/clickhouse-server/config.d/zookeeper.xml + rm /etc/clickhouse-server/config.d/test_keeper_port.xml fi ln -sf $SRC_PATH/client_config.xml $DEST_CLIENT_PATH/config.xml From e62e256e77a99d6ceaf4b2b2efb98e491c634a0f Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sat, 13 Mar 2021 13:22:48 +0300 Subject: [PATCH 11/60] fix --- docker/test/stateful/run.sh | 22 +++++++++++ docker/test/stateless/clusters2.xml | 22 +++++++++++ docker/test/stateless/enable_test_keeper1.xml | 38 +++++++++++++++++++ docker/test/stateless/run.sh | 38 +++++-------------- docker/test/stateless/use_test_keeper.xml | 16 ++++++++ tests/clickhouse-test | 2 +- tests/config/config.d/database_replicated.xml | 6 +++ tests/config/install.sh | 22 +++++++++++ 8 files changed, 137 insertions(+), 29 deletions(-) create mode 100644 docker/test/stateless/clusters2.xml create mode 100644 docker/test/stateless/enable_test_keeper1.xml create mode 100644 docker/test/stateless/use_test_keeper.xml diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 6b90a9e7e37..195cfbfa3d3 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -13,6 +13,22 @@ dpkg -i package_folder/clickhouse-test_*.deb function start() { + if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server2/config.xml --daemon \ + -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ + --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ + --mysql_port 19004 \ + --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 + + sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server3/config.xml --daemon \ + -- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \ + --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ + --mysql_port 29004 \ + --test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3 + fi + counter=0 until clickhouse-client --query "SELECT 1" do @@ -73,3 +89,9 @@ mv /var/log/clickhouse-server/stderr.log /test_output/ ||: if [[ -n "$WITH_COVERAGE" ]] && [[ "$WITH_COVERAGE" -eq 1 ]]; then tar -chf /test_output/clickhouse_coverage.tar.gz /profraw ||: fi +if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + pigz < /var/log/clickhouse-server/clickhouse-server2.log > /test_output/clickhouse-server2.log.gz ||: + pigz < /var/log/clickhouse-server/clickhouse-server3.log > /test_output/clickhouse-server3.log.gz ||: + mv /var/log/clickhouse-server/stderr2.log /test_output/ ||: + mv /var/log/clickhouse-server/stderr3.log /test_output/ ||: +fi diff --git a/docker/test/stateless/clusters2.xml b/docker/test/stateless/clusters2.xml new file mode 100644 index 00000000000..73b6274df12 --- /dev/null +++ b/docker/test/stateless/clusters2.xml @@ -0,0 +1,22 @@ + + + + + + localhost + 9000 + + + localhost + 19000 + + + + + localhost + 29000 + + + + + diff --git a/docker/test/stateless/enable_test_keeper1.xml b/docker/test/stateless/enable_test_keeper1.xml new file mode 100644 index 00000000000..6c9669a42fd --- /dev/null +++ b/docker/test/stateless/enable_test_keeper1.xml @@ -0,0 +1,38 @@ + + + 9181 + 1 + + + 5000 + 10000 + trace + + + + + 1 + localhost + 44444 + true + 3 + + + 2 + localhost + 44445 + true + true + 2 + + + 3 + localhost + 44446 + true + true + 1 + + + + diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 0d67df8226c..dfa05e1a354 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -11,20 +11,6 @@ dpkg -i package_folder/clickhouse-test_*.deb # install test configs /usr/share/clickhouse-test/config/install.sh -#cp /use_test_keeper.xml /etc/clickhouse-server/config.d/zookeeper.xml -#cp /enable_test_keeper1.xml /etc/clickhouse-server/config.d/test_keeper_port.xml -#cp /clusters2.xml /etc/clickhouse-server/config.d/ - -#mkdir /etc/clickhouse-server2 -#chown clickhouse /etc/clickhouse-server2 -#chgrp clickhouse /etc/clickhouse-server2 -#sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server2 -#rm /etc/clickhouse-server2/config.d/macros.xml -#sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|r1|r2|" > /etc/clickhouse-server2/config.d/macros.xml - -#cat /usr/bin/clickhouse-test | sed "s| ENGINE=Replicated('/test/clickhouse/db/{}', 's1', 'r1')| ON CLUSTER test_cluster_database_replicated ENGINE=Replicated('/test/clickhouse/db/{}', '{{shard}}', '{{replica}}')|" > /usr/bin/clickhouse-test-tmp -#mv /usr/bin/clickhouse-test-tmp /usr/bin/clickhouse-test -#chmod a+x /usr/bin/clickhouse-test # For flaky check we also enable thread fuzzer if [ "$NUM_TRIES" -gt "1" ]; then @@ -53,17 +39,7 @@ else fi if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - # There is a bug in config reloading, so we cannot override macros using --macros.replica r2 - # And we have to copy configs... - mkdir /etc/clickhouse-server2 - chown clickhouse /etc/clickhouse-server2 - chgrp clickhouse /etc/clickhouse-server2 - sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server2 - rm /etc/clickhouse-server2/config.d/macros.xml - sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|r1|r2|" > /etc/clickhouse-server2/config.d/macros.xml - sudo mkdir /var/lib/clickhouse2 - sudo chmod a=rwx /var/lib/clickhouse2 sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server2/config.xml --daemon \ -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ @@ -71,6 +47,14 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] --mysql_port 19004 \ --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 \ --macros.replica r2 # It doesn't work :( + + sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server3/config.xml --daemon \ + -- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \ + --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ + --mysql_port 29004 \ + --test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3 \ + --macros.shard s2 # It doesn't work :( fi sleep 5 @@ -104,10 +88,6 @@ function run_tests() | tee -a test_output/test_result.txt } -#clickhouse-client --port 9000 -q "SELECT * FROM system.macros" -#clickhouse-client --port 19000 -q "SELECT * FROM system.macros" -#clickhouse-client --port 19000 -q "SELECT 2" - export -f run_tests timeout "$MAX_RUN_TIME" bash -c run_tests ||: @@ -124,5 +104,7 @@ tar -chf /test_output/query_log_dump.tar /var/lib/clickhouse/data/system/query_l if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then pigz < /var/log/clickhouse-server/clickhouse-server2.log > /test_output/clickhouse-server2.log.gz ||: + pigz < /var/log/clickhouse-server/clickhouse-server3.log > /test_output/clickhouse-server3.log.gz ||: mv /var/log/clickhouse-server/stderr2.log /test_output/ ||: + mv /var/log/clickhouse-server/stderr3.log /test_output/ ||: fi diff --git a/docker/test/stateless/use_test_keeper.xml b/docker/test/stateless/use_test_keeper.xml new file mode 100644 index 00000000000..26ec47e50e8 --- /dev/null +++ b/docker/test/stateless/use_test_keeper.xml @@ -0,0 +1,16 @@ + + + + localhost + 9181 + + + localhost + 19181 + + + localhost + 29181 + + + diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 212503a3e71..6d7ca807992 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -107,7 +107,7 @@ def remove_control_characters(s): def get_db_engine(args, database_name): if args.replicated_database: - return " ON CLUSTER test_cluster_database_replicated ENGINE=Replicated('/test/clickhouse/db/{}', 's1', 'r1')".format(database_name) + return " ON CLUSTER test_cluster_database_replicated ENGINE=Replicated('/test/clickhouse/db/{}', '{{shard}}', '{{replica}}')".format(database_name) if args.db_engine: return " ENGINE=" + args.db_engine return "" # Will use default engine diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml index 7392968b415..5b09a13ca4b 100644 --- a/tests/config/config.d/database_replicated.xml +++ b/tests/config/config.d/database_replicated.xml @@ -63,6 +63,12 @@ 19000 + + + localhost + 29000 + + diff --git a/tests/config/install.sh b/tests/config/install.sh index d0f978a3315..76cca9956cb 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -68,6 +68,28 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] ln -sf $SRC_PATH/config.d/database_replicated.xml $DEST_SERVER_PATH/config.d/ rm /etc/clickhouse-server/config.d/zookeeper.xml rm /etc/clickhouse-server/config.d/test_keeper_port.xml + + # There is a bug in config reloading, so we cannot override macros using --macros.replica r2 + # And we have to copy configs... + mkdir /etc/clickhouse-server2 + mkdir /etc/clickhouse-server3 + chown clickhouse /etc/clickhouse-server2 + chown clickhouse /etc/clickhouse-server3 + chgrp clickhouse /etc/clickhouse-server2 + chgrp clickhouse /etc/clickhouse-server3 + sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server2 + sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server3 + rm /etc/clickhouse-server2/config.d/macros.xml + rm /etc/clickhouse-server3/config.d/macros.xml + sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|r1|r2|" > /etc/clickhouse-server2/config.d/macros.xml + sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|s1|s2|" > /etc/clickhouse-server3/config.d/macros.xml + + sudo mkdir /var/lib/clickhouse2 + sudo mkdir /var/lib/clickhouse3 + sudo chown clickhouse /var/lib/clickhouse2 + sudo chown clickhouse /var/lib/clickhouse3 + sudo chgrp clickhouse /var/lib/clickhouse2 + sudo chgrp clickhouse /var/lib/clickhouse3 fi ln -sf $SRC_PATH/client_config.xml $DEST_CLIENT_PATH/config.xml From 735acf27d87afa546c5e3b0265941280678adce8 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 16 Mar 2021 19:39:31 +0300 Subject: [PATCH 12/60] try run stateful tests --- docker/test/stateful/run.sh | 64 +++++++++++++------ docker/test/stateless/clusters2.xml | 22 ------- docker/test/stateless/enable_test_keeper1.xml | 38 ----------- .../process_functional_tests_result.py | 12 +++- docker/test/stateless/run.sh | 15 ++--- docker/test/stateless/use_test_keeper.xml | 16 ----- tests/clickhouse-test | 2 + tests/queries/skip_list.json | 4 +- 8 files changed, 68 insertions(+), 105 deletions(-) delete mode 100644 docker/test/stateless/clusters2.xml delete mode 100644 docker/test/stateless/enable_test_keeper1.xml delete mode 100644 docker/test/stateless/use_test_keeper.xml diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 195cfbfa3d3..e2edb4a0238 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -14,14 +14,17 @@ dpkg -i package_folder/clickhouse-test_*.deb function start() { if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server2/config.xml --daemon \ + # NOTE We run "clickhouse server" instead of "clickhouse-server" + # to make "pidof clickhouse-server" return single pid of the main instance. + # We wil run main instance using "service clickhouse-server start" + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ --mysql_port 19004 \ --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 - sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server3/config.xml --daemon \ + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server3/config.xml --daemon \ -- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \ --logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \ --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ @@ -51,9 +54,8 @@ start /s3downloader --dataset-names $DATASETS chmod 777 -R /var/lib/clickhouse clickhouse-client --query "SHOW DATABASES" -clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary" -clickhouse-client --query "CREATE DATABASE test" +clickhouse-client --query "ATTACH DATABASE datasets ENGINE = Ordinary" service clickhouse-server restart # Wait for server to start accepting connections @@ -63,24 +65,50 @@ for _ in {1..120}; do done clickhouse-client --query "SHOW TABLES FROM datasets" -clickhouse-client --query "SHOW TABLES FROM test" -clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" -clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" -clickhouse-client --query "SHOW TABLES FROM test" - -if grep -q -- "--use-skip-list" /usr/bin/clickhouse-test ; then - SKIP_LIST_OPT="--use-skip-list" -fi - -# We can have several additional options so we path them as array because it's -# more idiologically correct. -read -ra ADDITIONAL_OPTIONS <<< "${ADDITIONAL_OPTIONS:-}" if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - ADDITIONAL_OPTIONS+=('--replicated-database') + clickhouse-client --query "CREATE DATABASE test ON CLUSTER 'test_cluster_database_replicated' + ENGINE=Replicated('/test/clickhouse/db/test', '{shard}', '{replica}')" + + clickhouse-client --query "CREATE TABLE test.hits AS datasets.hits_v1" + clickhouse-client --query "CREATE TABLE test.visits AS datasets.visits_v1" + + clickhouse-client --query "INSERT INTO test.hits SELECT * FROM datasets.hits_v1" + clickhouse-client --query "INSERT INTO test.visits SELECT * FROM datasets.visits_v1" + + clickhouse-client --query "DROP TABLE datasets.hits_v1" + clickhouse-client --query "DROP TABLE datasets.visits_v1" + + MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours) + MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited) +else + clickhouse-client --query "CREATE DATABASE test" + clickhouse-client --query "SHOW TABLES FROM test" + clickhouse-client --query "RENAME TABLE datasets.hits_v1 TO test.hits" + clickhouse-client --query "RENAME TABLE datasets.visits_v1 TO test.visits" fi -clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --print-time "$SKIP_LIST_OPT" "${ADDITIONAL_OPTIONS[@]}" "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt +clickhouse-client --query "SHOW TABLES FROM test" +clickhouse-client --query "SELECT count() FROM test.hits" +clickhouse-client --query "SELECT count() FROM test.visits" + +function run_tests() +{ + set -x + # We can have several additional options so we path them as array because it's + # more idiologically correct. + read -ra ADDITIONAL_OPTIONS <<< "${ADDITIONAL_OPTIONS:-}" + + if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + ADDITIONAL_OPTIONS+=('--replicated-database') + fi + + clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ + "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt +} + +export -f run_tests +timeout "$MAX_RUN_TIME" bash -c run_tests ||: ./process_functional_tests_result.py || echo -e "failure\tCannot parse results" > /test_output/check_status.tsv diff --git a/docker/test/stateless/clusters2.xml b/docker/test/stateless/clusters2.xml deleted file mode 100644 index 73b6274df12..00000000000 --- a/docker/test/stateless/clusters2.xml +++ /dev/null @@ -1,22 +0,0 @@ - - - - - - localhost - 9000 - - - localhost - 19000 - - - - - localhost - 29000 - - - - - diff --git a/docker/test/stateless/enable_test_keeper1.xml b/docker/test/stateless/enable_test_keeper1.xml deleted file mode 100644 index 6c9669a42fd..00000000000 --- a/docker/test/stateless/enable_test_keeper1.xml +++ /dev/null @@ -1,38 +0,0 @@ - - - 9181 - 1 - - - 5000 - 10000 - trace - - - - - 1 - localhost - 44444 - true - 3 - - - 2 - localhost - 44445 - true - true - 2 - - - 3 - localhost - 44446 - true - true - 1 - - - - diff --git a/docker/test/stateless/process_functional_tests_result.py b/docker/test/stateless/process_functional_tests_result.py index 27210ef9b80..02adf108212 100755 --- a/docker/test/stateless/process_functional_tests_result.py +++ b/docker/test/stateless/process_functional_tests_result.py @@ -12,6 +12,8 @@ UNKNOWN_SIGN = "[ UNKNOWN " SKIPPED_SIGN = "[ SKIPPED " HUNG_SIGN = "Found hung queries in processlist" +NO_TASK_TIMEOUT_SIGN = "All tests have finished" + def process_test_log(log_path): total = 0 skipped = 0 @@ -19,10 +21,13 @@ def process_test_log(log_path): failed = 0 success = 0 hung = False + task_timeout = True test_results = [] with open(log_path, 'r') as test_file: for line in test_file: line = line.strip() + if NO_TASK_TIMEOUT_SIGN in line: + task_timeout = False if HUNG_SIGN in line: hung = True if any(sign in line for sign in (OK_SIGN, FAIL_SING, UNKNOWN_SIGN, SKIPPED_SIGN)): @@ -52,7 +57,7 @@ def process_test_log(log_path): else: success += int(OK_SIGN in line) test_results.append((test_name, "OK", test_time)) - return total, skipped, unknown, failed, success, hung, test_results + return total, skipped, unknown, failed, success, hung, task_timeout, test_results def process_result(result_path): test_results = [] @@ -68,7 +73,7 @@ def process_result(result_path): state = "error" if result_path and os.path.exists(result_path): - total, skipped, unknown, failed, success, hung, test_results = process_test_log(result_path) + total, skipped, unknown, failed, success, hung, task_timeout, test_results = process_test_log(result_path) is_flacky_check = 1 < int(os.environ.get('NUM_TRIES', 1)) # If no tests were run (success == 0) it indicates an error (e.g. server did not start or crashed immediately) # But it's Ok for "flaky checks" - they can contain just one test for check which is marked as skipped. @@ -78,6 +83,9 @@ def process_result(result_path): if hung: description = "Some queries hung, " state = "failure" + elif task_timeout: + description = "Timeout, " + state = "failure" else: description = "" diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index dfa05e1a354..494be95b490 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -40,7 +40,7 @@ fi if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server2/config.xml --daemon \ + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ @@ -48,23 +48,23 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 \ --macros.replica r2 # It doesn't work :( - sudo -E -u clickhouse /usr/bin/clickhouse-server --config /etc/clickhouse-server3/config.xml --daemon \ + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server3/config.xml --daemon \ -- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \ --logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \ --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ --mysql_port 29004 \ --test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3 \ --macros.shard s2 # It doesn't work :( + + MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours) + MAX_RUN_TIME=$((MAX_RUN_TIME != 0 ? MAX_RUN_TIME : 9000)) # set to 2.5 hours if 0 (unlimited) fi sleep 5 -if grep -q -- "--use-skip-list" /usr/bin/clickhouse-test; then - SKIP_LIST_OPT="--use-skip-list" -fi - function run_tests() { + set -x # We can have several additional options so we path them as array because it's # more idiologically correct. read -ra ADDITIONAL_OPTIONS <<< "${ADDITIONAL_OPTIONS:-}" @@ -82,8 +82,7 @@ function run_tests() fi clickhouse-test --testname --shard --zookeeper --hung-check --print-time \ - --test-runs "$NUM_TRIES" \ - "$SKIP_LIST_OPT" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ + --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt } diff --git a/docker/test/stateless/use_test_keeper.xml b/docker/test/stateless/use_test_keeper.xml deleted file mode 100644 index 26ec47e50e8..00000000000 --- a/docker/test/stateless/use_test_keeper.xml +++ /dev/null @@ -1,16 +0,0 @@ - - - - localhost - 9181 - - - localhost - 19181 - - - localhost - 29181 - - - diff --git a/tests/clickhouse-test b/tests/clickhouse-test index fce8fe65f30..56e311a5e11 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -843,6 +843,8 @@ def main(args): if total_tests_run == 0: print("No tests were run.") sys.exit(1) + else: + print("All tests have finished.") sys.exit(exit_code) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 957108cc9cc..4b613369cc2 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -135,6 +135,7 @@ "00626_replace_partition_from_table_zookeeper", "00626_replace_partition_from_table", "00152_insert_different_granularity", + "00054_merge_tree_partitions", /// Old syntax is not allowed "01062_alter_on_mutataion_zookeeper", "00925_zookeeper_empty_replicated_merge_tree_optimize_final", @@ -150,7 +151,8 @@ "00083_create_merge_tree_zookeeper", "00062_replicated_merge_tree_alter_zookeeper", /// Does not support renaming of multiple tables in single query - "00634_rename_view" + "00634_rename_view", + "00140_rename" ], "polymorphic-parts": [ "01508_partition_pruning_long", /// bug, shoud be fixed From ff79be8af90e57feaf3dfb8d6c017349cbeb9403 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 16 Mar 2021 23:01:20 +0300 Subject: [PATCH 13/60] fix --- docker/test/stateful/run.sh | 2 +- docker/test/stateless/run.sh | 2 +- src/Common/Macros.cpp | 3 + src/Common/Macros.h | 1 + src/Databases/DatabaseReplicated.cpp | 72 ++++++++++++++++--- src/Databases/DatabaseReplicated.h | 2 + tests/config/config.d/database_replicated.xml | 1 + 7 files changed, 73 insertions(+), 10 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index e2edb4a0238..3f840d9bfec 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -103,7 +103,7 @@ function run_tests() ADDITIONAL_OPTIONS+=('--replicated-database') fi - clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --print-time "${ADDITIONAL_OPTIONS[@]}" \ + clickhouse-test --testname --shard --zookeeper --no-stateless --hung-check --use-skip-list --print-time "${ADDITIONAL_OPTIONS[@]}" \ "$SKIP_TESTS_OPTION" 2>&1 | ts '%Y-%m-%d %H:%M:%S' | tee test_output/test_result.txt } diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index 494be95b490..df8b78fddc0 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -82,7 +82,7 @@ function run_tests() fi clickhouse-test --testname --shard --zookeeper --hung-check --print-time \ - --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ + --use-skip-list --test-runs "$NUM_TRIES" "${ADDITIONAL_OPTIONS[@]}" 2>&1 \ | ts '%Y-%m-%d %H:%M:%S' \ | tee -a test_output/test_result.txt } diff --git a/src/Common/Macros.cpp b/src/Common/Macros.cpp index b8e25499c0b..7882449b595 100644 --- a/src/Common/Macros.cpp +++ b/src/Common/Macros.cpp @@ -78,7 +78,10 @@ String Macros::expand(const String & s, /// Prefer explicit macros over implicit. if (it != macros.end() && !info.expand_special_macros_only) + { res += it->second; + info.expanded_other = true; + } else if (macro_name == "database" && !info.table_id.database_name.empty()) { res += info.table_id.database_name; diff --git a/src/Common/Macros.h b/src/Common/Macros.h index 3082452e297..9298dbfc2d5 100644 --- a/src/Common/Macros.h +++ b/src/Common/Macros.h @@ -40,6 +40,7 @@ public: bool expanded_database = false; bool expanded_table = false; bool expanded_uuid = false; + bool expanded_other = false; bool has_unknown = false; }; diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 83609606bb8..61eabf6ebea 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -23,6 +23,7 @@ #include #include #include +#include namespace DB { @@ -314,20 +315,65 @@ void DatabaseReplicated::loadStoredObjects(Context & context, bool has_force_res ddl_worker->startup(); } -BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, const Context & query_context) +void DatabaseReplicated::checkQueryValid(const ASTPtr & query, const Context & query_context) const { - if (is_readonly) - throw Exception(ErrorCodes::NO_ZOOKEEPER, "Database is in readonly mode, because it cannot connect to ZooKeeper"); - - if (query_context.getClientInfo().query_kind != ClientInfo::QueryKind::INITIAL_QUERY) - throw Exception(ErrorCodes::INCORRECT_QUERY, "It's not initial query. ON CLUSTER is not allowed for Replicated database."); - /// Replicas will set correct name of current database in query context (database name can be different on replicas) - if (auto * ddl_query = query->as()) + if (auto * ddl_query = dynamic_cast(query.get())) { if (ddl_query->database != getDatabaseName()) throw Exception(ErrorCodes::UNKNOWN_DATABASE, "Database was renamed"); ddl_query->database.clear(); + + if (auto * create = query->as()) + { + bool replicated_table = create->storage && create->storage->engine && startsWith(create->storage->engine->name, "Replicated"); + if (!replicated_table || !create->storage->engine->arguments) + return; + + ASTs & args = create->storage->engine->arguments->children; + if (args.size() < 2) + return; + + ASTLiteral * arg1 = args[0]->as(); + ASTLiteral * arg2 = args[1]->as(); + if (!arg1 || !arg2 || arg1->value.getType() != Field::Types::String || arg2->value.getType() != Field::Types::String) + return; + + String maybe_path = arg1->value.get(); + String maybe_replica = arg2->value.get(); + + /// Looks like it's ReplicatedMergeTree with explicit zookeeper_path and replica_name arguments. + /// Let's ensure that some macros are used. + /// NOTE: we cannot check here that substituted values will be actually different on shards and replicas. + + Macros::MacroExpansionInfo info; + info.table_id = {getDatabaseName(), create->table, create->uuid}; + query_context.getMacros()->expand(maybe_path, info); + bool maybe_shard_macros = info.expanded_other; + info.expanded_other = false; + query_context.getMacros()->expand(maybe_replica, info); + bool maybe_replica_macros = info.expanded_other; + bool enable_functional_tests_helper = global_context.getConfigRef().has("_functional_tests_helper_database_replicated_replace_args_macros"); + + if (enable_functional_tests_helper) + { + if (maybe_path.empty() || maybe_path.back() != '/') + maybe_path += '/'; + arg1->value = maybe_path + "{shard}"; + arg2->value = maybe_replica + "{replica}"; + return; + } + + LOG_WARNING(log, "It's not recommended to explicitly specify zookeeper_path and replica_name in ReplicatedMergeTree arguments"); + + if (maybe_shard_macros && maybe_replica_macros) + return; + + throw Exception(ErrorCodes::INCORRECT_QUERY, + "Explicit zookeeper_path and replica_name are specified in ReplicatedMergeTree arguments. " + "If you really want to specify it explicitly, then you should use some macros " + "to distinguish different shards and replicas"); + } } if (const auto * query_alter = query->as()) @@ -348,7 +394,17 @@ BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, const "Use DETACH TABLE PERMANENTLY or SYSTEM RESTART REPLICA or set " "database_replicated_always_detach_permanently to 1"); } +} +BlockIO DatabaseReplicated::tryEnqueueReplicatedDDL(const ASTPtr & query, const Context & query_context) +{ + if (is_readonly) + throw Exception(ErrorCodes::NO_ZOOKEEPER, "Database is in readonly mode, because it cannot connect to ZooKeeper"); + + if (query_context.getClientInfo().query_kind != ClientInfo::QueryKind::INITIAL_QUERY) + throw Exception(ErrorCodes::INCORRECT_QUERY, "It's not initial query. ON CLUSTER is not allowed for Replicated database."); + + checkQueryValid(query, query_context); LOG_DEBUG(log, "Proposing query: {}", queryToString(query)); DDLLogEntry entry; diff --git a/src/Databases/DatabaseReplicated.h b/src/Databases/DatabaseReplicated.h index 8f2ccd27627..46bcdc71ef1 100644 --- a/src/Databases/DatabaseReplicated.h +++ b/src/Databases/DatabaseReplicated.h @@ -71,6 +71,8 @@ private: bool createDatabaseNodesInZooKeeper(const ZooKeeperPtr & current_zookeeper); void createReplicaNodesInZooKeeper(const ZooKeeperPtr & current_zookeeper); + void checkQueryValid(const ASTPtr & query, const Context & query_context) const; + void recoverLostReplica(const ZooKeeperPtr & current_zookeeper, UInt32 our_log_ptr, UInt32 max_log_ptr); std::map tryGetConsistentMetadataSnapshot(const ZooKeeperPtr & zookeeper, UInt32 & max_log_ptr); diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml index 5b09a13ca4b..a030dd1e861 100644 --- a/tests/config/config.d/database_replicated.xml +++ b/tests/config/config.d/database_replicated.xml @@ -72,4 +72,5 @@ + <_functional_tests_helper_database_replicated_replace_args_macros>1 From 69cbb99f2c651d01ff352a1b5ce822c0427ea755 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 17 Mar 2021 01:01:48 +0300 Subject: [PATCH 14/60] fix --- src/Interpreters/InterpreterAlterQuery.cpp | 1 + src/Interpreters/InterpreterDropQuery.cpp | 8 ++++---- src/Interpreters/InterpreterDropQuery.h | 4 ++-- 3 files changed, 7 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index 37eaecf9a90..a4672887eac 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -48,6 +48,7 @@ BlockIO InterpreterAlterQuery::execute() context.checkAccess(getRequiredAccess()); auto table_id = context.resolveStorageID(alter, Context::ResolveOrdinary); + query_ptr->as().database = table_id.database_name;; DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name); if (typeid_cast(database.get()) && context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY) diff --git a/src/Interpreters/InterpreterDropQuery.cpp b/src/Interpreters/InterpreterDropQuery.cpp index b30996b1dbf..276f61d244a 100644 --- a/src/Interpreters/InterpreterDropQuery.cpp +++ b/src/Interpreters/InterpreterDropQuery.cpp @@ -82,7 +82,7 @@ void InterpreterDropQuery::waitForTableToBeActuallyDroppedOrDetached(const ASTDr db->waitDetachedTableNotInUse(uuid_to_wait); } -BlockIO InterpreterDropQuery::executeToTable(const ASTDropQuery & query) +BlockIO InterpreterDropQuery::executeToTable(ASTDropQuery & query) { DatabasePtr database; UUID table_to_wait_on = UUIDHelpers::Nil; @@ -92,7 +92,7 @@ BlockIO InterpreterDropQuery::executeToTable(const ASTDropQuery & query) return res; } -BlockIO InterpreterDropQuery::executeToTableImpl(const ASTDropQuery & query, DatabasePtr & db, UUID & uuid_to_wait) +BlockIO InterpreterDropQuery::executeToTableImpl(ASTDropQuery & query, DatabasePtr & db, UUID & uuid_to_wait) { /// NOTE: it does not contain UUID, we will resolve it with locked DDLGuard auto table_id = StorageID(query); @@ -101,7 +101,7 @@ BlockIO InterpreterDropQuery::executeToTableImpl(const ASTDropQuery & query, Dat if (context.tryResolveStorageID(table_id, Context::ResolveExternal)) return executeToTemporaryTable(table_id.getTableName(), query.kind); else - table_id.database_name = context.getCurrentDatabase(); + query.database = table_id.database_name = context.getCurrentDatabase(); } if (query.temporary) @@ -212,7 +212,7 @@ BlockIO InterpreterDropQuery::executeToDictionary( String database_name = context.resolveDatabase(database_name_); auto ddl_guard = (!no_ddl_lock ? DatabaseCatalog::instance().getDDLGuard(database_name, dictionary_name) : nullptr); - + query_ptr->as()->database = database_name; DatabasePtr database = tryGetDatabase(database_name, if_exists); bool is_drop_or_detach_database = query_ptr->as()->table.empty(); diff --git a/src/Interpreters/InterpreterDropQuery.h b/src/Interpreters/InterpreterDropQuery.h index d51ce3293ec..4a67857767f 100644 --- a/src/Interpreters/InterpreterDropQuery.h +++ b/src/Interpreters/InterpreterDropQuery.h @@ -34,8 +34,8 @@ private: BlockIO executeToDatabase(const ASTDropQuery & query); BlockIO executeToDatabaseImpl(const ASTDropQuery & query, DatabasePtr & database, std::vector & uuids_to_wait); - BlockIO executeToTable(const ASTDropQuery & query); - BlockIO executeToTableImpl(const ASTDropQuery & query, DatabasePtr & db, UUID & uuid_to_wait); + BlockIO executeToTable(ASTDropQuery & query); + BlockIO executeToTableImpl(ASTDropQuery & query, DatabasePtr & db, UUID & uuid_to_wait); static void waitForTableToBeActuallyDroppedOrDetached(const ASTDropQuery & query, const DatabasePtr & db, const UUID & uuid_to_wait); From 7ae5e51ee9a65cbeefcda2d57876228a405d03c8 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 17 Mar 2021 01:50:37 +0300 Subject: [PATCH 15/60] Update InterpreterAlterQuery.cpp --- src/Interpreters/InterpreterAlterQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterAlterQuery.cpp b/src/Interpreters/InterpreterAlterQuery.cpp index a4672887eac..3ad7702548e 100644 --- a/src/Interpreters/InterpreterAlterQuery.cpp +++ b/src/Interpreters/InterpreterAlterQuery.cpp @@ -48,7 +48,7 @@ BlockIO InterpreterAlterQuery::execute() context.checkAccess(getRequiredAccess()); auto table_id = context.resolveStorageID(alter, Context::ResolveOrdinary); - query_ptr->as().database = table_id.database_name;; + query_ptr->as().database = table_id.database_name; DatabasePtr database = DatabaseCatalog::instance().getDatabase(table_id.database_name); if (typeid_cast(database.get()) && context.getClientInfo().query_kind != ClientInfo::QueryKind::SECONDARY_QUERY) From 91068b782bb9c5b0fe66408bcb8ae9014e1d26f0 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Wed, 17 Mar 2021 17:29:24 +0300 Subject: [PATCH 16/60] fix --- src/Interpreters/DDLTask.cpp | 11 +++++++++++ src/Interpreters/DDLTask.h | 3 ++- tests/integration/test_replicated_database/test.py | 11 ++++++++++- 3 files changed, 23 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/DDLTask.cpp b/src/Interpreters/DDLTask.cpp index 1cfd113e81f..d446422124a 100644 --- a/src/Interpreters/DDLTask.cpp +++ b/src/Interpreters/DDLTask.cpp @@ -344,6 +344,17 @@ String DatabaseReplicatedTask::getShardID() const return database->shard_name; } +void DatabaseReplicatedTask::parseQueryFromEntry(const Context & context) +{ + DDLTaskBase::parseQueryFromEntry(context); + if (auto * ddl_query = dynamic_cast(query.get())) + { + /// Update database name with actual name of local database + assert(ddl_query->database.empty()); + ddl_query->database = database->getDatabaseName(); + } +} + std::unique_ptr DatabaseReplicatedTask::makeQueryContext(Context & from_context, const ZooKeeperPtr & zookeeper) { auto query_context = DDLTaskBase::makeQueryContext(from_context, zookeeper); diff --git a/src/Interpreters/DDLTask.h b/src/Interpreters/DDLTask.h index b794668f802..a333bb1af45 100644 --- a/src/Interpreters/DDLTask.h +++ b/src/Interpreters/DDLTask.h @@ -93,7 +93,7 @@ struct DDLTaskBase DDLTaskBase(const DDLTaskBase &) = delete; virtual ~DDLTaskBase() = default; - void parseQueryFromEntry(const Context & context); + virtual void parseQueryFromEntry(const Context & context); virtual String getShardID() const = 0; @@ -134,6 +134,7 @@ struct DatabaseReplicatedTask : public DDLTaskBase DatabaseReplicatedTask(const String & name, const String & path, DatabaseReplicated * database_); String getShardID() const override; + void parseQueryFromEntry(const Context & context) override; std::unique_ptr makeQueryContext(Context & from_context, const ZooKeeperPtr & zookeeper) override; DatabaseReplicated * database; diff --git a/tests/integration/test_replicated_database/test.py b/tests/integration/test_replicated_database/test.py index f02457b144a..70d779ea737 100644 --- a/tests/integration/test_replicated_database/test.py +++ b/tests/integration/test_replicated_database/test.py @@ -35,8 +35,17 @@ def started_cluster(): cluster.shutdown() def test_create_replicated_table(started_cluster): + assert "Explicit zookeeper_path and replica_name are specified" in \ + main_node.query_and_get_error("CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) " + "ENGINE=ReplicatedMergeTree('/test/tmp', 'r') ORDER BY k PARTITION BY toYYYYMM(d);") + + assert "Explicit zookeeper_path and replica_name are specified" in \ + main_node.query_and_get_error("CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) " + "ENGINE=ReplicatedMergeTree('/test/tmp', 'r', d, k, 8192);") + assert "Old syntax is not allowed" in \ - main_node.query_and_get_error("CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree('/test/tmp', 'r', d, k, 8192);") + main_node.query_and_get_error("CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) " + "ENGINE=ReplicatedMergeTree('/test/tmp/{shard}', '{replica}', d, k, 8192);") main_node.query("CREATE TABLE testdb.replicated_table (d Date, k UInt64, i32 Int32) ENGINE=ReplicatedMergeTree ORDER BY k PARTITION BY toYYYYMM(d);") From 710c491f63bc24f26e266766a6b693e3c50fec19 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 17 Mar 2021 19:50:56 +0300 Subject: [PATCH 17/60] Update database_replicated.xml --- tests/config/config.d/database_replicated.xml | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml index a030dd1e861..3bbae6031c9 100644 --- a/tests/config/config.d/database_replicated.xml +++ b/tests/config/config.d/database_replicated.xml @@ -22,6 +22,7 @@ 5000 10000 trace + false From fdae70df6ca2e5ddd71d9cae39139af02f5acbd4 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 18 Mar 2021 15:49:31 +0300 Subject: [PATCH 18/60] fix tests --- docker/test/stateful/run.sh | 16 +++++++-------- docker/test/stateless/run.sh | 16 +++++++-------- src/Databases/DatabaseReplicated.cpp | 15 +++++++------- tests/clickhouse-test | 9 ++++++++- tests/config/install.sh | 20 +++++++++---------- ...ted_minimalistic_part_header_zookeeper.sql | 8 ++++---- ...0953_zookeeper_suetin_deduplication_bug.sh | 14 ++++++------- ...ion_collapsing_attach_detach_zookeeper.sql | 4 ++-- ...icated_mutations_empty_partition.reference | 2 +- ...6_replicated_mutations_empty_partition.sql | 6 +++--- ...ter_mutations_kill_many_replicas.reference | 10 +++++----- ...rent_alter_mutations_kill_many_replicas.sh | 8 +++++--- .../01700_system_zookeeper_path_in.reference | 6 ++---- .../01700_system_zookeeper_path_in.sql | 16 ++++++++------- tests/queries/skip_list.json | 8 ++++++++ 15 files changed, 88 insertions(+), 70 deletions(-) diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 3f840d9bfec..3682686bc99 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -17,16 +17,16 @@ function start() # NOTE We run "clickhouse server" instead of "clickhouse-server" # to make "pidof clickhouse-server" return single pid of the main instance. # We wil run main instance using "service clickhouse-server start" - sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ - -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ - --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \ + -- --path /var/lib/clickhouse1/ --logger.stderr /var/log/clickhouse-server/stderr1.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server1.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server1.err.log \ --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ --mysql_port 19004 \ --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 - sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server3/config.xml --daemon \ - -- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \ - --logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \ + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ + -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ --mysql_port 29004 \ --test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3 @@ -118,8 +118,8 @@ if [[ -n "$WITH_COVERAGE" ]] && [[ "$WITH_COVERAGE" -eq 1 ]]; then tar -chf /test_output/clickhouse_coverage.tar.gz /profraw ||: fi if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + pigz < /var/log/clickhouse-server/clickhouse-server1.log > /test_output/clickhouse-server1.log.gz ||: pigz < /var/log/clickhouse-server/clickhouse-server2.log > /test_output/clickhouse-server2.log.gz ||: - pigz < /var/log/clickhouse-server/clickhouse-server3.log > /test_output/clickhouse-server3.log.gz ||: + mv /var/log/clickhouse-server/stderr1.log /test_output/ ||: mv /var/log/clickhouse-server/stderr2.log /test_output/ ||: - mv /var/log/clickhouse-server/stderr3.log /test_output/ ||: fi diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index df8b78fddc0..fda9e61dd29 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -40,17 +40,17 @@ fi if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then - sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ - -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ - --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server1/config.xml --daemon \ + -- --path /var/lib/clickhouse1/ --logger.stderr /var/log/clickhouse-server/stderr1.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server1.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server1.err.log \ --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ --mysql_port 19004 \ --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 \ --macros.replica r2 # It doesn't work :( - sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server3/config.xml --daemon \ - -- --path /var/lib/clickhouse3/ --logger.stderr /var/log/clickhouse-server/stderr3.log \ - --logger.log /var/log/clickhouse-server/clickhouse-server3.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server3.err.log \ + sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ + -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ + --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ --mysql_port 29004 \ --test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3 \ @@ -102,8 +102,8 @@ tar -chf /test_output/text_log_dump.tar /var/lib/clickhouse/data/system/text_log tar -chf /test_output/query_log_dump.tar /var/lib/clickhouse/data/system/query_log ||: if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]]; then + pigz < /var/log/clickhouse-server/clickhouse-server1.log > /test_output/clickhouse-server1.log.gz ||: pigz < /var/log/clickhouse-server/clickhouse-server2.log > /test_output/clickhouse-server2.log.gz ||: - pigz < /var/log/clickhouse-server/clickhouse-server3.log > /test_output/clickhouse-server3.log.gz ||: + mv /var/log/clickhouse-server/stderr1.log /test_output/ ||: mv /var/log/clickhouse-server/stderr2.log /test_output/ ||: - mv /var/log/clickhouse-server/stderr3.log /test_output/ ||: fi diff --git a/src/Databases/DatabaseReplicated.cpp b/src/Databases/DatabaseReplicated.cpp index 61eabf6ebea..9ae03ad0900 100644 --- a/src/Databases/DatabaseReplicated.cpp +++ b/src/Databases/DatabaseReplicated.cpp @@ -355,20 +355,21 @@ void DatabaseReplicated::checkQueryValid(const ASTPtr & query, const Context & q bool maybe_replica_macros = info.expanded_other; bool enable_functional_tests_helper = global_context.getConfigRef().has("_functional_tests_helper_database_replicated_replace_args_macros"); + if (!enable_functional_tests_helper) + LOG_WARNING(log, "It's not recommended to explicitly specify zookeeper_path and replica_name in ReplicatedMergeTree arguments"); + + if (maybe_shard_macros && maybe_replica_macros) + return; + if (enable_functional_tests_helper) { if (maybe_path.empty() || maybe_path.back() != '/') maybe_path += '/'; - arg1->value = maybe_path + "{shard}"; - arg2->value = maybe_replica + "{replica}"; + arg1->value = maybe_path + "auto_{shard}"; + arg2->value = maybe_replica + "auto_{replica}"; return; } - LOG_WARNING(log, "It's not recommended to explicitly specify zookeeper_path and replica_name in ReplicatedMergeTree arguments"); - - if (maybe_shard_macros && maybe_replica_macros) - return; - throw Exception(ErrorCodes::INCORRECT_QUERY, "Explicit zookeeper_path and replica_name are specified in ReplicatedMergeTree arguments. " "If you really want to specify it explicitly, then you should use some macros " diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 56e311a5e11..c4d75c65289 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -194,6 +194,9 @@ def run_single_test(args, ext, server_logs_level, client_options, case_file, std os.system("LC_ALL=C sed -i -e 's/{test_db}/default/g' {file}".format(test_db=database, file=stdout_file)) if not args.show_db_name: os.system("LC_ALL=C sed -i -e 's/{test_db}/default/g' {file}".format(test_db=database, file=stderr_file)) + if args.replicated_database: + os.system("LC_ALL=C sed -i -e 's|/auto_{{shard}}||g' {file}".format(file=stdout_file)) + os.system("LC_ALL=C sed -i -e 's|auto_{{replica}}||g' {file}".format(file=stdout_file)) stdout = open(stdout_file, 'rb').read() if os.path.exists(stdout_file) else b'' stdout = str(stdout, errors='replace', encoding='utf-8') @@ -209,8 +212,12 @@ def need_retry(stderr): def get_processlist(args): try: + query = b"SHOW PROCESSLIST FORMAT Vertical" + if args.replicated_database: + query = b"SELECT materialize((hostName(), tcpPort())) as host, * " \ + b"FROM clusterAllReplicas('r', system.processes) WHERE query NOT LIKE '%system.processes%' FORMAT Vertical" clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - (stdout, _) = clickhouse_proc.communicate((b"SHOW PROCESSLIST FORMAT Vertical"), timeout=10) + (stdout, _) = clickhouse_proc.communicate(query, timeout=10) return False, stdout.decode('utf-8') except Exception as ex: print("Exception", ex) diff --git a/tests/config/install.sh b/tests/config/install.sh index 76cca9956cb..0fc590b7ab9 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -71,25 +71,25 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] # There is a bug in config reloading, so we cannot override macros using --macros.replica r2 # And we have to copy configs... + mkdir /etc/clickhouse-server1 mkdir /etc/clickhouse-server2 - mkdir /etc/clickhouse-server3 + chown clickhouse /etc/clickhouse-server1 chown clickhouse /etc/clickhouse-server2 - chown clickhouse /etc/clickhouse-server3 + chgrp clickhouse /etc/clickhouse-server1 chgrp clickhouse /etc/clickhouse-server2 - chgrp clickhouse /etc/clickhouse-server3 + sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server1 sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server2 - sudo -u clickhouse cp -r /etc/clickhouse-server/* /etc/clickhouse-server3 + rm /etc/clickhouse-server1/config.d/macros.xml rm /etc/clickhouse-server2/config.d/macros.xml - rm /etc/clickhouse-server3/config.d/macros.xml - sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|r1|r2|" > /etc/clickhouse-server2/config.d/macros.xml - sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|s1|s2|" > /etc/clickhouse-server3/config.d/macros.xml + sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|r1|r2|" > /etc/clickhouse-server1/config.d/macros.xml + sudo -u clickhouse cat /etc/clickhouse-server/config.d/macros.xml | sed "s|s1|s2|" > /etc/clickhouse-server2/config.d/macros.xml + sudo mkdir /var/lib/clickhouse1 sudo mkdir /var/lib/clickhouse2 - sudo mkdir /var/lib/clickhouse3 + sudo chown clickhouse /var/lib/clickhouse1 sudo chown clickhouse /var/lib/clickhouse2 - sudo chown clickhouse /var/lib/clickhouse3 + sudo chgrp clickhouse /var/lib/clickhouse1 sudo chgrp clickhouse /var/lib/clickhouse2 - sudo chgrp clickhouse /var/lib/clickhouse3 fi ln -sf $SRC_PATH/client_config.xml $DEST_CLIENT_PATH/config.xml diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql index 0fd760d73d5..163ebf1af23 100644 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql @@ -4,13 +4,13 @@ DROP TABLE IF EXISTS part_header_r2; SET replication_alter_partitions_sync = 2; CREATE TABLE part_header_r1(x UInt32, y UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header', '1') ORDER BY x + ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header/{shard}', '1{replica}') ORDER BY x SETTINGS use_minimalistic_part_header_in_zookeeper = 0, old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0; CREATE TABLE part_header_r2(x UInt32, y UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header', '2') ORDER BY x + ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header/{shard}', '2{replica}') ORDER BY x SETTINGS use_minimalistic_part_header_in_zookeeper = 1, old_parts_lifetime = 1, cleanup_delay_period = 0, @@ -39,10 +39,10 @@ SELECT sleep(3) FORMAT Null; SELECT '*** Test part removal ***'; SELECT '*** replica 1 ***'; SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r1'; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/replicas/1/parts'; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/s1/replicas/1r1/parts'; SELECT '*** replica 2 ***'; SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r2'; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/replicas/1/parts'; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/s1/replicas/1r1/parts'; SELECT '*** Test ALTER ***'; ALTER TABLE part_header_r1 MODIFY COLUMN y String; diff --git a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh index bbc2d957937..bf116689ce4 100755 --- a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh +++ b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh @@ -15,7 +15,7 @@ CREATE TABLE elog ( engine_id UInt32, referrer String ) -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00953/elog', 'test') +ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00953/elog/{shard}', '{replica}') PARTITION BY date ORDER BY (engine_id) SETTINGS replicated_deduplication_window = 2, cleanup_delay_period=4, cleanup_delay_period_random_add=0;" @@ -28,35 +28,35 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 3, 'h $CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # 3 rows -count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'") +count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'") while [[ $count != 2 ]] do sleep 1 - count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'") + count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'") done $CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 1, 'hello')" $CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # 4 rows -count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'") +count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'") while [[ $count != 2 ]] do sleep 1 - count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'") + count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'") done $CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 2, 'hello')" $CLICKHOUSE_CLIENT --query="SELECT count(*) from elog" # 5 rows -count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'") +count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'") while [[ $count != 2 ]] do sleep 1 - count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/blocks'") + count=$($CLICKHOUSE_CLIENT --query="SELECT COUNT(*) FROM system.zookeeper where path = '/clickhouse/tables/test_00953/elog/s1/blocks'") done $CLICKHOUSE_CLIENT --query="INSERT INTO elog VALUES (toDate('2018-10-01'), 2, 'hello')" diff --git a/tests/queries/0_stateless/01357_version_collapsing_attach_detach_zookeeper.sql b/tests/queries/0_stateless/01357_version_collapsing_attach_detach_zookeeper.sql index 0086ec5c2a3..d8249a603ff 100644 --- a/tests/queries/0_stateless/01357_version_collapsing_attach_detach_zookeeper.sql +++ b/tests/queries/0_stateless/01357_version_collapsing_attach_detach_zookeeper.sql @@ -8,13 +8,13 @@ CREATE TABLE versioned_collapsing_table( sign Int8, version UInt16 ) -ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/versioned_collapsing_table', '1', sign, version) +ENGINE = ReplicatedVersionedCollapsingMergeTree('/clickhouse/versioned_collapsing_table/{shard}', '{replica}', sign, version) PARTITION BY d ORDER BY (key1, key2); INSERT INTO versioned_collapsing_table VALUES (toDate('2019-10-10'), 1, 1, 'Hello', -1, 1); -SELECT value FROM system.zookeeper WHERE path = '/clickhouse/versioned_collapsing_table' and name = 'metadata'; +SELECT value FROM system.zookeeper WHERE path = '/clickhouse/versioned_collapsing_table/s1' and name = 'metadata'; SELECT COUNT() FROM versioned_collapsing_table; diff --git a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference index f79be33624b..1f7146dfe48 100644 --- a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference +++ b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference @@ -2,4 +2,4 @@ 10 10 24 -CREATE TABLE default.replicated_mutations_empty_partitions\n(\n `key` UInt64,\n `value` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01586_replicated_mutations_empty_partitions\', \'1\')\nPARTITION BY key\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.replicated_mutations_empty_partitions\n(\n `key` UInt64,\n `value` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01586_replicated_mutations_empty_partitions/{shard}\', \'{replica}\')\nPARTITION BY key\nORDER BY key\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql index 659cc060f32..63a3069c518 100644 --- a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql +++ b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql @@ -5,7 +5,7 @@ CREATE TABLE replicated_mutations_empty_partitions key UInt64, value String ) -ENGINE = ReplicatedMergeTree('/clickhouse/test/01586_replicated_mutations_empty_partitions', '1') +ENGINE = ReplicatedMergeTree('/clickhouse/test/01586_replicated_mutations_empty_partitions/{shard}', '{replica}') ORDER BY key PARTITION by key; @@ -13,7 +13,7 @@ INSERT INTO replicated_mutations_empty_partitions SELECT number, toString(number SELECT count(distinct value) FROM replicated_mutations_empty_partitions; -SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/block_numbers'; +SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/s1/block_numbers'; ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '3'; ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '4'; @@ -21,7 +21,7 @@ ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '5'; ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '9'; -- still ten records -SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/block_numbers'; +SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/s1/block_numbers'; ALTER TABLE replicated_mutations_empty_partitions MODIFY COLUMN value UInt64 SETTINGS replication_alter_partitions_sync=2; diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.reference b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.reference index cb1eace24a2..0d6ae8338a8 100644 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.reference +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.reference @@ -4,13 +4,13 @@ 499999500000 499999500000 Metadata version on replica 1 equal with first replica, OK -CREATE TABLE default.concurrent_kill_1\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill\', \'1\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_1\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill/{shard}\', \'{replica}1\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 2 equal with first replica, OK -CREATE TABLE default.concurrent_kill_2\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill\', \'2\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_2\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill/{shard}\', \'{replica}2\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 3 equal with first replica, OK -CREATE TABLE default.concurrent_kill_3\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill\', \'3\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_3\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill/{shard}\', \'{replica}3\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 4 equal with first replica, OK -CREATE TABLE default.concurrent_kill_4\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill\', \'4\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_4\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill/{shard}\', \'{replica}4\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 5 equal with first replica, OK -CREATE TABLE default.concurrent_kill_5\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill\', \'5\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_5\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test_01593_concurrent_kill/{shard}\', \'{replica}5\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 499999500000 diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.sh b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.sh index bfa68328c06..b2570cf97f3 100755 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.sh +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.sh @@ -11,7 +11,9 @@ for i in $(seq $REPLICAS); do done for i in $(seq $REPLICAS); do - $CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_kill_$i (key UInt64, value String) ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_01593_concurrent_kill', '$i') ORDER BY key SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000" + $CLICKHOUSE_CLIENT --query "CREATE TABLE concurrent_kill_$i (key UInt64, value String) ENGINE = + ReplicatedMergeTree('/clickhouse/tables/test_01593_concurrent_kill/{shard}', '{replica}$i') ORDER BY key + SETTINGS max_replicated_mutations_in_queue=1000, number_of_free_entries_in_pool_to_execute_mutation=0,max_replicated_merges_in_queue=1000" done $CLICKHOUSE_CLIENT --query "INSERT INTO concurrent_kill_1 SELECT number, toString(number) FROM numbers(1000000)" @@ -77,9 +79,9 @@ while true; do done -metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/test_01593_concurrent_kill/replicas/$i/' and name = 'metadata_version'") +metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/test_01593_concurrent_kill/s1/replicas/r1$i/' and name = 'metadata_version'") for i in $(seq $REPLICAS); do - replica_metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/test_01593_concurrent_kill/replicas/$i/' and name = 'metadata_version'") + replica_metadata_version=$($CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/test_01593_concurrent_kill/s1/replicas/r1$i/' and name = 'metadata_version'") if [ "$metadata_version" != "$replica_metadata_version" ]; then echo "Metadata version on replica $i differs from the first replica, FAIL" else diff --git a/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference b/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference index 2fc177c812e..e491dd9e091 100644 --- a/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference +++ b/tests/queries/0_stateless/01700_system_zookeeper_path_in.reference @@ -1,16 +1,14 @@ block_numbers blocks -1 +r1 ======== block_numbers blocks -1 +r1 ======== block_numbers blocks ======== -1 failed_parts last_part -leader_election-0000000000 parallel diff --git a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql index d4126098c7c..4eb1a73ef6b 100644 --- a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql +++ b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql @@ -3,17 +3,19 @@ DROP TABLE IF EXISTS sample_table; CREATE TABLE sample_table ( key UInt64 ) -ENGINE ReplicatedMergeTree('/clickhouse/01700_system_zookeeper_path_in', '1') +ENGINE ReplicatedMergeTree('/clickhouse/01700_system_zookeeper_path_in/{shard}', '{replica}') ORDER BY tuple(); -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in' AND name like 'block%' ORDER BY name; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/replicas' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/s1' AND name like 'block%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/s1/replicas' AND name LIKE '%r1%' ORDER BY name; SELECT '========'; -SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in') AND name LIKE 'block%' ORDER BY name; -SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/replicas') ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1') AND name LIKE 'block%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE '%r1%' ORDER BY name; SELECT '========'; -SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in','/clickhouse/01700_system_zookeeper_path_in/replicas') AND name LIKE 'block%' ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1', + '/clickhouse/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE 'block%' ORDER BY name; SELECT '========'; -SELECT name FROM system.zookeeper WHERE path IN (SELECT concat('/clickhouse/01700_system_zookeeper_path_in/', name) FROM system.zookeeper WHERE (path = '/clickhouse/01700_system_zookeeper_path_in')) ORDER BY name; +SELECT name FROM system.zookeeper WHERE path IN (SELECT concat('/clickhouse/01700_system_zookeeper_path_in/s1/', name) + FROM system.zookeeper WHERE (name != 'replicas' AND name NOT LIKE 'leader_election%' AND path = '/clickhouse/01700_system_zookeeper_path_in/s1')) ORDER BY name; DROP TABLE IF EXISTS sample_table; diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index b1e032e348d..f0fab6b42df 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -117,6 +117,14 @@ "01148_zookeeper_path_macros_unfolding", "01294_system_distributed_on_cluster", "01269_create_with_null", + "01451_replicated_detach_drop_and_quorum", + "01188_attach_table_from_path", + /// user_files + "01721_engine_file_truncate_on_insert", + /// Fails due to additional replicas or shards + "01650_drop_part_and_deduplication_zookeeper", + "01532_execute_merges_on_single_replica", + "01509_parallel_quorum_insert_no_replicas", /// grep -c "01018_ddl_dictionaries_bad_queries", "00908_bloom_filter_index", From 10696fed4dc9eb7524d04bc2cfc5cdbda8f6ff66 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Thu, 18 Mar 2021 17:21:39 +0300 Subject: [PATCH 19/60] Update 00953_zookeeper_suetin_deduplication_bug.sh --- .../0_stateless/00953_zookeeper_suetin_deduplication_bug.sh | 6 +----- 1 file changed, 1 insertion(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh index 98d0f5fad42..baa2b0cf53f 100755 --- a/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh +++ b/tests/queries/0_stateless/00953_zookeeper_suetin_deduplication_bug.sh @@ -15,11 +15,7 @@ CREATE TABLE elog ( engine_id UInt32, referrer String ) -<<<<<<< HEAD -ENGINE = ReplicatedMergeTree('/clickhouse/tables/test_00953/elog/{shard}', '{replica}') -======= -ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog', 'test') ->>>>>>> master +ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/elog/{shard}', '{replica}') PARTITION BY date ORDER BY (engine_id) SETTINGS replicated_deduplication_window = 2, cleanup_delay_period=4, cleanup_delay_period_random_add=0;" From ce31d4608b9a633567a1ec949e84bbacca3a0364 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Thu, 18 Mar 2021 22:13:21 +0300 Subject: [PATCH 20/60] fix tests --- src/Databases/DatabaseReplicatedWorker.cpp | 2 +- ...01396_inactive_replica_cleanup_nodes_zookeeper.sh | 12 ++++++------ tests/queries/skip_list.json | 3 ++- 3 files changed, 9 insertions(+), 8 deletions(-) diff --git a/src/Databases/DatabaseReplicatedWorker.cpp b/src/Databases/DatabaseReplicatedWorker.cpp index b69e76697b0..63f4d89b216 100644 --- a/src/Databases/DatabaseReplicatedWorker.cpp +++ b/src/Databases/DatabaseReplicatedWorker.cpp @@ -125,7 +125,7 @@ String DatabaseReplicatedDDLWorker::tryEnqueueAndExecuteEntry(DDLLogEntry & entr }); if (!processed) - throw Exception(ErrorCodes::UNFINISHED, "Timeout: Cannot enqueue query on this replica," + throw Exception(ErrorCodes::UNFINISHED, "Timeout: Cannot enqueue query on this replica, " "most likely because replica is busy with previous queue entries"); } diff --git a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh index b604ace85cc..693580bc270 100755 --- a/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh +++ b/tests/queries/0_stateless/01396_inactive_replica_cleanup_nodes_zookeeper.sh @@ -12,8 +12,8 @@ SCALE=5000 $CLICKHOUSE_CLIENT -n --query " DROP TABLE IF EXISTS r1; DROP TABLE IF EXISTS r2; - CREATE TABLE r1 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', '1') ORDER BY x SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 1, parts_to_throw_insert = 100000, max_replicated_logs_to_keep = 10; - CREATE TABLE r2 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r', '2') ORDER BY x SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 1, parts_to_throw_insert = 100000, max_replicated_logs_to_keep = 10; + CREATE TABLE r1 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{shard}', '1{replica}') ORDER BY x SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 1, parts_to_throw_insert = 100000, max_replicated_logs_to_keep = 10; + CREATE TABLE r2 (x UInt64) ENGINE = ReplicatedMergeTree('/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/{shard}', '2{replica}') ORDER BY x SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 1, parts_to_throw_insert = 100000, max_replicated_logs_to_keep = 10; DETACH TABLE r2; " @@ -29,16 +29,16 @@ for _ in {1..60}; do done -$CLICKHOUSE_CLIENT --query "SELECT numChildren < $((SCALE / 4)) FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r' AND name = 'log'"; +$CLICKHOUSE_CLIENT --query "SELECT numChildren < $((SCALE / 4)) FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1' AND name = 'log'"; echo -e '\n---\n'; -$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r/replicas/1' AND name = 'is_lost'"; -$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r/replicas/2' AND name = 'is_lost'"; +$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/1r1' AND name = 'is_lost'"; +$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/2r1' AND name = 'is_lost'"; echo -e '\n---\n'; $CLICKHOUSE_CLIENT --query "ATTACH TABLE r2" $CLICKHOUSE_CLIENT --receive_timeout 600 --query "SYSTEM SYNC REPLICA r2" # Need to increase timeout, otherwise it timed out in debug build -$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/r/replicas/2' AND name = 'is_lost'"; +$CLICKHOUSE_CLIENT --query "SELECT value FROM system.zookeeper WHERE path = '/clickhouse/tables/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/s1/replicas/2r1' AND name = 'is_lost'"; $CLICKHOUSE_CLIENT -n --query " DROP TABLE IF EXISTS r1; diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 66f10021359..28d6d2da939 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -119,12 +119,13 @@ "01269_create_with_null", "01451_replicated_detach_drop_and_quorum", "01188_attach_table_from_path", + "01149_zookeeper_mutation_stuck_after_replace_partition", /// user_files "01721_engine_file_truncate_on_insert", /// Fails due to additional replicas or shards + "quorum", "01650_drop_part_and_deduplication_zookeeper", "01532_execute_merges_on_single_replica", - "01509_parallel_quorum_insert_no_replicas", /// grep -c "01018_ddl_dictionaries_bad_queries", "00908_bloom_filter_index", From 60ef7147bcc17e6715f87b8051556b16e04e8daa Mon Sep 17 00:00:00 2001 From: tavplubix Date: Fri, 19 Mar 2021 07:24:43 +0300 Subject: [PATCH 21/60] trigger CI --- tests/queries/0_stateless/01700_system_zookeeper_path_in.sql | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql index 4eb1a73ef6b..02457a956a1 100644 --- a/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql +++ b/tests/queries/0_stateless/01700_system_zookeeper_path_in.sql @@ -8,6 +8,7 @@ ORDER BY tuple(); SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/s1' AND name like 'block%' ORDER BY name; SELECT name FROM system.zookeeper WHERE path = '/clickhouse/01700_system_zookeeper_path_in/s1/replicas' AND name LIKE '%r1%' ORDER BY name; + SELECT '========'; SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1') AND name LIKE 'block%' ORDER BY name; SELECT name FROM system.zookeeper WHERE path IN ('/clickhouse/01700_system_zookeeper_path_in/s1/replicas') AND name LIKE '%r1%' ORDER BY name; From d03b1312eb2e559d996973163cfa994881118f7a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 22 Mar 2021 17:31:43 +0300 Subject: [PATCH 22/60] update skip_list --- tests/queries/skip_list.json | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/skip_list.json b/tests/queries/skip_list.json index 28d6d2da939..5d11dc871d6 100644 --- a/tests/queries/skip_list.json +++ b/tests/queries/skip_list.json @@ -126,6 +126,8 @@ "quorum", "01650_drop_part_and_deduplication_zookeeper", "01532_execute_merges_on_single_replica", + "00652_replicated_mutations_default_database_zookeeper", + "00620_optimize_on_nonleader_replica_zookeeper", /// grep -c "01018_ddl_dictionaries_bad_queries", "00908_bloom_filter_index", From e378c0bf8a805e904e6f8f931cc311ad79c12c4b Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Fri, 19 Mar 2021 16:20:19 +0300 Subject: [PATCH 23/60] Disk S3 possibility to migrate to restorable schema --- src/Disks/S3/DiskS3.cpp | 149 ++++++++++++++++-- src/Disks/S3/DiskS3.h | 23 ++- .../config.d/storage_conf_not_restorable.xml | 35 ++++ .../test_merge_tree_s3_restore/test.py | 58 ++++++- 4 files changed, 250 insertions(+), 15 deletions(-) create mode 100644 tests/integration/test_merge_tree_s3_restore/configs/config.d/storage_conf_not_restorable.xml diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index bb9966eb6ff..bb15df9b9e3 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -648,7 +648,7 @@ void DiskS3::moveFile(const String & from_path, const String & to_path) if (send_metadata) { auto revision = ++revision_counter; - const DiskS3::ObjectMetadata object_metadata { + const ObjectMetadata object_metadata { {"from_path", from_path}, {"to_path", to_path} }; @@ -942,7 +942,16 @@ void DiskS3::startup() LOG_INFO(&Poco::Logger::get("DiskS3"), "Starting up disk {}", name); - /// Find last revision. + if (readSchemaVersion(bucket, s3_root_path) < RESTORABLE_SCHEMA_VERSION) + migrateToRestorableSchema(); + + findLastRevision(); + + LOG_INFO(&Poco::Logger::get("DiskS3"), "Disk {} started up", name); +} + +void DiskS3::findLastRevision() +{ UInt64 l = 0, r = LATEST_REVISION; while (l < r) { @@ -954,8 +963,8 @@ void DiskS3::startup() LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Check object with revision {}", revision); /// Check file or operation with such revision exists. - if (checkObjectExists(s3_root_path + "r" + revision_str) - || checkObjectExists(s3_root_path + "operations/r" + revision_str)) + if (checkObjectExists(bucket, s3_root_path + "r" + revision_str) + || checkObjectExists(bucket, s3_root_path + "operations/r" + revision_str)) l = revision; else r = revision - 1; @@ -964,10 +973,124 @@ void DiskS3::startup() LOG_INFO(&Poco::Logger::get("DiskS3"), "Found last revision number {} for disk {}", revision_counter, name); } -bool DiskS3::checkObjectExists(const String & prefix) +int DiskS3::readSchemaVersion(const String & source_bucket, const String & source_path) +{ + int version = 0; + if (!checkObjectExists(source_bucket, source_path + SCHEMA_VERSION_OBJECT)) + return version; + + ReadBufferFromS3 buffer (client, source_bucket, source_path + SCHEMA_VERSION_OBJECT); + readIntText(version, buffer); + + return version; +} + +void DiskS3::saveSchemaVersion(const int & version) +{ + WriteBufferFromS3 buffer (client, bucket, s3_root_path + SCHEMA_VERSION_OBJECT, min_upload_part_size, max_single_part_upload_size); + writeIntText(version, buffer); + buffer.finalize(); +} + +void DiskS3::updateObjectMetadata(const String & key, const ObjectMetadata & metadata) +{ + Aws::S3::Model::CopyObjectRequest request; + request.SetCopySource(bucket + "/" + key); + request.SetBucket(bucket); + request.SetKey(key); + request.SetMetadata(metadata); + + auto outcome = client->CopyObject(request); + throwIfError(outcome); +} + +void DiskS3::migrateFileToRestorableSchema(const String & path) +{ + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Migrate file {} to restorable schema", metadata_path + path); + + auto meta = readMeta(path); + + for (const auto & [key, _] : meta.s3_objects) + { + ObjectMetadata metadata { + {"path", path} + }; + updateObjectMetadata(s3_root_path + key, metadata); + } +} + +void DiskS3::migrateToRestorableSchemaRecursive(const String & path, Futures & results) +{ + checkStackSize(); /// This is needed to prevent stack overflow in case of cyclic symlinks. + + LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Migrate directory {} to restorable schema", metadata_path + path); + + bool dir_contains_only_files = true; + for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + if (isDirectory(it->path())) + { + dir_contains_only_files = false; + break; + } + + /// The whole directory can be migrated asynchronously. + if (dir_contains_only_files) + { + auto result = getExecutor().execute([this, path] + { + for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + migrateFileToRestorableSchema(it->path()); + }); + + results.push_back(std::move(result)); + } + else + { + for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + if (!isDirectory(it->path())) + { + auto source_path = it->path(); + auto result = getExecutor().execute([this, source_path] + { + migrateFileToRestorableSchema(source_path); + }); + + results.push_back(std::move(result)); + } + else + migrateToRestorableSchemaRecursive(it->path(), results); + } +} + +void DiskS3::migrateToRestorableSchema() +{ + try + { + LOG_INFO(&Poco::Logger::get("DiskS3"), "Start migration to restorable schema for disk {}", name); + + Futures results; + + migrateToRestorableSchemaRecursive("data/", results); + + for (auto & result : results) + result.wait(); + for (auto & result : results) + result.get(); + + saveSchemaVersion(RESTORABLE_SCHEMA_VERSION); + } + catch (const Exception & e) + { + LOG_ERROR(&Poco::Logger::get("DiskS3"), "Failed to migrate to restorable schema. Code: {}, e.displayText() = {}, Stack trace:\n\n{}", e.code(), e.displayText(), e.getStackTraceString()); + + throw; + } +} + +bool DiskS3::checkObjectExists(const String & source_bucket, const String & prefix) { Aws::S3::Model::ListObjectsV2Request request; - request.SetBucket(bucket); + request.SetBucket(source_bucket); request.SetPrefix(prefix); request.SetMaxKeys(1); @@ -1048,7 +1171,7 @@ struct DiskS3::RestoreInformation void DiskS3::readRestoreInformation(DiskS3::RestoreInformation & restore_information) { - ReadBufferFromFile buffer(metadata_path + restore_file_name, 512); + ReadBufferFromFile buffer(metadata_path + RESTORE_FILE_NAME, 512); buffer.next(); /// Empty file - just restore all metadata. @@ -1083,7 +1206,7 @@ void DiskS3::readRestoreInformation(DiskS3::RestoreInformation & restore_informa void DiskS3::restore() { - if (!exists(restore_file_name)) + if (!exists(RESTORE_FILE_NAME)) return; try @@ -1110,15 +1233,21 @@ void DiskS3::restore() throw Exception("Restoring to the same bucket is allowed only if source path is not a sub-path of configured path in S3 disk", ErrorCodes::BAD_ARGUMENTS); } - ///TODO: Cleanup FS and bucket if previous restore was failed. + if (readSchemaVersion(information.source_bucket, information.source_path) < RESTORABLE_SCHEMA_VERSION) + throw Exception("Source bucket doesn't have restorable schema.", ErrorCodes::BAD_ARGUMENTS); LOG_INFO(&Poco::Logger::get("DiskS3"), "Starting to restore disk {}. Revision: {}, Source bucket: {}, Source path: {}", name, information.revision, information.source_bucket, information.source_path); + LOG_INFO(&Poco::Logger::get("DiskS3"), "Removing old metadata..."); + + bool cleanup_s3 = information.source_bucket != bucket || information.source_path != s3_root_path; + removeSharedRecursive("data/", !cleanup_s3); + restoreFiles(information.source_bucket, information.source_path, information.revision); restoreFileOperations(information.source_bucket, information.source_path, information.revision); - Poco::File restore_file(metadata_path + restore_file_name); + Poco::File restore_file(metadata_path + RESTORE_FILE_NAME); restore_file.remove(); LOG_INFO(&Poco::Logger::get("DiskS3"), "Restore disk {} finished", name); diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 5d9effa16fa..4a5f93866bd 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -25,6 +25,7 @@ class DiskS3 : public IDisk { public: using ObjectMetadata = std::map; + using Futures = std::vector>; friend class DiskS3Reservation; @@ -148,7 +149,16 @@ private: void createFileOperationObject(const String & operation_name, UInt64 revision, const ObjectMetadata & metadata); static String revisionToString(UInt64 revision); - bool checkObjectExists(const String & prefix); + bool checkObjectExists(const String & source_bucket, const String & prefix); + void findLastRevision(); + + int readSchemaVersion(const String & source_bucket, const String & source_path); + void saveSchemaVersion(const int & version); + void updateObjectMetadata(const String & key, const ObjectMetadata & metadata); + void migrateFileToRestorableSchema(const String & path); + void migrateToRestorableSchemaRecursive(const String & path, Futures & results); + void migrateToRestorableSchema(); + Aws::S3::Model::HeadObjectResult headObject(const String & source_bucket, const String & key); void listObjects(const String & source_bucket, const String & source_path, std::function callback); void copyObject(const String & src_bucket, const String & src_key, const String & dst_bucket, const String & dst_key); @@ -168,7 +178,7 @@ private: std::shared_ptr proxy_configuration; const String bucket; const String s3_root_path; - const String metadata_path; + String metadata_path; size_t min_upload_part_size; size_t max_single_part_upload_size; size_t min_bytes_for_seek; @@ -179,16 +189,21 @@ private: std::mutex reservation_mutex; std::atomic revision_counter; - static constexpr UInt64 LATEST_REVISION = (static_cast(1)) << 63; + static constexpr UInt64 LATEST_REVISION = std::numeric_limits::max(); static constexpr UInt64 UNKNOWN_REVISION = 0; /// File at path {metadata_path}/restore contains metadata restore information - const String restore_file_name = "restore"; + inline static const String RESTORE_FILE_NAME = "restore"; /// The number of keys listed in one request (1000 is max value) int list_object_keys_size; /// Key has format: ../../r{revision}-{operation} const re2::RE2 key_regexp {".*/r(\\d+)-(\\w+).*"}; + + /// Object contains information about schema version. + inline static const String SCHEMA_VERSION_OBJECT = ".SCHEMA_VERSION"; + /// Version with possibility to backup-restore metadata. + static constexpr int RESTORABLE_SCHEMA_VERSION = 1; }; } diff --git a/tests/integration/test_merge_tree_s3_restore/configs/config.d/storage_conf_not_restorable.xml b/tests/integration/test_merge_tree_s3_restore/configs/config.d/storage_conf_not_restorable.xml new file mode 100644 index 00000000000..c682ddae785 --- /dev/null +++ b/tests/integration/test_merge_tree_s3_restore/configs/config.d/storage_conf_not_restorable.xml @@ -0,0 +1,35 @@ + + + + + s3 + http://minio1:9001/root/another_data/ + minio + minio123 + false + 1 + 0 + + + local + / + + + + + +
+ s3 +
+ + hdd + +
+
+
+
+ + + 0 + +
diff --git a/tests/integration/test_merge_tree_s3_restore/test.py b/tests/integration/test_merge_tree_s3_restore/test.py index 346d9aced3f..0d08e2686b9 100644 --- a/tests/integration/test_merge_tree_s3_restore/test.py +++ b/tests/integration/test_merge_tree_s3_restore/test.py @@ -1,3 +1,4 @@ +import os import logging import random import string @@ -10,6 +11,20 @@ logging.getLogger().setLevel(logging.INFO) logging.getLogger().addHandler(logging.StreamHandler()) +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +CONFIG_PATH = os.path.join(SCRIPT_DIR, './_instances/node_not_restorable/configs/config.d/storage_conf_not_restorable.xml') + + +def replace_config(old, new): + config = open(CONFIG_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.writelines(config_lines) + config.close() + + @pytest.fixture(scope="module") def cluster(): try: @@ -26,6 +41,10 @@ def cluster(): "configs/config.d/storage_conf_another_bucket_path.xml", "configs/config.d/bg_processing_pool_conf.xml", "configs/config.d/log_conf.xml"], user_configs=[], stay_alive=True) + cluster.add_instance("node_not_restorable", main_configs=[ + "configs/config.d/storage_conf_not_restorable.xml", + "configs/config.d/bg_processing_pool_conf.xml", + "configs/config.d/log_conf.xml"], user_configs=[], stay_alive=True) logging.info("Starting cluster...") cluster.start() logging.info("Cluster started") @@ -103,7 +122,7 @@ def get_revision_counter(node, backup_number): def drop_table(cluster): yield - node_names = ["node", "node_another_bucket", "node_another_bucket_path"] + node_names = ["node", "node_another_bucket", "node_another_bucket_path", "node_not_restorable"] for node_name in node_names: node = cluster.instances[node_name] @@ -311,3 +330,40 @@ def test_restore_mutations(cluster): assert node_another_bucket.query("SELECT sum(id) FROM s3.test FORMAT Values") == "({})".format(0) assert node_another_bucket.query("SELECT sum(counter) FROM s3.test FORMAT Values") == "({})".format(4096 * 2) assert node_another_bucket.query("SELECT sum(counter) FROM s3.test WHERE id > 0 FORMAT Values") == "({})".format(4096) + + +def test_migrate_to_restorable_schema(cluster): + node = cluster.instances["node_not_restorable"] + + create_table(node, "test") + + node.query("INSERT INTO s3.test VALUES {}".format(generate_values('2020-01-03', 4096))) + node.query("INSERT INTO s3.test VALUES {}".format(generate_values('2020-01-04', 4096, -1))) + node.query("INSERT INTO s3.test VALUES {}".format(generate_values('2020-01-05', 4096))) + node.query("INSERT INTO s3.test VALUES {}".format(generate_values('2020-01-05', 4096, -1))) + + replace_config("false", "true") + + node.restart_clickhouse() + + node.query("INSERT INTO s3.test VALUES {}".format(generate_values('2020-01-06', 4096))) + node.query("INSERT INTO s3.test VALUES {}".format(generate_values('2020-01-06', 4096, -1))) + + node.query("ALTER TABLE s3.test FREEZE") + revision = get_revision_counter(node, 1) + + assert revision != 0 + + node_another_bucket = cluster.instances["node_another_bucket"] + + create_table(node_another_bucket, "test") + + # Restore to revision before mutation. + node_another_bucket.stop_clickhouse() + drop_s3_metadata(node_another_bucket) + purge_s3(cluster, cluster.minio_bucket_2) + create_restore_file(node_another_bucket, revision=revision, bucket="root", path="another_data") + node_another_bucket.start_clickhouse(10) + + assert node_another_bucket.query("SELECT count(*) FROM s3.test FORMAT Values") == "({})".format(4096 * 6) + assert node_another_bucket.query("SELECT sum(id) FROM s3.test FORMAT Values") == "({})".format(0) From 7e462a77a7f4cf1eb4a3dcd9377287dcc510fa39 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 24 Mar 2021 13:05:43 +0300 Subject: [PATCH 24/60] Disk S3 possibility to migrate to restorable schema --- src/Disks/S3/DiskS3.cpp | 21 ++++++++++++++++--- .../test_merge_tree_s3_restore/test.py | 2 ++ 2 files changed, 20 insertions(+), 3 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index bb15df9b9e3..6adae24d15d 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -966,6 +966,8 @@ void DiskS3::findLastRevision() if (checkObjectExists(bucket, s3_root_path + "r" + revision_str) || checkObjectExists(bucket, s3_root_path + "operations/r" + revision_str)) l = revision; + else if (revision == 0) + r = 0; else r = revision - 1; } @@ -999,6 +1001,7 @@ void DiskS3::updateObjectMetadata(const String & key, const ObjectMetadata & met request.SetBucket(bucket); request.SetKey(key); request.SetMetadata(metadata); + request.SetMetadataDirective(Aws::S3::Model::MetadataDirective::REPLACE); auto outcome = client->CopyObject(request); throwIfError(outcome); @@ -1070,7 +1073,10 @@ void DiskS3::migrateToRestorableSchema() Futures results; - migrateToRestorableSchemaRecursive("data/", results); + if (exists("data")) + migrateToRestorableSchemaRecursive("data/", results); + if (exists("store")) + migrateToRestorableSchemaRecursive("store/", results); for (auto & result : results) result.wait(); @@ -1242,7 +1248,10 @@ void DiskS3::restore() LOG_INFO(&Poco::Logger::get("DiskS3"), "Removing old metadata..."); bool cleanup_s3 = information.source_bucket != bucket || information.source_path != s3_root_path; - removeSharedRecursive("data/", !cleanup_s3); + if (exists("data")) + removeSharedRecursive("data/", !cleanup_s3); + if (exists("store")) + removeSharedRecursive("data/", !cleanup_s3); restoreFiles(information.source_bucket, information.source_path, information.revision); restoreFileOperations(information.source_bucket, information.source_path, information.revision); @@ -1250,6 +1259,8 @@ void DiskS3::restore() Poco::File restore_file(metadata_path + RESTORE_FILE_NAME); restore_file.remove(); + saveSchemaVersion(RESTORABLE_SCHEMA_VERSION); + LOG_INFO(&Poco::Logger::get("DiskS3"), "Restore disk {} finished", name); } catch (const Exception & e) @@ -1315,7 +1326,11 @@ void DiskS3::processRestoreFiles(const String & source_bucket, const String & so /// Restore file if object has 'path' in metadata. auto path_entry = object_metadata.find("path"); if (path_entry == object_metadata.end()) - throw Exception("Failed to restore key " + key + " because it doesn't have 'path' in metadata", ErrorCodes::S3_ERROR); + { + /// Such keys can remain after migration, we can skip them. + LOG_WARNING(&Poco::Logger::get("DiskS3"), "Skip key {} because it doesn't have 'path' in metadata", key); + continue; + } const auto & path = path_entry->second; diff --git a/tests/integration/test_merge_tree_s3_restore/test.py b/tests/integration/test_merge_tree_s3_restore/test.py index 0d08e2686b9..c0ebce68480 100644 --- a/tests/integration/test_merge_tree_s3_restore/test.py +++ b/tests/integration/test_merge_tree_s3_restore/test.py @@ -94,6 +94,8 @@ def create_table(node, table_name, additional_settings=None): def purge_s3(cluster, bucket): minio = cluster.minio_client for obj in list(minio.list_objects(bucket, recursive=True)): + if str(obj.object_name).find(".SCHEMA_VERSION") != -1: + continue minio.remove_object(bucket, obj.object_name) From 9f9419cf144643750463a2113edac46fc51f0a44 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 24 Mar 2021 13:08:37 +0300 Subject: [PATCH 25/60] Disk S3 possibility to migrate to restorable schema --- src/Disks/S3/DiskS3.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 6adae24d15d..94804985762 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -958,6 +958,9 @@ void DiskS3::findLastRevision() LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Check revision in bounds {}-{}", l, r); auto revision = l + (r - l + 1) / 2; + if (revision == 0) + break; + auto revision_str = revisionToString(revision); LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Check object with revision {}", revision); @@ -966,8 +969,6 @@ void DiskS3::findLastRevision() if (checkObjectExists(bucket, s3_root_path + "r" + revision_str) || checkObjectExists(bucket, s3_root_path + "operations/r" + revision_str)) l = revision; - else if (revision == 0) - r = 0; else r = revision - 1; } From 0c0ddefc75c69768d482242f47982226044001f9 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Wed, 24 Mar 2021 13:12:47 +0300 Subject: [PATCH 26/60] Disk S3 possibility to migrate to restorable schema --- src/Disks/S3/DiskS3.cpp | 14 ++++++-------- src/Disks/S3/DiskS3.h | 2 ++ 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 94804985762..74c859efb75 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -1074,10 +1074,9 @@ void DiskS3::migrateToRestorableSchema() Futures results; - if (exists("data")) - migrateToRestorableSchemaRecursive("data/", results); - if (exists("store")) - migrateToRestorableSchemaRecursive("store/", results); + for (const auto & root : data_roots) + if (exists(root)) + migrateToRestorableSchemaRecursive(root + '/', results); for (auto & result : results) result.wait(); @@ -1249,10 +1248,9 @@ void DiskS3::restore() LOG_INFO(&Poco::Logger::get("DiskS3"), "Removing old metadata..."); bool cleanup_s3 = information.source_bucket != bucket || information.source_path != s3_root_path; - if (exists("data")) - removeSharedRecursive("data/", !cleanup_s3); - if (exists("store")) - removeSharedRecursive("data/", !cleanup_s3); + for (const auto & root : data_roots) + if (exists(root)) + removeSharedRecursive(root + '/', !cleanup_s3); restoreFiles(information.source_bucket, information.source_path, information.revision); restoreFileOperations(information.source_bucket, information.source_path, information.revision); diff --git a/src/Disks/S3/DiskS3.h b/src/Disks/S3/DiskS3.h index 4a5f93866bd..5624fd2eee1 100644 --- a/src/Disks/S3/DiskS3.h +++ b/src/Disks/S3/DiskS3.h @@ -204,6 +204,8 @@ private: inline static const String SCHEMA_VERSION_OBJECT = ".SCHEMA_VERSION"; /// Version with possibility to backup-restore metadata. static constexpr int RESTORABLE_SCHEMA_VERSION = 1; + /// Directories with data. + const std::vector data_roots {"data", "store"}; }; } From 9db74c4dc0194ccad5e6cbe0c7bce1baff5a1ae5 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 20 Mar 2021 22:40:24 +0300 Subject: [PATCH 27/60] Add stress test for distributed queries It may founds issue like in [1]: 2021.03.18 19:05:38.783328 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} executeQuery: (from 127.0.0.1:40918, using production parser) select * from dist where key = 0; 2021.03.18 19:05:38.783760 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} StorageDistributed (dist): Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): [1] 2021.03.18 19:05:38.784012 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} ContextAccess (default): Access granted: SELECT(key) ON default.dist 2021.03.18 19:05:38.784410 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} ContextAccess (default): Access granted: SELECT(key) ON default.dist 2021.03.18 19:05:38.784488 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} StorageDistributed (dist): Disabling force_optimize_skip_unused_shards for nested queries (force_optimize_skip_unused_shards_nesting exceeded) 2021.03.18 19:05:38.784572 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} InterpreterSelectQuery: Complete -> Complete 2021.03.18 19:05:38.819063 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} executeQuery: Read 20 rows, 80.00 B in 0.035687783 sec., 560 rows/sec., 2.19 KiB/sec. 2021.03.18 19:05:38.827842 [ 245 ] {4b1f5ec0-bf2d-478c-a2e1-d312531db206} MemoryTracker: Peak memory usage (for query): 0.00 B. 2021.03.18 19:05:38.867752 [ 547 ] {} BaseDaemon: ######################################## 2021.03.18 19:05:38.867959 [ 547 ] {} BaseDaemon: (version 21.4.1.1, build id: A0ADEC175BD65E58EA012C47C265E661C32D23B5) (from thread 245) (query_id: 4b1f5ec0-bf2d-478c-a2e1-d312531db206) Received signal Aborted (6) 2021.03.18 19:05:38.868733 [ 547 ] {} BaseDaemon: 2021.03.18 19:05:38.868958 [ 547 ] {} BaseDaemon: Stack trace: 0x7fd1394be18b 0x7fd13949d859 0x10c4c99b 0xd434ee1 0xd434f1a 2021.03.18 19:05:38.870135 [ 547 ] {} BaseDaemon: 3. gsignal @ 0x4618b in /usr/lib/x86_64-linux-gnu/libc-2.31.so 2021.03.18 19:05:38.870383 [ 547 ] {} BaseDaemon: 4. abort @ 0x25859 in /usr/lib/x86_64-linux-gnu/libc-2.31.so 2021.03.18 19:05:38.886783 [ 547 ] {} BaseDaemon: 5. /work3/azat/ch/clickhouse/.cmake/../contrib/libunwind/src/UnwindLevel1.c:396: _Unwind_Resume @ 0x10c4c99b in /usr/bin/clickhouse 2021.03.18 19:05:47.200208 [ 547 ] {} BaseDaemon: 6. ? @ 0xd434ee1 in /usr/bin/clickhouse 2021.03.18 19:05:47.348738 [ 547 ] {} BaseDaemon: 7.1. inlined from /work3/azat/ch/clickhouse/.cmake/../contrib/boost/boost/context/fiber_fcontext.hpp:253: boost::context::fiber::~fiber() 2021.03.18 19:05:47.349118 [ 547 ] {} BaseDaemon: 7.2. inlined from ../contrib/boost/boost/context/fiber_fcontext.hpp:252: boost::context::detail::fiber_record::run(void*) 2021.03.18 19:05:47.349163 [ 547 ] {} BaseDaemon: 7. ../contrib/boost/boost/context/fiber_fcontext.hpp:80: void boost::context::detail::fiber_entry >(boost::context::detail::transfer_t) @ 0xd434f1a in /usr/bin/clickhouse 2021.03.18 19:05:47.618174 [ 547 ] {} BaseDaemon: Calculated checksum of the binary: FF3BA83D0CD648741EEEC242CB1966D9. There is no information about the reference checksum. [1]: https://clickhouse-test-reports.s3.yandex.net/0/1b2ed51ff5e4a3dc45567d4967108f43f680c884/stress_test_(debug).html#fail1 --- .../__init__.py | 0 .../configs/remote_servers.xml | 42 +++++++ .../test_distributed_queries_stress/test.py | 103 ++++++++++++++++++ 3 files changed, 145 insertions(+) create mode 100644 tests/integration/test_distributed_queries_stress/__init__.py create mode 100644 tests/integration/test_distributed_queries_stress/configs/remote_servers.xml create mode 100644 tests/integration/test_distributed_queries_stress/test.py diff --git a/tests/integration/test_distributed_queries_stress/__init__.py b/tests/integration/test_distributed_queries_stress/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_distributed_queries_stress/configs/remote_servers.xml b/tests/integration/test_distributed_queries_stress/configs/remote_servers.xml new file mode 100644 index 00000000000..7d00cebccfc --- /dev/null +++ b/tests/integration/test_distributed_queries_stress/configs/remote_servers.xml @@ -0,0 +1,42 @@ + + 1000 + + + + + true + + node1_r1 + 9000 + + + node1_r2 + 9000 + + + + + + true + + node1_r1 + 9000 + + + node1_r2 + 9000 + + + + + node2_r1 + 9000 + + + node2_r2 + 9000 + + + + + diff --git a/tests/integration/test_distributed_queries_stress/test.py b/tests/integration/test_distributed_queries_stress/test.py new file mode 100644 index 00000000000..dcc4943f7e6 --- /dev/null +++ b/tests/integration/test_distributed_queries_stress/test.py @@ -0,0 +1,103 @@ +# pylint: disable=redefined-outer-name +# pylint: disable=unused-argument +# pylint: disable=line-too-long + +import shlex +import itertools +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node1_r1 = cluster.add_instance('node1_r1', main_configs=['configs/remote_servers.xml']) +node2_r1 = cluster.add_instance('node2_r1', main_configs=['configs/remote_servers.xml']) +node1_r2 = cluster.add_instance('node1_r2', main_configs=['configs/remote_servers.xml']) +node2_r2 = cluster.add_instance('node2_r2', main_configs=['configs/remote_servers.xml']) + +def run_benchmark(payload, settings): + node1_r1.exec_in_container([ + 'bash', '-c', 'echo {} | '.format(shlex.quote(payload.strip())) + ' '.join([ + 'clickhouse', 'benchmark', + '--concurrency=100', + '--cumulative', + '--delay=0', + # NOTE: with current matrix even 3 seconds it huge... + '--timelimit=3', + # tune some basic timeouts + '--hedged_connection_timeout_ms=200', + '--connect_timeout_with_failover_ms=200', + '--connections_with_failover_max_tries=5', + *settings, + ]) + ]) + +@pytest.fixture(scope='module') +def started_cluster(): + try: + cluster.start() + + for _, instance in cluster.instances.items(): + instance.query(""" + create table if not exists data ( + key Int, + /* just to increase block size */ + v1 UInt64, + v2 UInt64, + v3 UInt64, + v4 UInt64, + v5 UInt64, + v6 UInt64, + v7 UInt64, + v8 UInt64, + v9 UInt64, + v10 UInt64, + v11 UInt64, + v12 UInt64 + ) Engine=MergeTree() order by key partition by key%5; + insert into data (key) select * from numbers(10); + + create table if not exists dist_one as data engine=Distributed(one_shard, currentDatabase(), data, key); + create table if not exists dist_one_over_dist as data engine=Distributed(one_shard, currentDatabase(), dist_one, yandexConsistentHash(key, 2)); + + create table if not exists dist_two as data engine=Distributed(two_shards, currentDatabase(), data, key); + create table if not exists dist_two_over_dist as data engine=Distributed(two_shards, currentDatabase(), dist_two, yandexConsistentHash(key, 2)); + """) + yield cluster + finally: + cluster.shutdown() + +# since it includes started_cluster fixture at first start +@pytest.mark.timeout(60) +@pytest.mark.parametrize('table,settings', itertools.product( + [ # tables + 'dist_one', + 'dist_one_over_dist', + 'dist_two', + 'dist_two_over_dist', + ], + [ # settings + *list(itertools.combinations([ + '', # defaults + '--prefer_localhost_replica=0', + '--async_socket_for_remote=0', + '--use_hedged_requests=0', + '--optimize_skip_unused_shards=1', + '--distributed_group_by_no_merge=2', + '--optimize_distributed_group_by_sharding_key=1', + + # TODO: enlarge test matrix (but first those values to accept ms): + # + # - sleep_in_send_tables_status + # - sleep_in_send_data + ], 2)) + # TODO: more combinations that just 2 + ], +)) +def test_stress_distributed(table, settings, started_cluster): + payload = f''' + select * from {table} where key = 0; + select * from {table} where key = 1; + select * from {table} where key = 2; + select * from {table} where key = 3; + select * from {table}; + ''' + run_benchmark(payload, settings) From db505553a496b7d51ce73a8b078e9062866d7f3a Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 26 Mar 2021 10:09:51 +0300 Subject: [PATCH 28/60] Reads file as String - description of the new function. --- docs/en/sql-reference/functions/files.md | 35 ++++++++++++++++++++++++ 1 file changed, 35 insertions(+) create mode 100644 docs/en/sql-reference/functions/files.md diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md new file mode 100644 index 00000000000..3fb89db929d --- /dev/null +++ b/docs/en/sql-reference/functions/files.md @@ -0,0 +1,35 @@ +--- +toc_priority: 68 +toc_title: Files +--- + +# Functions for Working with Files {#functions-for-working-with-files} + +## file() {#file} + +Reads file as a String. The file can contain subqueries, condition, names of columns and any other information that will be read as one line. + +**Syntax** + +``` sql +file(filename) +``` + +**Arguments** + +- `filename` — The name of the file to read. The file must be located in the user's directory specified in `user_files_path` settings. + +**Example** + +Inserting data from files a.txt and b.txt in the table as separate rows. + +Query: + +``` sql +INSERT INTO table SELECT file('a.txt'), file('b.txt') +``` + +**See Also** + +- [user_files_path][../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path](#) + From 9f6ad165c84270166fa43b15ef0a19be4c040e9d Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 26 Mar 2021 11:22:43 +0300 Subject: [PATCH 29/60] Updated description --- docs/en/sql-reference/functions/files.md | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 3fb89db929d..a0da5a2c82d 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -5,23 +5,23 @@ toc_title: Files # Functions for Working with Files {#functions-for-working-with-files} -## file() {#file} +## file(path) {#file} -Reads file as a String. The file can contain subqueries, condition, names of columns and any other information that will be read as one line. +Reads file as a String. The file can contain subqueries, condition, names of columns and any other information that will be read as one line. **Syntax** ``` sql -file(filename) +file(path) ``` **Arguments** -- `filename` — The name of the file to read. The file must be located in the user's directory specified in `user_files_path` settings. +- `path` — The relative path to the file from `user_files_path`. **Example** -Inserting data from files a.txt and b.txt in the table as separate rows. +Inserting data from files a.txt and b.txt into a table as different rows. Query: @@ -31,5 +31,6 @@ INSERT INTO table SELECT file('a.txt'), file('b.txt') **See Also** -- [user_files_path][../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path](#) +- [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path) +- [file](../table-functions/file/) From 25052fc3f2a648efd268283c419ef218b16cdac8 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 26 Mar 2021 11:47:41 +0300 Subject: [PATCH 30/60] Reads file as a String --- docs/en/sql-reference/functions/files.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index a0da5a2c82d..55ecf7d3aba 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -1,5 +1,5 @@ --- -toc_priority: 68 +toc_priority: 43 toc_title: Files --- @@ -17,7 +17,7 @@ file(path) **Arguments** -- `path` — The relative path to the file from `user_files_path`. +- `path` — The relative path to the file from `user_files_path`. Path to file support following globs in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. **Example** From e4bf63f4704e347bf48f0df859e7035921784407 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Fri, 26 Mar 2021 12:25:33 +0300 Subject: [PATCH 31/60] Link for see also --- docs/en/sql-reference/functions/files.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 55ecf7d3aba..47b07d9ac9d 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -32,5 +32,5 @@ INSERT INTO table SELECT file('a.txt'), file('b.txt') **See Also** - [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path) -- [file](../table-functions/file/) +- [file](../table-functions/file.md) From 8d3f6be3704f7d34fe04a7adc7f60661e896310e Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Sun, 28 Mar 2021 21:26:51 +0300 Subject: [PATCH 32/60] minor fixes --- docs/en/sql-reference/functions/files.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 47b07d9ac9d..0e30d1bdd07 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -5,7 +5,7 @@ toc_title: Files # Functions for Working with Files {#functions-for-working-with-files} -## file(path) {#file} +## file {#file} Reads file as a String. The file can contain subqueries, condition, names of columns and any other information that will be read as one line. @@ -17,7 +17,7 @@ file(path) **Arguments** -- `path` — The relative path to the file from `user_files_path`. Path to file support following globs in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. +- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following wildcards in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. **Example** @@ -26,7 +26,7 @@ Inserting data from files a.txt and b.txt into a table as different rows. Query: ``` sql -INSERT INTO table SELECT file('a.txt'), file('b.txt') +INSERT INTO table SELECT file('a.txt'), file('b.txt'); ``` **See Also** From 83b8f4e2ea8832782e8e871eb0049bccf3eb569d Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Mon, 29 Mar 2021 22:27:06 +0300 Subject: [PATCH 33/60] Translate to Russian --- docs/ru/sql-reference/functions/files.md | 34 ++++++++++++++++++++++++ 1 file changed, 34 insertions(+) create mode 100644 docs/ru/sql-reference/functions/files.md diff --git a/docs/ru/sql-reference/functions/files.md b/docs/ru/sql-reference/functions/files.md new file mode 100644 index 00000000000..7252a3a18e2 --- /dev/null +++ b/docs/ru/sql-reference/functions/files.md @@ -0,0 +1,34 @@ +--- +toc_priority: 43 +toc_title: Files +--- + +# Функции для работы с файлами {#funktsii-dlia-raboty-s-failami} + +## file {#file} + +Читает файл как строку. Файл может содержать подзапросы, условие, названия столбцов и любую другую информацию, которая будет прочитана как одна строка. + +**Синтаксис** + +``` sql +file(path) +``` + +**Аргументы** + +- `path` — относительный путь до файла от [user_files_path](../../sql-reference/table-functions/file.md#server_configuration_parameters-user_files_path). Путь к файлу поддерживает следующие шаблоны в режиме доступа только для чтения `*`, `?`, `{abc,def}` и `{N..M}`, где `N`, `M` — числа, `'abc', 'def'` — строки. + +**Примеры** + +Вставка данных из файлов a.txt и b.txt в таблицу в виде отдельных строк: + +``` sql +INSERT INTO table SELECT file('a.txt'), file('b.txt'); +``` + +**Смотрите также** + +- [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path) +- [file](../table-functions/file.md) + From 9925110713c47611bccc8185a830b7f5c7ca0341 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 29 Mar 2021 23:04:50 +0300 Subject: [PATCH 34/60] rename other occurrances --- .../config/config.d/zzz-perf-comparison-tweaks-config.xml | 2 +- docker/test/stateful/run.sh | 4 ++-- docker/test/stateless/run.sh | 4 ++-- tests/config/config.d/database_replicated.xml | 4 ++-- tests/config/install.sh | 2 +- 5 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml b/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml index ee2006201b0..31f5b739c6d 100644 --- a/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml +++ b/docker/test/performance-comparison/config/config.d/zzz-perf-comparison-tweaks-config.xml @@ -3,7 +3,7 @@ - + :: diff --git a/docker/test/stateful/run.sh b/docker/test/stateful/run.sh index 3682686bc99..9e210dc92a2 100755 --- a/docker/test/stateful/run.sh +++ b/docker/test/stateful/run.sh @@ -22,14 +22,14 @@ function start() --logger.log /var/log/clickhouse-server/clickhouse-server1.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server1.err.log \ --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ --mysql_port 19004 \ - --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 + --keeper_server.tcp_port 19181 --keeper_server.server_id 2 sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ -- --path /var/lib/clickhouse2/ --logger.stderr /var/log/clickhouse-server/stderr2.log \ --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ --mysql_port 29004 \ - --test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3 + --keeper_server.tcp_port 29181 --keeper_server.server_id 3 fi counter=0 diff --git a/docker/test/stateless/run.sh b/docker/test/stateless/run.sh index fda9e61dd29..539b8d52cdd 100755 --- a/docker/test/stateless/run.sh +++ b/docker/test/stateless/run.sh @@ -45,7 +45,7 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] --logger.log /var/log/clickhouse-server/clickhouse-server1.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server1.err.log \ --tcp_port 19000 --tcp_port_secure 19440 --http_port 18123 --https_port 18443 --interserver_http_port 19009 --tcp_with_proxy_port 19010 \ --mysql_port 19004 \ - --test_keeper_server.tcp_port 19181 --test_keeper_server.server_id 2 \ + --keeper_server.tcp_port 19181 --keeper_server.server_id 2 \ --macros.replica r2 # It doesn't work :( sudo -E -u clickhouse /usr/bin/clickhouse server --config /etc/clickhouse-server2/config.xml --daemon \ @@ -53,7 +53,7 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] --logger.log /var/log/clickhouse-server/clickhouse-server2.log --logger.errorlog /var/log/clickhouse-server/clickhouse-server2.err.log \ --tcp_port 29000 --tcp_port_secure 29440 --http_port 28123 --https_port 28443 --interserver_http_port 29009 --tcp_with_proxy_port 29010 \ --mysql_port 29004 \ - --test_keeper_server.tcp_port 29181 --test_keeper_server.server_id 3 \ + --keeper_server.tcp_port 29181 --keeper_server.server_id 3 \ --macros.shard s2 # It doesn't work :( MAX_RUN_TIME=$((MAX_RUN_TIME < 9000 ? MAX_RUN_TIME : 9000)) # min(MAX_RUN_TIME, 2.5 hours) diff --git a/tests/config/config.d/database_replicated.xml b/tests/config/config.d/database_replicated.xml index 3bbae6031c9..ed5845bad48 100644 --- a/tests/config/config.d/database_replicated.xml +++ b/tests/config/config.d/database_replicated.xml @@ -14,7 +14,7 @@ - + 9181 1 @@ -50,7 +50,7 @@ 1
- + diff --git a/tests/config/install.sh b/tests/config/install.sh index d5396179de9..9c4f8caca07 100755 --- a/tests/config/install.sh +++ b/tests/config/install.sh @@ -67,7 +67,7 @@ if [[ -n "$USE_DATABASE_REPLICATED" ]] && [[ "$USE_DATABASE_REPLICATED" -eq 1 ]] ln -sf $SRC_PATH/users.d/database_replicated.xml $DEST_SERVER_PATH/users.d/ ln -sf $SRC_PATH/config.d/database_replicated.xml $DEST_SERVER_PATH/config.d/ rm /etc/clickhouse-server/config.d/zookeeper.xml - rm /etc/clickhouse-server/config.d/test_keeper_port.xml + rm /etc/clickhouse-server/config.d/keeper_port.xml # There is a bug in config reloading, so we cannot override macros using --macros.replica r2 # And we have to copy configs... From 85e4cfa731b246151c99d7ecc8eaf1f565031cf3 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Mar 2021 00:04:03 +0300 Subject: [PATCH 35/60] Run performance tests with enabled mmap IO --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 2986564b398..bc708e414ec 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -141,7 +141,7 @@ class IColumn; M(UInt64, optimize_min_equality_disjunction_chain_length, 3, "The minimum length of the expression `expr = x1 OR ... expr = xN` for optimization ", 0) \ \ M(UInt64, min_bytes_to_use_direct_io, 0, "The minimum number of bytes for reading the data with O_DIRECT option during SELECT queries execution. 0 - disabled.", 0) \ - M(UInt64, min_bytes_to_use_mmap_io, 0, "The minimum number of bytes for reading the data with mmap option during SELECT queries execution. 0 - disabled.", 0) \ + M(UInt64, min_bytes_to_use_mmap_io, (64 * 1024 * 1024), "The minimum number of bytes for reading the data with mmap option during SELECT queries execution. 0 - disabled.", 0) \ M(Bool, checksum_on_read, true, "Validate checksums on reading. It is enabled by default and should be always enabled in production. Please do not expect any benefits in disabling this setting. It may only be used for experiments and benchmarks. The setting only applicable for tables of MergeTree family. Checksums are always validated for other table engines and when receiving data over network.", 0) \ \ M(Bool, force_index_by_date, 0, "Throw an exception if there is a partition key in a table, and it is not used.", 0) \ From 466c70fb7cc75c3c1ef3f0d34adc829834c796ad Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Mar 2021 02:00:25 +0300 Subject: [PATCH 36/60] Code simplification --- src/Functions/partitionId.cpp | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/src/Functions/partitionId.cpp b/src/Functions/partitionId.cpp index b6d9d1bf4e6..c8411fb1860 100644 --- a/src/Functions/partitionId.cpp +++ b/src/Functions/partitionId.cpp @@ -45,21 +45,15 @@ public: virtual ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t input_rows_count) const override { + Block sample_block(arguments); size_t size = arguments.size(); - Columns columns; - Block sample_block; - for (const auto & argument : arguments) - { - sample_block.insert(argument); - columns.push_back(argument.column); - } auto result_column = ColumnString::create(); for (size_t j = 0; j < input_rows_count; ++j) { Row row(size); for (size_t i = 0; i < size; ++i) - columns[i]->get(j, row[i]); + arguments[i].column->get(j, row[i]); MergeTreePartition partition(std::move(row)); result_column->insert(partition.getID(sample_block)); } From e9c2309c05884f061e45561264e1481771107165 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Tue, 30 Mar 2021 02:02:53 +0300 Subject: [PATCH 37/60] A comment --- src/Functions/partitionId.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Functions/partitionId.cpp b/src/Functions/partitionId.cpp index c8411fb1860..39d6f1aff84 100644 --- a/src/Functions/partitionId.cpp +++ b/src/Functions/partitionId.cpp @@ -16,6 +16,7 @@ namespace ErrorCodes /** partitionId(x, y, ...) is a function that computes partition ids of arguments. + * The function is slow and should not be called for large amount of rows. */ class FunctionPartitionId : public IFunction { From 6d2f4dd82eaaf3615a7a6f40f6c7b1487c85f0bd Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Tue, 30 Mar 2021 14:51:19 +0800 Subject: [PATCH 38/60] MySQL is started only once with MaterializeMySQL integration test --- .../test_materialize_mysql_database/test.py | 65 +++++++++++++------ 1 file changed, 44 insertions(+), 21 deletions(-) diff --git a/tests/integration/test_materialize_mysql_database/test.py b/tests/integration/test_materialize_mysql_database/test.py index ced9a978d02..730305a6f16 100644 --- a/tests/integration/test_materialize_mysql_database/test.py +++ b/tests/integration/test_materialize_mysql_database/test.py @@ -42,7 +42,7 @@ class MySQLNodeInstance: if not os.path.exists(self.instances_dir): os.mkdir(self.instances_dir) self.docker_logs_path = p.join(self.instances_dir, 'docker_mysql.log') - + self.start_up = False def alloc_connection(self): if self.mysql_connection is None: @@ -78,12 +78,16 @@ class MySQLNodeInstance: return cursor.fetchall() def start_and_wait(self): + if self.start_up: + return + run_and_check(['docker-compose', - '-p', cluster.project_name, - '-f', self.docker_compose, - 'up', '--no-recreate', '-d', - ]) + '-p', cluster.project_name, + '-f', self.docker_compose, + 'up', '--no-recreate', '-d', + ]) self.wait_mysql_to_start(120) + self.start_up = True def close(self): if self.mysql_connection is not None: @@ -99,6 +103,8 @@ class MySQLNodeInstance: except Exception as e: print("Unable to get logs from docker mysql.") + self.start_up = False + def wait_mysql_to_start(self, timeout=60): start = time.time() while time.time() - start < timeout: @@ -113,32 +119,32 @@ class MySQLNodeInstance: run_and_check(['docker-compose', 'ps', '--services', 'all']) raise Exception("Cannot wait MySQL container") + +mysql_5_7_docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_5_7_for_materialize_mysql.yml') +mysql_5_7_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 3308, mysql_5_7_docker_compose) + +mysql_8_0_docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_8_0_for_materialize_mysql.yml') +mysql_8_0_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33308, mysql_8_0_docker_compose) + + @pytest.fixture(scope="module") def started_mysql_5_7(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_5_7_for_materialize_mysql.yml') - mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 3308, docker_compose) - try: - mysql_node.start_and_wait() - yield mysql_node + mysql_5_7_node.start_and_wait() + yield mysql_5_7_node finally: - mysql_node.close() - run_and_check(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes', - '--remove-orphans']) + mysql_5_7_node.close() + run_and_check(['docker-compose', '-p', cluster.project_name, '-f', mysql_5_7_docker_compose, 'down', '--volumes', '--remove-orphans']) @pytest.fixture(scope="module") def started_mysql_8_0(): - docker_compose = os.path.join(DOCKER_COMPOSE_PATH, 'docker_compose_mysql_8_0_for_materialize_mysql.yml') - mysql_node = MySQLNodeInstance('root', 'clickhouse', '127.0.0.1', 33308, docker_compose) - try: - mysql_node.start_and_wait() - yield mysql_node + mysql_8_0_node.start_and_wait() + yield mysql_8_0_node finally: - mysql_node.close() - run_and_check(['docker-compose', '-p', cluster.project_name, '-f', docker_compose, 'down', '--volumes', - '--remove-orphans']) + mysql_8_0_node.close() + run_and_check(['docker-compose', '-p', cluster.project_name, '-f', mysql_8_0_docker_compose, 'down', '--volumes', '--remove-orphans']) @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) @@ -146,11 +152,13 @@ def test_materialize_database_dml_with_mysql_5_7(started_cluster, started_mysql_ materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_5_7, "mysql1") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_dml_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.dml_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") materialize_with_ddl.materialize_mysql_database_with_datetime_and_decimal(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") @@ -163,6 +171,7 @@ def test_materialize_database_ddl_with_mysql_5_7(started_cluster, started_mysql_ materialize_with_ddl.alter_rename_table_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_5_7, "mysql1") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.drop_table_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") @@ -179,10 +188,12 @@ def test_materialize_database_ddl_with_mysql_8_0(started_cluster, started_mysql_ materialize_with_ddl.alter_modify_column_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_empty_transaction_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_5_7, "mysql1") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_materialize_database_ddl_with_empty_transaction_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.query_event_with_empty_transaction(clickhouse_node, started_mysql_8_0, "mysql8_0") @@ -217,52 +228,64 @@ def test_materialize_database_err_sync_user_privs_5_7(started_cluster, started_m def test_materialize_database_err_sync_user_privs_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.err_sync_user_privs_with_materialize_mysql_database(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_network_partition_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_5_7, "mysql1") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_network_partition_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.network_partition_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_kill_sync_thread_restore_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_5_7, "mysql1") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_kill_sync_thread_restore_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.mysql_kill_sync_thread_restore_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql1") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_mysql_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.mysql_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_clickhouse_killed_while_insert_5_7(started_cluster, started_mysql_5_7, clickhouse_node): materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_5_7, "mysql1") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_atomic]) def test_clickhouse_killed_while_insert_8_0(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.clickhouse_killed_while_insert(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_utf8mb4(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_5_7, "mysql1") materialize_with_ddl.utf8mb4_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_system_parts_table(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.system_parts_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_multi_table_update(started_cluster, started_mysql_8_0, started_mysql_5_7, clickhouse_node): materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_5_7, "mysql1") materialize_with_ddl.multi_table_update_test(clickhouse_node, started_mysql_8_0, "mysql8_0") + @pytest.mark.parametrize(('clickhouse_node'), [node_db_ordinary, node_db_ordinary]) def test_system_tables_table(started_cluster, started_mysql_8_0, clickhouse_node): materialize_with_ddl.system_tables_test(clickhouse_node, started_mysql_8_0, "mysql8_0") From 517ee6f142bbe46ebbabec11a865afa0a99eb41e Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 30 Mar 2021 10:45:51 +0300 Subject: [PATCH 39/60] rename --- tests/jepsen.clickhouse-keeper/doc/intro.md | 2 +- tests/jepsen.clickhouse-keeper/project.clj | 2 +- tests/jepsen.clickhouse-keeper/test/jepsen/keeper_test.clj | 2 +- utils/keeper-data-dumper/main.cpp | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/jepsen.clickhouse-keeper/doc/intro.md b/tests/jepsen.clickhouse-keeper/doc/intro.md index c6e5ccbd04a..09ce235c467 100644 --- a/tests/jepsen.clickhouse-keeper/doc/intro.md +++ b/tests/jepsen.clickhouse-keeper/doc/intro.md @@ -1,3 +1,3 @@ -# Introduction to jepsen.nukeeper +# Introduction to jepsen.keeper TODO: write [great documentation](http://jacobian.org/writing/what-to-write/) diff --git a/tests/jepsen.clickhouse-keeper/project.clj b/tests/jepsen.clickhouse-keeper/project.clj index c9b24e0ce2c..c38767a767d 100644 --- a/tests/jepsen.clickhouse-keeper/project.clj +++ b/tests/jepsen.clickhouse-keeper/project.clj @@ -1,4 +1,4 @@ -(defproject jepsen.nukeeper "0.1.0-SNAPSHOT" +(defproject jepsen.keeper "0.1.0-SNAPSHOT" :injections [(.. System (setProperty "zookeeper.request.timeout" "10000"))] :description "A jepsen tests for ClickHouse Keeper" :url "https://clickhouse.tech/" diff --git a/tests/jepsen.clickhouse-keeper/test/jepsen/keeper_test.clj b/tests/jepsen.clickhouse-keeper/test/jepsen/keeper_test.clj index 35b1bab7b38..25333605351 100644 --- a/tests/jepsen.clickhouse-keeper/test/jepsen/keeper_test.clj +++ b/tests/jepsen.clickhouse-keeper/test/jepsen/keeper_test.clj @@ -15,7 +15,7 @@ (dorun (map (fn [v] (zk/delete conn v)) (take 10 (zk-range))))) (deftest a-test - (testing "nukeeper connection" + (testing "keeper connection" (.setLevel (LoggerFactory/getLogger "org.apache.zookeeper") Level/OFF) (let [conn (zk/connect "localhost:9181" :timeout-msec 5000)] diff --git a/utils/keeper-data-dumper/main.cpp b/utils/keeper-data-dumper/main.cpp index 74b5a6ddf93..11db6fc61bc 100644 --- a/utils/keeper-data-dumper/main.cpp +++ b/utils/keeper-data-dumper/main.cpp @@ -58,7 +58,7 @@ int main(int argc, char *argv[]) Poco::Logger::root().setChannel(channel); Poco::Logger::root().setLevel("trace"); } - auto * logger = &Poco::Logger::get("nukeeper-dumper"); + auto * logger = &Poco::Logger::get("keeper-dumper"); ResponsesQueue queue; SnapshotsQueue snapshots_queue{1}; CoordinationSettingsPtr settings = std::make_shared(); From 676ea1efc3903103056defdadee8ce654ea9e3c1 Mon Sep 17 00:00:00 2001 From: Tatiana Kirillova Date: Tue, 30 Mar 2021 11:52:38 +0300 Subject: [PATCH 40/60] The translate to Russian --- docs/en/sql-reference/functions/files.md | 2 +- docs/ru/sql-reference/functions/files.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 0e30d1bdd07..4cf1141e1ad 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -7,7 +7,7 @@ toc_title: Files ## file {#file} -Reads file as a String. The file can contain subqueries, condition, names of columns and any other information that will be read as one line. +Reads file as a String. The file can contain subqueries, conditions, names of columns and any other information that will be read as one line. **Syntax** diff --git a/docs/ru/sql-reference/functions/files.md b/docs/ru/sql-reference/functions/files.md index 7252a3a18e2..8db29e0c4e6 100644 --- a/docs/ru/sql-reference/functions/files.md +++ b/docs/ru/sql-reference/functions/files.md @@ -1,13 +1,13 @@ --- toc_priority: 43 -toc_title: Files +toc_title: "Функции для работы с файлами" --- # Функции для работы с файлами {#funktsii-dlia-raboty-s-failami} ## file {#file} -Читает файл как строку. Файл может содержать подзапросы, условие, названия столбцов и любую другую информацию, которая будет прочитана как одна строка. +Читает файл как строку. Файл может содержать подзапросы, условия, названия столбцов и любую другую информацию, которая будет прочитана как одна строка. **Синтаксис** From 0bb369d5e2935b76e30b71acda6fd13ceb88cd07 Mon Sep 17 00:00:00 2001 From: Pavel Kovalenko Date: Tue, 30 Mar 2021 13:21:46 +0300 Subject: [PATCH 41/60] Code review fixes. --- src/Disks/S3/DiskS3.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Disks/S3/DiskS3.cpp b/src/Disks/S3/DiskS3.cpp index 74c859efb75..211a9a63a74 100644 --- a/src/Disks/S3/DiskS3.cpp +++ b/src/Disks/S3/DiskS3.cpp @@ -1030,7 +1030,7 @@ void DiskS3::migrateToRestorableSchemaRecursive(const String & path, Futures & r LOG_DEBUG(&Poco::Logger::get("DiskS3"), "Migrate directory {} to restorable schema", metadata_path + path); bool dir_contains_only_files = true; - for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + for (auto it = iterateDirectory(path); it->isValid(); it->next()) if (isDirectory(it->path())) { dir_contains_only_files = false; @@ -1042,7 +1042,7 @@ void DiskS3::migrateToRestorableSchemaRecursive(const String & path, Futures & r { auto result = getExecutor().execute([this, path] { - for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + for (auto it = iterateDirectory(path); it->isValid(); it->next()) migrateFileToRestorableSchema(it->path()); }); @@ -1050,7 +1050,7 @@ void DiskS3::migrateToRestorableSchemaRecursive(const String & path, Futures & r } else { - for (auto it{iterateDirectory(path)}; it->isValid(); it->next()) + for (auto it = iterateDirectory(path); it->isValid(); it->next()) if (!isDirectory(it->path())) { auto source_path = it->path(); From 4f896d2b076c78b5ee2d030e643082923f287441 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Tue, 30 Mar 2021 19:46:23 +0300 Subject: [PATCH 42/60] fix tests --- .../MergeTree/registerStorageMergeTree.cpp | 30 +++++++++++++++++++ ...ted_minimalistic_part_header_zookeeper.sql | 8 ++--- ...icated_mutations_empty_partition.reference | 2 +- ...6_replicated_mutations_empty_partition.sql | 6 ++-- ...tations_kill_many_replicas_long.reference} | 0 ...lter_mutations_kill_many_replicas_long.sh} | 0 6 files changed, 38 insertions(+), 8 deletions(-) rename tests/queries/0_stateless/{01593_concurrent_alter_mutations_kill_many_replicas.reference => 01593_concurrent_alter_mutations_kill_many_replicas_long.reference} (100%) rename tests/queries/0_stateless/{01593_concurrent_alter_mutations_kill_many_replicas.sh => 01593_concurrent_alter_mutations_kill_many_replicas_long.sh} (100%) diff --git a/src/Storages/MergeTree/registerStorageMergeTree.cpp b/src/Storages/MergeTree/registerStorageMergeTree.cpp index 6cde5245735..6dd005736f0 100644 --- a/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -20,6 +20,7 @@ #include #include +#include namespace DB @@ -410,6 +411,35 @@ static StoragePtr create(const StorageFactory::Arguments & args) throw Exception(msg, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); } + if (is_extended_storage_def) + { + /// Allow expressions in engine arguments. + /// In new syntax argument can be literal or identifier or array/tuple of identifiers. + size_t arg_idx = 0; + try + { + for (; arg_idx < engine_args.size(); ++arg_idx) + { + auto & arg = engine_args[arg_idx]; + auto * arg_func = arg->as(); + if (!arg_func) + continue; + + /// If we got ASTFunction, let's evaluate it and replace with ASTLiteral. + /// Do not try evaluate array or tuple, because it's array or tuple of column identifiers. + if (arg_func->name == "array" || arg_func->name == "tuple") + continue; + Field value = evaluateConstantExpression(arg, args.local_context).first; + arg = std::make_shared(value); + } + } + catch (Exception & e) + { + throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot evaluate engine argument {}: {} {}", + arg_idx, e.message(), getMergeTreeVerboseHelp(is_extended_storage_def)); + } + } + /// For Replicated. String zookeeper_path; String replica_name; diff --git a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql index 163ebf1af23..63897e225ce 100644 --- a/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql +++ b/tests/queries/0_stateless/00814_replicated_minimalistic_part_header_zookeeper.sql @@ -4,13 +4,13 @@ DROP TABLE IF EXISTS part_header_r2; SET replication_alter_partitions_sync = 2; CREATE TABLE part_header_r1(x UInt32, y UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header/{shard}', '1{replica}') ORDER BY x + ENGINE ReplicatedMergeTree('/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/{shard}', '1{replica}') ORDER BY x SETTINGS use_minimalistic_part_header_in_zookeeper = 0, old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0; CREATE TABLE part_header_r2(x UInt32, y UInt32) - ENGINE ReplicatedMergeTree('/clickhouse/tables/test_00814/part_header/{shard}', '2{replica}') ORDER BY x + ENGINE ReplicatedMergeTree('/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/{shard}', '2{replica}') ORDER BY x SETTINGS use_minimalistic_part_header_in_zookeeper = 1, old_parts_lifetime = 1, cleanup_delay_period = 0, @@ -39,10 +39,10 @@ SELECT sleep(3) FORMAT Null; SELECT '*** Test part removal ***'; SELECT '*** replica 1 ***'; SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r1'; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/s1/replicas/1r1/parts'; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/s1/replicas/1r1/parts'; SELECT '*** replica 2 ***'; SELECT name FROM system.parts WHERE active AND database = currentDatabase() AND table = 'part_header_r2'; -SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/test_00814/part_header/s1/replicas/1r1/parts'; +SELECT name FROM system.zookeeper WHERE path = '/clickhouse/tables/'||currentDatabase()||'/test_00814/part_header/s1/replicas/1r1/parts'; SELECT '*** Test ALTER ***'; ALTER TABLE part_header_r1 MODIFY COLUMN y String; diff --git a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference index 1f7146dfe48..2f204867c41 100644 --- a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference +++ b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.reference @@ -2,4 +2,4 @@ 10 10 24 -CREATE TABLE default.replicated_mutations_empty_partitions\n(\n `key` UInt64,\n `value` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/01586_replicated_mutations_empty_partitions/{shard}\', \'{replica}\')\nPARTITION BY key\nORDER BY key\nSETTINGS index_granularity = 8192 +CREATE TABLE default.replicated_mutations_empty_partitions\n(\n `key` UInt64,\n `value` UInt64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/test/default/01586_replicated_mutations_empty_partitions/{shard}\', \'{replica}\')\nPARTITION BY key\nORDER BY key\nSETTINGS index_granularity = 8192 diff --git a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql index 63a3069c518..73245fe49ec 100644 --- a/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql +++ b/tests/queries/0_stateless/01586_replicated_mutations_empty_partition.sql @@ -5,7 +5,7 @@ CREATE TABLE replicated_mutations_empty_partitions key UInt64, value String ) -ENGINE = ReplicatedMergeTree('/clickhouse/test/01586_replicated_mutations_empty_partitions/{shard}', '{replica}') +ENGINE = ReplicatedMergeTree('/clickhouse/test/'||currentDatabase()||'/01586_replicated_mutations_empty_partitions/{shard}', '{replica}') ORDER BY key PARTITION by key; @@ -13,7 +13,7 @@ INSERT INTO replicated_mutations_empty_partitions SELECT number, toString(number SELECT count(distinct value) FROM replicated_mutations_empty_partitions; -SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/s1/block_numbers'; +SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/'||currentDatabase()||'/01586_replicated_mutations_empty_partitions/s1/block_numbers'; ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '3'; ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '4'; @@ -21,7 +21,7 @@ ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '5'; ALTER TABLE replicated_mutations_empty_partitions DROP PARTITION '9'; -- still ten records -SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/01586_replicated_mutations_empty_partitions/s1/block_numbers'; +SELECT count() FROM system.zookeeper WHERE path = '/clickhouse/test/'||currentDatabase()||'/01586_replicated_mutations_empty_partitions/s1/block_numbers'; ALTER TABLE replicated_mutations_empty_partitions MODIFY COLUMN value UInt64 SETTINGS replication_alter_partitions_sync=2; diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.reference b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference similarity index 100% rename from tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.reference rename to tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.sh b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh similarity index 100% rename from tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas.sh rename to tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.sh From fb8fdf7aa1ce8ef0b54a4af1d8cca4189cd4a4c3 Mon Sep 17 00:00:00 2001 From: kirillikoff Date: Tue, 30 Mar 2021 20:10:03 +0300 Subject: [PATCH 43/60] Update docs/en/sql-reference/functions/files.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/files.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 4cf1141e1ad..c081daeae7c 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -7,7 +7,7 @@ toc_title: Files ## file {#file} -Reads file as a String. The file can contain subqueries, conditions, names of columns and any other information that will be read as one line. +Reads file as a String. The file content is not parsed, so any information is read as one string and placed into the specified column. **Syntax** @@ -33,4 +33,3 @@ INSERT INTO table SELECT file('a.txt'), file('b.txt'); - [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path) - [file](../table-functions/file.md) - From 3573a2ba4bfc397a5879c048c80b1d9df1233a2b Mon Sep 17 00:00:00 2001 From: kirillikoff Date: Tue, 30 Mar 2021 20:10:15 +0300 Subject: [PATCH 44/60] Update docs/en/sql-reference/functions/files.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/files.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index c081daeae7c..5bc84649c89 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -17,7 +17,7 @@ file(path) **Arguments** -- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following wildcards in read-only mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. +- `path` — The relative path to the file from [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Path to file support following wildcards: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc', 'def'` — strings. **Example** From cde71fbbe5eb839f710100f858a1f06f46b037dd Mon Sep 17 00:00:00 2001 From: kirillikoff Date: Tue, 30 Mar 2021 20:11:30 +0300 Subject: [PATCH 45/60] Update docs/en/sql-reference/functions/files.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/en/sql-reference/functions/files.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/functions/files.md b/docs/en/sql-reference/functions/files.md index 5bc84649c89..9cbf8932465 100644 --- a/docs/en/sql-reference/functions/files.md +++ b/docs/en/sql-reference/functions/files.md @@ -21,7 +21,7 @@ file(path) **Example** -Inserting data from files a.txt and b.txt into a table as different rows. +Inserting data from files a.txt and b.txt into a table as strings: Query: From e74e3df04ab1509246dfade37b9c28df30a862da Mon Sep 17 00:00:00 2001 From: kirillikoff Date: Tue, 30 Mar 2021 20:12:39 +0300 Subject: [PATCH 46/60] Update docs/ru/sql-reference/functions/files.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/files.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/functions/files.md b/docs/ru/sql-reference/functions/files.md index 8db29e0c4e6..d4152a52b31 100644 --- a/docs/ru/sql-reference/functions/files.md +++ b/docs/ru/sql-reference/functions/files.md @@ -7,7 +7,7 @@ toc_title: "Функции для работы с файлами" ## file {#file} -Читает файл как строку. Файл может содержать подзапросы, условия, названия столбцов и любую другую информацию, которая будет прочитана как одна строка. +Читает файл как строку. Содержимое файла не разбирается (не парсится) и записывается в указанную колонку в виде единой строки. **Синтаксис** @@ -31,4 +31,3 @@ INSERT INTO table SELECT file('a.txt'), file('b.txt'); - [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path) - [file](../table-functions/file.md) - From 3676f5d7983dd43d33344dbb2904e440613038c7 Mon Sep 17 00:00:00 2001 From: kirillikoff Date: Tue, 30 Mar 2021 20:12:57 +0300 Subject: [PATCH 47/60] Update docs/ru/sql-reference/functions/files.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/files.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/files.md b/docs/ru/sql-reference/functions/files.md index d4152a52b31..d7792da1929 100644 --- a/docs/ru/sql-reference/functions/files.md +++ b/docs/ru/sql-reference/functions/files.md @@ -21,7 +21,7 @@ file(path) **Примеры** -Вставка данных из файлов a.txt и b.txt в таблицу в виде отдельных строк: +Вставка данных из файлов a.txt и b.txt в таблицу в виде строк: ``` sql INSERT INTO table SELECT file('a.txt'), file('b.txt'); From 049019bdb21df5d25a6a2239903fdba8167d9009 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Tue, 30 Mar 2021 20:27:58 +0300 Subject: [PATCH 48/60] Update clickhouse-test --- tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/clickhouse-test b/tests/clickhouse-test index b7b72e0de41..afb02e7ea0b 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -227,7 +227,7 @@ def get_processlist(args): query = b"SELECT materialize((hostName(), tcpPort())) as host, * " \ b"FROM clusterAllReplicas('r', system.processes) WHERE query NOT LIKE '%system.processes%' FORMAT Vertical" clickhouse_proc = Popen(shlex.split(args.client), stdin=PIPE, stdout=PIPE, stderr=PIPE) - (stdout, _) = clickhouse_proc.communicate((b"SHOW PROCESSLIST FORMAT Vertical"), timeout=20) + (stdout, _) = clickhouse_proc.communicate((query), timeout=20) return False, stdout.decode('utf-8') except Exception as ex: print("Exception", ex) From 2ab2de6fd3bb30d2eb05b2fb456ce55fa1d6ae62 Mon Sep 17 00:00:00 2001 From: kirillikoff Date: Tue, 30 Mar 2021 20:34:21 +0300 Subject: [PATCH 49/60] Update docs/ru/sql-reference/functions/files.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/sql-reference/functions/files.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/functions/files.md b/docs/ru/sql-reference/functions/files.md index d7792da1929..9cb659375b9 100644 --- a/docs/ru/sql-reference/functions/files.md +++ b/docs/ru/sql-reference/functions/files.md @@ -17,7 +17,7 @@ file(path) **Аргументы** -- `path` — относительный путь до файла от [user_files_path](../../sql-reference/table-functions/file.md#server_configuration_parameters-user_files_path). Путь к файлу поддерживает следующие шаблоны в режиме доступа только для чтения `*`, `?`, `{abc,def}` и `{N..M}`, где `N`, `M` — числа, `'abc', 'def'` — строки. +- `path` — относительный путь до файла от [user_files_path](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-user_files_path). Путь к файлу может включать следующие символы подстановки и шаблоны: `*`, `?`, `{abc,def}` и `{N..M}`, где `N`, `M` — числа, `'abc', 'def'` — строки. **Примеры** From a712b34eed2d0a7a9c39c085aac9be73ec9e4725 Mon Sep 17 00:00:00 2001 From: tavplubix Date: Wed, 31 Mar 2021 00:06:43 +0300 Subject: [PATCH 50/60] Update 01593_concurrent_alter_mutations_kill_many_replicas_long.reference --- ...t_alter_mutations_kill_many_replicas_long.reference | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference index 06e637bc736..f7c65e36be4 100644 --- a/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference +++ b/tests/queries/0_stateless/01593_concurrent_alter_mutations_kill_many_replicas_long.reference @@ -4,13 +4,13 @@ 499999500000 499999500000 Metadata version on replica 1 equal with first replica, OK -CREATE TABLE default.concurrent_kill_1\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_default/{shard}\', \'{replica}1\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_1\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_long_default/{shard}\', \'{replica}1\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 2 equal with first replica, OK -CREATE TABLE default.concurrent_kill_2\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_default/{shard}\', \'{replica}2\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_2\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_long_default/{shard}\', \'{replica}2\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 3 equal with first replica, OK -CREATE TABLE default.concurrent_kill_3\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_default/{shard}\', \'{replica}3\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_3\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_long_default/{shard}\', \'{replica}3\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 4 equal with first replica, OK -CREATE TABLE default.concurrent_kill_4\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_default/{shard}\', \'{replica}4\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_4\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_long_default/{shard}\', \'{replica}4\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 Metadata version on replica 5 equal with first replica, OK -CREATE TABLE default.concurrent_kill_5\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_default/{shard}\', \'{replica}5\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 +CREATE TABLE default.concurrent_kill_5\n(\n `key` UInt64,\n `value` Int64\n)\nENGINE = ReplicatedMergeTree(\'/clickhouse/tables/01593_concurrent_alter_mutations_kill_many_replicas_long_default/{shard}\', \'{replica}5\')\nORDER BY key\nSETTINGS max_replicated_mutations_in_queue = 1000, number_of_free_entries_in_pool_to_execute_mutation = 0, max_replicated_merges_in_queue = 1000, index_granularity = 8192 499999500000 From c3b4587d9d85cbe1f33620b56b71e522057dd3bf Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Mar 2021 03:54:40 +0300 Subject: [PATCH 51/60] Fix MSan report in function "range" --- src/Functions/array/range.cpp | 53 ++++++++++++++++++++++++++--------- 1 file changed, 40 insertions(+), 13 deletions(-) diff --git a/src/Functions/array/range.cpp b/src/Functions/array/range.cpp index 1a057458175..ac968786923 100644 --- a/src/Functions/array/range.cpp +++ b/src/Functions/array/range.cpp @@ -22,6 +22,11 @@ namespace ErrorCodes } +/** Generates array + * range(size): [0, size) + * range(start, end): [start, end) + * range(start, end, step): [start, end) with step increments. + */ class FunctionRange : public IFunction { public: @@ -40,9 +45,9 @@ private: { if (arguments.size() > 3 || arguments.empty()) { - throw Exception{"Function " + getName() + " needs 1..3 arguments; passed " - + std::to_string(arguments.size()) + ".", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Function {} needs 1..3 arguments; passed {}.", + getName(), arguments.size()); } for (const auto & arg : arguments) @@ -339,6 +344,18 @@ private: ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t input_rows_count) const override { + DataTypePtr elem_type = checkAndGetDataType(result_type.get())->getNestedType(); + WhichDataType which(elem_type); + + if (!which.isUInt8() + && !which.isUInt16() + && !which.isUInt32() + && !which.isUInt64()) + { + throw Exception{"Illegal columns of arguments of function " + getName() + + ", the function only implemented for unsigned integers up to 64 bit", ErrorCodes::ILLEGAL_COLUMN}; + } + ColumnPtr res; if (arguments.size() == 1) { @@ -356,22 +373,24 @@ private: Columns columns_holder(3); ColumnRawPtrs column_ptrs(3); - const auto return_type = checkAndGetDataType(result_type.get())->getNestedType(); - for (size_t i = 0; i < arguments.size(); ++i) { if (i == 1) - columns_holder[i] = castColumn(arguments[i], return_type)->convertToFullColumnIfConst(); + columns_holder[i] = castColumn(arguments[i], elem_type)->convertToFullColumnIfConst(); else - columns_holder[i] = castColumn(arguments[i], return_type); + columns_holder[i] = castColumn(arguments[i], elem_type); column_ptrs[i] = columns_holder[i].get(); } - // for step column, defaults to 1 + /// Step is one by default. if (arguments.size() == 2) { - columns_holder[2] = return_type->createColumnConst(input_rows_count, 1); + /// Convert a column with constant 1 to the result type. + columns_holder[2] = castColumn( + {DataTypeUInt8().createColumnConst(input_rows_count, 1), std::make_shared(), {}}, + elem_type); + column_ptrs[2] = columns_holder[2].get(); } @@ -385,7 +404,9 @@ private: if ((res = executeConstStartStep(column_ptrs[1], start, step, input_rows_count)) || (res = executeConstStartStep(column_ptrs[1], start, step, input_rows_count)) || (res = executeConstStartStep(column_ptrs[1], start, step, input_rows_count)) || - (res = executeConstStartStep(column_ptrs[1], start, step, input_rows_count))) {} + (res = executeConstStartStep(column_ptrs[1], start, step, input_rows_count))) + { + } } else if (is_start_const && !is_step_const) { @@ -394,7 +415,9 @@ private: if ((res = executeConstStart(column_ptrs[1], column_ptrs[2], start, input_rows_count)) || (res = executeConstStart(column_ptrs[1], column_ptrs[2], start, input_rows_count)) || (res = executeConstStart(column_ptrs[1], column_ptrs[2], start, input_rows_count)) || - (res = executeConstStart(column_ptrs[1], column_ptrs[2], start, input_rows_count))) {} + (res = executeConstStart(column_ptrs[1], column_ptrs[2], start, input_rows_count))) + { + } } else if (!is_start_const && is_step_const) { @@ -403,14 +426,18 @@ private: if ((res = executeConstStep(column_ptrs[0], column_ptrs[1], step, input_rows_count)) || (res = executeConstStep(column_ptrs[0], column_ptrs[1], step, input_rows_count)) || (res = executeConstStep(column_ptrs[0], column_ptrs[1], step, input_rows_count)) || - (res = executeConstStep(column_ptrs[0], column_ptrs[1], step, input_rows_count))) {} + (res = executeConstStep(column_ptrs[0], column_ptrs[1], step, input_rows_count))) + { + } } else { if ((res = executeGeneric(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count)) || (res = executeGeneric(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count)) || (res = executeGeneric(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count)) || - (res = executeGeneric(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))) {} + (res = executeGeneric(column_ptrs[0], column_ptrs[1], column_ptrs[2], input_rows_count))) + { + } } if (!res) From 27b2362f3004b65a814c535d4ed01d28522a3eef Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Mar 2021 03:55:41 +0300 Subject: [PATCH 52/60] Add a test --- tests/queries/0_stateless/01780_range_msan.reference | 0 tests/queries/0_stateless/01780_range_msan.sql | 1 + 2 files changed, 1 insertion(+) create mode 100644 tests/queries/0_stateless/01780_range_msan.reference create mode 100644 tests/queries/0_stateless/01780_range_msan.sql diff --git a/tests/queries/0_stateless/01780_range_msan.reference b/tests/queries/0_stateless/01780_range_msan.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/01780_range_msan.sql b/tests/queries/0_stateless/01780_range_msan.sql new file mode 100644 index 00000000000..dd0a35c3eea --- /dev/null +++ b/tests/queries/0_stateless/01780_range_msan.sql @@ -0,0 +1 @@ +SELECT range(toUInt256(1), 1); -- { serverError 44 } From 1b218465c5a2a1740ca21d955b8c907608d75e8b Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Mar 2021 04:41:25 +0300 Subject: [PATCH 53/60] Remove recursive submodules --- contrib/arrow | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/arrow b/contrib/arrow index 744bdfe188f..616b3dc76a0 160000 --- a/contrib/arrow +++ b/contrib/arrow @@ -1 +1 @@ -Subproject commit 744bdfe188f018e5e05f5deebd4e9ee0a7706cf4 +Subproject commit 616b3dc76a0c8450b4027ded8a78e9619d7c845f From fefe3e69e5b57ceab8e2ba4689b6377fb026ab81 Mon Sep 17 00:00:00 2001 From: zhang2014 Date: Wed, 31 Mar 2021 11:45:37 +0800 Subject: [PATCH 54/60] trigger CI again From 7709402422d56f2863a6ba360b38925755519cb9 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Mar 2021 08:15:58 +0300 Subject: [PATCH 55/60] Add more adopters --- docs/en/introduction/adopters.md | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 23f7b596851..4d84895afa3 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -15,6 +15,7 @@ toc_title: Adopters | Alibaba Cloud | Cloud | Managed Service | — | — | [Official Website](https://help.aliyun.com/product/144466.html) | | Aloha Browser | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://presentations.clickhouse.tech/meetup22/aloha.pdf) | | Amadeus | Travel | Analytics | — | — | [Press Release, April 2018](https://www.altinity.com/blog/2018/4/5/amadeus-technologies-launches-investment-and-insights-tool-based-on-machine-learning-and-strategy-algorithms) | +| ApiRoad | API marketplace | Analytics | — | — | [Blog post, Nov 2018, Mar 2020](https://pixeljets.com/blog/clickhouse-vs-elasticsearch/) | | Appsflyer | Mobile analytics | Main product | — | — | [Talk in Russian, July 2019](https://www.youtube.com/watch?v=M3wbRlcpBbY) | | ArenaData | Data Platform | Main product | — | — | [Slides in Russian, December 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup38/indexes.pdf) | | Avito | Classifieds | Monitoring | — | — | [Meetup, April 2020](https://www.youtube.com/watch?v=n1tm4j4W8ZQ) | @@ -37,6 +38,7 @@ toc_title: Adopters | CraiditX 氪信 | Finance AI | Analysis | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/udf.pptx) | | Crazypanda | Games | | — | — | Live session on ClickHouse meetup | | Criteo | Retail | Main product | — | — | [Slides in English, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup18/3_storetail.pptx) | +| Cryptology | Digital Assets Trading Platform | — | — | — | [Job advertisement, March 2021](https://career.habr.com/companies/cryptology/vacancies) | | Dataliance for China Telecom | Telecom | Analytics | — | — | [Slides in Chinese, January 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup12/telecom.pdf) | | Deutsche Bank | Finance | BI Analytics | — | — | [Slides in English, October 2019](https://bigdatadays.ru/wp-content/uploads/2019/10/D2-H3-3_Yakunin-Goihburg.pdf) | | Deeplay | Gaming Analytics | — | — | — | [Job advertisement, 2020](https://career.habr.com/vacancies/1000062568) | @@ -49,6 +51,7 @@ toc_title: Adopters | FunCorp | Games | | — | 14 bn records/day as of Jan 2021 | [Article](https://www.altinity.com/blog/migrating-from-redshift-to-clickhouse) | | Geniee | Ad network | Main product | — | — | [Blog post in Japanese, July 2017](https://tech.geniee.co.jp/entry/2017/07/20/160100) | | Genotek | Bioinformatics | Main product | — | — | [Video, August 2020](https://youtu.be/v3KyZbz9lEE) | +| Glaber | Monitoring | Main product | — | — | [Website](https://glaber.io/) | | HUYA | Video Streaming | Analytics | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/7.%20ClickHouse万亿数据分析实践%20李本旺(sundy-li)%20虎牙.pdf) | | ICA | FinTech | Risk Management | — | — | [Blog Post in English, Sep 2020](https://altinity.com/blog/clickhouse-vs-redshift-performance-for-fintech-risk-management?utm_campaign=ClickHouse%20vs%20RedShift&utm_content=143520807&utm_medium=social&utm_source=twitter&hss_channel=tw-3894792263) | | Idealista | Real Estate | Analytics | — | — | [Blog Post in English, April 2019](https://clickhouse.tech/blog/en/clickhouse-meetup-in-madrid-on-april-2-2019) | @@ -65,15 +68,18 @@ toc_title: Adopters | Lawrence Berkeley National Laboratory | Research | Traffic analysis | 1 server | 11.8 TiB | [Slides in English, April 2019](https://www.smitasin.com/presentations/2019-04-17_DOE-NSM.pdf) | | LifeStreet | Ad network | Main product | 75 servers (3 replicas) | 5.27 PiB | [Blog post in Russian, February 2017](https://habr.com/en/post/322620/) | | Mail.ru Cloud Solutions | Cloud services | Main product | — | — | [Article in Russian](https://mcs.mail.ru/help/db-create/clickhouse#) | +| MAXILECT | Ad Tech, Blockchain, ML, AI | — | — | — | [Job advertisement, 2021](https://www.linkedin.com/feed/update/urn:li:activity:6780842017229430784/) | | Marilyn | Advertising | Statistics | — | — | [Talk in Russian, June 2017](https://www.youtube.com/watch?v=iXlIgx2khwc) | | Mello | Marketing | Analytics | 1 server | — | [Article, Oct 2020](https://vc.ru/marketing/166180-razrabotka-tipovogo-otcheta-skvoznoy-analitiki) | | MessageBird | Telecommunications | Statistics | — | — | [Slides in English, November 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup20/messagebird.pdf) | | MindsDB | Machine Learning | Main Product | — | — | [Official Website](https://www.mindsdb.com/blog/machine-learning-models-as-tables-in-ch) |x | MUX | Online Video | Video Analytics | — | — | [Talk in English, August 2019](https://altinity.com/presentations/2019/8/13/how-clickhouse-became-the-default-analytics-database-for-mux/) | | MGID | Ad network | Web-analytics | — | — | [Blog post in Russian, April 2020](http://gs-studio.com/news-about-it/32777----clickhouse---c) | +| Netskope | Network Security | — | — | — | [Job advertisement, March 2021](https://www.mendeley.com/careers/job/senior-software-developer-backend-developer-1346348) | | NOC Project | Network Monitoring | Analytics | Main Product | — | [Official Website](https://getnoc.com/features/big-data/) | | Nuna Inc. | Health Data Analytics | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=170) | | OneAPM | Monitorings and Data Analysis | Main product | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/8.%20clickhouse在OneAPM的应用%20杜龙.pdf) | +| OZON | E-commerce | — | — | — | [Official website](https://job.ozon.ru/vacancy/razrabotchik-clickhouse-ekspluatatsiya-40991870/) | | Panelbear | Analytics | Monitoring and Analytics | — | — | [Tech Stack, November 2020](https://panelbear.com/blog/tech-stack/) | | Percent 百分点 | Analytics | Main Product | — | — | [Slides in Chinese, June 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup24/4.%20ClickHouse万亿数据双中心的设计与实践%20.pdf) | | Percona | Performance analysis | Percona Monitoring and Management | — | — | [Official website, Mar 2020](https://www.percona.com/blog/2020/03/30/advanced-query-analysis-in-percona-monitoring-and-management-with-direct-clickhouse-access/) | @@ -92,12 +98,14 @@ toc_title: Adopters | S7 Airlines | Airlines | Metrics, Logging | — | — | [Talk in Russian, March 2019](https://www.youtube.com/watch?v=nwG68klRpPg&t=15s) | | scireum GmbH | e-Commerce | Main product | — | — | [Talk in German, February 2020](https://www.youtube.com/watch?v=7QWAn5RbyR4) | | Segment | Data processing | Main product | 9 * i3en.3xlarge nodes 7.5TB NVME SSDs, 96GB Memory, 12 vCPUs | — | [Slides, 2019](https://slides.com/abraithwaite/segment-clickhouse) | +| sembot.io | Shopping Ads | — | — | — | A comment on LinkedIn, 2020 | | SEMrush | Marketing | Main product | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/5_semrush.pdf) | | Sentry | Software Development | Main product | — | — | [Blog Post in English, May 2019](https://blog.sentry.io/2019/05/16/introducing-snuba-sentrys-new-search-infrastructure) | | seo.do | Analytics | Main product | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/CH%20Presentation-%20Metehan%20Çetinkaya.pdf) | | SGK | Goverment Social Security | Analytics | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup35/ClickHouse%20Meetup-Ramazan%20POLAT.pdf) | | Sina | News | — | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/6.%20ClickHouse最佳实践%20高鹏_新浪.pdf) | | SMI2 | News | Analytics | — | — | [Blog Post in Russian, November 2017](https://habr.com/ru/company/smi2/blog/314558/) | +| Spark New Zealand | Telecommunications | Security Operations | — | — | [Blog Post, Feb 2020](https://blog.n0p.me/2020/02/2020-02-05-dnsmonster/) | | Splunk | Business Analytics | Main product | — | — | [Slides in English, January 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup12/splunk.pdf) | | Spotify | Music | Experimentation | — | — | [Slides, July 2018](https://www.slideshare.net/glebus/using-clickhouse-for-experimentation-104247173) | | Staffcop | Information Security | Main Product | — | — | [Official website, Documentation](https://www.staffcop.ru/sce43) | @@ -106,13 +114,16 @@ toc_title: Adopters | Tencent | Big Data | Data processing | — | — | [Slides in Chinese, October 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup19/5.%20ClickHouse大数据集群应用_李俊飞腾讯网媒事业部.pdf) | | Tencent | Messaging | Logging | — | — | [Talk in Chinese, November 2019](https://youtu.be/T-iVQRuw-QY?t=5050) | | Tencent Music Entertainment (TME) | BigData | Data processing | — | — | [Blog in Chinese, June 2020](https://cloud.tencent.com/developer/article/1637840) | +| Tinybird | Real-time Data Products | Data processing | — | — | [Official website](https://www.tinybird.co/) | | Traffic Stars | AD network | — | — | — | [Slides in Russian, May 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup15/lightning/ninja.pdf) | | Uber | Taxi | Logging | — | — | [Slides, February 2020](https://presentations.clickhouse.tech/meetup40/uber.pdf) | | VKontakte | Social Network | Statistics, Logging | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/3_vk.pdf) | | Walmart Labs | Internet, Retail | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=144) | | Wargaming | Games | | — | — | [Interview](https://habr.com/en/post/496954/) | +| Wildberries | E-commerce | | — | — | [Official website](https://it.wildberries.ru/) | | Wisebits | IT Solutions | Analytics | — | — | [Slides in Russian, May 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup22/strategies.pdf) | | Workato | Automation Software | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=334) | +| Xenoss | Marketing, Advertising | — | — | — | [Instagram, March 2021](https://www.instagram.com/p/CNATV7qBgB1/) | | Xiaoxin Tech | Education | Common purpose | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/sync-clickhouse-with-mysql-mongodb.pptx) | | Ximalaya | Audio sharing | OLAP | — | — | [Slides in English, November 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup33/ximalaya.pdf) | | Yandex Cloud | Public Cloud | Main product | — | — | [Talk in Russian, December 2019](https://www.youtube.com/watch?v=pgnak9e_E0o) | @@ -122,7 +133,9 @@ toc_title: Adopters | ЦВТ | Software Development | Metrics, Logging | — | — | [Blog Post, March 2019, in Russian](https://vc.ru/dev/62715-kak-my-stroili-monitoring-na-prometheus-clickhouse-i-elk) | | МКБ | Bank | Web-system monitoring | — | — | [Slides in Russian, September 2019](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup28/mkb.pdf) | | ЦФТ | Banking, Financial products, Payments | — | — | — | [Meetup in Russian, April 2020](https://team.cft.ru/events/162) | +| Цифровой Рабочий | Industrial IoT, Analytics | — | — | — | [Blog post in Russian, March 2021](https://habr.com/en/company/croc/blog/548018/) | | kakaocorp | Internet company | — | — | — | [if(kakao)2020 conference](https://if.kakao.com/session/117) | +| ООО «МПЗ Богородский» | Agriculture | — | — | — | [Article in Russian, November 2020](https://cloud.yandex.ru/cases/okraina) | | Tesla | Electric vehicle and clean energy company | — | — | — | [Vacancy description, March 2021](https://news.ycombinator.com/item?id=26306170) | [Original article](https://clickhouse.tech/docs/en/introduction/adopters/) From 14c4cc2ccbad455af022e99b4d2d8b60fba0240d Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Mar 2021 08:19:15 +0300 Subject: [PATCH 56/60] Update ya.make --- src/Functions/ya.make | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/ya.make b/src/Functions/ya.make index e57935743eb..acabd7b3c99 100644 --- a/src/Functions/ya.make +++ b/src/Functions/ya.make @@ -210,7 +210,7 @@ SRCS( cbrt.cpp coalesce.cpp concat.cpp - connectionID.cpp + connectionId.cpp convertCharset.cpp cos.cpp cosh.cpp @@ -373,7 +373,7 @@ SRCS( now.cpp now64.cpp nullIf.cpp - partitionID.cpp + partitionId.cpp pi.cpp plus.cpp pointInEllipses.cpp From 32cca8cfb6440b88d187023ecb31f508877af1cc Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 31 Mar 2021 08:31:47 +0300 Subject: [PATCH 57/60] More --- docs/en/introduction/adopters.md | 3 +++ 1 file changed, 3 insertions(+) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index 4d84895afa3..ebc90211594 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -12,8 +12,10 @@ toc_title: Adopters |---------|----------|---------|--------------|------------------------------------------------------------------------------|-----------| | 2gis | Maps | Monitoring | — | — | [Talk in Russian, July 2019](https://youtu.be/58sPkXfq6nw) | | Admiral | Martech | Engagement Management | — | — | [Webinar Slides, June 2020](https://altinity.com/presentations/2020/06/16/big-data-in-real-time-how-clickhouse-powers-admirals-visitor-relationships-for-publishers) | +| AdScribe | Ads | TV Analytics | — | — | [A quote from CTO](https://altinity.com/24x7-support/) | | Alibaba Cloud | Cloud | Managed Service | — | — | [Official Website](https://help.aliyun.com/product/144466.html) | | Aloha Browser | Mobile App | Browser backend | — | — | [Slides in Russian, May 2019](https://presentations.clickhouse.tech/meetup22/aloha.pdf) | +| Altinity | Cloud, SaaS | Main product | — | — | [Official Website](https://altinity.com/) | | Amadeus | Travel | Analytics | — | — | [Press Release, April 2018](https://www.altinity.com/blog/2018/4/5/amadeus-technologies-launches-investment-and-insights-tool-based-on-machine-learning-and-strategy-algorithms) | | ApiRoad | API marketplace | Analytics | — | — | [Blog post, Nov 2018, Mar 2020](https://pixeljets.com/blog/clickhouse-vs-elasticsearch/) | | Appsflyer | Mobile analytics | Main product | — | — | [Talk in Russian, July 2019](https://www.youtube.com/watch?v=M3wbRlcpBbY) | @@ -118,6 +120,7 @@ toc_title: Adopters | Traffic Stars | AD network | — | — | — | [Slides in Russian, May 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup15/lightning/ninja.pdf) | | Uber | Taxi | Logging | — | — | [Slides, February 2020](https://presentations.clickhouse.tech/meetup40/uber.pdf) | | VKontakte | Social Network | Statistics, Logging | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/3_vk.pdf) | +| VMWare | Cloud | VeloCloud, SDN | — | — | [Product documentation](https://docs.vmware.com/en/vRealize-Operations-Manager/8.3/com.vmware.vcom.metrics.doc/GUID-A9AD72E1-C948-4CA2-971B-919385AB3CA8.html) | | Walmart Labs | Internet, Retail | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=144) | | Wargaming | Games | | — | — | [Interview](https://habr.com/en/post/496954/) | | Wildberries | E-commerce | | — | — | [Official website](https://it.wildberries.ru/) | From 94d3ebb5eadce50719b1717d4889dc1669887a54 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 31 Mar 2021 09:19:35 +0300 Subject: [PATCH 58/60] Update adopters.md --- docs/en/introduction/adopters.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/introduction/adopters.md b/docs/en/introduction/adopters.md index ebc90211594..2c1a077d112 100644 --- a/docs/en/introduction/adopters.md +++ b/docs/en/introduction/adopters.md @@ -120,7 +120,7 @@ toc_title: Adopters | Traffic Stars | AD network | — | — | — | [Slides in Russian, May 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup15/lightning/ninja.pdf) | | Uber | Taxi | Logging | — | — | [Slides, February 2020](https://presentations.clickhouse.tech/meetup40/uber.pdf) | | VKontakte | Social Network | Statistics, Logging | — | — | [Slides in Russian, August 2018](https://github.com/ClickHouse/clickhouse-presentations/blob/master/meetup17/3_vk.pdf) | -| VMWare | Cloud | VeloCloud, SDN | — | — | [Product documentation](https://docs.vmware.com/en/vRealize-Operations-Manager/8.3/com.vmware.vcom.metrics.doc/GUID-A9AD72E1-C948-4CA2-971B-919385AB3CA8.html) | +| VMWare | Cloud | VeloCloud, SDN | — | — | [Product documentation](https://docs.vmware.com/en/vRealize-Operations-Manager/8.3/com.vmware.vcom.metrics.doc/GUID-A9AD72E1-C948-4CA2-971B-919385AB3CA8.html) | | Walmart Labs | Internet, Retail | — | — | — | [Talk in English, July 2020](https://youtu.be/GMiXCMFDMow?t=144) | | Wargaming | Games | | — | — | [Interview](https://habr.com/en/post/496954/) | | Wildberries | E-commerce | | — | — | [Official website](https://it.wildberries.ru/) | From 9fd3cd2a212b02f38e8c48d0a9cd5b654ca80221 Mon Sep 17 00:00:00 2001 From: MyroTk Date: Wed, 31 Mar 2021 08:34:52 +0200 Subject: [PATCH 59/60] Adding memory info to testflows RBAC --- tests/testflows/rbac/helper/common.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/tests/testflows/rbac/helper/common.py b/tests/testflows/rbac/helper/common.py index c140e01f34f..a234a62eabd 100755 --- a/tests/testflows/rbac/helper/common.py +++ b/tests/testflows/rbac/helper/common.py @@ -28,6 +28,10 @@ def instrument_clickhouse_server_log(self, node=None, clickhouse_server_log="/va try: with And("adding test name start message to the clickhouse-server.log"): node.command(f"echo -e \"\\n-- start: {current().name} --\\n\" >> {clickhouse_server_log}") + with And("dump memory info"): + node.command(f"echo -e \"\\n-- {current().name} -- top --\\n\" && top -bn1") + node.command(f"echo -e \"\\n-- {current().name} -- df --\\n\" && df -h") + node.command(f"echo -e \"\\n-- {current().name} -- free --\\n\" && free -mh") yield finally: From 061e3c7d81c75788355f94ec9bf3492d73206dab Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 31 Mar 2021 18:28:58 +0800 Subject: [PATCH 60/60] Correctly place debug helpers --- cmake/warnings.cmake | 5 ----- src/CMakeLists.txt | 5 +++++ 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/cmake/warnings.cmake b/cmake/warnings.cmake index 8122e9ef31e..a398c59e981 100644 --- a/cmake/warnings.cmake +++ b/cmake/warnings.cmake @@ -11,11 +11,6 @@ if (NOT MSVC) set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} -Wextra") endif () -if (USE_DEBUG_HELPERS) - set (INCLUDE_DEBUG_HELPERS "-I${ClickHouse_SOURCE_DIR}/base -include ${ClickHouse_SOURCE_DIR}/src/Core/iostream_debug_helpers.h") - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${INCLUDE_DEBUG_HELPERS}") -endif () - # Add some warnings that are not available even with -Wall -Wextra -Wpedantic. # Intended for exploration of new compiler warnings that may be found useful. # Applies to clang only diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index 805941f7dcc..7a3d2861da7 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -27,6 +27,11 @@ configure_file (Common/config.h.in ${CONFIG_COMMON}) configure_file (Common/config_version.h.in ${CONFIG_VERSION}) configure_file (Core/config_core.h.in ${CMAKE_CURRENT_BINARY_DIR}/Core/include/config_core.h) +if (USE_DEBUG_HELPERS) + set (INCLUDE_DEBUG_HELPERS "-I${ClickHouse_SOURCE_DIR}/base -include ${ClickHouse_SOURCE_DIR}/src/Core/iostream_debug_helpers.h") + set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${INCLUDE_DEBUG_HELPERS}") +endif () + if (COMPILER_GCC) # If we leave this optimization enabled, gcc-7 replaces a pair of SSE intrinsics (16 byte load, store) with a call to memcpy. # It leads to slow code. This is compiler bug. It looks like this: