From fa8aafa94222da68791492561aaa0ee70e395249 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 22 May 2024 21:28:33 +0000 Subject: [PATCH 001/162] Local plan for parallel replicas: save --- .../ClusterProxy/executeQuery.cpp | 83 +++++++++++++++---- src/Interpreters/ClusterProxy/executeQuery.h | 9 +- src/Planner/PlannerJoinTree.cpp | 4 +- .../QueryPlan/DistributedCreateLocalPlan.cpp | 2 - .../QueryPlan/DistributedCreateLocalPlan.h | 5 -- .../QueryPlan/ParallelReplicasLocalPlan.cpp | 78 +++++++++++++++++ .../QueryPlan/ParallelReplicasLocalPlan.h | 19 +++++ src/Processors/QueryPlan/ReadFromRemote.cpp | 7 +- src/Processors/QueryPlan/ReadFromRemote.h | 4 +- 9 files changed, 182 insertions(+), 29 deletions(-) create mode 100644 src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp create mode 100644 src/Processors/QueryPlan/ParallelReplicasLocalPlan.h diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index d1701d268f1..71912fa1081 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -25,6 +25,7 @@ #include #include #include +#include namespace DB { @@ -403,7 +404,8 @@ void executeQueryWithParallelReplicas( QueryProcessingStage::Enum processed_stage, const ASTPtr & query_ast, ContextPtr context, - std::shared_ptr storage_limits) + std::shared_ptr storage_limits, + QueryPlanStepPtr read_from_merge_tree) { const auto & settings = context->getSettingsRef(); @@ -486,21 +488,66 @@ void executeQueryWithParallelReplicas( auto coordinator = std::make_shared( new_cluster->getShardsInfo().begin()->getAllNodeCount(), settings.parallel_replicas_mark_segment_size); auto external_tables = new_context->getExternalTables(); - auto read_from_remote = std::make_unique( - query_ast, - new_cluster, - storage_id, - std::move(coordinator), - header, - processed_stage, - new_context, - getThrottler(new_context), - std::move(scalars), - std::move(external_tables), - getLogger("ReadFromParallelRemoteReplicasStep"), - std::move(storage_limits)); - query_plan.addStep(std::move(read_from_remote)); + if (settings.allow_experimental_analyzer) + { + auto read_from_remote = std::make_unique( + query_ast, + new_cluster, + storage_id, + std::move(coordinator), + header, + processed_stage, + new_context, + getThrottler(new_context), + std::move(scalars), + std::move(external_tables), + getLogger("ReadFromParallelRemoteReplicasStep"), + std::move(storage_limits), + /*exclude_local_replica*/ true); + + auto remote_plan = std::make_unique(); + remote_plan->addStep(std::move(read_from_remote)); + + auto local_plan = createLocalPlanForParallelReplicas( + query_ast, + header, + new_context, + processed_stage, + coordinator, + std::move(read_from_merge_tree), + /*has_missing_objects=*/false); + + DataStreams input_streams; + input_streams.reserve(2); + input_streams.emplace_back(local_plan->getCurrentDataStream()); + input_streams.emplace_back(remote_plan->getCurrentDataStream()); + + std::vector plans; + plans.emplace_back(std::move(local_plan)); + plans.emplace_back(std::move(remote_plan)); + + auto union_step = std::make_unique(std::move(input_streams)); + query_plan.unitePlans(std::move(union_step), std::move(plans)); + } + else { + auto read_from_remote = std::make_unique( + query_ast, + new_cluster, + storage_id, + std::move(coordinator), + header, + processed_stage, + new_context, + getThrottler(new_context), + std::move(scalars), + std::move(external_tables), + getLogger("ReadFromParallelRemoteReplicasStep"), + std::move(storage_limits), + /*exclude_local_replica*/ false); + + query_plan.addStep(std::move(read_from_remote)); + } } void executeQueryWithParallelReplicas( @@ -510,7 +557,8 @@ void executeQueryWithParallelReplicas( const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context, ContextPtr context, - std::shared_ptr storage_limits) + std::shared_ptr storage_limits, + QueryPlanStepPtr read_from_merge_tree) { QueryTreeNodePtr modified_query_tree = query_tree->clone(); rewriteJoinToGlobalJoin(modified_query_tree, context); @@ -520,7 +568,8 @@ void executeQueryWithParallelReplicas( = InterpreterSelectQueryAnalyzer::getSampleBlock(modified_query_tree, context, SelectQueryOptions(processed_stage).analyze()); auto modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree); - executeQueryWithParallelReplicas(query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits); + executeQueryWithParallelReplicas( + query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits, std::move(read_from_merge_tree)); } void executeQueryWithParallelReplicas( diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index 6548edf8939..1b38d1921b1 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -30,6 +30,9 @@ using QueryTreeNodePtr = std::shared_ptr; class PlannerContext; using PlannerContextPtr = std::shared_ptr; +class IQueryPlanStep; +using QueryPlanStepPtr = std::unique_ptr; + namespace ClusterProxy { @@ -73,7 +76,8 @@ void executeQueryWithParallelReplicas( QueryProcessingStage::Enum processed_stage, const ASTPtr & query_ast, ContextPtr context, - std::shared_ptr storage_limits); + std::shared_ptr storage_limits, + QueryPlanStepPtr read_from_merge_tree = nullptr); void executeQueryWithParallelReplicas( QueryPlan & query_plan, @@ -90,7 +94,8 @@ void executeQueryWithParallelReplicas( const QueryTreeNodePtr & query_tree, const PlannerContextPtr & planner_context, ContextPtr context, - std::shared_ptr storage_limits); + std::shared_ptr storage_limits, + QueryPlanStepPtr read_from_merge_tree); } } diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index a6e4a8ebcde..275461fa109 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -934,6 +934,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres { from_stage = QueryProcessingStage::WithMergeableState; QueryPlan query_plan_parallel_replicas; + QueryPlanStepPtr reading_step = std::move(node->step); ClusterProxy::executeQueryWithParallelReplicas( query_plan_parallel_replicas, storage->getStorageID(), @@ -941,7 +942,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres table_expression_query_info.query_tree, table_expression_query_info.planner_context, query_context, - table_expression_query_info.storage_limits); + table_expression_query_info.storage_limits, + std::move(reading_step)); query_plan = std::move(query_plan_parallel_replicas); const Block & query_plan_header = query_plan.getCurrentDataStream().header; diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index d4545482477..1f4f271fa6e 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -1,8 +1,6 @@ #include -#include #include -#include #include #include #include diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h index 50545d9ae81..f59123a7d88 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.h +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.h @@ -1,17 +1,12 @@ #pragma once #include -#include #include #include -#include namespace DB { -class PreparedSets; -using PreparedSetsPtr = std::shared_ptr; - std::unique_ptr createLocalPlan( const ASTPtr & query_ast, const Block & header, diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp new file mode 100644 index 00000000000..4d78e049b58 --- /dev/null +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -0,0 +1,78 @@ +#include + +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace +{ + +void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects) +{ + if (blocksHaveEqualStructure(plan.getCurrentDataStream().header, header)) + return; + + auto mode = has_missing_objects ? ActionsDAG::MatchColumnsMode::Position : ActionsDAG::MatchColumnsMode::Name; + + auto get_converting_dag = [mode](const Block & block_, const Block & header_) + { + /// Convert header structure to expected. + /// Also we ignore constants from result and replace it with constants from header. + /// It is needed for functions like `now64()` or `randConstant()` because their values may be different. + return ActionsDAG::makeConvertingActions( + block_.getColumnsWithTypeAndName(), + header_.getColumnsWithTypeAndName(), + mode, + true); + }; + + auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header); + auto converting = std::make_unique(plan.getCurrentDataStream(), convert_actions_dag); + plan.addStep(std::move(converting)); +} + +} + +std::unique_ptr createLocalPlanForParallelReplicas( + const ASTPtr & query_ast, + const Block & header, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + ParallelReplicasReadingCoordinatorPtr /*coordinator*/, + QueryPlanStepPtr /*read_from_merge_tree*/, + bool has_missing_objects) +{ + checkStackSize(); + + auto query_plan = std::make_unique(); + auto new_context = Context::createCopy(context); + + /// Do not push down limit to local plan, as it will break `rows_before_limit_at_least` counter. + if (processed_stage == QueryProcessingStage::WithMergeableStateAfterAggregationAndLimit) + processed_stage = QueryProcessingStage::WithMergeableStateAfterAggregation; + + /// Do not apply AST optimizations, because query + /// is already optimized and some optimizations + /// can be applied only for non-distributed tables + /// and we can produce query, inconsistent with remote plans. + auto select_query_options = SelectQueryOptions(processed_stage).ignoreASTOptimizations(); + + /// For Analyzer, identifier in GROUP BY/ORDER BY/LIMIT BY lists has been resolved to + /// ConstantNode in QueryTree if it is an alias of a constant, so we should not replace + /// ConstantNode with ProjectionNode again(https://github.com/ClickHouse/ClickHouse/issues/62289). + new_context->setSetting("enable_positional_arguments", Field(false)); + auto interpreter = InterpreterSelectQueryAnalyzer(query_ast, new_context, select_query_options); + query_plan = std::make_unique(std::move(interpreter).extractQueryPlan()); + + addConvertingActions(*query_plan, header, has_missing_objects); + return query_plan; +} + +} diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h new file mode 100644 index 00000000000..89d2019f807 --- /dev/null +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h @@ -0,0 +1,19 @@ +#pragma once + +#include +#include +#include +#include + +namespace DB +{ + +std::unique_ptr createLocalPlanForParallelReplicas( + const ASTPtr & query_ast, + const Block & header, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + ParallelReplicasReadingCoordinatorPtr coordinator, + QueryPlanStepPtr read_from_merge_tree, + bool has_missing_objects); +} diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index b4e35af85d6..6e6edfa1208 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -369,7 +369,8 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( Scalars scalars_, Tables external_tables_, LoggerPtr log_, - std::shared_ptr storage_limits_) + std::shared_ptr storage_limits_, + bool exclude_local_replica_) : ISourceStep(DataStream{.header = std::move(header_)}) , cluster(cluster_) , query_ast(query_ast_) @@ -382,6 +383,7 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( , external_tables{external_tables_} , storage_limits(std::move(storage_limits_)) , log(log_) + , exclude_local_replica(exclude_local_replica_) { chassert(cluster->getShardCount() == 1); @@ -410,6 +412,9 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder const auto & shard = cluster->getShardsInfo().at(0); size_t all_replicas_count = current_settings.max_parallel_replicas; + if (exclude_local_replica) + --all_replicas_count; + if (all_replicas_count > shard.getAllNodeCount()) { LOG_INFO( diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index eb15269155a..442da098a17 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -78,7 +78,8 @@ public: Scalars scalars_, Tables external_tables_, LoggerPtr log_, - std::shared_ptr storage_limits_); + std::shared_ptr storage_limits_, + bool exclude_local_replica = false); String getName() const override { return "ReadFromRemoteParallelReplicas"; } @@ -101,6 +102,7 @@ private: Tables external_tables; std::shared_ptr storage_limits; LoggerPtr log; + bool exclude_local_replica; }; } From be08ebd0f4755c475385fa3460c32eaea6f71312 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 24 May 2024 13:01:30 +0000 Subject: [PATCH 002/162] Fix --- src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index 4d78e049b58..64646960824 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -68,6 +68,7 @@ std::unique_ptr createLocalPlanForParallelReplicas( /// ConstantNode in QueryTree if it is an alias of a constant, so we should not replace /// ConstantNode with ProjectionNode again(https://github.com/ClickHouse/ClickHouse/issues/62289). new_context->setSetting("enable_positional_arguments", Field(false)); + new_context->setSetting("allow_experimental_parallel_reading_from_replicas", Field(0)); auto interpreter = InterpreterSelectQueryAnalyzer(query_ast, new_context, select_query_options); query_plan = std::make_unique(std::move(interpreter).extractQueryPlan()); From 5cdf8d336cf509f7af1031a1b9a856d06b5f2ac2 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 27 May 2024 12:22:18 +0000 Subject: [PATCH 003/162] Local reading step from merge tree --- .../ClusterProxy/executeQuery.cpp | 2 +- .../QueryPlan/ParallelReplicasLocalPlan.cpp | 217 +++++++++++++++++- .../QueryPlan/ReadFromMergeTree.cpp | 45 +++- src/Processors/QueryPlan/ReadFromMergeTree.h | 12 +- src/Storages/MergeTree/RequestResponse.h | 1 - 5 files changed, 264 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 71912fa1081..e12e531ab51 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -495,7 +495,7 @@ void executeQueryWithParallelReplicas( query_ast, new_cluster, storage_id, - std::move(coordinator), + coordinator, header, processed_stage, new_context, diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index 64646960824..2bfd8965269 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -1,12 +1,32 @@ #include #include +#include "Storages/MergeTree/RequestResponse.h" #include #include #include #include #include #include +#include +#include +#include +#include +#include +#include +#include +#include + +#include + +#include + +namespace ProfileEvents +{ + extern const Event SelectedParts; + extern const Event SelectedRanges; + extern const Event SelectedMarks; +} namespace DB { @@ -40,13 +60,179 @@ void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missi } +class ReadFromMergeTreeCoordinated : public ISourceStep +{ +public: + ReadFromMergeTreeCoordinated(QueryPlanStepPtr read_from_merge_tree_, ParallelReplicasReadingCoordinatorPtr coordinator_) + : ISourceStep(read_from_merge_tree_->getOutputStream()) + , read_from_merge_tree(std::move(read_from_merge_tree_)) + , coordinator(std::move(coordinator_)) + { + } + + void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; + String getName() const override { return "ReadFromLocalParallelReplica"; } + +private: + QueryPlanStepPtr read_from_merge_tree; + ParallelReplicasReadingCoordinatorPtr coordinator; +}; + +void ReadFromMergeTreeCoordinated::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & /*settings*/) +{ + ReadFromMergeTree & reading = *typeid_cast(read_from_merge_tree.get()); + + auto result = reading.getAnalysisResult(); + const auto & query_info = reading.getQueryInfo(); + const auto & data = reading.data; + const auto & context = reading.getContext(); + const auto & storage_snapshot = reading.getStorageSnapshot(); + + if (reading.enable_remove_parts_from_snapshot_optimization) + { + /// Do not keep data parts in snapshot. + /// They are stored separately, and some could be released after PK analysis. + reading.storage_snapshot->data = std::make_unique(); + } + + LOG_DEBUG( + reading.log, + "Selected {}/{} parts by partition key, {} parts by primary key, {}/{} marks by primary key, {} marks to read from {} ranges", + result.parts_before_pk, + result.total_parts, + result.selected_parts, + result.selected_marks_pk, + result.total_marks_pk, + result.selected_marks, + result.selected_ranges); + + // Adding partition info to QueryAccessInfo. + if (context->hasQueryContext() && !query_info.is_internal) + { + Names partition_names; + for (const auto & part : result.parts_with_ranges) + { + partition_names.emplace_back( + fmt::format("{}.{}", data.getStorageID().getFullNameNotQuoted(), part.data_part->info.partition_id)); + } + context->getQueryContext()->addQueryAccessInfo(partition_names); + + if (storage_snapshot->projection) + context->getQueryContext()->addQueryAccessInfo( + Context::QualifiedProjectionName{.storage_id = data.getStorageID(), .projection_name = storage_snapshot->projection->name}); + } + + ProfileEvents::increment(ProfileEvents::SelectedParts, result.selected_parts); + ProfileEvents::increment(ProfileEvents::SelectedRanges, result.selected_ranges); + ProfileEvents::increment(ProfileEvents::SelectedMarks, result.selected_marks); + + auto query_id_holder = MergeTreeDataSelectExecutor::checkLimits(data, result, context); + + // TODO: check this on plan level, we should be here if there is nothing to read + if (result.parts_with_ranges.empty()) + { + pipeline.init(Pipe(std::make_shared(getOutputStream().header))); + return; + } + + /// Projection, that needed to drop columns, which have appeared by execution + /// of some extra expressions, and to allow execute the same expressions later. + /// NOTE: It may lead to double computation of expressions. + ActionsDAGPtr result_projection; + + Pipe pipe = reading.spreadMarkRanges(std::move(result.parts_with_ranges), reading.requested_num_streams, result, result_projection); + + for (const auto & processor : pipe.getProcessors()) + processor->setStorageLimits(query_info.storage_limits); + + if (pipe.empty()) + { + pipeline.init(Pipe(std::make_shared(getOutputStream().header))); + return; + } + + if (result.sampling.use_sampling) + { + auto sampling_actions = std::make_shared(result.sampling.filter_expression); + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared( + header, + sampling_actions, + result.sampling.filter_function->getColumnName(), + false); + }); + } + + Block cur_header = pipe.getHeader(); + + auto append_actions = [&result_projection](ActionsDAGPtr actions) + { + if (!result_projection) + result_projection = std::move(actions); + else + result_projection = ActionsDAG::merge(std::move(*result_projection), std::move(*actions)); + }; + + if (result_projection) + cur_header = result_projection->updateHeader(cur_header); + + /// Extra columns may be returned (for example, if sampling is used). + /// Convert pipe to step header structure. + if (!isCompatibleHeader(cur_header, getOutputStream().header)) + { + auto converting = ActionsDAG::makeConvertingActions( + cur_header.getColumnsWithTypeAndName(), + getOutputStream().header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name); + + append_actions(std::move(converting)); + } + + if (result_projection) + { + auto projection_actions = std::make_shared(result_projection); + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, projection_actions); + }); + } + + /// Some extra columns could be added by sample/final/in-order/etc + /// Remove them from header if not needed. + if (!blocksHaveEqualStructure(pipe.getHeader(), getOutputStream().header)) + { + auto convert_actions_dag = ActionsDAG::makeConvertingActions( + pipe.getHeader().getColumnsWithTypeAndName(), + getOutputStream().header.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Name, + true); + + auto converting_dag_expr = std::make_shared(convert_actions_dag); + + pipe.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, converting_dag_expr); + }); + } + + for (const auto & processor : pipe.getProcessors()) + processors.emplace_back(processor); + + pipeline.init(std::move(pipe)); + pipeline.addContext(context); + // Attach QueryIdHolder if needed + if (query_id_holder) + pipeline.setQueryIdHolder(std::move(query_id_holder)); +} + std::unique_ptr createLocalPlanForParallelReplicas( const ASTPtr & query_ast, const Block & header, ContextPtr context, QueryProcessingStage::Enum processed_stage, - ParallelReplicasReadingCoordinatorPtr /*coordinator*/, - QueryPlanStepPtr /*read_from_merge_tree*/, + ParallelReplicasReadingCoordinatorPtr coordinator, + QueryPlanStepPtr read_from_merge_tree, bool has_missing_objects) { checkStackSize(); @@ -72,6 +258,33 @@ std::unique_ptr createLocalPlanForParallelReplicas( auto interpreter = InterpreterSelectQueryAnalyzer(query_ast, new_context, select_query_options); query_plan = std::make_unique(std::move(interpreter).extractQueryPlan()); + QueryPlan::Node * node = query_plan->getRootNode(); + ReadFromMergeTree * reading = nullptr; + while (node) + { + reading = typeid_cast(node->step.get()); + if (reading) + break; + + if (!node->children.empty()) + node = node->children.at(0); + } + + chassert(reading); + + MergeTreeAllRangesCallback all_ranges_cb = [coordinator](InitialAllRangesAnnouncement announcement) + { + chassert(coordinator); + coordinator->handleInitialAllRangesAnnouncement(std::move(announcement)); + }; + + MergeTreeReadTaskCallback read_task_cb = [coordinator](ParallelReadRequest req) -> std::optional + { return coordinator->handleRequest(std::move(req)); }; + + const auto * analyzed_merge_tree = typeid_cast(read_from_merge_tree.get()); + auto read_from_merge_tree_parallel_replicas = reading->createLocalParallelReplicasReadingStep(analyzed_merge_tree, true, all_ranges_cb, read_task_cb); + node->step = std::move(read_from_merge_tree_parallel_replicas); + addConvertingActions(*query_plan, header, has_missing_objects); return query_plan; } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 6f0fa55c349..21303cf2af2 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -272,7 +272,9 @@ ReadFromMergeTree::ReadFromMergeTree( std::shared_ptr max_block_numbers_to_read_, LoggerPtr log_, AnalysisResultPtr analyzed_result_ptr_, - bool enable_parallel_reading) + bool enable_parallel_reading_, + std::optional all_ranges_callback_, + std::optional read_task_callback_) : SourceStepWithFilter(DataStream{.header = MergeTreeSelectProcessor::transformHeader( storage_snapshot_->getSampleBlockForColumns(all_column_names_), query_info_.prewhere_info)}, all_column_names_, query_info_, storage_snapshot_, context_) @@ -291,13 +293,20 @@ ReadFromMergeTree::ReadFromMergeTree( , max_block_numbers_to_read(std::move(max_block_numbers_to_read_)) , log(std::move(log_)) , analyzed_result_ptr(analyzed_result_ptr_) - , is_parallel_reading_from_replicas(enable_parallel_reading) + , is_parallel_reading_from_replicas(enable_parallel_reading_) , enable_remove_parts_from_snapshot_optimization(query_info_.merge_tree_enable_remove_parts_from_snapshot_optimization) { if (is_parallel_reading_from_replicas) { - all_ranges_callback = context->getMergeTreeAllRangesCallback(); - read_task_callback = context->getMergeTreeReadTaskCallback(); + if (all_ranges_callback_) + all_ranges_callback = all_ranges_callback_.value(); + else + all_ranges_callback = context->getMergeTreeAllRangesCallback(); + + if (read_task_callback_) + read_task_callback = read_task_callback_.value(); + else + read_task_callback = context->getMergeTreeReadTaskCallback(); } const auto & settings = context->getSettingsRef(); @@ -331,11 +340,31 @@ ReadFromMergeTree::ReadFromMergeTree( enable_vertical_final); } +std::unique_ptr ReadFromMergeTree::createLocalParallelReplicasReadingStep( + const ReadFromMergeTree * analyzed_merge_tree, + bool enable_parallel_reading_, + std::optional all_ranges_callback_, + std::optional read_task_callback_) +{ + return std::make_unique( + prepared_parts, + alter_conversions_for_parts, + all_column_names, + data, + getQueryInfo(), + getStorageSnapshot(), + getContext(), + block_size.max_block_size_rows, + requested_num_streams, + max_block_numbers_to_read, + log, + analyzed_merge_tree->analyzed_result_ptr, + enable_parallel_reading_, + all_ranges_callback_, + read_task_callback_); +} -Pipe ReadFromMergeTree::readFromPoolParallelReplicas( - RangesInDataParts parts_with_range, - Names required_columns, - PoolSettings pool_settings) +Pipe ReadFromMergeTree::readFromPoolParallelReplicas(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings) { const auto & client_info = context->getClientInfo(); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 5d7879e8dee..bba5293e863 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -119,7 +119,15 @@ public: std::shared_ptr max_block_numbers_to_read_, LoggerPtr log_, AnalysisResultPtr analyzed_result_ptr_, - bool enable_parallel_reading); + bool enable_parallel_reading_, + std::optional all_ranges_callback_ = std::nullopt, + std::optional read_task_callback_ = std::nullopt); + + std::unique_ptr createLocalParallelReplicasReadingStep( + const ReadFromMergeTree * analyzed_merge_tree, + bool enable_parallel_reading_, + std::optional all_ranges_callback_, + std::optional read_task_callback_); static constexpr auto name = "ReadFromMergeTree"; String getName() const override { return name; } @@ -282,6 +290,8 @@ private: std::optional read_task_callback; bool enable_vertical_final = false; bool enable_remove_parts_from_snapshot_optimization = true; + + friend class ReadFromMergeTreeCoordinated; }; } diff --git a/src/Storages/MergeTree/RequestResponse.h b/src/Storages/MergeTree/RequestResponse.h index 3a5bfde6c20..5f5516a6804 100644 --- a/src/Storages/MergeTree/RequestResponse.h +++ b/src/Storages/MergeTree/RequestResponse.h @@ -1,6 +1,5 @@ #pragma once -#include #include #include From 29346f607398dba5cb7796d53dc525fd937af104 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 28 May 2024 12:16:18 +0000 Subject: [PATCH 004/162] Init coordinator separately --- src/Planner/PlannerJoinTree.cpp | 9 +++++--- .../QueryPlan/ParallelReplicasLocalPlan.cpp | 23 ++++++++++++++++++- src/Processors/QueryPlan/ReadFromMergeTree.h | 6 +++++ .../ParallelReplicasReadingCoordinator.h | 2 +- 4 files changed, 35 insertions(+), 5 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 275461fa109..01455d4b955 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -892,16 +892,19 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres if (!node->children.empty()) node = node->children.at(0); + else + node = nullptr; } chassert(reading); + auto result_ptr = reading->selectRangesToRead(); + UInt64 rows_to_read = result_ptr->selected_rows; + reading->setAnalyzedResult(std::move(result_ptr)); + // (2) if it's ReadFromMergeTree - run index analysis and check number of rows to read if (settings.parallel_replicas_min_number_of_rows_per_replica > 0) { - auto result_ptr = reading->selectRangesToRead(); - - UInt64 rows_to_read = result_ptr->selected_rows; if (table_expression_query_info.limit > 0 && table_expression_query_info.limit < rows_to_read) rows_to_read = table_expression_query_info.limit; diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index 2bfd8965269..1c5e0861530 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -268,10 +268,32 @@ std::unique_ptr createLocalPlanForParallelReplicas( if (!node->children.empty()) node = node->children.at(0); + else + node = nullptr; } chassert(reading); + const auto * analyzed_merge_tree = typeid_cast(read_from_merge_tree.get()); + if (!analyzed_merge_tree->hasAnalyzedResult()) + analyzed_merge_tree->selectRangesToRead(); + + switch (analyzed_merge_tree->getReadType()) + { + case ReadFromMergeTree::ReadType::Default: + coordinator->initialize(CoordinationMode::Default); + break; + case ReadFromMergeTree::ReadType::InOrder: + coordinator->initialize(CoordinationMode::WithOrder); + break; + case ReadFromMergeTree::ReadType::InReverseOrder: + coordinator->initialize(CoordinationMode::ReverseOrder); + break; + case ReadFromMergeTree::ReadType::ParallelReplicas: + chassert(false); + UNREACHABLE(); + } + MergeTreeAllRangesCallback all_ranges_cb = [coordinator](InitialAllRangesAnnouncement announcement) { chassert(coordinator); @@ -281,7 +303,6 @@ std::unique_ptr createLocalPlanForParallelReplicas( MergeTreeReadTaskCallback read_task_cb = [coordinator](ParallelReadRequest req) -> std::optional { return coordinator->handleRequest(std::move(req)); }; - const auto * analyzed_merge_tree = typeid_cast(read_from_merge_tree.get()); auto read_from_merge_tree_parallel_replicas = reading->createLocalParallelReplicasReadingStep(analyzed_merge_tree, true, all_ranges_cb, read_task_cb); node->step = std::move(read_from_merge_tree_parallel_replicas); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index bba5293e863..adc3818c3ab 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -203,6 +203,12 @@ public: void applyFilters(ActionDAGNodes added_filter_nodes) override; + ReadType getReadType() const + { + chassert(analyzed_result_ptr); + return analyzed_result_ptr->read_type; + } + private: static AnalysisResultPtr selectRangesToReadImpl( MergeTreeData::DataPartsVector parts, diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h index 60343988f03..ff72decbf8d 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h @@ -30,8 +30,8 @@ public: /// needed to report total rows to read void setProgressCallback(ProgressCallback callback); -private: void initialize(CoordinationMode mode); +private: std::mutex mutex; size_t replicas_count{0}; From 1aa5d70d2a65b860f0d469a6c7c00b9f4129b07d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 28 May 2024 13:03:24 +0000 Subject: [PATCH 005/162] Fix style --- src/Interpreters/ClusterProxy/executeQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index e12e531ab51..3d08219155f 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -530,7 +530,8 @@ void executeQueryWithParallelReplicas( auto union_step = std::make_unique(std::move(input_streams)); query_plan.unitePlans(std::move(union_step), std::move(plans)); } - else { + else + { auto read_from_remote = std::make_unique( query_ast, new_cluster, From a9b485a2c1b8eb85fa04a5fef50b1258b0b63102 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 28 May 2024 19:05:09 +0000 Subject: [PATCH 006/162] Disable temporary PR inorder test --- .../02898_parallel_replicas_progress_bar.reference | 6 ------ .../02898_parallel_replicas_progress_bar.sql | 10 +++++----- 2 files changed, 5 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.reference b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.reference index c66597436f3..380aac4dbe8 100644 --- a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.reference +++ b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.reference @@ -1,8 +1,2 @@ 3000 1000 3999 2499.5 1 -1998 2944475297004403859 -1999 254596732598015005 -2000 6863370867519437063 -2001 17844331710293705251 -2002 1587587338113897332 -1 diff --git a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql index d8bfec12b3a..42f8091db08 100644 --- a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql +++ b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql @@ -26,12 +26,12 @@ WHERE query_id in (select query_id from system.query_log where current_database AND message LIKE '%Total rows to read: 3000%' SETTINGS allow_experimental_parallel_reading_from_replicas=0; -- reading in order coordinator -SELECT k, sipHash64(v) FROM t1 order by k limit 5 offset 998 SETTINGS optimize_read_in_order=1, log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b'; +-- SELECT k, sipHash64(v) FROM t1 order by k limit 5 offset 998 SETTINGS optimize_read_in_order=1, log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b'; -SYSTEM FLUSH LOGS; -SELECT count() > 0 FROM system.text_log -WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b') - AND message LIKE '%Updated total rows to read: added % rows, total 3000 rows%' SETTINGS allow_experimental_parallel_reading_from_replicas=0; +-- SYSTEM FLUSH LOGS; +-- SELECT count() > 0 FROM system.text_log +-- WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b') +-- AND message LIKE '%Updated total rows to read: added % rows, total 3000 rows%' SETTINGS allow_experimental_parallel_reading_from_replicas=0; DROP TABLE t1 SYNC; DROP TABLE t2 SYNC; From d756729f384ff9a6fbf93f12934ed0f98b54b7a6 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 28 May 2024 21:04:33 +0000 Subject: [PATCH 007/162] Use local working set for parallel replicas --- .../QueryPlan/ParallelReplicasLocalPlan.cpp | 42 +++++++++---------- .../QueryPlan/ReadFromMergeTree.cpp | 29 +++++-------- src/Processors/QueryPlan/ReadFromMergeTree.h | 10 +++-- src/Processors/QueryPlan/ReadFromRemote.cpp | 19 ++++++--- .../MergeTree/MergeTreeSelectProcessor.h | 5 --- 5 files changed, 52 insertions(+), 53 deletions(-) diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index 1c5e0861530..48184cb2b5f 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -273,37 +273,37 @@ std::unique_ptr createLocalPlanForParallelReplicas( } chassert(reading); - const auto * analyzed_merge_tree = typeid_cast(read_from_merge_tree.get()); - if (!analyzed_merge_tree->hasAnalyzedResult()) - analyzed_merge_tree->selectRangesToRead(); + chassert(analyzed_merge_tree->hasAnalyzedResult()); + CoordinationMode mode = CoordinationMode::Default; switch (analyzed_merge_tree->getReadType()) { - case ReadFromMergeTree::ReadType::Default: - coordinator->initialize(CoordinationMode::Default); - break; - case ReadFromMergeTree::ReadType::InOrder: - coordinator->initialize(CoordinationMode::WithOrder); - break; - case ReadFromMergeTree::ReadType::InReverseOrder: - coordinator->initialize(CoordinationMode::ReverseOrder); - break; - case ReadFromMergeTree::ReadType::ParallelReplicas: - chassert(false); - UNREACHABLE(); + case ReadFromMergeTree::ReadType::Default: + mode = CoordinationMode::Default; + break; + case ReadFromMergeTree::ReadType::InOrder: + mode = CoordinationMode::WithOrder; + break; + case ReadFromMergeTree::ReadType::InReverseOrder: + mode = CoordinationMode::ReverseOrder; + break; + case ReadFromMergeTree::ReadType::ParallelReplicas: + chassert(false); + UNREACHABLE(); } - MergeTreeAllRangesCallback all_ranges_cb = [coordinator](InitialAllRangesAnnouncement announcement) - { - chassert(coordinator); - coordinator->handleInitialAllRangesAnnouncement(std::move(announcement)); - }; + const auto number_of_local_replica = new_context->getSettingsRef().max_parallel_replicas - 1; + coordinator->handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement( + mode, analyzed_merge_tree->getAnalysisResult().parts_with_ranges.getDescriptions(), number_of_local_replica)); + + MergeTreeAllRangesCallback all_ranges_cb = [coordinator](InitialAllRangesAnnouncement) {}; MergeTreeReadTaskCallback read_task_cb = [coordinator](ParallelReadRequest req) -> std::optional { return coordinator->handleRequest(std::move(req)); }; - auto read_from_merge_tree_parallel_replicas = reading->createLocalParallelReplicasReadingStep(analyzed_merge_tree, true, all_ranges_cb, read_task_cb); + auto read_from_merge_tree_parallel_replicas + = reading->createLocalParallelReplicasReadingStep(analyzed_merge_tree, true, all_ranges_cb, read_task_cb, number_of_local_replica); node->step = std::move(read_from_merge_tree_parallel_replicas); addConvertingActions(*query_plan, header, has_missing_objects); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 198236c4f49..e3e09673431 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -273,8 +273,9 @@ ReadFromMergeTree::ReadFromMergeTree( LoggerPtr log_, AnalysisResultPtr analyzed_result_ptr_, bool enable_parallel_reading_, - std::optional all_ranges_callback_, - std::optional read_task_callback_) + std::optional all_ranges_callback_, + std::optional read_task_callback_, + std::optional number_of_current_replica_) : SourceStepWithFilter(DataStream{.header = MergeTreeSelectProcessor::transformHeader( storage_snapshot_->getSampleBlockForColumns(all_column_names_), query_info_.prewhere_info)}, all_column_names_, query_info_, storage_snapshot_, context_) @@ -295,18 +296,12 @@ ReadFromMergeTree::ReadFromMergeTree( , analyzed_result_ptr(analyzed_result_ptr_) , is_parallel_reading_from_replicas(enable_parallel_reading_) , enable_remove_parts_from_snapshot_optimization(query_info_.merge_tree_enable_remove_parts_from_snapshot_optimization) + , number_of_current_replica(number_of_current_replica_) { if (is_parallel_reading_from_replicas) { - if (all_ranges_callback_) - all_ranges_callback = all_ranges_callback_.value(); - else - all_ranges_callback = context->getMergeTreeAllRangesCallback(); - - if (read_task_callback_) - read_task_callback = read_task_callback_.value(); - else - read_task_callback = context->getMergeTreeReadTaskCallback(); + all_ranges_callback = all_ranges_callback_.value_or(context->getMergeTreeAllRangesCallback()); + read_task_callback = read_task_callback_.value_or(context->getMergeTreeReadTaskCallback()); } const auto & settings = context->getSettingsRef(); @@ -344,7 +339,8 @@ std::unique_ptr ReadFromMergeTree::createLocalParallelReplica const ReadFromMergeTree * analyzed_merge_tree, bool enable_parallel_reading_, std::optional all_ranges_callback_, - std::optional read_task_callback_) + std::optional read_task_callback_, + std::optional number_of_current_replica_) { return std::make_unique( prepared_parts, @@ -361,7 +357,8 @@ std::unique_ptr ReadFromMergeTree::createLocalParallelReplica analyzed_merge_tree->analyzed_result_ptr, enable_parallel_reading_, all_ranges_callback_, - read_task_callback_); + read_task_callback_, + number_of_current_replica_); } Pipe ReadFromMergeTree::readFromPoolParallelReplicas(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings) @@ -372,9 +369,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas(RangesInDataParts parts_wit { .all_callback = all_ranges_callback.value(), .callback = read_task_callback.value(), - .count_participating_replicas = client_info.count_participating_replicas, - .number_of_current_replica = client_info.number_of_current_replica, - .columns_to_read = required_columns, + .number_of_current_replica = number_of_current_replica.value_or(client_info.number_of_current_replica), }; /// We have a special logic for local replica. It has to read less data, because in some cases it should @@ -545,9 +540,7 @@ Pipe ReadFromMergeTree::readInOrder( { .all_callback = all_ranges_callback.value(), .callback = read_task_callback.value(), - .count_participating_replicas = client_info.count_participating_replicas, .number_of_current_replica = client_info.number_of_current_replica, - .columns_to_read = required_columns, }; const auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index adc3818c3ab..8cabc6822c1 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -121,13 +121,15 @@ public: AnalysisResultPtr analyzed_result_ptr_, bool enable_parallel_reading_, std::optional all_ranges_callback_ = std::nullopt, - std::optional read_task_callback_ = std::nullopt); + std::optional read_task_callback_ = std::nullopt, + std::optional number_of_current_replica_ = std::nullopt); std::unique_ptr createLocalParallelReplicasReadingStep( const ReadFromMergeTree * analyzed_merge_tree, bool enable_parallel_reading_, std::optional all_ranges_callback_, - std::optional read_task_callback_); + std::optional read_task_callback_, + std::optional number_of_current_replica_); static constexpr auto name = "ReadFromMergeTree"; String getName() const override { return name; } @@ -192,6 +194,7 @@ public: bool hasAnalyzedResult() const { return analyzed_result_ptr != nullptr; } void setAnalyzedResult(AnalysisResultPtr analyzed_result_ptr_) { analyzed_result_ptr = std::move(analyzed_result_ptr_); } + ReadFromMergeTree::AnalysisResult getAnalysisResult() const; const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; } const std::vector & getAlterConvertionsForParts() const { return alter_conversions_for_parts; } @@ -286,8 +289,6 @@ private: Pipe spreadMarkRangesAmongStreamsFinal( RangesInDataParts && parts, size_t num_streams, const Names & origin_column_names, const Names & column_names, ActionsDAGPtr & out_projection); - ReadFromMergeTree::AnalysisResult getAnalysisResult() const; - AnalysisResultPtr analyzed_result_ptr; VirtualFields shared_virtual_fields; @@ -296,6 +297,7 @@ private: std::optional read_task_callback; bool enable_vertical_final = false; bool enable_remove_parts_from_snapshot_optimization = true; + std::optional number_of_current_replica; friend class ReadFromMergeTreeCoordinated; }; diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 485d5f675ab..23d8f2b496f 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -390,8 +390,18 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( std::vector description; description.push_back(fmt::format("query: {}", formattedAST(query_ast))); - for (const auto & pool : cluster->getShardsInfo().front().per_replica_pools) - description.push_back(fmt::format("Replica: {}", pool->getHost())); + bool first_local = false; + for (const auto & addr : cluster->getShardsAddresses().front()) + { + /// skip first local + if (exclude_local_replica && addr.is_local && !first_local) + { + first_local = true; + continue; + } + + description.push_back(fmt::format("Replica: {}", addr.host_name)); + } setStepDescription(boost::algorithm::join(description, ", ")); } @@ -414,9 +424,6 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder const auto & shard = cluster->getShardsInfo().at(0); size_t all_replicas_count = current_settings.max_parallel_replicas; - if (exclude_local_replica) - --all_replicas_count; - if (all_replicas_count > shard.getAllNodeCount()) { LOG_INFO( @@ -427,6 +434,8 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder shard.getAllNodeCount()); all_replicas_count = shard.getAllNodeCount(); } + if (exclude_local_replica) + --all_replicas_count; std::vector shuffled_pool; if (all_replicas_count < shard.getAllNodeCount()) diff --git a/src/Storages/MergeTree/MergeTreeSelectProcessor.h b/src/Storages/MergeTree/MergeTreeSelectProcessor.h index 8f41f5deacb..03ca30dd5b3 100644 --- a/src/Storages/MergeTree/MergeTreeSelectProcessor.h +++ b/src/Storages/MergeTree/MergeTreeSelectProcessor.h @@ -26,12 +26,7 @@ struct ParallelReadingExtension { MergeTreeAllRangesCallback all_callback; MergeTreeReadTaskCallback callback; - size_t count_participating_replicas{0}; size_t number_of_current_replica{0}; - /// This is needed to estimate the number of bytes - /// between a pair of marks to perform one request - /// over the network for a 1Gb of data. - Names columns_to_read; }; /// Base class for MergeTreeThreadSelectAlgorithm and MergeTreeSelectAlgorithm From 40aab93db123840042a0f5f4703ae26f4e2ab507 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 29 May 2024 20:55:02 +0000 Subject: [PATCH 008/162] Correct replica id for inorder case --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index e3e09673431..9516072f269 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -540,7 +540,7 @@ Pipe ReadFromMergeTree::readInOrder( { .all_callback = all_ranges_callback.value(), .callback = read_task_callback.value(), - .number_of_current_replica = client_info.number_of_current_replica, + .number_of_current_replica = number_of_current_replica.value_or(client_info.number_of_current_replica), }; const auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; From e8a1a800dcdc44001c6c9f08f78390df9b3133e3 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 29 May 2024 20:57:16 +0000 Subject: [PATCH 009/162] Fix crash with JOINs --- src/Interpreters/ClusterProxy/executeQuery.cpp | 8 ++++---- src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp | 9 ++++++++- 2 files changed, 12 insertions(+), 5 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 464ce2ec586..0eeae112062 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -405,7 +405,7 @@ void executeQueryWithParallelReplicas( const ASTPtr & query_ast, ContextPtr context, std::shared_ptr storage_limits, - QueryPlanStepPtr read_from_merge_tree) + QueryPlanStepPtr analyzed_read_from_merge_tree) { auto logger = getLogger("executeQueryWithParallelReplicas"); LOG_DEBUG(logger, "Executing read from {}, header {}, query ({}), stage {} with parallel replicas", @@ -519,7 +519,7 @@ void executeQueryWithParallelReplicas( new_context, processed_stage, coordinator, - std::move(read_from_merge_tree), + std::move(analyzed_read_from_merge_tree), /*has_missing_objects=*/false); DataStreams input_streams; @@ -563,7 +563,7 @@ void executeQueryWithParallelReplicas( const PlannerContextPtr & planner_context, ContextPtr context, std::shared_ptr storage_limits, - QueryPlanStepPtr read_from_merge_tree) + QueryPlanStepPtr analyzed_read_from_merge_tree) { QueryTreeNodePtr modified_query_tree = query_tree->clone(); rewriteJoinToGlobalJoin(modified_query_tree, context); @@ -574,7 +574,7 @@ void executeQueryWithParallelReplicas( auto modified_query_ast = queryNodeToDistributedSelectQuery(modified_query_tree); executeQueryWithParallelReplicas( - query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits, std::move(read_from_merge_tree)); + query_plan, storage_id, header, processed_stage, modified_query_ast, context, storage_limits, std::move(analyzed_read_from_merge_tree)); } void executeQueryWithParallelReplicas( diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index 48184cb2b5f..e847a0ece26 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -273,7 +273,14 @@ std::unique_ptr createLocalPlanForParallelReplicas( } chassert(reading); - const auto * analyzed_merge_tree = typeid_cast(read_from_merge_tree.get()); + if (!read_from_merge_tree) + read_from_merge_tree = std::move(node->step); + + auto * analyzed_merge_tree = typeid_cast(read_from_merge_tree.get()); + /// if no analysis is done yet, let's do it (happens with JOINs) + if (!analyzed_merge_tree->hasAnalyzedResult()) + analyzed_merge_tree->setAnalyzedResult(analyzed_merge_tree->selectRangesToRead()); + chassert(analyzed_merge_tree->hasAnalyzedResult()); CoordinationMode mode = CoordinationMode::Default; From eece76bc412b4973d7f13a19c46e0f24c5580d02 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 1 Jun 2024 20:53:33 +0000 Subject: [PATCH 010/162] Fix build --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 4 ---- 1 file changed, 4 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 1b57515d850..a969b84dbe8 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -540,11 +540,7 @@ Pipe ReadFromMergeTree::readInOrder( { .all_callback = all_ranges_callback.value(), .callback = read_task_callback.value(), -<<<<<<< HEAD .number_of_current_replica = number_of_current_replica.value_or(client_info.number_of_current_replica), -======= - .number_of_current_replica = client_info.number_of_current_replica, ->>>>>>> origin/pr-plan-rewrite }; const auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; From 6476e9ad9bb2557f5010e4358e4881c80281a18c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 5 Jun 2024 06:45:56 +0000 Subject: [PATCH 011/162] Temporary test fix --- .../queries/0_stateless/02841_parallel_replicas_summary.sh | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02841_parallel_replicas_summary.sh b/tests/queries/0_stateless/02841_parallel_replicas_summary.sh index c82d2c8b0c0..bff56feacbf 100755 --- a/tests/queries/0_stateless/02841_parallel_replicas_summary.sh +++ b/tests/queries/0_stateless/02841_parallel_replicas_summary.sh @@ -27,6 +27,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE replicas_summary (n Int64) ENGINE = Mer query_id_base="02841_summary_$CLICKHOUSE_DATABASE" +# TODO: rethink the test, for now temporary disable allow_experimental_analyzer echo " SELECT * FROM replicas_summary @@ -36,7 +37,8 @@ echo " cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, - interactive_delay=0 + interactive_delay=0, + allow_experimental_analyzer=0 "\ | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_0" --data-binary @- -vvv 2>&1 \ | grep "Summary" | grep -cv '"read_rows":"0"' @@ -50,7 +52,8 @@ echo " cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, - interactive_delay=99999999999 + interactive_delay=99999999999, + allow_experimental_analyzer=0 "\ | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_high" --data-binary @- -vvv 2>&1 \ | grep "Summary" | grep -cv '"read_rows":"0"' From 3210d0f4716b5ac820d6f267437d13bf9fb02011 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 5 Jun 2024 11:56:12 +0000 Subject: [PATCH 012/162] Simple renaming --- src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index e847a0ece26..8ea826b861c 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -232,7 +232,7 @@ std::unique_ptr createLocalPlanForParallelReplicas( ContextPtr context, QueryProcessingStage::Enum processed_stage, ParallelReplicasReadingCoordinatorPtr coordinator, - QueryPlanStepPtr read_from_merge_tree, + QueryPlanStepPtr analyzed_read_from_merge_tree, bool has_missing_objects) { checkStackSize(); @@ -273,10 +273,10 @@ std::unique_ptr createLocalPlanForParallelReplicas( } chassert(reading); - if (!read_from_merge_tree) - read_from_merge_tree = std::move(node->step); + if (!analyzed_read_from_merge_tree) + analyzed_read_from_merge_tree = std::move(node->step); - auto * analyzed_merge_tree = typeid_cast(read_from_merge_tree.get()); + auto * analyzed_merge_tree = typeid_cast(analyzed_read_from_merge_tree.get()); /// if no analysis is done yet, let's do it (happens with JOINs) if (!analyzed_merge_tree->hasAnalyzedResult()) analyzed_merge_tree->setAnalyzedResult(analyzed_merge_tree->selectRangesToRead()); From f8d4aabfe0df44638184017ee90bfbe63ee79e90 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 6 Jun 2024 14:28:17 +0000 Subject: [PATCH 013/162] Initiliaze working set on pipelie initialization, right after analysis --- src/Planner/PlannerJoinTree.cpp | 8 ++-- .../QueryPlan/ParallelReplicasLocalPlan.cpp | 39 ++++--------------- .../QueryPlan/ReadFromMergeTree.cpp | 39 +++++++++++++++---- src/Processors/QueryPlan/ReadFromMergeTree.h | 6 ++- 4 files changed, 48 insertions(+), 44 deletions(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 25206763a40..26e78ea69ac 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -908,13 +908,13 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres chassert(reading); - auto result_ptr = reading->selectRangesToRead(); - UInt64 rows_to_read = result_ptr->selected_rows; - reading->setAnalyzedResult(std::move(result_ptr)); - // (2) if it's ReadFromMergeTree - run index analysis and check number of rows to read if (settings.parallel_replicas_min_number_of_rows_per_replica > 0) { + auto result_ptr = reading->selectRangesToRead(); + UInt64 rows_to_read = result_ptr->selected_rows; + reading->setAnalyzedResult(std::move(result_ptr)); + if (table_expression_query_info.limit > 0 && table_expression_query_info.limit < rows_to_read) rows_to_read = table_expression_query_info.limit; diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index 8ea826b861c..27d86f656c5 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -273,44 +273,21 @@ std::unique_ptr createLocalPlanForParallelReplicas( } chassert(reading); - if (!analyzed_read_from_merge_tree) - analyzed_read_from_merge_tree = std::move(node->step); - auto * analyzed_merge_tree = typeid_cast(analyzed_read_from_merge_tree.get()); - /// if no analysis is done yet, let's do it (happens with JOINs) - if (!analyzed_merge_tree->hasAnalyzedResult()) - analyzed_merge_tree->setAnalyzedResult(analyzed_merge_tree->selectRangesToRead()); + ReadFromMergeTree * analyzed_merge_tree = nullptr; + if (analyzed_read_from_merge_tree.get()) + analyzed_merge_tree = typeid_cast(analyzed_read_from_merge_tree.get()); - chassert(analyzed_merge_tree->hasAnalyzedResult()); - - CoordinationMode mode = CoordinationMode::Default; - switch (analyzed_merge_tree->getReadType()) - { - case ReadFromMergeTree::ReadType::Default: - mode = CoordinationMode::Default; - break; - case ReadFromMergeTree::ReadType::InOrder: - mode = CoordinationMode::WithOrder; - break; - case ReadFromMergeTree::ReadType::InReverseOrder: - mode = CoordinationMode::ReverseOrder; - break; - case ReadFromMergeTree::ReadType::ParallelReplicas: - chassert(false); - UNREACHABLE(); - } - - const auto number_of_local_replica = new_context->getSettingsRef().max_parallel_replicas - 1; - coordinator->handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement( - mode, analyzed_merge_tree->getAnalysisResult().parts_with_ranges.getDescriptions(), number_of_local_replica)); - - MergeTreeAllRangesCallback all_ranges_cb = [coordinator](InitialAllRangesAnnouncement) {}; + MergeTreeAllRangesCallback all_ranges_cb + = [coordinator](InitialAllRangesAnnouncement announcement) { coordinator->handleInitialAllRangesAnnouncement(announcement); }; MergeTreeReadTaskCallback read_task_cb = [coordinator](ParallelReadRequest req) -> std::optional { return coordinator->handleRequest(std::move(req)); }; + const auto number_of_local_replica = new_context->getSettingsRef().max_parallel_replicas - 1; + auto read_from_merge_tree_parallel_replicas - = reading->createLocalParallelReplicasReadingStep(analyzed_merge_tree, true, all_ranges_cb, read_task_cb, number_of_local_replica); + = reading->createLocalParallelReplicasReadingStep(analyzed_merge_tree, all_ranges_cb, read_task_cb, number_of_local_replica); node->step = std::move(read_from_merge_tree_parallel_replicas); addConvertingActions(*query_plan, header, has_missing_objects); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 7a13f072e4a..1aa48dbcff8 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -337,7 +337,6 @@ ReadFromMergeTree::ReadFromMergeTree( std::unique_ptr ReadFromMergeTree::createLocalParallelReplicasReadingStep( const ReadFromMergeTree * analyzed_merge_tree, - bool enable_parallel_reading_, std::optional all_ranges_callback_, std::optional read_task_callback_, std::optional number_of_current_replica_) @@ -354,8 +353,8 @@ std::unique_ptr ReadFromMergeTree::createLocalParallelReplica requested_num_streams, max_block_numbers_to_read, log, - analyzed_merge_tree->analyzed_result_ptr, - enable_parallel_reading_, + (analyzed_merge_tree ? analyzed_merge_tree->analyzed_result_ptr : nullptr), + true, all_ranges_callback_, read_task_callback_, number_of_current_replica_); @@ -1424,11 +1423,8 @@ static void buildIndexes( const auto & settings = context->getSettingsRef(); - indexes.emplace(ReadFromMergeTree::Indexes{{ - filter_actions_dag, - context, - primary_key_column_names, - primary_key.expression}, {}, {}, {}, {}, false, {}}); + indexes.emplace( + ReadFromMergeTree::Indexes{KeyCondition{filter_actions_dag, context, primary_key_column_names, primary_key.expression}}); if (metadata_snapshot->hasPartitionKey()) { @@ -1951,6 +1947,33 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons { auto result = getAnalysisResult(); + if (is_parallel_reading_from_replicas && context->canUseParallelReplicasOnInitiator()) + { + CoordinationMode mode = CoordinationMode::Default; + switch (result.read_type) + { + case ReadFromMergeTree::ReadType::Default: + mode = CoordinationMode::Default; + break; + case ReadFromMergeTree::ReadType::InOrder: + mode = CoordinationMode::WithOrder; + break; + case ReadFromMergeTree::ReadType::InReverseOrder: + mode = CoordinationMode::ReverseOrder; + break; + case ReadFromMergeTree::ReadType::ParallelReplicas: + chassert(false); + UNREACHABLE(); + } + + chassert(number_of_current_replica.has_value()); + chassert(all_ranges_callback.has_value()); + + /// initialize working set from local replica + all_ranges_callback.value()( + InitialAllRangesAnnouncement(mode, result.parts_with_ranges.getDescriptions(), number_of_current_replica.value())); + } + if (enable_remove_parts_from_snapshot_optimization) { /// Do not keep data parts in snapshot. diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index b541481fa62..dc1d20cc807 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -126,7 +126,6 @@ public: std::unique_ptr createLocalParallelReplicasReadingStep( const ReadFromMergeTree * analyzed_merge_tree, - bool enable_parallel_reading_, std::optional all_ranges_callback_, std::optional read_task_callback_, std::optional number_of_current_replica_); @@ -151,6 +150,11 @@ public: struct Indexes { + explicit Indexes(KeyCondition key_condition_) + : key_condition(std::move(key_condition_)) + , use_skip_indexes(false) + {} + KeyCondition key_condition; std::optional partition_pruner; std::optional minmax_idx_condition; From 4f37fafe426bbefe55aabc36faf386dc047e8517 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 6 Jun 2024 15:57:12 +0000 Subject: [PATCH 014/162] Fix 02771_parallel_replicas_analyzer --- .../0_stateless/02771_parallel_replicas_analyzer.reference | 2 +- .../queries/0_stateless/02771_parallel_replicas_analyzer.sql | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference index 5bf3520ccdb..f87b3244f09 100644 --- a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference +++ b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.reference @@ -9,4 +9,4 @@ 7885388429666205427 8124171311239967992 1 1 -- Simple query with analyzer and pure parallel replicas\nSELECT number\nFROM join_inner_table__fuzz_146_replicated\n SETTINGS\n allow_experimental_analyzer = 1,\n max_parallel_replicas = 2,\n cluster_for_parallel_replicas = \'test_cluster_one_shard_three_replicas_localhost\',\n allow_experimental_parallel_reading_from_replicas = 1; -0 2 SELECT `__table1`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` AS `__table1` +0 1 SELECT `__table1`.`number` AS `number` FROM `default`.`join_inner_table__fuzz_146_replicated` AS `__table1` diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql index 88a0d2163d6..7e27507ada9 100644 --- a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql +++ b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql @@ -1,5 +1,5 @@ -- Tags: zookeeper - +DROP TABLE IF EXISTS join_inner_table__fuzz_146_replicated; CREATE TABLE join_inner_table__fuzz_146_replicated ( `id` UUID, @@ -49,3 +49,5 @@ WHERE ) GROUP BY is_initial_query, query ORDER BY is_initial_query DESC, c, query; + +DROP TABLE join_inner_table__fuzz_146_replicated; From 7be90470d5c33877426cfc22cb5b3ad949a523ad Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 6 Jun 2024 16:49:05 +0000 Subject: [PATCH 015/162] Fix 02731_parallel_replicas_join_subquery --- .../02731_parallel_replicas_join_subquery.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference index 451f0d6d485..f0d096d2072 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.reference @@ -31,7 +31,7 @@ 29 2j&S)ba?XG QuQj 17163829389637435056 3 UlI+1 14144472852965836438 =============== QUERIES EXECUTED BY PARALLEL INNER QUERY ALONE =============== -0 3 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2`, toUInt64(min(`__table1`.`time`)) AS `start_ts` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` ORDER BY `__table1`.`key` ASC, `__table1`.`value1` ASC, `__table1`.`value2` ASC LIMIT _CAST(10, \'UInt64\') +0 2 SELECT `__table1`.`key` AS `key`, `__table1`.`value1` AS `value1`, `__table1`.`value2` AS `value2`, toUInt64(min(`__table1`.`time`)) AS `start_ts` FROM `default`.`join_inner_table` AS `__table1` PREWHERE (`__table1`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table1`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table1`.`key`, `__table1`.`value1`, `__table1`.`value2` ORDER BY `__table1`.`key` ASC, `__table1`.`value1` ASC, `__table1`.`value2` ASC LIMIT _CAST(10, \'UInt64\') 0 3 SELECT `key`, `value1`, `value2`, toUInt64(min(`time`)) AS `start_ts` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` ORDER BY `key` ASC, `value1` ASC, `value2` ASC LIMIT 10 1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=0; 1 1 -- Parallel inner query alone\nSELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\nFROM join_inner_table\nPREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\nGROUP BY key, value1, value2\nORDER BY key, value1, value2\nLIMIT 10\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=1; @@ -58,7 +58,7 @@ U c 10 UlI+1 10 bX?}ix [ Ny]2 G 10 t _CAST(1610517366120, \'UInt64\')) GROUP BY `__table3`.`key`, `__table3`.`value1`, `__table3`.`value2`) AS `__table2` USING (`key`) GROUP BY `__table1`.`key`, `__table2`.`value1`, `__table2`.`value2` +0 2 SELECT `__table2`.`value1` AS `value1`, `__table2`.`value2` AS `value2`, count() AS `count` FROM `default`.`join_outer_table` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`key` AS `key`, `__table3`.`value1` AS `value1`, `__table3`.`value2` AS `value2` FROM `default`.`join_inner_table` AS `__table3` PREWHERE (`__table3`.`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`__table3`.`number` > _CAST(1610517366120, \'UInt64\')) GROUP BY `__table3`.`key`, `__table3`.`value1`, `__table3`.`value2`) AS `__table2` USING (`key`) GROUP BY `__table1`.`key`, `__table2`.`value1`, `__table2`.`value2` 0 3 SELECT `key`, `value1`, `value2` FROM `default`.`join_inner_table` PREWHERE (`id` = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (`number` > toUInt64(\'1610517366120\')) GROUP BY `key`, `value1`, `value2` 0 3 SELECT `value1`, `value2`, count() AS `count` FROM `default`.`join_outer_table` ALL INNER JOIN `_data_` USING (`key`) GROUP BY `key`, `value1`, `value2` 1 1 -- Parallel full query\nSELECT\n value1,\n value2,\n avg(count) AS avg\nFROM\n (\n SELECT\n key,\n value1,\n value2,\n count() AS count\n FROM join_outer_table\n INNER JOIN\n (\n SELECT\n key,\n value1,\n value2,\n toUInt64(min(time)) AS start_ts\n FROM join_inner_table\n PREWHERE (id = \'833c9e22-c245-4eb5-8745-117a9a1f26b1\') AND (number > toUInt64(\'1610517366120\'))\n GROUP BY key, value1, value2\n ) USING (key)\n GROUP BY key, value1, value2\n )\nGROUP BY value1, value2\nORDER BY value1, value2\nSETTINGS allow_experimental_parallel_reading_from_replicas = 1, allow_experimental_analyzer=0; From 44fecf66ca3cd1969b2fd03d30861dac92df5ed0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 6 Jun 2024 17:11:42 +0000 Subject: [PATCH 016/162] Fix 02967_parallel_replicas_joins_and_analyzer --- ...llel_replicas_joins_and_analyzer.reference | 365 +++++++++++++++--- 1 file changed, 303 insertions(+), 62 deletions(-) diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference index 100e4e500cd..baec06244e2 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference @@ -19,8 +19,20 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x 14 14 14 14 0 0 15 15 0 0 0 0 explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; -Expression - ReadFromRemoteParallelReplicas +Union + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + ReadFromMemoryStorage + Expression + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas -- -- The same query with cte; with sub1 as (select x, y from tab1 where x != 2), @@ -55,8 +67,22 @@ select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; Expression Sorting - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Sorting + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + ReadFromMemoryStorage + Expression + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas -- -- GROUP BY should work up to WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -80,8 +106,22 @@ Expression Sorting Expression MergingAggregated - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Aggregating + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + ReadFromMemoryStorage + Expression + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas -- -- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -118,10 +158,22 @@ Expression Sorting Expression Join - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- ORDER BY in sub1 : sub1 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2 order by y), @@ -160,12 +212,24 @@ Expression Join Expression Join + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub3: sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -202,12 +266,24 @@ Expression Join Expression Join + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub5: sub5 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -242,14 +318,26 @@ Expression Sorting Expression Join - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas Expression Join - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- Subqueries for IN allowed with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -284,8 +372,28 @@ select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; Expression Sorting - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Sorting + Expression + Join + Expression + Join + Expression + CreatingSets + Expression + Expression + ReadFromMergeTree + CreatingSet + Expression + Filter + ReadFromSystemNumbers + Expression + ReadFromMemoryStorage + Expression + ReadFromMemoryStorage + Expression + ReadFromRemoteParallelReplicas -- -- Subqueries for IN are not allowed with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -333,10 +441,18 @@ Expression Expression Filter ReadFromSystemNumbers + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas set parallel_replicas_prefer_local_join = 1; -- A query with only INNER/LEFT joins is fully send to replicas. JOIN is executed in GLOBAL mode. select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; @@ -356,8 +472,22 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x 14 14 14 14 0 0 15 15 0 0 0 0 explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; -Expression - ReadFromRemoteParallelReplicas +Union + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- The same query with cte; with sub1 as (select x, y from tab1 where x != 2), @@ -392,8 +522,24 @@ select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; Expression Sorting - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Sorting + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- GROUP BY should work up to WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -417,8 +563,24 @@ Expression Sorting Expression MergingAggregated - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Aggregating + Expression + Join + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- ORDER BY in sub3 : sub1 is fully pushed, sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -455,10 +617,23 @@ Expression Sorting Expression Join - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Join + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- ORDER BY in sub1 : sub1 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2 order by y), @@ -497,12 +672,24 @@ Expression Join Expression Join + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub3: sub3 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -539,12 +726,24 @@ Expression Join Expression Join + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- RIGHT JOIN in sub5: sub5 -> WithMergableStage with sub1 as (select x, y from tab1 where x != 2), @@ -579,14 +778,26 @@ Expression Sorting Expression Join - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas Expression Join - Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- Subqueries for IN allowed with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -621,8 +832,30 @@ select * from sub5 order by x SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; Expression Sorting - Expression - ReadFromRemoteParallelReplicas + Union + Expression + Sorting + Expression + Join + Expression + Join + Expression + CreatingSets + Expression + Expression + ReadFromMergeTree + CreatingSet + Expression + Filter + ReadFromSystemNumbers + Expression + Expression + ReadFromMergeTree + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas -- -- Subqueries for IN are not allowed with sub1 as (select x, y from tab1 where x in (select number from numbers(16) where number != 2)), @@ -670,7 +903,15 @@ Expression Expression Filter ReadFromSystemNumbers + Union + Expression + Expression + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas + Union + Expression Expression - ReadFromRemoteParallelReplicas - Expression - ReadFromRemoteParallelReplicas + ReadFromMergeTree + Expression + ReadFromRemoteParallelReplicas From 09995013085511b206388816f896cdf251535ee4 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 7 Jun 2024 14:35:06 +0000 Subject: [PATCH 017/162] Fix crash in in-order coordinator --- src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index f3318a48883..9eea40c4c5f 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -887,7 +887,9 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa ++stats[announcement.replica_num].number_of_requests; - if (new_rows_to_read > 0) + /// FIXME: this code updating total_rows_to_read but it needs to be done only once since we're taking working set from initiator + /// util I missing something, it seems this code is not necessary if working set is taken from initiator (todo: check it) + if (new_rows_to_read > 0 && progress_callback) { Progress progress; progress.total_rows_to_read = new_rows_to_read; From 29d56b61b3425effb0f1fedc0c51319d8fa4062b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 7 Jun 2024 14:54:00 +0000 Subject: [PATCH 018/162] fix 02404_memory_bound_merging --- .../02404_memory_bound_merging.reference | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.reference b/tests/queries/0_stateless/02404_memory_bound_merging.reference index 0409c48f846..3daa99f91c6 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.reference +++ b/tests/queries/0_stateless/02404_memory_bound_merging.reference @@ -113,12 +113,18 @@ ExpressionTransform (Expression) ExpressionTransform × 4 (MergingAggregated) - Resize 1 → 4 - SortingAggregatedTransform 4 → 1 - MergingAggregatedBucketTransform × 4 - Resize 1 → 4 - GroupingAggregatedTransform 3 → 1 - (ReadFromRemoteParallelReplicas) + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + FinishAggregatingInOrderTransform 3 → 1 + (Union) + (Aggregating) + FinalizeAggregatedTransform + AggregatingInOrderTransform + (Expression) + ExpressionTransform + (ReadFromMergeTree) + MergeTreeSelect(pool: ReadPoolParallelReplicasInOrder, algorithm: InOrder) 0 → 1 + (ReadFromRemoteParallelReplicas) select a, count() from pr_t group by a order by a limit 5 offset 500; 500 1000 501 1000 From 23ca8608ba12dc9221effc760ecf11b3e2aefdf0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 11 Jun 2024 13:25:19 +0000 Subject: [PATCH 019/162] Fix 03006_parallel_replicas_prewhere --- .../03006_parallel_replicas_prewhere.reference | 1 + .../0_stateless/03006_parallel_replicas_prewhere.sql | 12 +++++++++++- 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference index e69de29bb2d..8b8d0e5d565 100644 --- a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference +++ b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.reference @@ -0,0 +1 @@ +default url_na_log 1 130000 130 diff --git a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql index 4b84646c034..afe6a00cc4d 100644 --- a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql +++ b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql @@ -21,11 +21,21 @@ SELECT FROM numbers(130000) SETTINGS max_insert_block_size = 200000; +SET max_block_size = 1048576, max_threads = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, parallel_replicas_min_number_of_rows_per_replica=10000; + EXPLAIN ESTIMATE SELECT count() FROM url_na_log PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20') -SETTINGS max_block_size = 1048576, max_threads = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', max_parallel_replicas = 3, parallel_replicas_min_number_of_rows_per_replica=10000; +SETTINGS allow_experimental_analyzer=0; + +-- here parallel replicas uses local snapshot as working set +-- so, the estimation can be done +EXPLAIN ESTIMATE +SELECT count() +FROM url_na_log +PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20') +SETTINGS allow_experimental_analyzer=1; DROP POLICY url_na_log_policy0 ON url_na_log; DROP TABLE url_na_log; From d6dec38103cce55ba6c026ed8d56148eeed16e46 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 11 Jun 2024 14:37:42 +0000 Subject: [PATCH 020/162] Fix test_parallel_replicas_over_distributed --- src/Interpreters/ClusterProxy/executeQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 0eeae112062..cde33697915 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -493,7 +493,8 @@ void executeQueryWithParallelReplicas( new_cluster->getShardsInfo().begin()->getAllNodeCount(), settings.parallel_replicas_mark_segment_size); auto external_tables = new_context->getExternalTables(); - if (settings.allow_experimental_analyzer) + /// do not build local plan for distributed queries for now (address it later) + if (settings.allow_experimental_analyzer && !shard_num) { auto read_from_remote = std::make_unique( query_ast, From f999eed3767ba5e98ee9de7d1f94d13fcab20ebc Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 11 Jun 2024 16:49:11 +0000 Subject: [PATCH 021/162] Fix 02967_parallel_replicas_join_algo_and_analyzer --- ...llel_replicas_join_algo_and_analyzer.reference | 15 --------------- 1 file changed, 15 deletions(-) diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.reference b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.reference index d7fa419aeab..d911335c6c0 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.reference +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer.reference @@ -23,10 +23,8 @@ simple (global) join with analyzer and parallel replicas 4200048 4200048 4200048 -1400016 4200054 4200054 4200054 -1400018 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` (stage: WithMergeableState) DefaultCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) DefaultCoordinator: Coordination done simple (local) join with analyzer and parallel replicas @@ -41,7 +39,6 @@ simple (local) join with analyzer and parallel replicas 4200048 4200048 4200048 -1400016 4200054 4200054 4200054 -1400018 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) DefaultCoordinator: Coordination done simple (local) join with analyzer and parallel replicas and full sorting merge join @@ -56,7 +53,6 @@ simple (local) join with analyzer and parallel replicas and full sorting merge j 4200048 4200048 4200048 -1400016 4200054 4200054 4200054 -1400018 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4`) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(700000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) WithOrderCoordinator: Coordination done nested join with analyzer @@ -83,7 +79,6 @@ nested join with analyzer and parallel replicas, both local 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4` ALL INNER JOIN (SELECT `__table6`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table6`) AS `__table5` ON `__table4`.`key` = `__table5`.`key` SETTINGS parallel_replicas_prefer_local_join = 1) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` ALL INNER JOIN (SELECT `__table4`.`key` AS `key`, `__table4`.`value` AS `value` FROM `default`.`num_2` AS `__table4` ALL INNER JOIN (SELECT `__table6`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table6`) AS `__table5` ON `__table4`.`key` = `__table5`.`key` SETTINGS parallel_replicas_prefer_local_join = 1) AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) WithOrderCoordinator: Coordination done nested join with analyzer and parallel replicas, both global @@ -98,10 +93,8 @@ nested join with analyzer and parallel replicas, both global 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) DefaultCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) DefaultCoordinator: Coordination done nested join with analyzer and parallel replicas, global + local @@ -116,10 +109,8 @@ nested join with analyzer and parallel replicas, global + local 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table3`) AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` ALL INNER JOIN (SELECT `__table3`.`number` * 7 AS `key` FROM numbers(100000.) AS `__table3`) AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) DefaultCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) DefaultCoordinator: Coordination done nested join with analyzer and parallel replicas, both local, both full sorting merge join @@ -134,10 +125,8 @@ nested join with analyzer and parallel replicas, both local, both full sorting m 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) WithOrderCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) WithOrderCoordinator: Coordination done nested join with analyzer and parallel replicas, both local, both full sorting and hash join @@ -152,10 +141,8 @@ nested join with analyzer and parallel replicas, both local, both full sorting a 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) DefaultCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) WithOrderCoordinator: Coordination done nested join with analyzer and parallel replicas, both local, both full sorting and hash join @@ -170,8 +157,6 @@ nested join with analyzer and parallel replicas, both local, both full sorting a 420336 420336 420336 -140112 420378 420378 420378 -140126 SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value` FROM `default`.`num_2` AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table2` ON `__table1`.`key` = `__table2`.`key` (stage: WithMergeableState) WithOrderCoordinator: Coordination done SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) -SELECT `__table1`.`key` AS `key`, `__table1`.`value` AS `value`, `__table3`.`key` AS `r.key`, `__table3`.`value` AS `r.value` FROM (SELECT `__table2`.`key` AS `key`, `__table2`.`value` AS `value` FROM `default`.`num_1` AS `__table2`) AS `__table1` GLOBAL ALL INNER JOIN `_data_` AS `__table3` ON `__table1`.`key` = `__table3`.`key` ORDER BY `__table1`.`key` ASC LIMIT _CAST(10000, 'UInt64'), _CAST(10, 'UInt64') (stage: WithMergeableState) DefaultCoordinator: Coordination done From f46deb4e793647b505514eb1a826dc94d7a69207 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 11 Jun 2024 16:49:51 +0000 Subject: [PATCH 022/162] Fix clang-tidy --- .../QueryPlan/ParallelReplicasLocalPlan.cpp | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index 27d86f656c5..16cca6a7f5d 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -1,26 +1,22 @@ #include #include -#include "Storages/MergeTree/RequestResponse.h" -#include #include +#include #include +#include +#include #include #include -#include #include -#include #include -#include #include -#include +#include #include -#include - +#include +#include #include -#include - namespace ProfileEvents { extern const Event SelectedParts; From da4ed273e304cf01010289b374074818e04771d7 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 11 Jun 2024 19:24:50 +0000 Subject: [PATCH 023/162] Fix 02811_parallel_replicas_prewhere_count --- src/Planner/PlannerJoinTree.cpp | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 9d1301305b8..15d0fa45891 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -959,6 +959,19 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres std::move(reading_step)); query_plan = std::move(query_plan_parallel_replicas); } + else { + QueryPlan query_plan_no_parallel_replicas; + storage->read( + query_plan_no_parallel_replicas, + columns_names, + storage_snapshot, + table_expression_query_info, + query_context, + from_stage, + max_block_size, + max_streams); + query_plan = std::move(query_plan_no_parallel_replicas); + } } const auto & alias_column_expressions = table_expression_data.getAliasColumnExpressions(); From eccd56c9b93f6ac3a8ef5c23c4bbc5ddc8a77fb6 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 11 Jun 2024 20:16:00 +0000 Subject: [PATCH 024/162] Fix style check --- src/Planner/PlannerJoinTree.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 15d0fa45891..7a8ad50b9ab 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -959,7 +959,8 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres std::move(reading_step)); query_plan = std::move(query_plan_parallel_replicas); } - else { + else + { QueryPlan query_plan_no_parallel_replicas; storage->read( query_plan_no_parallel_replicas, From 164b5e47a81ae3ad0719d00319f5193fd59ee680 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 12 Jun 2024 10:30:15 +0000 Subject: [PATCH 025/162] Fix 02769_parallel_replicas_unavailable_shards --- .../0_stateless/02769_parallel_replicas_unavailable_shards.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql index 1a75e000349..34678fdd146 100644 --- a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql +++ b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql @@ -2,7 +2,7 @@ DROP TABLE IF EXISTS test_parallel_replicas_unavailable_shards; CREATE TABLE test_parallel_replicas_unavailable_shards (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); INSERT INTO test_parallel_replicas_unavailable_shards SELECT * FROM numbers(10); -SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=11, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1; +SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=11, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1, allow_experimental_analyzer=0; SET send_logs_level='error'; SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79'; From 048cbb17a6b016c20128bfa156d52a7554974fdc Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 13 Jun 2024 19:55:30 +0000 Subject: [PATCH 026/162] Comment to 02769_parallel_replicas_unavailable_shards.sql --- .../02769_parallel_replicas_unavailable_shards.sql | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql index 34678fdd146..337b05263d0 100644 --- a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql +++ b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql @@ -2,9 +2,11 @@ DROP TABLE IF EXISTS test_parallel_replicas_unavailable_shards; CREATE TABLE test_parallel_replicas_unavailable_shards (n UInt64) ENGINE=MergeTree() ORDER BY tuple(); INSERT INTO test_parallel_replicas_unavailable_shards SELECT * FROM numbers(10); -SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=11, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1, allow_experimental_analyzer=0; +SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=11, cluster_for_parallel_replicas='parallel_replicas', parallel_replicas_for_non_replicated_merge_tree=1; SET send_logs_level='error'; -SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79'; +-- with local plan for initiator, the query can be executed fast on initator, we can simply not come to the point where unavailable replica can be detected +-- therefore disable local plan for now +SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79', allow_experimental_analyzer=0; SYSTEM FLUSH LOGS; From 245476b34bbbf7b0fd24fffa54539fa9c2b3b490 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 14 Jun 2024 12:02:47 +0000 Subject: [PATCH 027/162] Consistent replica id assignment --- .../QueryPlan/ParallelReplicasLocalPlan.cpp | 3 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 34 ++++++++++++++++--- 2 files changed, 30 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index 16cca6a7f5d..bd79fc38ae9 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -280,8 +280,7 @@ std::unique_ptr createLocalPlanForParallelReplicas( MergeTreeReadTaskCallback read_task_cb = [coordinator](ParallelReadRequest req) -> std::optional { return coordinator->handleRequest(std::move(req)); }; - const auto number_of_local_replica = new_context->getSettingsRef().max_parallel_replicas - 1; - + const auto number_of_local_replica = 0; auto read_from_merge_tree_parallel_replicas = reading->createLocalParallelReplicasReadingStep(analyzed_merge_tree, all_ranges_cb, read_task_cb, number_of_local_replica); node->step = std::move(read_from_merge_tree_parallel_replicas); diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 23d8f2b496f..6949c35e0ca 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -434,14 +434,11 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder shard.getAllNodeCount()); all_replicas_count = shard.getAllNodeCount(); } - if (exclude_local_replica) - --all_replicas_count; std::vector shuffled_pool; if (all_replicas_count < shard.getAllNodeCount()) { shuffled_pool = shard.pool->getShuffledPools(current_settings); - shuffled_pool.resize(all_replicas_count); } else { @@ -451,10 +448,37 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func); } - for (size_t i=0; i < all_replicas_count; ++i) + std::vector pools_to_use; + if (exclude_local_replica) { - IConnections::ReplicaInfo replica_info + std::vector local_addr_possitions; + for (auto & pool : shuffled_pool) { + const auto & hostname = pool.pool->getHost(); + auto it = std::find_if( + begin(shard.local_addresses), + end(shard.local_addresses), + [&hostname](const Cluster::Address & local_addr) { return hostname == local_addr.host_name; }); + if (it != shard.local_addresses.end()) + pool.pool.reset(); + } + } + for (const auto & pool : shuffled_pool) + { + if (pool.pool) + pools_to_use.push_back(pool.pool); + } + + if (pools_to_use.size() > all_replicas_count) + pools_to_use.resize(all_replicas_count); + else + all_replicas_count = pools_to_use.size(); + + /// local replicas has number 0 + size_t offset = (exclude_local_replica ? 1 : 0); + for (size_t i = 0 + offset; i < all_replicas_count + offset; ++i) + { + IConnections::ReplicaInfo replica_info{ .all_replicas_count = all_replicas_count, /// we should use this number specifically because efficiency of data distribution by consistent hash depends on it. .number_of_current_replica = i, From c2d38b3c93c2182f096b21558db2b7f574fa435c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 14 Jun 2024 14:06:12 +0000 Subject: [PATCH 028/162] Fix crash --- src/Processors/QueryPlan/ReadFromRemote.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 6949c35e0ca..1a96668e30e 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -474,9 +474,17 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder else all_replicas_count = pools_to_use.size(); + chassert(all_replicas_count == pools_to_use.size()); + + if (exclude_local_replica && !pools_to_use.empty()) + pools_to_use.resize(all_replicas_count - 1); + + if (pools_to_use.empty()) + return; + /// local replicas has number 0 size_t offset = (exclude_local_replica ? 1 : 0); - for (size_t i = 0 + offset; i < all_replicas_count + offset; ++i) + for (size_t i = 0 + offset; i < all_replicas_count; ++i) { IConnections::ReplicaInfo replica_info{ .all_replicas_count = all_replicas_count, @@ -484,7 +492,7 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder .number_of_current_replica = i, }; - addPipeForSingeReplica(pipes, shuffled_pool[i].pool, replica_info); + addPipeForSingeReplica(pipes, pools_to_use[i - offset], replica_info); } auto pipe = Pipe::unitePipes(std::move(pipes)); From 70a33e633cd8acd0ec7220b5ce6e45d9c7bd4e9e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 14 Jun 2024 19:53:04 +0000 Subject: [PATCH 029/162] Fix 02784_parallel_replicas_automatic_decision --- .../02784_parallel_replicas_automatic_decision.sh | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index 8a3b34e5cfa..dd9b5c3f6d9 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -10,12 +10,14 @@ function were_parallel_replicas_used () { $CLICKHOUSE_CLIENT --query " SELECT initial_query_id, - concat('Used parallel replicas: ', (countIf(initial_query_id != query_id) != 0)::bool::String) as used + concat('Used parallel replicas: ', (ProfileEvents['ParallelReplicasUsedCount'] > 0)::bool::String) as used FROM system.query_log WHERE event_date >= yesterday() AND initial_query_id LIKE '$1%' - GROUP BY initial_query_id - ORDER BY min(event_time_microseconds) ASC + AND query_id = initial_query_id + AND type = 'QueryFinish' + AND current_database = '$CLICKHOUSE_DATABASE' + ORDER BY event_time_microseconds ASC FORMAT TSV" } @@ -48,7 +50,6 @@ function run_query_with_pure_parallel_replicas () { --query "$3" \ --query_id "${1}_pure" \ --max_parallel_replicas 3 \ - --prefer_localhost_replica 1 \ --cluster_for_parallel_replicas "parallel_replicas" \ --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ From c87bfe102e36612ebb4b474eb822543de0cf011e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 17 Jun 2024 12:24:14 +0000 Subject: [PATCH 030/162] Fix: correct local replica exclusion - fixes 02731_parallel_replicas_join_subquery --- .../ClusterProxy/executeQuery.cpp | 8 +++++-- src/Processors/QueryPlan/ReadFromRemote.cpp | 21 +++++++++++++++---- 2 files changed, 23 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index cde33697915..0937e121426 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -489,8 +489,12 @@ void executeQueryWithParallelReplicas( "`cluster_for_parallel_replicas` setting refers to cluster with several shards. Expected a cluster with one shard"); } - auto coordinator = std::make_shared( - new_cluster->getShardsInfo().begin()->getAllNodeCount(), settings.parallel_replicas_mark_segment_size); + auto replica_count = new_cluster->getShardsInfo().begin()->getAllNodeCount(); + if (settings.max_parallel_replicas < replica_count) + replica_count = settings.max_parallel_replicas; + + auto coordinator = std::make_shared(replica_count, settings.parallel_replicas_mark_segment_size); + auto external_tables = new_context->getExternalTables(); /// do not build local plan for distributed queries for now (address it later) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 1a96668e30e..f7cb88154af 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -449,6 +449,7 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder } std::vector pools_to_use; + pools_to_use.reserve(shuffled_pool.size()); if (exclude_local_replica) { std::vector local_addr_possitions; @@ -460,7 +461,9 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder end(shard.local_addresses), [&hostname](const Cluster::Address & local_addr) { return hostname == local_addr.host_name; }); if (it != shard.local_addresses.end()) + { pool.pool.reset(); + } } } for (const auto & pool : shuffled_pool) @@ -469,12 +472,14 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder pools_to_use.push_back(pool.pool); } + LOG_DEBUG( + getLogger("ReadFromParallelRemoteReplicasStep"), + "Number of pools to use is {}. Originally {}", + pools_to_use.size(), + shuffled_pool.size()); + if (pools_to_use.size() > all_replicas_count) pools_to_use.resize(all_replicas_count); - else - all_replicas_count = pools_to_use.size(); - - chassert(all_replicas_count == pools_to_use.size()); if (exclude_local_replica && !pools_to_use.empty()) pools_to_use.resize(all_replicas_count - 1); @@ -482,6 +487,14 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder if (pools_to_use.empty()) return; + { + String pool_addresses; + for (const auto & pool : pools_to_use) + pool_addresses += pool->getAddress() + ";"; + + LOG_DEBUG(getLogger("ReadFromParallelRemoteReplicasStep"), "Addresses to use: {}", pool_addresses); + } + /// local replicas has number 0 size_t offset = (exclude_local_replica ? 1 : 0); for (size_t i = 0 + offset; i < all_replicas_count; ++i) From c286188419df31cdc17ac29f341a99b723da6360 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 18 Jun 2024 10:39:13 +0000 Subject: [PATCH 031/162] Update 00177_memory_bound_merging.reference --- .../queries/1_stateful/00177_memory_bound_merging.reference | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/queries/1_stateful/00177_memory_bound_merging.reference b/tests/queries/1_stateful/00177_memory_bound_merging.reference index 5689152d60a..4c7505d1123 100644 --- a/tests/queries/1_stateful/00177_memory_bound_merging.reference +++ b/tests/queries/1_stateful/00177_memory_bound_merging.reference @@ -10,6 +10,8 @@ http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-18 http:/ http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-19 http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 2014-03-20 http://auto.ru/chatay-baranta_bound-in-thankYou=ru/tver/zhanny 1 -SortingAggregatedTransform MergingAggregatedBucketTransform -GroupingAggregatedTransform +FinishAggregatingInOrderTransform +FinalizeAggregatedTransform +AggregatingInOrderTransform +MergeTreeSelect From 366de07856cd25d7212ae8106d68994cd84a2de3 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 18 Jun 2024 10:39:50 +0000 Subject: [PATCH 032/162] Remove prefer_localhost_replica --- .../0_stateless/02731_parallel_replicas_join_subquery.sql | 1 - tests/queries/1_stateful/00177_memory_bound_merging.sh | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql index 7693d0da295..e91e2a19526 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql @@ -22,7 +22,6 @@ SELECT LIMIT 100; SET max_parallel_replicas = 3; -SET prefer_localhost_replica = 1; SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; SET joined_subquery_requires_alias = 0; diff --git a/tests/queries/1_stateful/00177_memory_bound_merging.sh b/tests/queries/1_stateful/00177_memory_bound_merging.sh index d5cd1a05cd8..74b5e80e059 100755 --- a/tests/queries/1_stateful/00177_memory_bound_merging.sh +++ b/tests/queries/1_stateful/00177_memory_bound_merging.sh @@ -55,7 +55,7 @@ test2() { test3() { $CLICKHOUSE_CLIENT -nq " SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; - SET max_threads = 16, prefer_localhost_replica = 1, read_in_order_two_level_merge_threshold = 1000, query_plan_aggregation_in_order = 1, distributed_aggregation_memory_efficient = 1; + SET max_threads = 16, read_in_order_two_level_merge_threshold = 1000, query_plan_aggregation_in_order = 1, distributed_aggregation_memory_efficient = 1; SELECT replaceRegexpOne(explain, '^ *(\w+).*', '\\1') FROM ( From 0ecefa6973ecb1356e117d944d26f26fb204d8c9 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 18 Jun 2024 11:07:15 +0000 Subject: [PATCH 033/162] Better replicas notation in plan description --- src/Processors/QueryPlan/ReadFromRemote.cpp | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index f7cb88154af..26c57c4a760 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -387,8 +387,8 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( { chassert(cluster->getShardCount() == 1); - std::vector description; - description.push_back(fmt::format("query: {}", formattedAST(query_ast))); + std::vector replicas; + replicas.reserve(cluster->getShardsAddresses().front().size()); bool first_local = false; for (const auto & addr : cluster->getShardsAddresses().front()) @@ -400,10 +400,16 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( continue; } - description.push_back(fmt::format("Replica: {}", addr.host_name)); + /// replace hostname with replica name if the hostname started with replica namespace, + /// it makes description shorter and more readable + if (!addr.database_replica_name.empty() && addr.host_name.starts_with(addr.database_replica_name)) + replicas.push_back(fmt::format("{}", addr.database_replica_name)); + else + replicas.push_back(fmt::format("{}", addr.host_name)); } - setStepDescription(boost::algorithm::join(description, ", ")); + auto description = fmt::format("Query: {} Replicas: ", formattedAST(query_ast)) + boost::algorithm::join(replicas, ", "); + setStepDescription(std::move(description)); } void ReadFromParallelRemoteReplicasStep::enforceSorting(SortDescription output_sort_description) From dba6ea078f259433dd91a0b2ddc88b6a4f3c143d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 18 Jun 2024 19:47:07 +0000 Subject: [PATCH 034/162] Setting parallel_replicas_local_plan --- src/Core/Settings.h | 1 + src/Interpreters/ClusterProxy/executeQuery.cpp | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 6197a7cf6e1..c16efc5717b 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -216,6 +216,7 @@ class IColumn; M(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, "Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas'", 0) \ M(Bool, parallel_replicas_prefer_local_join, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN.", 0) \ M(UInt64, parallel_replicas_mark_segment_size, 128, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing", 0) \ + M(Bool, parallel_replicas_local_plan, true, "Build local plan for local replica", 0) \ \ M(Bool, skip_unavailable_shards, false, "If true, ClickHouse silently skips unavailable shards. Shard is marked as unavailable when: 1) The shard cannot be reached due to a connection failure. 2) Shard is unresolvable through DNS. 3) Table does not exist on the shard.", 0) \ \ diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 0937e121426..ab25da090d6 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -498,7 +498,7 @@ void executeQueryWithParallelReplicas( auto external_tables = new_context->getExternalTables(); /// do not build local plan for distributed queries for now (address it later) - if (settings.allow_experimental_analyzer && !shard_num) + if (settings.allow_experimental_analyzer && settings.parallel_replicas_local_plan && !shard_num) { auto read_from_remote = std::make_unique( query_ast, From 9658d37d49e3c47cd306657abb8a314f47c4011d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 19 Jun 2024 11:27:50 +0000 Subject: [PATCH 035/162] Add parallel_replicas_local_plan to settings history --- src/Core/SettingsChangesHistory.h | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Core/SettingsChangesHistory.h b/src/Core/SettingsChangesHistory.h index eddf83f7912..1d197db9e31 100644 --- a/src/Core/SettingsChangesHistory.h +++ b/src/Core/SettingsChangesHistory.h @@ -113,6 +113,7 @@ static const std::map Date: Mon, 24 Jun 2024 14:31:05 +0000 Subject: [PATCH 036/162] Cleanup --- src/Processors/Executors/PipelineExecutor.h | 1 - .../QueryPlan/ParallelReplicasLocalPlan.cpp | 173 ------------------ 2 files changed, 174 deletions(-) diff --git a/src/Processors/Executors/PipelineExecutor.h b/src/Processors/Executors/PipelineExecutor.h index 03f0f7f1a0a..ae119355cb5 100644 --- a/src/Processors/Executors/PipelineExecutor.h +++ b/src/Processors/Executors/PipelineExecutor.h @@ -9,7 +9,6 @@ #include #include -#include #include diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index bd79fc38ae9..bd5a4793872 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -17,13 +17,6 @@ #include #include -namespace ProfileEvents -{ - extern const Event SelectedParts; - extern const Event SelectedRanges; - extern const Event SelectedMarks; -} - namespace DB { @@ -56,172 +49,6 @@ void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missi } -class ReadFromMergeTreeCoordinated : public ISourceStep -{ -public: - ReadFromMergeTreeCoordinated(QueryPlanStepPtr read_from_merge_tree_, ParallelReplicasReadingCoordinatorPtr coordinator_) - : ISourceStep(read_from_merge_tree_->getOutputStream()) - , read_from_merge_tree(std::move(read_from_merge_tree_)) - , coordinator(std::move(coordinator_)) - { - } - - void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings) override; - String getName() const override { return "ReadFromLocalParallelReplica"; } - -private: - QueryPlanStepPtr read_from_merge_tree; - ParallelReplicasReadingCoordinatorPtr coordinator; -}; - -void ReadFromMergeTreeCoordinated::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & /*settings*/) -{ - ReadFromMergeTree & reading = *typeid_cast(read_from_merge_tree.get()); - - auto result = reading.getAnalysisResult(); - const auto & query_info = reading.getQueryInfo(); - const auto & data = reading.data; - const auto & context = reading.getContext(); - const auto & storage_snapshot = reading.getStorageSnapshot(); - - if (reading.enable_remove_parts_from_snapshot_optimization) - { - /// Do not keep data parts in snapshot. - /// They are stored separately, and some could be released after PK analysis. - reading.storage_snapshot->data = std::make_unique(); - } - - LOG_DEBUG( - reading.log, - "Selected {}/{} parts by partition key, {} parts by primary key, {}/{} marks by primary key, {} marks to read from {} ranges", - result.parts_before_pk, - result.total_parts, - result.selected_parts, - result.selected_marks_pk, - result.total_marks_pk, - result.selected_marks, - result.selected_ranges); - - // Adding partition info to QueryAccessInfo. - if (context->hasQueryContext() && !query_info.is_internal) - { - Names partition_names; - for (const auto & part : result.parts_with_ranges) - { - partition_names.emplace_back( - fmt::format("{}.{}", data.getStorageID().getFullNameNotQuoted(), part.data_part->info.partition_id)); - } - context->getQueryContext()->addQueryAccessInfo(partition_names); - - if (storage_snapshot->projection) - context->getQueryContext()->addQueryAccessInfo( - Context::QualifiedProjectionName{.storage_id = data.getStorageID(), .projection_name = storage_snapshot->projection->name}); - } - - ProfileEvents::increment(ProfileEvents::SelectedParts, result.selected_parts); - ProfileEvents::increment(ProfileEvents::SelectedRanges, result.selected_ranges); - ProfileEvents::increment(ProfileEvents::SelectedMarks, result.selected_marks); - - auto query_id_holder = MergeTreeDataSelectExecutor::checkLimits(data, result, context); - - // TODO: check this on plan level, we should be here if there is nothing to read - if (result.parts_with_ranges.empty()) - { - pipeline.init(Pipe(std::make_shared(getOutputStream().header))); - return; - } - - /// Projection, that needed to drop columns, which have appeared by execution - /// of some extra expressions, and to allow execute the same expressions later. - /// NOTE: It may lead to double computation of expressions. - ActionsDAGPtr result_projection; - - Pipe pipe = reading.spreadMarkRanges(std::move(result.parts_with_ranges), reading.requested_num_streams, result, result_projection); - - for (const auto & processor : pipe.getProcessors()) - processor->setStorageLimits(query_info.storage_limits); - - if (pipe.empty()) - { - pipeline.init(Pipe(std::make_shared(getOutputStream().header))); - return; - } - - if (result.sampling.use_sampling) - { - auto sampling_actions = std::make_shared(result.sampling.filter_expression); - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared( - header, - sampling_actions, - result.sampling.filter_function->getColumnName(), - false); - }); - } - - Block cur_header = pipe.getHeader(); - - auto append_actions = [&result_projection](ActionsDAGPtr actions) - { - if (!result_projection) - result_projection = std::move(actions); - else - result_projection = ActionsDAG::merge(std::move(*result_projection), std::move(*actions)); - }; - - if (result_projection) - cur_header = result_projection->updateHeader(cur_header); - - /// Extra columns may be returned (for example, if sampling is used). - /// Convert pipe to step header structure. - if (!isCompatibleHeader(cur_header, getOutputStream().header)) - { - auto converting = ActionsDAG::makeConvertingActions( - cur_header.getColumnsWithTypeAndName(), - getOutputStream().header.getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name); - - append_actions(std::move(converting)); - } - - if (result_projection) - { - auto projection_actions = std::make_shared(result_projection); - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, projection_actions); - }); - } - - /// Some extra columns could be added by sample/final/in-order/etc - /// Remove them from header if not needed. - if (!blocksHaveEqualStructure(pipe.getHeader(), getOutputStream().header)) - { - auto convert_actions_dag = ActionsDAG::makeConvertingActions( - pipe.getHeader().getColumnsWithTypeAndName(), - getOutputStream().header.getColumnsWithTypeAndName(), - ActionsDAG::MatchColumnsMode::Name, - true); - - auto converting_dag_expr = std::make_shared(convert_actions_dag); - - pipe.addSimpleTransform([&](const Block & header) - { - return std::make_shared(header, converting_dag_expr); - }); - } - - for (const auto & processor : pipe.getProcessors()) - processors.emplace_back(processor); - - pipeline.init(std::move(pipe)); - pipeline.addContext(context); - // Attach QueryIdHolder if needed - if (query_id_holder) - pipeline.setQueryIdHolder(std::move(query_id_holder)); -} - std::unique_ptr createLocalPlanForParallelReplicas( const ASTPtr & query_ast, const Block & header, From f62873b1735bc22e8a24a1ed91a43edbf405fea4 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 25 Jun 2024 12:29:21 +0000 Subject: [PATCH 037/162] Fix 02404_memory_bound_merging --- .../02404_memory_bound_merging.reference | 21 ++++++++++++++++++- .../02404_memory_bound_merging.sql | 3 ++- 2 files changed, 22 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.reference b/tests/queries/0_stateless/02404_memory_bound_merging.reference index 3daa99f91c6..b2195051c0b 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.reference +++ b/tests/queries/0_stateless/02404_memory_bound_merging.reference @@ -100,7 +100,26 @@ select a, count() from dist_t_different_dbs group by a, b order by a limit 5 off 504 2000 1000000 -- { echoOn } -- -explain pipeline select a from pr_t group by a order by a limit 5 offset 500; +explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings allow_experimental_analyzer=0; +(Expression) +ExpressionTransform + (Limit) + Limit + (Sorting) + MergingSortedTransform 4 → 1 + MergeSortingTransform × 4 + LimitsCheckingTransform × 4 + PartialSortingTransform × 4 + (Expression) + ExpressionTransform × 4 + (MergingAggregated) + Resize 1 → 4 + SortingAggregatedTransform 4 → 1 + MergingAggregatedBucketTransform × 4 + Resize 1 → 4 + GroupingAggregatedTransform 3 → 1 + (ReadFromRemoteParallelReplicas) +explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings allow_experimental_analyzer=1, parallel_replicas_local_plan=1; (Expression) ExpressionTransform (Limit) diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.sql b/tests/queries/0_stateless/02404_memory_bound_merging.sql index 5e017e79309..a12ccb2ffb3 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.sql +++ b/tests/queries/0_stateless/02404_memory_bound_merging.sql @@ -72,7 +72,8 @@ set distributed_aggregation_memory_efficient=1; select count() from pr_t; -- { echoOn } -- -explain pipeline select a from pr_t group by a order by a limit 5 offset 500; +explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings allow_experimental_analyzer=0; +explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings allow_experimental_analyzer=1, parallel_replicas_local_plan=1; select a, count() from pr_t group by a order by a limit 5 offset 500; select a, count() from pr_t group by a, b order by a limit 5 offset 500; From 96b68cb920600cb1a080cefd5f7e82de332ac8dd Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 25 Jun 2024 13:33:36 +0000 Subject: [PATCH 038/162] Update setting in 02404_memory_bound_merging --- tests/queries/0_stateless/02404_memory_bound_merging.reference | 2 +- tests/queries/0_stateless/02404_memory_bound_merging.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.reference b/tests/queries/0_stateless/02404_memory_bound_merging.reference index b2195051c0b..e02684c3b95 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.reference +++ b/tests/queries/0_stateless/02404_memory_bound_merging.reference @@ -100,7 +100,7 @@ select a, count() from dist_t_different_dbs group by a, b order by a limit 5 off 504 2000 1000000 -- { echoOn } -- -explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings allow_experimental_analyzer=0; +explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings parallel_replicas_local_plan=0; (Expression) ExpressionTransform (Limit) diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.sql b/tests/queries/0_stateless/02404_memory_bound_merging.sql index a12ccb2ffb3..2958cc22ce0 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.sql +++ b/tests/queries/0_stateless/02404_memory_bound_merging.sql @@ -72,7 +72,7 @@ set distributed_aggregation_memory_efficient=1; select count() from pr_t; -- { echoOn } -- -explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings allow_experimental_analyzer=0; +explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings parallel_replicas_local_plan=0; explain pipeline select a from pr_t group by a order by a limit 5 offset 500 settings allow_experimental_analyzer=1, parallel_replicas_local_plan=1; select a, count() from pr_t group by a order by a limit 5 offset 500; From 318af3af95845166ff7cd9c38c6a2cfd1e97ccfd Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 25 Jun 2024 14:31:11 +0000 Subject: [PATCH 039/162] Cleanup --- .../QueryPlan/ParallelReplicasLocalPlan.cpp | 34 ++----------------- .../QueryPlan/ParallelReplicasLocalPlan.h | 3 +- .../QueryPlan/ReadFromMergeTree.cpp | 10 +++--- src/Processors/QueryPlan/ReadFromMergeTree.h | 9 ++--- .../ParallelReplicasReadingCoordinator.h | 2 +- ...9_parallel_replicas_unavailable_shards.sql | 2 +- 6 files changed, 14 insertions(+), 46 deletions(-) diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index bd5a4793872..6822d8b0a71 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -20,34 +20,7 @@ namespace DB { -namespace -{ - -void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects) -{ - if (blocksHaveEqualStructure(plan.getCurrentDataStream().header, header)) - return; - - auto mode = has_missing_objects ? ActionsDAG::MatchColumnsMode::Position : ActionsDAG::MatchColumnsMode::Name; - - auto get_converting_dag = [mode](const Block & block_, const Block & header_) - { - /// Convert header structure to expected. - /// Also we ignore constants from result and replace it with constants from header. - /// It is needed for functions like `now64()` or `randConstant()` because their values may be different. - return ActionsDAG::makeConvertingActions( - block_.getColumnsWithTypeAndName(), - header_.getColumnsWithTypeAndName(), - mode, - true); - }; - - auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header); - auto converting = std::make_unique(plan.getCurrentDataStream(), convert_actions_dag); - plan.addStep(std::move(converting)); -} - -} +void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects); std::unique_ptr createLocalPlanForParallelReplicas( const ASTPtr & query_ast, @@ -55,8 +28,7 @@ std::unique_ptr createLocalPlanForParallelReplicas( ContextPtr context, QueryProcessingStage::Enum processed_stage, ParallelReplicasReadingCoordinatorPtr coordinator, - QueryPlanStepPtr analyzed_read_from_merge_tree, - bool has_missing_objects) + QueryPlanStepPtr analyzed_read_from_merge_tree) { checkStackSize(); @@ -112,7 +84,7 @@ std::unique_ptr createLocalPlanForParallelReplicas( = reading->createLocalParallelReplicasReadingStep(analyzed_merge_tree, all_ranges_cb, read_task_cb, number_of_local_replica); node->step = std::move(read_from_merge_tree_parallel_replicas); - addConvertingActions(*query_plan, header, has_missing_objects); + addConvertingActions(*query_plan, header, /*has_missing_objects=*/false); return query_plan; } diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h index 89d2019f807..123754458a1 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h @@ -14,6 +14,5 @@ std::unique_ptr createLocalPlanForParallelReplicas( ContextPtr context, QueryProcessingStage::Enum processed_stage, ParallelReplicasReadingCoordinatorPtr coordinator, - QueryPlanStepPtr read_from_merge_tree, - bool has_missing_objects); + QueryPlanStepPtr read_from_merge_tree); } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 71caece30ed..1de701cff0b 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -339,9 +339,9 @@ ReadFromMergeTree::ReadFromMergeTree( std::unique_ptr ReadFromMergeTree::createLocalParallelReplicasReadingStep( const ReadFromMergeTree * analyzed_merge_tree, std::optional all_ranges_callback_, - std::optional read_task_callback_, - std::optional number_of_current_replica_) + std::optional read_task_callback_) { + const auto number_of_local_replica = 0; return std::make_unique( prepared_parts, alter_conversions_for_parts, @@ -356,9 +356,9 @@ std::unique_ptr ReadFromMergeTree::createLocalParallelReplica log, (analyzed_merge_tree ? analyzed_merge_tree->analyzed_result_ptr : nullptr), true, - all_ranges_callback_, - read_task_callback_, - number_of_current_replica_); + all_ranges_callback, + read_task_callback, + number_of_local_replica); } Pipe ReadFromMergeTree::readFromPoolParallelReplicas(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 079f95be908..59a85413dd9 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -123,14 +123,13 @@ public: AnalysisResultPtr analyzed_result_ptr_, bool enable_parallel_reading_, std::optional all_ranges_callback_ = std::nullopt, - std::optional read_task_callback_ = std::nullopt, + std::optional read_task_callback_ = std::nullopt); std::optional number_of_current_replica_ = std::nullopt); std::unique_ptr createLocalParallelReplicasReadingStep( const ReadFromMergeTree * analyzed_merge_tree, - std::optional all_ranges_callback_, - std::optional read_task_callback_, - std::optional number_of_current_replica_); + std::optional all_ranges_callback, + std::optional read_task_callback); static constexpr auto name = "ReadFromMergeTree"; String getName() const override { return name; } @@ -291,8 +290,6 @@ private: bool enable_vertical_final = false; bool enable_remove_parts_from_snapshot_optimization = true; std::optional number_of_current_replica; - - friend class ReadFromMergeTreeCoordinated; }; } diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h index afde5f336af..8b463fda395 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.h @@ -30,8 +30,8 @@ public: /// needed to report total rows to read void setProgressCallback(ProgressCallback callback); - void initialize(CoordinationMode mode); private: + void initialize(CoordinationMode mode); std::mutex mutex; const size_t replicas_count{0}; diff --git a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql index 337b05263d0..7dbc389b55c 100644 --- a/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql +++ b/tests/queries/0_stateless/02769_parallel_replicas_unavailable_shards.sql @@ -6,7 +6,7 @@ SET allow_experimental_parallel_reading_from_replicas=2, max_parallel_replicas=1 SET send_logs_level='error'; -- with local plan for initiator, the query can be executed fast on initator, we can simply not come to the point where unavailable replica can be detected -- therefore disable local plan for now -SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79', allow_experimental_analyzer=0; +SELECT count() FROM test_parallel_replicas_unavailable_shards WHERE NOT ignore(*) SETTINGS log_comment = '02769_7b513191-5082-4073-8568-53b86a49da79', parallel_replicas_local_plan=0; SYSTEM FLUSH LOGS; From 510cb961a1d41b592f986b98f2c40a360cd83314 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 25 Jun 2024 14:47:38 +0000 Subject: [PATCH 040/162] Fix --- src/Interpreters/ClusterProxy/executeQuery.cpp | 3 +-- src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp | 5 ----- src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp | 3 +-- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 4 ++-- src/Processors/QueryPlan/ReadFromMergeTree.h | 4 ++-- 5 files changed, 6 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index ab25da090d6..da88f16b504 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -524,8 +524,7 @@ void executeQueryWithParallelReplicas( new_context, processed_stage, coordinator, - std::move(analyzed_read_from_merge_tree), - /*has_missing_objects=*/false); + std::move(analyzed_read_from_merge_tree)); DataStreams input_streams; input_streams.reserve(2); diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index 1f4f271fa6e..4d4ac69ec25 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -9,9 +9,6 @@ namespace DB { -namespace -{ - void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects) { if (blocksHaveEqualStructure(plan.getCurrentDataStream().header, header)) @@ -36,8 +33,6 @@ void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missi plan.addStep(std::move(converting)); } -} - std::unique_ptr createLocalPlan( const ASTPtr & query_ast, const Block & header, diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index 6822d8b0a71..67c10985408 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -79,9 +79,8 @@ std::unique_ptr createLocalPlanForParallelReplicas( MergeTreeReadTaskCallback read_task_cb = [coordinator](ParallelReadRequest req) -> std::optional { return coordinator->handleRequest(std::move(req)); }; - const auto number_of_local_replica = 0; auto read_from_merge_tree_parallel_replicas - = reading->createLocalParallelReplicasReadingStep(analyzed_merge_tree, all_ranges_cb, read_task_cb, number_of_local_replica); + = reading->createLocalParallelReplicasReadingStep(analyzed_merge_tree, all_ranges_cb, read_task_cb); node->step = std::move(read_from_merge_tree_parallel_replicas); addConvertingActions(*query_plan, header, /*has_missing_objects=*/false); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 1de701cff0b..0ed06639f8c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -356,8 +356,8 @@ std::unique_ptr ReadFromMergeTree::createLocalParallelReplica log, (analyzed_merge_tree ? analyzed_merge_tree->analyzed_result_ptr : nullptr), true, - all_ranges_callback, - read_task_callback, + all_ranges_callback_, + read_task_callback_, number_of_local_replica); } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 59a85413dd9..fd66a8af126 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -123,12 +123,12 @@ public: AnalysisResultPtr analyzed_result_ptr_, bool enable_parallel_reading_, std::optional all_ranges_callback_ = std::nullopt, - std::optional read_task_callback_ = std::nullopt); + std::optional read_task_callback_ = std::nullopt, std::optional number_of_current_replica_ = std::nullopt); std::unique_ptr createLocalParallelReplicasReadingStep( const ReadFromMergeTree * analyzed_merge_tree, - std::optional all_ranges_callback, + std::optional all_ranges_callback_, std::optional read_task_callback); static constexpr auto name = "ReadFromMergeTree"; From 0eff924475af085f055fb04f514ea656446c9996 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 25 Jun 2024 15:11:02 +0000 Subject: [PATCH 041/162] Use parallel_replicas_local_plan in test --- .../queries/0_stateless/03006_parallel_replicas_prewhere.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql index afe6a00cc4d..de43302690a 100644 --- a/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql +++ b/tests/queries/0_stateless/03006_parallel_replicas_prewhere.sql @@ -27,7 +27,7 @@ EXPLAIN ESTIMATE SELECT count() FROM url_na_log PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20') -SETTINGS allow_experimental_analyzer=0; +SETTINGS parallel_replicas_local_plan=0; -- here parallel replicas uses local snapshot as working set -- so, the estimation can be done @@ -35,7 +35,7 @@ EXPLAIN ESTIMATE SELECT count() FROM url_na_log PREWHERE (DateVisit >= toFixedString('2022-08-10', 10)) AND (DateVisit <= '2022-08-20') -SETTINGS allow_experimental_analyzer=1; +SETTINGS allow_experimental_analyzer=1, parallel_replicas_local_plan=1; DROP POLICY url_na_log_policy0 ON url_na_log; DROP TABLE url_na_log; From 64cfe1628f74c3797455c26b47770012fe5119b1 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 25 Jun 2024 15:21:44 +0000 Subject: [PATCH 042/162] Cleanup ReadFromMergeTree.h --- src/Processors/QueryPlan/ReadFromMergeTree.h | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index fd66a8af126..99205c1de1d 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -199,7 +199,6 @@ public: AnalysisResultPtr getAnalyzedResult() const { return analyzed_result_ptr; } void setAnalyzedResult(AnalysisResultPtr analyzed_result_ptr_) { analyzed_result_ptr = std::move(analyzed_result_ptr_); } - ReadFromMergeTree::AnalysisResult getAnalysisResult() const; const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; } const std::vector & getAlterConvertionsForParts() const { return alter_conversions_for_parts; } @@ -211,12 +210,6 @@ public: void applyFilters(ActionDAGNodes added_filter_nodes) override; - ReadType getReadType() const - { - chassert(analyzed_result_ptr); - return analyzed_result_ptr->read_type; - } - private: int getSortDirection() const { @@ -281,6 +274,8 @@ private: Pipe spreadMarkRangesAmongStreamsFinal( RangesInDataParts && parts, size_t num_streams, const Names & origin_column_names, const Names & column_names, ActionsDAGPtr & out_projection); + ReadFromMergeTree::AnalysisResult getAnalysisResult() const; + mutable AnalysisResultPtr analyzed_result_ptr; VirtualFields shared_virtual_fields; From 3d35d31655edd271dcdc1b6e3a562edbcb8c17fd Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 25 Jun 2024 16:11:04 +0000 Subject: [PATCH 043/162] More cleanup --- src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp | 10 +++++++--- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 7 ++++--- src/Processors/QueryPlan/ReadFromMergeTree.h | 4 ++-- 3 files changed, 13 insertions(+), 8 deletions(-) diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index 67c10985408..5f48a12072b 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -69,9 +69,13 @@ std::unique_ptr createLocalPlanForParallelReplicas( chassert(reading); - ReadFromMergeTree * analyzed_merge_tree = nullptr; + ReadFromMergeTree::AnalysisResultPtr analyzed_result_ptr; if (analyzed_read_from_merge_tree.get()) - analyzed_merge_tree = typeid_cast(analyzed_read_from_merge_tree.get()); + { + auto * analyzed_merge_tree = typeid_cast(analyzed_read_from_merge_tree.get()); + if (analyzed_merge_tree) + analyzed_result_ptr = analyzed_merge_tree->getAnalyzedResult(); + } MergeTreeAllRangesCallback all_ranges_cb = [coordinator](InitialAllRangesAnnouncement announcement) { coordinator->handleInitialAllRangesAnnouncement(announcement); }; @@ -80,7 +84,7 @@ std::unique_ptr createLocalPlanForParallelReplicas( { return coordinator->handleRequest(std::move(req)); }; auto read_from_merge_tree_parallel_replicas - = reading->createLocalParallelReplicasReadingStep(analyzed_merge_tree, all_ranges_cb, read_task_cb); + = reading->createLocalParallelReplicasReadingStep(analyzed_result_ptr, all_ranges_cb, read_task_cb); node->step = std::move(read_from_merge_tree_parallel_replicas); addConvertingActions(*query_plan, header, /*has_missing_objects=*/false); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 0ed06639f8c..95c337e2150 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -337,11 +337,12 @@ ReadFromMergeTree::ReadFromMergeTree( } std::unique_ptr ReadFromMergeTree::createLocalParallelReplicasReadingStep( - const ReadFromMergeTree * analyzed_merge_tree, + AnalysisResultPtr analyzed_result_ptr_, std::optional all_ranges_callback_, std::optional read_task_callback_) { const auto number_of_local_replica = 0; + const bool enable_parallel_reading = true; return std::make_unique( prepared_parts, alter_conversions_for_parts, @@ -354,8 +355,8 @@ std::unique_ptr ReadFromMergeTree::createLocalParallelReplica requested_num_streams, max_block_numbers_to_read, log, - (analyzed_merge_tree ? analyzed_merge_tree->analyzed_result_ptr : nullptr), - true, + analyzed_result_ptr_, + enable_parallel_reading, all_ranges_callback_, read_task_callback_, number_of_local_replica); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 99205c1de1d..c9c6030d207 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -127,9 +127,9 @@ public: std::optional number_of_current_replica_ = std::nullopt); std::unique_ptr createLocalParallelReplicasReadingStep( - const ReadFromMergeTree * analyzed_merge_tree, + AnalysisResultPtr analyzed_result_ptr_, std::optional all_ranges_callback_, - std::optional read_task_callback); + std::optional read_task_callback_); static constexpr auto name = "ReadFromMergeTree"; String getName() const override { return name; } From 7c6293a747a813742c663d59342c0c0cc9db4956 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 26 Jun 2024 10:12:09 +0000 Subject: [PATCH 044/162] Fix test_parallel_replicas_no_replicas --- tests/integration/test_parallel_replicas_no_replicas/test.py | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/integration/test_parallel_replicas_no_replicas/test.py b/tests/integration/test_parallel_replicas_no_replicas/test.py index 9f716459643..73ab6fde5a3 100644 --- a/tests/integration/test_parallel_replicas_no_replicas/test.py +++ b/tests/integration/test_parallel_replicas_no_replicas/test.py @@ -46,5 +46,6 @@ def test_skip_all_replicas(start_cluster, skip_unavailable_shards): "max_parallel_replicas": 3, "cluster_for_parallel_replicas": cluster_name, "skip_unavailable_shards": skip_unavailable_shards, + "parallel_replicas_local_plan": 0, }, ) From 372b948d34fe24257e096a6b7d6c9ea03ba0023a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 27 Jun 2024 20:30:05 +0000 Subject: [PATCH 045/162] Fix: progress bar, reading in order --- src/Interpreters/ClusterProxy/executeQuery.cpp | 16 ++++++++-------- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 10 ++++++---- ...2898_parallel_replicas_progress_bar.reference | 6 ++++++ .../02898_parallel_replicas_progress_bar.sql | 10 +++++----- 4 files changed, 25 insertions(+), 17 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index da88f16b504..9302baf57ca 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -500,6 +500,14 @@ void executeQueryWithParallelReplicas( /// do not build local plan for distributed queries for now (address it later) if (settings.allow_experimental_analyzer && settings.parallel_replicas_local_plan && !shard_num) { + auto local_plan = createLocalPlanForParallelReplicas( + query_ast, + header, + new_context, + processed_stage, + coordinator, + std::move(analyzed_read_from_merge_tree)); + auto read_from_remote = std::make_unique( query_ast, new_cluster, @@ -518,14 +526,6 @@ void executeQueryWithParallelReplicas( auto remote_plan = std::make_unique(); remote_plan->addStep(std::move(read_from_remote)); - auto local_plan = createLocalPlanForParallelReplicas( - query_ast, - header, - new_context, - processed_stage, - coordinator, - std::move(analyzed_read_from_merge_tree)); - DataStreams input_streams; input_streams.reserve(2); input_streams.emplace_back(local_plan->getCurrentDataStream()); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 7cf574ddf5c..e4e251b694e 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -586,12 +586,14 @@ Pipe ReadFromMergeTree::readInOrder( context); } - /// Actually it means that parallel reading from replicas enabled - /// and we have to collaborate with initiator. + /// Actually it means that parallel reading from replicas enabled and read snapshot is not local - + /// we can't rely on local snapshot /// In this case we won't set approximate rows, because it will be accounted multiple times. /// Also do not count amount of read rows if we read in order of sorting key, /// because we don't know actual amount of read rows in case when limit is set. - bool set_rows_approx = !is_parallel_reading_from_replicas && !reader_settings.read_in_order; + const UInt64 in_order_limit = query_info.input_order_info ? query_info.input_order_info->limit : 0; + const bool set_total_rows_approx + = !(is_parallel_reading_from_replicas && context->canUseParallelReplicasOnFollower()) && !in_order_limit; Pipes pipes; for (size_t i = 0; i < parts_with_ranges.size(); ++i) @@ -621,7 +623,7 @@ Pipe ReadFromMergeTree::readInOrder( processor->addPartLevelToChunk(isQueryWithFinal()); auto source = std::make_shared(std::move(processor)); - if (set_rows_approx) + if (set_total_rows_approx) source->addTotalRowsApprox(total_rows); pipes.emplace_back(std::move(source)); diff --git a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.reference b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.reference index 380aac4dbe8..c66597436f3 100644 --- a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.reference +++ b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.reference @@ -1,2 +1,8 @@ 3000 1000 3999 2499.5 1 +1998 2944475297004403859 +1999 254596732598015005 +2000 6863370867519437063 +2001 17844331710293705251 +2002 1587587338113897332 +1 diff --git a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql index 42f8091db08..d3bf228e0fb 100644 --- a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql +++ b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql @@ -26,12 +26,12 @@ WHERE query_id in (select query_id from system.query_log where current_database AND message LIKE '%Total rows to read: 3000%' SETTINGS allow_experimental_parallel_reading_from_replicas=0; -- reading in order coordinator --- SELECT k, sipHash64(v) FROM t1 order by k limit 5 offset 998 SETTINGS optimize_read_in_order=1, log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b'; +SELECT k, sipHash64(v) FROM t1 order by k limit 5 offset 998 SETTINGS optimize_read_in_order=1, parallel_replicas_local_plan=0, log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b'; --- SYSTEM FLUSH LOGS; --- SELECT count() > 0 FROM system.text_log --- WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b') --- AND message LIKE '%Updated total rows to read: added % rows, total 3000 rows%' SETTINGS allow_experimental_parallel_reading_from_replicas=0; +SYSTEM FLUSH LOGS; +SELECT count() > 0 FROM system.text_log +WHERE query_id in (select query_id from system.query_log where current_database = currentDatabase() AND log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b') + AND message LIKE '%Updated total rows to read: added % rows, total 3000 rows%' SETTINGS allow_experimental_parallel_reading_from_replicas=0; DROP TABLE t1 SYNC; DROP TABLE t2 SYNC; From ebcf455f4a88043234fb19770bfc29958789c090 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 28 Jun 2024 22:17:34 +0000 Subject: [PATCH 046/162] Fix: progress bar for reading in order --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 9 +++++---- .../MergeTree/ParallelReplicasReadingCoordinator.cpp | 6 +++--- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index e4e251b694e..08c6989242e 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -592,8 +592,10 @@ Pipe ReadFromMergeTree::readInOrder( /// Also do not count amount of read rows if we read in order of sorting key, /// because we don't know actual amount of read rows in case when limit is set. const UInt64 in_order_limit = query_info.input_order_info ? query_info.input_order_info->limit : 0; - const bool set_total_rows_approx - = !(is_parallel_reading_from_replicas && context->canUseParallelReplicasOnFollower()) && !in_order_limit; + const bool parallel_replicas_remote_plan_for_initiator = is_parallel_reading_from_replicas + && !context->getSettingsRef().parallel_replicas_local_plan && context->canUseParallelReplicasOnInitiator(); + const bool parallel_replicas_follower = is_parallel_reading_from_replicas && context->canUseParallelReplicasOnFollower(); + const bool set_total_rows_approx = !parallel_replicas_follower && !parallel_replicas_remote_plan_for_initiator && !in_order_limit; Pipes pipes; for (size_t i = 0; i < parts_with_ranges.size(); ++i) @@ -1968,8 +1970,7 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons mode = CoordinationMode::ReverseOrder; break; case ReadFromMergeTree::ReadType::ParallelReplicas: - chassert(false); - UNREACHABLE(); + throw Exception(ErrorCodes::LOGICAL_ERROR, "Read type can't be ParallelReplicas on initiator"); } chassert(number_of_current_replica.has_value()); diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 601ab3aeb70..f66cfdafa1a 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -888,9 +888,8 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa ++stats[announcement.replica_num].number_of_requests; - /// FIXME: this code updating total_rows_to_read but it needs to be done only once since we're taking working set from initiator - /// util I missing something, it seems this code is not necessary if working set is taken from initiator (todo: check it) - if (new_rows_to_read > 0 && progress_callback) + // progress_callback is not set when local plan is used for initiator + if (progress_callback && new_rows_to_read > 0) { Progress progress; progress.total_rows_to_read = new_rows_to_read; @@ -1052,6 +1051,7 @@ void ParallelReplicasReadingCoordinator::initialize(CoordinationMode mode) break; } + // progress_callback is not set when local plan is used for initiator if (progress_callback) pimpl->setProgressCallback(std::move(progress_callback)); From 37fbf905dda2cdeb683488b46ff364531d1ac2d5 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 30 Jun 2024 20:33:43 +0000 Subject: [PATCH 047/162] Use only local snapshot for in order coordinator --- .../ParallelReplicasReadingCoordinator.cpp | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index f66cfdafa1a..b510da13d90 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -836,6 +836,7 @@ public: Parts all_parts_to_read; size_t total_rows_to_read = 0; + bool state_initialized{false}; LoggerPtr log = getLogger(fmt::format("{}{}", magic_enum::enum_name(mode), "Coordinator")); }; @@ -857,6 +858,11 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa { LOG_TRACE(log, "Received an announcement {}", announcement.describe()); + ++stats[announcement.replica_num].number_of_requests; + + if (state_initialized) + return; + size_t new_rows_to_read = 0; /// To get rid of duplicates @@ -886,7 +892,7 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa std::sort(ranges.begin(), ranges.end()); } - ++stats[announcement.replica_num].number_of_requests; + state_initialized = true; // progress_callback is not set when local plan is used for initiator if (progress_callback && new_rows_to_read > 0) @@ -923,8 +929,15 @@ ParallelReadResponse InOrderCoordinator::handleRequest(ParallelReadRequest if (global_part_it == all_parts_to_read.end()) continue; + if (global_part_it->replicas.empty()) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Part {} requested by replica {} is not registered in working set", + part.info.getPartNameV1(), + request.replica_num); + if (!global_part_it->replicas.contains(request.replica_num)) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Part {} doesn't exist on replica {} according to the global state", part.info.getPartNameV1(), request.replica_num); + continue; size_t current_mark_size = 0; From 6b3750ff83f0801322ac0870a30339e34e9182aa Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 1 Jul 2024 08:28:54 +0000 Subject: [PATCH 048/162] Clean up --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 3 ++- .../queries/0_stateless/02841_parallel_replicas_summary.sh | 6 +++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 7d798e2a399..8ecb36aab7f 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -1954,7 +1954,8 @@ void ReadFromMergeTree::initializePipeline(QueryPipelineBuilder & pipeline, cons { auto result = getAnalysisResult(); - if (is_parallel_reading_from_replicas && context->canUseParallelReplicasOnInitiator()) + if (is_parallel_reading_from_replicas && context->canUseParallelReplicasOnInitiator() + && context->getSettingsRef().parallel_replicas_local_plan) { CoordinationMode mode = CoordinationMode::Default; switch (result.read_type) diff --git a/tests/queries/0_stateless/02841_parallel_replicas_summary.sh b/tests/queries/0_stateless/02841_parallel_replicas_summary.sh index bff56feacbf..01e2883c547 100755 --- a/tests/queries/0_stateless/02841_parallel_replicas_summary.sh +++ b/tests/queries/0_stateless/02841_parallel_replicas_summary.sh @@ -27,7 +27,7 @@ $CLICKHOUSE_CLIENT --query "CREATE TABLE replicas_summary (n Int64) ENGINE = Mer query_id_base="02841_summary_$CLICKHOUSE_DATABASE" -# TODO: rethink the test, for now temporary disable allow_experimental_analyzer +# TODO: rethink the test, for now temporary disable parallel_replicas_local_plan echo " SELECT * FROM replicas_summary @@ -38,7 +38,7 @@ echo " allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, interactive_delay=0, - allow_experimental_analyzer=0 + parallel_replicas_local_plan=0 "\ | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_0" --data-binary @- -vvv 2>&1 \ | grep "Summary" | grep -cv '"read_rows":"0"' @@ -53,7 +53,7 @@ echo " allow_experimental_parallel_reading_from_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, interactive_delay=99999999999, - allow_experimental_analyzer=0 + parallel_replicas_local_plan=0 "\ | ${CLICKHOUSE_CURL} -sS "${CLICKHOUSE_URL}&wait_end_of_query=1&query_id=${query_id_base}_interactive_high" --data-binary @- -vvv 2>&1 \ | grep "Summary" | grep -cv '"read_rows":"0"' From 105d39b09f241f4b48b70425ba07288d32ae107f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 2 Jul 2024 15:06:19 +0000 Subject: [PATCH 049/162] Twick number of threads --- src/Planner/PlannerJoinTree.cpp | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 094562a2837..a4a8dd08561 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -958,6 +958,18 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres table_expression_query_info.storage_limits, std::move(reading_step)); query_plan = std::move(query_plan_parallel_replicas); + + if (settings.parallel_replicas_local_plan) + { + const auto old_max_threads = query_plan.getMaxThreads(); + query_plan.setMaxThreads(old_max_threads * 2); + + LOG_TRACE( + getLogger("Planner"), + "Increase max threads from {} to {} to have similar number of threads to remote plan", + old_max_threads, + query_plan.getMaxThreads()); + } } else { From 04c3661b0b18eb4be82ecf4261ded70fa52d7bcc Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 12 Jul 2024 13:42:25 +0000 Subject: [PATCH 050/162] Randomize parallel_replicas_local_plan --- src/Interpreters/ClusterProxy/executeQuery.cpp | 2 -- src/Planner/PlannerJoinTree.cpp | 12 ------------ tests/clickhouse-test | 1 + .../02731_parallel_replicas_join_subquery.sql | 1 + .../0_stateless/02771_parallel_replicas_analyzer.sql | 2 ++ 5 files changed, 4 insertions(+), 14 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 832f04913c7..6039c545085 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -29,8 +29,6 @@ #include #include #include -#include -#include #include #include diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 1d9972ae03f..6d8d58fc8ef 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -998,18 +998,6 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres table_expression_query_info.storage_limits, std::move(reading_step)); query_plan = std::move(query_plan_parallel_replicas); - - if (settings.parallel_replicas_local_plan) - { - const auto old_max_threads = query_plan.getMaxThreads(); - query_plan.setMaxThreads(old_max_threads * 2); - - LOG_TRACE( - getLogger("Planner"), - "Increase max threads from {} to {} to have similar number of threads to remote plan", - old_max_threads, - query_plan.getMaxThreads()); - } } else { diff --git a/tests/clickhouse-test b/tests/clickhouse-test index 958dde0606f..77e984aa960 100755 --- a/tests/clickhouse-test +++ b/tests/clickhouse-test @@ -832,6 +832,7 @@ class SettingsRandomizer: "cross_join_min_bytes_to_compress": lambda: random.choice([0, 1, 100000000]), "min_external_table_block_size_bytes": lambda: random.choice([0, 1, 100000000]), "max_parsing_threads": lambda: random.choice([0, 1, 10]), + "parallel_replicas_local_plan": lambda: random.randint(0, 1), } @staticmethod diff --git a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql index e91e2a19526..f15e7631753 100644 --- a/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql +++ b/tests/queries/0_stateless/02731_parallel_replicas_join_subquery.sql @@ -23,6 +23,7 @@ LIMIT 100; SET max_parallel_replicas = 3; SET cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; +SET parallel_replicas_local_plan = 1; SET joined_subquery_requires_alias = 0; SELECT '=============== INNER QUERY (NO PARALLEL) ==============='; diff --git a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql index 7e27507ada9..3a19768c0cc 100644 --- a/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql +++ b/tests/queries/0_stateless/02771_parallel_replicas_analyzer.sql @@ -17,6 +17,8 @@ INSERT INTO join_inner_table__fuzz_146_replicated SELECT CAST('833c9e22-c245-4eb5-8745-117a9a1f26b1', 'UUID') AS id, CAST(rowNumberInAllBlocks(), 'String') AS key, * FROM generateRandom('number Int64, value1 String, value2 String, time Int64', 1, 10, 2) LIMIT 10; +SET parallel_replicas_local_plan = 1; + -- Simple query with analyzer and pure parallel replicas SELECT number FROM join_inner_table__fuzz_146_replicated From 4e44ecf286b85d56ba4c17bc6877d62fff98828b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 12 Jul 2024 13:52:33 +0000 Subject: [PATCH 051/162] Fixate setting in 02967_parallel_replicas_joins_and_analyzer --- .../02967_parallel_replicas_joins_and_analyzer.sql.j2 | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 index 54505b147a3..fd03e4c0857 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.sql.j2 @@ -10,6 +10,8 @@ insert into tab1 select number, number, number from numbers(16); insert into tab2 select number * 2, number * 2 from numbers(8); insert into tab3 select number * 4, number * 4 from numbers(4); +set parallel_replicas_local_plan=1; + {% for use_global_in in [0, 1] -%} -- { echoOn } From e22d6035fa3496108b2c9619dcb3fa54c131bf6e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 12 Jul 2024 14:11:04 +0000 Subject: [PATCH 052/162] Cleanup --- tests/queries/0_stateless/02404_memory_bound_merging.sql | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/queries/0_stateless/02404_memory_bound_merging.sql b/tests/queries/0_stateless/02404_memory_bound_merging.sql index 0167c27ae44..112640a2e2c 100644 --- a/tests/queries/0_stateless/02404_memory_bound_merging.sql +++ b/tests/queries/0_stateless/02404_memory_bound_merging.sql @@ -66,7 +66,6 @@ insert into pr_t select number % 1000, number % 1000 from numbers_mt(1e6); set allow_experimental_parallel_reading_from_replicas = 1; set parallel_replicas_for_non_replicated_merge_tree = 1; set max_parallel_replicas = 3; -set use_hedged_requests = 0; set cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost'; set distributed_aggregation_memory_efficient=1; From b952a11f35c93b4df1e97325119c6295791ec748 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 12 Jul 2024 21:20:23 +0000 Subject: [PATCH 053/162] Fix 00177_memory_bound_merging --- tests/queries/1_stateful/00177_memory_bound_merging.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/1_stateful/00177_memory_bound_merging.sh b/tests/queries/1_stateful/00177_memory_bound_merging.sh index 74b5e80e059..1110ab9a61d 100755 --- a/tests/queries/1_stateful/00177_memory_bound_merging.sh +++ b/tests/queries/1_stateful/00177_memory_bound_merging.sh @@ -64,7 +64,7 @@ test3() { FROM test.hits WHERE CounterID = 1704509 AND UserID = 4322253409885123546 GROUP BY URL, EventDate - SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3 + SETTINGS optimize_aggregation_in_order = 1, enable_memory_bound_merging_of_aggregation_results = 1, allow_experimental_parallel_reading_from_replicas = 1, parallel_replicas_for_non_replicated_merge_tree = 1, max_parallel_replicas = 3, parallel_replicas_local_plan=1 ) WHERE explain LIKE '%Aggr%Transform%' OR explain LIKE '%InOrder%'" } From 32dc3fe8d1c0384221cd1f27adc53bfe45a01e28 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 13 Jul 2024 21:19:14 +0000 Subject: [PATCH 054/162] Simplify code --- src/Interpreters/ClusterProxy/executeQuery.cpp | 4 +--- src/Processors/QueryPlan/ReadFromRemote.cpp | 10 ++-------- .../02898_parallel_replicas_progress_bar.sql | 1 + 3 files changed, 4 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 6039c545085..79f4344e6cc 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -519,9 +519,7 @@ void executeQueryWithParallelReplicas( "`cluster_for_parallel_replicas` setting refers to cluster with several shards. Expected a cluster with one shard"); } - auto replica_count = new_cluster->getShardsInfo().begin()->getAllNodeCount(); - if (settings.max_parallel_replicas < replica_count) - replica_count = settings.max_parallel_replicas; + const auto replica_count = std::min(settings.max_parallel_replicas.value, new_cluster->getShardsInfo().begin()->getAllNodeCount()); auto coordinator = std::make_shared(replica_count, settings.parallel_replicas_mark_segment_size); diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 26c57c4a760..3da22265c5c 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -458,7 +458,6 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder pools_to_use.reserve(shuffled_pool.size()); if (exclude_local_replica) { - std::vector local_addr_possitions; for (auto & pool : shuffled_pool) { const auto & hostname = pool.pool->getHost(); @@ -466,17 +465,12 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder begin(shard.local_addresses), end(shard.local_addresses), [&hostname](const Cluster::Address & local_addr) { return hostname == local_addr.host_name; }); - if (it != shard.local_addresses.end()) + if (it == shard.local_addresses.end()) { - pool.pool.reset(); + pools_to_use.push_back(pool.pool); } } } - for (const auto & pool : shuffled_pool) - { - if (pool.pool) - pools_to_use.push_back(pool.pool); - } LOG_DEBUG( getLogger("ReadFromParallelRemoteReplicasStep"), diff --git a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql index d3bf228e0fb..9348ea1dc32 100644 --- a/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql +++ b/tests/queries/0_stateless/02898_parallel_replicas_progress_bar.sql @@ -26,6 +26,7 @@ WHERE query_id in (select query_id from system.query_log where current_database AND message LIKE '%Total rows to read: 3000%' SETTINGS allow_experimental_parallel_reading_from_replicas=0; -- reading in order coordinator +-- disable parallel_replicas_local_plan since the test relay on traces which only present in case of no local plan SELECT k, sipHash64(v) FROM t1 order by k limit 5 offset 998 SETTINGS optimize_read_in_order=1, parallel_replicas_local_plan=0, log_comment='02898_inorder_190aed82-2423-413b-ad4c-24dcca50f65b'; SYSTEM FLUSH LOGS; From 0c986cca7c91a462ec77fc6a198e65fdb0ef4b10 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 14 Jul 2024 16:37:00 +0000 Subject: [PATCH 055/162] Fix in-order coordinator + register all replicas for parts in working set --- .../MergeTree/ParallelReplicasReadingCoordinator.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 14c706c77ef..ff309f17893 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -863,9 +863,6 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa ++stats[announcement.replica_num].number_of_requests; - if (state_initialized) - return; - size_t new_rows_to_read = 0; /// To get rid of duplicates @@ -874,13 +871,16 @@ void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRa auto the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), [&part] (const Part & other) { return other.description.info == part.info; }); - /// We have the same part - add the info about presence on current replica to it + /// We have the same part - add the info about presence on the corresponding replica to it if (the_same_it != all_parts_to_read.end()) { the_same_it->replicas.insert(announcement.replica_num); continue; } + if (state_initialized) + continue; + auto covering_or_the_same_it = std::find_if(all_parts_to_read.begin(), all_parts_to_read.end(), [&part] (const Part & other) { return other.description.info.contains(part.info) || part.info.contains(other.description.info); }); From 9f20e33d50810f20ea17355512abed0a04a495a2 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 14 Jul 2024 17:52:51 +0000 Subject: [PATCH 056/162] Fix --- src/Processors/QueryPlan/ReadFromRemote.cpp | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 3da22265c5c..712f714376b 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -456,9 +456,9 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder std::vector pools_to_use; pools_to_use.reserve(shuffled_pool.size()); - if (exclude_local_replica) + for (const auto & pool : shuffled_pool) { - for (auto & pool : shuffled_pool) + if (exclude_local_replica) { const auto & hostname = pool.pool->getHost(); auto it = std::find_if( @@ -466,9 +466,11 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder end(shard.local_addresses), [&hostname](const Cluster::Address & local_addr) { return hostname == local_addr.host_name; }); if (it == shard.local_addresses.end()) - { pools_to_use.push_back(pool.pool); - } + } + else + { + pools_to_use.push_back(pool.pool); } } From f7befaf68e19d60fe3678c15b7367b815ce6da17 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 14 Jul 2024 18:46:14 +0000 Subject: [PATCH 057/162] Polish + comments --- src/Processors/QueryPlan/ReadFromRemote.cpp | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 712f714376b..c05bda62648 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -474,18 +474,17 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder } } + pools_to_use.resize(std::min(pools_to_use.size(), all_replicas_count)); + // if local plan is used for local replica, we should exclude one remote replica + if (exclude_local_replica && !pools_to_use.empty()) + pools_to_use.resize(all_replicas_count - 1); + LOG_DEBUG( getLogger("ReadFromParallelRemoteReplicasStep"), "Number of pools to use is {}. Originally {}", pools_to_use.size(), shuffled_pool.size()); - if (pools_to_use.size() > all_replicas_count) - pools_to_use.resize(all_replicas_count); - - if (exclude_local_replica && !pools_to_use.empty()) - pools_to_use.resize(all_replicas_count - 1); - if (pools_to_use.empty()) return; @@ -497,7 +496,7 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder LOG_DEBUG(getLogger("ReadFromParallelRemoteReplicasStep"), "Addresses to use: {}", pool_addresses); } - /// local replicas has number 0 + /// when using local plan for local replica, local replica has 0 number size_t offset = (exclude_local_replica ? 1 : 0); for (size_t i = 0 + offset; i < all_replicas_count; ++i) { From 261ff133b82a61885a54e10ac890be2b8eb44831 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 15 Jul 2024 12:03:12 +0000 Subject: [PATCH 058/162] Fix 02784_parallel_replicas_automatic_decision_join --- .../02784_parallel_replicas_automatic_decision.sh | 1 - ...parallel_replicas_automatic_decision_join.reference | 10 +++++----- .../02784_parallel_replicas_automatic_decision_join.sh | 10 +++++----- 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index dd9b5c3f6d9..e5658f31a34 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -6,7 +6,6 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CLICKHOUSE_CLIENT_TRACE=${CLICKHOUSE_CLIENT/"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"/"--send_logs_level=trace"} function were_parallel_replicas_used () { - # Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it $CLICKHOUSE_CLIENT --query " SELECT initial_query_id, diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference index 521e3e2edbc..595b35db610 100644 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.reference @@ -4,8 +4,8 @@ 02784_automatic_parallel_replicas_join-default_simple_join_1M_pure 10 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 3 estimated parallel replicas 02784_automatic_parallel_replicas_join-default_simple_join_300k_pure 33 estimated parallel replicas -02784_automatic_parallel_replicas_join-default_simple_join_0_pure Distinct parallel subqueries: 2 Used parallel replicas: true -02784_automatic_parallel_replicas_join-default_simple_join_10M_pure Distinct parallel subqueries: 0 Used parallel replicas: false -02784_automatic_parallel_replicas_join-default_simple_join_5M_pure Distinct parallel subqueries: 1 Used parallel replicas: true -02784_automatic_parallel_replicas_join-default_simple_join_1M_pure Distinct parallel subqueries: 1 Used parallel replicas: true -02784_automatic_parallel_replicas_join-default_simple_join_300k_pure Distinct parallel subqueries: 2 Used parallel replicas: true +02784_automatic_parallel_replicas_join-default_simple_join_0_pure Used parallel replicas: true +02784_automatic_parallel_replicas_join-default_simple_join_10M_pure Used parallel replicas: false +02784_automatic_parallel_replicas_join-default_simple_join_5M_pure Used parallel replicas: true +02784_automatic_parallel_replicas_join-default_simple_join_1M_pure Used parallel replicas: true +02784_automatic_parallel_replicas_join-default_simple_join_300k_pure Used parallel replicas: true diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh index 801cd22b79f..238d63e6980 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision_join.sh @@ -9,17 +9,17 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) CLICKHOUSE_CLIENT_TRACE=${CLICKHOUSE_CLIENT/"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"/"--send_logs_level=trace"} function were_parallel_replicas_used () { - # Not using current_database = '$CLICKHOUSE_DATABASE' as nested parallel queries aren't run with it $CLICKHOUSE_CLIENT --query " SELECT initial_query_id, - concat('Distinct parallel subqueries: ' , countDistinctIf(query, initial_query_id != query_id)::String) as subqueries_parallelized, - concat('Used parallel replicas: ', (countIf(initial_query_id != query_id) != 0)::bool::String) as used + concat('Used parallel replicas: ', (ProfileEvents['ParallelReplicasUsedCount'] > 0)::bool::String) as used FROM system.query_log WHERE event_date >= yesterday() AND initial_query_id LIKE '$1%' - GROUP BY initial_query_id - ORDER BY min(event_time_microseconds) ASC + AND query_id = initial_query_id + AND type = 'QueryFinish' + AND current_database = '$CLICKHOUSE_DATABASE' + ORDER BY event_time_microseconds ASC FORMAT TSV" } From 02a2f509163e5df370440b4eed792dcbd995942c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 15 Jul 2024 18:37:19 +0000 Subject: [PATCH 059/162] Fix clang warning --- src/Interpreters/ClusterProxy/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 79f4344e6cc..4a637db3887 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -519,7 +519,7 @@ void executeQueryWithParallelReplicas( "`cluster_for_parallel_replicas` setting refers to cluster with several shards. Expected a cluster with one shard"); } - const auto replica_count = std::min(settings.max_parallel_replicas.value, new_cluster->getShardsInfo().begin()->getAllNodeCount()); + const auto replica_count = std::min(settings.max_parallel_replicas.value, new_cluster->getShardsInfo().begin()->getAllNodeCount()); auto coordinator = std::make_shared(replica_count, settings.parallel_replicas_mark_segment_size); From 860b18c251a3492682d410fbcc54576e87f4ca25 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 15 Jul 2024 20:40:55 +0000 Subject: [PATCH 060/162] Update test for in-(reverse)-order coordinator --- .../ParallelReplicasReadingCoordinator.cpp | 2 +- src/Storages/MergeTree/RequestResponse.cpp | 4 +--- ...02950_parallel_replicas_used_count.reference | 10 ++++++++-- .../02950_parallel_replicas_used_count.sql | 17 ++++++++++++----- 4 files changed, 22 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index ff309f17893..e1c0d87837a 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -918,7 +918,7 @@ ParallelReadResponse InOrderCoordinator::handleRequest(ParallelReadRequest "Replica {} decided to read in {} mode, not in {}. This is a bug", request.replica_num, magic_enum::enum_name(request.mode), magic_enum::enum_name(mode)); - LOG_TRACE(log, "Got request from replica {}, data {}", request.replica_num, request.describe()); + LOG_TRACE(log, "Got read request: {}", request.describe()); ParallelReadResponse response; response.description = request.description; diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp index 2ce0e20dcd2..33cd935c88b 100644 --- a/src/Storages/MergeTree/RequestResponse.cpp +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -44,9 +44,7 @@ void ParallelReadRequest::serialize(WriteBuffer & out) const String ParallelReadRequest::describe() const { - String result; - result += fmt::format("replica_num: {} \n", replica_num); - result += fmt::format("min_num_of_marks: {} \n", min_number_of_marks); + String result = fmt::format("replica_num {}, min_num_of_marks {}, ", replica_num, min_number_of_marks); result += description.describe(); return result; } diff --git a/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference b/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference index 21b7b527b7a..cc8284bfeac 100644 --- a/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference +++ b/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference @@ -1,8 +1,14 @@ -100 4950 +10000 49995000 1 89 90 91 92 93 -1 +3 +93 +92 +91 +90 +89 +3 diff --git a/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql b/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql index 22f55acd365..69f1dc47ded 100644 --- a/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql +++ b/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql @@ -2,11 +2,12 @@ DROP TABLE IF EXISTS test; CREATE TABLE test (k UInt64, v String) ENGINE = MergeTree -ORDER BY k; +ORDER BY k +SETTINGS index_granularity=1; -INSERT INTO test SELECT number, toString(number) FROM numbers(100); +INSERT INTO test SELECT number, toString(number) FROM numbers(10_000); -SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, prefer_localhost_replica = 0, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; +SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; -- default coordinator SELECT count(), sum(k) @@ -14,12 +15,18 @@ FROM test SETTINGS log_comment = '02950_parallel_replicas_used_replicas_count'; SYSTEM FLUSH LOGS; -SELECT ProfileEvents['ParallelReplicasUsedCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; +SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; -- In order coordinator -SELECT k FROM test order by k limit 5 offset 89 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_2'; +SELECT k FROM test order by k limit 5 offset 89 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_2', merge_tree_min_rows_for_concurrent_read=1, max_threads=1; SYSTEM FLUSH LOGS; SELECT ProfileEvents['ParallelReplicasUsedCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_2' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; +-- In reverse order coordinator +SELECT k FROM test order by k desc limit 5 offset 9906 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_3', merge_tree_min_rows_for_concurrent_read=1, max_threads=1; + +SYSTEM FLUSH LOGS; +SELECT ProfileEvents['ParallelReplicasUsedCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_3' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; + DROP TABLE test; From 69c1e68359f08ee648eba819f445cbde9b8c82e0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 16 Jul 2024 07:48:40 +0000 Subject: [PATCH 061/162] Fix clang-tidy warning: remove include duplicate --- src/Interpreters/Context.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index fc1e87e7b7e..2602afd8b78 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -51,7 +51,6 @@ #include #include #include -#include #include #include #include From 27db36cd4fbc2b0d74e982a3992778c62ccc2482 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 16 Jul 2024 08:46:24 +0000 Subject: [PATCH 062/162] Separate converting actions in separte source file --- .../QueryPlan/ConvertingActions.cpp | 32 +++++++++++++++++++ src/Processors/QueryPlan/ConvertingActions.h | 9 ++++++ .../QueryPlan/DistributedCreateLocalPlan.cpp | 27 +--------------- .../QueryPlan/ParallelReplicasLocalPlan.cpp | 5 +-- 4 files changed, 45 insertions(+), 28 deletions(-) create mode 100644 src/Processors/QueryPlan/ConvertingActions.cpp create mode 100644 src/Processors/QueryPlan/ConvertingActions.h diff --git a/src/Processors/QueryPlan/ConvertingActions.cpp b/src/Processors/QueryPlan/ConvertingActions.cpp new file mode 100644 index 00000000000..ff106ff08c1 --- /dev/null +++ b/src/Processors/QueryPlan/ConvertingActions.cpp @@ -0,0 +1,32 @@ +#include +#include +#include + +namespace DB +{ + +void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects) +{ + if (blocksHaveEqualStructure(plan.getCurrentDataStream().header, header)) + return; + + auto mode = has_missing_objects ? ActionsDAG::MatchColumnsMode::Position : ActionsDAG::MatchColumnsMode::Name; + + auto get_converting_dag = [mode](const Block & block_, const Block & header_) + { + /// Convert header structure to expected. + /// Also we ignore constants from result and replace it with constants from header. + /// It is needed for functions like `now64()` or `randConstant()` because their values may be different. + return ActionsDAG::makeConvertingActions( + block_.getColumnsWithTypeAndName(), + header_.getColumnsWithTypeAndName(), + mode, + true); + }; + + auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header); + auto converting = std::make_unique(plan.getCurrentDataStream(), convert_actions_dag); + plan.addStep(std::move(converting)); +} + +} diff --git a/src/Processors/QueryPlan/ConvertingActions.h b/src/Processors/QueryPlan/ConvertingActions.h new file mode 100644 index 00000000000..6bdf9b8af9a --- /dev/null +++ b/src/Processors/QueryPlan/ConvertingActions.h @@ -0,0 +1,9 @@ +#pragma once + +namespace DB +{ +class QueryPlan; +class Block; + +void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects); +} diff --git a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp index bad0380cb46..eb699858bdf 100644 --- a/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp +++ b/src/Processors/QueryPlan/DistributedCreateLocalPlan.cpp @@ -2,38 +2,13 @@ #include #include -#include #include #include -#include +#include namespace DB { -void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects) -{ - if (blocksHaveEqualStructure(plan.getCurrentDataStream().header, header)) - return; - - auto mode = has_missing_objects ? ActionsDAG::MatchColumnsMode::Position : ActionsDAG::MatchColumnsMode::Name; - - auto get_converting_dag = [mode](const Block & block_, const Block & header_) - { - /// Convert header structure to expected. - /// Also we ignore constants from result and replace it with constants from header. - /// It is needed for functions like `now64()` or `randConstant()` because their values may be different. - return ActionsDAG::makeConvertingActions( - block_.getColumnsWithTypeAndName(), - header_.getColumnsWithTypeAndName(), - mode, - true); - }; - - auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header); - auto converting = std::make_unique(plan.getCurrentDataStream(), convert_actions_dag); - plan.addStep(std::move(converting)); -} - std::unique_ptr createLocalPlan( const ASTPtr & query_ast, const Block & header, diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index 5f48a12072b..d2e862a3416 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -77,8 +78,8 @@ std::unique_ptr createLocalPlanForParallelReplicas( analyzed_result_ptr = analyzed_merge_tree->getAnalyzedResult(); } - MergeTreeAllRangesCallback all_ranges_cb - = [coordinator](InitialAllRangesAnnouncement announcement) { coordinator->handleInitialAllRangesAnnouncement(announcement); }; + MergeTreeAllRangesCallback all_ranges_cb = [coordinator](InitialAllRangesAnnouncement announcement) + { coordinator->handleInitialAllRangesAnnouncement(std::move(announcement)); }; MergeTreeReadTaskCallback read_task_cb = [coordinator](ParallelReadRequest req) -> std::optional { return coordinator->handleRequest(std::move(req)); }; From 4caf9b6e6fa53adb1e7208c992469175a9344c2f Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 16 Jul 2024 08:57:15 +0000 Subject: [PATCH 063/162] Better diagnostic message formatting --- src/Processors/QueryPlan/ReadFromRemote.cpp | 22 ++++++++------------- 1 file changed, 8 insertions(+), 14 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 3c6c651ae02..caa332c2ebc 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -22,7 +22,7 @@ #include #include -#include +#include namespace DB { @@ -391,15 +391,11 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( std::vector replicas; replicas.reserve(cluster->getShardsAddresses().front().size()); - bool first_local = false; for (const auto & addr : cluster->getShardsAddresses().front()) { /// skip first local - if (exclude_local_replica && addr.is_local && !first_local) - { - first_local = true; + if (exclude_local_replica && addr.is_local) continue; - } /// replace hostname with replica name if the hostname started with replica namespace, /// it makes description shorter and more readable @@ -409,7 +405,7 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( replicas.push_back(fmt::format("{}", addr.host_name)); } - auto description = fmt::format("Query: {} Replicas: ", formattedAST(query_ast)) + boost::algorithm::join(replicas, ", "); + auto description = fmt::format("Query: {} Replicas: {}", formattedAST(query_ast), fmt::join(replicas, ", ")); setStepDescription(std::move(description)); } @@ -489,13 +485,11 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder if (pools_to_use.empty()) return; - { - String pool_addresses; - for (const auto & pool : pools_to_use) - pool_addresses += pool->getAddress() + ";"; - - LOG_DEBUG(getLogger("ReadFromParallelRemoteReplicasStep"), "Addresses to use: {}", pool_addresses); - } + std::vector addresses; + addresses.reserve(pools_to_use.size()); + for (const auto & pool : pools_to_use) + addresses.emplace_back(pool->getAddress()); + LOG_DEBUG(getLogger("ReadFromParallelRemoteReplicasStep"), "Addresses to use: {}", fmt::join(addresses, ", ")); /// when using local plan for local replica, local replica has 0 number size_t offset = (exclude_local_replica ? 1 : 0); From 08dc1c8c37d287ddf3cf3a785b6c8334e773840a Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 16 Jul 2024 19:59:16 +0000 Subject: [PATCH 064/162] Fallback to local execution in case of cluster(shard) has only one node --- .../ClusterProxy/executeQuery.cpp | 39 ++++++++++++++++++- src/Interpreters/ClusterProxy/executeQuery.h | 2 + src/Planner/PlannerJoinTree.cpp | 2 +- 3 files changed, 41 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 4a637db3887..7207efd9ef6 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -488,12 +488,12 @@ void executeQueryWithParallelReplicas( shard_num = column->getUInt(0); } - const auto shard_count = not_optimized_cluster->getShardCount(); ClusterPtr new_cluster = not_optimized_cluster; /// if got valid shard_num from query initiator, then parallel replicas scope is the specified shard /// shards are numbered in order of appearance in the cluster config if (shard_num > 0) { + const auto shard_count = not_optimized_cluster->getShardCount(); if (shard_num > shard_count) throw Exception( ErrorCodes::LOGICAL_ERROR, @@ -702,6 +702,43 @@ void executeQueryWithParallelReplicasCustomKey( context, query_info.query, storage_id.getDatabaseName(), storage_id.getTableName(), /*table_function_ptr=*/nullptr); executeQueryWithParallelReplicasCustomKey(query_plan, storage_id, query_info, columns, snapshot, processed_stage, header, context); } + +bool canUseParallelReplicasOnInitiator(const ContextPtr & context) +{ + if (!context->canUseParallelReplicasOnInitiator()) + return false; + + auto cluster = context->getClusterForParallelReplicas(); + if (cluster->getShardCount() == 1) + return cluster->getShardsInfo()[0].getAllNodeCount() > 1; + + /// parallel replicas with distributed table + auto scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{}; + UInt64 shard_num = 0; /// shard_num is 1-based, so 0 - no shard specified + const auto it = scalars.find("_shard_num"); + if (it != scalars.end()) + { + const Block & block = it->second; + const auto & column = block.safeGetByPosition(0).column; + shard_num = column->getUInt(0); + } + if (shard_num > 0) + { + const auto shard_count = cluster->getShardCount(); + if (shard_num > shard_count) + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Shard number is greater than shard count: shard_num={} shard_count={} cluster={}", + shard_num, + shard_count, + cluster->getName()); + + return cluster->getShardsInfo().at(shard_num - 1).getAllNodeCount() > 1; + } + + return false; +} + } } diff --git a/src/Interpreters/ClusterProxy/executeQuery.h b/src/Interpreters/ClusterProxy/executeQuery.h index 2c63b7b89f0..2a21f3e8255 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.h +++ b/src/Interpreters/ClusterProxy/executeQuery.h @@ -58,6 +58,8 @@ using AdditionalShardFilterGenerator = std::function; AdditionalShardFilterGenerator getShardFilterGeneratorForCustomKey(const Cluster & cluster, ContextPtr context, const ColumnsDescription & columns); +bool canUseParallelReplicasOnInitiator(const ContextPtr & context); + /// Execute a distributed query, creating a query plan, from which the query pipeline can be built. /// `stream_factory` object encapsulates the logic of creating plans for a different type of query /// (currently SELECT, DESCRIBE). diff --git a/src/Planner/PlannerJoinTree.cpp b/src/Planner/PlannerJoinTree.cpp index 4c21f0d00cc..c1eb2b48045 100644 --- a/src/Planner/PlannerJoinTree.cpp +++ b/src/Planner/PlannerJoinTree.cpp @@ -920,7 +920,7 @@ JoinTreeQueryPlan buildQueryPlanForTableExpression(QueryTreeNodePtr table_expres query_plan = std::move(query_plan_parallel_replicas); } } - else if (query_context->canUseParallelReplicasOnInitiator()) + else if (ClusterProxy::canUseParallelReplicasOnInitiator(query_context)) { // (1) find read step QueryPlan::Node * node = query_plan.getRootNode(); From fc693cc982cce7b4aaf71933bd18aa751ed3b7a0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 16 Jul 2024 21:05:27 +0000 Subject: [PATCH 065/162] Fix 02982_parallel_replicas_unexpected_cluster --- src/Interpreters/ClusterProxy/executeQuery.cpp | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 7207efd9ef6..c0d156a8894 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -736,6 +736,12 @@ bool canUseParallelReplicasOnInitiator(const ContextPtr & context) return cluster->getShardsInfo().at(shard_num - 1).getAllNodeCount() > 1; } + if (cluster->getShardCount() > 1) + throw DB::Exception( + ErrorCodes::UNEXPECTED_CLUSTER, + "`cluster_for_parallel_replicas` setting refers to cluster with {} shards. Expected a cluster with one shard", + cluster->getShardCount()); + return false; } From a3d4fd9246b2e1953ed04d75840a9f3399d37ba4 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 17 Jul 2024 10:01:26 +0000 Subject: [PATCH 066/162] Fix 02950_parallel_replicas_used_count --- .../0_stateless/02950_parallel_replicas_used_count.reference | 4 ++-- .../0_stateless/02950_parallel_replicas_used_count.sql | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference b/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference index cc8284bfeac..c1265b7ca14 100644 --- a/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference +++ b/tests/queries/0_stateless/02950_parallel_replicas_used_count.reference @@ -5,10 +5,10 @@ 91 92 93 -3 +1 93 92 91 90 89 -3 +1 diff --git a/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql b/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql index 69f1dc47ded..4396ca60e0e 100644 --- a/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql +++ b/tests/queries/0_stateless/02950_parallel_replicas_used_count.sql @@ -21,12 +21,12 @@ SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHER SELECT k FROM test order by k limit 5 offset 89 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_2', merge_tree_min_rows_for_concurrent_read=1, max_threads=1; SYSTEM FLUSH LOGS; -SELECT ProfileEvents['ParallelReplicasUsedCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_2' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; +SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_2' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; -- In reverse order coordinator SELECT k FROM test order by k desc limit 5 offset 9906 SETTINGS optimize_read_in_order=1, log_comment='02950_parallel_replicas_used_replicas_count_3', merge_tree_min_rows_for_concurrent_read=1, max_threads=1; SYSTEM FLUSH LOGS; -SELECT ProfileEvents['ParallelReplicasUsedCount'] FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_3' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; +SELECT ProfileEvents['ParallelReplicasUsedCount'] > 0 FROM system.query_log WHERE type = 'QueryFinish' AND query_id IN (SELECT query_id FROM system.query_log WHERE current_database = currentDatabase() AND log_comment = '02950_parallel_replicas_used_replicas_count_3' AND type = 'QueryFinish' AND initial_query_id = query_id) SETTINGS allow_experimental_parallel_reading_from_replicas=0; DROP TABLE test; From baf3408b16b3634c59d6635afbb53b849cf55d0e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Wed, 17 Jul 2024 11:45:00 +0000 Subject: [PATCH 067/162] Fix: fallback to local plan w/o PR in case of empty table --- src/Interpreters/ClusterProxy/executeQuery.cpp | 8 +++++++- .../QueryPlan/ParallelReplicasLocalPlan.cpp | 11 ++++++----- src/Processors/QueryPlan/ParallelReplicasLocalPlan.h | 2 +- 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index c0d156a8894..a8cf3022a61 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -528,7 +528,7 @@ void executeQueryWithParallelReplicas( /// do not build local plan for distributed queries for now (address it later) if (settings.allow_experimental_analyzer && settings.parallel_replicas_local_plan && !shard_num) { - auto local_plan = createLocalPlanForParallelReplicas( + auto [local_plan, with_parallel_replicas] = createLocalPlanForParallelReplicas( query_ast, header, new_context, @@ -536,6 +536,12 @@ void executeQueryWithParallelReplicas( coordinator, std::move(analyzed_read_from_merge_tree)); + if (!with_parallel_replicas) + { + query_plan = std::move(*local_plan); + return; + } + auto read_from_remote = std::make_unique( query_ast, new_cluster, diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index d2e862a3416..e9d5ef90e30 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -21,9 +21,7 @@ namespace DB { -void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missing_objects); - -std::unique_ptr createLocalPlanForParallelReplicas( +std::pair, bool> createLocalPlanForParallelReplicas( const ASTPtr & query_ast, const Block & header, ContextPtr context, @@ -68,7 +66,9 @@ std::unique_ptr createLocalPlanForParallelReplicas( node = nullptr; } - chassert(reading); + if (!reading) + /// it can happened if merge tree table is empty, - it'll be replaced with ReadFromPreparedSource + return {std::move(query_plan), false}; ReadFromMergeTree::AnalysisResultPtr analyzed_result_ptr; if (analyzed_read_from_merge_tree.get()) @@ -89,7 +89,8 @@ std::unique_ptr createLocalPlanForParallelReplicas( node->step = std::move(read_from_merge_tree_parallel_replicas); addConvertingActions(*query_plan, header, /*has_missing_objects=*/false); - return query_plan; + + return {std::move(query_plan), true}; } } diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h index 123754458a1..2a49be6347a 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h @@ -8,7 +8,7 @@ namespace DB { -std::unique_ptr createLocalPlanForParallelReplicas( +std::pair, bool> createLocalPlanForParallelReplicas( const ASTPtr & query_ast, const Block & header, ContextPtr context, From 97f4ec2adbf614842e9b16badb69a8d5c642abe0 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 17 Jul 2024 16:59:35 +0200 Subject: [PATCH 068/162] Read cgroup memory usage in async metrics thread --- programs/keeper/Keeper.cpp | 26 ++-- programs/server/Server.cpp | 34 ++--- src/Common/AsynchronousMetrics.cpp | 13 +- src/Common/AsynchronousMetrics.h | 5 +- src/Common/CgroupsMemoryUsageObserver.cpp | 127 ++++-------------- src/Common/CgroupsMemoryUsageObserver.h | 19 +-- src/Common/MemoryTracker.cpp | 38 ++---- src/Common/MemoryTracker.h | 4 +- .../KeeperAsynchronousMetrics.cpp | 2 +- src/Interpreters/Context.cpp | 13 ++ src/Interpreters/Context.h | 4 + .../ServerAsynchronousMetrics.cpp | 2 +- 12 files changed, 112 insertions(+), 175 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 44c2daa33ad..3f6020ad48c 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -399,6 +399,18 @@ try registerDisks(/*global_skip_access_check=*/false); + auto cgroups_memory_observer_wait_time = config().getUInt64("keeper_server.cgroups_memory_observer_wait_time", 15); + try + { + auto cgroups_reader = createCgroupsReader(); + global_context->setCgroupsReader(createCgroupsReader()); + } + catch (...) + { + if (cgroups_memory_observer_wait_time != 0) + tryLogCurrentException(log, "Failed to create cgroups reader"); + } + /// This object will periodically calculate some metrics. KeeperAsynchronousMetrics async_metrics( global_context, @@ -622,21 +634,19 @@ try main_config_reloader->start(); std::optional cgroups_memory_usage_observer; - try + if (cgroups_memory_observer_wait_time != 0) { - auto wait_time = config().getUInt64("keeper_server.cgroups_memory_observer_wait_time", 15); - if (wait_time != 0) + auto cgroups_reader = global_context->getCgroupsReader(); + if (cgroups_reader) { - cgroups_memory_usage_observer.emplace(std::chrono::seconds(wait_time)); + cgroups_memory_usage_observer.emplace(std::chrono::seconds(cgroups_memory_observer_wait_time), global_context->getCgroupsReader()); /// Not calling cgroups_memory_usage_observer->setLimits() here (as for the normal ClickHouse server) because Keeper controls /// its memory usage by other means (via setting 'max_memory_usage_soft_limit'). cgroups_memory_usage_observer->setOnMemoryAmountAvailableChangedFn([&]() { main_config_reloader->reload(); }); cgroups_memory_usage_observer->startThread(); } - } - catch (Exception &) - { - tryLogCurrentException(log, "Disabling cgroup memory observer because of an error during initialization"); + else + LOG_ERROR(log, "Disabling cgroup memory observer because of an error during initialization of cgroups reader"); } diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 053ddaf8d8b..c52b1e037ec 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -897,6 +897,17 @@ try LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); }); + try + { + auto cgroups_reader = createCgroupsReader(); + global_context->setCgroupsReader(createCgroupsReader()); + } + catch (...) + { + if (server_settings.cgroups_memory_usage_observer_wait_time != 0) + tryLogCurrentException(log, "Failed to create cgroups reader"); + } + /// This object will periodically calculate some metrics. ServerAsynchronousMetrics async_metrics( global_context, @@ -1456,15 +1467,13 @@ try } std::optional cgroups_memory_usage_observer; - try + if (auto wait_time = server_settings.cgroups_memory_usage_observer_wait_time; wait_time != 0) { - auto wait_time = server_settings.cgroups_memory_usage_observer_wait_time; - if (wait_time != 0) - cgroups_memory_usage_observer.emplace(std::chrono::seconds(wait_time)); - } - catch (Exception &) - { - tryLogCurrentException(log, "Disabling cgroup memory observer because of an error during initialization"); + auto cgroups_reader = global_context->getCgroupsReader(); + if (cgroups_reader) + cgroups_memory_usage_observer.emplace(std::chrono::seconds(wait_time), std::move(cgroups_reader)); + else + LOG_ERROR(log, "Disabling cgroup memory observer because of an error during initialization of cgroups reader"); } std::string cert_path = config().getString("openSSL.server.certificateFile", ""); @@ -1532,15 +1541,6 @@ try total_memory_tracker.setDescription("(total)"); total_memory_tracker.setMetric(CurrentMetrics::MemoryTracking); - if (cgroups_memory_usage_observer) - { - double hard_limit_ratio = new_server_settings.cgroup_memory_watcher_hard_limit_ratio; - double soft_limit_ratio = new_server_settings.cgroup_memory_watcher_soft_limit_ratio; - cgroups_memory_usage_observer->setMemoryUsageLimits( - static_cast(max_server_memory_usage * hard_limit_ratio), - static_cast(max_server_memory_usage * soft_limit_ratio)); - } - size_t merges_mutations_memory_usage_soft_limit = new_server_settings.merges_mutations_memory_usage_soft_limit; size_t default_merges_mutations_server_memory_usage = static_cast(current_physical_server_memory * new_server_settings.merges_mutations_memory_usage_to_ram_ratio); diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 6309f6079f6..0953ad88697 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -57,10 +57,12 @@ static std::unique_ptr openFileIfExists(const std::stri AsynchronousMetrics::AsynchronousMetrics( unsigned update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_) + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + std::shared_ptr cgroups_reader_) : update_period(update_period_seconds) , log(getLogger("AsynchronousMetrics")) , protocol_server_metrics_func(protocol_server_metrics_func_) + , cgroups_reader(std::move(cgroups_reader_)) { #if defined(OS_LINUX) openFileIfExists("/proc/meminfo", meminfo); @@ -669,6 +671,13 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) free_memory_in_allocator_arenas = je_malloc_pdirty * getPageSize(); #endif + if (cgroups_reader != nullptr) + { + rss = cgroups_reader->readMemoryUsage(); + new_values["CgroupsMemoryUsage"] = { rss, + "The amount of physical memory used by the server process, reported by cgroups." }; + } + Int64 difference = rss - amount; /// Log only if difference is high. This is for convenience. The threshold is arbitrary. @@ -681,7 +690,7 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) ReadableSize(rss), ReadableSize(difference)); - MemoryTracker::setRSS(rss, free_memory_in_allocator_arenas); + MemoryTracker::setRSS(rss, /*has_free_memory_in_allocator_arenas_=*/free_memory_in_allocator_arenas > 0); } } diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index 10a972d2458..0b110f41fc3 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -68,7 +69,8 @@ public: AsynchronousMetrics( unsigned update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_); + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + std::shared_ptr cgroups_reader_); virtual ~AsynchronousMetrics(); @@ -91,6 +93,7 @@ private: virtual void logImpl(AsynchronousMetricValues &) {} ProtocolServerMetricsFunc protocol_server_metrics_func; + std::shared_ptr cgroups_reader; std::unique_ptr thread; diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index 02bde0d80b7..b12845df098 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -17,13 +17,6 @@ #include #include -#include "config.h" -#if USE_JEMALLOC -# include -#define STRINGIFY_HELPER(x) #x -#define STRINGIFY(x) STRINGIFY_HELPER(x) -#endif - using namespace DB; namespace fs = std::filesystem; @@ -155,15 +148,21 @@ std::optional getCgroupsV1Path() return {default_cgroups_mount / "memory"}; } -std::pair getCgroupsPath() +enum class CgroupsVersion : uint8_t +{ + V1, + V2 +}; + +std::pair getCgroupsPath() { auto v2_path = getCgroupsV2Path(); if (v2_path.has_value()) - return {*v2_path, CgroupsMemoryUsageObserver::CgroupsVersion::V2}; + return {*v2_path, CgroupsVersion::V2}; auto v1_path = getCgroupsV1Path(); if (v1_path.has_value()) - return {*v1_path, CgroupsMemoryUsageObserver::CgroupsVersion::V1}; + return {*v1_path, CgroupsVersion::V1}; throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot find cgroups v1 or v2 current memory file"); } @@ -173,22 +172,29 @@ std::pair getCgroupsPat namespace DB { -CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_) - : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_) +std::shared_ptr createCgroupsReader() { const auto [cgroup_path, version] = getCgroupsPath(); + LOG_INFO( + getLogger("CgroupsReader"), + "Will create cgroup reader from '{}' (cgroups version: {})", + cgroup_path, + (version == CgroupsVersion::V1) ? "v1" : "v2"); if (version == CgroupsVersion::V2) - cgroup_reader = std::make_unique(cgroup_path); + return std::make_shared(cgroup_path); else - cgroup_reader = std::make_unique(cgroup_path); + { + chassert(version == CgroupsVersion::V1); + return std::make_shared(cgroup_path); + } - LOG_INFO( - log, - "Will read the current memory usage from '{}' (cgroups version: {}), wait time is {} sec", - cgroup_path, - (version == CgroupsVersion::V1) ? "v1" : "v2", - wait_time.count()); +} + +CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_, std::shared_ptr cgroups_reader_) + : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_), cgroups_reader(std::move(cgroups_reader_)) +{ + cgroups_reader = createCgroupsReader(); } CgroupsMemoryUsageObserver::~CgroupsMemoryUsageObserver() @@ -196,58 +202,6 @@ CgroupsMemoryUsageObserver::~CgroupsMemoryUsageObserver() stopThread(); } -void CgroupsMemoryUsageObserver::setMemoryUsageLimits(uint64_t hard_limit_, uint64_t soft_limit_) -{ - std::lock_guard limit_lock(limit_mutex); - - if (hard_limit_ == hard_limit && soft_limit_ == soft_limit) - return; - - hard_limit = hard_limit_; - soft_limit = soft_limit_; - - on_hard_limit = [this, hard_limit_](bool up) - { - if (up) - { - LOG_WARNING(log, "Exceeded hard memory limit ({})", ReadableSize(hard_limit_)); - - /// Update current usage in memory tracker. Also reset free_memory_in_allocator_arenas to zero though we don't know if they are - /// really zero. Trying to avoid OOM ... - MemoryTracker::setRSS(hard_limit_, 0); - } - else - { - LOG_INFO(log, "Dropped below hard memory limit ({})", ReadableSize(hard_limit_)); - } - }; - - on_soft_limit = [this, soft_limit_](bool up) - { - if (up) - { - LOG_WARNING(log, "Exceeded soft memory limit ({})", ReadableSize(soft_limit_)); - -# if USE_JEMALLOC - LOG_INFO(log, "Purging jemalloc arenas"); - mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); -# endif - /// Reset current usage in memory tracker. Expect zero for free_memory_in_allocator_arenas as we just purged them. - uint64_t memory_usage = cgroup_reader->readMemoryUsage(); - LOG_TRACE(log, "Read current memory usage {} bytes ({}) from cgroups", memory_usage, ReadableSize(memory_usage)); - MemoryTracker::setRSS(memory_usage, 0); - - LOG_INFO(log, "Purged jemalloc arenas. Current memory usage is {}", ReadableSize(memory_usage)); - } - else - { - LOG_INFO(log, "Dropped below soft memory limit ({})", ReadableSize(soft_limit_)); - } - }; - - LOG_INFO(log, "Set new limits, soft limit: {}, hard limit: {}", ReadableSize(soft_limit_), ReadableSize(hard_limit_)); -} - void CgroupsMemoryUsageObserver::setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_) { std::lock_guard memory_amount_available_changed_lock(memory_amount_available_changed_mutex); @@ -301,35 +255,6 @@ void CgroupsMemoryUsageObserver::runThread() std::lock_guard memory_amount_available_changed_lock(memory_amount_available_changed_mutex); on_memory_amount_available_changed(); } - - std::lock_guard limit_lock(limit_mutex); - if (soft_limit > 0 && hard_limit > 0) - { - uint64_t memory_usage = cgroup_reader->readMemoryUsage(); - LOG_TRACE(log, "Read current memory usage {} bytes ({}) from cgroups", memory_usage, ReadableSize(memory_usage)); - if (memory_usage > hard_limit) - { - if (last_memory_usage <= hard_limit) - on_hard_limit(true); - } - else - { - if (last_memory_usage > hard_limit) - on_hard_limit(false); - } - - if (memory_usage > soft_limit) - { - if (last_memory_usage <= soft_limit) - on_soft_limit(true); - } - else - { - if (last_memory_usage > soft_limit) - on_soft_limit(false); - } - last_memory_usage = memory_usage; - } } catch (...) { diff --git a/src/Common/CgroupsMemoryUsageObserver.h b/src/Common/CgroupsMemoryUsageObserver.h index b848a2bff3c..078307a6fa0 100644 --- a/src/Common/CgroupsMemoryUsageObserver.h +++ b/src/Common/CgroupsMemoryUsageObserver.h @@ -16,6 +16,8 @@ struct ICgroupsReader virtual uint64_t readMemoryUsage() = 0; }; +std::shared_ptr createCgroupsReader(); + /// Does two things: /// 1. Periodically reads the memory usage of the process from Linux cgroups. /// You can specify soft or hard memory limits: @@ -35,19 +37,11 @@ struct ICgroupsReader class CgroupsMemoryUsageObserver { public: - using OnMemoryLimitFn = std::function; using OnMemoryAmountAvailableChangedFn = std::function; - enum class CgroupsVersion : uint8_t - { - V1, - V2 - }; - - explicit CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_); + explicit CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_, std::shared_ptr cgroups_reader_); ~CgroupsMemoryUsageObserver(); - void setMemoryUsageLimits(uint64_t hard_limit_, uint64_t soft_limit_); void setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_); void startThread(); @@ -58,22 +52,17 @@ private: const std::chrono::seconds wait_time; std::mutex limit_mutex; - size_t hard_limit TSA_GUARDED_BY(limit_mutex) = 0; - size_t soft_limit TSA_GUARDED_BY(limit_mutex) = 0; - OnMemoryLimitFn on_hard_limit TSA_GUARDED_BY(limit_mutex); - OnMemoryLimitFn on_soft_limit TSA_GUARDED_BY(limit_mutex); std::mutex memory_amount_available_changed_mutex; OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed TSA_GUARDED_BY(memory_amount_available_changed_mutex); - uint64_t last_memory_usage = 0; /// how much memory does the process use uint64_t last_available_memory_amount; /// how much memory can the process use void stopThread(); void runThread(); - std::unique_ptr cgroup_reader; + std::shared_ptr cgroups_reader; std::mutex thread_mutex; std::condition_variable cond; diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 28cfa98666a..eaf34d87ec5 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -26,7 +26,6 @@ #endif #include -#include #include #include #include @@ -123,7 +122,7 @@ static constexpr size_t log_peak_memory_usage_every = 1ULL << 30; MemoryTracker total_memory_tracker(nullptr, VariableContext::Global); MemoryTracker background_memory_tracker(&total_memory_tracker, VariableContext::User, false); -std::atomic MemoryTracker::free_memory_in_allocator_arenas; +std::atomic MemoryTracker::has_free_memory_in_allocator_arenas; MemoryTracker::MemoryTracker(VariableContext level_) : parent(&total_memory_tracker), level(level_) {} MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_) : parent(parent_), level(level_) {} @@ -204,7 +203,7 @@ void MemoryTracker::debugLogBigAllocationWithoutCheck(Int64 size [[maybe_unused] LOG_TEST(getLogger("MemoryTracker"), "Too big allocation ({} bytes) without checking memory limits, " "it may lead to OOM. Stack trace: {}", size, StackTrace().toString()); #else - return; /// Avoid trash logging in release builds + /// Avoid trash logging in release builds #endif } @@ -294,33 +293,18 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed } } - Int64 limit_to_check = current_hard_limit; - #if USE_JEMALLOC - if (level == VariableContext::Global && allow_use_jemalloc_memory.load(std::memory_order_relaxed)) + if (level == VariableContext::Global && will_be > soft_limit.load(std::memory_order_relaxed) + && has_free_memory_in_allocator_arenas.exchange(false)) { - /// Jemalloc arenas may keep some extra memory. - /// This memory was substucted from RSS to decrease memory drift. - /// In case memory is close to limit, try to pugre the arenas. - /// This is needed to avoid OOM, because some allocations are directly done with mmap. - Int64 current_free_memory_in_allocator_arenas = free_memory_in_allocator_arenas.load(std::memory_order_relaxed); - - if (current_free_memory_in_allocator_arenas > 0 && current_hard_limit && current_free_memory_in_allocator_arenas + will_be > current_hard_limit) - { - if (free_memory_in_allocator_arenas.exchange(-current_free_memory_in_allocator_arenas) > 0) - { - Stopwatch watch; - mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); - ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); - ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, watch.elapsedMicroseconds()); - } - } - - limit_to_check += abs(current_free_memory_in_allocator_arenas); + Stopwatch watch; + mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); + ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); + ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, watch.elapsedMicroseconds()); } #endif - if (unlikely(current_hard_limit && will_be > limit_to_check)) + if (unlikely(current_hard_limit && will_be > current_hard_limit)) { if (memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) { @@ -526,11 +510,11 @@ void MemoryTracker::reset() } -void MemoryTracker::setRSS(Int64 rss_, Int64 free_memory_in_allocator_arenas_) +void MemoryTracker::setRSS(Int64 rss_, bool has_free_memory_in_allocator_arenas_) { Int64 new_amount = rss_; total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); - free_memory_in_allocator_arenas.store(free_memory_in_allocator_arenas_, std::memory_order_relaxed); + has_free_memory_in_allocator_arenas.store(has_free_memory_in_allocator_arenas_, std::memory_order_relaxed); auto metric_loaded = total_memory_tracker.metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index fd32b631774..48d02fd1fc6 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -59,7 +59,7 @@ private: std::atomic profiler_limit {0}; std::atomic_bool allow_use_jemalloc_memory {true}; - static std::atomic free_memory_in_allocator_arenas; + static std::atomic has_free_memory_in_allocator_arenas; Int64 profiler_step = 0; @@ -252,7 +252,7 @@ public: /// Reset current counter to an RSS value. /// Jemalloc may have pre-allocated arenas, they are accounted in RSS. /// We can free this arenas in case of exception to avoid OOM. - static void setRSS(Int64 rss_, Int64 free_memory_in_allocator_arenas_); + static void setRSS(Int64 rss_, bool has_free_memory_in_allocator_arenas_); /// Prints info about peak memory consumption into log. void logPeakMemoryUsage(); diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 86166ffe31b..3e404b7152b 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -115,7 +115,7 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM KeeperAsynchronousMetrics::KeeperAsynchronousMetrics( ContextPtr context_, unsigned update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) - : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), context(std::move(context_)) + : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, context_->getCgroupsReader()), context(std::move(context_)) { } diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 2602afd8b78..771b8e9e558 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -5,6 +5,7 @@ #include #include #include +#include #include #include #include @@ -405,6 +406,8 @@ struct ContextSharedPart : boost::noncopyable std::unique_ptr cluster_discovery TSA_GUARDED_BY(clusters_mutex); size_t clusters_version TSA_GUARDED_BY(clusters_mutex) = 0; + std::shared_ptr cgroups_reader; + /// No lock required for async_insert_queue modified only during initialization std::shared_ptr async_insert_queue; @@ -5635,6 +5638,16 @@ const ServerSettings & Context::getServerSettings() const return shared->server_settings; } +void Context::setCgroupsReader(std::shared_ptr cgroups_reader_) +{ + shared->cgroups_reader = std::move(cgroups_reader_); +} + +std::shared_ptr Context::getCgroupsReader() const +{ + return shared->cgroups_reader; +} + uint64_t HTTPContext::getMaxHstsAge() const { return context->getSettingsRef().hsts_max_age; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 284cac50769..f183a72e8e2 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -150,6 +150,7 @@ class ServerType; template class MergeTreeBackgroundExecutor; class AsyncLoader; +struct ICgroupsReader; struct TemporaryTableHolder; using TemporaryTablesMapping = std::map>; @@ -1344,6 +1345,9 @@ public: const ServerSettings & getServerSettings() const; + void setCgroupsReader(std::shared_ptr cgroups_reader_); + std::shared_ptr getCgroupsReader() const; + private: std::shared_ptr getSettingsConstraintsAndCurrentProfilesWithLock() const; diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 872a9f864df..6ee0168bede 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -57,7 +57,7 @@ ServerAsynchronousMetrics::ServerAsynchronousMetrics( unsigned heavy_metrics_update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) : WithContext(global_context_) - , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_) + , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, getContext()->getCgroupsReader()) , heavy_metric_update_period(heavy_metrics_update_period_seconds) { /// sanity check From 21009577d867d493a6ceda48987c060498774f94 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 18 Jul 2024 09:01:08 +0200 Subject: [PATCH 069/162] Dedicated memory background thread --- programs/server/Server.cpp | 39 ++++++++++++++++++++++++--- src/CMakeLists.txt | 2 +- src/Common/AsynchronousMetrics.cpp | 43 ------------------------------ src/Common/Jemalloc.cpp | 16 ----------- src/Common/Jemalloc.h | 29 ++++++++++++++++++++ src/Common/MemoryTracker.cpp | 33 +++++++++-------------- src/Common/MemoryTracker.h | 1 - 7 files changed, 78 insertions(+), 85 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index c52b1e037ec..ca46338d1c1 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -447,9 +447,12 @@ void checkForUsersNotInMainConfig( } } +namespace +{ + /// Unused in other builds #if defined(OS_LINUX) -static String readLine(const String & path) +String readLine(const String & path) { ReadBufferFromFile in(path); String contents; @@ -457,7 +460,7 @@ static String readLine(const String & path) return contents; } -static int readNumber(const String & path) +int readNumber(const String & path) { ReadBufferFromFile in(path); int result; @@ -467,7 +470,7 @@ static int readNumber(const String & path) #endif -static void sanityChecks(Server & server) +void sanityChecks(Server & server) { std::string data_path = getCanonicalPath(server.config().getString("path", DBMS_DEFAULT_PATH)); std::string logs_path = server.config().getString("logger.log", ""); @@ -588,6 +591,31 @@ static void sanityChecks(Server & server) } } +[[noreturn]] void backgroundMemoryThread() +{ + std::mutex mutex; + std::condition_variable cv; + + std::unique_lock lock(mutex); + while (true) + { + cv.wait_for(lock, std::chrono::microseconds(200)); + uint64_t epoch = 0; + mallctl("epoch", nullptr, nullptr, &epoch, sizeof(epoch)); + auto maybe_resident = getJemallocValue("stats.resident"); + if (!maybe_resident.has_value()) + continue; + + Int64 resident = *maybe_resident; + //LOG_INFO(getLogger("JEmalloc"), "Resident {}", ReadableSize(resident)); + MemoryTracker::setRSS(resident, false); + if (resident > total_memory_tracker.getHardLimit()) + purgeJemallocArenas(); + } +} + +} + void loadStartupScripts(const Poco::Util::AbstractConfiguration & config, ContextMutablePtr context, Poco::Logger * log) { try @@ -877,6 +905,11 @@ try total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); } + ThreadFromGlobalPool background_memory_thread([] + { + backgroundMemoryThread(); + }); + Poco::ThreadPool server_pool( /* minCapacity */3, /* maxCapacity */server_settings.max_connections, diff --git a/src/CMakeLists.txt b/src/CMakeLists.txt index d985595154c..bfa41eacea1 100644 --- a/src/CMakeLists.txt +++ b/src/CMakeLists.txt @@ -174,7 +174,7 @@ add_library (clickhouse_new_delete STATIC Common/new_delete.cpp) target_link_libraries (clickhouse_new_delete PRIVATE clickhouse_common_io) if (TARGET ch_contrib::jemalloc) target_link_libraries (clickhouse_new_delete PRIVATE ch_contrib::jemalloc) - target_link_libraries (clickhouse_common_io PRIVATE ch_contrib::jemalloc) + target_link_libraries (clickhouse_common_io PUBLIC ch_contrib::jemalloc) target_link_libraries (clickhouse_storages_system PRIVATE ch_contrib::jemalloc) endif() diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index 0953ad88697..b3e53c29d4a 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -649,49 +649,6 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) "The amount of virtual memory mapped for the use of stack and for the allocated memory, in bytes." " It is unspecified whether it includes the per-thread stacks and most of the allocated memory, that is allocated with the 'mmap' system call." " This metric exists only for completeness reasons. I recommend to use the `MemoryResident` metric for monitoring."}; - - /// We must update the value of total_memory_tracker periodically. - /// Otherwise it might be calculated incorrectly - it can include a "drift" of memory amount. - /// See https://github.com/ClickHouse/ClickHouse/issues/10293 - { - Int64 amount = total_memory_tracker.get(); - Int64 peak = total_memory_tracker.getPeak(); - Int64 rss = data.resident; - Int64 free_memory_in_allocator_arenas = 0; - -#if USE_JEMALLOC - /// According to jemalloc man, pdirty is: - /// - /// Number of pages within unused extents that are potentially - /// dirty, and for which madvise() or similar has not been called. - /// - /// So they will be subtracted from RSS to make accounting more - /// accurate, since those pages are not really RSS but a memory - /// that can be used at anytime via jemalloc. - free_memory_in_allocator_arenas = je_malloc_pdirty * getPageSize(); -#endif - - if (cgroups_reader != nullptr) - { - rss = cgroups_reader->readMemoryUsage(); - new_values["CgroupsMemoryUsage"] = { rss, - "The amount of physical memory used by the server process, reported by cgroups." }; - } - - Int64 difference = rss - amount; - - /// Log only if difference is high. This is for convenience. The threshold is arbitrary. - if (difference >= 1048576 || difference <= -1048576) - LOG_TRACE(log, - "MemoryTracking: was {}, peak {}, free memory in arenas {}, will set to {} (RSS), difference: {}", - ReadableSize(amount), - ReadableSize(peak), - ReadableSize(free_memory_in_allocator_arenas), - ReadableSize(rss), - ReadableSize(difference)); - - MemoryTracker::setRSS(rss, /*has_free_memory_in_allocator_arenas_=*/free_memory_in_allocator_arenas > 0); - } } { diff --git a/src/Common/Jemalloc.cpp b/src/Common/Jemalloc.cpp index d7cc246db6a..d8ff9268cca 100644 --- a/src/Common/Jemalloc.cpp +++ b/src/Common/Jemalloc.cpp @@ -5,7 +5,6 @@ #include #include #include -#include #define STRINGIFY_HELPER(x) #x #define STRINGIFY(x) STRINGIFY_HELPER(x) @@ -26,7 +25,6 @@ namespace ErrorCodes void purgeJemallocArenas() { - LOG_TRACE(getLogger("SystemJemalloc"), "Purging unused memory"); Stopwatch watch; mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); @@ -46,20 +44,6 @@ void checkJemallocProfilingEnabled() "set: MALLOC_CONF=background_thread:true,prof:true"); } -template -void setJemallocValue(const char * name, T value) -{ - T old_value; - size_t old_value_size = sizeof(T); - if (mallctl(name, &old_value, &old_value_size, reinterpret_cast(&value), sizeof(T))) - { - LOG_WARNING(getLogger("Jemalloc"), "mallctl for {} failed", name); - return; - } - - LOG_INFO(getLogger("Jemalloc"), "Value for {} set to {} (from {})", name, value, old_value); -} - void setJemallocProfileActive(bool value) { checkJemallocProfilingEnabled(); diff --git a/src/Common/Jemalloc.h b/src/Common/Jemalloc.h index 499a906fd3d..0c533711f78 100644 --- a/src/Common/Jemalloc.h +++ b/src/Common/Jemalloc.h @@ -5,6 +5,8 @@ #if USE_JEMALLOC #include +#include +#include namespace DB { @@ -21,6 +23,33 @@ void setJemallocBackgroundThreads(bool enabled); void setJemallocMaxBackgroundThreads(size_t max_threads); +template +void setJemallocValue(const char * name, T value) +{ + T old_value; + size_t old_value_size = sizeof(T); + if (mallctl(name, &old_value, &old_value_size, reinterpret_cast(&value), sizeof(T))) + { + LOG_WARNING(getLogger("Jemalloc"), "mallctl for {} failed", name); + return; + } + + LOG_INFO(getLogger("Jemalloc"), "Value for {} set to {} (from {})", name, value, old_value); +} + +template +std::optional getJemallocValue(const char * name) +{ + T value; + size_t value_size = sizeof(T); + if (mallctl(name, &value, &value_size, nullptr, 0)) + { + LOG_WARNING(getLogger("Jemalloc"), "mallctl for {} failed", name); + return std::nullopt; + } + return value; +} + } #endif diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index eaf34d87ec5..a541eeeb25f 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -20,9 +20,6 @@ #if USE_JEMALLOC # include -#define STRINGIFY_HELPER(x) #x -#define STRINGIFY(x) STRINGIFY_HELPER(x) - #endif #include @@ -126,11 +123,11 @@ std::atomic MemoryTracker::has_free_memory_in_allocator_arenas; MemoryTracker::MemoryTracker(VariableContext level_) : parent(&total_memory_tracker), level(level_) {} MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_) : parent(parent_), level(level_) {} + MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_, bool log_peak_memory_usage_in_destructor_) - : parent(parent_) - , log_peak_memory_usage_in_destructor(log_peak_memory_usage_in_destructor_) - , level(level_) -{} + : parent(parent_), log_peak_memory_usage_in_destructor(log_peak_memory_usage_in_destructor_), level(level_) +{ +} MemoryTracker::~MemoryTracker() { @@ -200,8 +197,12 @@ void MemoryTracker::debugLogBigAllocationWithoutCheck(Int64 size [[maybe_unused] return; MemoryTrackerBlockerInThread blocker(VariableContext::Global); - LOG_TEST(getLogger("MemoryTracker"), "Too big allocation ({} bytes) without checking memory limits, " - "it may lead to OOM. Stack trace: {}", size, StackTrace().toString()); + LOG_TEST( + getLogger("MemoryTracker"), + "Too big allocation ({} bytes) without checking memory limits, " + "it may lead to OOM. Stack trace: {}", + size, + StackTrace().toString()); #else /// Avoid trash logging in release builds #endif @@ -293,17 +294,6 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed } } -#if USE_JEMALLOC - if (level == VariableContext::Global && will_be > soft_limit.load(std::memory_order_relaxed) - && has_free_memory_in_allocator_arenas.exchange(false)) - { - Stopwatch watch; - mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); - ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); - ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, watch.elapsedMicroseconds()); - } -#endif - if (unlikely(current_hard_limit && will_be > current_hard_limit)) { if (memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) @@ -513,7 +503,8 @@ void MemoryTracker::reset() void MemoryTracker::setRSS(Int64 rss_, bool has_free_memory_in_allocator_arenas_) { Int64 new_amount = rss_; - total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); + if (rss_) + total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); has_free_memory_in_allocator_arenas.store(has_free_memory_in_allocator_arenas_, std::memory_order_relaxed); auto metric_loaded = total_memory_tracker.metric.load(std::memory_order_relaxed); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 48d02fd1fc6..257ed7d0629 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -2,7 +2,6 @@ #include #include -#include #include #include #include From 9a43183eb39dc056186432e6478e050db5045ecf Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 18 Jul 2024 10:31:24 +0200 Subject: [PATCH 070/162] Finish background memory thread --- programs/keeper/Keeper.cpp | 30 ++++------ programs/server/Server.cpp | 59 ++++--------------- src/Common/AsynchronousMetrics.cpp | 19 ++---- src/Common/AsynchronousMetrics.h | 3 +- src/Common/CgroupsMemoryUsageObserver.cpp | 8 +-- src/Common/CgroupsMemoryUsageObserver.h | 24 +++----- src/Common/Jemalloc.h | 42 +++++++++---- src/Common/MemoryTracker.cpp | 10 +--- src/Common/MemoryTracker.h | 13 +--- src/Common/MemoryWorker.cpp | 49 +++++++++++++++ src/Common/MemoryWorker.h | 34 +++++++++++ .../KeeperAsynchronousMetrics.cpp | 2 +- src/Core/ServerSettings.h | 1 + src/Interpreters/Context.cpp | 12 ---- src/Interpreters/Context.h | 3 - .../ServerAsynchronousMetrics.cpp | 2 +- .../System/StorageSystemServerSettings.cpp | 1 - tests/integration/test_memory_limit/test.py | 1 - 18 files changed, 158 insertions(+), 155 deletions(-) create mode 100644 src/Common/MemoryWorker.cpp create mode 100644 src/Common/MemoryWorker.h diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index 3f6020ad48c..87f126a0046 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -11,6 +11,7 @@ #include #include #include +#include #include #include #include @@ -371,6 +372,8 @@ try LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); }); + MemoryWorker memory_worker(config().getUInt64("memory_worker_period_ms", 100)); + static ServerErrorHandler error_handler; Poco::ErrorHandler::set(&error_handler); @@ -399,18 +402,6 @@ try registerDisks(/*global_skip_access_check=*/false); - auto cgroups_memory_observer_wait_time = config().getUInt64("keeper_server.cgroups_memory_observer_wait_time", 15); - try - { - auto cgroups_reader = createCgroupsReader(); - global_context->setCgroupsReader(createCgroupsReader()); - } - catch (...) - { - if (cgroups_memory_observer_wait_time != 0) - tryLogCurrentException(log, "Failed to create cgroups reader"); - } - /// This object will periodically calculate some metrics. KeeperAsynchronousMetrics async_metrics( global_context, @@ -634,21 +625,22 @@ try main_config_reloader->start(); std::optional cgroups_memory_usage_observer; - if (cgroups_memory_observer_wait_time != 0) + try { - auto cgroups_reader = global_context->getCgroupsReader(); - if (cgroups_reader) + auto wait_time = config().getUInt64("keeper_server.cgroups_memory_observer_wait_time", 15); + if (wait_time != 0) { - cgroups_memory_usage_observer.emplace(std::chrono::seconds(cgroups_memory_observer_wait_time), global_context->getCgroupsReader()); + cgroups_memory_usage_observer.emplace(std::chrono::seconds(wait_time)); /// Not calling cgroups_memory_usage_observer->setLimits() here (as for the normal ClickHouse server) because Keeper controls /// its memory usage by other means (via setting 'max_memory_usage_soft_limit'). cgroups_memory_usage_observer->setOnMemoryAmountAvailableChangedFn([&]() { main_config_reloader->reload(); }); cgroups_memory_usage_observer->startThread(); } - else - LOG_ERROR(log, "Disabling cgroup memory observer because of an error during initialization of cgroups reader"); } - + catch (Exception &) + { + tryLogCurrentException(log, "Disabling cgroup memory observer because of an error during initialization"); + } LOG_INFO(log, "Ready for connections."); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index ca46338d1c1..ae445477c3a 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include @@ -24,6 +23,7 @@ #include #include #include +#include #include #include #include @@ -110,6 +110,8 @@ #include #include +#include + #include "config.h" #include @@ -591,29 +593,6 @@ void sanityChecks(Server & server) } } -[[noreturn]] void backgroundMemoryThread() -{ - std::mutex mutex; - std::condition_variable cv; - - std::unique_lock lock(mutex); - while (true) - { - cv.wait_for(lock, std::chrono::microseconds(200)); - uint64_t epoch = 0; - mallctl("epoch", nullptr, nullptr, &epoch, sizeof(epoch)); - auto maybe_resident = getJemallocValue("stats.resident"); - if (!maybe_resident.has_value()) - continue; - - Int64 resident = *maybe_resident; - //LOG_INFO(getLogger("JEmalloc"), "Resident {}", ReadableSize(resident)); - MemoryTracker::setRSS(resident, false); - if (resident > total_memory_tracker.getHardLimit()) - purgeJemallocArenas(); - } -} - } void loadStartupScripts(const Poco::Util::AbstractConfiguration & config, ContextMutablePtr context, Poco::Logger * log) @@ -905,11 +884,6 @@ try total_memory_tracker.setSampleMaxAllocationSize(server_settings.total_memory_profiler_sample_max_allocation_size); } - ThreadFromGlobalPool background_memory_thread([] - { - backgroundMemoryThread(); - }); - Poco::ThreadPool server_pool( /* minCapacity */3, /* maxCapacity */server_settings.max_connections, @@ -930,16 +904,7 @@ try LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); }); - try - { - auto cgroups_reader = createCgroupsReader(); - global_context->setCgroupsReader(createCgroupsReader()); - } - catch (...) - { - if (server_settings.cgroups_memory_usage_observer_wait_time != 0) - tryLogCurrentException(log, "Failed to create cgroups reader"); - } + MemoryWorker memory_worker(global_context->getServerSettings().memory_worker_period_ms); /// This object will periodically calculate some metrics. ServerAsynchronousMetrics async_metrics( @@ -1500,13 +1465,15 @@ try } std::optional cgroups_memory_usage_observer; - if (auto wait_time = server_settings.cgroups_memory_usage_observer_wait_time; wait_time != 0) + try { - auto cgroups_reader = global_context->getCgroupsReader(); - if (cgroups_reader) - cgroups_memory_usage_observer.emplace(std::chrono::seconds(wait_time), std::move(cgroups_reader)); - else - LOG_ERROR(log, "Disabling cgroup memory observer because of an error during initialization of cgroups reader"); + auto wait_time = server_settings.cgroups_memory_usage_observer_wait_time; + if (wait_time != 0) + cgroups_memory_usage_observer.emplace(std::chrono::seconds(wait_time)); + } + catch (Exception &) + { + tryLogCurrentException(log, "Disabling cgroup memory observer because of an error during initialization"); } std::string cert_path = config().getString("openSSL.server.certificateFile", ""); @@ -1602,8 +1569,6 @@ try background_memory_tracker.setDescription("(background)"); background_memory_tracker.setMetric(CurrentMetrics::MergesMutationsMemoryTracking); - total_memory_tracker.setAllowUseJemallocMemory(new_server_settings.allow_use_jemalloc_memory); - auto * global_overcommit_tracker = global_context->getGlobalOvercommitTracker(); total_memory_tracker.setOvercommitTracker(global_overcommit_tracker); diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index b3e53c29d4a..a5c9875188b 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -57,12 +58,10 @@ static std::unique_ptr openFileIfExists(const std::stri AsynchronousMetrics::AsynchronousMetrics( unsigned update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_, - std::shared_ptr cgroups_reader_) + const ProtocolServerMetricsFunc & protocol_server_metrics_func_) : update_period(update_period_seconds) , log(getLogger("AsynchronousMetrics")) , protocol_server_metrics_func(protocol_server_metrics_func_) - , cgroups_reader(std::move(cgroups_reader_)) { #if defined(OS_LINUX) openFileIfExists("/proc/meminfo", meminfo); @@ -378,23 +377,13 @@ void AsynchronousMetrics::run() namespace { -uint64_t updateJemallocEpoch() -{ - uint64_t value = 0; - size_t size = sizeof(value); - mallctl("epoch", &value, &size, &value, size); - return value; -} - template Value saveJemallocMetricImpl( AsynchronousMetricValues & values, const std::string & jemalloc_full_name, const std::string & clickhouse_full_name) { - Value value{}; - size_t size = sizeof(value); - mallctl(jemalloc_full_name.c_str(), &value, &size, nullptr, 0); + auto value = getJemallocValue(jemalloc_full_name.c_str()); values[clickhouse_full_name] = AsynchronousMetricValue(value, "An internal metric of the low-level memory allocator (jemalloc). See https://jemalloc.net/jemalloc.3.html"); return value; } @@ -604,7 +593,7 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) // 'epoch' is a special mallctl -- it updates the statistics. Without it, all // the following calls will return stale values. It increments and returns // the current epoch number, which might be useful to log as a sanity check. - auto epoch = updateJemallocEpoch(); + auto epoch = getJemallocValue("epoch"); new_values["jemalloc.epoch"] = { epoch, "An internal incremental update number of the statistics of jemalloc (Jason Evans' memory allocator), used in all other `jemalloc` metrics." }; // Collect the statistics themselves. diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index 0b110f41fc3..bc379d4e92b 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -69,8 +69,7 @@ public: AsynchronousMetrics( unsigned update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_, - std::shared_ptr cgroups_reader_); + const ProtocolServerMetricsFunc & protocol_server_metrics_func_); virtual ~AsynchronousMetrics(); diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index b12845df098..ab7ca69ca04 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -191,11 +191,9 @@ std::shared_ptr createCgroupsReader() } -CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_, std::shared_ptr cgroups_reader_) - : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_), cgroups_reader(std::move(cgroups_reader_)) -{ - cgroups_reader = createCgroupsReader(); -} +CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_) + : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_), cgroups_reader(createCgroupsReader()) +{} CgroupsMemoryUsageObserver::~CgroupsMemoryUsageObserver() { diff --git a/src/Common/CgroupsMemoryUsageObserver.h b/src/Common/CgroupsMemoryUsageObserver.h index 078307a6fa0..33e0f167a59 100644 --- a/src/Common/CgroupsMemoryUsageObserver.h +++ b/src/Common/CgroupsMemoryUsageObserver.h @@ -18,28 +18,20 @@ struct ICgroupsReader std::shared_ptr createCgroupsReader(); -/// Does two things: -/// 1. Periodically reads the memory usage of the process from Linux cgroups. -/// You can specify soft or hard memory limits: -/// - When the soft memory limit is hit, drop jemalloc cache. -/// - When the hard memory limit is hit, update MemoryTracking metric to throw memory exceptions faster. -/// The goal of this is to avoid that the process hits the maximum allowed memory limit at which there is a good -/// chance that the Limux OOM killer terminates it. All of this is done is because internal memory tracking in -/// ClickHouse can unfortunately under-estimate the actually used memory. -/// 2. Periodically reads the the maximum memory available to the process (which can change due to cgroups settings). -/// You can specify a callback to react on changes. The callback typically reloads the configuration, i.e. Server -/// or Keeper configuration file. This reloads settings 'max_server_memory_usage' (Server) and 'max_memory_usage_soft_limit' -/// (Keeper) from which various other internal limits are calculated, including the soft and hard limits for (1.). -/// The goal of this is to provide elasticity when the container is scaled-up/scaled-down. The mechanism (polling -/// cgroups) is quite implicit, unfortunately there is currently no better way to communicate memory threshold changes -/// to the database. +/// Periodically reads the the maximum memory available to the process (which can change due to cgroups settings). +/// You can specify a callback to react on changes. The callback typically reloads the configuration, i.e. Server +/// or Keeper configuration file. This reloads settings 'max_server_memory_usage' (Server) and 'max_memory_usage_soft_limit' +/// (Keeper) from which various other internal limits are calculated, including the soft and hard limits for (1.). +/// The goal of this is to provide elasticity when the container is scaled-up/scaled-down. The mechanism (polling +/// cgroups) is quite implicit, unfortunately there is currently no better way to communicate memory threshold changes +/// to the database. #if defined(OS_LINUX) class CgroupsMemoryUsageObserver { public: using OnMemoryAmountAvailableChangedFn = std::function; - explicit CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_, std::shared_ptr cgroups_reader_); + explicit CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_); ~CgroupsMemoryUsageObserver(); void setOnMemoryAmountAvailableChangedFn(OnMemoryAmountAvailableChangedFn on_memory_amount_available_changed_); diff --git a/src/Common/Jemalloc.h b/src/Common/Jemalloc.h index 0c533711f78..629d039b483 100644 --- a/src/Common/Jemalloc.h +++ b/src/Common/Jemalloc.h @@ -28,28 +28,46 @@ void setJemallocValue(const char * name, T value) { T old_value; size_t old_value_size = sizeof(T); - if (mallctl(name, &old_value, &old_value_size, reinterpret_cast(&value), sizeof(T))) - { - LOG_WARNING(getLogger("Jemalloc"), "mallctl for {} failed", name); - return; - } - + mallctl(name, &old_value, &old_value_size, reinterpret_cast(&value), sizeof(T)); LOG_INFO(getLogger("Jemalloc"), "Value for {} set to {} (from {})", name, value, old_value); } template -std::optional getJemallocValue(const char * name) +T getJemallocValue(const char * name) { T value; size_t value_size = sizeof(T); - if (mallctl(name, &value, &value_size, nullptr, 0)) - { - LOG_WARNING(getLogger("Jemalloc"), "mallctl for {} failed", name); - return std::nullopt; - } + mallctl(name, &value, &value_size, nullptr, 0); return value; } +template +struct JemallocMibCache +{ + explicit JemallocMibCache(const char * name) + { + mallctlnametomib(name, mib, &mib_length); + } + + void setValue(T value) + { + mallctlbymib(mib, mib_length, nullptr, nullptr, reinterpret_cast(&value), sizeof(T)); + } + + T getValue() + { + T value; + size_t value_size = sizeof(T); + mallctlbymib(mib, mib_length, &value, &value_size, nullptr, 0); + return value; + } + +private: + static constexpr size_t max_mib_length = 4; + size_t mib[max_mib_length]; + size_t mib_length = max_mib_length; +}; + } #endif diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index a541eeeb25f..e237c3a0d33 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -108,8 +108,6 @@ void AllocationTrace::onFreeImpl(void * ptr, size_t size) const namespace ProfileEvents { extern const Event QueryMemoryLimitExceeded; - extern const Event MemoryAllocatorPurge; - extern const Event MemoryAllocatorPurgeTimeMicroseconds; } using namespace std::chrono_literals; @@ -119,8 +117,6 @@ static constexpr size_t log_peak_memory_usage_every = 1ULL << 30; MemoryTracker total_memory_tracker(nullptr, VariableContext::Global); MemoryTracker background_memory_tracker(&total_memory_tracker, VariableContext::User, false); -std::atomic MemoryTracker::has_free_memory_in_allocator_arenas; - MemoryTracker::MemoryTracker(VariableContext level_) : parent(&total_memory_tracker), level(level_) {} MemoryTracker::MemoryTracker(MemoryTracker * parent_, VariableContext level_) : parent(parent_), level(level_) {} @@ -500,12 +496,10 @@ void MemoryTracker::reset() } -void MemoryTracker::setRSS(Int64 rss_, bool has_free_memory_in_allocator_arenas_) +void MemoryTracker::setRSS(Int64 rss_) { Int64 new_amount = rss_; - if (rss_) - total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); - has_free_memory_in_allocator_arenas.store(has_free_memory_in_allocator_arenas_, std::memory_order_relaxed); + total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); auto metric_loaded = total_memory_tracker.metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 257ed7d0629..4085bb321ed 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -56,9 +56,6 @@ private: std::atomic soft_limit {0}; std::atomic hard_limit {0}; std::atomic profiler_limit {0}; - std::atomic_bool allow_use_jemalloc_memory {true}; - - static std::atomic has_free_memory_in_allocator_arenas; Int64 profiler_step = 0; @@ -153,14 +150,6 @@ public: { return soft_limit.load(std::memory_order_relaxed); } - void setAllowUseJemallocMemory(bool value) - { - allow_use_jemalloc_memory.store(value, std::memory_order_relaxed); - } - bool getAllowUseJemallocMmemory() const - { - return allow_use_jemalloc_memory.load(std::memory_order_relaxed); - } /** Set limit if it was not set. * Otherwise, set limit to new value, if new value is greater than previous limit. @@ -251,7 +240,7 @@ public: /// Reset current counter to an RSS value. /// Jemalloc may have pre-allocated arenas, they are accounted in RSS. /// We can free this arenas in case of exception to avoid OOM. - static void setRSS(Int64 rss_, bool has_free_memory_in_allocator_arenas_); + static void setRSS(Int64 rss_); /// Prints info about peak memory consumption into log. void logPeakMemoryUsage(); diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp new file mode 100644 index 00000000000..dce47b83667 --- /dev/null +++ b/src/Common/MemoryWorker.cpp @@ -0,0 +1,49 @@ +#include "Common/ThreadPool.h" +#include + +#include +#include + +namespace DB +{ + +#if USE_JEMALLOC +MemoryWorker::MemoryWorker(uint64_t period_ms_) + : period_ms(period_ms_) +{ + background_thread = ThreadFromGlobalPool([this] { backgroundThread(); }); +} + +MemoryWorker::~MemoryWorker() +{ + { + std::unique_lock lock(mutex); + shutdown = true; + } + cv.notify_all(); + + if (background_thread.joinable()) + background_thread.join(); +} + +void MemoryWorker::backgroundThread() +{ + JemallocMibCache epoch_mib("epoch"); + JemallocMibCache resident_mib("stats.resident"); + std::unique_lock lock(mutex); + while (true) + { + cv.wait_for(lock, period_ms, [this] { return shutdown; }); + if (shutdown) + return; + + epoch_mib.setValue(0); + Int64 resident = resident_mib.getValue(); + MemoryTracker::setRSS(resident); + if (resident > total_memory_tracker.getHardLimit()) + purgeJemallocArenas(); + } +} +#endif + +} diff --git a/src/Common/MemoryWorker.h b/src/Common/MemoryWorker.h new file mode 100644 index 00000000000..8048194d9cd --- /dev/null +++ b/src/Common/MemoryWorker.h @@ -0,0 +1,34 @@ +#pragma once + +#include + +#include "config.h" + +namespace DB +{ + +#if USE_JEMALLOC +class MemoryWorker +{ +public: + explicit MemoryWorker(uint64_t period_ms_); + + ~MemoryWorker(); +private: + void backgroundThread(); + + ThreadFromGlobalPool background_thread; + + std::mutex mutex; + std::condition_variable cv; + bool shutdown = false; + + std::chrono::milliseconds period_ms; +}; +#else +class MemoryWorker +{ +}; +#endif + +} diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 3e404b7152b..86166ffe31b 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -115,7 +115,7 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM KeeperAsynchronousMetrics::KeeperAsynchronousMetrics( ContextPtr context_, unsigned update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) - : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, context_->getCgroupsReader()), context(std::move(context_)) + : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), context(std::move(context_)) { } diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index 28b32a6e6a5..aaea0388239 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -157,6 +157,7 @@ namespace DB M(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `system.*_log` tables before the startup. It can be helpful if some startup scripts depend on these tables.", 0) \ M(Double, gwp_asan_force_sample_probability, 0.0003, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ M(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \ + M(UInt64, memory_worker_period_ms, 100, "Period of background memory worker which corrects memory tracker memory usages and cleans up unused pages during higher memory usage.", 0) \ /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp diff --git a/src/Interpreters/Context.cpp b/src/Interpreters/Context.cpp index 6a24e049998..f70ccfd77be 100644 --- a/src/Interpreters/Context.cpp +++ b/src/Interpreters/Context.cpp @@ -406,8 +406,6 @@ struct ContextSharedPart : boost::noncopyable std::unique_ptr cluster_discovery TSA_GUARDED_BY(clusters_mutex); size_t clusters_version TSA_GUARDED_BY(clusters_mutex) = 0; - std::shared_ptr cgroups_reader; - /// No lock required for async_insert_queue modified only during initialization std::shared_ptr async_insert_queue; @@ -5631,16 +5629,6 @@ const ServerSettings & Context::getServerSettings() const return shared->server_settings; } -void Context::setCgroupsReader(std::shared_ptr cgroups_reader_) -{ - shared->cgroups_reader = std::move(cgroups_reader_); -} - -std::shared_ptr Context::getCgroupsReader() const -{ - return shared->cgroups_reader; -} - uint64_t HTTPContext::getMaxHstsAge() const { return context->getSettingsRef().hsts_max_age; diff --git a/src/Interpreters/Context.h b/src/Interpreters/Context.h index 33b742d20ad..0e3c0591c12 100644 --- a/src/Interpreters/Context.h +++ b/src/Interpreters/Context.h @@ -1344,9 +1344,6 @@ public: const ServerSettings & getServerSettings() const; - void setCgroupsReader(std::shared_ptr cgroups_reader_); - std::shared_ptr getCgroupsReader() const; - private: std::shared_ptr getSettingsConstraintsAndCurrentProfilesWithLock() const; diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 6ee0168bede..872a9f864df 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -57,7 +57,7 @@ ServerAsynchronousMetrics::ServerAsynchronousMetrics( unsigned heavy_metrics_update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) : WithContext(global_context_) - , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, getContext()->getCgroupsReader()) + , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_) , heavy_metric_update_period(heavy_metrics_update_period_seconds) { /// sanity check diff --git a/src/Storages/System/StorageSystemServerSettings.cpp b/src/Storages/System/StorageSystemServerSettings.cpp index d242b6de4ec..ee99c472620 100644 --- a/src/Storages/System/StorageSystemServerSettings.cpp +++ b/src/Storages/System/StorageSystemServerSettings.cpp @@ -63,7 +63,6 @@ void StorageSystemServerSettings::fillData(MutableColumns & res_columns, Context /// current setting values, one needs to ask the components directly. std::unordered_map> changeable_settings = { {"max_server_memory_usage", {std::to_string(total_memory_tracker.getHardLimit()), ChangeableWithoutRestart::Yes}}, - {"allow_use_jemalloc_memory", {std::to_string(total_memory_tracker.getAllowUseJemallocMmemory()), ChangeableWithoutRestart::Yes}}, {"max_table_size_to_drop", {std::to_string(context->getMaxTableSizeToDrop()), ChangeableWithoutRestart::Yes}}, {"max_partition_size_to_drop", {std::to_string(context->getMaxPartitionSizeToDrop()), ChangeableWithoutRestart::Yes}}, diff --git a/tests/integration/test_memory_limit/test.py b/tests/integration/test_memory_limit/test.py index 6d6745711da..db68a38c1b1 100644 --- a/tests/integration/test_memory_limit/test.py +++ b/tests/integration/test_memory_limit/test.py @@ -13,7 +13,6 @@ node = cluster.add_instance( "configs/async_metrics_no.xml", ], mem_limit="4g", - env_variables={"MALLOC_CONF": "dirty_decay_ms:0"}, ) From 9ec1fd1ab769b2f6c6ad713b4e747a89bde48b78 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 18 Jul 2024 11:29:03 +0200 Subject: [PATCH 071/162] Fix non-jemalloc builds --- src/Common/MemoryWorker.cpp | 2 +- src/Common/MemoryWorker.h | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index dce47b83667..e5ebbe3b979 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -1,8 +1,8 @@ -#include "Common/ThreadPool.h" #include #include #include +#include namespace DB { diff --git a/src/Common/MemoryWorker.h b/src/Common/MemoryWorker.h index 8048194d9cd..5f02fd0b1d0 100644 --- a/src/Common/MemoryWorker.h +++ b/src/Common/MemoryWorker.h @@ -28,6 +28,8 @@ private: #else class MemoryWorker { +public: + explicit MemoryWorker(uint64_t /*period_ms_*/) {} }; #endif From 05c7dc582a48d738bed82bcb24d3a7619fec8bc9 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 18 Jul 2024 13:40:03 +0200 Subject: [PATCH 072/162] Add some comments --- src/Common/Jemalloc.h | 3 +++ src/Common/MemoryWorker.h | 6 ++++++ 2 files changed, 9 insertions(+) diff --git a/src/Common/Jemalloc.h b/src/Common/Jemalloc.h index 629d039b483..dfa265c5e59 100644 --- a/src/Common/Jemalloc.h +++ b/src/Common/Jemalloc.h @@ -41,6 +41,9 @@ T getJemallocValue(const char * name) return value; } +/// Each mallctl call consists of string name lookup which can be expensive. +/// This can be avoided by translating name to "Management Information Base" (MIB) +/// and using it in mallctlbymib calls template struct JemallocMibCache { diff --git a/src/Common/MemoryWorker.h b/src/Common/MemoryWorker.h index 5f02fd0b1d0..6c0a578aa61 100644 --- a/src/Common/MemoryWorker.h +++ b/src/Common/MemoryWorker.h @@ -8,6 +8,12 @@ namespace DB { #if USE_JEMALLOC +/// Correct MemoryTracker based on stats.resident read from jemalloc. +/// This requires jemalloc built with --enable-stats which we use. +/// The worker spawns a background thread which moves the jemalloc epoch (updates internal stats), +/// and fetches the current stats.resident whose value is sent to global MemoryTracker. +/// Additionally, if the current memory usage is higher than global hard limit, +/// jemalloc's dirty pages are forcefully purged. class MemoryWorker { public: From e47fe15968b84a3f9ea5a911bbad2a942fd27188 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 18 Jul 2024 12:22:03 +0000 Subject: [PATCH 073/162] Simplify condition --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 06455526f1b..e967b575acb 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -591,10 +591,9 @@ Pipe ReadFromMergeTree::readInOrder( /// If parallel replicas enabled, set total rows in progress here only on initiator with local plan /// Otherwise rows will counted multiple times const UInt64 in_order_limit = query_info.input_order_info ? query_info.input_order_info->limit : 0; - const bool parallel_replicas_remote_plan_for_initiator = is_parallel_reading_from_replicas - && !context->getSettingsRef().parallel_replicas_local_plan && context->canUseParallelReplicasOnInitiator(); - const bool parallel_replicas_follower = is_parallel_reading_from_replicas && context->canUseParallelReplicasOnFollower(); - const bool set_total_rows_approx = !parallel_replicas_follower && !parallel_replicas_remote_plan_for_initiator; + const bool parallel_replicas_local_plan_for_initiator = is_parallel_reading_from_replicas + && context->getSettingsRef().parallel_replicas_local_plan && context->canUseParallelReplicasOnInitiator(); + const bool set_total_rows_approx = !is_parallel_reading_from_replicas || parallel_replicas_local_plan_for_initiator; Pipes pipes; for (size_t i = 0; i < parts_with_ranges.size(); ++i) From 7523dafc067c000ffe329808f1f409c2a52d3e9e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 18 Jul 2024 13:09:07 +0000 Subject: [PATCH 074/162] Fix after incorrect merge conflict resolution --- src/Processors/QueryPlan/ReadFromRemote.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 4599343f5c1..fdf35be6460 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -493,7 +493,7 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder /// when using local plan for local replica, local replica has 0 number size_t offset = (exclude_local_replica ? 1 : 0); - for (size_t i = 0; i < max_replicas_to_use; ++i) + for (size_t i = 0 + offset; i < pools_to_use.size(); ++i) { IConnections::ReplicaInfo replica_info{ /// we should use this number specifically because efficiency of data distribution by consistent hash depends on it. From 7ad07657a32415bd1b9df5541452d46e3b1e0c3d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 18 Jul 2024 14:16:44 +0000 Subject: [PATCH 075/162] Fix --- src/Processors/QueryPlan/ReadFromRemote.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index fdf35be6460..79ad436c37e 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -493,7 +493,7 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder /// when using local plan for local replica, local replica has 0 number size_t offset = (exclude_local_replica ? 1 : 0); - for (size_t i = 0 + offset; i < pools_to_use.size(); ++i) + for (size_t i = 0 + offset; i < max_replicas_to_use; ++i) { IConnections::ReplicaInfo replica_info{ /// we should use this number specifically because efficiency of data distribution by consistent hash depends on it. From 7d66f400b25a5bf0f2f43f06c6ed432d1c572fb6 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 18 Jul 2024 16:51:49 +0200 Subject: [PATCH 076/162] Better --- src/Common/Jemalloc.h | 5 +++++ src/Common/MemoryTracker.cpp | 13 ++++++++----- src/Common/MemoryTracker.h | 8 ++++---- src/Common/MemoryWorker.cpp | 27 +++++++++++++++++++++++++-- src/Common/ProfileEvents.cpp | 3 +++ 5 files changed, 45 insertions(+), 11 deletions(-) diff --git a/src/Common/Jemalloc.h b/src/Common/Jemalloc.h index dfa265c5e59..22a94a44eba 100644 --- a/src/Common/Jemalloc.h +++ b/src/Common/Jemalloc.h @@ -65,6 +65,11 @@ struct JemallocMibCache return value; } + void run() + { + mallctlbymib(mib, mib_length, nullptr, nullptr, nullptr, 0); + } + private: static constexpr size_t max_mib_length = 4; size_t mib[max_mib_length]; diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index e237c3a0d33..49a3a6ef7ef 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -242,6 +242,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed * So, we allow over-allocations. */ Int64 will_be = size ? size + amount.fetch_add(size, std::memory_order_relaxed) : amount.load(std::memory_order_relaxed); + Int64 will_be_rss = size + rss.load(std::memory_order_relaxed); auto metric_loaded = metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end() && size) @@ -290,7 +291,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed } } - if (unlikely(current_hard_limit && will_be > current_hard_limit)) + if (unlikely(current_hard_limit && (will_be > current_hard_limit || will_be_rss > current_hard_limit))) { if (memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) { @@ -310,12 +311,13 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed throw DB::Exception( DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED, "Memory limit{}{} exceeded: " - "would use {} (attempt to allocate chunk of {} bytes), maximum: {}." + "would use {} (attempt to allocate chunk of {} bytes), current RSS {}, maximum: {}." "{}{}", description ? " " : "", description ? description : "", formatReadableSizeWithBinarySuffix(will_be), size, + formatReadableSizeWithBinarySuffix(rss.load(std::memory_order_relaxed)), formatReadableSizeWithBinarySuffix(current_hard_limit), overcommit_result == OvercommitResult::NONE ? "" : " OvercommitTracker decision: ", toDescription(overcommit_result)); @@ -496,17 +498,18 @@ void MemoryTracker::reset() } -void MemoryTracker::setRSS(Int64 rss_) +void MemoryTracker::updateValues(Int64 rss_, Int64 allocated_) { - Int64 new_amount = rss_; + Int64 new_amount = allocated_; total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); + total_memory_tracker.rss.store(rss_, std::memory_order_relaxed); auto metric_loaded = total_memory_tracker.metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) CurrentMetrics::set(metric_loaded, new_amount); bool log_memory_usage = true; - total_memory_tracker.updatePeak(rss_, log_memory_usage); + total_memory_tracker.updatePeak(new_amount, log_memory_usage); } diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 4085bb321ed..add8bcb43d2 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -57,6 +57,8 @@ private: std::atomic hard_limit {0}; std::atomic profiler_limit {0}; + std::atomic rss{0}; + Int64 profiler_step = 0; /// To test exception safety of calling code, memory tracker throws an exception on each memory allocation with specified probability. @@ -237,10 +239,8 @@ public: /// Reset the accumulated data. void reset(); - /// Reset current counter to an RSS value. - /// Jemalloc may have pre-allocated arenas, they are accounted in RSS. - /// We can free this arenas in case of exception to avoid OOM. - static void setRSS(Int64 rss_); + /// update values based on external information (e.g. jemalloc's stat) + static void updateValues(Int64 rss_, Int64 allocated_); /// Prints info about peak memory consumption into log. void logPeakMemoryUsage(); diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index e5ebbe3b979..ae488a47b67 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -3,11 +3,23 @@ #include #include #include +#include + +namespace ProfileEvents +{ + extern const Event MemoryAllocatorPurge; + extern const Event MemoryAllocatorPurgeTimeMicroseconds; + extern const Event MemoryWorkerRun; + extern const Event MemoryWorkerRunElapsedMicroseconds; +} namespace DB { #if USE_JEMALLOC +#define STRINGIFY_HELPER(x) #x +#define STRINGIFY(x) STRINGIFY_HELPER(x) + MemoryWorker::MemoryWorker(uint64_t period_ms_) : period_ms(period_ms_) { @@ -30,6 +42,8 @@ void MemoryWorker::backgroundThread() { JemallocMibCache epoch_mib("epoch"); JemallocMibCache resident_mib("stats.resident"); + JemallocMibCache allocated_mib("stats.allocated"); + JemallocMibCache purge_mib("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge"); std::unique_lock lock(mutex); while (true) { @@ -37,11 +51,20 @@ void MemoryWorker::backgroundThread() if (shutdown) return; + Stopwatch total_watch; epoch_mib.setValue(0); Int64 resident = resident_mib.getValue(); - MemoryTracker::setRSS(resident); if (resident > total_memory_tracker.getHardLimit()) - purgeJemallocArenas(); + { + Stopwatch purge_watch; + purge_mib.run(); + ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); + ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, purge_watch.elapsedMicroseconds()); + } + + MemoryTracker::updateValues(resident, allocated_mib.getValue()); + ProfileEvents::increment(ProfileEvents::MemoryWorkerRun); + ProfileEvents::increment(ProfileEvents::MemoryWorkerRunElapsedMicroseconds, total_watch.elapsedMicroseconds()); } } #endif diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 871ba7cab8b..d85c21fcded 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -778,6 +778,9 @@ The server successfully detected this situation and will download merged part fr M(GWPAsanAllocateSuccess, "Number of successful allocations done by GWPAsan") \ M(GWPAsanAllocateFailed, "Number of failed allocations done by GWPAsan (i.e. filled pool)") \ M(GWPAsanFree, "Number of free operations done by GWPAsan") \ + \ + M(MemoryWorkerRun, "Number of runs done by MemoryWorker in background") \ + M(MemoryWorkerRunElapsedMicroseconds, "Total time spent by MemoryWorker for background work") \ #ifdef APPLY_FOR_EXTERNAL_EVENTS From 6db09e884b9fcc516c6c4c0afe1dd3a33ec1ada9 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 19 Jul 2024 08:24:29 +0000 Subject: [PATCH 077/162] Simplify code --- src/Processors/QueryPlan/ReadFromRemote.cpp | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 79ad436c37e..8a2d773696b 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -491,16 +491,17 @@ void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder addresses.emplace_back(pool->getAddress()); LOG_DEBUG(getLogger("ReadFromParallelRemoteReplicasStep"), "Addresses to use: {}", fmt::join(addresses, ", ")); - /// when using local plan for local replica, local replica has 0 number - size_t offset = (exclude_local_replica ? 1 : 0); - for (size_t i = 0 + offset; i < max_replicas_to_use; ++i) + /// when using local plan for local replica, 0 is assigned to local replica as replica num, - in this case, starting from 1 here + size_t replica_num = (exclude_local_replica ? 1 : 0); + for (const auto & pool : pools_to_use) { IConnections::ReplicaInfo replica_info{ /// we should use this number specifically because efficiency of data distribution by consistent hash depends on it. - .number_of_current_replica = i, + .number_of_current_replica = replica_num, }; + ++replica_num; - addPipeForSingeReplica(pipes, pools_to_use[i - offset], replica_info); + addPipeForSingeReplica(pipes, pool, replica_info); } auto pipe = Pipe::unitePipes(std::move(pipes)); From daabf2275e3b04bc40949aade73f544f56d78931 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 19 Jul 2024 21:46:14 +0000 Subject: [PATCH 078/162] Remove wrong comment --- src/Processors/QueryPlan/ReadFromRemote.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 8a2d773696b..46d4aa29e70 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -393,7 +393,6 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( for (const auto & addr : cluster->getShardsAddresses().front()) { - /// skip first local if (exclude_local_replica && addr.is_local) continue; From 4e69cd0d5262fed5bbebaf30fd2f34d6cb2ad830 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 20 Jul 2024 20:53:18 +0000 Subject: [PATCH 079/162] Avoid getting callbacks from context for local plan --- .../QueryPlan/ParallelReplicasLocalPlan.cpp | 2 +- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 15 +++++++++++---- src/Processors/QueryPlan/ReadFromMergeTree.h | 4 ++-- 3 files changed, 14 insertions(+), 7 deletions(-) diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index e9d5ef90e30..e8ff0f417dc 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -85,7 +85,7 @@ std::pair, bool> createLocalPlanForParallelReplicas( { return coordinator->handleRequest(std::move(req)); }; auto read_from_merge_tree_parallel_replicas - = reading->createLocalParallelReplicasReadingStep(analyzed_result_ptr, all_ranges_cb, read_task_cb); + = reading->createLocalParallelReplicasReadingStep(analyzed_result_ptr, std::move(all_ranges_cb), std::move(read_task_cb)); node->step = std::move(read_from_merge_tree_parallel_replicas); addConvertingActions(*query_plan, header, /*has_missing_objects=*/false); diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index e967b575acb..6f0037684e9 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -303,8 +303,15 @@ ReadFromMergeTree::ReadFromMergeTree( { if (is_parallel_reading_from_replicas) { - all_ranges_callback = all_ranges_callback_.value_or(context->getMergeTreeAllRangesCallback()); - read_task_callback = read_task_callback_.value_or(context->getMergeTreeReadTaskCallback()); + if (all_ranges_callback_.has_value()) + all_ranges_callback = all_ranges_callback_.value(); + else + all_ranges_callback = context->getMergeTreeAllRangesCallback(); + + if (read_task_callback_.has_value()) + read_task_callback = read_task_callback_.value(); + else + read_task_callback = context->getMergeTreeReadTaskCallback(); } const auto & settings = context->getSettingsRef(); @@ -340,8 +347,8 @@ ReadFromMergeTree::ReadFromMergeTree( std::unique_ptr ReadFromMergeTree::createLocalParallelReplicasReadingStep( AnalysisResultPtr analyzed_result_ptr_, - std::optional all_ranges_callback_, - std::optional read_task_callback_) + MergeTreeAllRangesCallback all_ranges_callback_, + MergeTreeReadTaskCallback read_task_callback_) { const auto number_of_local_replica = 0; const bool enable_parallel_reading = true; diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index c9c6030d207..307b605c01c 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -128,8 +128,8 @@ public: std::unique_ptr createLocalParallelReplicasReadingStep( AnalysisResultPtr analyzed_result_ptr_, - std::optional all_ranges_callback_, - std::optional read_task_callback_); + MergeTreeAllRangesCallback all_ranges_callback_, + MergeTreeReadTaskCallback read_task_callback_); static constexpr auto name = "ReadFromMergeTree"; String getName() const override { return name; } From 2147a96475717f0af53dd62f487c011a5b9b933a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Sun, 21 Jul 2024 11:32:57 +0200 Subject: [PATCH 080/162] Better --- src/Common/AsynchronousMetrics.cpp | 4 +++ src/Common/MemoryTracker.cpp | 31 +++++++++++++++---- src/Common/MemoryTracker.h | 2 +- src/Common/MemoryWorker.cpp | 7 ++++- src/Coordination/KeeperDispatcher.cpp | 8 ++++- .../configs/keeper_config2.xml | 2 +- .../configs/keeper_config3.xml | 2 +- 7 files changed, 45 insertions(+), 11 deletions(-) diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index a5c9875188b..dc2f687004b 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -638,6 +638,10 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) "The amount of virtual memory mapped for the use of stack and for the allocated memory, in bytes." " It is unspecified whether it includes the per-thread stacks and most of the allocated memory, that is allocated with the 'mmap' system call." " This metric exists only for completeness reasons. I recommend to use the `MemoryResident` metric for monitoring."}; + +#if !USE_JEMALLOC + MemoryTracker::updateValues(data.resident, data.resident, /*force_update=*/true); +#endif } { diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 49a3a6ef7ef..07d6ba98745 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -221,6 +221,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed { /// For global memory tracker always update memory usage. amount.fetch_add(size, std::memory_order_relaxed); + rss.fetch_add(size, std::memory_order_relaxed); auto metric_loaded = metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) @@ -242,7 +243,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed * So, we allow over-allocations. */ Int64 will_be = size ? size + amount.fetch_add(size, std::memory_order_relaxed) : amount.load(std::memory_order_relaxed); - Int64 will_be_rss = size + rss.load(std::memory_order_relaxed); + Int64 will_be_rss = size ? size + rss.fetch_add(size, std::memory_order_relaxed) : rss.load(std::memory_order_relaxed); auto metric_loaded = metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end() && size) @@ -269,6 +270,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed { /// Revert amount.fetch_sub(size, std::memory_order_relaxed); + rss.fetch_sub(size, std::memory_order_relaxed); /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); @@ -291,7 +293,8 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed } } - if (unlikely(current_hard_limit && (will_be > current_hard_limit || will_be_rss > current_hard_limit))) + if (unlikely( + current_hard_limit && (will_be > current_hard_limit || (level == VariableContext::Global && will_be_rss > current_hard_limit)))) { if (memoryTrackerCanThrow(level, false) && throw_if_memory_exceeded) { @@ -303,6 +306,7 @@ AllocationTrace MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceed { /// Revert amount.fetch_sub(size, std::memory_order_relaxed); + rss.fetch_sub(size, std::memory_order_relaxed); /// Prevent recursion. Exception::ctor -> std::string -> new[] -> MemoryTracker::alloc MemoryTrackerBlockerInThread untrack_lock(VariableContext::Global); @@ -411,6 +415,7 @@ AllocationTrace MemoryTracker::free(Int64 size, double _sample_probability) { /// For global memory tracker always update memory usage. amount.fetch_sub(size, std::memory_order_relaxed); + rss.fetch_sub(size, std::memory_order_relaxed); auto metric_loaded = metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) CurrentMetrics::sub(metric_loaded, size); @@ -424,7 +429,12 @@ AllocationTrace MemoryTracker::free(Int64 size, double _sample_probability) } Int64 accounted_size = size; - if (level == VariableContext::Thread || level == VariableContext::Global) + if (level == VariableContext::Global) + { + amount.fetch_sub(accounted_size, std::memory_order_relaxed); + rss.fetch_sub(accounted_size, std::memory_order_relaxed); + } + else if (level == VariableContext::Thread) { /// Could become negative if memory allocated in this thread is freed in another one amount.fetch_sub(accounted_size, std::memory_order_relaxed); @@ -498,12 +508,21 @@ void MemoryTracker::reset() } -void MemoryTracker::updateValues(Int64 rss_, Int64 allocated_) +void MemoryTracker::updateValues(Int64 rss_, Int64 allocated_, bool force_update) { - Int64 new_amount = allocated_; - total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); total_memory_tracker.rss.store(rss_, std::memory_order_relaxed); + if (likely(!force_update && total_memory_tracker.amount.load(std::memory_order_relaxed) >= 0)) + return; + + Int64 new_amount = allocated_; + LOG_INFO( + getLogger("MemoryTracker"), + "Correcting the value of global memory tracker from {} to {}", + ReadableSize(total_memory_tracker.amount.load(std::memory_order_relaxed)), + ReadableSize(allocated_)); + total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); + auto metric_loaded = total_memory_tracker.metric.load(std::memory_order_relaxed); if (metric_loaded != CurrentMetrics::end()) CurrentMetrics::set(metric_loaded, new_amount); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index add8bcb43d2..4913be9781f 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -240,7 +240,7 @@ public: void reset(); /// update values based on external information (e.g. jemalloc's stat) - static void updateValues(Int64 rss_, Int64 allocated_); + static void updateValues(Int64 rss_, Int64 allocated_, bool force_update); /// Prints info about peak memory consumption into log. void logPeakMemoryUsage(); diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index ae488a47b67..23cd90178ff 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -44,6 +44,7 @@ void MemoryWorker::backgroundThread() JemallocMibCache resident_mib("stats.resident"); JemallocMibCache allocated_mib("stats.allocated"); JemallocMibCache purge_mib("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge"); + bool first_run = false; std::unique_lock lock(mutex); while (true) { @@ -62,9 +63,13 @@ void MemoryWorker::backgroundThread() ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, purge_watch.elapsedMicroseconds()); } - MemoryTracker::updateValues(resident, allocated_mib.getValue()); + /// force update the allocated stat from jemalloc for the first run to cover the allocations we missed + /// during initialization + MemoryTracker::updateValues(resident, allocated_mib.getValue(), first_run); ProfileEvents::increment(ProfileEvents::MemoryWorkerRun); ProfileEvents::increment(ProfileEvents::MemoryWorkerRunElapsedMicroseconds, total_watch.elapsedMicroseconds()); + + first_run = false; } } #endif diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 8c7e6405153..332662117d8 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -148,7 +148,13 @@ void KeeperDispatcher::requestThread() Int64 mem_soft_limit = keeper_context->getKeeperMemorySoftLimit(); if (configuration_and_settings->standalone_keeper && isExceedingMemorySoftLimit() && checkIfRequestIncreaseMem(request.request)) { - LOG_WARNING(log, "Processing requests refused because of max_memory_usage_soft_limit {}, the total used memory is {}, request type is {}", ReadableSize(mem_soft_limit), ReadableSize(total_memory_tracker.get()), request.request->getOpNum()); + LOG_WARNING( + log, + "Processing requests refused because of max_memory_usage_soft_limit {}, the total used memory is {}, request type " + "is {}", + ReadableSize(mem_soft_limit), + ReadableSize(total_memory_tracker.get()), + request.request->getOpNum()); addErrorResponses({request}, Coordination::Error::ZCONNECTIONLOSS); continue; } diff --git a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml index 25ececea3e8..e71b93379d0 100644 --- a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml +++ b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config2.xml @@ -16,7 +16,7 @@ az-zoo2 1 - 20000000 + 200000000 10000 diff --git a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml index 81e343b77c9..cf4a4686f2c 100644 --- a/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml +++ b/tests/integration/test_keeper_memory_soft_limit/configs/keeper_config3.xml @@ -13,7 +13,7 @@ 2181 3 - 20000000 + 200000000 10000 From 1c3f7d0fd0fbf27692ff29d8309382eae7b7a598 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 22 Jul 2024 14:58:00 +0200 Subject: [PATCH 081/162] Small fix --- programs/server/Server.cpp | 4 ++-- src/Common/MemoryWorker.cpp | 15 ++++++++++----- 2 files changed, 12 insertions(+), 7 deletions(-) diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 04480f0bfe9..5691d82e216 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -904,8 +904,6 @@ try LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); }); - MemoryWorker memory_worker(global_context->getServerSettings().memory_worker_period_ms); - /// This object will periodically calculate some metrics. ServerAsynchronousMetrics async_metrics( global_context, @@ -1198,6 +1196,8 @@ try FailPointInjection::enableFromGlobalConfig(config()); + MemoryWorker memory_worker(global_context->getServerSettings().memory_worker_period_ms); + int default_oom_score = 0; #if !defined(NDEBUG) diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index 23cd90178ff..2b945a30d3d 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -2,8 +2,9 @@ #include #include -#include #include +#include +#include namespace ProfileEvents { @@ -23,6 +24,7 @@ namespace DB MemoryWorker::MemoryWorker(uint64_t period_ms_) : period_ms(period_ms_) { + LOG_INFO(getLogger("MemoryWorker"), "Starting background memory thread with period of {}ms", period_ms.count()); background_thread = ThreadFromGlobalPool([this] { backgroundThread(); }); } @@ -42,9 +44,10 @@ void MemoryWorker::backgroundThread() { JemallocMibCache epoch_mib("epoch"); JemallocMibCache resident_mib("stats.resident"); + JemallocMibCache active_mib("stats.active"); JemallocMibCache allocated_mib("stats.allocated"); JemallocMibCache purge_mib("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge"); - bool first_run = false; + bool first_run = true; std::unique_lock lock(mutex); while (true) { @@ -55,6 +58,11 @@ void MemoryWorker::backgroundThread() Stopwatch total_watch; epoch_mib.setValue(0); Int64 resident = resident_mib.getValue(); + + /// force update the allocated stat from jemalloc for the first run to cover the allocations we missed + /// during initialization + MemoryTracker::updateValues(resident, allocated_mib.getValue(), first_run); + if (resident > total_memory_tracker.getHardLimit()) { Stopwatch purge_watch; @@ -63,9 +71,6 @@ void MemoryWorker::backgroundThread() ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, purge_watch.elapsedMicroseconds()); } - /// force update the allocated stat from jemalloc for the first run to cover the allocations we missed - /// during initialization - MemoryTracker::updateValues(resident, allocated_mib.getValue(), first_run); ProfileEvents::increment(ProfileEvents::MemoryWorkerRun); ProfileEvents::increment(ProfileEvents::MemoryWorkerRunElapsedMicroseconds, total_watch.elapsedMicroseconds()); From fa0f4543b688861a97843a8767d895f52b6dee9d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 22 Jul 2024 13:44:25 +0000 Subject: [PATCH 082/162] Fix marks multiplier for local replica to avoid overflow with fuzzer --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 6f0037684e9..5922550eaf3 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -384,7 +384,11 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas(RangesInDataParts parts_wit /// We have a special logic for local replica. It has to read less data, because in some cases it should /// merge states of aggregate functions or do some other important stuff other than reading from Disk. - const auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; + auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; + // avoid using multiplier if min marks to read is already too big (to avoid overflow) + if (pool_settings.min_marks_for_concurrent_read >= std::numeric_limits::max()) + multiplier = 1.0f; + if (auto result = pool_settings.min_marks_for_concurrent_read * multiplier; canConvertTo(result)) pool_settings.min_marks_for_concurrent_read = static_cast(result); else @@ -553,7 +557,11 @@ Pipe ReadFromMergeTree::readInOrder( .number_of_current_replica = number_of_current_replica.value_or(client_info.number_of_current_replica), }; - const auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; + auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; + // avoid using multiplier if min marks to read is already too big (to avoid overflow) + if (pool_settings.min_marks_for_concurrent_read >= std::numeric_limits::max()) + multiplier = 1.0f; + if (auto result = pool_settings.min_marks_for_concurrent_read * multiplier; canConvertTo(result)) pool_settings.min_marks_for_concurrent_read = static_cast(result); else From 4483f4b3894e846deb43fa4e7d0962eba2a2224b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 22 Jul 2024 16:39:07 +0000 Subject: [PATCH 083/162] Better logging for announcement --- .../MergeTree/ParallelReplicasReadingCoordinator.cpp | 7 +++---- src/Storages/MergeTree/RequestResponse.cpp | 5 +---- 2 files changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index e1c0d87837a..4e3c9b08fb0 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -433,12 +433,11 @@ void DefaultCoordinator::setProgressCallback() void DefaultCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) { - const auto replica_num = announcement.replica_num; - - LOG_DEBUG(log, "Initial request from replica {}: {}", announcement.replica_num, announcement.describe()); + LOG_DEBUG(log, "Initial request: {}", announcement.describe()); initializeReadingState(std::move(announcement)); + const auto replica_num = announcement.replica_num; if (replica_num >= stats.size()) throw Exception( ErrorCodes::LOGICAL_ERROR, "Replica number ({}) is bigger than total replicas count ({})", replica_num, stats.size()); @@ -859,7 +858,7 @@ void InOrderCoordinator::markReplicaAsUnavailable(size_t replica_number) template void InOrderCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) { - LOG_TRACE(log, "Received an announcement {}", announcement.describe()); + LOG_TRACE(log, "Received an announcement : {}", announcement.describe()); ++stats[announcement.replica_num].number_of_requests; diff --git a/src/Storages/MergeTree/RequestResponse.cpp b/src/Storages/MergeTree/RequestResponse.cpp index 33cd935c88b..bcdeb443a0b 100644 --- a/src/Storages/MergeTree/RequestResponse.cpp +++ b/src/Storages/MergeTree/RequestResponse.cpp @@ -129,10 +129,7 @@ void InitialAllRangesAnnouncement::serialize(WriteBuffer & out) const String InitialAllRangesAnnouncement::describe() { - String result; - result += description.describe(); - result += fmt::format("----------\nReceived from {} replica\n", replica_num); - return result; + return fmt::format("replica {}, mode {}, {}", replica_num, mode, description.describe()); } InitialAllRangesAnnouncement InitialAllRangesAnnouncement::deserialize(ReadBuffer & in) From 420716d702daf8885796ed1b9b2cf83c437d6e32 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 Jul 2024 06:38:37 +0000 Subject: [PATCH 084/162] Fix clang-tidy warning --- src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index 4e3c9b08fb0..a1b36fe1299 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -435,9 +435,10 @@ void DefaultCoordinator::doHandleInitialAllRangesAnnouncement(InitialAllRangesAn { LOG_DEBUG(log, "Initial request: {}", announcement.describe()); + const auto replica_num = announcement.replica_num; + initializeReadingState(std::move(announcement)); - const auto replica_num = announcement.replica_num; if (replica_num >= stats.size()) throw Exception( ErrorCodes::LOGICAL_ERROR, "Replica number ({}) is bigger than total replicas count ({})", replica_num, stats.size()); From d78cfd030fa8364456ac5283a6a1469703c53b40 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Mon, 22 Jul 2024 21:47:46 +0200 Subject: [PATCH 085/162] Use cgroups as source --- programs/keeper/Keeper.cpp | 8 +- programs/server/Server.cpp | 9 +- src/Common/AsynchronousMetrics.cpp | 32 +- src/Common/AsynchronousMetrics.h | 10 +- src/Common/CgroupsMemoryUsageObserver.cpp | 168 +---------- src/Common/CgroupsMemoryUsageObserver.h | 12 - src/Common/MemoryTracker.cpp | 9 +- src/Common/MemoryTracker.h | 3 +- src/Common/MemoryWorker.cpp | 280 ++++++++++++++++-- src/Common/MemoryWorker.h | 49 ++- .../KeeperAsynchronousMetrics.cpp | 9 +- src/Coordination/KeeperAsynchronousMetrics.h | 8 +- src/Core/ServerSettings.h | 2 +- .../ServerAsynchronousMetrics.cpp | 6 +- src/Interpreters/ServerAsynchronousMetrics.h | 5 +- 15 files changed, 367 insertions(+), 243 deletions(-) diff --git a/programs/keeper/Keeper.cpp b/programs/keeper/Keeper.cpp index b10d3f34623..d308e741311 100644 --- a/programs/keeper/Keeper.cpp +++ b/programs/keeper/Keeper.cpp @@ -376,7 +376,8 @@ try LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); }); - MemoryWorker memory_worker(config().getUInt64("memory_worker_period_ms", 100)); + MemoryWorker memory_worker(config().getUInt64("memory_worker_period_ms", 0)); + memory_worker.start(); static ServerErrorHandler error_handler; Poco::ErrorHandler::set(&error_handler); @@ -419,8 +420,9 @@ try for (const auto & server : *servers) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); return metrics; - } - ); + }, + /*update_jemalloc_epoch_=*/memory_worker.getSource() != MemoryWorker::MemoryUsageSource::Jemalloc, + /*update_rss_=*/memory_worker.getSource() == MemoryWorker::MemoryUsageSource::None); std::vector listen_hosts = DB::getMultipleValuesFromConfig(config(), "", "listen_host"); diff --git a/programs/server/Server.cpp b/programs/server/Server.cpp index 5691d82e216..1fc1df1494c 100644 --- a/programs/server/Server.cpp +++ b/programs/server/Server.cpp @@ -904,6 +904,8 @@ try LOG_INFO(log, "Background threads finished in {} ms", watch.elapsedMilliseconds()); }); + MemoryWorker memory_worker(global_context->getServerSettings().memory_worker_period_ms); + /// This object will periodically calculate some metrics. ServerAsynchronousMetrics async_metrics( global_context, @@ -922,8 +924,9 @@ try for (const auto & server : servers) metrics.emplace_back(ProtocolServerMetrics{server.getPortName(), server.currentThreads()}); return metrics; - } - ); + }, + /*update_jemalloc_epoch_=*/memory_worker.getSource() != MemoryWorker::MemoryUsageSource::Jemalloc, + /*update_rss_=*/memory_worker.getSource() == MemoryWorker::MemoryUsageSource::None); /// NOTE: global context should be destroyed *before* GlobalThreadPool::shutdown() /// Otherwise GlobalThreadPool::shutdown() will hang, since Context holds some threads. @@ -1196,7 +1199,7 @@ try FailPointInjection::enableFromGlobalConfig(config()); - MemoryWorker memory_worker(global_context->getServerSettings().memory_worker_period_ms); + memory_worker.start(); int default_oom_score = 0; diff --git a/src/Common/AsynchronousMetrics.cpp b/src/Common/AsynchronousMetrics.cpp index dc2f687004b..53b8e13eaaa 100644 --- a/src/Common/AsynchronousMetrics.cpp +++ b/src/Common/AsynchronousMetrics.cpp @@ -58,10 +58,14 @@ static std::unique_ptr openFileIfExists(const std::stri AsynchronousMetrics::AsynchronousMetrics( unsigned update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_) + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_) : update_period(update_period_seconds) , log(getLogger("AsynchronousMetrics")) , protocol_server_metrics_func(protocol_server_metrics_func_) + , update_jemalloc_epoch(update_jemalloc_epoch_) + , update_rss(update_rss_) { #if defined(OS_LINUX) openFileIfExists("/proc/meminfo", meminfo); @@ -377,6 +381,14 @@ void AsynchronousMetrics::run() namespace { +uint64_t updateJemallocEpoch() +{ + uint64_t value = 0; + size_t size = sizeof(value); + mallctl("epoch", &value, &size, &value, size); + return value; +} + template Value saveJemallocMetricImpl( AsynchronousMetricValues & values, @@ -593,8 +605,11 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) // 'epoch' is a special mallctl -- it updates the statistics. Without it, all // the following calls will return stale values. It increments and returns // the current epoch number, which might be useful to log as a sanity check. - auto epoch = getJemallocValue("epoch"); - new_values["jemalloc.epoch"] = { epoch, "An internal incremental update number of the statistics of jemalloc (Jason Evans' memory allocator), used in all other `jemalloc` metrics." }; + auto epoch = update_jemalloc_epoch ? updateJemallocEpoch() : getJemallocValue("epoch"); + new_values["jemalloc.epoch"] + = {epoch, + "An internal incremental update number of the statistics of jemalloc (Jason Evans' memory allocator), used in all other " + "`jemalloc` metrics."}; // Collect the statistics themselves. saveJemallocMetric(new_values, "allocated"); @@ -607,10 +622,10 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) saveJemallocMetric(new_values, "background_thread.num_threads"); saveJemallocMetric(new_values, "background_thread.num_runs"); saveJemallocMetric(new_values, "background_thread.run_intervals"); - saveJemallocProf(new_values, "active"); + saveJemallocProf(new_values, "active"); saveAllArenasMetric(new_values, "pactive"); - [[maybe_unused]] size_t je_malloc_pdirty = saveAllArenasMetric(new_values, "pdirty"); - [[maybe_unused]] size_t je_malloc_pmuzzy = saveAllArenasMetric(new_values, "pmuzzy"); + saveAllArenasMetric(new_values, "pdirty"); + saveAllArenasMetric(new_values, "pmuzzy"); saveAllArenasMetric(new_values, "dirty_purged"); saveAllArenasMetric(new_values, "muzzy_purged"); #endif @@ -639,9 +654,8 @@ void AsynchronousMetrics::update(TimePoint update_time, bool force_update) " It is unspecified whether it includes the per-thread stacks and most of the allocated memory, that is allocated with the 'mmap' system call." " This metric exists only for completeness reasons. I recommend to use the `MemoryResident` metric for monitoring."}; -#if !USE_JEMALLOC - MemoryTracker::updateValues(data.resident, data.resident, /*force_update=*/true); -#endif + if (update_rss) + MemoryTracker::updateRSS(data.resident); } { diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index bc379d4e92b..eb6ede7a558 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -7,10 +7,8 @@ #include #include -#include #include #include -#include #include #include #include @@ -69,7 +67,9 @@ public: AsynchronousMetrics( unsigned update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_); + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_); virtual ~AsynchronousMetrics(); @@ -92,7 +92,6 @@ private: virtual void logImpl(AsynchronousMetricValues &) {} ProtocolServerMetricsFunc protocol_server_metrics_func; - std::shared_ptr cgroups_reader; std::unique_ptr thread; @@ -113,6 +112,9 @@ private: MemoryStatisticsOS memory_stat TSA_GUARDED_BY(data_mutex); #endif + const bool update_jemalloc_epoch; + const bool update_rss; + #if defined(OS_LINUX) std::optional meminfo TSA_GUARDED_BY(data_mutex); std::optional loadavg TSA_GUARDED_BY(data_mutex); diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index ab7ca69ca04..afeac1808b2 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -13,12 +13,8 @@ #include #include -#include -#include -#include using namespace DB; -namespace fs = std::filesystem; namespace DB { @@ -29,170 +25,8 @@ extern const int FILE_DOESNT_EXIST; extern const int INCORRECT_DATA; } -} - -namespace -{ - -/// Format is -/// kernel 5 -/// rss 15 -/// [...] -uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & key) -{ - while (!buf.eof()) - { - std::string current_key; - readStringUntilWhitespace(current_key, buf); - if (current_key != key) - { - std::string dummy; - readStringUntilNewlineInto(dummy, buf); - buf.ignore(); - continue; - } - - assertChar(' ', buf); - uint64_t value = 0; - readIntText(value, buf); - return value; - } - - throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot find '{}' in '{}'", key, buf.getFileName()); -} - -struct CgroupsV1Reader : ICgroupsReader -{ - explicit CgroupsV1Reader(const fs::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { } - - uint64_t readMemoryUsage() override - { - std::lock_guard lock(mutex); - buf.rewind(); - return readMetricFromStatFile(buf, "rss"); - } - -private: - std::mutex mutex; - ReadBufferFromFile buf TSA_GUARDED_BY(mutex); -}; - -struct CgroupsV2Reader : ICgroupsReader -{ - explicit CgroupsV2Reader(const fs::path & stat_file_dir) - : current_buf(stat_file_dir / "memory.current"), stat_buf(stat_file_dir / "memory.stat") - { - } - - uint64_t readMemoryUsage() override - { - std::lock_guard lock(mutex); - current_buf.rewind(); - stat_buf.rewind(); - - int64_t mem_usage = 0; - /// memory.current contains a single number - /// the reason why we subtract it described here: https://github.com/ClickHouse/ClickHouse/issues/64652#issuecomment-2149630667 - readIntText(mem_usage, current_buf); - mem_usage -= readMetricFromStatFile(stat_buf, "inactive_file"); - chassert(mem_usage >= 0, "Negative memory usage"); - return mem_usage; - } - -private: - std::mutex mutex; - ReadBufferFromFile current_buf TSA_GUARDED_BY(mutex); - ReadBufferFromFile stat_buf TSA_GUARDED_BY(mutex); -}; - -/// Caveats: -/// - All of the logic in this file assumes that the current process is the only process in the -/// containing cgroup (or more precisely: the only process with significant memory consumption). -/// If this is not the case, then other processe's memory consumption may affect the internal -/// memory tracker ... -/// - Cgroups v1 and v2 allow nested cgroup hierarchies. As v1 is deprecated for over half a -/// decade and will go away at some point, hierarchical detection is only implemented for v2. -/// - I did not test what happens if a host has v1 and v2 simultaneously enabled. I believe such -/// systems existed only for a short transition period. - -std::optional getCgroupsV2Path() -{ - if (!cgroupsV2Enabled()) - return {}; - - if (!cgroupsV2MemoryControllerEnabled()) - return {}; - - fs::path current_cgroup = cgroupV2PathOfProcess(); - if (current_cgroup.empty()) - return {}; - - /// Return the bottom-most nested current memory file. If there is no such file at the current - /// level, try again at the parent level as memory settings are inherited. - while (current_cgroup != default_cgroups_mount.parent_path()) - { - const auto current_path = current_cgroup / "memory.current"; - const auto stat_path = current_cgroup / "memory.stat"; - if (fs::exists(current_path) && fs::exists(stat_path)) - return {current_cgroup}; - current_cgroup = current_cgroup.parent_path(); - } - return {}; -} - -std::optional getCgroupsV1Path() -{ - auto path = default_cgroups_mount / "memory/memory.stat"; - if (!fs::exists(path)) - return {}; - return {default_cgroups_mount / "memory"}; -} - -enum class CgroupsVersion : uint8_t -{ - V1, - V2 -}; - -std::pair getCgroupsPath() -{ - auto v2_path = getCgroupsV2Path(); - if (v2_path.has_value()) - return {*v2_path, CgroupsVersion::V2}; - - auto v1_path = getCgroupsV1Path(); - if (v1_path.has_value()) - return {*v1_path, CgroupsVersion::V1}; - - throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot find cgroups v1 or v2 current memory file"); -} - -} - -namespace DB -{ - -std::shared_ptr createCgroupsReader() -{ - const auto [cgroup_path, version] = getCgroupsPath(); - LOG_INFO( - getLogger("CgroupsReader"), - "Will create cgroup reader from '{}' (cgroups version: {})", - cgroup_path, - (version == CgroupsVersion::V1) ? "v1" : "v2"); - - if (version == CgroupsVersion::V2) - return std::make_shared(cgroup_path); - else - { - chassert(version == CgroupsVersion::V1); - return std::make_shared(cgroup_path); - } - -} - CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_) - : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_), cgroups_reader(createCgroupsReader()) + : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_) {} CgroupsMemoryUsageObserver::~CgroupsMemoryUsageObserver() diff --git a/src/Common/CgroupsMemoryUsageObserver.h b/src/Common/CgroupsMemoryUsageObserver.h index 33e0f167a59..3de83d6b437 100644 --- a/src/Common/CgroupsMemoryUsageObserver.h +++ b/src/Common/CgroupsMemoryUsageObserver.h @@ -3,21 +3,11 @@ #include #include -#include #include namespace DB { -struct ICgroupsReader -{ - virtual ~ICgroupsReader() = default; - - virtual uint64_t readMemoryUsage() = 0; -}; - -std::shared_ptr createCgroupsReader(); - /// Periodically reads the the maximum memory available to the process (which can change due to cgroups settings). /// You can specify a callback to react on changes. The callback typically reloads the configuration, i.e. Server /// or Keeper configuration file. This reloads settings 'max_server_memory_usage' (Server) and 'max_memory_usage_soft_limit' @@ -54,8 +44,6 @@ private: void runThread(); - std::shared_ptr cgroups_reader; - std::mutex thread_mutex; std::condition_variable cond; ThreadFromGlobalPool thread; diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 07d6ba98745..0ffae89ffa6 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -508,13 +508,13 @@ void MemoryTracker::reset() } -void MemoryTracker::updateValues(Int64 rss_, Int64 allocated_, bool force_update) +void MemoryTracker::updateRSS(Int64 rss_) { total_memory_tracker.rss.store(rss_, std::memory_order_relaxed); +} - if (likely(!force_update && total_memory_tracker.amount.load(std::memory_order_relaxed) >= 0)) - return; - +void MemoryTracker::updateAllocated(Int64 allocated_) +{ Int64 new_amount = allocated_; LOG_INFO( getLogger("MemoryTracker"), @@ -531,7 +531,6 @@ void MemoryTracker::updateValues(Int64 rss_, Int64 allocated_, bool force_update total_memory_tracker.updatePeak(new_amount, log_memory_usage); } - void MemoryTracker::setSoftLimit(Int64 value) { soft_limit.store(value, std::memory_order_relaxed); diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index 4913be9781f..d2db8489f19 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -240,7 +240,8 @@ public: void reset(); /// update values based on external information (e.g. jemalloc's stat) - static void updateValues(Int64 rss_, Int64 allocated_, bool force_update); + static void updateRSS(Int64 rss_); + static void updateAllocated(Int64 allocated_); /// Prints info about peak memory consumption into log. void logPeakMemoryUsage(); diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index 2b945a30d3d..42e797a80d6 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -1,11 +1,21 @@ #include +#include +#include +#include +#include #include #include #include #include #include +#include +#include +#include + +namespace fs = std::filesystem; + namespace ProfileEvents { extern const Event MemoryAllocatorPurge; @@ -17,14 +27,227 @@ namespace ProfileEvents namespace DB { -#if USE_JEMALLOC -#define STRINGIFY_HELPER(x) #x -#define STRINGIFY(x) STRINGIFY_HELPER(x) - -MemoryWorker::MemoryWorker(uint64_t period_ms_) - : period_ms(period_ms_) +namespace ErrorCodes { - LOG_INFO(getLogger("MemoryWorker"), "Starting background memory thread with period of {}ms", period_ms.count()); + extern const int FILE_DOESNT_EXIST; + extern const int INCORRECT_DATA; +} + +#if defined(OS_LINUX) +struct ICgroupsReader +{ + virtual ~ICgroupsReader() = default; + + virtual uint64_t readMemoryUsage() = 0; +}; + +namespace +{ + +/// Format is +/// kernel 5 +/// rss 15 +/// [...] +uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & key) +{ + while (!buf.eof()) + { + std::string current_key; + readStringUntilWhitespace(current_key, buf); + if (current_key != key) + { + std::string dummy; + readStringUntilNewlineInto(dummy, buf); + buf.ignore(); + continue; + } + + assertChar(' ', buf); + uint64_t value = 0; + readIntText(value, buf); + return value; + } + + LOG_ERROR(getLogger("CgroupsReader"), "Cannot find '{}' in '{}'", key, buf.getFileName()); + return 0; +} + +struct CgroupsV1Reader : ICgroupsReader +{ + explicit CgroupsV1Reader(const fs::path & stat_file_dir) : buf(stat_file_dir / "memory.stat") { } + + uint64_t readMemoryUsage() override + { + std::lock_guard lock(mutex); + buf.rewind(); + return readMetricFromStatFile(buf, "rss"); + } + +private: + std::mutex mutex; + ReadBufferFromFile buf TSA_GUARDED_BY(mutex); +}; + +struct CgroupsV2Reader : ICgroupsReader +{ + explicit CgroupsV2Reader(const fs::path & stat_file_dir) : stat_buf(stat_file_dir / "memory.stat") { } + + uint64_t readMemoryUsage() override + { + std::lock_guard lock(mutex); + stat_buf.rewind(); + return readMetricFromStatFile(stat_buf, "anon"); + } + +private: + std::mutex mutex; + ReadBufferFromFile stat_buf TSA_GUARDED_BY(mutex); +}; + +/// Caveats: +/// - All of the logic in this file assumes that the current process is the only process in the +/// containing cgroup (or more precisely: the only process with significant memory consumption). +/// If this is not the case, then other processe's memory consumption may affect the internal +/// memory tracker ... +/// - Cgroups v1 and v2 allow nested cgroup hierarchies. As v1 is deprecated for over half a +/// decade and will go away at some point, hierarchical detection is only implemented for v2. +/// - I did not test what happens if a host has v1 and v2 simultaneously enabled. I believe such +/// systems existed only for a short transition period. + +std::optional getCgroupsV2Path() +{ + if (!cgroupsV2Enabled()) + return {}; + + if (!cgroupsV2MemoryControllerEnabled()) + return {}; + + fs::path current_cgroup = cgroupV2PathOfProcess(); + if (current_cgroup.empty()) + return {}; + + /// Return the bottom-most nested current memory file. If there is no such file at the current + /// level, try again at the parent level as memory settings are inherited. + while (current_cgroup != default_cgroups_mount.parent_path()) + { + const auto current_path = current_cgroup / "memory.current"; + const auto stat_path = current_cgroup / "memory.stat"; + if (fs::exists(current_path) && fs::exists(stat_path)) + return {current_cgroup}; + current_cgroup = current_cgroup.parent_path(); + } + return {}; +} + +std::optional getCgroupsV1Path() +{ + auto path = default_cgroups_mount / "memory/memory.stat"; + if (!fs::exists(path)) + return {}; + return {default_cgroups_mount / "memory"}; +} + +enum class CgroupsVersion : uint8_t +{ + V1, + V2 +}; + +std::pair getCgroupsPath() +{ + auto v2_path = getCgroupsV2Path(); + if (v2_path.has_value()) + return {*v2_path, CgroupsVersion::V2}; + + auto v1_path = getCgroupsV1Path(); + if (v1_path.has_value()) + return {*v1_path, CgroupsVersion::V1}; + + throw Exception(ErrorCodes::FILE_DOESNT_EXIST, "Cannot find cgroups v1 or v2 current memory file"); +} + +std::shared_ptr createCgroupsReader() +{ + const auto [cgroup_path, version] = getCgroupsPath(); + LOG_INFO( + getLogger("CgroupsReader"), + "Will create cgroup reader from '{}' (cgroups version: {})", + cgroup_path, + (version == CgroupsVersion::V1) ? "v1" : "v2"); + + if (version == CgroupsVersion::V2) + return std::make_shared(cgroup_path); + else + { + chassert(version == CgroupsVersion::V1); + return std::make_shared(cgroup_path); + } + +} +#endif + +constexpr uint64_t cgroups_memory_usage_tick_ms{50}; +constexpr uint64_t jemalloc_memory_usage_tick_ms{100}; + +std::string_view sourceToString(MemoryWorker::MemoryUsageSource source) +{ + switch (source) + { + case MemoryWorker::MemoryUsageSource::Cgroups: return "Cgroups"; + case MemoryWorker::MemoryUsageSource::Jemalloc: return "Jemalloc"; + case MemoryWorker::MemoryUsageSource::None: return "None"; + } +} + +} + +/// We try to pick the best possible supported source for reading memory usage. +/// Supported sources in order of priority +/// - reading from cgroups' pseudo-files (fastest and most accurate) +/// - reading jemalloc's resident stat (doesn't take into account allocations that didn't use jemalloc) +/// Also, different tick rates are used because not all options are equally fast +MemoryWorker::MemoryWorker(uint64_t period_ms_) + : log(getLogger("MemoryWorker")) + , period_ms(period_ms_) +{ +#if defined(OS_LINUX) + try + { + cgroups_reader = createCgroupsReader(); + source = MemoryUsageSource::Cgroups; + if (period_ms == 0) + period_ms = cgroups_memory_usage_tick_ms; + + return; + } + catch (...) + { + tryLogCurrentException(log, "Cannot use cgroups reader"); + } +#endif + +#if USE_JEMALLOC + source = MemoryUsageSource::Jemalloc; + if (period_ms == 0) + period_ms = jemalloc_memory_usage_tick_ms; +#endif +} + +MemoryWorker::MemoryUsageSource MemoryWorker::getSource() +{ + return source; +} + +void MemoryWorker::start() +{ + if (source == MemoryUsageSource::None) + return; + + LOG_INFO( + getLogger("MemoryWorker"), + "Starting background memory thread with period of {}ms, using {} as source", + period_ms, + sourceToString(source)); background_thread = ThreadFromGlobalPool([this] { backgroundThread(); }); } @@ -40,29 +263,39 @@ MemoryWorker::~MemoryWorker() background_thread.join(); } +uint64_t MemoryWorker::getMemoryUsage() +{ + switch (source) + { + case MemoryUsageSource::Cgroups: + return cgroups_reader->readMemoryUsage(); + case MemoryUsageSource::Jemalloc: + return resident_mib.getValue(); + case MemoryUsageSource::None: + throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Trying to fetch memory usage while no memory source can be used"); + } +} + void MemoryWorker::backgroundThread() { - JemallocMibCache epoch_mib("epoch"); - JemallocMibCache resident_mib("stats.resident"); - JemallocMibCache active_mib("stats.active"); - JemallocMibCache allocated_mib("stats.allocated"); - JemallocMibCache purge_mib("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge"); + std::chrono::milliseconds chrono_period_ms{period_ms}; bool first_run = true; std::unique_lock lock(mutex); while (true) { - cv.wait_for(lock, period_ms, [this] { return shutdown; }); + cv.wait_for(lock, chrono_period_ms, [this] { return shutdown; }); if (shutdown) return; Stopwatch total_watch; - epoch_mib.setValue(0); - Int64 resident = resident_mib.getValue(); - /// force update the allocated stat from jemalloc for the first run to cover the allocations we missed - /// during initialization - MemoryTracker::updateValues(resident, allocated_mib.getValue(), first_run); + if (source == MemoryUsageSource::Jemalloc) + epoch_mib.setValue(0); + Int64 resident = getMemoryUsage(); + MemoryTracker::updateRSS(resident); + +#if USE_JEMALLOC if (resident > total_memory_tracker.getHardLimit()) { Stopwatch purge_watch; @@ -71,12 +304,19 @@ void MemoryWorker::backgroundThread() ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, purge_watch.elapsedMicroseconds()); } + if (unlikely(first_run || total_memory_tracker.get() < 0)) + { + if (source != MemoryUsageSource::Jemalloc) + epoch_mib.setValue(0); + + MemoryTracker::updateAllocated(allocated_mib.getValue()); + } +#endif + ProfileEvents::increment(ProfileEvents::MemoryWorkerRun); ProfileEvents::increment(ProfileEvents::MemoryWorkerRunElapsedMicroseconds, total_watch.elapsedMicroseconds()); - first_run = false; } } -#endif } diff --git a/src/Common/MemoryWorker.h b/src/Common/MemoryWorker.h index 6c0a578aa61..6fde93d63ad 100644 --- a/src/Common/MemoryWorker.h +++ b/src/Common/MemoryWorker.h @@ -1,13 +1,14 @@ #pragma once +#include #include - -#include "config.h" +#include namespace DB { -#if USE_JEMALLOC +struct ICgroupsReader; + /// Correct MemoryTracker based on stats.resident read from jemalloc. /// This requires jemalloc built with --enable-stats which we use. /// The worker spawns a background thread which moves the jemalloc epoch (updates internal stats), @@ -19,8 +20,21 @@ class MemoryWorker public: explicit MemoryWorker(uint64_t period_ms_); + enum class MemoryUsageSource : uint8_t + { + None, + Cgroups, + Jemalloc + }; + + MemoryUsageSource getSource(); + + void start(); + ~MemoryWorker(); private: + uint64_t getMemoryUsage(); + void backgroundThread(); ThreadFromGlobalPool background_thread; @@ -29,14 +43,27 @@ private: std::condition_variable cv; bool shutdown = false; - std::chrono::milliseconds period_ms; -}; -#else -class MemoryWorker -{ -public: - explicit MemoryWorker(uint64_t /*period_ms_*/) {} -}; + LoggerPtr log; + + uint64_t period_ms; + + MemoryUsageSource source{MemoryUsageSource::None}; + +#if defined(OS_LINUX) + std::shared_ptr cgroups_reader; #endif +#if USE_JEMALLOC + JemallocMibCache epoch_mib{"epoch"}; + JemallocMibCache resident_mib{"stats.resident"}; + JemallocMibCache allocated_mib{"stats.allocated"}; + +#define STRINGIFY_HELPER(x) #x +#define STRINGIFY(x) STRINGIFY_HELPER(x) + JemallocMibCache purge_mib{"arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge"}; +#undef STRINGIFY +#undef STRINGIFY_HELPER +#endif +}; + } diff --git a/src/Coordination/KeeperAsynchronousMetrics.cpp b/src/Coordination/KeeperAsynchronousMetrics.cpp index 86166ffe31b..157858f3c44 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.cpp +++ b/src/Coordination/KeeperAsynchronousMetrics.cpp @@ -114,8 +114,13 @@ void updateKeeperInformation(KeeperDispatcher & keeper_dispatcher, AsynchronousM } KeeperAsynchronousMetrics::KeeperAsynchronousMetrics( - ContextPtr context_, unsigned update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_) - : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_), context(std::move(context_)) + ContextPtr context_, + unsigned update_period_seconds, + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_) + : AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, update_jemalloc_epoch_, update_rss_) + , context(std::move(context_)) { } diff --git a/src/Coordination/KeeperAsynchronousMetrics.h b/src/Coordination/KeeperAsynchronousMetrics.h index ec0e60cbb6e..a2ab7cab756 100644 --- a/src/Coordination/KeeperAsynchronousMetrics.h +++ b/src/Coordination/KeeperAsynchronousMetrics.h @@ -13,9 +13,13 @@ class KeeperAsynchronousMetrics : public AsynchronousMetrics { public: KeeperAsynchronousMetrics( - ContextPtr context_, unsigned update_period_seconds, const ProtocolServerMetricsFunc & protocol_server_metrics_func_); - ~KeeperAsynchronousMetrics() override; + ContextPtr context_, + unsigned update_period_seconds, + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_); + ~KeeperAsynchronousMetrics() override; private: ContextPtr context; diff --git a/src/Core/ServerSettings.h b/src/Core/ServerSettings.h index aaea0388239..ea5a3f19638 100644 --- a/src/Core/ServerSettings.h +++ b/src/Core/ServerSettings.h @@ -157,7 +157,7 @@ namespace DB M(Bool, prepare_system_log_tables_on_startup, false, "If true, ClickHouse creates all configured `system.*_log` tables before the startup. It can be helpful if some startup scripts depend on these tables.", 0) \ M(Double, gwp_asan_force_sample_probability, 0.0003, "Probability that an allocation from specific places will be sampled by GWP Asan (i.e. PODArray allocations)", 0) \ M(UInt64, config_reload_interval_ms, 2000, "How often clickhouse will reload config and check for new changes", 0) \ - M(UInt64, memory_worker_period_ms, 100, "Period of background memory worker which corrects memory tracker memory usages and cleans up unused pages during higher memory usage.", 0) \ + M(UInt64, memory_worker_period_ms, 0, "Tick period of background memory worker which corrects memory tracker memory usages and cleans up unused pages during higher memory usage. If set to 0, default value will be used depending on the memory usage source", 0) \ /// If you add a setting which can be updated at runtime, please update 'changeable_settings' map in StorageSystemServerSettings.cpp diff --git a/src/Interpreters/ServerAsynchronousMetrics.cpp b/src/Interpreters/ServerAsynchronousMetrics.cpp index 872a9f864df..079029695c9 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.cpp +++ b/src/Interpreters/ServerAsynchronousMetrics.cpp @@ -55,9 +55,11 @@ ServerAsynchronousMetrics::ServerAsynchronousMetrics( ContextPtr global_context_, unsigned update_period_seconds, unsigned heavy_metrics_update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_) + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_) : WithContext(global_context_) - , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_) + , AsynchronousMetrics(update_period_seconds, protocol_server_metrics_func_, update_jemalloc_epoch_, update_rss_) , heavy_metric_update_period(heavy_metrics_update_period_seconds) { /// sanity check diff --git a/src/Interpreters/ServerAsynchronousMetrics.h b/src/Interpreters/ServerAsynchronousMetrics.h index e3c83dc748e..5fab419a32b 100644 --- a/src/Interpreters/ServerAsynchronousMetrics.h +++ b/src/Interpreters/ServerAsynchronousMetrics.h @@ -14,7 +14,10 @@ public: ContextPtr global_context_, unsigned update_period_seconds, unsigned heavy_metrics_update_period_seconds, - const ProtocolServerMetricsFunc & protocol_server_metrics_func_); + const ProtocolServerMetricsFunc & protocol_server_metrics_func_, + bool update_jemalloc_epoch_, + bool update_rss_); + ~ServerAsynchronousMetrics() override; private: From 5a1b96ac8453f73de5e891e3a9f235fd96270b50 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 23 Jul 2024 10:52:14 +0200 Subject: [PATCH 086/162] Style fix --- src/Common/CgroupsMemoryUsageObserver.cpp | 6 ------ src/Common/MemoryWorker.cpp | 2 +- 2 files changed, 1 insertion(+), 7 deletions(-) diff --git a/src/Common/CgroupsMemoryUsageObserver.cpp b/src/Common/CgroupsMemoryUsageObserver.cpp index afeac1808b2..16d5d1cccde 100644 --- a/src/Common/CgroupsMemoryUsageObserver.cpp +++ b/src/Common/CgroupsMemoryUsageObserver.cpp @@ -19,12 +19,6 @@ using namespace DB; namespace DB { -namespace ErrorCodes -{ -extern const int FILE_DOESNT_EXIST; -extern const int INCORRECT_DATA; -} - CgroupsMemoryUsageObserver::CgroupsMemoryUsageObserver(std::chrono::seconds wait_time_) : log(getLogger("CgroupsMemoryUsageObserver")), wait_time(wait_time_) {} diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index 42e797a80d6..ddc3fd783f4 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -30,7 +30,7 @@ namespace DB namespace ErrorCodes { extern const int FILE_DOESNT_EXIST; - extern const int INCORRECT_DATA; + extern const int LOGICAL_ERROR; } #if defined(OS_LINUX) From 5b51a35e015336227dcc5b25445fefbc3da3059c Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 23 Jul 2024 11:31:34 +0200 Subject: [PATCH 087/162] Add unused variables --- src/Common/AsynchronousMetrics.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/AsynchronousMetrics.h b/src/Common/AsynchronousMetrics.h index eb6ede7a558..fedba4e55be 100644 --- a/src/Common/AsynchronousMetrics.h +++ b/src/Common/AsynchronousMetrics.h @@ -112,8 +112,8 @@ private: MemoryStatisticsOS memory_stat TSA_GUARDED_BY(data_mutex); #endif - const bool update_jemalloc_epoch; - const bool update_rss; + [[maybe_unused]] const bool update_jemalloc_epoch; + [[maybe_unused]] const bool update_rss; #if defined(OS_LINUX) std::optional meminfo TSA_GUARDED_BY(data_mutex); From ade79cfd7a8465bf6e332f3b2ca6143ba652251a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 23 Jul 2024 11:50:56 +0200 Subject: [PATCH 088/162] More fixes --- src/Common/MemoryWorker.cpp | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index ddc3fd783f4..8169bc7d177 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -186,9 +186,6 @@ std::shared_ptr createCgroupsReader() } #endif -constexpr uint64_t cgroups_memory_usage_tick_ms{50}; -constexpr uint64_t jemalloc_memory_usage_tick_ms{100}; - std::string_view sourceToString(MemoryWorker::MemoryUsageSource source) { switch (source) @@ -213,6 +210,8 @@ MemoryWorker::MemoryWorker(uint64_t period_ms_) #if defined(OS_LINUX) try { + static constexpr uint64_t cgroups_memory_usage_tick_ms{50}; + cgroups_reader = createCgroupsReader(); source = MemoryUsageSource::Cgroups; if (period_ms == 0) @@ -227,6 +226,8 @@ MemoryWorker::MemoryWorker(uint64_t period_ms_) #endif #if USE_JEMALLOC + static constexpr uint64_t jemalloc_memory_usage_tick_ms{100}; + source = MemoryUsageSource::Jemalloc; if (period_ms == 0) period_ms = jemalloc_memory_usage_tick_ms; @@ -270,7 +271,11 @@ uint64_t MemoryWorker::getMemoryUsage() case MemoryUsageSource::Cgroups: return cgroups_reader->readMemoryUsage(); case MemoryUsageSource::Jemalloc: +#if USE_JEMALLOC return resident_mib.getValue(); +#else + return 0; +#endif case MemoryUsageSource::None: throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Trying to fetch memory usage while no memory source can be used"); } @@ -279,7 +284,7 @@ uint64_t MemoryWorker::getMemoryUsage() void MemoryWorker::backgroundThread() { std::chrono::milliseconds chrono_period_ms{period_ms}; - bool first_run = true; + [[maybe_unused]] bool first_run = true; std::unique_lock lock(mutex); while (true) { @@ -289,8 +294,10 @@ void MemoryWorker::backgroundThread() Stopwatch total_watch; +#if USE_JEMALLOC if (source == MemoryUsageSource::Jemalloc) epoch_mib.setValue(0); +#endif Int64 resident = getMemoryUsage(); MemoryTracker::updateRSS(resident); From 842b51c782f0cc5cf9bd7b695c466933310a7e6c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 Jul 2024 13:30:40 +0000 Subject: [PATCH 089/162] Try assigning replica numbers consistently independent of initiator --- .../ClusterProxy/executeQuery.cpp | 67 +++++++++++- .../QueryPlan/ParallelReplicasLocalPlan.cpp | 7 +- .../QueryPlan/ParallelReplicasLocalPlan.h | 3 +- .../QueryPlan/ReadFromMergeTree.cpp | 8 +- src/Processors/QueryPlan/ReadFromMergeTree.h | 3 +- src/Processors/QueryPlan/ReadFromRemote.cpp | 102 ++++-------------- src/Processors/QueryPlan/ReadFromRemote.h | 6 +- .../ParallelReplicasReadingCoordinator.cpp | 3 +- 8 files changed, 101 insertions(+), 98 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index a8cf3022a61..80f6d7a864b 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -40,6 +40,7 @@ namespace ErrorCodes extern const int TOO_LARGE_DISTRIBUTED_DEPTH; extern const int LOGICAL_ERROR; extern const int UNEXPECTED_CLUSTER; + extern const int INCONSISTENT_CLUSTER_DEFINITION; } namespace ClusterProxy @@ -519,22 +520,75 @@ void executeQueryWithParallelReplicas( "`cluster_for_parallel_replicas` setting refers to cluster with several shards. Expected a cluster with one shard"); } - const auto replica_count = std::min(settings.max_parallel_replicas.value, new_cluster->getShardsInfo().begin()->getAllNodeCount()); + const auto & shard = new_cluster->getShardsInfo().at(0); + size_t max_replicas_to_use = settings.max_parallel_replicas; + if (max_replicas_to_use > shard.getAllNodeCount()) + { + LOG_INFO( + getLogger("ReadFromParallelRemoteReplicasStep"), + "The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). " + "Will use the latter number to execute the query.", + settings.max_parallel_replicas, + shard.getAllNodeCount()); + max_replicas_to_use = shard.getAllNodeCount(); + } - auto coordinator = std::make_shared(replica_count, settings.parallel_replicas_mark_segment_size); + auto coordinator = std::make_shared(max_replicas_to_use, settings.parallel_replicas_mark_segment_size); auto external_tables = new_context->getExternalTables(); + std::vector shuffled_pool; + if (max_replicas_to_use < shard.getAllNodeCount()) + { + shuffled_pool = shard.pool->getShuffledPools(settings); + shuffled_pool.resize(max_replicas_to_use); + } + else + { + /// if all replicas in cluster are used for query execution + /// try to preserve replicas order as in cluster definition + /// it's important for data locality during query execution + auto priority_func = [](size_t i) { return Priority{static_cast(i)}; }; + shuffled_pool = shard.pool->getShuffledPools(settings, priority_func); + } + + std::vector pools_to_use; + pools_to_use.reserve(shuffled_pool.size()); + for (auto & pool : shuffled_pool) + pools_to_use.emplace_back(std::move(pool.pool)); + /// do not build local plan for distributed queries for now (address it later) if (settings.allow_experimental_analyzer && settings.parallel_replicas_local_plan && !shard_num) { + /// find local replica index in pool, to assign it as replica number + std::optional local_replica_number; + for (size_t i = 0, s = pools_to_use.size(); i < s; ++i) + { + const auto & hostname = pools_to_use[i]->getHost(); + const auto found = std::find_if( + begin(shard.local_addresses), + end(shard.local_addresses), + [&hostname](const Cluster::Address & local_addr) { return hostname == local_addr.host_name; }); + if (found != shard.local_addresses.end()) + { + local_replica_number = i; + break; + } + } + if (!local_replica_number) + throw Exception( + ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION, + "Local replica is not found in '{}' cluster definition, see 'cluster_for_parallel_replicas' setting", + new_cluster->getName()); + auto [local_plan, with_parallel_replicas] = createLocalPlanForParallelReplicas( query_ast, header, new_context, processed_stage, coordinator, - std::move(analyzed_read_from_merge_tree)); + std::move(analyzed_read_from_merge_tree), + local_replica_number.value()); if (!with_parallel_replicas) { @@ -542,6 +596,8 @@ void executeQueryWithParallelReplicas( return; } + LOG_DEBUG(logger, "Local replica got replica number {}", local_replica_number.value()); + auto read_from_remote = std::make_unique( query_ast, new_cluster, @@ -555,7 +611,8 @@ void executeQueryWithParallelReplicas( std::move(external_tables), getLogger("ReadFromParallelRemoteReplicasStep"), std::move(storage_limits), - /*exclude_local_replica*/ true); + std::move(pools_to_use), + local_replica_number); auto remote_plan = std::make_unique(); remote_plan->addStep(std::move(read_from_remote)); @@ -587,7 +644,7 @@ void executeQueryWithParallelReplicas( std::move(external_tables), getLogger("ReadFromParallelRemoteReplicasStep"), std::move(storage_limits), - /*exclude_local_replica*/ false); + std::move(pools_to_use)); query_plan.addStep(std::move(read_from_remote)); } diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp index e8ff0f417dc..050044edd3a 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.cpp @@ -27,7 +27,8 @@ std::pair, bool> createLocalPlanForParallelReplicas( ContextPtr context, QueryProcessingStage::Enum processed_stage, ParallelReplicasReadingCoordinatorPtr coordinator, - QueryPlanStepPtr analyzed_read_from_merge_tree) + QueryPlanStepPtr analyzed_read_from_merge_tree, + size_t replica_number) { checkStackSize(); @@ -84,8 +85,8 @@ std::pair, bool> createLocalPlanForParallelReplicas( MergeTreeReadTaskCallback read_task_cb = [coordinator](ParallelReadRequest req) -> std::optional { return coordinator->handleRequest(std::move(req)); }; - auto read_from_merge_tree_parallel_replicas - = reading->createLocalParallelReplicasReadingStep(analyzed_result_ptr, std::move(all_ranges_cb), std::move(read_task_cb)); + auto read_from_merge_tree_parallel_replicas = reading->createLocalParallelReplicasReadingStep( + analyzed_result_ptr, std::move(all_ranges_cb), std::move(read_task_cb), replica_number); node->step = std::move(read_from_merge_tree_parallel_replicas); addConvertingActions(*query_plan, header, /*has_missing_objects=*/false); diff --git a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h index 2a49be6347a..ab0bbeaeeff 100644 --- a/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h +++ b/src/Processors/QueryPlan/ParallelReplicasLocalPlan.h @@ -14,5 +14,6 @@ std::pair, bool> createLocalPlanForParallelReplicas( ContextPtr context, QueryProcessingStage::Enum processed_stage, ParallelReplicasReadingCoordinatorPtr coordinator, - QueryPlanStepPtr read_from_merge_tree); + QueryPlanStepPtr read_from_merge_tree, + size_t replica_number); } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 5922550eaf3..aea9d02fe01 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -348,9 +348,9 @@ ReadFromMergeTree::ReadFromMergeTree( std::unique_ptr ReadFromMergeTree::createLocalParallelReplicasReadingStep( AnalysisResultPtr analyzed_result_ptr_, MergeTreeAllRangesCallback all_ranges_callback_, - MergeTreeReadTaskCallback read_task_callback_) + MergeTreeReadTaskCallback read_task_callback_, + size_t replica_number) { - const auto number_of_local_replica = 0; const bool enable_parallel_reading = true; return std::make_unique( prepared_parts, @@ -364,11 +364,11 @@ std::unique_ptr ReadFromMergeTree::createLocalParallelReplica requested_num_streams, max_block_numbers_to_read, log, - analyzed_result_ptr_, + std::move(analyzed_result_ptr_), enable_parallel_reading, all_ranges_callback_, read_task_callback_, - number_of_local_replica); + replica_number); } Pipe ReadFromMergeTree::readFromPoolParallelReplicas(RangesInDataParts parts_with_range, Names required_columns, PoolSettings pool_settings) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.h b/src/Processors/QueryPlan/ReadFromMergeTree.h index 307b605c01c..69e20a7864d 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.h +++ b/src/Processors/QueryPlan/ReadFromMergeTree.h @@ -129,7 +129,8 @@ public: std::unique_ptr createLocalParallelReplicasReadingStep( AnalysisResultPtr analyzed_result_ptr_, MergeTreeAllRangesCallback all_ranges_callback_, - MergeTreeReadTaskCallback read_task_callback_); + MergeTreeReadTaskCallback read_task_callback_, + size_t replica_number); static constexpr auto name = "ReadFromMergeTree"; String getName() const override { return name; } diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index 46d4aa29e70..3df46eb1987 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -371,7 +371,8 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( Tables external_tables_, LoggerPtr log_, std::shared_ptr storage_limits_, - bool exclude_local_replica_) + std::vector pools_to_use_, + std::optional exclude_pool_index_) : ISourceStep(DataStream{.header = std::move(header_)}) , cluster(cluster_) , query_ast(query_ast_) @@ -384,24 +385,20 @@ ReadFromParallelRemoteReplicasStep::ReadFromParallelRemoteReplicasStep( , external_tables{external_tables_} , storage_limits(std::move(storage_limits_)) , log(log_) - , exclude_local_replica(exclude_local_replica_) + , pools_to_use(std::move(pools_to_use_)) + , exclude_pool_index(exclude_pool_index_) { chassert(cluster->getShardCount() == 1); std::vector replicas; - replicas.reserve(cluster->getShardsAddresses().front().size()); + replicas.reserve(pools_to_use.size()); - for (const auto & addr : cluster->getShardsAddresses().front()) + for (size_t i = 0, l = pools_to_use.size(); i < l; ++i) { - if (exclude_local_replica && addr.is_local) + if (exclude_pool_index.has_value() && i == exclude_pool_index) continue; - /// replace hostname with replica name if the hostname started with replica namespace, - /// it makes description shorter and more readable - if (!addr.database_replica_name.empty() && addr.host_name.starts_with(addr.database_replica_name)) - replicas.push_back(fmt::format("{}", addr.database_replica_name)); - else - replicas.push_back(fmt::format("{}", addr.host_name)); + replicas.push_back(pools_to_use[i]->getAddress()); } auto description = fmt::format("Query: {} Replicas: {}", formattedAST(query_ast), fmt::join(replicas, ", ")); @@ -421,86 +418,29 @@ void ReadFromParallelRemoteReplicasStep::enforceAggregationInOrder() void ReadFromParallelRemoteReplicasStep::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) { Pipes pipes; - const Settings & current_settings = context->getSettingsRef(); - auto timeouts = ConnectionTimeouts::getTCPTimeoutsWithFailover(current_settings); - - const auto & shard = cluster->getShardsInfo().at(0); - size_t max_replicas_to_use = current_settings.max_parallel_replicas; - if (max_replicas_to_use > shard.getAllNodeCount()) - { - LOG_INFO( - getLogger("ReadFromParallelRemoteReplicasStep"), - "The number of replicas requested ({}) is bigger than the real number available in the cluster ({}). " - "Will use the latter number to execute the query.", - current_settings.max_parallel_replicas, - shard.getAllNodeCount()); - max_replicas_to_use = shard.getAllNodeCount(); - } - - std::vector shuffled_pool; - if (max_replicas_to_use < shard.getAllNodeCount()) - { - shuffled_pool = shard.pool->getShuffledPools(current_settings); - } - else - { - /// try to preserve replicas order if all replicas in cluster are used for query execution - /// it's important for data locality during query execution - auto priority_func = [](size_t i) { return Priority{static_cast(i)}; }; - shuffled_pool = shard.pool->getShuffledPools(current_settings, priority_func); - } - - std::vector pools_to_use; - pools_to_use.reserve(shuffled_pool.size()); - for (const auto & pool : shuffled_pool) - { - if (exclude_local_replica) - { - const auto & hostname = pool.pool->getHost(); - auto it = std::find_if( - begin(shard.local_addresses), - end(shard.local_addresses), - [&hostname](const Cluster::Address & local_addr) { return hostname == local_addr.host_name; }); - if (it == shard.local_addresses.end()) - pools_to_use.push_back(pool.pool); - } - else - { - pools_to_use.push_back(pool.pool); - } - } - - pools_to_use.resize(std::min(pools_to_use.size(), max_replicas_to_use)); - // if local plan is used for local replica, we should exclude one remote replica - if (exclude_local_replica && !pools_to_use.empty()) - pools_to_use.resize(max_replicas_to_use - 1); - - LOG_DEBUG( - getLogger("ReadFromParallelRemoteReplicasStep"), - "Number of pools to use is {}. Originally {}", - pools_to_use.size(), - shuffled_pool.size()); - - if (pools_to_use.empty()) - return; std::vector addresses; addresses.reserve(pools_to_use.size()); - for (const auto & pool : pools_to_use) - addresses.emplace_back(pool->getAddress()); + for (size_t i = 0, l = pools_to_use.size(); i < l; ++i) + { + if (exclude_pool_index.has_value() && i == exclude_pool_index) + continue; + + addresses.emplace_back(pools_to_use[i]->getAddress()); + } LOG_DEBUG(getLogger("ReadFromParallelRemoteReplicasStep"), "Addresses to use: {}", fmt::join(addresses, ", ")); - /// when using local plan for local replica, 0 is assigned to local replica as replica num, - in this case, starting from 1 here - size_t replica_num = (exclude_local_replica ? 1 : 0); - for (const auto & pool : pools_to_use) + for (size_t i = 0, l = pools_to_use.size(); i < l; ++i) { + if (exclude_pool_index.has_value() && i == exclude_pool_index) + continue; + IConnections::ReplicaInfo replica_info{ /// we should use this number specifically because efficiency of data distribution by consistent hash depends on it. - .number_of_current_replica = replica_num, + .number_of_current_replica = i, }; - ++replica_num; - addPipeForSingeReplica(pipes, pool, replica_info); + addPipeForSingeReplica(pipes, pools_to_use[i], replica_info); } auto pipe = Pipe::unitePipes(std::move(pipes)); diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index 442da098a17..74389c8f9eb 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -79,7 +79,8 @@ public: Tables external_tables_, LoggerPtr log_, std::shared_ptr storage_limits_, - bool exclude_local_replica = false); + std::vector pools_to_use, + std::optional exclude_pool_index_ = std::nullopt); String getName() const override { return "ReadFromRemoteParallelReplicas"; } @@ -102,7 +103,8 @@ private: Tables external_tables; std::shared_ptr storage_limits; LoggerPtr log; - bool exclude_local_replica; + std::vector pools_to_use; + std::optional exclude_pool_index; }; } diff --git a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp index a1b36fe1299..a1264eabdea 100644 --- a/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp +++ b/src/Storages/MergeTree/ParallelReplicasReadingCoordinator.cpp @@ -185,7 +185,8 @@ public: void handleInitialAllRangesAnnouncement(InitialAllRangesAnnouncement announcement) { if (++sent_initial_requests > replicas_count) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Initiator received more initial requests than there are replicas"); + throw Exception( + ErrorCodes::LOGICAL_ERROR, "Initiator received more initial requests than there are replicas: replica_num={}", announcement.replica_num); doHandleInitialAllRangesAnnouncement(std::move(announcement)); } From 04d80ec2763a0677f32e9f6190cf2bcda0ebf8b7 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 23 Jul 2024 16:13:07 +0200 Subject: [PATCH 090/162] Fix non-linux build --- src/Common/MemoryWorker.cpp | 4 ++-- src/Common/MemoryWorker.h | 2 -- 2 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index 8169bc7d177..c576772d303 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -33,7 +33,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -#if defined(OS_LINUX) struct ICgroupsReader { virtual ~ICgroupsReader() = default; @@ -44,6 +43,7 @@ struct ICgroupsReader namespace { +#if defined(OS_LINUX) /// Format is /// kernel 5 /// rss 15 @@ -269,7 +269,7 @@ uint64_t MemoryWorker::getMemoryUsage() switch (source) { case MemoryUsageSource::Cgroups: - return cgroups_reader->readMemoryUsage(); + return cgroups_reader != nullptr ? cgroups_reader->readMemoryUsage() : 0; case MemoryUsageSource::Jemalloc: #if USE_JEMALLOC return resident_mib.getValue(); diff --git a/src/Common/MemoryWorker.h b/src/Common/MemoryWorker.h index 6fde93d63ad..b1b0495bf14 100644 --- a/src/Common/MemoryWorker.h +++ b/src/Common/MemoryWorker.h @@ -49,9 +49,7 @@ private: MemoryUsageSource source{MemoryUsageSource::None}; -#if defined(OS_LINUX) std::shared_ptr cgroups_reader; -#endif #if USE_JEMALLOC JemallocMibCache epoch_mib{"epoch"}; From f5630f86e4853c06db1267628e4a8c0e80d62889 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 Jul 2024 15:43:25 +0000 Subject: [PATCH 091/162] Fix: resize pools_to_use correctly (keep local replica in it) --- .../ClusterProxy/executeQuery.cpp | 35 +++++++++++++------ 1 file changed, 24 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 80f6d7a864b..44474a11b50 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -540,14 +540,15 @@ void executeQueryWithParallelReplicas( std::vector shuffled_pool; if (max_replicas_to_use < shard.getAllNodeCount()) { + // will be shuffled according to `load_balancing` setting shuffled_pool = shard.pool->getShuffledPools(settings); - shuffled_pool.resize(max_replicas_to_use); } else { - /// if all replicas in cluster are used for query execution - /// try to preserve replicas order as in cluster definition - /// it's important for data locality during query execution + /// If all replicas in cluster are used for query execution, + /// try to preserve replicas order as in cluster definition. + /// It's important for data locality during query execution + /// independently of the query initiator auto priority_func = [](size_t i) { return Priority{static_cast(i)}; }; shuffled_pool = shard.pool->getShuffledPools(settings, priority_func); } @@ -560,8 +561,8 @@ void executeQueryWithParallelReplicas( /// do not build local plan for distributed queries for now (address it later) if (settings.allow_experimental_analyzer && settings.parallel_replicas_local_plan && !shard_num) { - /// find local replica index in pool, to assign it as replica number - std::optional local_replica_number; + /// find local replica index in pool + std::optional local_replica_index; for (size_t i = 0, s = pools_to_use.size(); i < s; ++i) { const auto & hostname = pools_to_use[i]->getHost(); @@ -571,16 +572,25 @@ void executeQueryWithParallelReplicas( [&hostname](const Cluster::Address & local_addr) { return hostname == local_addr.host_name; }); if (found != shard.local_addresses.end()) { - local_replica_number = i; + local_replica_index = i; break; } } - if (!local_replica_number) + if (!local_replica_index) throw Exception( ErrorCodes::INCONSISTENT_CLUSTER_DEFINITION, "Local replica is not found in '{}' cluster definition, see 'cluster_for_parallel_replicas' setting", new_cluster->getName()); + // resize the pool but keep local replicas in it (and udpate its index) + chassert(max_replicas_to_use <= pools_to_use.size()); + if (local_replica_index >= max_replicas_to_use) + { + std::swap(pools_to_use[max_replicas_to_use - 1], pools_to_use[local_replica_index.value()]); + local_replica_index = max_replicas_to_use - 1; + } + pools_to_use.resize(max_replicas_to_use); + auto [local_plan, with_parallel_replicas] = createLocalPlanForParallelReplicas( query_ast, header, @@ -588,7 +598,7 @@ void executeQueryWithParallelReplicas( processed_stage, coordinator, std::move(analyzed_read_from_merge_tree), - local_replica_number.value()); + local_replica_index.value()); if (!with_parallel_replicas) { @@ -596,7 +606,7 @@ void executeQueryWithParallelReplicas( return; } - LOG_DEBUG(logger, "Local replica got replica number {}", local_replica_number.value()); + LOG_DEBUG(logger, "Local replica got replica number {}", local_replica_index.value()); auto read_from_remote = std::make_unique( query_ast, @@ -612,7 +622,7 @@ void executeQueryWithParallelReplicas( getLogger("ReadFromParallelRemoteReplicasStep"), std::move(storage_limits), std::move(pools_to_use), - local_replica_number); + local_replica_index); auto remote_plan = std::make_unique(); remote_plan->addStep(std::move(read_from_remote)); @@ -631,6 +641,9 @@ void executeQueryWithParallelReplicas( } else { + chassert(max_replicas_to_use <= pools_to_use.size()); + pools_to_use.resize(max_replicas_to_use); + auto read_from_remote = std::make_unique( query_ast, new_cluster, From 87ad12e5fceec5c412964fbf0a6336a4d3da7662 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 Jul 2024 17:16:11 +0000 Subject: [PATCH 092/162] Fix typo --- src/Interpreters/ClusterProxy/executeQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/ClusterProxy/executeQuery.cpp b/src/Interpreters/ClusterProxy/executeQuery.cpp index 44474a11b50..444a872f491 100644 --- a/src/Interpreters/ClusterProxy/executeQuery.cpp +++ b/src/Interpreters/ClusterProxy/executeQuery.cpp @@ -582,7 +582,7 @@ void executeQueryWithParallelReplicas( "Local replica is not found in '{}' cluster definition, see 'cluster_for_parallel_replicas' setting", new_cluster->getName()); - // resize the pool but keep local replicas in it (and udpate its index) + // resize the pool but keep local replicas in it (and update its index) chassert(max_replicas_to_use <= pools_to_use.size()); if (local_replica_index >= max_replicas_to_use) { From f49412e967ec549d8a49e53ce3bfba27e13994fc Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 23 Jul 2024 17:16:40 +0000 Subject: [PATCH 093/162] Change local replica position in `parallel_replicas` cluster --- tests/config/config.d/clusters.xml | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/config/config.d/clusters.xml b/tests/config/config.d/clusters.xml index af26565d7e6..ff42b5828e3 100644 --- a/tests/config/config.d/clusters.xml +++ b/tests/config/config.d/clusters.xml @@ -115,10 +115,6 @@ false - - 127.0.0.1 - 9000 - 127.0.0.2 9000 @@ -147,6 +143,10 @@ 127.0.0.8 9000 + + 127.0.0.1 + 9000 + 127.0.0.9 9000 From ec5459a60d0be15e7bc100bd257e2b5c65baf594 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Jul 2024 09:15:34 +0200 Subject: [PATCH 094/162] Update allocated with resident if no jemalloc --- src/Common/MemoryWorker.cpp | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index c576772d303..75d0e7c32d8 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -310,15 +310,19 @@ void MemoryWorker::backgroundThread() ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, purge_watch.elapsedMicroseconds()); } +#endif if (unlikely(first_run || total_memory_tracker.get() < 0)) { +#if USE_JEMALLOC if (source != MemoryUsageSource::Jemalloc) epoch_mib.setValue(0); MemoryTracker::updateAllocated(allocated_mib.getValue()); - } +#elif defined(OS_LINUX) + MemoryTracker::updateAllocated(resident); #endif + } ProfileEvents::increment(ProfileEvents::MemoryWorkerRun); ProfileEvents::increment(ProfileEvents::MemoryWorkerRunElapsedMicroseconds, total_watch.elapsedMicroseconds()); From e2e4c8ee0f8fcf1d6ef4d566c2fa4f9ee2123a56 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Wed, 24 Jul 2024 10:21:09 +0200 Subject: [PATCH 095/162] Better --- src/Common/MemoryWorker.cpp | 22 ++++++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index 84ccffb8e90..e148f7f8f49 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -65,11 +65,25 @@ Metrics readAllMetricsFromStatFile(ReadBufferFromFile & buf) return metrics; } -uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, const std::string & key) +uint64_t readMetricFromStatFile(ReadBufferFromFile & buf, std::string_view key) { - const auto all_metrics = readAllMetricsFromStatFile(buf); - if (const auto it = all_metrics.find(key); it != all_metrics.end()) - return it->second; + while (!buf.eof()) + { + std::string current_key; + readStringUntilWhitespace(current_key, buf); + if (current_key != key) + { + std::string dummy; + readStringUntilNewlineInto(dummy, buf); + buf.ignore(); + continue; + } + + assertChar(' ', buf); + uint64_t value = 0; + readIntText(value, buf); + return value; + } LOG_ERROR(getLogger("CgroupsReader"), "Cannot find '{}' in '{}'", key, buf.getFileName()); return 0; } From f449c2fea0487abbe262c10fc5af9a99df1bc822 Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Thu, 25 Jul 2024 08:45:08 +0200 Subject: [PATCH 096/162] Fix --- src/Common/MemoryTracker.h | 5 +++++ src/Common/MemoryWorker.cpp | 8 +++----- src/Coordination/KeeperDispatcher.cpp | 3 ++- src/Coordination/KeeperServer.cpp | 2 +- 4 files changed, 11 insertions(+), 7 deletions(-) diff --git a/src/Common/MemoryTracker.h b/src/Common/MemoryTracker.h index d2db8489f19..f15465a20c1 100644 --- a/src/Common/MemoryTracker.h +++ b/src/Common/MemoryTracker.h @@ -120,6 +120,11 @@ public: return amount.load(std::memory_order_relaxed); } + Int64 getRSS() const + { + return rss.load(std::memory_order_relaxed); + } + // Merges and mutations may pass memory ownership to other threads thus in the end of execution // MemoryTracker for background task may have a non-zero counter. // This method is intended to fix the counter inside of background_memory_tracker. diff --git a/src/Common/MemoryWorker.cpp b/src/Common/MemoryWorker.cpp index e148f7f8f49..1b869ed9d6b 100644 --- a/src/Common/MemoryWorker.cpp +++ b/src/Common/MemoryWorker.cpp @@ -312,7 +312,7 @@ uint64_t MemoryWorker::getMemoryUsage() void MemoryWorker::backgroundThread() { std::chrono::milliseconds chrono_period_ms{period_ms}; - bool first_run = true; + [[maybe_unused]] bool first_run = true; std::unique_lock lock(mutex); while (true) { @@ -340,17 +340,15 @@ void MemoryWorker::backgroundThread() } #endif +#if USE_JEMALLOC if (unlikely(first_run || total_memory_tracker.get() < 0)) { -#if USE_JEMALLOC if (source != MemoryUsageSource::Jemalloc) epoch_mib.setValue(0); MemoryTracker::updateAllocated(allocated_mib.getValue()); -#elif defined(OS_LINUX) - MemoryTracker::updateAllocated(resident); -#endif } +#endif ProfileEvents::increment(ProfileEvents::MemoryWorkerRun); ProfileEvents::increment(ProfileEvents::MemoryWorkerRunElapsedMicroseconds, total_watch.elapsedMicroseconds()); diff --git a/src/Coordination/KeeperDispatcher.cpp b/src/Coordination/KeeperDispatcher.cpp index 4c2ccb8db64..893bb8e6082 100644 --- a/src/Coordination/KeeperDispatcher.cpp +++ b/src/Coordination/KeeperDispatcher.cpp @@ -150,10 +150,11 @@ void KeeperDispatcher::requestThread() { LOG_WARNING( log, - "Processing requests refused because of max_memory_usage_soft_limit {}, the total used memory is {}, request type " + "Processing requests refused because of max_memory_usage_soft_limit {}, the total allocated memory is {}, RSS is {}, request type " "is {}", ReadableSize(mem_soft_limit), ReadableSize(total_memory_tracker.get()), + ReadableSize(total_memory_tracker.getRSS()), request.request->getOpNum()); addErrorResponses({request}, Coordination::Error::ZCONNECTIONLOSS); continue; diff --git a/src/Coordination/KeeperServer.cpp b/src/Coordination/KeeperServer.cpp index d40e5ef2e50..ad9e8d32caa 100644 --- a/src/Coordination/KeeperServer.cpp +++ b/src/Coordination/KeeperServer.cpp @@ -599,7 +599,7 @@ bool KeeperServer::isLeaderAlive() const bool KeeperServer::isExceedingMemorySoftLimit() const { Int64 mem_soft_limit = keeper_context->getKeeperMemorySoftLimit(); - return mem_soft_limit > 0 && total_memory_tracker.get() >= mem_soft_limit; + return mem_soft_limit > 0 && std::max(total_memory_tracker.get(), total_memory_tracker.getRSS()) >= mem_soft_limit; } /// TODO test whether taking failed peer in count From ce134830b702b117724235de28afdea2b5db0dfc Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 29 Jul 2024 17:15:15 +0000 Subject: [PATCH 097/162] Disable pr local plan by default --- src/Core/Settings.h | 2 +- src/Core/SettingsChangesHistory.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 4a5dec8dcc8..e175d7f76af 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -934,7 +934,7 @@ class IColumn; M(UInt64, parallel_replicas_min_number_of_rows_per_replica, 0, "Limit the number of replicas used in a query to (estimated rows to read / min_number_of_rows_per_replica). The max is still limited by 'max_parallel_replicas'", 0) \ M(Bool, parallel_replicas_prefer_local_join, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN.", 0) \ M(UInt64, parallel_replicas_mark_segment_size, 128, "Parts virtually divided into segments to be distributed between replicas for parallel reading. This setting controls the size of these segments. Not recommended to change until you're absolutely sure in what you're doing", 0) \ - M(Bool, parallel_replicas_local_plan, true, "Build local plan for local replica", 0) \ + M(Bool, parallel_replicas_local_plan, false, "Build local plan for local replica", 0) \ \ M(Bool, allow_experimental_inverted_index, false, "If it is set to true, allow to use experimental inverted index.", 0) \ M(Bool, allow_experimental_full_text_index, false, "If it is set to true, allow to use experimental full-text index.", 0) \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 24427dab1f3..ee9cfd3f28c 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -58,7 +58,7 @@ String ClickHouseVersion::toString() const static std::initializer_list> settings_changes_history_initializer = { {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"parallel_replicas_local_plan", false, true, "Use local plan for local replica in a query with parallel replicas"}, + {"parallel_replicas_local_plan", false, false, "Use local plan for local replica in a query with parallel replicas"}, {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, From 9501bc8a7551daa74cd37363b0047bb42c029697 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 29 Jul 2024 20:33:06 +0000 Subject: [PATCH 098/162] Fix forgotten conflict --- src/Processors/QueryPlan/ReadFromRemote.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromRemote.cpp b/src/Processors/QueryPlan/ReadFromRemote.cpp index cec460ef3bc..3df46eb1987 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.cpp +++ b/src/Processors/QueryPlan/ReadFromRemote.cpp @@ -21,13 +21,8 @@ #include #include #include -<<<<<<< HEAD #include -======= -#include -#include ->>>>>>> origin/master namespace DB { From d711ec118c41689d6f9694048bf279c3dbd2e1db Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 30 Jul 2024 09:04:59 +0000 Subject: [PATCH 099/162] Fix build --- src/Processors/QueryPlan/ConvertingActions.cpp | 2 +- src/Processors/QueryPlan/ReadFromRemote.h | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ConvertingActions.cpp b/src/Processors/QueryPlan/ConvertingActions.cpp index ff106ff08c1..b9703ef59cd 100644 --- a/src/Processors/QueryPlan/ConvertingActions.cpp +++ b/src/Processors/QueryPlan/ConvertingActions.cpp @@ -25,7 +25,7 @@ void addConvertingActions(QueryPlan & plan, const Block & header, bool has_missi }; auto convert_actions_dag = get_converting_dag(plan.getCurrentDataStream().header, header); - auto converting = std::make_unique(plan.getCurrentDataStream(), convert_actions_dag); + auto converting = std::make_unique(plan.getCurrentDataStream(), std::move(convert_actions_dag)); plan.addStep(std::move(converting)); } diff --git a/src/Processors/QueryPlan/ReadFromRemote.h b/src/Processors/QueryPlan/ReadFromRemote.h index 2dc4d2f8ed5..74389c8f9eb 100644 --- a/src/Processors/QueryPlan/ReadFromRemote.h +++ b/src/Processors/QueryPlan/ReadFromRemote.h @@ -70,6 +70,7 @@ public: ASTPtr query_ast_, ClusterPtr cluster_, const StorageID & storage_id_, + ParallelReplicasReadingCoordinatorPtr coordinator_, Block header_, QueryProcessingStage::Enum stage_, ContextMutablePtr context_, From bce0f46254d382f78c9a36be1371ed731aeb7880 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 30 Jul 2024 16:26:01 +0000 Subject: [PATCH 100/162] Simplify tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh + enable parallel_replicas_local_plan by default --- ...allel_replicas_join_algo_and_analyzer_3.sh | 52 ++++++------------- 1 file changed, 17 insertions(+), 35 deletions(-) diff --git a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh index e49a340ab67..81c6e8bed18 100755 --- a/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh +++ b/tests/queries/0_stateless/02967_parallel_replicas_join_algo_and_analyzer_3.sh @@ -21,23 +21,21 @@ insert into num_2 select number * 3, -number from numbers(1.5e6); echo echo "nested join with analyzer and parallel replicas, both global" -$CLICKHOUSE_CLIENT -q " -select * from (select key, value from num_1) l -inner join (select key, value from num_2 inner join - (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r -on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" +PARALLEL_REPLICAS_SETTINGS="allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_local_plan=1" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | +SETTINGS allow_experimental_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0" + +$CLICKHOUSE_CLIENT -q " +select * from (select key, value from num_1) l +inner join (select key, value from num_2 inner join + (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r +on l.key = r.key order by l.key limit 10 offset 10000 +SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -51,18 +49,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" +SETTINGS allow_experimental_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | +SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -77,18 +71,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" +SETTINGS allow_experimental_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 | +SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -102,18 +92,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" +SETTINGS allow_experimental_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 | +SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -127,18 +113,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'" +SETTINGS allow_experimental_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='hash'" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'" 2>&1 | +SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='hash'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' From c5a2f3aafbe6373a662f80f715e56a302a7fdd8b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 1 Aug 2024 14:21:54 +0000 Subject: [PATCH 101/162] Limit min_marks_for_concurrent_read --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 772bf427892..ec64db4139f 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -383,9 +383,12 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas(RangesInDataParts parts_wit /// We have a special logic for local replica. It has to read less data, because in some cases it should /// merge states of aggregate functions or do some other important stuff other than reading from Disk. auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; - // avoid using multiplier if min marks to read is already too big (to avoid overflow) - if (pool_settings.min_marks_for_concurrent_read >= std::numeric_limits::max()) + if (pool_settings.min_marks_for_concurrent_read > std::numeric_limits::max()) + { + /// limit min marks to read in case it's big, happened in test since due to settings randomzation + pool_settings.min_marks_for_concurrent_read = std::numeric_limits::max(); multiplier = 1.0f; + } if (auto result = pool_settings.min_marks_for_concurrent_read * multiplier; canConvertTo(result)) pool_settings.min_marks_for_concurrent_read = static_cast(result); @@ -556,9 +559,12 @@ Pipe ReadFromMergeTree::readInOrder( }; auto multiplier = context->getSettingsRef().parallel_replicas_single_task_marks_count_multiplier; - // avoid using multiplier if min marks to read is already too big (to avoid overflow) - if (pool_settings.min_marks_for_concurrent_read >= std::numeric_limits::max()) + if (pool_settings.min_marks_for_concurrent_read > std::numeric_limits::max()) + { + /// limit min marks to read in case it's big, happened in test since due to settings randomzation + pool_settings.min_marks_for_concurrent_read = std::numeric_limits::max(); multiplier = 1.0f; + } if (auto result = pool_settings.min_marks_for_concurrent_read * multiplier; canConvertTo(result)) pool_settings.min_marks_for_concurrent_read = static_cast(result); From 85b8c11175ff77176e6bbf4bd0b5821a34a14338 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 1 Aug 2024 20:32:03 +0000 Subject: [PATCH 102/162] Fix flaky 0_stateless/02784_parallel_replicas_automatic_decision - avoid max_threads randomzation --- .../0_stateless/02784_parallel_replicas_automatic_decision.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh index e5658f31a34..ef41a5d6277 100755 --- a/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh +++ b/tests/queries/0_stateless/02784_parallel_replicas_automatic_decision.sh @@ -53,6 +53,7 @@ function run_query_with_pure_parallel_replicas () { --allow_experimental_parallel_reading_from_replicas 1 \ --parallel_replicas_for_non_replicated_merge_tree 1 \ --parallel_replicas_min_number_of_rows_per_replica "$2" \ + --max_threads 5 \ |& grep "It is enough work for" | awk '{ print substr($7, 2, length($7) - 2) "\t" $20 " estimated parallel replicas" }' } From b6f5eb12115ad54ea9cfade80b71fce2052ce6c0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sun, 11 Aug 2024 23:15:19 +0000 Subject: [PATCH 103/162] Fix screwed merge --- src/Core/SettingsChangesHistory.cpp | 261 +----------------- ...allel_replicas_join_algo_and_analyzer_3.sh | 89 +----- ...llel_replicas_joins_and_analyzer.reference | 16 +- 3 files changed, 15 insertions(+), 351 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index e81786a902e..7cfc119f734 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -57,266 +57,6 @@ String ClickHouseVersion::toString() const /// Note: please check if the key already exists to prevent duplicate entries. static std::initializer_list> settings_changes_history_initializer = { - {"24.7", {{"output_format_parquet_write_page_index", false, true, "Add a possibility to write page index into parquet files."}, - {"parallel_replicas_local_plan", false, false, "Use local plan for local replica in a query with parallel replicas"}, - {"output_format_binary_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in RowBinaryWithNamesAndTypes output format"}, - {"input_format_binary_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in RowBinaryWithNamesAndTypes input format"}, - {"output_format_native_encode_types_in_binary_format", false, false, "Added new setting to allow to write type names in binary format in Native output format"}, - {"input_format_native_decode_types_in_binary_format", false, false, "Added new setting to allow to read type names in binary format in Native output format"}, - {"read_in_order_use_buffering", false, true, "Use buffering before merging while reading in order of primary key"}, - {"enable_named_columns_in_function_tuple", false, true, "Generate named tuples in function tuple() when all names are unique and can be treated as unquoted identifiers."}, - {"input_format_json_case_insensitive_column_matching", false, false, "Ignore case when matching JSON keys with CH columns."}, - {"optimize_trivial_insert_select", true, false, "The optimization does not make sense in many cases."}, - {"dictionary_validate_primary_key_type", false, false, "Validate primary key type for dictionaries. By default id type for simple layouts will be implicitly converted to UInt64."}, - {"collect_hash_table_stats_during_joins", false, true, "New setting."}, - {"max_size_to_preallocate_for_joins", 0, 100'000'000, "New setting."}, - {"input_format_orc_reader_time_zone_name", "GMT", "GMT", "The time zone name for ORC row reader, the default ORC row reader's time zone is GMT."}, - {"lightweight_mutation_projection_mode", "throw", "throw", "When lightweight delete happens on a table with projection(s), the possible operations include throw the exception as projection exists, or drop all projection related to this table then do lightweight delete."}, - {"database_replicated_allow_heavy_create", true, false, "Long-running DDL queries (CREATE AS SELECT and POPULATE) for Replicated database engine was forbidden"}, - {"query_plan_merge_filters", false, false, "Allow to merge filters in the query plan"}, - {"azure_sdk_max_retries", 10, 10, "Maximum number of retries in azure sdk"}, - {"azure_sdk_retry_initial_backoff_ms", 10, 10, "Minimal backoff between retries in azure sdk"}, - {"azure_sdk_retry_max_backoff_ms", 1000, 1000, "Maximal backoff between retries in azure sdk"}, - {"merge_tree_min_bytes_per_task_for_remote_reading", 4194304, 2097152, "Value is unified with `filesystem_prefetch_min_bytes_for_single_read_task`"}, - {"ignore_on_cluster_for_replicated_named_collections_queries", false, false, "Ignore ON CLUSTER clause for replicated named collections management queries."}, - {"backup_restore_s3_retry_attempts", 1000,1000, "Setting for Aws::Client::RetryStrategy, Aws::Client does retries itself, 0 means no retries. It takes place only for backup/restore."}, - {"postgresql_connection_attempt_timeout", 2, 2, "Allow to control 'connect_timeout' parameter of PostgreSQL connection."}, - {"postgresql_connection_pool_retries", 2, 2, "Allow to control the number of retries in PostgreSQL connection pool."} - }}, - {"24.6", {{"materialize_skip_indexes_on_insert", true, true, "Added new setting to allow to disable materialization of skip indexes on insert"}, - {"materialize_statistics_on_insert", true, true, "Added new setting to allow to disable materialization of statistics on insert"}, - {"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."}, - {"hdfs_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in HDFS engine instead of empty query result"}, - {"azure_throw_on_zero_files_match", false, false, "Allow to throw an error when ListObjects request cannot match any files in AzureBlobStorage engine instead of empty query result"}, - {"s3_validate_request_settings", true, true, "Allow to disable S3 request settings validation"}, - {"allow_experimental_full_text_index", false, false, "Enable experimental full-text index"}, - {"azure_skip_empty_files", false, false, "Allow to skip empty files in azure table engine"}, - {"hdfs_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in HDFS table engine"}, - {"azure_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in AzureBlobStorage table engine"}, - {"s3_ignore_file_doesnt_exist", false, false, "Allow to return 0 rows when the requested files don't exist instead of throwing an exception in S3 table engine"}, - {"s3_max_part_number", 10000, 10000, "Maximum part number number for s3 upload part"}, - {"s3_max_single_operation_copy_size", 32 * 1024 * 1024, 32 * 1024 * 1024, "Maximum size for a single copy operation in s3"}, - {"input_format_parquet_max_block_size", 8192, DEFAULT_BLOCK_SIZE, "Increase block size for parquet reader."}, - {"input_format_parquet_prefer_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Average block bytes output by parquet reader."}, - {"enable_blob_storage_log", true, true, "Write information about blob storage operations to system.blob_storage_log table"}, - {"allow_deprecated_snowflake_conversion_functions", true, false, "Disabled deprecated functions snowflakeToDateTime[64] and dateTime[64]ToSnowflake."}, - {"allow_statistic_optimize", false, false, "Old setting which popped up here being renamed."}, - {"allow_experimental_statistic", false, false, "Old setting which popped up here being renamed."}, - {"allow_statistics_optimize", false, false, "The setting was renamed. The previous name is `allow_statistic_optimize`."}, - {"allow_experimental_statistics", false, false, "The setting was renamed. The previous name is `allow_experimental_statistic`."}, - {"enable_vertical_final", false, true, "Enable vertical final by default again after fixing bug"}, - {"parallel_replicas_custom_key_range_lower", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards"}, - {"parallel_replicas_custom_key_range_upper", 0, 0, "Add settings to control the range filter when using parallel replicas with dynamic shards. A value of 0 disables the upper limit"}, - {"output_format_pretty_display_footer_column_names", 0, 1, "Add a setting to display column names in the footer if there are many rows. Threshold value is controlled by output_format_pretty_display_footer_column_names_min_rows."}, - {"output_format_pretty_display_footer_column_names_min_rows", 0, 50, "Add a setting to control the threshold value for setting output_format_pretty_display_footer_column_names_min_rows. Default 50."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.5", {{"allow_deprecated_error_prone_window_functions", true, false, "Allow usage of deprecated error prone window functions (neighbor, runningAccumulate, runningDifferenceStartingWithFirstValue, runningDifference)"}, - {"allow_experimental_join_condition", false, false, "Support join with inequal conditions which involve columns from both left and right table. e.g. t1.y < t2.y."}, - {"input_format_tsv_crlf_end_of_line", false, false, "Enables reading of CRLF line endings with TSV formats"}, - {"output_format_parquet_use_custom_encoder", false, true, "Enable custom Parquet encoder."}, - {"cross_join_min_rows_to_compress", 0, 10000000, "Minimal count of rows to compress block in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"cross_join_min_bytes_to_compress", 0, 1_GiB, "Minimal size of block to compress in CROSS JOIN. Zero value means - disable this threshold. This block is compressed when any of the two thresholds (by rows or by bytes) are reached."}, - {"http_max_chunk_size", 0, 0, "Internal limitation"}, - {"prefer_external_sort_block_bytes", 0, DEFAULT_BLOCK_SIZE * 256, "Prefer maximum block bytes for external sort, reduce the memory usage during merging."}, - {"input_format_force_null_for_omitted_fields", false, false, "Disable type-defaults for omitted fields when needed"}, - {"cast_string_to_dynamic_use_inference", false, false, "Add setting to allow converting String to Dynamic through parsing"}, - {"allow_experimental_dynamic_type", false, false, "Add new experimental Dynamic type"}, - {"azure_max_blocks_in_multipart_upload", 50000, 50000, "Maximum number of blocks in multipart upload for Azure."}, - }}, - {"24.4", {{"input_format_json_throw_on_bad_escape_sequence", true, true, "Allow to save JSON strings with bad escape sequences"}, - {"max_parsing_threads", 0, 0, "Add a separate setting to control number of threads in parallel parsing from files"}, - {"ignore_drop_queries_probability", 0, 0, "Allow to ignore drop queries in server with specified probability for testing purposes"}, - {"lightweight_deletes_sync", 2, 2, "The same as 'mutation_sync', but controls only execution of lightweight deletes"}, - {"query_cache_system_table_handling", "save", "throw", "The query cache no longer caches results of queries against system tables"}, - {"input_format_json_ignore_unnecessary_fields", false, true, "Ignore unnecessary fields and not parse them. Enabling this may not throw exceptions on json strings of invalid format or with duplicated fields"}, - {"input_format_hive_text_allow_variable_number_of_columns", false, true, "Ignore extra columns in Hive Text input (if file has more columns than expected) and treat missing fields in Hive Text input as default values."}, - {"allow_experimental_database_replicated", false, true, "Database engine Replicated is now in Beta stage"}, - {"temporary_data_in_cache_reserve_space_wait_lock_timeout_milliseconds", (10 * 60 * 1000), (10 * 60 * 1000), "Wait time to lock cache for sapce reservation in temporary data in filesystem cache"}, - {"optimize_rewrite_sum_if_to_count_if", false, true, "Only available for the analyzer, where it works correctly"}, - {"azure_allow_parallel_part_upload", "true", "true", "Use multiple threads for azure multipart upload."}, - {"max_recursive_cte_evaluation_depth", DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, DBMS_RECURSIVE_CTE_MAX_EVALUATION_DEPTH, "Maximum limit on recursive CTE evaluation depth"}, - {"query_plan_convert_outer_join_to_inner_join", false, true, "Allow to convert OUTER JOIN to INNER JOIN if filter after JOIN always filters default values"}, - }}, - {"24.3", {{"s3_connect_timeout_ms", 1000, 1000, "Introduce new dedicated setting for s3 connection timeout"}, - {"allow_experimental_shared_merge_tree", false, true, "The setting is obsolete"}, - {"use_page_cache_for_disks_without_file_cache", false, false, "Added userspace page cache"}, - {"read_from_page_cache_if_exists_otherwise_bypass_cache", false, false, "Added userspace page cache"}, - {"page_cache_inject_eviction", false, false, "Added userspace page cache"}, - {"default_table_engine", "None", "MergeTree", "Set default table engine to MergeTree for better usability"}, - {"input_format_json_use_string_type_for_ambiguous_paths_in_named_tuples_inference_from_objects", false, false, "Allow to use String type for ambiguous paths during named tuple inference from JSON objects"}, - {"traverse_shadow_remote_data_paths", false, false, "Traverse shadow directory when query system.remote_data_paths."}, - {"throw_if_deduplication_in_dependent_materialized_views_enabled_with_async_insert", false, true, "Deduplication in dependent materialized view cannot work together with async inserts."}, - {"parallel_replicas_allow_in_with_subquery", false, true, "If true, subquery for IN will be executed on every follower replica"}, - {"log_processors_profiles", false, true, "Enable by default"}, - {"function_locate_has_mysql_compatible_argument_order", false, true, "Increase compatibility with MySQL's locate function."}, - {"allow_suspicious_primary_key", true, false, "Forbid suspicious PRIMARY KEY/ORDER BY for MergeTree (i.e. SimpleAggregateFunction)"}, - {"filesystem_cache_reserve_space_wait_lock_timeout_milliseconds", 1000, 1000, "Wait time to lock cache for sapce reservation in filesystem cache"}, - {"max_parser_backtracks", 0, 1000000, "Limiting the complexity of parsing"}, - {"analyzer_compatibility_join_using_top_level_identifier", false, false, "Force to resolve identifier in JOIN USING from projection"}, - {"distributed_insert_skip_read_only_replicas", false, false, "If true, INSERT into Distributed will skip read-only replicas"}, - {"keeper_max_retries", 10, 10, "Max retries for general keeper operations"}, - {"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"}, - {"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"}, - {"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"}, - {"allow_experimental_analyzer", false, true, "Enable analyzer and planner by default."}, - {"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."}, - {"allow_get_client_http_header", false, false, "Introduced a new function."}, - {"output_format_pretty_row_numbers", false, true, "It is better for usability."}, - {"output_format_pretty_max_value_width_apply_for_single_value", true, false, "Single values in Pretty formats won't be cut."}, - {"output_format_parquet_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_orc_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_arrow_string_as_string", false, true, "ClickHouse allows arbitrary binary data in the String data type, which is typically UTF-8. Parquet/ORC/Arrow Strings only support UTF-8. That's why you can choose which Arrow's data type to use for the ClickHouse String data type - String or Binary. While Binary would be more correct and compatible, using String by default will correspond to user expectations in most cases."}, - {"output_format_parquet_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_orc_compression_method", "lz4", "zstd", "Parquet/ORC/Arrow support many compression methods, including lz4 and zstd. ClickHouse supports each and every compression method. Some inferior tools, such as 'duckdb', lack support for the faster `lz4` compression method, that's why we set zstd by default."}, - {"output_format_pretty_highlight_digit_groups", false, true, "If enabled and if output is a terminal, highlight every digit corresponding to the number of thousands, millions, etc. with underline."}, - {"geo_distance_returns_float64_on_float64_arguments", false, true, "Increase the default precision."}, - {"azure_max_inflight_parts_for_one_file", 20, 20, "The maximum number of a concurrent loaded parts in multipart upload request. 0 means unlimited."}, - {"azure_strict_upload_part_size", 0, 0, "The exact size of part to upload during multipart upload to Azure blob storage."}, - {"azure_min_upload_part_size", 16*1024*1024, 16*1024*1024, "The minimum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_max_upload_part_size", 5ull*1024*1024*1024, 5ull*1024*1024*1024, "The maximum size of part to upload during multipart upload to Azure blob storage."}, - {"azure_upload_part_size_multiply_factor", 2, 2, "Multiply azure_min_upload_part_size by this factor each time azure_multiply_parts_count_threshold parts were uploaded from a single write to Azure blob storage."}, - {"azure_upload_part_size_multiply_parts_count_threshold", 500, 500, "Each time this number of parts was uploaded to Azure blob storage, azure_min_upload_part_size is multiplied by azure_upload_part_size_multiply_factor."}, - {"output_format_csv_serialize_tuple_into_separate_columns", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_deserialize_separate_columns_into_tuple", true, true, "A new way of how interpret tuples in CSV format was added."}, - {"input_format_csv_try_infer_strings_from_quoted_tuples", true, true, "A new way of how interpret tuples in CSV format was added."}, - }}, - {"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"}, - {"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"}, - {"output_format_values_escape_quote_with_quote", false, false, "If true escape ' with '', otherwise quoted with \\'"}, - {"output_format_pretty_single_large_number_tip_threshold", 0, 1'000'000, "Print a readable number tip on the right side of the table if the block consists of a single number which exceeds this value (except 0)"}, - {"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"}, - {"query_plan_optimize_prewhere", true, true, "Allow to push down filter to PREWHERE expression for supported storages"}, - {"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."}, - {"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"}, - {"async_insert_use_adaptive_busy_timeout", false, true, "Use adaptive asynchronous insert timeout"}, - {"async_insert_busy_timeout_min_ms", 50, 50, "The minimum value of the asynchronous insert timeout in milliseconds; it also serves as the initial value, which may be increased later by the adaptive algorithm"}, - {"async_insert_busy_timeout_max_ms", 200, 200, "The minimum value of the asynchronous insert timeout in milliseconds; async_insert_busy_timeout_ms is aliased to async_insert_busy_timeout_max_ms"}, - {"async_insert_busy_timeout_increase_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout increases"}, - {"async_insert_busy_timeout_decrease_rate", 0.2, 0.2, "The exponential growth rate at which the adaptive asynchronous insert timeout decreases"}, - {"format_template_row_format", "", "", "Template row format string can be set directly in query"}, - {"format_template_resultset_format", "", "", "Template result set format string can be set in query"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", true, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}, - {"azure_max_single_part_copy_size", 256*1024*1024, 256*1024*1024, "The maximum size of object to copy using single part copy to Azure blob storage."}, - {"min_external_table_block_size_rows", DEFAULT_INSERT_BLOCK_SIZE, DEFAULT_INSERT_BLOCK_SIZE, "Squash blocks passed to external table to specified size in rows, if blocks are not big enough"}, - {"min_external_table_block_size_bytes", DEFAULT_INSERT_BLOCK_SIZE * 256, DEFAULT_INSERT_BLOCK_SIZE * 256, "Squash blocks passed to external table to specified size in bytes, if blocks are not big enough."}, - {"parallel_replicas_prefer_local_join", true, true, "If true, and JOIN can be executed with parallel replicas algorithm, and all storages of right JOIN part are *MergeTree, local JOIN will be used instead of GLOBAL JOIN."}, - {"optimize_time_filter_with_preimage", true, true, "Optimize Date and DateTime predicates by converting functions into equivalent comparisons without conversions (e.g. toYear(col) = 2023 -> col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, {"24.12", { } @@ -345,6 +85,7 @@ static std::initializer_list>>>>>> origin/master +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r on l.key = r.key order by l.key limit 10 offset 10000 -<<<<<<< HEAD -SETTINGS allow_experimental_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0" - -$CLICKHOUSE_CLIENT -q " -select * from (select key, value from num_1) l -inner join (select key, value from num_2 inner join - (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=0) r -on l.key = r.key order by l.key limit 10 offset 10000 -SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0" 2>&1 | -======= -SETTINGS enable_analyzer=1, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | ->>>>>>> origin/master +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, send_logs_level='trace', parallel_replicas_prefer_local_join=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -66,26 +49,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r on l.key = r.key order by l.key limit 10 offset 10000 -<<<<<<< HEAD -SETTINGS allow_experimental_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0" -======= -SETTINGS enable_analyzer=1, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" ->>>>>>> origin/master +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings parallel_replicas_prefer_local_join=1) r on l.key = r.key order by l.key limit 10 offset 10000 -<<<<<<< HEAD -SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0" 2>&1 | -======= -SETTINGS enable_analyzer=1, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', parallel_replicas_prefer_local_join=0" 2>&1 | ->>>>>>> origin/master +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, send_logs_level='trace', parallel_replicas_prefer_local_join=0" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -100,26 +71,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -<<<<<<< HEAD -SETTINGS allow_experimental_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'" -======= -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" ->>>>>>> origin/master +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -<<<<<<< HEAD -SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'" 2>&1 | -======= -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 | ->>>>>>> origin/master +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, send_logs_level='trace', join_algorithm='full_sorting_merge'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -133,26 +92,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r on l.key = r.key order by l.key limit 10 offset 10000 -<<<<<<< HEAD -SETTINGS allow_experimental_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'" -======= -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" ->>>>>>> origin/master +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='hash') r on l.key = r.key order by l.key limit 10 offset 10000 -<<<<<<< HEAD -SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='full_sorting_merge'" 2>&1 | -======= -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='full_sorting_merge'" 2>&1 | ->>>>>>> origin/master +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, send_logs_level='trace', join_algorithm='full_sorting_merge'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' @@ -166,26 +113,14 @@ select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -<<<<<<< HEAD -SETTINGS allow_experimental_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='hash'" -======= -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'" ->>>>>>> origin/master +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='hash'" $CLICKHOUSE_CLIENT -q " select * from (select key, value from num_1) l inner join (select key, value from num_2 inner join (select number * 7 as key from numbers(1e5)) as nn on num_2.key = nn.key settings join_algorithm='full_sorting_merge') r on l.key = r.key order by l.key limit 10 offset 10000 -<<<<<<< HEAD -SETTINGS allow_experimental_analyzer=1, send_logs_level='trace', $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, join_algorithm='hash'" 2>&1 | -======= -SETTINGS enable_analyzer=1, parallel_replicas_prefer_local_join=0, send_logs_level='trace', -allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, -cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', join_algorithm='hash'" 2>&1 | ->>>>>>> origin/master +SETTINGS enable_analyzer=1, $PARALLEL_REPLICAS_SETTINGS, parallel_replicas_prefer_local_join=0, send_logs_level='trace', join_algorithm='hash'" 2>&1 | grep "executeQuery\|.*Coordinator: Coordination done" | grep -o "SELECT.*WithMergeableState)\|.*Coordinator: Coordination done" | sed -re 's/_data_[[:digit:]]+_[[:digit:]]+/_data_/g' diff --git a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference index 453caf1cffe..ba67acd7d0b 100644 --- a/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference +++ b/tests/queries/0_stateless/02967_parallel_replicas_joins_and_analyzer.reference @@ -18,8 +18,7 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x 13 13 0 0 0 0 14 14 14 14 0 0 15 15 0 0 0 0 -<<<<<<< HEAD -explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Union Expression Join @@ -34,11 +33,6 @@ Union ReadFromMemoryStorage Expression ReadFromRemoteParallelReplicas -======= -explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -Expression - ReadFromRemoteParallelReplicas ->>>>>>> origin/master -- -- The same query with cte; with sub1 as (select x, y from tab1 where x != 2), @@ -477,8 +471,7 @@ select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x 13 13 0 0 0 0 14 14 14 14 0 0 15 15 0 0 0 0 -<<<<<<< HEAD -explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', allow_experimental_analyzer=1; +explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; Union Expression Join @@ -495,11 +488,6 @@ Union ReadFromMergeTree Expression ReadFromRemoteParallelReplicas -======= -explain description=0 select x, y, r.y, z, rr.z, a from (select l.x, l.y, r.y, r.z as z from (select x, y from tab1 where x != 2) l any left join (select y, z from tab2 where y != 4) r on l.y = r.y) ll any left join (select z, a from tab3 where z != 8) rr on ll.z = rr.z SETTINGS allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 2, parallel_replicas_for_non_replicated_merge_tree = 1, prefer_localhost_replica = 1, cluster_for_parallel_replicas = 'test_cluster_one_shard_three_replicas_localhost', enable_analyzer=1; -Expression - ReadFromRemoteParallelReplicas ->>>>>>> origin/master -- -- The same query with cte; with sub1 as (select x, y from tab1 where x != 2), From 8d6e2e26a57764d2dfc133d8b063578848fd3e2e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 13 Aug 2024 14:29:52 +0000 Subject: [PATCH 104/162] New test for asan issue with pr --- .../03222_pr_asan_index_granularity.reference | 100 ++++++++++++++++++ .../03222_pr_asan_index_granularity.sql | 14 +++ 2 files changed, 114 insertions(+) create mode 100644 tests/queries/0_stateless/03222_pr_asan_index_granularity.reference create mode 100644 tests/queries/0_stateless/03222_pr_asan_index_granularity.sql diff --git a/tests/queries/0_stateless/03222_pr_asan_index_granularity.reference b/tests/queries/0_stateless/03222_pr_asan_index_granularity.reference new file mode 100644 index 00000000000..662880055bd --- /dev/null +++ b/tests/queries/0_stateless/03222_pr_asan_index_granularity.reference @@ -0,0 +1,100 @@ +0 18 9899 +0 18 9898 +0 18 9897 +0 18 9896 +0 18 9895 +0 18 9894 +0 18 9893 +0 18 9892 +0 18 9891 +0 18 9890 +0 18 9889 +0 18 9888 +0 18 9887 +0 18 9886 +0 18 9885 +0 18 9884 +0 18 9883 +0 18 9882 +0 18 9881 +0 18 9880 +0 18 9879 +0 18 9878 +0 18 9877 +0 18 9876 +0 18 9875 +0 18 9874 +0 18 9873 +0 18 9872 +0 18 9871 +0 18 9870 +0 18 9869 +0 18 9868 +0 18 9867 +0 18 9866 +0 18 9865 +0 18 9864 +0 18 9863 +0 18 9862 +0 18 9861 +0 18 9860 +0 18 9859 +0 18 9858 +0 18 9857 +0 18 9856 +0 18 9855 +0 18 9854 +0 18 9853 +0 18 9852 +0 18 9851 +0 18 9850 +0 18 9849 +0 18 9848 +0 18 9847 +0 18 9846 +0 18 9845 +0 18 9844 +0 18 9843 +0 18 9842 +0 18 9841 +0 18 9840 +0 18 9839 +0 18 9838 +0 18 9837 +0 18 9836 +0 18 9835 +0 18 9834 +0 18 9833 +0 18 9832 +0 18 9831 +0 18 9830 +0 18 9829 +0 18 9828 +0 18 9827 +0 18 9826 +0 18 9825 +0 18 9824 +0 18 9823 +0 18 9822 +0 18 9821 +0 18 9820 +0 18 9819 +0 18 9818 +0 18 9817 +0 18 9816 +0 18 9815 +0 18 9814 +0 18 9813 +0 18 9812 +0 18 9811 +0 18 9810 +0 18 9809 +0 18 9808 +0 18 9807 +0 18 9806 +0 18 9805 +0 18 9804 +0 18 9803 +0 18 9802 +0 18 9801 +0 18 9800 diff --git a/tests/queries/0_stateless/03222_pr_asan_index_granularity.sql b/tests/queries/0_stateless/03222_pr_asan_index_granularity.sql new file mode 100644 index 00000000000..b7f37dd2856 --- /dev/null +++ b/tests/queries/0_stateless/03222_pr_asan_index_granularity.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS test; + +CREATE TABLE test (k UInt64, v String) +ENGINE = MergeTree +ORDER BY k +SETTINGS index_granularity=1; + +INSERT INTO test SELECT number, toString(number) FROM numbers(10_000); + +SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; + +SELECT 0, materialize(18), k FROM test PREWHERE toNullable(toNullable(11)) WHERE toNullable(11) ORDER BY k DESC NULLS LAST LIMIT 100, 100 SETTINGS optimize_read_in_order = 1, merge_tree_min_rows_for_concurrent_read = 9223372036854775806, max_threads = 1; + +-- DROP TABLE test; From 99dcf7e60ebf91858e4b16d9235bb1623a78b9b0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 13 Aug 2024 15:57:09 +0000 Subject: [PATCH 105/162] Add 03222_pr_final_not_supported.sql --- .../03222_pr_final_not_supported.reference | 0 .../03222_pr_final_not_supported.sql | 22 +++++++++++++++++++ 2 files changed, 22 insertions(+) create mode 100644 tests/queries/0_stateless/03222_pr_final_not_supported.reference create mode 100644 tests/queries/0_stateless/03222_pr_final_not_supported.sql diff --git a/tests/queries/0_stateless/03222_pr_final_not_supported.reference b/tests/queries/0_stateless/03222_pr_final_not_supported.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03222_pr_final_not_supported.sql b/tests/queries/0_stateless/03222_pr_final_not_supported.sql new file mode 100644 index 00000000000..a018bfc53b4 --- /dev/null +++ b/tests/queries/0_stateless/03222_pr_final_not_supported.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS test_00808; + +CREATE TABLE test_00808 +( + `date` Date, + `id` Int8, + `name` String, + `value` Int64, + `sign` Int8 +) +ENGINE = CollapsingMergeTree(sign) +ORDER BY (id, date) +SETTINGS index_granularity = 62918, min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0., replace_long_file_name_to_hash = false, max_file_name_length = 79, min_bytes_for_full_part_storage = 536870912, compact_parts_max_bytes_to_buffer = 525685803, compact_parts_max_granules_to_buffer = 148, compact_parts_merge_max_bytes_to_prefetch_part = 23343456, merge_max_block_size = 22854, old_parts_lifetime = 93., prefer_fetch_merged_part_size_threshold = 3610162434, vertical_merge_algorithm_min_rows_to_activate = 1000000, vertical_merge_algorithm_min_columns_to_activate = 1, min_merge_bytes_to_use_direct_io = 1, index_granularity_bytes = 9550183, concurrent_part_removal_threshold = 34, allow_vertical_merges_from_compact_to_wide_parts = false, cache_populated_by_fetch = false, marks_compress_block_size = 97756, primary_key_compress_block_size = 80902; + +INSERT INTO test_00808 VALUES('2000-01-01', 1, 'test string 1', 1, 1); +INSERT INTO test_00808 VALUES('2000-01-01', 2, 'test string 2', 2, 1); + +SET enable_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; + +SELECT * FROM (SELECT * FROM test_00808 FINAL) WHERE id = 1; -- { serverError SUPPORT_IS_DISABLED } + +DROP TABLE test_00808; From 4448880bbe11d80047de2c3988165b4b54710f1e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 13 Aug 2024 16:15:28 +0000 Subject: [PATCH 106/162] Remove settings 03222_pr_final_not_supported --- tests/queries/0_stateless/03222_pr_final_not_supported.sql | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/03222_pr_final_not_supported.sql b/tests/queries/0_stateless/03222_pr_final_not_supported.sql index a018bfc53b4..6c2e05c12a9 100644 --- a/tests/queries/0_stateless/03222_pr_final_not_supported.sql +++ b/tests/queries/0_stateless/03222_pr_final_not_supported.sql @@ -9,13 +9,12 @@ CREATE TABLE test_00808 `sign` Int8 ) ENGINE = CollapsingMergeTree(sign) -ORDER BY (id, date) -SETTINGS index_granularity = 62918, min_bytes_for_wide_part = 0, ratio_of_defaults_for_sparse_serialization = 0., replace_long_file_name_to_hash = false, max_file_name_length = 79, min_bytes_for_full_part_storage = 536870912, compact_parts_max_bytes_to_buffer = 525685803, compact_parts_max_granules_to_buffer = 148, compact_parts_merge_max_bytes_to_prefetch_part = 23343456, merge_max_block_size = 22854, old_parts_lifetime = 93., prefer_fetch_merged_part_size_threshold = 3610162434, vertical_merge_algorithm_min_rows_to_activate = 1000000, vertical_merge_algorithm_min_columns_to_activate = 1, min_merge_bytes_to_use_direct_io = 1, index_granularity_bytes = 9550183, concurrent_part_removal_threshold = 34, allow_vertical_merges_from_compact_to_wide_parts = false, cache_populated_by_fetch = false, marks_compress_block_size = 97756, primary_key_compress_block_size = 80902; +ORDER BY (id, date); INSERT INTO test_00808 VALUES('2000-01-01', 1, 'test string 1', 1, 1); INSERT INTO test_00808 VALUES('2000-01-01', 2, 'test string 2', 2, 1); -SET enable_analyzer=1, allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; +SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree=1, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; SELECT * FROM (SELECT * FROM test_00808 FINAL) WHERE id = 1; -- { serverError SUPPORT_IS_DISABLED } From 62e45a4c8fefcd846d531c7c4d52e4277435d3c3 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 13 Aug 2024 20:55:20 +0000 Subject: [PATCH 107/162] Test 03222_parallel_replicas_min_marks_to_read_overflow --- ..._replicas_min_marks_to_read_overflow.reference | 10 ++++++++++ ...rallel_replicas_min_marks_to_read_overflow.sql | 15 +++++++++++++++ 2 files changed, 25 insertions(+) create mode 100644 tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.reference create mode 100644 tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.sql diff --git a/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.reference b/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.reference new file mode 100644 index 00000000000..7fafd4d13ea --- /dev/null +++ b/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.reference @@ -0,0 +1,10 @@ +100 100 +101 101 +102 102 +103 103 +104 104 +105 105 +106 106 +107 107 +108 108 +109 109 diff --git a/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.sql b/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.sql new file mode 100644 index 00000000000..112373e5db2 --- /dev/null +++ b/tests/queries/0_stateless/03222_parallel_replicas_min_marks_to_read_overflow.sql @@ -0,0 +1,15 @@ +DROP TABLE IF EXISTS test__fuzz_22 SYNC; + +CREATE TABLE test__fuzz_22 (k Float32, v String) ENGINE = ReplicatedMergeTree('/clickhouse/03222/{database}/test__fuzz_22', 'r1') ORDER BY k SETTINGS index_granularity = 1; + +INSERT INTO test__fuzz_22 SELECT number, toString(number) FROM numbers(10_000); + +SET allow_experimental_parallel_reading_from_replicas = 2, max_parallel_replicas = 3, cluster_for_parallel_replicas='test_cluster_one_shard_three_replicas_localhost'; + +SELECT k, v +FROM test__fuzz_22 +ORDER BY k +LIMIT 100, 10 +SETTINGS merge_tree_min_rows_for_concurrent_read = 9223372036854775806; + +DROP TABLE test__fuzz_22 SYNC; From 1a8f45464c1ef81dc8c2c16c8fbba2e696762962 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 13 Aug 2024 21:05:29 +0000 Subject: [PATCH 108/162] Fix min_marks_to_read overflow --- src/Storages/MergeTree/MergeTreeIndexGranularity.cpp | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp index 2a45ab1d927..2f9a4a47b11 100644 --- a/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexGranularity.cpp @@ -103,8 +103,16 @@ size_t MergeTreeIndexGranularity::countMarksForRows(size_t from_mark, size_t num /// This is a heuristic to respect min_marks_to_read which is ignored by MergeTreeReadPool in case of remote disk. /// See comment in IMergeTreeSelectAlgorithm. - if (min_marks_to_read && from_mark + 2 * min_marks_to_read <= to_mark) - to_mark = from_mark + min_marks_to_read; + if (min_marks_to_read) + { + // check that ... + bool overflow = ((1ULL << 63) & min_marks_to_read); // further multiplication by 2 will not overflow + if (!overflow) + overflow = (std::numeric_limits::max() - from_mark) < 2 * min_marks_to_read; // further addition will not overflow + + if (!overflow && from_mark + 2 * min_marks_to_read <= to_mark) + to_mark = from_mark + min_marks_to_read; + } return getRowsCountInRange(from_mark, std::max(1UL, to_mark)) - offset_in_rows; } From a3978b24d0ee8847e5b332a08014ced896edae2c Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Thu, 15 Aug 2024 11:11:49 +0000 Subject: [PATCH 109/162] Fix for subqueries with FINAL --- src/Planner/findParallelReplicasQuery.cpp | 6 ++++++ ... => 03222_parallel_replicas_final_in_subquery.reference} | 0 ...ed.sql => 03222_parallel_replicas_final_in_subquery.sql} | 0 3 files changed, 6 insertions(+) rename tests/queries/0_stateless/{03222_pr_final_not_supported.reference => 03222_parallel_replicas_final_in_subquery.reference} (100%) rename tests/queries/0_stateless/{03222_pr_final_not_supported.sql => 03222_parallel_replicas_final_in_subquery.sql} (100%) diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index 39edb1e6516..cd02e01c392 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -52,7 +52,13 @@ std::stack getSupportingParallelReplicasQuery(const IQueryTre const auto & storage = table_node.getStorage(); /// Here we check StorageDummy as well, to support a query tree with replaced storages. if (std::dynamic_pointer_cast(storage) || typeid_cast(storage.get())) + { + /// parallel replicas is not supported with FINAL + if (table_node.getTableExpressionModifiers()->hasFinal()) + return {}; + return res; + } return {}; } diff --git a/tests/queries/0_stateless/03222_pr_final_not_supported.reference b/tests/queries/0_stateless/03222_parallel_replicas_final_in_subquery.reference similarity index 100% rename from tests/queries/0_stateless/03222_pr_final_not_supported.reference rename to tests/queries/0_stateless/03222_parallel_replicas_final_in_subquery.reference diff --git a/tests/queries/0_stateless/03222_pr_final_not_supported.sql b/tests/queries/0_stateless/03222_parallel_replicas_final_in_subquery.sql similarity index 100% rename from tests/queries/0_stateless/03222_pr_final_not_supported.sql rename to tests/queries/0_stateless/03222_parallel_replicas_final_in_subquery.sql From 29fffd34ac5ab769153563f9c83b134cbaefb117 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 16 Aug 2024 11:06:20 +0000 Subject: [PATCH 110/162] Fix: check if table expression is present --- src/Planner/findParallelReplicasQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Planner/findParallelReplicasQuery.cpp b/src/Planner/findParallelReplicasQuery.cpp index cd02e01c392..25481d06670 100644 --- a/src/Planner/findParallelReplicasQuery.cpp +++ b/src/Planner/findParallelReplicasQuery.cpp @@ -54,7 +54,7 @@ std::stack getSupportingParallelReplicasQuery(const IQueryTre if (std::dynamic_pointer_cast(storage) || typeid_cast(storage.get())) { /// parallel replicas is not supported with FINAL - if (table_node.getTableExpressionModifiers()->hasFinal()) + if (table_node.getTableExpressionModifiers() && table_node.getTableExpressionModifiers()->hasFinal()) return {}; return res; From 37c3f4a8707598022d94ce5b20c10893882cf58c Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Fri, 10 May 2024 12:18:06 +0800 Subject: [PATCH 111/162] add threshold for table rows --- src/Core/Settings.h | 2 + src/Core/SettingsChangesHistory.cpp | 253 +++++++++++ src/Interpreters/HashJoin/AddedColumns.cpp | 63 +++ src/Interpreters/HashJoin/AddedColumns.h | 6 + src/Interpreters/HashJoin/HashJoin.cpp | 84 +++- src/Interpreters/HashJoin/HashJoin.h | 7 +- src/Interpreters/HashJoin/HashJoinMethods.h | 424 +++++++++++++++++- src/Interpreters/IJoin.h | 1 + src/Interpreters/RowRefs.h | 1 + .../Transforms/JoiningTransform.cpp | 6 +- 10 files changed, 841 insertions(+), 6 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index d8837d26e54..c1433ca7250 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -922,6 +922,8 @@ class IColumn; M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \ M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \ M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \ + M(Int32, join_to_sort_perkey_rows_threshold, 40, "The lower limit of per-key average rows in the right table to determine whether to sort it in hash join.", 0) \ + M(Int32, join_to_sort_table_rows_threshold, 10000, "The upper limit of rows in the right table to determine whether to sort it in hash join.", 0) \ M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, use_hive_partitioning, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines.", 0)\ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 2415323b4a0..b975c6b2fad 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -516,6 +516,259 @@ static std::initializer_list col >= '2023-01-01' AND col <= '2023-12-31')"}, + {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, + {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, + {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, + {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, + {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, + {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, + }}, + {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, + {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, + {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, + {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, + {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, + {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, + {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, + {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, + {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, + {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, + {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, + {"enable_vertical_final", false, true, "Use vertical final by default"}, + {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, + {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, + {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, + {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, + {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, + {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, + {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, + {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, + {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, + {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, + {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, + {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, + {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, + {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, + {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, + {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, + {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, + {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, + {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, + {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, + {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, + {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, + {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, + {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, + {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, + {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, + {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, + {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, + {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, + {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, + {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, + {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, + {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, + {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, + {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, + {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, + {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, + {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, + {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, + {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, + {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, + {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, + {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, + {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, + {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, + {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, + {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, + {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, + {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, + {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, + {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, + {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, + {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, + {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, + {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, + {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, + {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, + {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, + {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, + {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, + {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, + {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, + {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, + {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, + {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, + {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, + {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, + {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, + {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, + {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, + {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, + {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, + {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, + {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, + {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, + {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, + {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, + {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, + {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, + {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, + {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, + {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, + {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, + {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, + {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, + {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, + {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, + {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, + {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, + {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, + {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, + {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, + {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, }; diff --git a/src/Interpreters/HashJoin/AddedColumns.cpp b/src/Interpreters/HashJoin/AddedColumns.cpp index 21cb6e401ed..d70781d2fb3 100644 --- a/src/Interpreters/HashJoin/AddedColumns.cpp +++ b/src/Interpreters/HashJoin/AddedColumns.cpp @@ -20,10 +20,13 @@ void AddedColumns::buildOutput() {} template<> void AddedColumns::buildJoinGetOutput() {} +<<<<<<< HEAD template<> template void AddedColumns::buildOutputFromBlocks() {} +======= +>>>>>>> add threshold for table rows template<> void AddedColumns::buildOutput() @@ -32,9 +35,15 @@ void AddedColumns::buildOutput() buildOutputFromBlocks(); else { +<<<<<<< HEAD if (join_data_avg_perkey_rows < output_by_row_list_threshold) buildOutputFromBlocks(); else +======= + if (join_data_avg_perkey_rows < sort_right_perkey_rows_threshold) + buildOutputFromBlocks(); + else if (join_data_sorted) +>>>>>>> add threshold for table rows { for (size_t i = 0; i < this->size(); ++i) { @@ -44,14 +53,19 @@ void AddedColumns::buildOutput() if (row_ref_i) { const RowRefList * row_ref_list = reinterpret_cast(row_ref_i); +<<<<<<< HEAD for (auto it = row_ref_list->begin(); it.ok(); ++it) col->insertFrom(*it->block->getByPosition(right_indexes[i]).column, it->row_num); +======= + col->insertRangeFrom(*row_ref_list->block->getByPosition(right_indexes[i]).column, row_ref_list->row_num, row_ref_list->rows); +>>>>>>> add threshold for table rows } else type_name[i].type->insertDefaultInto(*col); } } } +<<<<<<< HEAD } } @@ -74,6 +88,25 @@ void AddedColumns::buildJoinGetOutput() nullable_col->insertFromNotNullable(*column_from_block.column, row_ref->row_num); else col->insertFrom(*column_from_block.column, row_ref->row_num); +======= + else + { + for (size_t i = 0; i < this->size(); ++i) + { + auto & col = columns[i]; + for (auto row_ref_i : lazy_output.row_refs) + { + if (row_ref_i) + { + const RowRefList * row_ref_list = reinterpret_cast(row_ref_i); + for (auto it = row_ref_list->begin(); it.ok(); ++it) + col->insertFrom(*it->block->getByPosition(right_indexes[i]).column, it->row_num); + } + else + type_name[i].type->insertDefaultInto(*col); + } + } +>>>>>>> add threshold for table rows } } } @@ -82,7 +115,11 @@ template<> template void AddedColumns::buildOutputFromBlocks() { +<<<<<<< HEAD if (this->size() == 0) +======= + if (this->size() == 0) +>>>>>>> add threshold for table rows return; std::vector blocks; std::vector row_nums; @@ -123,6 +160,32 @@ void AddedColumns::buildOutputFromBlocks() col->insertFrom(*blocks[j]->getByPosition(right_indexes[i]).column, row_nums[j]); else type_name[i].type->insertDefaultInto(*col); +<<<<<<< HEAD +======= + } + } +} + +template<> +void AddedColumns::buildJoinGetOutput() +{ + for (size_t i = 0; i < this->size(); ++i) + { + auto & col = columns[i]; + for (auto row_ref_i : lazy_output.row_refs) + { + if (!row_ref_i) + { + type_name[i].type->insertDefaultInto(*col); + continue; + } + const auto * row_ref = reinterpret_cast(row_ref_i); + const auto & column_from_block = row_ref->block->getByPosition(right_indexes[i]); + if (auto * nullable_col = typeid_cast(col.get()); nullable_col && !column_from_block.column->isNullable()) + nullable_col->insertFromNotNullable(*column_from_block.column, row_ref->row_num); + else + col->insertFrom(*column_from_block.column, row_ref->row_num); +>>>>>>> add threshold for table rows } } } diff --git a/src/Interpreters/HashJoin/AddedColumns.h b/src/Interpreters/HashJoin/AddedColumns.h index f1b95a63be6..5ae69fbbf66 100644 --- a/src/Interpreters/HashJoin/AddedColumns.h +++ b/src/Interpreters/HashJoin/AddedColumns.h @@ -196,6 +196,12 @@ private: } } + /** Build output from the blocks that extract from `RowRef` or `RowRefList`, to avoid block cache miss which may cause performance slow down. + * And This problem would happen it we directly build output from `RowRef` or `RowRefList`. + */ + template + void buildOutputFromBlocks(); + MutableColumns columns; bool is_join_get; std::vector right_indexes; diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 9c07a71e614..6f332118f8a 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -649,7 +649,6 @@ bool HashJoin::addBlockToJoin(const Block & source_block_, bool check_limits) } data->keys_to_join = total_rows; shrinkStoredBlocksToFit(total_bytes); - return table_join->sizeLimits().check(total_rows, total_bytes, "JOIN", ErrorCodes::SET_SIZE_LIMIT_EXCEEDED); } @@ -1361,4 +1360,87 @@ bool HashJoin::needUsedFlagsForPerRightTableRow(std::shared_ptr table return false; } +template +void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]]) +{ + constexpr JoinFeatures join_features; + if constexpr (join_features.is_all_join && (join_features.left || join_features.inner)) + { + auto merge_rows_into_one_block = [&](BlocksList & blocks, RowRefList & rows_ref) + { + auto it = rows_ref.begin(); + if (it.ok()) + { + if (blocks.empty() || blocks.back().rows() > DEFAULT_BLOCK_SIZE) + blocks.emplace_back(it->block->cloneEmpty()); + } + else + { + return; + } + auto & block = blocks.back(); + size_t start_row = block.rows(); + for (; it.ok(); ++it) + { + for (size_t i = 0; i < block.columns(); ++i) + { + auto & col = *(block.getByPosition(i).column->assumeMutable()); + col.insertFrom(*it->block->getByPosition(i).column, it->row_num); + } + } + if (block.rows() > start_row) + { + RowRefList new_rows_ref(&block, start_row, block.rows() - start_row); + rows_ref = std::move(new_rows_ref); + } + }; + + auto visit_rows_map = [&](BlocksList & blocks, MapsAll & rows_map) + { + switch (data->type) + { + #define M(TYPE) \ + case Type::TYPE: \ + {\ + rows_map.TYPE->forEachMapped([&](RowRefList & rows_ref) { merge_rows_into_one_block(blocks, rows_ref); }); \ + break; \ + } + APPLY_FOR_JOIN_VARIANTS(M) + #undef M + default: + break; + } + }; + BlocksList sorted_blocks; + visit_rows_map(sorted_blocks, map); + data->blocks.swap(sorted_blocks); + } +} + +void HashJoin::tryRerangeRightTableData() +{ + if ((kind != JoinKind::Inner && kind != JoinKind::Left) || strictness != JoinStrictness::All || table_join->getMixedJoinExpression()) + return; + + if (!data || data->sorted || data->blocks.empty() || data->maps.size() > 1) + return; + + if (data->keys_to_join == 0) + data->keys_to_join = getTotalRowCount(); + if (sample_block_with_columns_to_add.columns() == 0 || data->rows_to_join > table_join->sortRightTableRowsThreshold() || data->avgPerKeyRows() < table_join->sortRightPerkeyRowsThreshold()) + { + LOG_DEBUG(log, "The joined right table total rows :{}, total keys :{}, columns added:{}", + data->rows_to_join, data->keys_to_join, sample_block_with_columns_to_add.columns()); + return; + } + std::cout << "sort right table rows" << std::endl; + joinDispatch( + kind, + strictness, + data->maps.front(), + [&](auto kind_, auto strictness_, auto & map_) { tryRerangeRightTableDataImpl(map_); }); + std::cout << "sort right finished" << std::endl; + data->sorted = true; +} + } diff --git a/src/Interpreters/HashJoin/HashJoin.h b/src/Interpreters/HashJoin/HashJoin.h index d645b8e9273..230343691ea 100644 --- a/src/Interpreters/HashJoin/HashJoin.h +++ b/src/Interpreters/HashJoin/HashJoin.h @@ -345,11 +345,12 @@ public: size_t blocks_allocated_size = 0; size_t blocks_nullmaps_allocated_size = 0; - /// Number of rows of right table to join size_t rows_to_join = 0; /// Number of keys of right table to join size_t keys_to_join = 0; + /// Whether the right table reranged by key + bool sorted = false; size_t avgPerKeyRows() const { @@ -465,6 +466,10 @@ private: void validateAdditionalFilterExpression(std::shared_ptr additional_filter_expression); bool needUsedFlagsForPerRightTableRow(std::shared_ptr table_join_) const; + + void tryRerangeRightTableData() override; + template + void tryRerangeRightTableDataImpl(Map & map); }; } diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 97ad57d26ea..9d94c3f62c2 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -121,7 +121,142 @@ private: std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, - JoinStuff::JoinUsedFlags & used_flags); + JoinStuff::JoinUsedFlags & used_flags) + { + constexpr JoinFeatures join_features; + + size_t rows = added_columns.rows_to_add; + if constexpr (need_filter) + added_columns.filter = IColumn::Filter(rows, 0); + if constexpr (!flag_per_row && (STRICTNESS == JoinStrictness::All || (STRICTNESS == JoinStrictness::Semi && KIND == JoinKind::Right))) + added_columns.output_by_row_list = true; + + Arena pool; + + if constexpr (join_features.need_replication) + added_columns.offsets_to_replicate = std::make_unique(rows); + + IColumn::Offset current_offset = 0; + size_t max_joined_block_rows = added_columns.max_joined_block_rows; + size_t i = 0; + for (; i < rows; ++i) + { + if constexpr (join_features.need_replication) + { + if (unlikely(current_offset >= max_joined_block_rows)) + { + added_columns.offsets_to_replicate->resize_assume_reserved(i); + added_columns.filter.resize_assume_reserved(i); + break; + } + } + + bool right_row_found = false; + + KnownRowsHolder known_rows; + for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx) + { + const auto & join_keys = added_columns.join_on_keys[onexpr_idx]; + if (join_keys.null_map && (*join_keys.null_map)[i]) + continue; + + bool row_acceptable = !join_keys.isRowFiltered(i); + using FindResult = typename KeyGetter::FindResult; + auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult(); + + if (find_result.isFound()) + { + right_row_found = true; + auto & mapped = find_result.getMapped(); + if constexpr (join_features.is_asof_join) + { + const IColumn & left_asof_key = added_columns.leftAsofKey(); + + auto row_ref = mapped->findAsof(left_asof_key, i); + if (row_ref && row_ref->block) + { + setUsed(added_columns.filter, i); + if constexpr (flag_per_row) + used_flags.template setUsed(row_ref->block, row_ref->row_num, 0); + else + used_flags.template setUsed(find_result); + + added_columns.appendFromBlock(row_ref, join_features.add_missing); + } + else + addNotFoundRow(added_columns, current_offset); + } + else if constexpr (join_features.is_all_join) + { + setUsed(added_columns.filter, i); + used_flags.template setUsed(find_result); + auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; + addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); + } + else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right) + { + /// Use first appeared left key + it needs left columns replication + bool used_once = used_flags.template setUsedOnce(find_result); + if (used_once) + { + auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; + setUsed(added_columns.filter, i); + addFoundRowAll( + mapped, added_columns, current_offset, known_rows, used_flags_opt); + } + } + else if constexpr (join_features.is_any_join && KIND == JoinKind::Inner) + { + bool used_once = used_flags.template setUsedOnce(find_result); + + /// Use first appeared left key only + if (used_once) + { + setUsed(added_columns.filter, i); + added_columns.appendFromBlock(&mapped, join_features.add_missing); + } + + break; + } + else if constexpr (join_features.is_any_join && join_features.full) + { + /// TODO + } + else if constexpr (join_features.is_anti_join) + { + if constexpr (join_features.right && join_features.need_flags) + used_flags.template setUsed(find_result); + } + else /// ANY LEFT, SEMI LEFT, old ANY (RightAny) + { + setUsed(added_columns.filter, i); + used_flags.template setUsed(find_result); + added_columns.appendFromBlock(&mapped, join_features.add_missing); + + if (join_features.is_any_or_semi_join) + { + break; + } + } + } + } + + if (!right_row_found) + { + if constexpr (join_features.is_anti_join && join_features.left) + setUsed(added_columns.filter, i); + addNotFoundRow(added_columns, current_offset); + } + + if constexpr (join_features.need_replication) + { + (*added_columns.offsets_to_replicate)[i] = current_offset; + } + } + + added_columns.applyLazyDefaults(); + return i; + } template static void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]]); @@ -131,7 +266,120 @@ private: size_t left_start_row, const std::vector & selected_rows, const std::vector & row_replicate_offset, - AddedColumns & added_columns); + AddedColumns & added_columns) + { + ColumnPtr result_column; + do + { + if (selected_rows.empty()) + { + result_column = ColumnUInt8::create(); + break; + } + const Block & sample_right_block = *((*selected_rows.begin())->block); + if (!sample_right_block || !added_columns.additional_filter_expression) + { + auto filter = ColumnUInt8::create(); + filter->insertMany(1, selected_rows.size()); + result_column = std::move(filter); + break; + } + + auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes(); + if (required_cols.empty()) + { + Block block; + added_columns.additional_filter_expression->execute(block); + result_column = block.getByPosition(0).column->cloneResized(selected_rows.size()); + break; + } + NameSet required_column_names; + for (auto & col : required_cols) + required_column_names.insert(col.name); + + Block executed_block; + size_t right_col_pos = 0; + for (const auto & col : sample_right_block.getColumnsWithTypeAndName()) + { + if (required_column_names.contains(col.name)) + { + auto new_col = col.column->cloneEmpty(); + for (const auto & selected_row : selected_rows) + { + const auto & src_col = selected_row->block->getByPosition(right_col_pos); + new_col->insertFrom(*src_col.column, selected_row->row_num); + } + executed_block.insert({std::move(new_col), col.type, col.name}); + } + right_col_pos += 1; + } + if (!executed_block) + { + result_column = ColumnUInt8::create(); + break; + } + + for (const auto & col_name : required_column_names) + { + const auto * src_col = added_columns.left_block.findByName(col_name); + if (!src_col) + continue; + auto new_col = src_col->column->cloneEmpty(); + size_t prev_left_offset = 0; + for (size_t i = 1; i < row_replicate_offset.size(); ++i) + { + const size_t & left_offset = row_replicate_offset[i]; + size_t rows = left_offset - prev_left_offset; + if (rows) + new_col->insertManyFrom(*src_col->column, left_start_row + i - 1, rows); + prev_left_offset = left_offset; + } + executed_block.insert({std::move(new_col), src_col->type, col_name}); + } + if (!executed_block) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "required columns: [{}], but not found any in left/right table. right table: {}, left table: {}", + required_cols.toString(), + sample_right_block.dumpNames(), + added_columns.left_block.dumpNames()); + } + + for (const auto & col : executed_block.getColumnsWithTypeAndName()) + if (!col.column || !col.type) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal nullptr column in input block: {}", executed_block.dumpStructure()); + + added_columns.additional_filter_expression->execute(executed_block); + result_column = executed_block.getByPosition(0).column->convertToFullColumnIfConst(); + executed_block.clear(); + } while (false); + + result_column = result_column->convertToFullIfNeeded(); + if (result_column->isNullable()) + { + /// Convert Nullable(UInt8) to UInt8 ensuring that nulls are zeros + /// Trying to avoid copying data, since we are the only owner of the column. + ColumnPtr mask_column = assert_cast(*result_column).getNullMapColumnPtr(); + + MutableColumnPtr mutable_column; + { + ColumnPtr nested_column = assert_cast(*result_column).getNestedColumnPtr(); + result_column.reset(); + mutable_column = IColumn::mutate(std::move(nested_column)); + } + + auto & column_data = assert_cast(*mutable_column).getData(); + const auto & mask_column_data = assert_cast(*mask_column).getData(); + for (size_t i = 0; i < column_data.size(); ++i) + { + if (mask_column_data[i]) + column_data[i] = 0; + } + return mutable_column; + } + return result_column; + } /// First to collect all matched rows refs by join keys, then filter out rows which are not true in additional filter expression. template @@ -141,7 +389,177 @@ private: AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]], bool need_filter [[maybe_unused]], - bool flag_per_row [[maybe_unused]]); + bool need_flags [[maybe_unused]], + bool add_missing [[maybe_unused]], + bool flag_per_row [[maybe_unused]]) + { + size_t left_block_rows = added_columns.rows_to_add; + if (need_filter) + added_columns.filter = IColumn::Filter(left_block_rows, 0); + + std::unique_ptr pool; + + if constexpr (need_replication) + added_columns.offsets_to_replicate = std::make_unique(left_block_rows); + + std::vector row_replicate_offset; + row_replicate_offset.reserve(left_block_rows); + + using FindResult = typename KeyGetter::FindResult; + size_t max_joined_block_rows = added_columns.max_joined_block_rows; + size_t left_row_iter = 0; + PreSelectedRows selected_rows; + selected_rows.reserve(left_block_rows); + std::vector find_results; + find_results.reserve(left_block_rows); + bool exceeded_max_block_rows = false; + IColumn::Offset total_added_rows = 0; + IColumn::Offset current_added_rows = 0; + + auto collect_keys_matched_rows_refs = [&]() + { + pool = std::make_unique(); + find_results.clear(); + row_replicate_offset.clear(); + row_replicate_offset.push_back(0); + current_added_rows = 0; + selected_rows.clear(); + for (; left_row_iter < left_block_rows; ++left_row_iter) + { + if constexpr (need_replication) + { + if (unlikely(total_added_rows + current_added_rows >= max_joined_block_rows)) + { + break; + } + } + KnownRowsHolder all_flag_known_rows; + KnownRowsHolder single_flag_know_rows; + for (size_t join_clause_idx = 0; join_clause_idx < added_columns.join_on_keys.size(); ++join_clause_idx) + { + const auto & join_keys = added_columns.join_on_keys[join_clause_idx]; + if (join_keys.null_map && (*join_keys.null_map)[left_row_iter]) + continue; + + bool row_acceptable = !join_keys.isRowFiltered(left_row_iter); + auto find_result = row_acceptable + ? key_getter_vector[join_clause_idx].findKey(*(mapv[join_clause_idx]), left_row_iter, *pool) + : FindResult(); + + if (find_result.isFound()) + { + auto & mapped = find_result.getMapped(); + find_results.push_back(find_result); + if (flag_per_row) + addFoundRowAll(mapped, selected_rows, current_added_rows, all_flag_known_rows, nullptr); + else + addFoundRowAll(mapped, selected_rows, current_added_rows, single_flag_know_rows, nullptr); + } + } + row_replicate_offset.push_back(current_added_rows); + } + }; + + auto copy_final_matched_rows = [&](size_t left_start_row, ColumnPtr filter_col) + { + const PaddedPODArray & filter_flags = assert_cast(*filter_col).getData(); + + size_t prev_replicated_row = 0; + auto selected_right_row_it = selected_rows.begin(); + size_t find_result_index = 0; + for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i) + { + bool any_matched = false; + /// For all right join, flag_per_row is true, we need mark used flags for each row. + if (flag_per_row) + { + for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) + { + if (filter_flags[replicated_row]) + { + any_matched = true; + added_columns.appendFromBlock(*selected_right_row_it, add_missing); + total_added_rows += 1; + if (need_flags) + used_flags.template setUsed((*selected_right_row_it)->block, (*selected_right_row_it)->row_num, 0); + } + ++selected_right_row_it; + } + } + else + { + for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) + { + if (filter_flags[replicated_row]) + { + any_matched = true; + added_columns.appendFromBlock(*selected_right_row_it, add_missing); + total_added_rows += 1; + } + ++selected_right_row_it; + } + } + if (!any_matched) + { + if (add_missing) + addNotFoundRow(added_columns, total_added_rows); + else + addNotFoundRow(added_columns, total_added_rows); + } + else + { + if (!flag_per_row && need_flags) + used_flags.template setUsed(find_results[find_result_index]); + if (need_filter) + setUsed(added_columns.filter, left_start_row + i - 1); + if (add_missing) + added_columns.applyLazyDefaults(); + } + find_result_index += (prev_replicated_row != row_replicate_offset[i]); + + if constexpr (need_replication) + { + (*added_columns.offsets_to_replicate)[left_start_row + i - 1] = total_added_rows; + } + prev_replicated_row = row_replicate_offset[i]; + } + }; + + while (left_row_iter < left_block_rows && !exceeded_max_block_rows) + { + auto left_start_row = left_row_iter; + collect_keys_matched_rows_refs(); + if (selected_rows.size() != current_added_rows || row_replicate_offset.size() != left_row_iter - left_start_row + 1) + { + throw Exception( + ErrorCodes::LOGICAL_ERROR, + "Sizes are mismatched. selected_rows.size:{}, current_added_rows:{}, row_replicate_offset.size:{}, left_row_iter: {}, " + "left_start_row: {}", + selected_rows.size(), + current_added_rows, + row_replicate_offset.size(), + left_row_iter, + left_start_row); + } + auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns); + copy_final_matched_rows(left_start_row, filter_col); + + if constexpr (need_replication) + { + // Add a check for current_added_rows to avoid run the filter expression on too small size batch. + if (total_added_rows >= max_joined_block_rows || current_added_rows < 1024) + exceeded_max_block_rows = true; + } + } + + if constexpr (need_replication) + { + added_columns.offsets_to_replicate->resize_assume_reserved(left_row_iter); + added_columns.filter.resize_assume_reserved(left_row_iter); + } + added_columns.applyLazyDefaults(); + return left_row_iter; + } /// Cut first num_rows rows from block in place and returns block with remaining rows static Block sliceBlock(Block & block, size_t num_rows); diff --git a/src/Interpreters/IJoin.h b/src/Interpreters/IJoin.h index 7374348da50..8f648de2538 100644 --- a/src/Interpreters/IJoin.h +++ b/src/Interpreters/IJoin.h @@ -115,6 +115,7 @@ public: /// Peek next stream of delayed joined blocks. virtual IBlocksStreamPtr getDelayedBlocks() { return nullptr; } virtual bool hasDelayedBlocks() const { return false; } + virtual void tryRerangeRightTableData() {} virtual IBlocksStreamPtr getNonJoinedBlocks(const Block & left_sample_block, const Block & result_sample_block, UInt64 max_block_size) const = 0; diff --git a/src/Interpreters/RowRefs.h b/src/Interpreters/RowRefs.h index 7c98c47dd11..f8ac68191d6 100644 --- a/src/Interpreters/RowRefs.h +++ b/src/Interpreters/RowRefs.h @@ -123,6 +123,7 @@ struct RowRefList : RowRef RowRefList() {} /// NOLINT RowRefList(const Block * block_, size_t row_num_) : RowRef(block_, row_num_), rows(1) {} + RowRefList(const Block * block_, size_t row_start_, size_t rows_) : RowRef(block_, row_start_), rows(static_cast(rows_)) {} ForwardIterator begin() const { return ForwardIterator(this); } diff --git a/src/Processors/Transforms/JoiningTransform.cpp b/src/Processors/Transforms/JoiningTransform.cpp index ca204bcb482..f2fb6327129 100644 --- a/src/Processors/Transforms/JoiningTransform.cpp +++ b/src/Processors/Transforms/JoiningTransform.cpp @@ -299,13 +299,17 @@ IProcessor::Status FillingRightJoinSideTransform::prepare() void FillingRightJoinSideTransform::work() { - auto block = inputs.front().getHeader().cloneWithColumns(chunk.detachColumns()); + auto & input = inputs.front(); + auto block = input.getHeader().cloneWithColumns(chunk.detachColumns()); if (for_totals) join->setTotals(block); else stop_reading = !join->addBlockToJoin(block); + if (input.isFinished()) + join->tryRerangeRightTableData(); + set_totals = for_totals; } From 29c94195e1830e098b3973b43ff6272012735dac Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Mon, 12 Aug 2024 18:01:40 +0800 Subject: [PATCH 112/162] add setting tests/performance/all_join_opt.xml --- src/Core/SettingsChangesHistory.cpp | 255 +----------- src/Interpreters/HashJoin/HashJoin.cpp | 5 +- src/Interpreters/HashJoin/HashJoinMethods.h | 426 +------------------- tests/performance/all_join_opt.xml | 8 +- 4 files changed, 11 insertions(+), 683 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index b975c6b2fad..9ddf40e87b1 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -95,6 +95,8 @@ static std::initializer_list col >= '2023-01-01' AND col <= '2023-12-31')"}, - {"extract_key_value_pairs_max_pairs_per_row", 0, 0, "Max number of pairs that can be produced by the `extractKeyValuePairs` function. Used as a safeguard against consuming too much memory."}, - {"default_view_definer", "CURRENT_USER", "CURRENT_USER", "Allows to set default `DEFINER` option while creating a view"}, - {"default_materialized_view_sql_security", "DEFINER", "DEFINER", "Allows to set a default value for SQL SECURITY option when creating a materialized view"}, - {"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"}, - {"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - {"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."}, - }}, - {"24.1", {{"print_pretty_type_names", false, true, "Better user experience."}, - {"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"}, - {"output_format_arrow_use_signed_indexes_for_dictionary", false, true, "Use signed indexes type for Arrow dictionaries by default as it's recommended"}, - {"allow_experimental_variant_type", false, false, "Add new experimental Variant type"}, - {"use_variant_as_common_type", false, false, "Allow to use Variant in if/multiIf if there is no common type"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"parallel_replicas_mark_segment_size", 128, 128, "Add new setting to control segment size in new parallel replicas coordinator implementation"}, - {"ignore_materialized_views_with_dropped_target_table", false, false, "Add new setting to allow to ignore materialized views with dropped target table"}, - {"output_format_compression_level", 3, 3, "Allow to change compression level in the query output"}, - {"output_format_compression_zstd_window_log", 0, 0, "Allow to change zstd window log in the query output when zstd compression is used"}, - {"enable_zstd_qat_codec", false, false, "Add new ZSTD_QAT codec"}, - {"enable_vertical_final", false, true, "Use vertical final by default"}, - {"output_format_arrow_use_64_bit_indexes_for_dictionary", false, false, "Allow to use 64 bit indexes type in Arrow dictionaries"}, - {"max_rows_in_set_to_optimize_join", 100000, 0, "Disable join optimization as it prevents from read in order optimization"}, - {"output_format_pretty_color", true, "auto", "Setting is changed to allow also for auto value, disabling ANSI escapes if output is not a tty"}, - {"function_visible_width_behavior", 0, 1, "We changed the default behavior of `visibleWidth` to be more precise"}, - {"max_estimated_execution_time", 0, 0, "Separate max_execution_time and max_estimated_execution_time"}, - {"iceberg_engine_ignore_schema_evolution", false, false, "Allow to ignore schema evolution in Iceberg table engine"}, - {"optimize_injective_functions_in_group_by", false, true, "Replace injective functions by it's arguments in GROUP BY section in analyzer"}, - {"update_insert_deduplication_token_in_dependent_materialized_views", false, false, "Allow to update insert deduplication token with table identifier during insert in dependent materialized views"}, - {"azure_max_unexpected_write_error_retries", 4, 4, "The maximum number of retries in case of unexpected errors during Azure blob storage write"}, - {"split_parts_ranges_into_intersecting_and_non_intersecting_final", false, true, "Allow to split parts ranges into intersecting and non intersecting during FINAL optimization"}, - {"split_intersecting_parts_ranges_into_layers_final", true, true, "Allow to split intersecting parts ranges into layers during FINAL optimization"}}}, - {"23.12", {{"allow_suspicious_ttl_expressions", true, false, "It is a new setting, and in previous versions the behavior was equivalent to allowing."}, - {"input_format_parquet_allow_missing_columns", false, true, "Allow missing columns in Parquet files by default"}, - {"input_format_orc_allow_missing_columns", false, true, "Allow missing columns in ORC files by default"}, - {"input_format_arrow_allow_missing_columns", false, true, "Allow missing columns in Arrow files by default"}}}, - {"23.11", {{"parsedatetime_parse_without_leading_zeros", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.9", {{"optimize_group_by_constant_keys", false, true, "Optimize group by constant keys by default"}, - {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, - {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, - {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, - {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}, - {"input_format_json_try_infer_numbers_from_strings", true, false, "Don't infer numbers from strings in JSON formats by default to prevent possible parsing errors"}, - {"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}}, - {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, - {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, - {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, - {"http_receive_timeout", 180, 30, "See http_send_timeout."}}}, - {"23.5", {{"input_format_parquet_preserve_order", true, false, "Allow Parquet reader to reorder rows for better parallelism."}, - {"parallelize_output_from_storages", false, true, "Allow parallelism when executing queries that read from file/url/s3/etc. This may reorder rows."}, - {"use_with_fill_by_sorting_prefix", false, true, "Columns preceding WITH FILL columns in ORDER BY clause form sorting prefix. Rows with different values in sorting prefix are filled independently"}, - {"output_format_parquet_compliant_nested_types", false, true, "Change an internal field name in output Parquet file schema."}}}, - {"23.4", {{"allow_suspicious_indices", true, false, "If true, index can defined with identical expressions"}, - {"allow_nonconst_timezone_arguments", true, false, "Allow non-const timezone arguments in certain time-related functions like toTimeZone(), fromUnixTimestamp*(), snowflakeToDateTime*()."}, - {"connect_timeout_with_failover_ms", 50, 1000, "Increase default connect timeout because of async connect"}, - {"connect_timeout_with_failover_secure_ms", 100, 1000, "Increase default secure connect timeout because of async connect"}, - {"hedged_connection_timeout_ms", 100, 50, "Start new connection in hedged requests after 50 ms instead of 100 to correspond with previous connect timeout"}, - {"formatdatetime_f_prints_single_zero", true, false, "Improved compatibility with MySQL DATE_FORMAT()/STR_TO_DATE()"}, - {"formatdatetime_parsedatetime_m_is_month_name", false, true, "Improved compatibility with MySQL DATE_FORMAT/STR_TO_DATE"}}}, - {"23.3", {{"output_format_parquet_version", "1.0", "2.latest", "Use latest Parquet format version for output format"}, - {"input_format_json_ignore_unknown_keys_in_named_tuple", false, true, "Improve parsing JSON objects as named tuples"}, - {"input_format_native_allow_types_conversion", false, true, "Allow types conversion in Native input forma"}, - {"output_format_arrow_compression_method", "none", "lz4_frame", "Use lz4 compression in Arrow output format by default"}, - {"output_format_parquet_compression_method", "snappy", "lz4", "Use lz4 compression in Parquet output format by default"}, - {"output_format_orc_compression_method", "none", "lz4_frame", "Use lz4 compression in ORC output format by default"}, - {"async_query_sending_for_remote", false, true, "Create connections and send query async across shards"}}}, - {"23.2", {{"output_format_parquet_fixed_string_as_fixed_byte_array", false, true, "Use Parquet FIXED_LENGTH_BYTE_ARRAY type for FixedString by default"}, - {"output_format_arrow_fixed_string_as_fixed_byte_array", false, true, "Use Arrow FIXED_SIZE_BINARY type for FixedString by default"}, - {"query_plan_remove_redundant_distinct", false, true, "Remove redundant Distinct step in query plan"}, - {"optimize_duplicate_order_by_and_distinct", true, false, "Remove duplicate ORDER BY and DISTINCT if it's possible"}, - {"insert_keeper_max_retries", 0, 20, "Enable reconnections to Keeper on INSERT, improve reliability"}}}, - {"23.1", {{"input_format_json_read_objects_as_strings", 0, 1, "Enable reading nested json objects as strings while object type is experimental"}, - {"input_format_json_defaults_for_missing_elements_in_named_tuple", false, true, "Allow missing elements in JSON objects while reading named tuples by default"}, - {"input_format_csv_detect_header", false, true, "Detect header in CSV format by default"}, - {"input_format_tsv_detect_header", false, true, "Detect header in TSV format by default"}, - {"input_format_custom_detect_header", false, true, "Detect header in CustomSeparated format by default"}, - {"query_plan_remove_redundant_sorting", false, true, "Remove redundant sorting in query plan. For example, sorting steps related to ORDER BY clauses in subqueries"}}}, - {"22.12", {{"max_size_to_preallocate_for_aggregation", 10'000'000, 100'000'000, "This optimizes performance"}, - {"query_plan_aggregation_in_order", 0, 1, "Enable some refactoring around query plan"}, - {"format_binary_max_string_size", 0, 1_GiB, "Prevent allocating large amount of memory"}}}, - {"22.11", {{"use_structure_from_insertion_table_in_table_functions", 0, 2, "Improve using structure from insertion table in table functions"}}}, - {"22.9", {{"force_grouping_standard_compatibility", false, true, "Make GROUPING function output the same as in SQL standard and other DBMS"}}}, - {"22.7", {{"cross_to_inner_join_rewrite", 1, 2, "Force rewrite comma join to inner"}, - {"enable_positional_arguments", false, true, "Enable positional arguments feature by default"}, - {"format_csv_allow_single_quotes", true, false, "Most tools don't treat single quote in CSV specially, don't do it by default too"}}}, - {"22.6", {{"output_format_json_named_tuples_as_objects", false, true, "Allow to serialize named tuples as JSON objects in JSON formats by default"}, - {"input_format_skip_unknown_fields", false, true, "Optimize reading subset of columns for some input formats"}}}, - {"22.5", {{"memory_overcommit_ratio_denominator", 0, 1073741824, "Enable memory overcommit feature by default"}, - {"memory_overcommit_ratio_denominator_for_user", 0, 1073741824, "Enable memory overcommit feature by default"}}}, - {"22.4", {{"allow_settings_after_format_in_insert", true, false, "Do not allow SETTINGS after FORMAT for INSERT queries because ClickHouse interpret SETTINGS as some values, which is misleading"}}}, - {"22.3", {{"cast_ipv4_ipv6_default_on_conversion_error", true, false, "Make functions cast(value, 'IPv4') and cast(value, 'IPv6') behave same as toIPv4 and toIPv6 functions"}}}, - {"21.12", {{"stream_like_engine_allow_direct_select", true, false, "Do not allow direct select for Kafka/RabbitMQ/FileLog by default"}}}, - {"21.9", {{"output_format_decimal_trailing_zeros", true, false, "Do not output trailing zeros in text representation of Decimal types by default for better looking output"}, - {"use_hedged_requests", false, true, "Enable Hedged Requests feature by default"}}}, - {"21.7", {{"legacy_column_name_of_tuple_literal", true, false, "Add this setting only for compatibility reasons. It makes sense to set to 'true', while doing rolling update of cluster from version lower than 21.7 to higher"}}}, - {"21.5", {{"async_socket_for_remote", false, true, "Fix all problems and turn on asynchronous reads from socket for remote queries by default again"}}}, - {"21.3", {{"async_socket_for_remote", true, false, "Turn off asynchronous reads from socket for remote queries because of some problems"}, - {"optimize_normalize_count_variants", false, true, "Rewrite aggregate functions that semantically equals to count() as count() by default"}, - {"normalize_function_names", false, true, "Normalize function names to their canonical names, this was needed for projection query routing"}}}, - {"21.2", {{"enable_global_with_statement", false, true, "Propagate WITH statements to UNION queries and all subqueries by default"}}}, - {"21.1", {{"insert_quorum_parallel", false, true, "Use parallel quorum inserts by default. It is significantly more convenient to use than sequential quorum inserts"}, - {"input_format_null_as_default", false, true, "Allow to insert NULL as default for input formats by default"}, - {"optimize_on_insert", false, true, "Enable data optimization on INSERT by default for better user experience"}, - {"use_compact_format_in_distributed_parts_names", false, true, "Use compact format for async INSERT into Distributed tables by default"}}}, - {"20.10", {{"format_regexp_escaping_rule", "Escaped", "Raw", "Use Raw as default escaping rule for Regexp format to male the behaviour more like to what users expect"}}}, - {"20.7", {{"show_table_uuid_in_table_create_query_if_not_nil", true, false, "Stop showing UID of the table in its CREATE query for Engine=Atomic"}}}, - {"20.5", {{"input_format_with_names_use_header", false, true, "Enable using header with names for formats with WithNames/WithNamesAndTypes suffixes"}, - {"allow_suspicious_codecs", true, false, "Don't allow to specify meaningless compression codecs"}}}, - {"20.4", {{"validate_polygons", false, true, "Throw exception if polygon is invalid in function pointInPolygon by default instead of returning possibly wrong results"}}}, - {"19.18", {{"enable_scalar_subquery_optimization", false, true, "Prevent scalar subqueries from (de)serializing large scalar values and possibly avoid running the same subquery more than once"}}}, - {"19.14", {{"any_join_distinct_right_table_keys", true, false, "Disable ANY RIGHT and ANY FULL JOINs by default to avoid inconsistency"}}}, - {"19.12", {{"input_format_defaults_for_omitted_fields", false, true, "Enable calculation of complex default expressions for omitted fields for some input formats, because it should be the expected behaviour"}}}, - {"19.5", {{"max_partitions_per_insert_block", 0, 100, "Add a limit for the number of partitions in one block"}}}, - {"18.12.17", {{"enable_optimize_predicate_expression", 0, 1, "Optimize predicates to subqueries by default"}}}, }; diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 6f332118f8a..e394b9913b5 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -1363,7 +1363,7 @@ bool HashJoin::needUsedFlagsForPerRightTableRow(std::shared_ptr table template void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]]) { - constexpr JoinFeatures join_features; + constexpr JoinFeatures join_features; if constexpr (join_features.is_all_join && (join_features.left || join_features.inner)) { auto merge_rows_into_one_block = [&](BlocksList & blocks, RowRefList & rows_ref) @@ -1433,13 +1433,12 @@ void HashJoin::tryRerangeRightTableData() data->rows_to_join, data->keys_to_join, sample_block_with_columns_to_add.columns()); return; } - std::cout << "sort right table rows" << std::endl; joinDispatch( kind, strictness, data->maps.front(), + false, [&](auto kind_, auto strictness_, auto & map_) { tryRerangeRightTableDataImpl(map_); }); - std::cout << "sort right finished" << std::endl; data->sorted = true; } diff --git a/src/Interpreters/HashJoin/HashJoinMethods.h b/src/Interpreters/HashJoin/HashJoinMethods.h index 9d94c3f62c2..c5b54a62f36 100644 --- a/src/Interpreters/HashJoin/HashJoinMethods.h +++ b/src/Interpreters/HashJoin/HashJoinMethods.h @@ -83,7 +83,6 @@ public: const Block & block_with_columns_to_add, const MapsTemplateVector & maps_, bool is_join_get = false); - private: template static KeyGetter createKeyGetter(const ColumnRawPtrs & key_columns, const Sizes & key_sizes); @@ -121,142 +120,7 @@ private: std::vector && key_getter_vector, const std::vector & mapv, AddedColumns & added_columns, - JoinStuff::JoinUsedFlags & used_flags) - { - constexpr JoinFeatures join_features; - - size_t rows = added_columns.rows_to_add; - if constexpr (need_filter) - added_columns.filter = IColumn::Filter(rows, 0); - if constexpr (!flag_per_row && (STRICTNESS == JoinStrictness::All || (STRICTNESS == JoinStrictness::Semi && KIND == JoinKind::Right))) - added_columns.output_by_row_list = true; - - Arena pool; - - if constexpr (join_features.need_replication) - added_columns.offsets_to_replicate = std::make_unique(rows); - - IColumn::Offset current_offset = 0; - size_t max_joined_block_rows = added_columns.max_joined_block_rows; - size_t i = 0; - for (; i < rows; ++i) - { - if constexpr (join_features.need_replication) - { - if (unlikely(current_offset >= max_joined_block_rows)) - { - added_columns.offsets_to_replicate->resize_assume_reserved(i); - added_columns.filter.resize_assume_reserved(i); - break; - } - } - - bool right_row_found = false; - - KnownRowsHolder known_rows; - for (size_t onexpr_idx = 0; onexpr_idx < added_columns.join_on_keys.size(); ++onexpr_idx) - { - const auto & join_keys = added_columns.join_on_keys[onexpr_idx]; - if (join_keys.null_map && (*join_keys.null_map)[i]) - continue; - - bool row_acceptable = !join_keys.isRowFiltered(i); - using FindResult = typename KeyGetter::FindResult; - auto find_result = row_acceptable ? key_getter_vector[onexpr_idx].findKey(*(mapv[onexpr_idx]), i, pool) : FindResult(); - - if (find_result.isFound()) - { - right_row_found = true; - auto & mapped = find_result.getMapped(); - if constexpr (join_features.is_asof_join) - { - const IColumn & left_asof_key = added_columns.leftAsofKey(); - - auto row_ref = mapped->findAsof(left_asof_key, i); - if (row_ref && row_ref->block) - { - setUsed(added_columns.filter, i); - if constexpr (flag_per_row) - used_flags.template setUsed(row_ref->block, row_ref->row_num, 0); - else - used_flags.template setUsed(find_result); - - added_columns.appendFromBlock(row_ref, join_features.add_missing); - } - else - addNotFoundRow(added_columns, current_offset); - } - else if constexpr (join_features.is_all_join) - { - setUsed(added_columns.filter, i); - used_flags.template setUsed(find_result); - auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; - addFoundRowAll(mapped, added_columns, current_offset, known_rows, used_flags_opt); - } - else if constexpr ((join_features.is_any_join || join_features.is_semi_join) && join_features.right) - { - /// Use first appeared left key + it needs left columns replication - bool used_once = used_flags.template setUsedOnce(find_result); - if (used_once) - { - auto used_flags_opt = join_features.need_flags ? &used_flags : nullptr; - setUsed(added_columns.filter, i); - addFoundRowAll( - mapped, added_columns, current_offset, known_rows, used_flags_opt); - } - } - else if constexpr (join_features.is_any_join && KIND == JoinKind::Inner) - { - bool used_once = used_flags.template setUsedOnce(find_result); - - /// Use first appeared left key only - if (used_once) - { - setUsed(added_columns.filter, i); - added_columns.appendFromBlock(&mapped, join_features.add_missing); - } - - break; - } - else if constexpr (join_features.is_any_join && join_features.full) - { - /// TODO - } - else if constexpr (join_features.is_anti_join) - { - if constexpr (join_features.right && join_features.need_flags) - used_flags.template setUsed(find_result); - } - else /// ANY LEFT, SEMI LEFT, old ANY (RightAny) - { - setUsed(added_columns.filter, i); - used_flags.template setUsed(find_result); - added_columns.appendFromBlock(&mapped, join_features.add_missing); - - if (join_features.is_any_or_semi_join) - { - break; - } - } - } - } - - if (!right_row_found) - { - if constexpr (join_features.is_anti_join && join_features.left) - setUsed(added_columns.filter, i); - addNotFoundRow(added_columns, current_offset); - } - - if constexpr (join_features.need_replication) - { - (*added_columns.offsets_to_replicate)[i] = current_offset; - } - } - - added_columns.applyLazyDefaults(); - return i; - } + JoinStuff::JoinUsedFlags & used_flags); template static void setUsed(IColumn::Filter & filter [[maybe_unused]], size_t pos [[maybe_unused]]); @@ -266,120 +130,7 @@ private: size_t left_start_row, const std::vector & selected_rows, const std::vector & row_replicate_offset, - AddedColumns & added_columns) - { - ColumnPtr result_column; - do - { - if (selected_rows.empty()) - { - result_column = ColumnUInt8::create(); - break; - } - const Block & sample_right_block = *((*selected_rows.begin())->block); - if (!sample_right_block || !added_columns.additional_filter_expression) - { - auto filter = ColumnUInt8::create(); - filter->insertMany(1, selected_rows.size()); - result_column = std::move(filter); - break; - } - - auto required_cols = added_columns.additional_filter_expression->getRequiredColumnsWithTypes(); - if (required_cols.empty()) - { - Block block; - added_columns.additional_filter_expression->execute(block); - result_column = block.getByPosition(0).column->cloneResized(selected_rows.size()); - break; - } - NameSet required_column_names; - for (auto & col : required_cols) - required_column_names.insert(col.name); - - Block executed_block; - size_t right_col_pos = 0; - for (const auto & col : sample_right_block.getColumnsWithTypeAndName()) - { - if (required_column_names.contains(col.name)) - { - auto new_col = col.column->cloneEmpty(); - for (const auto & selected_row : selected_rows) - { - const auto & src_col = selected_row->block->getByPosition(right_col_pos); - new_col->insertFrom(*src_col.column, selected_row->row_num); - } - executed_block.insert({std::move(new_col), col.type, col.name}); - } - right_col_pos += 1; - } - if (!executed_block) - { - result_column = ColumnUInt8::create(); - break; - } - - for (const auto & col_name : required_column_names) - { - const auto * src_col = added_columns.left_block.findByName(col_name); - if (!src_col) - continue; - auto new_col = src_col->column->cloneEmpty(); - size_t prev_left_offset = 0; - for (size_t i = 1; i < row_replicate_offset.size(); ++i) - { - const size_t & left_offset = row_replicate_offset[i]; - size_t rows = left_offset - prev_left_offset; - if (rows) - new_col->insertManyFrom(*src_col->column, left_start_row + i - 1, rows); - prev_left_offset = left_offset; - } - executed_block.insert({std::move(new_col), src_col->type, col_name}); - } - if (!executed_block) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "required columns: [{}], but not found any in left/right table. right table: {}, left table: {}", - required_cols.toString(), - sample_right_block.dumpNames(), - added_columns.left_block.dumpNames()); - } - - for (const auto & col : executed_block.getColumnsWithTypeAndName()) - if (!col.column || !col.type) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Illegal nullptr column in input block: {}", executed_block.dumpStructure()); - - added_columns.additional_filter_expression->execute(executed_block); - result_column = executed_block.getByPosition(0).column->convertToFullColumnIfConst(); - executed_block.clear(); - } while (false); - - result_column = result_column->convertToFullIfNeeded(); - if (result_column->isNullable()) - { - /// Convert Nullable(UInt8) to UInt8 ensuring that nulls are zeros - /// Trying to avoid copying data, since we are the only owner of the column. - ColumnPtr mask_column = assert_cast(*result_column).getNullMapColumnPtr(); - - MutableColumnPtr mutable_column; - { - ColumnPtr nested_column = assert_cast(*result_column).getNestedColumnPtr(); - result_column.reset(); - mutable_column = IColumn::mutate(std::move(nested_column)); - } - - auto & column_data = assert_cast(*mutable_column).getData(); - const auto & mask_column_data = assert_cast(*mask_column).getData(); - for (size_t i = 0; i < column_data.size(); ++i) - { - if (mask_column_data[i]) - column_data[i] = 0; - } - return mutable_column; - } - return result_column; - } + AddedColumns & added_columns); /// First to collect all matched rows refs by join keys, then filter out rows which are not true in additional filter expression. template @@ -389,177 +140,7 @@ private: AddedColumns & added_columns, JoinStuff::JoinUsedFlags & used_flags [[maybe_unused]], bool need_filter [[maybe_unused]], - bool need_flags [[maybe_unused]], - bool add_missing [[maybe_unused]], - bool flag_per_row [[maybe_unused]]) - { - size_t left_block_rows = added_columns.rows_to_add; - if (need_filter) - added_columns.filter = IColumn::Filter(left_block_rows, 0); - - std::unique_ptr pool; - - if constexpr (need_replication) - added_columns.offsets_to_replicate = std::make_unique(left_block_rows); - - std::vector row_replicate_offset; - row_replicate_offset.reserve(left_block_rows); - - using FindResult = typename KeyGetter::FindResult; - size_t max_joined_block_rows = added_columns.max_joined_block_rows; - size_t left_row_iter = 0; - PreSelectedRows selected_rows; - selected_rows.reserve(left_block_rows); - std::vector find_results; - find_results.reserve(left_block_rows); - bool exceeded_max_block_rows = false; - IColumn::Offset total_added_rows = 0; - IColumn::Offset current_added_rows = 0; - - auto collect_keys_matched_rows_refs = [&]() - { - pool = std::make_unique(); - find_results.clear(); - row_replicate_offset.clear(); - row_replicate_offset.push_back(0); - current_added_rows = 0; - selected_rows.clear(); - for (; left_row_iter < left_block_rows; ++left_row_iter) - { - if constexpr (need_replication) - { - if (unlikely(total_added_rows + current_added_rows >= max_joined_block_rows)) - { - break; - } - } - KnownRowsHolder all_flag_known_rows; - KnownRowsHolder single_flag_know_rows; - for (size_t join_clause_idx = 0; join_clause_idx < added_columns.join_on_keys.size(); ++join_clause_idx) - { - const auto & join_keys = added_columns.join_on_keys[join_clause_idx]; - if (join_keys.null_map && (*join_keys.null_map)[left_row_iter]) - continue; - - bool row_acceptable = !join_keys.isRowFiltered(left_row_iter); - auto find_result = row_acceptable - ? key_getter_vector[join_clause_idx].findKey(*(mapv[join_clause_idx]), left_row_iter, *pool) - : FindResult(); - - if (find_result.isFound()) - { - auto & mapped = find_result.getMapped(); - find_results.push_back(find_result); - if (flag_per_row) - addFoundRowAll(mapped, selected_rows, current_added_rows, all_flag_known_rows, nullptr); - else - addFoundRowAll(mapped, selected_rows, current_added_rows, single_flag_know_rows, nullptr); - } - } - row_replicate_offset.push_back(current_added_rows); - } - }; - - auto copy_final_matched_rows = [&](size_t left_start_row, ColumnPtr filter_col) - { - const PaddedPODArray & filter_flags = assert_cast(*filter_col).getData(); - - size_t prev_replicated_row = 0; - auto selected_right_row_it = selected_rows.begin(); - size_t find_result_index = 0; - for (size_t i = 1, n = row_replicate_offset.size(); i < n; ++i) - { - bool any_matched = false; - /// For all right join, flag_per_row is true, we need mark used flags for each row. - if (flag_per_row) - { - for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) - { - if (filter_flags[replicated_row]) - { - any_matched = true; - added_columns.appendFromBlock(*selected_right_row_it, add_missing); - total_added_rows += 1; - if (need_flags) - used_flags.template setUsed((*selected_right_row_it)->block, (*selected_right_row_it)->row_num, 0); - } - ++selected_right_row_it; - } - } - else - { - for (size_t replicated_row = prev_replicated_row; replicated_row < row_replicate_offset[i]; ++replicated_row) - { - if (filter_flags[replicated_row]) - { - any_matched = true; - added_columns.appendFromBlock(*selected_right_row_it, add_missing); - total_added_rows += 1; - } - ++selected_right_row_it; - } - } - if (!any_matched) - { - if (add_missing) - addNotFoundRow(added_columns, total_added_rows); - else - addNotFoundRow(added_columns, total_added_rows); - } - else - { - if (!flag_per_row && need_flags) - used_flags.template setUsed(find_results[find_result_index]); - if (need_filter) - setUsed(added_columns.filter, left_start_row + i - 1); - if (add_missing) - added_columns.applyLazyDefaults(); - } - find_result_index += (prev_replicated_row != row_replicate_offset[i]); - - if constexpr (need_replication) - { - (*added_columns.offsets_to_replicate)[left_start_row + i - 1] = total_added_rows; - } - prev_replicated_row = row_replicate_offset[i]; - } - }; - - while (left_row_iter < left_block_rows && !exceeded_max_block_rows) - { - auto left_start_row = left_row_iter; - collect_keys_matched_rows_refs(); - if (selected_rows.size() != current_added_rows || row_replicate_offset.size() != left_row_iter - left_start_row + 1) - { - throw Exception( - ErrorCodes::LOGICAL_ERROR, - "Sizes are mismatched. selected_rows.size:{}, current_added_rows:{}, row_replicate_offset.size:{}, left_row_iter: {}, " - "left_start_row: {}", - selected_rows.size(), - current_added_rows, - row_replicate_offset.size(), - left_row_iter, - left_start_row); - } - auto filter_col = buildAdditionalFilter(left_start_row, selected_rows, row_replicate_offset, added_columns); - copy_final_matched_rows(left_start_row, filter_col); - - if constexpr (need_replication) - { - // Add a check for current_added_rows to avoid run the filter expression on too small size batch. - if (total_added_rows >= max_joined_block_rows || current_added_rows < 1024) - exceeded_max_block_rows = true; - } - } - - if constexpr (need_replication) - { - added_columns.offsets_to_replicate->resize_assume_reserved(left_row_iter); - added_columns.filter.resize_assume_reserved(left_row_iter); - } - added_columns.applyLazyDefaults(); - return left_row_iter; - } + bool flag_per_row [[maybe_unused]]); /// Cut first num_rows rows from block in place and returns block with remaining rows static Block sliceBlock(Block & block, size_t num_rows); @@ -617,4 +198,3 @@ extern template class HashJoinMethods; extern template class HashJoinMethods; } - diff --git a/tests/performance/all_join_opt.xml b/tests/performance/all_join_opt.xml index 0ab9c39f67c..2ecd76ee976 100644 --- a/tests/performance/all_join_opt.xml +++ b/tests/performance/all_join_opt.xml @@ -5,10 +5,10 @@ INSERT INTO test SELECT number % 10000, number % 10000, number % 10000 FROM numbers(10000000) INSERT INTO test1 SELECT number % 1000 , number % 1000, number % 1000 FROM numbers(100000) - SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b - SELECT MAX(test1.a) FROM test LEFT JOIN test1 on test.b = test1.b - SELECT MAX(test1.a) FROM test RIGHT JOIN test1 on test.b = test1.b - SELECT MAX(test1.a) FROM test FULL JOIN test1 on test.b = test1.b + SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b SETTINGS join_to_sort_table_rows_threshold=100000 + SELECT MAX(test1.a) FROM test LEFT JOIN test1 on test.b = test1.b SETTINGS join_to_sort_table_rows_threshold=100000 + SELECT MAX(test1.a) FROM test RIGHT JOIN test1 on test.b = test1.b SETTINGS join_to_sort_table_rows_threshold=100000 + SELECT MAX(test1.a) FROM test FULL JOIN test1 on test.b = test1.b SETTINGS join_to_sort_table_rows_threshold=100000 DROP TABLE IF EXISTS test DROP TABLE IF EXISTS test1 From b8e967ff9c5b03dcf9376ad316f0a73af133cb90 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Tue, 13 Aug 2024 09:41:19 +0800 Subject: [PATCH 113/162] add allowReadCaseInsentitive func --- src/Processors/Formats/IRowInputFormat.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/Processors/Formats/IRowInputFormat.h b/src/Processors/Formats/IRowInputFormat.h index f8796df8604..07014bec452 100644 --- a/src/Processors/Formats/IRowInputFormat.h +++ b/src/Processors/Formats/IRowInputFormat.h @@ -68,6 +68,8 @@ protected: virtual bool allowSyncAfterError() const { return false; } virtual void syncAfterError(); + virtual bool allReadColumnCaseInsensitive() const { return false; } + /// In case of parse error, try to roll back and parse last one or two rows very carefully /// and collect as much as possible diagnostic information about error. /// If not implemented, returns empty string. From cfa4ca6fb122580b98e4f4630dc14fb047ba6ccb Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Tue, 13 Aug 2024 09:46:53 +0800 Subject: [PATCH 114/162] remove useless code --- src/Processors/Formats/IRowInputFormat.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/src/Processors/Formats/IRowInputFormat.h b/src/Processors/Formats/IRowInputFormat.h index 07014bec452..f8796df8604 100644 --- a/src/Processors/Formats/IRowInputFormat.h +++ b/src/Processors/Formats/IRowInputFormat.h @@ -68,8 +68,6 @@ protected: virtual bool allowSyncAfterError() const { return false; } virtual void syncAfterError(); - virtual bool allReadColumnCaseInsensitive() const { return false; } - /// In case of parse error, try to roll back and parse last one or two rows very carefully /// and collect as much as possible diagnostic information about error. /// If not implemented, returns empty string. From add486b62a45a615b1d1c2ee08a945d08b984943 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Tue, 20 Aug 2024 17:33:08 +0800 Subject: [PATCH 115/162] rebase and reslove conflict --- src/Core/SettingsChangesHistory.cpp | 2 +- src/Interpreters/HashJoin/AddedColumns.cpp | 82 +++++----------------- src/Interpreters/HashJoin/AddedColumns.h | 8 +-- src/Interpreters/HashJoin/HashJoin.cpp | 4 +- src/Interpreters/TableJoin.cpp | 2 + src/Interpreters/TableJoin.h | 4 ++ tests/performance/all_join_opt.xml | 8 +-- 7 files changed, 33 insertions(+), 77 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 9ddf40e87b1..392f0dbc2ee 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -94,7 +94,7 @@ static std::initializer_list::buildOutput() {} template<> void AddedColumns::buildJoinGetOutput() {} -<<<<<<< HEAD template<> template void AddedColumns::buildOutputFromBlocks() {} -======= ->>>>>>> add threshold for table rows template<> void AddedColumns::buildOutput() @@ -35,15 +32,9 @@ void AddedColumns::buildOutput() buildOutputFromBlocks(); else { -<<<<<<< HEAD if (join_data_avg_perkey_rows < output_by_row_list_threshold) buildOutputFromBlocks(); - else -======= - if (join_data_avg_perkey_rows < sort_right_perkey_rows_threshold) - buildOutputFromBlocks(); else if (join_data_sorted) ->>>>>>> add threshold for table rows { for (size_t i = 0; i < this->size(); ++i) { @@ -53,19 +44,31 @@ void AddedColumns::buildOutput() if (row_ref_i) { const RowRefList * row_ref_list = reinterpret_cast(row_ref_i); -<<<<<<< HEAD - for (auto it = row_ref_list->begin(); it.ok(); ++it) - col->insertFrom(*it->block->getByPosition(right_indexes[i]).column, it->row_num); -======= col->insertRangeFrom(*row_ref_list->block->getByPosition(right_indexes[i]).column, row_ref_list->row_num, row_ref_list->rows); ->>>>>>> add threshold for table rows } else type_name[i].type->insertDefaultInto(*col); } } } -<<<<<<< HEAD + else + { + for (size_t i = 0; i < this->size(); ++i) + { + auto & col = columns[i]; + for (auto row_ref_i : lazy_output.row_refs) + { + if (row_ref_i) + { + const RowRefList * row_ref_list = reinterpret_cast(row_ref_i); + for (auto it = row_ref_list->begin(); it.ok(); ++it) + col->insertFrom(*it->block->getByPosition(right_indexes[i]).column, it->row_num); + } + else + type_name[i].type->insertDefaultInto(*col); + } + } + } } } @@ -88,25 +91,6 @@ void AddedColumns::buildJoinGetOutput() nullable_col->insertFromNotNullable(*column_from_block.column, row_ref->row_num); else col->insertFrom(*column_from_block.column, row_ref->row_num); -======= - else - { - for (size_t i = 0; i < this->size(); ++i) - { - auto & col = columns[i]; - for (auto row_ref_i : lazy_output.row_refs) - { - if (row_ref_i) - { - const RowRefList * row_ref_list = reinterpret_cast(row_ref_i); - for (auto it = row_ref_list->begin(); it.ok(); ++it) - col->insertFrom(*it->block->getByPosition(right_indexes[i]).column, it->row_num); - } - else - type_name[i].type->insertDefaultInto(*col); - } - } ->>>>>>> add threshold for table rows } } } @@ -115,11 +99,7 @@ template<> template void AddedColumns::buildOutputFromBlocks() { -<<<<<<< HEAD if (this->size() == 0) -======= - if (this->size() == 0) ->>>>>>> add threshold for table rows return; std::vector blocks; std::vector row_nums; @@ -160,32 +140,6 @@ void AddedColumns::buildOutputFromBlocks() col->insertFrom(*blocks[j]->getByPosition(right_indexes[i]).column, row_nums[j]); else type_name[i].type->insertDefaultInto(*col); -<<<<<<< HEAD -======= - } - } -} - -template<> -void AddedColumns::buildJoinGetOutput() -{ - for (size_t i = 0; i < this->size(); ++i) - { - auto & col = columns[i]; - for (auto row_ref_i : lazy_output.row_refs) - { - if (!row_ref_i) - { - type_name[i].type->insertDefaultInto(*col); - continue; - } - const auto * row_ref = reinterpret_cast(row_ref_i); - const auto & column_from_block = row_ref->block->getByPosition(right_indexes[i]); - if (auto * nullable_col = typeid_cast(col.get()); nullable_col && !column_from_block.column->isNullable()) - nullable_col->insertFromNotNullable(*column_from_block.column, row_ref->row_num); - else - col->insertFrom(*column_from_block.column, row_ref->row_num); ->>>>>>> add threshold for table rows } } } diff --git a/src/Interpreters/HashJoin/AddedColumns.h b/src/Interpreters/HashJoin/AddedColumns.h index 5ae69fbbf66..3f90b215602 100644 --- a/src/Interpreters/HashJoin/AddedColumns.h +++ b/src/Interpreters/HashJoin/AddedColumns.h @@ -115,6 +115,7 @@ public: } join_data_avg_perkey_rows = join.getJoinedData()->avgPerKeyRows(); output_by_row_list_threshold = join.getTableJoin().outputByRowListPerkeyRowsThreshold(); + join_data_sorted = join.getJoinedData()->sorted; } size_t size() const { return columns.size(); } @@ -147,6 +148,7 @@ public: std::unique_ptr offsets_to_replicate; bool need_filter = false; bool output_by_row_list = false; + bool join_data_sorted = false; size_t join_data_avg_perkey_rows = 0; size_t output_by_row_list_threshold = 0; IColumn::Filter filter; @@ -196,12 +198,6 @@ private: } } - /** Build output from the blocks that extract from `RowRef` or `RowRefList`, to avoid block cache miss which may cause performance slow down. - * And This problem would happen it we directly build output from `RowRef` or `RowRefList`. - */ - template - void buildOutputFromBlocks(); - MutableColumns columns; bool is_join_get; std::vector right_indexes; diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index e394b9913b5..59888d7a71d 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -1422,12 +1422,12 @@ void HashJoin::tryRerangeRightTableData() if ((kind != JoinKind::Inner && kind != JoinKind::Left) || strictness != JoinStrictness::All || table_join->getMixedJoinExpression()) return; - if (!data || data->sorted || data->blocks.empty() || data->maps.size() > 1) + if (!data || data->sorted || data->blocks.empty() || data->maps.size() > 1 || data->rows_to_join > table_join->sortRightTableRowsThreshold() || data->avgPerKeyRows() < table_join->sortRightPerkeyRowsThreshold()) return; if (data->keys_to_join == 0) data->keys_to_join = getTotalRowCount(); - if (sample_block_with_columns_to_add.columns() == 0 || data->rows_to_join > table_join->sortRightTableRowsThreshold() || data->avgPerKeyRows() < table_join->sortRightPerkeyRowsThreshold()) + if (sample_block_with_columns_to_add.columns() == 0) { LOG_DEBUG(log, "The joined right table total rows :{}, total keys :{}, columns added:{}", data->rows_to_join, data->keys_to_join, sample_block_with_columns_to_add.columns()); diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 138085f0710..8bcaef77939 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -116,6 +116,8 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary , max_files_to_merge(settings.join_on_disk_max_files_to_merge) , temporary_files_codec(settings.temporary_files_codec) , output_by_rowlist_perkey_rows_threshold(settings.join_output_by_rowlist_perkey_rows_threshold) + , sort_right_perkey_rows_threshold(settings.join_to_sort_perkey_rows_threshold) + , sort_right_table_rows_threshold(settings.join_to_sort_table_rows_threshold) , max_memory_usage(settings.max_memory_usage) , tmp_volume(tmp_volume_) , tmp_data(tmp_data_) diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 4d626084d81..09d7f0f2b2a 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -149,6 +149,8 @@ private: const size_t max_files_to_merge = 0; const String temporary_files_codec = "LZ4"; const size_t output_by_rowlist_perkey_rows_threshold = 0; + const size_t sort_right_perkey_rows_threshold = 0; + const size_t sort_right_table_rows_threshold = 0; /// Value if setting max_memory_usage for query, can be used when max_bytes_in_join is not specified. size_t max_memory_usage = 0; @@ -297,6 +299,8 @@ public: } size_t outputByRowListPerkeyRowsThreshold() const { return output_by_rowlist_perkey_rows_threshold; } + size_t sortRightPerkeyRowsThreshold() const { return sort_right_perkey_rows_threshold; } + size_t sortRightTableRowsThreshold() const { return sort_right_table_rows_threshold; } size_t defaultMaxBytes() const { return default_max_bytes; } size_t maxJoinedBlockRows() const { return max_joined_block_rows; } size_t maxRowsInRightBlock() const { return partial_merge_join_rows_in_right_blocks; } diff --git a/tests/performance/all_join_opt.xml b/tests/performance/all_join_opt.xml index 2ecd76ee976..0ab9c39f67c 100644 --- a/tests/performance/all_join_opt.xml +++ b/tests/performance/all_join_opt.xml @@ -5,10 +5,10 @@ INSERT INTO test SELECT number % 10000, number % 10000, number % 10000 FROM numbers(10000000) INSERT INTO test1 SELECT number % 1000 , number % 1000, number % 1000 FROM numbers(100000) - SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b SETTINGS join_to_sort_table_rows_threshold=100000 - SELECT MAX(test1.a) FROM test LEFT JOIN test1 on test.b = test1.b SETTINGS join_to_sort_table_rows_threshold=100000 - SELECT MAX(test1.a) FROM test RIGHT JOIN test1 on test.b = test1.b SETTINGS join_to_sort_table_rows_threshold=100000 - SELECT MAX(test1.a) FROM test FULL JOIN test1 on test.b = test1.b SETTINGS join_to_sort_table_rows_threshold=100000 + SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b + SELECT MAX(test1.a) FROM test LEFT JOIN test1 on test.b = test1.b + SELECT MAX(test1.a) FROM test RIGHT JOIN test1 on test.b = test1.b + SELECT MAX(test1.a) FROM test FULL JOIN test1 on test.b = test1.b DROP TABLE IF EXISTS test DROP TABLE IF EXISTS test1 From d12aac7d12de9632e4a97ae7240a0e0dabe5ef59 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 23 Aug 2024 20:11:33 +0000 Subject: [PATCH 116/162] Test 03228_pr_subquery_view_order_by PR + subquery with view and order by --- .../03228_pr_subquery_view_order_by.reference | 20 +++++++++++++++++++ .../03228_pr_subquery_view_order_by.sql | 18 +++++++++++++++++ 2 files changed, 38 insertions(+) create mode 100644 tests/queries/0_stateless/03228_pr_subquery_view_order_by.reference create mode 100644 tests/queries/0_stateless/03228_pr_subquery_view_order_by.sql diff --git a/tests/queries/0_stateless/03228_pr_subquery_view_order_by.reference b/tests/queries/0_stateless/03228_pr_subquery_view_order_by.reference new file mode 100644 index 00000000000..aad720e27cf --- /dev/null +++ b/tests/queries/0_stateless/03228_pr_subquery_view_order_by.reference @@ -0,0 +1,20 @@ +300 +299 +298 +297 +296 +295 +294 +293 +292 +291 +290 +289 +288 +287 +286 +285 +284 +283 +282 +281 diff --git a/tests/queries/0_stateless/03228_pr_subquery_view_order_by.sql b/tests/queries/0_stateless/03228_pr_subquery_view_order_by.sql new file mode 100644 index 00000000000..b85392e0521 --- /dev/null +++ b/tests/queries/0_stateless/03228_pr_subquery_view_order_by.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS view1; +DROP TABLE IF EXISTS table1; +CREATE TABLE table1 (number UInt64) ENGINE=MergeTree ORDER BY number SETTINGS index_granularity=1; +INSERT INTO table1 SELECT number FROM numbers(1, 300); +CREATE VIEW view1 AS SELECT number FROM table1; + +SELECT * +FROM +( + SELECT * + FROM view1 +) +ORDER BY number DESC +LIMIT 20 +SETTINGS cluster_for_parallel_replicas = 'parallel_replicas', allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree = 1, parallel_replicas_local_plan = 1, query_plan_lift_up_union = 0; + +DROP TABLE view1; +DROP TABLE table1; From 3c29f27dd2a1cc8d4649860a9b5021f8784e467b Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 30 Aug 2024 20:04:43 +0000 Subject: [PATCH 117/162] Fix 03228_pr_subquery_view_order_by --- .../QueryPlan/Optimizations/liftUpUnion.cpp | 1 + .../Optimizations/optimizeReadInOrder.cpp | 23 +++++++++++++++++++ .../03228_pr_subquery_view_order_by.sql | 2 +- 3 files changed, 25 insertions(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp index c48551732c9..43cf166002e 100644 --- a/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp +++ b/src/Processors/QueryPlan/Optimizations/liftUpUnion.cpp @@ -50,6 +50,7 @@ size_t tryLiftUpUnion(QueryPlan::Node * parent_node, QueryPlan::Nodes & nodes) expr_node.step = std::make_unique( expr_node.children.front()->step->getOutputStream(), expression->getExpression().clone()); + expr_node.step->setStepDescription(expression->getStepDescription()); } /// - Expression - Something diff --git a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp index 5df7d7b4e82..2eac1896066 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizeReadInOrder.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -899,6 +900,18 @@ AggregationInputOrder buildInputOrderInfo(AggregatingStep & aggregating, QueryPl return {}; } +static bool readingFromParallelReplicas(const QueryPlan::Node * node) +{ + IQueryPlanStep * step = node->step.get(); + while (!node->children.empty()) + { + step = node->children.front()->step.get(); + node = node->children.front(); + } + + return typeid_cast(step); +} + void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) { if (node.children.size() != 1) @@ -924,6 +937,16 @@ void optimizeReadInOrder(QueryPlan::Node & node, QueryPlan::Nodes & nodes) std::vector infos; infos.reserve(node.children.size()); + for (const auto * child : union_node->children) + { + /// in case of parallel replicas + /// avoid applying read-in-order optimization for local replica + /// since it will lead to different parallel replicas modes + /// between local and remote nodes + if (readingFromParallelReplicas(child)) + return; + } + for (auto * child : union_node->children) { infos.push_back(buildInputOrderInfo(*sorting, *child, steps_to_update)); diff --git a/tests/queries/0_stateless/03228_pr_subquery_view_order_by.sql b/tests/queries/0_stateless/03228_pr_subquery_view_order_by.sql index b85392e0521..804a97f737f 100644 --- a/tests/queries/0_stateless/03228_pr_subquery_view_order_by.sql +++ b/tests/queries/0_stateless/03228_pr_subquery_view_order_by.sql @@ -12,7 +12,7 @@ FROM ) ORDER BY number DESC LIMIT 20 -SETTINGS cluster_for_parallel_replicas = 'parallel_replicas', allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree = 1, parallel_replicas_local_plan = 1, query_plan_lift_up_union = 0; +SETTINGS cluster_for_parallel_replicas = 'parallel_replicas', allow_experimental_parallel_reading_from_replicas = 1, max_parallel_replicas = 3, parallel_replicas_for_non_replicated_merge_tree = 1, parallel_replicas_local_plan = 1; DROP TABLE view1; DROP TABLE table1; From 81b8f8594c044b1de6e1b18877e1d9b80401f617 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 30 Aug 2024 21:23:07 +0000 Subject: [PATCH 118/162] Fix tests --- ...imize_distributed_group_by_sharding_key.reference | 4 ++-- .../02496_remove_redundant_sorting.reference | 12 ++++++------ ...02496_remove_redundant_sorting_analyzer.reference | 12 ++++++------ ...2500_remove_redundant_distinct_analyzer.reference | 4 ++-- 4 files changed, 16 insertions(+), 16 deletions(-) diff --git a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference index e786532f25a..a807bf7096e 100644 --- a/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference +++ b/tests/queries/0_stateless/01952_optimize_distributed_group_by_sharding_key.reference @@ -21,7 +21,7 @@ Expression (Projection) Union Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))) ReadFromSystemNumbers - Expression + Expression (Before LIMIT BY) ReadFromRemote (Read from remote replica) explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized Union @@ -96,7 +96,7 @@ Expression (Project names) LimitBy Expression ((Before LIMIT BY + (Projection + (Change column names to column identifiers + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Project names + (Projection + (Change column names to column identifiers + (Project names + (Projection + Change column names to column identifiers))))))))))) ReadFromSystemNumbers - Expression + Expression (Before LIMIT BY) ReadFromRemote (Read from remote replica) explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized Union diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference index 4d004f2f78f..7824fd8cba9 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting.reference @@ -395,9 +395,9 @@ Expression ((Projection + Before ORDER BY)) Union Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 @@ -427,9 +427,9 @@ Expression ((Projection + Before ORDER BY)) Union Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) -- execute Nullable(Float64) 9007199254740994 @@ -459,9 +459,9 @@ Expression ((Projection + Before ORDER BY)) Union Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) - Expression (( + (Conversion before UNION + (Projection + Before ORDER BY)))) + Expression ((Before ORDER BY + (Conversion before UNION + (Projection + Before ORDER BY)))) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 diff --git a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference index dd5ac7bf706..3c68d14fdf2 100644 --- a/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference +++ b/tests/queries/0_stateless/02496_remove_redundant_sorting_analyzer.reference @@ -394,9 +394,9 @@ Expression ((Project names + Projection)) Union Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 @@ -426,9 +426,9 @@ Expression ((Project names + Projection)) Union Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) -- execute Nullable(Float64) 9007199254740994 @@ -458,9 +458,9 @@ Expression ((Project names + Projection)) Union Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) - Expression (( + ( + ( + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) + Expression ((Before ORDER BY + (Projection + (Change column names to column identifiers + (Conversion before UNION + (Project names + (Projection + Change column names to column identifiers))))))) ReadFromStorage (SystemOne) -- execute Float64 9007199254740994 diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference index b79f6310166..27b01cf1158 100644 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct_analyzer.reference @@ -54,7 +54,7 @@ Expression (Project names) Distinct (Preliminary DISTINCT) Expression ((Projection + Change column names to column identifiers)) ReadFromSystemNumbers - Expression (( + ( + Project names))) + Expression ((Projection + (Change column names to column identifiers + Project names))) Distinct (DISTINCT) Distinct (Preliminary DISTINCT) Expression ((Projection + Change column names to column identifiers)) @@ -542,7 +542,7 @@ Expression (Project names) Distinct (Preliminary DISTINCT) Expression ((Projection + Change column names to column identifiers)) ReadFromSystemNumbers - Expression (( + ( + Project names))) + Expression ((Projection + (Change column names to column identifiers + Project names))) Distinct (DISTINCT) Distinct (Preliminary DISTINCT) Expression ((Projection + Change column names to column identifiers)) From bfa27d6650484063b0f8c9b340f5a85b3df2c27d Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Sat, 31 Aug 2024 06:11:50 +0000 Subject: [PATCH 119/162] Fix 02500_remove_redundant_distinct --- .../0_stateless/02500_remove_redundant_distinct.reference | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/queries/0_stateless/02500_remove_redundant_distinct.reference b/tests/queries/0_stateless/02500_remove_redundant_distinct.reference index d7623cd5541..9bb8f4a4017 100644 --- a/tests/queries/0_stateless/02500_remove_redundant_distinct.reference +++ b/tests/queries/0_stateless/02500_remove_redundant_distinct.reference @@ -53,7 +53,7 @@ Expression (Projection) Distinct (Preliminary DISTINCT) Expression (Before ORDER BY) ReadFromSystemNumbers - Expression (( + Projection)) + Expression ((Before ORDER BY + Projection)) Distinct Distinct (Preliminary DISTINCT) Expression (Before ORDER BY) @@ -536,7 +536,7 @@ Expression (Projection) Distinct (Preliminary DISTINCT) Expression (Before ORDER BY) ReadFromSystemNumbers - Expression (( + Projection)) + Expression ((Before ORDER BY + Projection)) Distinct Distinct (Preliminary DISTINCT) Expression (Before ORDER BY) From db8ce31bb785ae6a843089a02396abc2f986003e Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Mon, 2 Sep 2024 14:08:22 +0000 Subject: [PATCH 120/162] Increase flaky check timeout --- tests/ci/ci_config.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/ci/ci_config.py b/tests/ci/ci_config.py index 7b0a7850576..fb4780d48c9 100644 --- a/tests/ci/ci_config.py +++ b/tests/ci/ci_config.py @@ -470,7 +470,7 @@ class CI: JobNames.STATELESS_TEST_FLAKY_ASAN: CommonJobConfigs.STATELESS_TEST.with_properties( required_builds=[BuildNames.PACKAGE_ASAN], pr_only=True, - timeout=3600, + timeout=3 * 3600, # TODO: approach with reference job names does not work because digest may not be calculated if job skipped in wf # reference_job_name=JobNames.STATELESS_TEST_RELEASE, ), From 7e444136bbad7e80f3a1905bbea7fa4c7e9a8337 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 2 Sep 2024 18:27:24 +0200 Subject: [PATCH 121/162] Use QueryPlan for horizontal part of merge --- src/Interpreters/MutationsInterpreter.cpp | 13 +- src/Storages/MergeTree/MergeTask.cpp | 412 ++++++++++++------ .../MergeTree/MergeTreeSequentialSource.cpp | 51 ++- .../MergeTree/MergeTreeSequentialSource.h | 3 + 4 files changed, 334 insertions(+), 145 deletions(-) diff --git a/src/Interpreters/MutationsInterpreter.cpp b/src/Interpreters/MutationsInterpreter.cpp index 0b93b5989b1..a8d45caeeaf 100644 --- a/src/Interpreters/MutationsInterpreter.cpp +++ b/src/Interpreters/MutationsInterpreter.cpp @@ -1217,9 +1217,16 @@ void MutationsInterpreter::Source::read( createReadFromPartStep( MergeTreeSequentialSourceType::Mutation, - plan, *data, storage_snapshot, - part, required_columns, - apply_deleted_mask_, std::move(filter), context_, + plan, + *data, storage_snapshot, + part, + required_columns, + nullptr, + apply_deleted_mask_, + std::move(filter), + false, + false, + context_, getLogger("MutationsInterpreter")); } else diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index fa86bb31629..3bee2ecb0d9 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -38,6 +38,11 @@ #include #include #include +#include +#include +#include +#include +#include #include #include #include @@ -1206,12 +1211,204 @@ bool MergeTask::execute() } +/// Apply merge strategy (Ordinary, Colapsing, Aggregating, etc) to the stream +class ApplyMergeStep : public ITransformingStep /// TODO: is this transformation step? +{ +public: + ApplyMergeStep( + const DataStream & input_stream_, + const SortDescription & sort_description_, + const Names partition_key_columns_, + const MergeTreeData::MergingParams & merging_params_, + WriteBuffer * rows_sources_write_buf_, + UInt64 merge_block_size_rows_, + UInt64 merge_block_size_bytes_, + bool blocks_are_granules_size_, + bool cleanup_) + : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + , sort_description(sort_description_) + , partition_key_columns(partition_key_columns_) + , merging_params(merging_params_) + , rows_sources_write_buf(rows_sources_write_buf_) + , merge_block_size_rows(merge_block_size_rows_) + , merge_block_size_bytes(merge_block_size_bytes_) + , blocks_are_granules_size(blocks_are_granules_size_) + , cleanup(cleanup_) + {} + + String getName() const override { return "ApplyMergePolicy"; } + + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & /*pipelineSettings*/) override + { + /// The order of the streams is important: when the key is matched, the elements go in the order of the source stream number. + /// In the merged part, the lines with the same key must be in the ascending order of the identifier of original part, + /// that is going in insertion order. + ProcessorPtr merged_transform; + +// /// There is no sense to have the block size bigger than one granule for merge operations. +// const UInt64 merge_block_size_rows = data_settings->merge_max_block_size; +// const UInt64 merge_block_size_bytes = data_settings->merge_max_block_size_bytes; + + const auto &header = pipeline.getHeader(); + const auto input_streams_count = pipeline.getNumStreams(); + + switch (merging_params.mode) + { + case MergeTreeData::MergingParams::Ordinary: + merged_transform = std::make_shared( + header, + input_streams_count, + sort_description, + merge_block_size_rows, + merge_block_size_bytes, + SortingQueueStrategy::Default, + /* limit_= */0, + /* always_read_till_end_= */false, + rows_sources_write_buf, + blocks_are_granules_size); + break; + + case MergeTreeData::MergingParams::Collapsing: + merged_transform = std::make_shared( + header, input_streams_count, sort_description, merging_params.sign_column, false, + merge_block_size_rows, merge_block_size_bytes, rows_sources_write_buf, blocks_are_granules_size); + break; + + case MergeTreeData::MergingParams::Summing: + merged_transform = std::make_shared( + header, input_streams_count, sort_description, merging_params.columns_to_sum, partition_key_columns, merge_block_size_rows, merge_block_size_bytes); + break; + + case MergeTreeData::MergingParams::Aggregating: + merged_transform = std::make_shared(header, input_streams_count, sort_description, merge_block_size_rows, merge_block_size_bytes); + break; + + case MergeTreeData::MergingParams::Replacing: + merged_transform = std::make_shared( + header, input_streams_count, sort_description, merging_params.is_deleted_column, merging_params.version_column, + merge_block_size_rows, merge_block_size_bytes, rows_sources_write_buf, blocks_are_granules_size, + cleanup); + break; + + case MergeTreeData::MergingParams::Graphite: + merged_transform = std::make_shared( + header, input_streams_count, sort_description, merge_block_size_rows, merge_block_size_bytes, + merging_params.graphite_params, time_of_merge); + break; + + case MergeTreeData::MergingParams::VersionedCollapsing: + merged_transform = std::make_shared( + header, input_streams_count, sort_description, merging_params.sign_column, + merge_block_size_rows, merge_block_size_bytes, rows_sources_write_buf, blocks_are_granules_size); + break; + } + + pipeline.addTransform(std::move(merged_transform)); + +#ifndef NDEBUG + if (!sort_description.empty()) + { + pipeline.addSimpleTransform([&](const Block & header_) + { + auto transform = std::make_shared(header_, sort_description); + return transform; + }); + } +#endif + } + + void updateOutputStream() override + { + output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); + output_stream->sort_description = sort_description; + + /// TODO: is this correct? +// if (partition_key_columns.empty()) + output_stream->sort_scope = DataStream::SortScope::Global; +// else +// output_stream->sort_scope = DataStream::SortScope::Stream; + } + +private: + SortDescription sort_description; + Names partition_key_columns; + MergeTreeData::MergingParams merging_params{}; + WriteBuffer * rows_sources_write_buf; + const UInt64 merge_block_size_rows; + const UInt64 merge_block_size_bytes; + bool blocks_are_granules_size; + bool cleanup{false}; + time_t time_of_merge{0}; +}; + + +class MaterializingStep : public ITransformingStep /// TODO: is this transformation step? +{ +public: + explicit MaterializingStep( + const DataStream & input_stream_) + : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + {} + + String getName() const override { return "Materializing"; } + + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override + { + pipeline.addTransform(std::make_shared(input_streams.front().header)); + } + + void updateOutputStream() override + { + /// TODO: can this be simplified? + output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); + output_stream->sort_description = input_streams.front().sort_description; + } +}; + + +class TTLStep : public ITransformingStep +{ +public: + TTLStep( + const DataStream & input_stream_, + const ContextPtr & context_, + const MergeTreeData & storage_, + const StorageMetadataPtr & metadata_snapshot_, + const MergeTreeData::MutableDataPartPtr & data_part_, + time_t current_time, + bool force_) + : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + { + transform = std::make_shared(context_, input_stream_.header, storage_, metadata_snapshot_, data_part_, current_time, force_); + subqueries_for_sets = transform->getSubqueries(); + } + + String getName() const override { return "Materializing"; } + + PreparedSets::Subqueries getSubqueries() { return std::move(subqueries_for_sets); } + + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override + { + pipeline.addTransform(transform); + } + + void updateOutputStream() override + { + // TODO: implement? + } + +private: + std::shared_ptr transform; + PreparedSets::Subqueries subqueries_for_sets; +}; + + void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() { /** Read from all parts, merge and write into a new one. * In passing, we calculate expression for sorting. */ - Pipes pipes; + global_ctx->watch_prev_elapsed = 0; /// We count total amount of bytes in parts @@ -1238,143 +1435,92 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() global_ctx->horizontal_stage_progress = std::make_unique( ctx->column_sizes ? ctx->column_sizes->keyColumnsWeight() : 1.0); + auto sorting_key_expression_dag = global_ctx->metadata_snapshot->getSortingKey().expression->getActionsDAG().clone(); + + /// Read from all parts + std::vector plans; for (const auto & part : global_ctx->future_part->parts) { - Pipe pipe = createMergeTreeSequentialSource( + /// TODO: this is just for debugging purposes, remove it later + if (part->getMarksCount() == 0) + LOG_DEBUG(ctx->log, "Part {} is empty", part->name); + + auto plan_for_part = std::make_unique(); + createReadFromPartStep( MergeTreeSequentialSourceType::Merge, + *plan_for_part, *global_ctx->data, global_ctx->storage_snapshot, part, global_ctx->merging_columns.getNames(), - /*mark_ranges=*/ {}, global_ctx->input_rows_filtered, /*apply_deleted_mask=*/ true, + /*filter=*/ std::nullopt, ctx->read_with_direct_io, - /*prefetch=*/ false); + /*prefetch=*/ false, + global_ctx->context, + ctx->log); if (global_ctx->metadata_snapshot->hasSortingKey()) { - pipe.addSimpleTransform([this](const Block & header) - { - return std::make_shared(header, global_ctx->metadata_snapshot->getSortingKey().expression); - }); + /// Calculate sorting key expressions so that they are available for merge sorting. + auto calculate_sorting_key_expression_step = std::make_unique( + plan_for_part->getCurrentDataStream(), + sorting_key_expression_dag.clone()); /// TODO: can we avoid cloning here? + plan_for_part->addStep(std::move(calculate_sorting_key_expression_step)); } - pipes.emplace_back(std::move(pipe)); + plans.emplace_back(std::move(plan_for_part)); } + QueryPlan merge_parts_query_plan; - Names sort_columns = global_ctx->metadata_snapshot->getSortingKeyColumns(); - SortDescription sort_description; - sort_description.compile_sort_description = global_ctx->data->getContext()->getSettingsRef().compile_sort_description; - sort_description.min_count_to_compile_sort_description = global_ctx->data->getContext()->getSettingsRef().min_count_to_compile_sort_description; - - size_t sort_columns_size = sort_columns.size(); - sort_description.reserve(sort_columns_size); - - Names partition_key_columns = global_ctx->metadata_snapshot->getPartitionKey().column_names; - - Block header = pipes.at(0).getHeader(); - for (size_t i = 0; i < sort_columns_size; ++i) - sort_description.emplace_back(sort_columns[i], 1, 1); - -#ifndef NDEBUG - if (!sort_description.empty()) + /// Union of all parts streams { - for (size_t i = 0; i < pipes.size(); ++i) - { - auto & pipe = pipes[i]; - pipe.addSimpleTransform([&](const Block & header_) - { - auto transform = std::make_shared(header_, sort_description); - transform->setDescription(global_ctx->future_part->parts[i]->name); - return transform; - }); - } + DataStreams input_streams; + input_streams.reserve(plans.size()); + for (auto & plan : plans) + input_streams.emplace_back(plan->getCurrentDataStream()); + + auto union_step = std::make_unique(std::move(input_streams)); + merge_parts_query_plan.unitePlans(std::move(union_step), std::move(plans)); } -#endif - /// The order of the streams is important: when the key is matched, the elements go in the order of the source stream number. - /// In the merged part, the lines with the same key must be in the ascending order of the identifier of original part, - /// that is going in insertion order. - ProcessorPtr merged_transform; - - /// If merge is vertical we cannot calculate it - ctx->blocks_are_granules_size = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical); - - /// There is no sense to have the block size bigger than one granule for merge operations. - const UInt64 merge_block_size_rows = data_settings->merge_max_block_size; - const UInt64 merge_block_size_bytes = data_settings->merge_max_block_size_bytes; - - switch (ctx->merging_params.mode) + /// Merge { - case MergeTreeData::MergingParams::Ordinary: - merged_transform = std::make_shared( - header, - pipes.size(), - sort_description, - merge_block_size_rows, - merge_block_size_bytes, - SortingQueueStrategy::Default, - /* limit_= */0, - /* always_read_till_end_= */false, - ctx->rows_sources_write_buf.get(), - ctx->blocks_are_granules_size); - break; + Names sort_columns = global_ctx->metadata_snapshot->getSortingKeyColumns(); + SortDescription sort_description; + sort_description.compile_sort_description = global_ctx->data->getContext()->getSettingsRef().compile_sort_description; + sort_description.min_count_to_compile_sort_description = global_ctx->data->getContext()->getSettingsRef().min_count_to_compile_sort_description; - case MergeTreeData::MergingParams::Collapsing: - merged_transform = std::make_shared( - header, pipes.size(), sort_description, ctx->merging_params.sign_column, false, - merge_block_size_rows, merge_block_size_bytes, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size); - break; + size_t sort_columns_size = sort_columns.size(); + sort_description.reserve(sort_columns_size); - case MergeTreeData::MergingParams::Summing: - merged_transform = std::make_shared( - header, pipes.size(), sort_description, ctx->merging_params.columns_to_sum, partition_key_columns, merge_block_size_rows, merge_block_size_bytes); - break; + Names partition_key_columns = global_ctx->metadata_snapshot->getPartitionKey().column_names; - case MergeTreeData::MergingParams::Aggregating: - merged_transform = std::make_shared(header, pipes.size(), sort_description, merge_block_size_rows, merge_block_size_bytes); - break; + for (size_t i = 0; i < sort_columns_size; ++i) + sort_description.emplace_back(sort_columns[i], 1, 1); - case MergeTreeData::MergingParams::Replacing: - if (global_ctx->cleanup && !data_settings->allow_experimental_replacing_merge_with_cleanup) - throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental merges with CLEANUP are not allowed"); + /// If merge is vertical we cannot calculate it + ctx->blocks_are_granules_size = (global_ctx->chosen_merge_algorithm == MergeAlgorithm::Vertical); - merged_transform = std::make_shared( - header, pipes.size(), sort_description, ctx->merging_params.is_deleted_column, ctx->merging_params.version_column, - merge_block_size_rows, merge_block_size_bytes, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size, - global_ctx->cleanup); - break; + if (global_ctx->cleanup && !data_settings->allow_experimental_replacing_merge_with_cleanup) + throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental merges with CLEANUP are not allowed"); - case MergeTreeData::MergingParams::Graphite: - merged_transform = std::make_shared( - header, pipes.size(), sort_description, merge_block_size_rows, merge_block_size_bytes, - ctx->merging_params.graphite_params, global_ctx->time_of_merge); - break; - - case MergeTreeData::MergingParams::VersionedCollapsing: - merged_transform = std::make_shared( - header, pipes.size(), sort_description, ctx->merging_params.sign_column, - merge_block_size_rows, merge_block_size_bytes, ctx->rows_sources_write_buf.get(), ctx->blocks_are_granules_size); - break; + auto merge_step = std::make_unique( + merge_parts_query_plan.getCurrentDataStream(), + sort_description, + partition_key_columns, + ctx->merging_params, + ctx->rows_sources_write_buf.get(), + data_settings->merge_max_block_size, + data_settings->merge_max_block_size_bytes, + ctx->blocks_are_granules_size, + global_ctx->cleanup); + merge_step->setStepDescription("Merge sorted parts"); + merge_parts_query_plan.addStep(std::move(merge_step)); } - auto builder = std::make_unique(); - builder->init(Pipe::unitePipes(std::move(pipes))); - builder->addTransform(std::move(merged_transform)); - -#ifndef NDEBUG - if (!sort_description.empty()) - { - builder->addSimpleTransform([&](const Block & header_) - { - auto transform = std::make_shared(header_, sort_description); - return transform; - }); - } -#endif - if (global_ctx->deduplicate) { const auto & virtuals = *global_ctx->data->getVirtualsPtr(); @@ -1383,44 +1529,56 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() /// If deduplicate_by_columns is empty, add all columns except virtuals. if (global_ctx->deduplicate_by_columns.empty()) { - for (const auto & column : global_ctx->merging_columns) + for (const auto & column_name : global_ctx->merging_columns.getNames()) { - if (virtuals.tryGet(column.name, VirtualsKind::Persistent)) + if (virtuals.tryGet(column_name, VirtualsKind::Persistent)) continue; - global_ctx->deduplicate_by_columns.emplace_back(column.name); + global_ctx->deduplicate_by_columns.emplace_back(column_name); } } - if (DistinctSortedTransform::isApplicable(header, sort_description, global_ctx->deduplicate_by_columns)) - builder->addTransform(std::make_shared( - builder->getHeader(), sort_description, SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns)); - else - builder->addTransform(std::make_shared( - builder->getHeader(), SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns)); + auto deduplication_step = std::make_unique( + merge_parts_query_plan.getCurrentDataStream(), + SizeLimits(), 0 /*limit_hint*/, + global_ctx->deduplicate_by_columns, + false, + true /*TODO: ??*/); + deduplication_step->setStepDescription("Deduplication step"); + merge_parts_query_plan.addStep(std::move(deduplication_step)); } PreparedSets::Subqueries subqueries; + /// TTL step if (ctx->need_remove_expired_values) { - auto transform = std::make_shared(global_ctx->context, builder->getHeader(), *global_ctx->data, global_ctx->metadata_snapshot, global_ctx->new_data_part, global_ctx->time_of_merge, ctx->force_ttl); - subqueries = transform->getSubqueries(); - builder->addTransform(std::move(transform)); + auto ttl_step = std::make_unique( + merge_parts_query_plan.getCurrentDataStream(), global_ctx->context, *global_ctx->data, global_ctx->metadata_snapshot, global_ctx->new_data_part, global_ctx->time_of_merge, ctx->force_ttl); + subqueries = ttl_step->getSubqueries(); + ttl_step->setStepDescription("TTL step"); + merge_parts_query_plan.addStep(std::move(ttl_step)); } + /// Secondary indices expressions if (!global_ctx->merging_skip_indexes.empty()) { - builder->addTransform(std::make_shared( - builder->getHeader(), - global_ctx->merging_skip_indexes.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), - global_ctx->data->getContext()))); - - builder->addTransform(std::make_shared(builder->getHeader())); + auto indices_expression_dag = global_ctx->merging_skip_indexes.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext())->getActionsDAG().clone(); + auto calculate_indices_expression_step = std::make_unique( + merge_parts_query_plan.getCurrentDataStream(), + std::move(indices_expression_dag)); + merge_parts_query_plan.addStep(std::move(calculate_indices_expression_step)); + /// TODO: what is the purpose of MaterializingTransform in the original code? + merge_parts_query_plan.addStep(std::make_unique(merge_parts_query_plan.getCurrentDataStream())); } if (!subqueries.empty()) - builder = addCreatingSetsTransform(std::move(builder), std::move(subqueries), global_ctx->context); + addCreatingSetsStep(merge_parts_query_plan, std::move(subqueries), global_ctx->context); + + auto pipelineSettings = BuildQueryPipelineSettings::fromContext(global_ctx->context); + auto builder = merge_parts_query_plan.buildQueryPipeline( + QueryPlanOptimizationSettings::fromContext(global_ctx->context), + pipelineSettings); global_ctx->merged_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); /// Dereference unique_ptr and pass horizontal_stage_progress by reference diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp index 39aa191a3d2..444a59b5590 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.cpp +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.cpp @@ -347,8 +347,11 @@ public: const StorageSnapshotPtr & storage_snapshot_, MergeTreeData::DataPartPtr data_part_, Names columns_to_read_, + std::shared_ptr> filtered_rows_count_, bool apply_deleted_mask_, std::optional filter_, + bool read_with_direct_io_, + bool prefetch_, ContextPtr context_, LoggerPtr log_) : ISourceStep(DataStream{.header = storage_snapshot_->getSampleBlockForColumns(columns_to_read_)}) @@ -357,8 +360,11 @@ public: , storage_snapshot(storage_snapshot_) , data_part(std::move(data_part_)) , columns_to_read(std::move(columns_to_read_)) + , filtered_rows_count(std::move(filtered_rows_count_)) , apply_deleted_mask(apply_deleted_mask_) , filter(std::move(filter_)) + , read_with_direct_io(read_with_direct_io_) + , prefetch(prefetch_) , context(std::move(context_)) , log(log_) { @@ -401,24 +407,27 @@ public: data_part, columns_to_read, std::move(mark_ranges), - /*filtered_rows_count=*/ nullptr, + filtered_rows_count, apply_deleted_mask, - /*read_with_direct_io=*/ false, - /*prefetch=*/ false); + read_with_direct_io, + prefetch); pipeline.init(Pipe(std::move(source))); } private: - MergeTreeSequentialSourceType type; + const MergeTreeSequentialSourceType type; const MergeTreeData & storage; - StorageSnapshotPtr storage_snapshot; - MergeTreeData::DataPartPtr data_part; - Names columns_to_read; - bool apply_deleted_mask; - std::optional filter; - ContextPtr context; - LoggerPtr log; + const StorageSnapshotPtr storage_snapshot; + const MergeTreeData::DataPartPtr data_part; + const Names columns_to_read; + const std::shared_ptr> filtered_rows_count; + const bool apply_deleted_mask; + const std::optional filter; + const bool read_with_direct_io; + const bool prefetch; + const ContextPtr context; + const LoggerPtr log; }; void createReadFromPartStep( @@ -428,15 +437,27 @@ void createReadFromPartStep( const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, Names columns_to_read, + std::shared_ptr> filtered_rows_count, bool apply_deleted_mask, std::optional filter, + bool read_with_direct_io, + bool prefetch, ContextPtr context, LoggerPtr log) { - auto reading = std::make_unique(type, - storage, storage_snapshot, std::move(data_part), - std::move(columns_to_read), apply_deleted_mask, - std::move(filter), std::move(context), log); + auto reading = std::make_unique( + type, + storage, + storage_snapshot, + std::move(data_part), + std::move(columns_to_read), + filtered_rows_count, + apply_deleted_mask, + std::move(filter), + read_with_direct_io, + prefetch, + std::move(context), + log); plan.addStep(std::move(reading)); } diff --git a/src/Storages/MergeTree/MergeTreeSequentialSource.h b/src/Storages/MergeTree/MergeTreeSequentialSource.h index 1b05512b9a3..543d1f60d10 100644 --- a/src/Storages/MergeTree/MergeTreeSequentialSource.h +++ b/src/Storages/MergeTree/MergeTreeSequentialSource.h @@ -37,8 +37,11 @@ void createReadFromPartStep( const StorageSnapshotPtr & storage_snapshot, MergeTreeData::DataPartPtr data_part, Names columns_to_read, + std::shared_ptr> filtered_rows_count, bool apply_deleted_mask, std::optional filter, + bool read_with_direct_io, + bool prefetch, ContextPtr context, LoggerPtr log); From 13f4eb3fac6c2c0781351ee5db382383193b2af5 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 2 Sep 2024 22:24:53 +0200 Subject: [PATCH 122/162] Fix for graphite merge mode --- src/Storages/MergeTree/MergeTask.cpp | 25 ++++++++++++------------- src/Storages/MergeTree/MergeTask.h | 2 +- 2 files changed, 13 insertions(+), 14 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 3bee2ecb0d9..fb5bbc4729c 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1224,7 +1224,8 @@ public: UInt64 merge_block_size_rows_, UInt64 merge_block_size_bytes_, bool blocks_are_granules_size_, - bool cleanup_) + bool cleanup_, + time_t time_of_merge_) : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? , sort_description(sort_description_) , partition_key_columns(partition_key_columns_) @@ -1234,6 +1235,7 @@ public: , merge_block_size_bytes(merge_block_size_bytes_) , blocks_are_granules_size(blocks_are_granules_size_) , cleanup(cleanup_) + , time_of_merge(time_of_merge_) {} String getName() const override { return "ApplyMergePolicy"; } @@ -1245,10 +1247,6 @@ public: /// that is going in insertion order. ProcessorPtr merged_transform; -// /// There is no sense to have the block size bigger than one granule for merge operations. -// const UInt64 merge_block_size_rows = data_settings->merge_max_block_size; -// const UInt64 merge_block_size_bytes = data_settings->merge_max_block_size_bytes; - const auto &header = pipeline.getHeader(); const auto input_streams_count = pipeline.getNumStreams(); @@ -1330,15 +1328,15 @@ public: } private: - SortDescription sort_description; - Names partition_key_columns; - MergeTreeData::MergingParams merging_params{}; + const SortDescription sort_description; + const Names partition_key_columns; + const MergeTreeData::MergingParams merging_params{}; WriteBuffer * rows_sources_write_buf; const UInt64 merge_block_size_rows; const UInt64 merge_block_size_bytes; - bool blocks_are_granules_size; - bool cleanup{false}; - time_t time_of_merge{0}; + const bool blocks_are_granules_size; + const bool cleanup{false}; + const time_t time_of_merge{0}; }; @@ -1403,7 +1401,7 @@ private: }; -void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() +void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const { /** Read from all parts, merge and write into a new one. * In passing, we calculate expression for sorting. @@ -1516,7 +1514,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() data_settings->merge_max_block_size, data_settings->merge_max_block_size_bytes, ctx->blocks_are_granules_size, - global_ctx->cleanup); + global_ctx->cleanup, + global_ctx->time_of_merge); merge_step->setStepDescription("Merge sorted parts"); merge_parts_query_plan.addStep(std::move(merge_step)); } diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index c80995888d4..a5d7851932c 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -291,7 +291,7 @@ private: bool executeMergeProjections(); MergeAlgorithm chooseMergeAlgorithm() const; - void createMergedStream(); + void createMergedStream() const; void extractMergingAndGatheringColumns() const; void setRuntimeContext(StageRuntimeContextPtr local, StageRuntimeContextPtr global) override From 48cacd6f310c107c1dd0239a7639527adb054b69 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 2 Sep 2024 22:36:42 +0200 Subject: [PATCH 123/162] Use query plan for column vertical merges --- src/Storages/MergeTree/MergeTask.cpp | 175 ++++++++++++++++++++------- src/Storages/MergeTree/MergeTask.h | 5 +- 2 files changed, 133 insertions(+), 47 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index fb5bbc4729c..75fd61ae4be 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -804,35 +804,106 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const bool all_parts_on_remote_disks = std::ranges::all_of(global_ctx->future_part->parts, [](const auto & part) { return part->isStoredOnRemoteDisk(); }); ctx->use_prefetch = all_parts_on_remote_disks && global_ctx->data->getSettings()->vertical_merge_remote_filesystem_prefetch; - if (ctx->use_prefetch && ctx->it_name_and_type != global_ctx->gathering_columns.end()) - ctx->prepared_pipe = createPipeForReadingOneColumn(ctx->it_name_and_type->name); +// if (ctx->use_prefetch && ctx->it_name_and_type != global_ctx->gathering_columns.end()) +// ctx->prepared_pipe = createPipeForReadingOneColumn(ctx->it_name_and_type->name); return false; } -Pipe MergeTask::VerticalMergeStage::createPipeForReadingOneColumn(const String & column_name) const +QueryPlan MergeTask::VerticalMergeStage::createPlanForReadingOneColumn(const String & column_name) const { - Pipes pipes; - for (size_t part_num = 0; part_num < global_ctx->future_part->parts.size(); ++part_num) + /// Read from all parts + std::vector plans; + for (const auto & part : global_ctx->future_part->parts) { - Pipe pipe = createMergeTreeSequentialSource( + auto plan_for_part = std::make_unique(); + createReadFromPartStep( MergeTreeSequentialSourceType::Merge, + *plan_for_part, *global_ctx->data, global_ctx->storage_snapshot, - global_ctx->future_part->parts[part_num], + part, Names{column_name}, - /*mark_ranges=*/ {}, global_ctx->input_rows_filtered, /*apply_deleted_mask=*/ true, + std::nullopt, ctx->read_with_direct_io, - ctx->use_prefetch); + ctx->use_prefetch, + global_ctx->context, + getLogger("VerticalMergeStage")); - pipes.emplace_back(std::move(pipe)); + plans.emplace_back(std::move(plan_for_part)); } - return Pipe::unitePipes(std::move(pipes)); + QueryPlan merge_parts_query_plan; + + /// Union of all parts streams + { + DataStreams input_streams; + input_streams.reserve(plans.size()); + for (auto & plan : plans) + input_streams.emplace_back(plan->getCurrentDataStream()); + + auto union_step = std::make_unique(std::move(input_streams)); + merge_parts_query_plan.unitePlans(std::move(union_step), std::move(plans)); + } + + return merge_parts_query_plan; } +/// Gathers values from all parts for one column using rows sources temporary file +class ColumnGathererStep : public ITransformingStep +{ +public: + ColumnGathererStep( + const DataStream & input_stream_, + CompressedReadBufferFromFile * rows_sources_read_buf_, + UInt64 merge_block_size_rows_, + UInt64 merge_block_size_bytes_, + bool is_result_sparse_) + : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + , rows_sources_read_buf(rows_sources_read_buf_) + , merge_block_size_rows(merge_block_size_rows_) + , merge_block_size_bytes(merge_block_size_bytes_) + , is_result_sparse(is_result_sparse_) + {} + + String getName() const override { return "ColumnGatherer"; } + + void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & /*pipelineSettings*/) override + { + const auto &header = pipeline.getHeader(); + const auto input_streams_count = pipeline.getNumStreams(); + + rows_sources_read_buf->seek(0, 0); + + auto transform = std::make_unique( + header, + input_streams_count, + *rows_sources_read_buf, + merge_block_size_rows, + merge_block_size_bytes, + is_result_sparse); + + pipeline.addTransform(std::move(transform)); + } + + void updateOutputStream() override + { + output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); + + /// TODO: is this correct? + output_stream->sort_scope = DataStream::SortScope::None; + } + +private: + MergeTreeData::MergingParams merging_params{}; + CompressedReadBufferFromFile * rows_sources_read_buf; + const UInt64 merge_block_size_rows; + const UInt64 merge_block_size_bytes; + const bool is_result_sparse; +}; + void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const { const auto & column_name = ctx->it_name_and_type->name; @@ -840,50 +911,64 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const ctx->progress_before = global_ctx->merge_list_element_ptr->progress.load(std::memory_order_relaxed); global_ctx->column_progress = std::make_unique(ctx->progress_before, ctx->column_sizes->columnWeight(column_name)); - Pipe pipe; - if (ctx->prepared_pipe) - { - pipe = std::move(*ctx->prepared_pipe); +// Pipe pipe; +//// if (ctx->prepared_pipe) +//// { +//// pipe = std::move(*ctx->prepared_pipe); +//// +//// auto next_column_it = std::next(ctx->it_name_and_type); +//// if (next_column_it != global_ctx->gathering_columns.end()) +//// ctx->prepared_pipe = createPipeForReadingOneColumn(next_column_it->name); +//// } +//// else +// { +// pipe = createPipeForReadingOneColumn(column_name); +// } - auto next_column_it = std::next(ctx->it_name_and_type); - if (next_column_it != global_ctx->gathering_columns.end()) - ctx->prepared_pipe = createPipeForReadingOneColumn(next_column_it->name); - } - else + auto merge_column_query_plan = createPlanForReadingOneColumn(column_name); + + /// Add column gatherer step { - pipe = createPipeForReadingOneColumn(column_name); +// ctx->rows_sources_read_buf->seek(0, 0); + bool is_result_sparse = global_ctx->new_data_part->getSerialization(column_name)->getKind() == ISerialization::Kind::SPARSE; + const auto data_settings = global_ctx->data->getSettings(); + auto merge_step = std::make_unique( + merge_column_query_plan.getCurrentDataStream(), + ctx->rows_sources_read_buf.get(), //global_ctx->rows_sources_temporary_file_name, + data_settings->merge_max_block_size, + data_settings->merge_max_block_size_bytes, + is_result_sparse); + merge_step->setStepDescription("Gather column"); + merge_column_query_plan.addStep(std::move(merge_step)); } - ctx->rows_sources_read_buf->seek(0, 0); - bool is_result_sparse = global_ctx->new_data_part->getSerialization(column_name)->getKind() == ISerialization::Kind::SPARSE; - - const auto data_settings = global_ctx->data->getSettings(); - auto transform = std::make_unique( - pipe.getHeader(), - pipe.numOutputPorts(), - *ctx->rows_sources_read_buf, - data_settings->merge_max_block_size, - data_settings->merge_max_block_size_bytes, - is_result_sparse); - - pipe.addTransform(std::move(transform)); - + /// Add expression step for indexes MergeTreeIndices indexes_to_recalc; - auto indexes_it = global_ctx->skip_indexes_by_column.find(column_name); - - if (indexes_it != global_ctx->skip_indexes_by_column.end()) + IndicesDescription indexes_to_recalc_description; { - indexes_to_recalc = MergeTreeIndexFactory::instance().getMany(indexes_it->second); + auto indexes_it = global_ctx->skip_indexes_by_column.find(column_name); - pipe.addTransform(std::make_shared( - pipe.getHeader(), - indexes_it->second.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), - global_ctx->data->getContext()))); + if (indexes_it != global_ctx->skip_indexes_by_column.end()) + { + indexes_to_recalc_description = indexes_it->second; + indexes_to_recalc = MergeTreeIndexFactory::instance().getMany(indexes_it->second); - pipe.addTransform(std::make_shared(pipe.getHeader())); + auto indices_expression_dag = indexes_it->second.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext())->getActionsDAG().clone(); + auto calculate_indices_expression_step = std::make_unique( + merge_column_query_plan.getCurrentDataStream(), + std::move(indices_expression_dag)); + merge_column_query_plan.addStep(std::move(calculate_indices_expression_step)); + } } - ctx->column_parts_pipeline = QueryPipeline(std::move(pipe)); + { + auto pipelineSettings = BuildQueryPipelineSettings::fromContext(global_ctx->context); + auto builder = merge_column_query_plan.buildQueryPipeline( + QueryPlanOptimizationSettings::fromContext(global_ctx->context), + pipelineSettings); + + ctx->column_parts_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + } /// Dereference unique_ptr ctx->column_parts_pipeline.setProgressCallback(MergeProgressCallback( diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index a5d7851932c..b36f5f832d9 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -333,7 +333,8 @@ private: Float64 progress_before = 0; std::unique_ptr column_to{nullptr}; - std::optional prepared_pipe; +// TODO: is this really needed for prefetch? +// std::optional prepared_pipe; size_t max_delayed_streams = 0; bool use_prefetch = false; std::list> delayed_streams; @@ -378,7 +379,7 @@ private: bool executeVerticalMergeForOneColumn() const; void finalizeVerticalMergeForOneColumn() const; - Pipe createPipeForReadingOneColumn(const String & column_name) const; + QueryPlan createPlanForReadingOneColumn(const String & column_name) const; VerticalMergeRuntimeContextPtr ctx; GlobalRuntimeContextPtr global_ctx; From d28cba981ccd6a58939854a0204d654c6075337d Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 3 Sep 2024 08:59:01 +0200 Subject: [PATCH 124/162] Fix clang_tidy --- src/Storages/MergeTree/MergeTask.cpp | 6 +++--- src/Storages/MergeTree/MergeTask.h | 8 ++++---- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 75fd61ae4be..cafc11fc34d 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -195,7 +195,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::extractMergingAndGatheringColu } } -bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() +bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() const { ProfileEvents::increment(ProfileEvents::Merge); @@ -657,7 +657,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::constructTaskForProjectionPart } -bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() // NOLINT +bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() const { /// In case if there are no projections we didn't construct a task if (!ctx->merge_projection_parts_task_ptr) @@ -676,7 +676,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeMergeProjections() // N return true; } -bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() +bool MergeTask::ExecuteAndFinalizeHorizontalPart::executeImpl() const { Stopwatch watch(CLOCK_MONOTONIC_COARSE); UInt64 step_time_ms = global_ctx->data->getSettings()->background_task_preferred_step_execution_time_ms.totalMilliseconds(); diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index b36f5f832d9..a30ab4712d5 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -268,12 +268,12 @@ private: { bool execute() override; - bool prepare(); - bool executeImpl(); + bool prepare() const; + bool executeImpl() const; void finalize() const; /// NOTE: Using pointer-to-member instead of std::function and lambda makes stacktraces much more concise and readable - using ExecuteAndFinalizeHorizontalPartSubtasks = std::array; + using ExecuteAndFinalizeHorizontalPartSubtasks = std::array; const ExecuteAndFinalizeHorizontalPartSubtasks subtasks { @@ -288,7 +288,7 @@ private: void calculateProjections(const Block & block) const; void finalizeProjections() const; void constructTaskForProjectionPartsMerge() const; - bool executeMergeProjections(); + bool executeMergeProjections() const; MergeAlgorithm chooseMergeAlgorithm() const; void createMergedStream() const; From 6a6935cb84a31493def51cf5d65954bec75f587e Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 3 Sep 2024 13:09:18 +0200 Subject: [PATCH 125/162] Cleanup --- src/Storages/MergeTree/MergeTask.cpp | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index cafc11fc34d..6f5b8301d4a 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1466,7 +1466,7 @@ public: subqueries_for_sets = transform->getSubqueries(); } - String getName() const override { return "Materializing"; } + String getName() const override { return "TTL"; } PreparedSets::Subqueries getSubqueries() { return std::move(subqueries_for_sets); } @@ -1524,9 +1524,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const std::vector plans; for (const auto & part : global_ctx->future_part->parts) { - /// TODO: this is just for debugging purposes, remove it later if (part->getMarksCount() == 0) - LOG_DEBUG(ctx->log, "Part {} is empty", part->name); + LOG_TRACE(ctx->log, "Part {} is empty", part->name); auto plan_for_part = std::make_unique(); createReadFromPartStep( @@ -1613,12 +1612,12 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const /// If deduplicate_by_columns is empty, add all columns except virtuals. if (global_ctx->deduplicate_by_columns.empty()) { - for (const auto & column_name : global_ctx->merging_columns.getNames()) + for (const auto & column : global_ctx->merging_columns) { - if (virtuals.tryGet(column_name, VirtualsKind::Persistent)) + if (virtuals.tryGet(column.name, VirtualsKind::Persistent)) continue; - global_ctx->deduplicate_by_columns.emplace_back(column_name); + global_ctx->deduplicate_by_columns.emplace_back(column.name); } } From a1cec53b7c2a6508277280bd8c36f90dfe661560 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 3 Sep 2024 14:54:05 +0200 Subject: [PATCH 126/162] Fix updateOutputStream and Traits --- src/Storages/MergeTree/MergeTask.cpp | 106 ++++++++++++++++++++------- 1 file changed, 78 insertions(+), 28 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 6f5b8301d4a..1bf1573fc1f 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -861,7 +861,7 @@ public: UInt64 merge_block_size_rows_, UInt64 merge_block_size_bytes_, bool is_result_sparse_) - : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , rows_sources_read_buf(rows_sources_read_buf_) , merge_block_size_rows(merge_block_size_rows_) , merge_block_size_bytes(merge_block_size_bytes_) @@ -891,12 +891,24 @@ public: void updateOutputStream() override { output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); - - /// TODO: is this correct? - output_stream->sort_scope = DataStream::SortScope::None; } private: + static Traits getTraits() + { + return ITransformingStep::Traits + { + { + .returns_single_stream = true, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = false, + } + }; + } + MergeTreeData::MergingParams merging_params{}; CompressedReadBufferFromFile * rows_sources_read_buf; const UInt64 merge_block_size_rows; @@ -962,10 +974,9 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const } { - auto pipelineSettings = BuildQueryPipelineSettings::fromContext(global_ctx->context); - auto builder = merge_column_query_plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(global_ctx->context), - pipelineSettings); + auto pipeline_settings = BuildQueryPipelineSettings::fromContext(global_ctx->context); + auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context); + auto builder = merge_column_query_plan.buildQueryPipeline(optimization_settings, pipeline_settings); ctx->column_parts_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); } @@ -1297,7 +1308,7 @@ bool MergeTask::execute() /// Apply merge strategy (Ordinary, Colapsing, Aggregating, etc) to the stream -class ApplyMergeStep : public ITransformingStep /// TODO: is this transformation step? +class ApplyMergeStep : public ITransformingStep { public: ApplyMergeStep( @@ -1311,7 +1322,7 @@ public: bool blocks_are_granules_size_, bool cleanup_, time_t time_of_merge_) - : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + : ITransformingStep(input_stream_, input_stream_.header, getTraits()) , sort_description(sort_description_) , partition_key_columns(partition_key_columns_) , merging_params(merging_params_) @@ -1403,16 +1414,24 @@ public: void updateOutputStream() override { output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); - output_stream->sort_description = sort_description; - - /// TODO: is this correct? -// if (partition_key_columns.empty()) - output_stream->sort_scope = DataStream::SortScope::Global; -// else -// output_stream->sort_scope = DataStream::SortScope::Stream; } private: + static Traits getTraits() + { + return ITransformingStep::Traits + { + { + .returns_single_stream = true, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = false, + } + }; + } + const SortDescription sort_description; const Names partition_key_columns; const MergeTreeData::MergingParams merging_params{}; @@ -1425,12 +1444,12 @@ private: }; -class MaterializingStep : public ITransformingStep /// TODO: is this transformation step? +class MaterializingStep : public ITransformingStep { public: explicit MaterializingStep( const DataStream & input_stream_) - : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + : ITransformingStep(input_stream_, input_stream_.header, getTraits()) {} String getName() const override { return "Materializing"; } @@ -1442,9 +1461,23 @@ public: void updateOutputStream() override { - /// TODO: can this be simplified? output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); - output_stream->sort_description = input_streams.front().sort_description; + } + +private: + static Traits getTraits() + { + return ITransformingStep::Traits + { + { + .returns_single_stream = true, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = true, + } + }; } }; @@ -1460,7 +1493,7 @@ public: const MergeTreeData::MutableDataPartPtr & data_part_, time_t current_time, bool force_) - : ITransformingStep(input_stream_, input_stream_.header, Traits{}) // TODO proper traits? + : ITransformingStep(input_stream_, input_stream_.header, getTraits()) { transform = std::make_shared(context_, input_stream_.header, storage_, metadata_snapshot_, data_part_, current_time, force_); subqueries_for_sets = transform->getSubqueries(); @@ -1477,10 +1510,25 @@ public: void updateOutputStream() override { - // TODO: implement? + output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); } private: + static Traits getTraits() + { + return ITransformingStep::Traits + { + { + .returns_single_stream = true, + .preserves_number_of_streams = true, + .preserves_sorting = true, + }, + { + .preserves_number_of_rows = false, + } + }; + } + std::shared_ptr transform; PreparedSets::Subqueries subqueries_for_sets; }; @@ -1658,12 +1706,14 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const if (!subqueries.empty()) addCreatingSetsStep(merge_parts_query_plan, std::move(subqueries), global_ctx->context); - auto pipelineSettings = BuildQueryPipelineSettings::fromContext(global_ctx->context); - auto builder = merge_parts_query_plan.buildQueryPipeline( - QueryPlanOptimizationSettings::fromContext(global_ctx->context), - pipelineSettings); + { + auto pipeline_settings = BuildQueryPipelineSettings::fromContext(global_ctx->context); + auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context); + auto builder = merge_parts_query_plan.buildQueryPipeline(optimization_settings, pipeline_settings); + + global_ctx->merged_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + } - global_ctx->merged_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); /// Dereference unique_ptr and pass horizontal_stage_progress by reference global_ctx->merged_pipeline.setProgressCallback(MergeProgressCallback(global_ctx->merge_list_element_ptr, global_ctx->watch_prev_elapsed, *global_ctx->horizontal_stage_progress)); /// Is calculated inside MergeProgressCallback. From 8361724539408d95f9757e00047919d70ea50bbd Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 3 Sep 2024 17:02:25 +0200 Subject: [PATCH 127/162] Build pipeline for next column for prefetching --- src/Storages/MergeTree/MergeTask.cpp | 140 +++++++++++++-------------- src/Storages/MergeTree/MergeTask.h | 14 ++- 2 files changed, 81 insertions(+), 73 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 1bf1573fc1f..a4104672de7 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -804,53 +804,12 @@ bool MergeTask::VerticalMergeStage::prepareVerticalMergeForAllColumns() const bool all_parts_on_remote_disks = std::ranges::all_of(global_ctx->future_part->parts, [](const auto & part) { return part->isStoredOnRemoteDisk(); }); ctx->use_prefetch = all_parts_on_remote_disks && global_ctx->data->getSettings()->vertical_merge_remote_filesystem_prefetch; -// if (ctx->use_prefetch && ctx->it_name_and_type != global_ctx->gathering_columns.end()) -// ctx->prepared_pipe = createPipeForReadingOneColumn(ctx->it_name_and_type->name); + if (ctx->use_prefetch && ctx->it_name_and_type != global_ctx->gathering_columns.end()) + ctx->prepared_pipeline = createPipelineForReadingOneColumn(ctx->it_name_and_type->name); return false; } -QueryPlan MergeTask::VerticalMergeStage::createPlanForReadingOneColumn(const String & column_name) const -{ - /// Read from all parts - std::vector plans; - for (const auto & part : global_ctx->future_part->parts) - { - auto plan_for_part = std::make_unique(); - createReadFromPartStep( - MergeTreeSequentialSourceType::Merge, - *plan_for_part, - *global_ctx->data, - global_ctx->storage_snapshot, - part, - Names{column_name}, - global_ctx->input_rows_filtered, - /*apply_deleted_mask=*/ true, - std::nullopt, - ctx->read_with_direct_io, - ctx->use_prefetch, - global_ctx->context, - getLogger("VerticalMergeStage")); - - plans.emplace_back(std::move(plan_for_part)); - } - - QueryPlan merge_parts_query_plan; - - /// Union of all parts streams - { - DataStreams input_streams; - input_streams.reserve(plans.size()); - for (auto & plan : plans) - input_streams.emplace_back(plan->getCurrentDataStream()); - - auto union_step = std::make_unique(std::move(input_streams)); - merge_parts_query_plan.unitePlans(std::move(union_step), std::move(plans)); - } - - return merge_parts_query_plan; -} - /// Gathers values from all parts for one column using rows sources temporary file class ColumnGathererStep : public ITransformingStep { @@ -916,32 +875,46 @@ private: const bool is_result_sparse; }; -void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const +MergeTask::VerticalMergeRuntimeContext::PreparedColumnPipeline MergeTask::VerticalMergeStage::createPipelineForReadingOneColumn(const String & column_name) const { - const auto & column_name = ctx->it_name_and_type->name; + /// Read from all parts + std::vector plans; + for (const auto & part : global_ctx->future_part->parts) + { + auto plan_for_part = std::make_unique(); + createReadFromPartStep( + MergeTreeSequentialSourceType::Merge, + *plan_for_part, + *global_ctx->data, + global_ctx->storage_snapshot, + part, + Names{column_name}, + global_ctx->input_rows_filtered, + /*apply_deleted_mask=*/ true, + std::nullopt, + ctx->read_with_direct_io, + ctx->use_prefetch, + global_ctx->context, + getLogger("VerticalMergeStage")); - ctx->progress_before = global_ctx->merge_list_element_ptr->progress.load(std::memory_order_relaxed); - global_ctx->column_progress = std::make_unique(ctx->progress_before, ctx->column_sizes->columnWeight(column_name)); + plans.emplace_back(std::move(plan_for_part)); + } -// Pipe pipe; -//// if (ctx->prepared_pipe) -//// { -//// pipe = std::move(*ctx->prepared_pipe); -//// -//// auto next_column_it = std::next(ctx->it_name_and_type); -//// if (next_column_it != global_ctx->gathering_columns.end()) -//// ctx->prepared_pipe = createPipeForReadingOneColumn(next_column_it->name); -//// } -//// else -// { -// pipe = createPipeForReadingOneColumn(column_name); -// } + QueryPlan merge_column_query_plan; - auto merge_column_query_plan = createPlanForReadingOneColumn(column_name); + /// Union of all parts streams + { + DataStreams input_streams; + input_streams.reserve(plans.size()); + for (auto & plan : plans) + input_streams.emplace_back(plan->getCurrentDataStream()); + + auto union_step = std::make_unique(std::move(input_streams)); + merge_column_query_plan.unitePlans(std::move(union_step), std::move(plans)); + } /// Add column gatherer step { -// ctx->rows_sources_read_buf->seek(0, 0); bool is_result_sparse = global_ctx->new_data_part->getSerialization(column_name)->getKind() == ISerialization::Kind::SPARSE; const auto data_settings = global_ctx->data->getSettings(); auto merge_step = std::make_unique( @@ -973,13 +946,36 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const } } - { - auto pipeline_settings = BuildQueryPipelineSettings::fromContext(global_ctx->context); - auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context); - auto builder = merge_column_query_plan.buildQueryPipeline(optimization_settings, pipeline_settings); + auto pipeline_settings = BuildQueryPipelineSettings::fromContext(global_ctx->context); + auto optimization_settings = QueryPlanOptimizationSettings::fromContext(global_ctx->context); + auto builder = merge_column_query_plan.buildQueryPipeline(optimization_settings, pipeline_settings); - ctx->column_parts_pipeline = QueryPipelineBuilder::getPipeline(std::move(*builder)); + return {QueryPipelineBuilder::getPipeline(std::move(*builder)), std::move(indexes_to_recalc)}; +} + +void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const +{ + const auto & column_name = ctx->it_name_and_type->name; + + ctx->progress_before = global_ctx->merge_list_element_ptr->progress.load(std::memory_order_relaxed); + global_ctx->column_progress = std::make_unique(ctx->progress_before, ctx->column_sizes->columnWeight(column_name)); + + VerticalMergeRuntimeContext::PreparedColumnPipeline column_pipepline; + if (ctx->prepared_pipeline) + { + column_pipepline = std::move(*ctx->prepared_pipeline); + + /// Prepare next column pipeline to initiate prefetching + auto next_column_it = std::next(ctx->it_name_and_type); + if (next_column_it != global_ctx->gathering_columns.end()) + ctx->prepared_pipeline = createPipelineForReadingOneColumn(next_column_it->name); } + else + { + column_pipepline = createPipelineForReadingOneColumn(column_name); + } + + ctx->column_parts_pipeline = std::move(column_pipepline.pipeline); /// Dereference unique_ptr ctx->column_parts_pipeline.setProgressCallback(MergeProgressCallback( @@ -997,12 +993,16 @@ void MergeTask::VerticalMergeStage::prepareVerticalMergeForOneColumn() const global_ctx->metadata_snapshot, columns_list, ctx->compression_codec, - indexes_to_recalc, + column_pipepline.indexes_to_recalc, getStatisticsForColumns(columns_list, global_ctx->metadata_snapshot), &global_ctx->written_offset_columns, global_ctx->to->getIndexGranularity()); ctx->column_elems_written = 0; + + /// rows_sources_read_buf is reused for each column so we need to rewind it explicitly each time + /// This sharing also prevents from from running multiple merge of individual columns in parallel. + ctx->rows_sources_read_buf->seek(0, 0); } @@ -1673,8 +1673,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const merge_parts_query_plan.getCurrentDataStream(), SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns, - false, - true /*TODO: ??*/); + false /*pre_distinct*/, + true /*optimize_distinct_in_order TODO: looks like it shoud be enabled*/); deduplication_step->setStepDescription("Deduplication step"); merge_parts_query_plan.addStep(std::move(deduplication_step)); } diff --git a/src/Storages/MergeTree/MergeTask.h b/src/Storages/MergeTree/MergeTask.h index a30ab4712d5..bbe53c34c7e 100644 --- a/src/Storages/MergeTree/MergeTask.h +++ b/src/Storages/MergeTree/MergeTask.h @@ -333,8 +333,16 @@ private: Float64 progress_before = 0; std::unique_ptr column_to{nullptr}; -// TODO: is this really needed for prefetch? -// std::optional prepared_pipe; + + /// Used for prefetching. Right before starting merge of a column we create a pipeline for the next column + /// and it initiates prefetching of the first range of that column. + struct PreparedColumnPipeline + { + QueryPipeline pipeline; + MergeTreeIndices indexes_to_recalc; + }; + + std::optional prepared_pipeline; size_t max_delayed_streams = 0; bool use_prefetch = false; std::list> delayed_streams; @@ -379,7 +387,7 @@ private: bool executeVerticalMergeForOneColumn() const; void finalizeVerticalMergeForOneColumn() const; - QueryPlan createPlanForReadingOneColumn(const String & column_name) const; + VerticalMergeRuntimeContext::PreparedColumnPipeline createPipelineForReadingOneColumn(const String & column_name) const; VerticalMergeRuntimeContextPtr ctx; GlobalRuntimeContextPtr global_ctx; From 472e6eb856e338332fbebb2519066f093c18a15f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 3 Sep 2024 17:16:43 +0200 Subject: [PATCH 128/162] typo --- src/Storages/MergeTree/MergeTask.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index a4104672de7..576ea341877 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1674,7 +1674,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const SizeLimits(), 0 /*limit_hint*/, global_ctx->deduplicate_by_columns, false /*pre_distinct*/, - true /*optimize_distinct_in_order TODO: looks like it shoud be enabled*/); + true /*optimize_distinct_in_order TODO: looks like it should be enabled*/); deduplication_step->setStepDescription("Deduplication step"); merge_parts_query_plan.addStep(std::move(deduplication_step)); } From fc1228e3a40509c0566c89c5348c94664e3ce219 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Tue, 3 Sep 2024 20:45:43 +0000 Subject: [PATCH 129/162] Update settings history --- src/Core/SettingsChangesHistory.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 8c26fe8da44..3d0ecc32f79 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -76,7 +76,8 @@ static std::initializer_list Date: Wed, 4 Sep 2024 16:05:13 +0800 Subject: [PATCH 130/162] review fix --- src/Core/Settings.h | 5 ++-- src/Core/SettingsChangesHistory.cpp | 5 ++-- src/Interpreters/HashJoin/AddedColumns.h | 8 +++--- src/Interpreters/HashJoin/HashJoin.cpp | 26 +++++++++++++------ src/Interpreters/TableJoin.cpp | 1 + src/Interpreters/TableJoin.h | 2 ++ tests/performance/all_join_opt.xml | 4 +-- .../experimental_settings_ignore.txt | 1 + 8 files changed, 34 insertions(+), 18 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c1433ca7250..386fd4e74ee 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -922,8 +922,9 @@ class IColumn; M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \ M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \ M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \ - M(Int32, join_to_sort_perkey_rows_threshold, 40, "The lower limit of per-key average rows in the right table to determine whether to sort it in hash join.", 0) \ - M(Int32, join_to_sort_table_rows_threshold, 10000, "The upper limit of rows in the right table to determine whether to sort it in hash join.", 0) \ + M(Int32, join_to_sort_perkey_rows_threshold, 40, "Rerange the right table by key in hash join when the per-key average rows of it exceed this value (means the table keys is dense) and its number of rows is not too many(controlled by `join_to_sort_table_rows_threshold`), to make the join output by the key's batch, which would improve performance.", 0) \ + M(Int32, join_to_sort_table_rows_threshold, 10000, "Rerange the right table by key in hash join when its number of rows not exceed this value and the table keys is dense (controlled by `join_to_sort_perkey_rows_threshold`), to make the join performance improve as output by the key's batch, but not cost too much on the table reranging.", 0) \ + M(Bool, allow_experimental_inner_join_right_table_sorting, false, "If it is set to true, and the conditions of `join_to_sort_perkey_rows_threshold` and `join_to_sort_perkey_rows_threshold` are met, then we will try to rerange the right table by key to improve the performance in hash join.", 0) \ M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, use_hive_partitioning, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines.", 0)\ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 392f0dbc2ee..2a38af85b08 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -95,8 +95,9 @@ static std::initializer_listavgPerKeyRows()) + , output_by_row_list_threshold(join.getTableJoin().outputByRowListPerkeyRowsThreshold()) + , join_data_sorted(join.getJoinedData()->sorted) , is_join_get(is_join_get_) { size_t num_columns_to_add = block_with_columns_to_add.columns(); @@ -113,9 +116,6 @@ public: if (columns[j]->isNullable() && !saved_column->isNullable()) nullable_column_ptrs[j] = typeid_cast(columns[j].get()); } - join_data_avg_perkey_rows = join.getJoinedData()->avgPerKeyRows(); - output_by_row_list_threshold = join.getTableJoin().outputByRowListPerkeyRowsThreshold(); - join_data_sorted = join.getJoinedData()->sorted; } size_t size() const { return columns.size(); } @@ -148,9 +148,9 @@ public: std::unique_ptr offsets_to_replicate; bool need_filter = false; bool output_by_row_list = false; - bool join_data_sorted = false; size_t join_data_avg_perkey_rows = 0; size_t output_by_row_list_threshold = 0; + bool join_data_sorted = false; IColumn::Filter filter; void reserve(bool need_replicate) diff --git a/src/Interpreters/HashJoin/HashJoin.cpp b/src/Interpreters/HashJoin/HashJoin.cpp index 59888d7a71d..5b095368131 100644 --- a/src/Interpreters/HashJoin/HashJoin.cpp +++ b/src/Interpreters/HashJoin/HashJoin.cpp @@ -1364,14 +1364,16 @@ template void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]]) { constexpr JoinFeatures join_features; - if constexpr (join_features.is_all_join && (join_features.left || join_features.inner)) + if constexpr (!join_features.is_all_join || (!join_features.left && !join_features.inner)) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Only left or inner join table can be reranged."); + else { auto merge_rows_into_one_block = [&](BlocksList & blocks, RowRefList & rows_ref) { auto it = rows_ref.begin(); if (it.ok()) { - if (blocks.empty() || blocks.back().rows() > DEFAULT_BLOCK_SIZE) + if (blocks.empty() || blocks.back().rows() >= DEFAULT_BLOCK_SIZE) blocks.emplace_back(it->block->cloneEmpty()); } else @@ -1384,7 +1386,7 @@ void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]]) { for (size_t i = 0; i < block.columns(); ++i) { - auto & col = *(block.getByPosition(i).column->assumeMutable()); + auto & col = block.getByPosition(i).column->assumeMutableRef(); col.insertFrom(*it->block->getByPosition(i).column, it->row_num); } } @@ -1419,26 +1421,34 @@ void HashJoin::tryRerangeRightTableDataImpl(Map & map [[maybe_unused]]) void HashJoin::tryRerangeRightTableData() { - if ((kind != JoinKind::Inner && kind != JoinKind::Left) || strictness != JoinStrictness::All || table_join->getMixedJoinExpression()) + if (!table_join->allowJoinSorting() || table_join->getMixedJoinExpression() || !isInnerOrLeft(kind) || strictness != JoinStrictness::All) return; + /// We should not rerange the right table on such conditions: + /// 1. the right table is already reranged by key or it is empty. + /// 2. the join clauses size is greater than 1, like `...join on a.key1=b.key1 or a.key2=b.key2`, we can not rerange the right table on different set of keys. + /// 3. the number of right table rows exceed the threshold, which may result in a significant cost for reranging and lead to performance degradation. + /// 4. the keys of right table is very sparse, which may result in insignificant performance improvement after reranging by key. if (!data || data->sorted || data->blocks.empty() || data->maps.size() > 1 || data->rows_to_join > table_join->sortRightTableRowsThreshold() || data->avgPerKeyRows() < table_join->sortRightPerkeyRowsThreshold()) return; if (data->keys_to_join == 0) data->keys_to_join = getTotalRowCount(); + + /// If the there is no columns to add, means no columns to output, then the rerange would not improve performance by using column's `insertRangeFrom` + /// to replace column's `insertFrom` to make the output. if (sample_block_with_columns_to_add.columns() == 0) { - LOG_DEBUG(log, "The joined right table total rows :{}, total keys :{}, columns added:{}", - data->rows_to_join, data->keys_to_join, sample_block_with_columns_to_add.columns()); + LOG_DEBUG(log, "The joined right table total rows :{}, total keys :{}", data->rows_to_join, data->keys_to_join); return; } - joinDispatch( + [[maybe_unused]] bool result = joinDispatch( kind, strictness, data->maps.front(), - false, + /*prefer_use_maps_all*/ false, [&](auto kind_, auto strictness_, auto & map_) { tryRerangeRightTableDataImpl(map_); }); + chassert(result); data->sorted = true; } diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 8bcaef77939..8d79b88190b 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -118,6 +118,7 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary , output_by_rowlist_perkey_rows_threshold(settings.join_output_by_rowlist_perkey_rows_threshold) , sort_right_perkey_rows_threshold(settings.join_to_sort_perkey_rows_threshold) , sort_right_table_rows_threshold(settings.join_to_sort_table_rows_threshold) + , allow_join_sorting(settings.allow_experimental_inner_join_right_table_sorting) , max_memory_usage(settings.max_memory_usage) , tmp_volume(tmp_volume_) , tmp_data(tmp_data_) diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index 09d7f0f2b2a..c7926271a67 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -151,6 +151,7 @@ private: const size_t output_by_rowlist_perkey_rows_threshold = 0; const size_t sort_right_perkey_rows_threshold = 0; const size_t sort_right_table_rows_threshold = 0; + const bool allow_join_sorting = false; /// Value if setting max_memory_usage for query, can be used when max_bytes_in_join is not specified. size_t max_memory_usage = 0; @@ -301,6 +302,7 @@ public: size_t outputByRowListPerkeyRowsThreshold() const { return output_by_rowlist_perkey_rows_threshold; } size_t sortRightPerkeyRowsThreshold() const { return sort_right_perkey_rows_threshold; } size_t sortRightTableRowsThreshold() const { return sort_right_table_rows_threshold; } + bool allowJoinSorting() const { return allow_join_sorting; } size_t defaultMaxBytes() const { return default_max_bytes; } size_t maxJoinedBlockRows() const { return max_joined_block_rows; } size_t maxRowsInRightBlock() const { return partial_merge_join_rows_in_right_blocks; } diff --git a/tests/performance/all_join_opt.xml b/tests/performance/all_join_opt.xml index 0ab9c39f67c..ed8805a2e5f 100644 --- a/tests/performance/all_join_opt.xml +++ b/tests/performance/all_join_opt.xml @@ -5,8 +5,8 @@ INSERT INTO test SELECT number % 10000, number % 10000, number % 10000 FROM numbers(10000000) INSERT INTO test1 SELECT number % 1000 , number % 1000, number % 1000 FROM numbers(100000) - SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b - SELECT MAX(test1.a) FROM test LEFT JOIN test1 on test.b = test1.b + SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b SETTINGS allow_experimental_inner_join_right_table_sorting=true, join_to_sort_table_rows_threshold=100000 + SELECT MAX(test1.a) FROM test LEFT JOIN test1 on test.b = test1.b SETTINGS allow_experimental_inner_join_right_table_sorting=true, join_to_sort_table_rows_threshold=100000 SELECT MAX(test1.a) FROM test RIGHT JOIN test1 on test.b = test1.b SELECT MAX(test1.a) FROM test FULL JOIN test1 on test.b = test1.b diff --git a/utils/check-style/experimental_settings_ignore.txt b/utils/check-style/experimental_settings_ignore.txt index 94c46cf562e..358374494a3 100644 --- a/utils/check-style/experimental_settings_ignore.txt +++ b/utils/check-style/experimental_settings_ignore.txt @@ -31,6 +31,7 @@ allow_experimental_statistics allow_experimental_time_series_table allow_experimental_undrop_table_query allow_experimental_usearch_index +allow_experimental_inner_join_right_table_sorting allow_get_client_http_header allow_introspection_functions allow_materialized_view_with_bad_select From 49548ed4d5b4fb30aeff0b020c5791537fbd21c1 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Wed, 4 Sep 2024 16:40:46 +0800 Subject: [PATCH 131/162] update the description --- src/Core/Settings.h | 4 ++-- src/Core/SettingsChangesHistory.cpp | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 386fd4e74ee..9b83507ce73 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -922,8 +922,8 @@ class IColumn; M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \ M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \ M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \ - M(Int32, join_to_sort_perkey_rows_threshold, 40, "Rerange the right table by key in hash join when the per-key average rows of it exceed this value (means the table keys is dense) and its number of rows is not too many(controlled by `join_to_sort_table_rows_threshold`), to make the join output by the key's batch, which would improve performance.", 0) \ - M(Int32, join_to_sort_table_rows_threshold, 10000, "Rerange the right table by key in hash join when its number of rows not exceed this value and the table keys is dense (controlled by `join_to_sort_perkey_rows_threshold`), to make the join performance improve as output by the key's batch, but not cost too much on the table reranging.", 0) \ + M(Int32, join_to_sort_perkey_rows_threshold, 40, "Rerange the right table by key in hash join when the per-key average rows of it exceed this value (means the table keys is dense) and its number of rows is not too many(controlled by `join_to_sort_table_rows_threshold`), to make the join output by the data batch of key, which would improve performance.", 0) \ + M(Int32, join_to_sort_table_rows_threshold, 10000, "Rerange the right table by key in hash join when its number of rows not exceed this value and the table keys is dense (controlled by `join_to_sort_perkey_rows_threshold`), to make the join performance improve as output by the data batch of key, but not cost too much on the table reranging.", 0) \ M(Bool, allow_experimental_inner_join_right_table_sorting, false, "If it is set to true, and the conditions of `join_to_sort_perkey_rows_threshold` and `join_to_sort_perkey_rows_threshold` are met, then we will try to rerange the right table by key to improve the performance in hash join.", 0) \ M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, use_hive_partitioning, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines.", 0)\ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 2a38af85b08..7bac9c314e7 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -95,8 +95,8 @@ static std::initializer_list Date: Thu, 5 Sep 2024 09:54:27 +0800 Subject: [PATCH 132/162] fix tests incompatible and add new test example --- tests/performance/all_join_opt.xml | 4 ++-- .../03228_join_to_rerange_right_table.reference | 2 ++ .../03228_join_to_rerange_right_table.sql | 14 ++++++++++++++ 3 files changed, 18 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03228_join_to_rerange_right_table.reference create mode 100644 tests/queries/0_stateless/03228_join_to_rerange_right_table.sql diff --git a/tests/performance/all_join_opt.xml b/tests/performance/all_join_opt.xml index ed8805a2e5f..0ab9c39f67c 100644 --- a/tests/performance/all_join_opt.xml +++ b/tests/performance/all_join_opt.xml @@ -5,8 +5,8 @@ INSERT INTO test SELECT number % 10000, number % 10000, number % 10000 FROM numbers(10000000) INSERT INTO test1 SELECT number % 1000 , number % 1000, number % 1000 FROM numbers(100000) - SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b SETTINGS allow_experimental_inner_join_right_table_sorting=true, join_to_sort_table_rows_threshold=100000 - SELECT MAX(test1.a) FROM test LEFT JOIN test1 on test.b = test1.b SETTINGS allow_experimental_inner_join_right_table_sorting=true, join_to_sort_table_rows_threshold=100000 + SELECT MAX(test1.a) FROM test INNER JOIN test1 on test.b = test1.b + SELECT MAX(test1.a) FROM test LEFT JOIN test1 on test.b = test1.b SELECT MAX(test1.a) FROM test RIGHT JOIN test1 on test.b = test1.b SELECT MAX(test1.a) FROM test FULL JOIN test1 on test.b = test1.b diff --git a/tests/queries/0_stateless/03228_join_to_rerange_right_table.reference b/tests/queries/0_stateless/03228_join_to_rerange_right_table.reference new file mode 100644 index 00000000000..b62923296e5 --- /dev/null +++ b/tests/queries/0_stateless/03228_join_to_rerange_right_table.reference @@ -0,0 +1,2 @@ +9 +9 diff --git a/tests/queries/0_stateless/03228_join_to_rerange_right_table.sql b/tests/queries/0_stateless/03228_join_to_rerange_right_table.sql new file mode 100644 index 00000000000..e9001f92405 --- /dev/null +++ b/tests/queries/0_stateless/03228_join_to_rerange_right_table.sql @@ -0,0 +1,14 @@ +drop table if exists test_left; +drop table if exists test_right; + +CREATE TABLE test_left (a Int64, b String, c LowCardinality(String)) ENGINE = MergeTree() ORDER BY a; +CREATE TABLE test_right (a Int64, b String, c LowCardinality(String)) ENGINE = MergeTree() ORDER BY a; + +INSERT INTO test_left SELECT number % 10000, number % 10000, number % 10000 FROM numbers(100000); +INSERT INTO test_right SELECT number % 10 , number % 10, number % 10 FROM numbers(10000); + +SELECT MAX(test_right.a) FROM test_left INNER JOIN test_right on test_left.b = test_right.b SETTINGS allow_experimental_inner_join_right_table_sorting=true; +SELECT MAX(test_right.a) FROM test_left LEFT JOIN test_right on test_left.b = test_right.b SETTINGS allow_experimental_inner_join_right_table_sorting=true; + +drop table test_left; +drop table test_right; From 20eaecc4f39adf73ac402c88d4a54d70f859453c Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Thu, 5 Sep 2024 13:50:26 +0200 Subject: [PATCH 133/162] Fix build --- src/Storages/MergeTree/MergeTask.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index e6d7b4656c9..398a9472456 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1585,8 +1585,8 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const std::vector plans; for (size_t i = 0; i < global_ctx->future_part->parts.size(); ++i) { - if (part->getMarksCount() == 0) - LOG_TRACE(ctx->log, "Part {} is empty", part->name); + if (global_ctx->future_part->parts[i]->getMarksCount() == 0) + LOG_TRACE(ctx->log, "Part {} is empty", global_ctx->future_part->parts[i]->name); auto plan_for_part = std::make_unique(); createReadFromPartStep( From f8b6025e078a1cf7327154f389b0c32ffbf77a94 Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Fri, 6 Sep 2024 09:43:32 +0800 Subject: [PATCH 134/162] rename to allow_experimental_join_right_table_sorting and modify comments --- src/Core/Settings.h | 6 +++--- src/Core/SettingsChangesHistory.cpp | 6 +++--- utils/check-style/experimental_settings_ignore.txt | 2 +- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 9b83507ce73..88555c67e24 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -922,9 +922,9 @@ class IColumn; M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \ M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \ M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \ - M(Int32, join_to_sort_perkey_rows_threshold, 40, "Rerange the right table by key in hash join when the per-key average rows of it exceed this value (means the table keys is dense) and its number of rows is not too many(controlled by `join_to_sort_table_rows_threshold`), to make the join output by the data batch of key, which would improve performance.", 0) \ - M(Int32, join_to_sort_table_rows_threshold, 10000, "Rerange the right table by key in hash join when its number of rows not exceed this value and the table keys is dense (controlled by `join_to_sort_perkey_rows_threshold`), to make the join performance improve as output by the data batch of key, but not cost too much on the table reranging.", 0) \ - M(Bool, allow_experimental_inner_join_right_table_sorting, false, "If it is set to true, and the conditions of `join_to_sort_perkey_rows_threshold` and `join_to_sort_perkey_rows_threshold` are met, then we will try to rerange the right table by key to improve the performance in hash join.", 0) \ + M(Int32, join_to_sort_perkey_rows_threshold, 40, "Rerange the right table by key in left or inner hash join when the per-key average rows of it exceed this value (means the table keys is dense) and its number of rows is not too many(controlled by `join_to_sort_table_rows_threshold`), to make the join output by the data batch of key, which would improve performance.", 0) \ + M(Int32, join_to_sort_table_rows_threshold, 10000, "Rerange the right table by key in left or inner hash join when its number of rows not exceed this value and the table keys is dense (controlled by `join_to_sort_perkey_rows_threshold`), to make the join performance improve as output by the data batch of key, but not cost too much on the table reranging.", 0) \ + M(Bool, allow_experimental_join_right_table_sorting, false, "If it is set to true, and the conditions of `join_to_sort_perkey_rows_threshold` and `join_to_sort_perkey_rows_threshold` are met, then we will try to rerange the right table by key to improve the performance in left or inner hash join.", 0) \ M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, use_hive_partitioning, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines.", 0)\ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 7bac9c314e7..4ac7a0f2d8d 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -95,9 +95,9 @@ static std::initializer_list Date: Fri, 6 Sep 2024 10:15:43 +0800 Subject: [PATCH 135/162] modify test --- src/Interpreters/TableJoin.cpp | 2 +- .../queries/0_stateless/03228_join_to_rerange_right_table.sql | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 8d79b88190b..519264dd0e4 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -118,7 +118,7 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary , output_by_rowlist_perkey_rows_threshold(settings.join_output_by_rowlist_perkey_rows_threshold) , sort_right_perkey_rows_threshold(settings.join_to_sort_perkey_rows_threshold) , sort_right_table_rows_threshold(settings.join_to_sort_table_rows_threshold) - , allow_join_sorting(settings.allow_experimental_inner_join_right_table_sorting) + , allow_join_sorting(settings.allow_experimental_join_right_table_sorting) , max_memory_usage(settings.max_memory_usage) , tmp_volume(tmp_volume_) , tmp_data(tmp_data_) diff --git a/tests/queries/0_stateless/03228_join_to_rerange_right_table.sql b/tests/queries/0_stateless/03228_join_to_rerange_right_table.sql index e9001f92405..f3ee0f0b933 100644 --- a/tests/queries/0_stateless/03228_join_to_rerange_right_table.sql +++ b/tests/queries/0_stateless/03228_join_to_rerange_right_table.sql @@ -7,8 +7,8 @@ CREATE TABLE test_right (a Int64, b String, c LowCardinality(String)) ENGINE = M INSERT INTO test_left SELECT number % 10000, number % 10000, number % 10000 FROM numbers(100000); INSERT INTO test_right SELECT number % 10 , number % 10, number % 10 FROM numbers(10000); -SELECT MAX(test_right.a) FROM test_left INNER JOIN test_right on test_left.b = test_right.b SETTINGS allow_experimental_inner_join_right_table_sorting=true; -SELECT MAX(test_right.a) FROM test_left LEFT JOIN test_right on test_left.b = test_right.b SETTINGS allow_experimental_inner_join_right_table_sorting=true; +SELECT MAX(test_right.a) FROM test_left INNER JOIN test_right on test_left.b = test_right.b SETTINGS allow_experimental_join_right_table_sorting=true; +SELECT MAX(test_right.a) FROM test_left LEFT JOIN test_right on test_left.b = test_right.b SETTINGS allow_experimental_join_right_table_sorting=true; drop table test_left; drop table test_right; From ebd9c294aa32db185e280cba7407f7e6ef2730d0 Mon Sep 17 00:00:00 2001 From: Igor Nikonov Date: Fri, 6 Sep 2024 20:09:48 +0000 Subject: [PATCH 136/162] Fix build --- src/Processors/QueryPlan/ReadFromMergeTree.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index f9accaaec57..218f0a61a48 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -355,7 +355,7 @@ std::unique_ptr ReadFromMergeTree::createLocalParallelReplica const bool enable_parallel_reading = true; return std::make_unique( prepared_parts, - alter_conversions_for_parts, + mutations_snapshot, all_column_names, data, getQueryInfo(), From 1bcc4ba823805bed282133fb7035b73598641fc6 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 9 Sep 2024 15:30:19 +0200 Subject: [PATCH 137/162] Renamed ApplyMergeStep into MergePartsStep --- src/Storages/MergeTree/MergeTask.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 398a9472456..3ca909a2d09 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1321,10 +1321,10 @@ bool MergeTask::execute() /// Apply merge strategy (Ordinary, Colapsing, Aggregating, etc) to the stream -class ApplyMergeStep : public ITransformingStep +class MergePartsStep : public ITransformingStep { public: - ApplyMergeStep( + MergePartsStep( const DataStream & input_stream_, const SortDescription & sort_description_, const Names partition_key_columns_, @@ -1347,7 +1347,7 @@ public: , time_of_merge(time_of_merge_) {} - String getName() const override { return "ApplyMergePolicy"; } + String getName() const override { return "MergeParts"; } void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & /*pipelineSettings*/) override { @@ -1651,7 +1651,7 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const if (global_ctx->cleanup && !data_settings->allow_experimental_replacing_merge_with_cleanup) throw Exception(ErrorCodes::SUPPORT_IS_DISABLED, "Experimental merges with CLEANUP are not allowed"); - auto merge_step = std::make_unique( + auto merge_step = std::make_unique( merge_parts_query_plan.getCurrentDataStream(), sort_description, partition_key_columns, From 8c1f434b1ac2c9fbb83561a43a6ee10f20d81974 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 9 Sep 2024 15:31:43 +0200 Subject: [PATCH 138/162] Do column materialization using ActionsDAG::addMaterializingOutputActions instead of a special step --- src/Storages/MergeTree/MergeTask.cpp | 42 +--------------------------- 1 file changed, 1 insertion(+), 41 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 3ca909a2d09..33cdff10b6a 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1456,45 +1456,6 @@ private: const time_t time_of_merge{0}; }; - -class MaterializingStep : public ITransformingStep -{ -public: - explicit MaterializingStep( - const DataStream & input_stream_) - : ITransformingStep(input_stream_, input_stream_.header, getTraits()) - {} - - String getName() const override { return "Materializing"; } - - void transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override - { - pipeline.addTransform(std::make_shared(input_streams.front().header)); - } - - void updateOutputStream() override - { - output_stream = createOutputStream(input_streams.front(), input_streams.front().header, getDataStreamTraits()); - } - -private: - static Traits getTraits() - { - return ITransformingStep::Traits - { - { - .returns_single_stream = true, - .preserves_number_of_streams = true, - .preserves_sorting = true, - }, - { - .preserves_number_of_rows = true, - } - }; - } -}; - - class TTLStep : public ITransformingStep { public: @@ -1709,12 +1670,11 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const if (!global_ctx->merging_skip_indexes.empty()) { auto indices_expression_dag = global_ctx->merging_skip_indexes.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext())->getActionsDAG().clone(); + indices_expression_dag.addMaterializingOutputActions(); /// Const columns cannot be written without materialization. auto calculate_indices_expression_step = std::make_unique( merge_parts_query_plan.getCurrentDataStream(), std::move(indices_expression_dag)); merge_parts_query_plan.addStep(std::move(calculate_indices_expression_step)); - /// TODO: what is the purpose of MaterializingTransform in the original code? - merge_parts_query_plan.addStep(std::make_unique(merge_parts_query_plan.getCurrentDataStream())); } if (!subqueries.empty()) From 4da1e10ac66059b47a89e69327ea79d487e2847f Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Mon, 9 Sep 2024 16:01:00 +0200 Subject: [PATCH 139/162] Move sorting key calculation step outside the loop --- src/Storages/MergeTree/MergeTask.cpp | 21 ++++++++++----------- 1 file changed, 10 insertions(+), 11 deletions(-) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 33cdff10b6a..9a1e749734c 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -1540,8 +1540,6 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const global_ctx->horizontal_stage_progress = std::make_unique( ctx->column_sizes ? ctx->column_sizes->keyColumnsWeight() : 1.0); - auto sorting_key_expression_dag = global_ctx->metadata_snapshot->getSortingKey().expression->getActionsDAG().clone(); - /// Read from all parts std::vector plans; for (size_t i = 0; i < global_ctx->future_part->parts.size(); ++i) @@ -1566,15 +1564,6 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const global_ctx->context, ctx->log); - if (global_ctx->metadata_snapshot->hasSortingKey()) - { - /// Calculate sorting key expressions so that they are available for merge sorting. - auto calculate_sorting_key_expression_step = std::make_unique( - plan_for_part->getCurrentDataStream(), - sorting_key_expression_dag.clone()); /// TODO: can we avoid cloning here? - plan_for_part->addStep(std::move(calculate_sorting_key_expression_step)); - } - plans.emplace_back(std::move(plan_for_part)); } @@ -1591,6 +1580,16 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream() const merge_parts_query_plan.unitePlans(std::move(union_step), std::move(plans)); } + if (global_ctx->metadata_snapshot->hasSortingKey()) + { + /// Calculate sorting key expressions so that they are available for merge sorting. + auto sorting_key_expression_dag = global_ctx->metadata_snapshot->getSortingKey().expression->getActionsDAG().clone(); + auto calculate_sorting_key_expression_step = std::make_unique( + merge_parts_query_plan.getCurrentDataStream(), + std::move(sorting_key_expression_dag)); + merge_parts_query_plan.addStep(std::move(calculate_sorting_key_expression_step)); + } + /// Merge { Names sort_columns = global_ctx->metadata_snapshot->getSortingKeyColumns(); From 597181c45e2395991cbb032c7eb2dc3542124e6c Mon Sep 17 00:00:00 2001 From: kevinyhzou Date: Tue, 10 Sep 2024 16:32:52 +0800 Subject: [PATCH 140/162] review --- src/Core/Settings.h | 6 +++--- src/Core/SettingsChangesHistory.cpp | 6 +++--- src/Interpreters/HashJoin/HashJoin.cpp | 2 +- src/Interpreters/TableJoin.cpp | 4 ++-- src/Interpreters/TableJoin.h | 8 ++++---- 5 files changed, 13 insertions(+), 13 deletions(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index 88555c67e24..28041089d9f 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -922,9 +922,9 @@ class IColumn; M(Bool, implicit_transaction, false, "If enabled and not already inside a transaction, wraps the query inside a full transaction (begin + commit or rollback)", 0) \ M(UInt64, grace_hash_join_initial_buckets, 1, "Initial number of grace hash join buckets", 0) \ M(UInt64, grace_hash_join_max_buckets, 1024, "Limit on the number of grace hash join buckets", 0) \ - M(Int32, join_to_sort_perkey_rows_threshold, 40, "Rerange the right table by key in left or inner hash join when the per-key average rows of it exceed this value (means the table keys is dense) and its number of rows is not too many(controlled by `join_to_sort_table_rows_threshold`), to make the join output by the data batch of key, which would improve performance.", 0) \ - M(Int32, join_to_sort_table_rows_threshold, 10000, "Rerange the right table by key in left or inner hash join when its number of rows not exceed this value and the table keys is dense (controlled by `join_to_sort_perkey_rows_threshold`), to make the join performance improve as output by the data batch of key, but not cost too much on the table reranging.", 0) \ - M(Bool, allow_experimental_join_right_table_sorting, false, "If it is set to true, and the conditions of `join_to_sort_perkey_rows_threshold` and `join_to_sort_perkey_rows_threshold` are met, then we will try to rerange the right table by key to improve the performance in left or inner hash join.", 0) \ + M(Int32, join_to_sort_minimum_perkey_rows, 40, "The lower limit of per-key average rows in the right table to determine whether to rerange the right table by key in left or inner join. This setting ensures that the optimization is not applied for sparse table keys", 0) \ + M(Int32, join_to_sort_maximum_table_rows, 10000, "The maximum number of rows in the right table to determine whether to rerange the right table by key in left or inner join.", 0) \ + M(Bool, allow_experimental_join_right_table_sorting, false, "If it is set to true, and the conditions of `join_to_sort_minimum_perkey_rows` and `join_to_sort_maximum_table_rows` are met, rerange the right table by key to improve the performance in left or inner hash join.", 0) \ M(Timezone, session_timezone, "", "This setting can be removed in the future due to potential caveats. It is experimental and is not suitable for production usage. The default timezone for current session or query. The server default timezone if empty.", 0) \ M(Bool, use_hive_partitioning, false, "Allows to use hive partitioning for File, URL, S3, AzureBlobStorage and HDFS engines.", 0)\ \ diff --git a/src/Core/SettingsChangesHistory.cpp b/src/Core/SettingsChangesHistory.cpp index 4ac7a0f2d8d..da29b6b11cd 100644 --- a/src/Core/SettingsChangesHistory.cpp +++ b/src/Core/SettingsChangesHistory.cpp @@ -95,9 +95,9 @@ static std::initializer_listsorted || data->blocks.empty() || data->maps.size() > 1 || data->rows_to_join > table_join->sortRightTableRowsThreshold() || data->avgPerKeyRows() < table_join->sortRightPerkeyRowsThreshold()) + if (!data || data->sorted || data->blocks.empty() || data->maps.size() > 1 || data->rows_to_join > table_join->sortRightMaximumTableRows() || data->avgPerKeyRows() < table_join->sortRightMinimumPerkeyRows()) return; if (data->keys_to_join == 0) diff --git a/src/Interpreters/TableJoin.cpp b/src/Interpreters/TableJoin.cpp index 519264dd0e4..59a0374051f 100644 --- a/src/Interpreters/TableJoin.cpp +++ b/src/Interpreters/TableJoin.cpp @@ -116,8 +116,8 @@ TableJoin::TableJoin(const Settings & settings, VolumePtr tmp_volume_, Temporary , max_files_to_merge(settings.join_on_disk_max_files_to_merge) , temporary_files_codec(settings.temporary_files_codec) , output_by_rowlist_perkey_rows_threshold(settings.join_output_by_rowlist_perkey_rows_threshold) - , sort_right_perkey_rows_threshold(settings.join_to_sort_perkey_rows_threshold) - , sort_right_table_rows_threshold(settings.join_to_sort_table_rows_threshold) + , sort_right_minimum_perkey_rows(settings.join_to_sort_minimum_perkey_rows) + , sort_right_maximum_table_rows(settings.join_to_sort_maximum_table_rows) , allow_join_sorting(settings.allow_experimental_join_right_table_sorting) , max_memory_usage(settings.max_memory_usage) , tmp_volume(tmp_volume_) diff --git a/src/Interpreters/TableJoin.h b/src/Interpreters/TableJoin.h index c7926271a67..e1bae55a4ed 100644 --- a/src/Interpreters/TableJoin.h +++ b/src/Interpreters/TableJoin.h @@ -149,8 +149,8 @@ private: const size_t max_files_to_merge = 0; const String temporary_files_codec = "LZ4"; const size_t output_by_rowlist_perkey_rows_threshold = 0; - const size_t sort_right_perkey_rows_threshold = 0; - const size_t sort_right_table_rows_threshold = 0; + const size_t sort_right_minimum_perkey_rows = 0; + const size_t sort_right_maximum_table_rows = 0; const bool allow_join_sorting = false; /// Value if setting max_memory_usage for query, can be used when max_bytes_in_join is not specified. @@ -300,8 +300,8 @@ public: } size_t outputByRowListPerkeyRowsThreshold() const { return output_by_rowlist_perkey_rows_threshold; } - size_t sortRightPerkeyRowsThreshold() const { return sort_right_perkey_rows_threshold; } - size_t sortRightTableRowsThreshold() const { return sort_right_table_rows_threshold; } + size_t sortRightMinimumPerkeyRows() const { return sort_right_minimum_perkey_rows; } + size_t sortRightMaximumTableRows() const { return sort_right_maximum_table_rows; } bool allowJoinSorting() const { return allow_join_sorting; } size_t defaultMaxBytes() const { return default_max_bytes; } size_t maxJoinedBlockRows() const { return max_joined_block_rows; } From 6719112b02e67ecc24346ac81fdb504f97e047a9 Mon Sep 17 00:00:00 2001 From: kssenii Date: Tue, 10 Sep 2024 15:08:58 +0200 Subject: [PATCH 141/162] Add azure-queue.md --- .../table-engines/integrations/azure-queue.md | 72 +++++++++++++++++++ .../engines/table-engines/integrations/s3.md | 2 +- .../table-engines/integrations/s3queue.md | 31 ++++---- 3 files changed, 88 insertions(+), 17 deletions(-) create mode 100644 docs/en/engines/table-engines/integrations/azure-queue.md diff --git a/docs/en/engines/table-engines/integrations/azure-queue.md b/docs/en/engines/table-engines/integrations/azure-queue.md new file mode 100644 index 00000000000..7c3d510ddcc --- /dev/null +++ b/docs/en/engines/table-engines/integrations/azure-queue.md @@ -0,0 +1,72 @@ +--- +slug: /en/engines/table-engines/integrations/azure-queue +sidebar_position: 181 +sidebar_label: AzureQueue +--- + +# AzureQueue Table Engine + +This engine provides an integration with [Azure Blob Storage](https://azure.microsoft.com/en-us/products/storage/blobs) ecosystem, allowing streaming data import. + +## Create Table {#creating-a-table} + +``` sql +CREATE TABLE test (name String, value UInt32) + ENGINE = AzureQueue(...) + [SETTINGS] + [mode = '',] + [after_processing = 'keep',] + [keeper_path = '',] + ... +``` + +**Engine parameters** + +`AzureQueue` parameters are the same as `AzureBlobStorage` table engine supports. See parameters section [here](../../../engines/table-engines/integrations/azureBlobStorage.md). + +**Example** + +```sql +CREATE TABLE s3queue_engine_table (name String, value UInt32) +ENGINE=AzureQueue('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/data/') +SETTINGS + mode = 'unordered' +``` + +## Settings {#settings} + +The set of supported settings is the same as for `S3Queue` table engine, but without `s3queue_` prefix. See [full list of settings settings](../../../engines/table-engines/integrations/s3queue.md#settings). + +## Description {#description} + +`SELECT` is not particularly useful for streaming import (except for debugging), because each file can be imported only once. It is more practical to create real-time threads using [materialized views](../../../sql-reference/statements/create/view.md). To do this: + +1. Use the engine to create a table for consuming from specified path in S3 and consider it a data stream. +2. Create a table with the desired structure. +3. Create a materialized view that converts data from the engine and puts it into a previously created table. + +When the `MATERIALIZED VIEW` joins the engine, it starts collecting data in the background. + +Example: + +``` sql + CREATE TABLE s3queue_engine_table (name String, value UInt32) + ENGINE=AzureQueue('', 'CSV', 'gzip') + SETTINGS + mode = 'unordered'; + + CREATE TABLE stats (name String, value UInt32) + ENGINE = MergeTree() ORDER BY name; + + CREATE MATERIALIZED VIEW consumer TO stats + AS SELECT name, value FROM s3queue_engine_table; + + SELECT * FROM stats ORDER BY name; +``` + +## Virtual columns {#virtual-columns} + +- `_path` — Path to the file. +- `_file` — Name of the file. + +For more information about virtual columns see [here](../../../engines/table-engines/index.md#table_engines-virtual_columns). diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 48a08dfa499..f02d0563491 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -35,7 +35,7 @@ CREATE TABLE s3_engine_table (name String, value UInt32) [SETTINGS ...] ``` -### Engine parameters +### Engine parameters {#parameters} - `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path). - `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed. diff --git a/docs/en/engines/table-engines/integrations/s3queue.md b/docs/en/engines/table-engines/integrations/s3queue.md index 06325fa15fb..f1957cfe1ce 100644 --- a/docs/en/engines/table-engines/integrations/s3queue.md +++ b/docs/en/engines/table-engines/integrations/s3queue.md @@ -5,6 +5,7 @@ sidebar_label: S3Queue --- # S3Queue Table Engine + This engine provides integration with [Amazon S3](https://aws.amazon.com/s3/) ecosystem and allows streaming import. This engine is similar to the [Kafka](../../../engines/table-engines/integrations/kafka.md), [RabbitMQ](../../../engines/table-engines/integrations/rabbitmq.md) engines, but provides S3-specific features. ## Create Table {#creating-a-table} @@ -16,27 +17,25 @@ CREATE TABLE s3_queue_engine_table (name String, value UInt32) [mode = '',] [after_processing = 'keep',] [keeper_path = '',] - [s3queue_loading_retries = 0,] - [s3queue_processing_threads_num = 1,] - [s3queue_enable_logging_to_s3queue_log = 0,] - [s3queue_polling_min_timeout_ms = 1000,] - [s3queue_polling_max_timeout_ms = 10000,] - [s3queue_polling_backoff_ms = 0,] - [s3queue_tracked_file_ttl_sec = 0,] - [s3queue_tracked_files_limit = 1000,] - [s3queue_cleanup_interval_min_ms = 10000,] - [s3queue_cleanup_interval_max_ms = 30000,] + [loading_retries = 0,] + [processing_threads_num = 1,] + [enable_logging_to_s3queue_log = 0,] + [polling_min_timeout_ms = 1000,] + [polling_max_timeout_ms = 10000,] + [polling_backoff_ms = 0,] + [tracked_file_ttl_sec = 0,] + [tracked_files_limit = 1000,] + [cleanup_interval_min_ms = 10000,] + [cleanup_interval_max_ms = 30000,] ``` -Starting with `24.7` settings without `s3queue_` prefix are also supported. +:::warning +Before `24.7`, it is required to use `s3queue_` prefix for all settings apart from `mode`, `after_processing` and `keeper_path`. +::: **Engine parameters** -- `path` — Bucket url with path to file. Supports following wildcards in readonly mode: `*`, `**`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, `'abc'`, `'def'` — strings. For more information see [below](#wildcards-in-path). -- `NOSIGN` - If this keyword is provided in place of credentials, all the requests will not be signed. -- `format` — The [format](../../../interfaces/formats.md#formats) of the file. -- `aws_access_key_id`, `aws_secret_access_key` - Long-term credentials for the [AWS](https://aws.amazon.com/) account user. You can use these to authenticate your requests. Parameter is optional. If credentials are not specified, they are used from the configuration file. For more information see [Using S3 for Data Storage](../mergetree-family/mergetree.md#table_engine-mergetree-s3). -- `compression` — Compression type. Supported values: `none`, `gzip/gz`, `brotli/br`, `xz/LZMA`, `zstd/zst`. Parameter is optional. By default, it will autodetect compression by file extension. +`S3Queue` parameters are the same as `S3` table engine supports. See parameters section [here](../../../engines/table-engines/integrations/s3.md#parameters). **Example** From acbeaa6d3339b7206c6f9e822445952fe664a807 Mon Sep 17 00:00:00 2001 From: Nikita Taranov Date: Tue, 10 Sep 2024 16:14:53 +0100 Subject: [PATCH 142/162] fix --- tests/integration/test_disks_app_func/test.py | 14 +++++++++++++- 1 file changed, 13 insertions(+), 1 deletion(-) diff --git a/tests/integration/test_disks_app_func/test.py b/tests/integration/test_disks_app_func/test.py index 56ea5c8846a..a4b2399e117 100644 --- a/tests/integration/test_disks_app_func/test.py +++ b/tests/integration/test_disks_app_func/test.py @@ -13,8 +13,20 @@ def started_cluster(): main_configs=["config.xml"], with_minio=True, ) - cluster.start() + + # local disk requires its `path` directory to exist. + # the two paths below belong to `test1` and `test2` disks + node = cluster.instances["disks_app_test"] + for path in ["path1", "path2"]: + node.exec_in_container( + [ + "bash", + "-c", + f"mkdir -p /var/lib/clickhouse/{path}", + ] + ) + yield cluster finally: From 04dcf73e8fb98fbcdee2ead46e7177c961e1b5f6 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 10 Sep 2024 15:35:59 +0000 Subject: [PATCH 143/162] Fix crash in sqidDecode --- src/Functions/sqid.cpp | 2 +- tests/queries/0_stateless/02933_sqid.reference | 1 + tests/queries/0_stateless/02933_sqid.sql | 7 +++++++ 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/src/Functions/sqid.cpp b/src/Functions/sqid.cpp index 0e133590b84..32434eb5e73 100644 --- a/src/Functions/sqid.cpp +++ b/src/Functions/sqid.cpp @@ -124,7 +124,7 @@ public: std::string_view sqid = col_non_const->getDataAt(i).toView(); std::vector integers = sqids.decode(String(sqid)); res_nested_data.insert(integers.begin(), integers.end()); - res_offsets_data.push_back(integers.size()); + res_offsets_data.push_back(i == 0 ? integers.size() : res_offsets_data.back() + integers.size()); } } else diff --git a/tests/queries/0_stateless/02933_sqid.reference b/tests/queries/0_stateless/02933_sqid.reference index a559bacb0ac..4597e2347e3 100644 --- a/tests/queries/0_stateless/02933_sqid.reference +++ b/tests/queries/0_stateless/02933_sqid.reference @@ -13,5 +13,6 @@ Td1EnWQo [1,2,3,4] XMbT -- invalid sqid [] +-- bug 69450 -- alias XMbT diff --git a/tests/queries/0_stateless/02933_sqid.sql b/tests/queries/0_stateless/02933_sqid.sql index 81d4b2bc35c..822fe33df51 100644 --- a/tests/queries/0_stateless/02933_sqid.sql +++ b/tests/queries/0_stateless/02933_sqid.sql @@ -25,5 +25,12 @@ SELECT sqidEncode(toNullable(materialize(1)), toLowCardinality(materialize(2))); SELECT '-- invalid sqid'; SELECT sqidDecode('invalid sqid'); +SELECT '-- bug 69450'; +DROP TABLE IF EXISTS tab; +CREATE TABLE tab (id String) ENGINE = MergeTree ORDER BY id; +INSERT INTO tab SELECT * FROM generateRandom() LIMIT 1000000; +SELECT sqidDecode(id) FROM tab FORMAT Null; +DROP TABLE tab; + SELECT '-- alias'; SELECT sqid(1, 2); From d8e670297bcdc1850260a1d60d4ee39b2a895a4d Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 10 Sep 2024 17:42:27 +0200 Subject: [PATCH 144/162] groupConcat consistency --- .../AggregateFunctionGroupConcat.cpp | 13 +++++++++++-- .../03235_groupArray_returns_string.reference | 1 + .../0_stateless/03235_groupArray_returns_string.sql | 10 ++++++++++ 3 files changed, 22 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/03235_groupArray_returns_string.reference create mode 100644 tests/queries/0_stateless/03235_groupArray_returns_string.sql diff --git a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp index 636ac80e350..8fb0b645096 100644 --- a/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp +++ b/src/AggregateFunctions/AggregateFunctionGroupConcat.cpp @@ -116,15 +116,17 @@ class GroupConcatImpl final SerializationPtr serialization; UInt64 limit; const String delimiter; + const DataTypePtr type; public: GroupConcatImpl(const DataTypePtr & data_type_, const Array & parameters_, UInt64 limit_, const String & delimiter_) : IAggregateFunctionDataHelper, GroupConcatImpl>( {data_type_}, parameters_, std::make_shared()) - , serialization(this->argument_types[0]->getDefaultSerialization()) , limit(limit_) , delimiter(delimiter_) + , type(data_type_) { + serialization = isFixedString(type) ? std::make_shared()->getDefaultSerialization() : this->argument_types[0]->getDefaultSerialization(); } String getName() const override { return name; } @@ -140,7 +142,14 @@ public: if (cur_data.data_size != 0) cur_data.insertChar(delimiter.c_str(), delimiter.size(), arena); - cur_data.insert(columns[0], serialization, row_num, arena); + if (isFixedString(type)) + { + ColumnWithTypeAndName col = {columns[0]->getPtr(), type, "column"}; + const auto & col_str = castColumn(col, std::make_shared()); + cur_data.insert(col_str.get(), serialization, row_num, arena); + } + else + cur_data.insert(columns[0], serialization, row_num, arena); } void merge(AggregateDataPtr __restrict place, ConstAggregateDataPtr rhs, Arena * arena) const override diff --git a/tests/queries/0_stateless/03235_groupArray_returns_string.reference b/tests/queries/0_stateless/03235_groupArray_returns_string.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/03235_groupArray_returns_string.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/03235_groupArray_returns_string.sql b/tests/queries/0_stateless/03235_groupArray_returns_string.sql new file mode 100644 index 00000000000..618ec6f839b --- /dev/null +++ b/tests/queries/0_stateless/03235_groupArray_returns_string.sql @@ -0,0 +1,10 @@ +CREATE TABLE t (st FixedString(54)) ENGINE=MergeTree ORDER BY (); + +INSERT INTO t VALUES +('abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRTUVWXYZ'), +('\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'), +('IIIIIIIIII\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0\0'); + +WITH (SELECT groupConcat(',')(st) FROM t) AS a, + (SELECT groupConcat(',')(st :: String) FROM t) AS b +SELECT equals(a, b); From f588e3c31bc2f097f168646f73ed078259265659 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 10 Sep 2024 17:43:50 +0200 Subject: [PATCH 145/162] rename tests --- ...ng.reference => 03235_groupArray_string_consistency.reference} | 0 ...returns_string.sql => 03235_groupArray_string_consistency.sql} | 0 2 files changed, 0 insertions(+), 0 deletions(-) rename tests/queries/0_stateless/{03235_groupArray_returns_string.reference => 03235_groupArray_string_consistency.reference} (100%) rename tests/queries/0_stateless/{03235_groupArray_returns_string.sql => 03235_groupArray_string_consistency.sql} (100%) diff --git a/tests/queries/0_stateless/03235_groupArray_returns_string.reference b/tests/queries/0_stateless/03235_groupArray_string_consistency.reference similarity index 100% rename from tests/queries/0_stateless/03235_groupArray_returns_string.reference rename to tests/queries/0_stateless/03235_groupArray_string_consistency.reference diff --git a/tests/queries/0_stateless/03235_groupArray_returns_string.sql b/tests/queries/0_stateless/03235_groupArray_string_consistency.sql similarity index 100% rename from tests/queries/0_stateless/03235_groupArray_returns_string.sql rename to tests/queries/0_stateless/03235_groupArray_string_consistency.sql From 080193cfc36909ab9a1016b70edc007c8dd3540f Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Tue, 10 Sep 2024 15:46:21 +0000 Subject: [PATCH 146/162] 14% more aesthetic code --- src/Functions/sqid.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Functions/sqid.cpp b/src/Functions/sqid.cpp index 32434eb5e73..074a34bd083 100644 --- a/src/Functions/sqid.cpp +++ b/src/Functions/sqid.cpp @@ -124,7 +124,7 @@ public: std::string_view sqid = col_non_const->getDataAt(i).toView(); std::vector integers = sqids.decode(String(sqid)); res_nested_data.insert(integers.begin(), integers.end()); - res_offsets_data.push_back(i == 0 ? integers.size() : res_offsets_data.back() + integers.size()); + res_offsets_data.push_back(res_offsets_data.back() + integers.size()); } } else From dcd1874520ce7934b3bd67e93b4441bf063138a6 Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 10 Sep 2024 18:12:56 +0200 Subject: [PATCH 147/162] Update docs/en/engines/table-engines/integrations/azure-queue.md Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- docs/en/engines/table-engines/integrations/azure-queue.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/engines/table-engines/integrations/azure-queue.md b/docs/en/engines/table-engines/integrations/azure-queue.md index 7c3d510ddcc..7e19bc92973 100644 --- a/docs/en/engines/table-engines/integrations/azure-queue.md +++ b/docs/en/engines/table-engines/integrations/azure-queue.md @@ -50,7 +50,7 @@ When the `MATERIALIZED VIEW` joins the engine, it starts collecting data in the Example: ``` sql - CREATE TABLE s3queue_engine_table (name String, value UInt32) + CREATE TABLE azure_queue_engine_table (name String, value UInt32) ENGINE=AzureQueue('', 'CSV', 'gzip') SETTINGS mode = 'unordered'; @@ -59,7 +59,7 @@ Example: ENGINE = MergeTree() ORDER BY name; CREATE MATERIALIZED VIEW consumer TO stats - AS SELECT name, value FROM s3queue_engine_table; + AS SELECT name, value FROM azure_queue_engine_table; SELECT * FROM stats ORDER BY name; ``` From b7e863b0e9b1891c5e4adaec61b5d51db6fc0fcd Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Tue, 10 Sep 2024 18:13:04 +0200 Subject: [PATCH 148/162] Update docs/en/engines/table-engines/integrations/azure-queue.md Co-authored-by: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> --- docs/en/engines/table-engines/integrations/azure-queue.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/azure-queue.md b/docs/en/engines/table-engines/integrations/azure-queue.md index 7e19bc92973..b5259336a8b 100644 --- a/docs/en/engines/table-engines/integrations/azure-queue.md +++ b/docs/en/engines/table-engines/integrations/azure-queue.md @@ -27,7 +27,7 @@ CREATE TABLE test (name String, value UInt32) **Example** ```sql -CREATE TABLE s3queue_engine_table (name String, value UInt32) +CREATE TABLE azure_queue_engine_table (name String, value UInt32) ENGINE=AzureQueue('DefaultEndpointsProtocol=http;AccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw==;BlobEndpoint=http://azurite1:10000/devstoreaccount1/data/') SETTINGS mode = 'unordered' From caab4dd8b903a8caaa45b0e9d811487110ac2560 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 10 Sep 2024 18:20:48 +0200 Subject: [PATCH 149/162] fix --- src/Interpreters/Squashing.cpp | 2 +- .../03236_squashing_high_memory.reference | 0 .../03236_squashing_high_memory.sql | 22 +++++++++++++++++++ 3 files changed, 23 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/03236_squashing_high_memory.reference create mode 100644 tests/queries/0_stateless/03236_squashing_high_memory.sql diff --git a/src/Interpreters/Squashing.cpp b/src/Interpreters/Squashing.cpp index 95b76c60063..c656a1a797b 100644 --- a/src/Interpreters/Squashing.cpp +++ b/src/Interpreters/Squashing.cpp @@ -45,7 +45,7 @@ Chunk Squashing::squash(Chunk && input_chunk) Chunk Squashing::add(Chunk && input_chunk) { - if (!input_chunk) + if (!input_chunk || input_chunk.getNumRows() == 0) return {}; /// Just read block is already enough. diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.reference b/tests/queries/0_stateless/03236_squashing_high_memory.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql new file mode 100644 index 00000000000..523281fb74a --- /dev/null +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -0,0 +1,22 @@ +DROP TABLE IF EXISTS id_values; + +DROP TABLE IF EXISTS test_table; + +CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS + SELECT arrayJoin(range(1000000)) AS id1, arrayJoin(range(1000)) AS id2; + +SET max_memory_usage = 1G; + +CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS +SELECT id_values.id1 AS id, + string_values.string_val1 AS string_val1, + string_values.string_val2 AS string_val2 +FROM id_values + JOIN (SELECT arrayJoin(range(10)) AS id1, + 'qwe' AS string_val1, + 'asd' AS string_val2) AS string_values + ON id_values.id1 = string_values.id1 + SETTINGS join_algorithm = 'hash'; + +DROP TABLE IF EXISTS id_values; +DROP TABLE IF EXISTS test_table; From df1821a5798b8d138c3da4b39517196ba06b447a Mon Sep 17 00:00:00 2001 From: Antonio Andelic Date: Tue, 10 Sep 2024 19:32:28 +0200 Subject: [PATCH 150/162] Update tests/queries/0_stateless/03236_squashing_high_memory.sql --- tests/queries/0_stateless/03236_squashing_high_memory.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index 523281fb74a..98b33b80e9f 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -3,7 +3,7 @@ DROP TABLE IF EXISTS id_values; DROP TABLE IF EXISTS test_table; CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS - SELECT arrayJoin(range(1000000)) AS id1, arrayJoin(range(1000)) AS id2; + SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; SET max_memory_usage = 1G; From 9b517a939fce931da57a7d0604dc366ef35bf3fd Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 10 Sep 2024 19:53:16 +0200 Subject: [PATCH 151/162] Update 03236_squashing_high_memory.sql --- tests/queries/0_stateless/03236_squashing_high_memory.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index 98b33b80e9f..78316597430 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -5,7 +5,7 @@ DROP TABLE IF EXISTS test_table; CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; -SET max_memory_usage = 1G; +SET max_memory_usage = '1G'; CREATE TABLE test_table ENGINE MergeTree ORDER BY id AS SELECT id_values.id1 AS id, From 63aebfa5d7d410e1b33a1f914d02a8e75f52d5ea Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 10 Sep 2024 20:15:36 +0200 Subject: [PATCH 152/162] Update 03236_squashing_high_memory.sql --- tests/queries/0_stateless/03236_squashing_high_memory.sql | 3 +++ 1 file changed, 3 insertions(+) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index 78316597430..df296cb6713 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -1,3 +1,6 @@ +-- Tags: no-fasttest +-- reason: test requires too many rows to read + DROP TABLE IF EXISTS id_values; DROP TABLE IF EXISTS test_table; From 2f15fcd23fb069a69d2f1a0caee36ae33f39fe45 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 10 Sep 2024 20:57:03 +0200 Subject: [PATCH 153/162] Test with sparse serialization, vertical merge and skip indices --- .../03175_sparse_and_skip_index.reference | 4 ++ .../03175_sparse_and_skip_index.sql | 45 +++++++++++++++++++ 2 files changed, 49 insertions(+) create mode 100644 tests/queries/0_stateless/03175_sparse_and_skip_index.reference create mode 100644 tests/queries/0_stateless/03175_sparse_and_skip_index.sql diff --git a/tests/queries/0_stateless/03175_sparse_and_skip_index.reference b/tests/queries/0_stateless/03175_sparse_and_skip_index.reference new file mode 100644 index 00000000000..619e98a152a --- /dev/null +++ b/tests/queries/0_stateless/03175_sparse_and_skip_index.reference @@ -0,0 +1,4 @@ +key Sparse +value Sparse +1000 +1 diff --git a/tests/queries/0_stateless/03175_sparse_and_skip_index.sql b/tests/queries/0_stateless/03175_sparse_and_skip_index.sql new file mode 100644 index 00000000000..4de6d1ac6df --- /dev/null +++ b/tests/queries/0_stateless/03175_sparse_and_skip_index.sql @@ -0,0 +1,45 @@ +DROP TABLE IF EXISTS t_bloom_filter; +CREATE TABLE t_bloom_filter( + key UInt64, + value UInt64, + + INDEX key_bf key TYPE bloom_filter(0.01) GRANULARITY 2147483648, -- bloom filter on sorting key column + INDEX value_bf value TYPE bloom_filter(0.01) GRANULARITY 2147483648 -- bloom filter on no-sorting column +) ENGINE=MergeTree ORDER BY key +SETTINGS + -- settings to trigger sparse serialization and vertical merge + ratio_of_defaults_for_sparse_serialization = 0.0 + ,vertical_merge_algorithm_min_rows_to_activate = 1 + ,vertical_merge_algorithm_min_columns_to_activate = 1 + ,allow_vertical_merges_from_compact_to_wide_parts = 1 + ,min_bytes_for_wide_part=0 +; + +SYSTEM STOP MERGES t_bloom_filter; + +-- Create at least one part +INSERT INTO t_bloom_filter +SELECT + number % 100 as key, -- 100 unique keys + rand() % 100 as value -- 100 unique values +FROM numbers(50_000); + +-- And another part +INSERT INTO t_bloom_filter +SELECT + number % 100 as key, -- 100 unique keys + rand() % 100 as value -- 100 unique values +FROM numbers(50_000, 50_000); + +SYSTEM START MERGES t_bloom_filter; + +-- Merge everything into a single part +OPTIMIZE TABLE t_bloom_filter FINAL; + +-- Check sparse serialization +SELECT column, serialization_kind FROM system.parts_columns WHERE database = currentDatabase() AND table = 't_bloom_filter' AND active ORDER BY column; + +SELECT COUNT() FROM t_bloom_filter WHERE key = 1; + +-- Check bloom filter non-zero size +SELECT COUNT() FROM system.parts WHERE database = currentDatabase() AND table = 't_bloom_filter' AND secondary_indices_uncompressed_bytes > 200 AND active; From d4aa06524b4e65e2bca4ff851ad54dd4468e5cc0 Mon Sep 17 00:00:00 2001 From: Alexander Gololobov Date: Tue, 10 Sep 2024 20:57:55 +0200 Subject: [PATCH 154/162] Add materialization when building indices in vertical merge --- src/Storages/MergeTree/MergeTask.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index 9a1e749734c..5c993504245 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -952,6 +952,7 @@ MergeTask::VerticalMergeRuntimeContext::PreparedColumnPipeline MergeTask::Vertic indexes_to_recalc = MergeTreeIndexFactory::instance().getMany(indexes_it->second); auto indices_expression_dag = indexes_it->second.getSingleExpressionForIndices(global_ctx->metadata_snapshot->getColumns(), global_ctx->data->getContext())->getActionsDAG().clone(); + indices_expression_dag.addMaterializingOutputActions(); /// Const columns cannot be written without materialization. auto calculate_indices_expression_step = std::make_unique( merge_column_query_plan.getCurrentDataStream(), std::move(indices_expression_dag)); From 3d36f6dce36595e8796f3d7361ca8ffc1e4bab38 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Tue, 10 Sep 2024 23:48:32 +0200 Subject: [PATCH 155/162] Update 03236_squashing_high_memory.sql --- tests/queries/0_stateless/03236_squashing_high_memory.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index df296cb6713..479ea87f798 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -6,7 +6,7 @@ DROP TABLE IF EXISTS id_values; DROP TABLE IF EXISTS test_table; CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS - SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; + SELECT arrayJoin(range(20000)) AS id1, arrayJoin(range(1000)) AS id2; SET max_memory_usage = '1G'; From fd3bd785b880644862f1437b99557d6643d78504 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 11 Sep 2024 11:17:06 +0200 Subject: [PATCH 156/162] Update 03236_squashing_high_memory.sql --- tests/queries/0_stateless/03236_squashing_high_memory.sql | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index 479ea87f798..e1c193c3ea6 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -1,12 +1,14 @@ -- Tags: no-fasttest -- reason: test requires too many rows to read +SET max_rows_to_read = '501G'; + DROP TABLE IF EXISTS id_values; DROP TABLE IF EXISTS test_table; CREATE TABLE id_values ENGINE MergeTree ORDER BY id1 AS - SELECT arrayJoin(range(20000)) AS id1, arrayJoin(range(1000)) AS id2; + SELECT arrayJoin(range(500000)) AS id1, arrayJoin(range(1000)) AS id2; SET max_memory_usage = '1G'; From 3921f910f5d0d7e69581a2cc7033213809af48aa Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 11 Sep 2024 10:33:23 +0200 Subject: [PATCH 157/162] Another attempt to address EAGAIN "Resource unavailable" --- docker/test/base/setup_export_logs.sh | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 3df9655701c..7ec1c31f04a 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -187,10 +187,15 @@ function setup_logs_replication ') echo -e "Creating remote destination table ${table}_${hash} with statement:" >&2 + echo "::group::${table}" # there's the only way big "$statement" can be printed without causing EAGAIN error # cat: write error: Resource temporarily unavailable - echo "$statement" | cat + statement_print="${statement}" + if [ "${#statement_print}" -gt 4000 ]; then + statement_print="${statement::1999}\n…\n${statement:${#statement}-1999}" + fi + echo "$statement_print" echo "::endgroup::" echo "$statement" | clickhouse-client --database_replicated_initial_query_timeout_sec=10 \ From af469ffde471ff64e8e0201ceb96ba6e1a0a29c4 Mon Sep 17 00:00:00 2001 From: Yarik Briukhovetskyi <114298166+yariks5s@users.noreply.github.com> Date: Wed, 11 Sep 2024 12:32:36 +0200 Subject: [PATCH 158/162] remove sanitizer builds --- tests/queries/0_stateless/03236_squashing_high_memory.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/03236_squashing_high_memory.sql b/tests/queries/0_stateless/03236_squashing_high_memory.sql index e1c193c3ea6..f6e5dbdef03 100644 --- a/tests/queries/0_stateless/03236_squashing_high_memory.sql +++ b/tests/queries/0_stateless/03236_squashing_high_memory.sql @@ -1,4 +1,4 @@ --- Tags: no-fasttest +-- Tags: no-fasttest, no-asan, no-tsan, no-msan, no-ubsan -- reason: test requires too many rows to read SET max_rows_to_read = '501G'; From c1830bc041a067d1fae1e8971091c453614eaca7 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 11 Sep 2024 13:08:58 +0200 Subject: [PATCH 159/162] Escape the `\n` in statements --- docker/test/base/setup_export_logs.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docker/test/base/setup_export_logs.sh b/docker/test/base/setup_export_logs.sh index 7ec1c31f04a..a39f96867be 100755 --- a/docker/test/base/setup_export_logs.sh +++ b/docker/test/base/setup_export_logs.sh @@ -195,7 +195,7 @@ function setup_logs_replication if [ "${#statement_print}" -gt 4000 ]; then statement_print="${statement::1999}\n…\n${statement:${#statement}-1999}" fi - echo "$statement_print" + echo -e "$statement_print" echo "::endgroup::" echo "$statement" | clickhouse-client --database_replicated_initial_query_timeout_sec=10 \ From 6f9d1631e26bbb0263962c6d50c65e9ee2be561a Mon Sep 17 00:00:00 2001 From: Kseniia Sumarokova <54203879+kssenii@users.noreply.github.com> Date: Wed, 11 Sep 2024 13:11:29 +0200 Subject: [PATCH 160/162] Update aspell-ignore --- utils/check-style/aspell-ignore/en/aspell-dict.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/utils/check-style/aspell-ignore/en/aspell-dict.txt b/utils/check-style/aspell-ignore/en/aspell-dict.txt index 796df6f1042..3467f21c812 100644 --- a/utils/check-style/aspell-ignore/en/aspell-dict.txt +++ b/utils/check-style/aspell-ignore/en/aspell-dict.txt @@ -24,6 +24,7 @@ Aggregatefunction AggregatingMergeTree AggregatorThreads AggregatorThreadsActive +AzureQueue Akka AlertManager Alexey From 9e335abe1735eeed100c4b964bf24220b6086bf9 Mon Sep 17 00:00:00 2001 From: Robert Schulze Date: Wed, 11 Sep 2024 13:05:24 +0000 Subject: [PATCH 161/162] CMake: Add comment about ICU data files --- contrib/icu-cmake/CMakeLists.txt | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/contrib/icu-cmake/CMakeLists.txt b/contrib/icu-cmake/CMakeLists.txt index afaa189701d..5714fef8347 100644 --- a/contrib/icu-cmake/CMakeLists.txt +++ b/contrib/icu-cmake/CMakeLists.txt @@ -481,6 +481,11 @@ if (ARCH_S390X) else() set(ICUDATA_SOURCE_FILE "${ICUDATA_SOURCE_DIR}/icudt75l_dat.S" ) endif() +# ^^ you might be confused how for different little endian platforms (x86, ARM) the same assembly files can be used. +# These files are indeed assembly but they only contain data ('.long' directive), which makes them portable accross CPUs. +# Only the endianness and the character set (ASCII, EBCDIC) makes a difference, also see +# https://unicode-org.github.io/icu/userguide/icu_data/#sharing-icu-data-between-platforms, 'Sharing ICU Data Between Platforms') +# (and as an experiment, try re-generating the data files on x86 vs. ARM, ... you'll get exactly the same files) set(ICUDATA_SOURCES "${ICUDATA_SOURCE_FILE}" From 370ce5ef6347addae66a4fd29b5b0d2b10c8ff76 Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Wed, 11 Sep 2024 22:37:30 +0200 Subject: [PATCH 162/162] Get rid of broken `get_commits().reversed` --- tests/ci/pr_info.py | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/tests/ci/pr_info.py b/tests/ci/pr_info.py index 5c051b093e0..d3c4ffb1e68 100644 --- a/tests/ci/pr_info.py +++ b/tests/ci/pr_info.py @@ -9,6 +9,7 @@ from urllib.parse import quote from unidiff import PatchSet # type: ignore from build_download_helper import get_gh_api +from ci_config import Labels from env_helper import ( GITHUB_EVENT_PATH, GITHUB_REPOSITORY, @@ -16,7 +17,6 @@ from env_helper import ( GITHUB_SERVER_URL, GITHUB_UPSTREAM_REPOSITORY, ) -from ci_config import Labels from get_robot_token import get_best_robot_token from github_helper import GitHub @@ -459,16 +459,18 @@ class PRInfo: sync_repo = gh.get_repo(GITHUB_REPOSITORY) sync_pr = sync_repo.get_pull(self.number) # Find the commit that is in both repos, upstream and cloud - sync_commits = sync_pr.get_commits().reversed - upstream_commits = upstream_pr.get_commits().reversed + # Do not ever use `reversed` here, otherwise the list of commits is not full + sync_commits = list(sync_pr.get_commits()) + upstream_commits = list(upstream_pr.get_commits()) # Github objects are compared by _url attribute. We can't compare them directly and # should compare commits by SHA1 upstream_shas = [c.sha for c in upstream_commits] logging.info("Commits in upstream PR:\n %s", ", ".join(upstream_shas)) sync_shas = [c.sha for c in sync_commits] - logging.info("Commits in sync PR:\n %s", ", ".join(reversed(sync_shas))) + logging.info("Commits in sync PR:\n %s", ", ".join(sync_shas)) - # find latest synced commit + # find latest synced commit, search from the latest + upstream_commits.reverse() last_synced_upstream_commit = None for commit in upstream_commits: if commit.sha in sync_shas: