From 76e9a8edfd430d1062c131569b62f44e796056f1 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 14 Feb 2020 10:12:04 +0300 Subject: [PATCH 001/183] At least something... --- dbms/src/Core/Settings.h | 1 + .../Interpreters/InterpreterSelectQuery.cpp | 46 +++++++++++++++++-- .../src/Interpreters/InterpreterSelectQuery.h | 5 +- dbms/src/Storages/ReadInOrderOptimizer.cpp | 43 ++++++++++++++++- dbms/src/Storages/ReadInOrderOptimizer.h | 25 +++++++++- dbms/src/Storages/SelectQueryInfo.h | 17 +++++++ 6 files changed, 127 insertions(+), 10 deletions(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index b7bd8b3f589..9099c16dbac 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -361,6 +361,7 @@ struct Settings : public SettingsCollection M(SettingBool, enable_debug_queries, false, "Enables debug queries such as AST.", 0) \ M(SettingBool, enable_unaligned_array_join, false, "Allow ARRAY JOIN with multiple arrays that have different sizes. When this settings is enabled, arrays will be resized to the longest one.", 0) \ M(SettingBool, optimize_read_in_order, true, "Enable ORDER BY optimization for reading data in corresponding order in MergeTree tables.", 0) \ + M(SettingBool, optimize_aggregation_in_order, true, "Enable GROUP BY optimization for aggregating data in corresponding order in MergeTree tables.", 0) \ M(SettingBool, low_cardinality_allow_in_native_format, true, "Use LowCardinality type in Native format. Otherwise, convert LowCardinality columns to ordinary for select query, and convert ordinary columns to required LowCardinality for insert query.", 0) \ M(SettingBool, cancel_http_readonly_queries_on_client_close, false, "Cancel HTTP readonly queries when a client closes the connection without waiting for response.", 0) \ M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only by 'mysql' and 'odbc' table functions.", 0) \ diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 72792499d1b..41a290f9392 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -791,13 +791,18 @@ InterpreterSelectQuery::analyzeExpressions( } } - bool has_stream_with_non_joned_rows = (res.before_join && res.before_join->getTableJoinAlgo()->hasStreamWithNonJoinedRows()); + bool has_stream_with_non_joined_rows = (res.before_join && res.before_join->getTableJoinAlgo()->hasStreamWithNonJoinedRows()); res.optimize_read_in_order = context.getSettingsRef().optimize_read_in_order && storage && query.orderBy() && !query_analyzer.hasAggregation() && !query.final() - && !has_stream_with_non_joned_rows; + && !has_stream_with_non_joined_rows; + + /// TODO correct conditions + res.optimize_aggregation_in_order = + context.getSettingsRef().optimize_aggregation_in_order + && storage && query.groupBy(); /// If there is aggregation, we execute expressions in SELECT and ORDER BY on the initiating server, otherwise on the source servers. query_analyzer.appendSelect(chain, only_types || (res.need_aggregate ? !res.second_stage : !res.first_stage)); @@ -929,6 +934,19 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, const Co return order_descr; } +static Names getGroupByDescription(const ASTSelectQuery & query, const Context & /*context*/) +{ + Names group_by_descr; + group_by_descr.reserve(query.groupBy()->children.size()); + + for (const auto & elem : query.groupBy()->children) + { + String name = elem->getColumnName(); + group_by_descr.push_back(name); + } + return group_by_descr; +} + static UInt64 getLimitUIntValue(const ASTPtr & node, const Context & context) { const auto & [field, type] = evaluateConstantExpression(node, context); @@ -1165,7 +1183,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS executeWhere(pipeline, expressions.before_where, expressions.remove_where_filter); if (expressions.need_aggregate) - executeAggregation(pipeline, expressions.before_aggregation, aggregate_overflow_row, aggregate_final); + executeAggregation(pipeline, expressions.before_aggregation, aggregate_overflow_row, aggregate_final, query_info.group_by_info); else { executeExpression(pipeline, expressions.before_order_and_select); @@ -1648,6 +1666,15 @@ void InterpreterSelectQuery::executeFetchColumns( query_info.input_sorting_info = query_info.order_by_optimizer->getInputOrder(storage); } + if (analysis_result.optimize_aggregation_in_order) + { + query_info.group_by_optimizer = std::make_shared( + getGroupByDescription(query, *context), + query_info.syntax_analyzer_result); + + query_info.group_by_info = query_info.group_by_optimizer->getGroupByCommonPrefix(storage); + } + BlockInputStreams streams; Pipes pipes; @@ -1861,7 +1888,7 @@ void InterpreterSelectQuery::executeWhere(QueryPipeline & pipeline, const Expres }); } -void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final) +void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, GroupByInfoPtr group_by_info) { pipeline.transform([&](auto & stream) { @@ -1883,6 +1910,15 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre const Settings & settings = context->getSettingsRef(); + if (group_by_info) { + + /// TODO optimization :) + +// for (const auto & elem : group_by_info->order_key_prefix_descr) { +// std::cerr << elem << " "; +// } +// std::cerr << "\n"; + } /** Two-level aggregation is useful in two cases: * 1. Parallel aggregation is done, and the results should be merged in parallel. * 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. @@ -1927,7 +1963,7 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre } -void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final) +void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, GroupByInfoPtr /*group_by_info*/) { pipeline.addSimpleTransform([&](const Block & header) { diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 56ab2aaa8f5..d781c841de8 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -163,6 +163,7 @@ private: bool remove_where_filter = false; bool optimize_read_in_order = false; + bool optimize_aggregation_in_order = false; ExpressionActionsPtr before_join; /// including JOIN ExpressionActionsPtr before_where; @@ -217,7 +218,7 @@ private: QueryPipeline & save_context_and_storage); void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter); - void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final); + void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, GroupByInfoPtr group_by_info); void executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final); void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final); void executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression); @@ -236,7 +237,7 @@ private: void executeMergeSorted(Pipeline & pipeline, const SortDescription & sort_description, UInt64 limit); void executeWhere(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_fiter); - void executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final); + void executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, GroupByInfoPtr group_by_info); void executeMergeAggregated(QueryPipeline & pipeline, bool overflow_row, bool final); void executeTotalsAndHaving(QueryPipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final); void executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); diff --git a/dbms/src/Storages/ReadInOrderOptimizer.cpp b/dbms/src/Storages/ReadInOrderOptimizer.cpp index 667ce095932..b841560d8e4 100644 --- a/dbms/src/Storages/ReadInOrderOptimizer.cpp +++ b/dbms/src/Storages/ReadInOrderOptimizer.cpp @@ -31,7 +31,7 @@ ReadInOrderOptimizer::ReadInOrderOptimizer( InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & storage) const { - const MergeTreeData * merge_tree = dynamic_cast(storage.get()); + const auto * merge_tree = dynamic_cast(storage.get()); if (!merge_tree || !merge_tree->hasSortingKey()) return {}; @@ -110,4 +110,45 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora return std::make_shared(std::move(order_key_prefix_descr), read_direction); } + +AggregateInOrderOptimizer::AggregateInOrderOptimizer( + const Names & group_by_description_, + const SyntaxAnalyzerResultPtr & syntax_result) + : group_by_description(group_by_description_) +{ + /// Not sure yet but let it be + for (const auto & elem : syntax_result->array_join_result_to_source) + forbidden_columns.insert(elem.first); +} + +GroupByInfoPtr AggregateInOrderOptimizer::getGroupByCommonPrefix(const StoragePtr &storage) const +{ + const auto * merge_tree = dynamic_cast(storage.get()); + if (!merge_tree || !merge_tree->hasSortingKey()) + return {}; + + Names group_by_common_prefix; + const auto & sorting_key_columns = merge_tree->getSortingKeyColumns(); + size_t prefix_size = std::min(group_by_description.size(), sorting_key_columns.size()); + + for (size_t i = 0; i < prefix_size; ++i) + { + if (forbidden_columns.count(group_by_description[i])) + break; + + if (group_by_description[i] == sorting_key_columns[i]) { + group_by_common_prefix.push_back(group_by_description[i]); + } + else { + /// TODO injective functions + break; + } + } + + if (group_by_common_prefix.empty()) + return {}; + + return std::make_shared(std::move(group_by_common_prefix)); +} + } diff --git a/dbms/src/Storages/ReadInOrderOptimizer.h b/dbms/src/Storages/ReadInOrderOptimizer.h index 8416d23a912..f19f2048868 100644 --- a/dbms/src/Storages/ReadInOrderOptimizer.h +++ b/dbms/src/Storages/ReadInOrderOptimizer.h @@ -23,10 +23,31 @@ public: InputSortingInfoPtr getInputOrder(const StoragePtr & storage) const; private: - /// Actions for every element of order expression to analyze functions for monotonicicy + /// Actions for every element of order expression to analyze functions for monotonicity ManyExpressionActions elements_actions; NameSet forbidden_columns; SortDescription required_sort_description; }; -} + + /** Helper class, that can analyze MergeTree order key + * and required group by description to get their + * common prefix, which is needed for + * performing reading in order of PK. + */ + class AggregateInOrderOptimizer + { + public: + AggregateInOrderOptimizer( + const Names & group_by_description, + const SyntaxAnalyzerResultPtr & syntax_result); + + GroupByInfoPtr getGroupByCommonPrefix(const StoragePtr & storage) const; + + private: + /// Actions for every element of order expression to analyze functions for monotonicity + NameSet forbidden_columns; + Names group_by_description; + }; + + } \ No newline at end of file diff --git a/dbms/src/Storages/SelectQueryInfo.h b/dbms/src/Storages/SelectQueryInfo.h index 84cf3a32aa1..39e5bb07453 100644 --- a/dbms/src/Storages/SelectQueryInfo.h +++ b/dbms/src/Storages/SelectQueryInfo.h @@ -2,6 +2,7 @@ #include #include +#include #include namespace DB @@ -51,9 +52,18 @@ struct InputSortingInfo bool operator !=(const InputSortingInfo & other) const { return !(*this == other); } }; +struct GroupByInfo +{ + Names order_key_prefix_descr; + + GroupByInfo(const Names & order_key_prefix_descr_) + : order_key_prefix_descr(order_key_prefix_descr_) {} +}; + using PrewhereInfoPtr = std::shared_ptr; using FilterInfoPtr = std::shared_ptr; using InputSortingInfoPtr = std::shared_ptr; +using GroupByInfoPtr = std::shared_ptr; struct SyntaxAnalyzerResult; using SyntaxAnalyzerResultPtr = std::shared_ptr; @@ -61,6 +71,9 @@ using SyntaxAnalyzerResultPtr = std::shared_ptr; class ReadInOrderOptimizer; using ReadInOrderOptimizerPtr = std::shared_ptr; +class AggregateInOrderOptimizer; +using AggregateInOrderOptimizerPtr = std::shared_ptr; + /** Query along with some additional data, * that can be used during query processing * inside storage engines. @@ -74,9 +87,13 @@ struct SelectQueryInfo PrewhereInfoPtr prewhere_info; ReadInOrderOptimizerPtr order_by_optimizer; + + AggregateInOrderOptimizerPtr group_by_optimizer; + /// We can modify it while reading from storage mutable InputSortingInfoPtr input_sorting_info; + GroupByInfoPtr group_by_info; /// Prepared sets are used for indices by storage engine. /// Example: x IN (1, 2, 3) PreparedSets sets; From 4bcf10b5a5ccf842ca5df2592e319a3385407864 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 14 Feb 2020 16:31:01 +0300 Subject: [PATCH 002/183] style --- .../Interpreters/InterpreterSelectQuery.cpp | 8 ++--- dbms/src/Storages/ReadInOrderOptimizer.cpp | 6 ++-- dbms/src/Storages/ReadInOrderOptimizer.h | 36 +++++++++---------- 3 files changed, 25 insertions(+), 25 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 41a290f9392..7f6a01067f3 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1910,15 +1910,15 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre const Settings & settings = context->getSettingsRef(); - if (group_by_info) { - + if (group_by_info) + { /// TODO optimization :) -// for (const auto & elem : group_by_info->order_key_prefix_descr) { +// for (const auto & elem : group_by_info->order_key_prefix_descr) // std::cerr << elem << " "; -// } // std::cerr << "\n"; } + /** Two-level aggregation is useful in two cases: * 1. Parallel aggregation is done, and the results should be merged in parallel. * 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. diff --git a/dbms/src/Storages/ReadInOrderOptimizer.cpp b/dbms/src/Storages/ReadInOrderOptimizer.cpp index b841560d8e4..1f261deda0f 100644 --- a/dbms/src/Storages/ReadInOrderOptimizer.cpp +++ b/dbms/src/Storages/ReadInOrderOptimizer.cpp @@ -136,10 +136,10 @@ GroupByInfoPtr AggregateInOrderOptimizer::getGroupByCommonPrefix(const StoragePt if (forbidden_columns.count(group_by_description[i])) break; - if (group_by_description[i] == sorting_key_columns[i]) { + if (group_by_description[i] == sorting_key_columns[i]) group_by_common_prefix.push_back(group_by_description[i]); - } - else { + else + { /// TODO injective functions break; } diff --git a/dbms/src/Storages/ReadInOrderOptimizer.h b/dbms/src/Storages/ReadInOrderOptimizer.h index f19f2048868..024c0abbfb0 100644 --- a/dbms/src/Storages/ReadInOrderOptimizer.h +++ b/dbms/src/Storages/ReadInOrderOptimizer.h @@ -30,24 +30,24 @@ private: }; - /** Helper class, that can analyze MergeTree order key - * and required group by description to get their - * common prefix, which is needed for - * performing reading in order of PK. - */ - class AggregateInOrderOptimizer - { - public: - AggregateInOrderOptimizer( - const Names & group_by_description, - const SyntaxAnalyzerResultPtr & syntax_result); +/** Helper class, that can analyze MergeTree order key +* and required group by description to get their +* common prefix, which is needed for +* performing reading in order of PK. +*/ +class AggregateInOrderOptimizer +{ +public: + AggregateInOrderOptimizer( + const Names & group_by_description, + const SyntaxAnalyzerResultPtr & syntax_result); - GroupByInfoPtr getGroupByCommonPrefix(const StoragePtr & storage) const; + GroupByInfoPtr getGroupByCommonPrefix(const StoragePtr & storage) const; - private: - /// Actions for every element of order expression to analyze functions for monotonicity - NameSet forbidden_columns; - Names group_by_description; - }; +private: + /// Actions for every element of order expression to analyze functions for monotonicity + NameSet forbidden_columns; + Names group_by_description; +}; - } \ No newline at end of file +} From 6cee50ab9780201e024f1c0844f6ab59af06048d Mon Sep 17 00:00:00 2001 From: Dmitry Date: Sun, 16 Feb 2020 22:46:45 +0300 Subject: [PATCH 003/183] removed extra structs --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 24 +++++++++-- dbms/src/Interpreters/ExpressionAnalyzer.h | 4 +- .../Interpreters/InterpreterSelectQuery.cpp | 28 +++++++------ .../src/Interpreters/InterpreterSelectQuery.h | 4 +- dbms/src/Storages/ReadInOrderOptimizer.cpp | 41 ------------------- dbms/src/Storages/ReadInOrderOptimizer.h | 21 ---------- dbms/src/Storages/SelectQueryInfo.h | 16 +------- 7 files changed, 42 insertions(+), 96 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index f131afb86c6..10ffda07555 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -693,7 +693,8 @@ bool SelectQueryExpressionAnalyzer::appendWhere(ExpressionActionsChain & chain, return true; } -bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only_types) +bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain, bool only_types, bool optimize_aggregation_in_order, + ManyExpressionActions & group_by_elements_actions) { const auto * select_query = getAggregatingQuery(); @@ -710,6 +711,16 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain getRootActions(asts[i], only_types, step.actions); } + if (optimize_aggregation_in_order) + { + auto all_columns = sourceWithJoinedColumns(); + for (auto & child : asts) + { + group_by_elements_actions.emplace_back(std::make_shared(all_columns, context)); + getRootActions(child, only_types, group_by_elements_actions.back()); + } + } + return true; } @@ -1051,7 +1062,12 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( if (need_aggregate) { - query_analyzer.appendGroupBy(chain, only_types || !first_stage); + /// TODO correct conditions + optimize_aggregation_in_order = + context.getSettingsRef().optimize_aggregation_in_order + && storage && query.groupBy(); + + query_analyzer.appendGroupBy(chain, only_types || !first_stage, optimize_aggregation_in_order, group_by_elements_actions); query_analyzer.appendAggregateFunctionsArguments(chain, only_types || !first_stage); before_aggregation = chain.getLastActions(); @@ -1064,13 +1080,13 @@ ExpressionAnalysisResult::ExpressionAnalysisResult( } } - bool has_stream_with_non_joned_rows = (before_join && before_join->getTableJoinAlgo()->hasStreamWithNonJoinedRows()); + bool has_stream_with_non_joined_rows = (before_join && before_join->getTableJoinAlgo()->hasStreamWithNonJoinedRows()); optimize_read_in_order = settings.optimize_read_in_order && storage && query.orderBy() && !query_analyzer.hasAggregation() && !query.final() - && !has_stream_with_non_joned_rows; + && !has_stream_with_non_joined_rows; /// If there is aggregation, we execute expressions in SELECT and ORDER BY on the initiating server, otherwise on the source servers. query_analyzer.appendSelect(chain, only_types || (need_aggregate ? !second_stage : !first_stage)); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index f262132c002..da38694a496 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -173,6 +173,7 @@ struct ExpressionAnalysisResult bool remove_where_filter = false; bool optimize_read_in_order = false; + bool optimize_aggregation_in_order = false; ExpressionActionsPtr before_join; /// including JOIN ExpressionActionsPtr before_where; @@ -194,6 +195,7 @@ struct ExpressionAnalysisResult ConstantFilterDescription where_constant_filter_description; /// Actions by every element of ORDER BY ManyExpressionActions order_by_elements_actions; + ManyExpressionActions group_by_elements_actions; ExpressionAnalysisResult() = default; @@ -301,7 +303,7 @@ private: /// Columns in `additional_required_columns` will not be removed (they can be used for e.g. sampling or FINAL modifier). bool appendPrewhere(ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns); bool appendWhere(ExpressionActionsChain & chain, bool only_types); - bool appendGroupBy(ExpressionActionsChain & chain, bool only_types); + bool appendGroupBy(ExpressionActionsChain & chain, bool only_types, bool optimize_aggregation_in_order, ManyExpressionActions &); void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types); /// After aggregation: diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 5148592da84..4d585109ed1 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -679,17 +679,18 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, const Co return order_descr; } -static Names getGroupByDescription(const ASTSelectQuery & query, const Context & /*context*/) +static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query, const Context & /*context*/) { - Names group_by_descr; - group_by_descr.reserve(query.groupBy()->children.size()); + SortDescription order_descr; + order_descr.reserve(query.groupBy()->children.size()); for (const auto & elem : query.groupBy()->children) { String name = elem->getColumnName(); - group_by_descr.push_back(name); + order_descr.emplace_back(name, 1, 1); } - return group_by_descr; + + return order_descr; } static UInt64 getLimitUIntValue(const ASTPtr & node, const Context & context) @@ -1413,11 +1414,12 @@ void InterpreterSelectQuery::executeFetchColumns( if (analysis_result.optimize_aggregation_in_order) { - query_info.group_by_optimizer = std::make_shared( - getGroupByDescription(query, *context), - query_info.syntax_analyzer_result); + query_info.group_by_optimizer = std::make_shared( + analysis_result.group_by_elements_actions, + getSortDescriptionFromGroupBy(query, *context), + query_info.syntax_analyzer_result); - query_info.group_by_info = query_info.group_by_optimizer->getGroupByCommonPrefix(storage); + query_info.group_by_info = query_info.group_by_optimizer->getInputOrder(storage); } @@ -1633,7 +1635,7 @@ void InterpreterSelectQuery::executeWhere(QueryPipeline & pipeline, const Expres }); } -void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, GroupByInfoPtr group_by_info) +void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr group_by_info) { pipeline.transform([&](auto & stream) { @@ -1656,9 +1658,9 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre if (group_by_info) { /// TODO optimization :) - +// std::cerr << "\n"; // for (const auto & elem : group_by_info->order_key_prefix_descr) -// std::cerr << elem << " "; +// std::cerr << elem.column_name << " "; // std::cerr << "\n"; } @@ -1706,7 +1708,7 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre } -void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, GroupByInfoPtr /*group_by_info*/) +void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr /*group_by_info*/) { pipeline.addSimpleTransform([&](const Block & header) { diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 8fc7df0039b..3e72fda362a 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -168,7 +168,7 @@ private: QueryPipeline & save_context_and_storage); void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter); - void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, GroupByInfoPtr group_by_info); + void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr sorting_info); void executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final); void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final); void executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression); @@ -187,7 +187,7 @@ private: void executeMergeSorted(Pipeline & pipeline, const SortDescription & sort_description, UInt64 limit); void executeWhere(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_fiter); - void executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, GroupByInfoPtr group_by_info); + void executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr sorting_info); void executeMergeAggregated(QueryPipeline & pipeline, bool overflow_row, bool final); void executeTotalsAndHaving(QueryPipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final); void executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); diff --git a/dbms/src/Storages/ReadInOrderOptimizer.cpp b/dbms/src/Storages/ReadInOrderOptimizer.cpp index 1f261deda0f..18652875f32 100644 --- a/dbms/src/Storages/ReadInOrderOptimizer.cpp +++ b/dbms/src/Storages/ReadInOrderOptimizer.cpp @@ -110,45 +110,4 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora return std::make_shared(std::move(order_key_prefix_descr), read_direction); } - -AggregateInOrderOptimizer::AggregateInOrderOptimizer( - const Names & group_by_description_, - const SyntaxAnalyzerResultPtr & syntax_result) - : group_by_description(group_by_description_) -{ - /// Not sure yet but let it be - for (const auto & elem : syntax_result->array_join_result_to_source) - forbidden_columns.insert(elem.first); -} - -GroupByInfoPtr AggregateInOrderOptimizer::getGroupByCommonPrefix(const StoragePtr &storage) const -{ - const auto * merge_tree = dynamic_cast(storage.get()); - if (!merge_tree || !merge_tree->hasSortingKey()) - return {}; - - Names group_by_common_prefix; - const auto & sorting_key_columns = merge_tree->getSortingKeyColumns(); - size_t prefix_size = std::min(group_by_description.size(), sorting_key_columns.size()); - - for (size_t i = 0; i < prefix_size; ++i) - { - if (forbidden_columns.count(group_by_description[i])) - break; - - if (group_by_description[i] == sorting_key_columns[i]) - group_by_common_prefix.push_back(group_by_description[i]); - else - { - /// TODO injective functions - break; - } - } - - if (group_by_common_prefix.empty()) - return {}; - - return std::make_shared(std::move(group_by_common_prefix)); -} - } diff --git a/dbms/src/Storages/ReadInOrderOptimizer.h b/dbms/src/Storages/ReadInOrderOptimizer.h index 024c0abbfb0..4f69831c49f 100644 --- a/dbms/src/Storages/ReadInOrderOptimizer.h +++ b/dbms/src/Storages/ReadInOrderOptimizer.h @@ -29,25 +29,4 @@ private: SortDescription required_sort_description; }; - -/** Helper class, that can analyze MergeTree order key -* and required group by description to get their -* common prefix, which is needed for -* performing reading in order of PK. -*/ -class AggregateInOrderOptimizer -{ -public: - AggregateInOrderOptimizer( - const Names & group_by_description, - const SyntaxAnalyzerResultPtr & syntax_result); - - GroupByInfoPtr getGroupByCommonPrefix(const StoragePtr & storage) const; - -private: - /// Actions for every element of order expression to analyze functions for monotonicity - NameSet forbidden_columns; - Names group_by_description; -}; - } diff --git a/dbms/src/Storages/SelectQueryInfo.h b/dbms/src/Storages/SelectQueryInfo.h index 39e5bb07453..d335398ada5 100644 --- a/dbms/src/Storages/SelectQueryInfo.h +++ b/dbms/src/Storages/SelectQueryInfo.h @@ -52,18 +52,9 @@ struct InputSortingInfo bool operator !=(const InputSortingInfo & other) const { return !(*this == other); } }; -struct GroupByInfo -{ - Names order_key_prefix_descr; - - GroupByInfo(const Names & order_key_prefix_descr_) - : order_key_prefix_descr(order_key_prefix_descr_) {} -}; - using PrewhereInfoPtr = std::shared_ptr; using FilterInfoPtr = std::shared_ptr; using InputSortingInfoPtr = std::shared_ptr; -using GroupByInfoPtr = std::shared_ptr; struct SyntaxAnalyzerResult; using SyntaxAnalyzerResultPtr = std::shared_ptr; @@ -71,8 +62,6 @@ using SyntaxAnalyzerResultPtr = std::shared_ptr; class ReadInOrderOptimizer; using ReadInOrderOptimizerPtr = std::shared_ptr; -class AggregateInOrderOptimizer; -using AggregateInOrderOptimizerPtr = std::shared_ptr; /** Query along with some additional data, * that can be used during query processing @@ -87,13 +76,12 @@ struct SelectQueryInfo PrewhereInfoPtr prewhere_info; ReadInOrderOptimizerPtr order_by_optimizer; - - AggregateInOrderOptimizerPtr group_by_optimizer; + ReadInOrderOptimizerPtr group_by_optimizer; /// We can modify it while reading from storage mutable InputSortingInfoPtr input_sorting_info; + mutable InputSortingInfoPtr group_by_info; - GroupByInfoPtr group_by_info; /// Prepared sets are used for indices by storage engine. /// Example: x IN (1, 2, 3) PreparedSets sets; From bbb6ed2307e1f451302e6c7d4625411758f41f51 Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Tue, 4 Feb 2020 23:47:30 +0300 Subject: [PATCH 004/183] Software events for perf_events metrics --- dbms/src/Common/ProfileEvents.cpp | 13 ++ dbms/src/Common/ThreadProfileEvents.cpp | 171 ++++++++++++++++++++++++ dbms/src/Common/ThreadProfileEvents.h | 31 ++++- dbms/src/Common/ThreadStatus.cpp | 1 + dbms/src/Common/ThreadStatus.h | 2 + 5 files changed, 217 insertions(+), 1 deletion(-) create mode 100644 dbms/src/Common/ThreadProfileEvents.cpp diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index 2b267f33f9d..22e4c7460fe 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -177,6 +177,19 @@ M(OSWriteBytes, "Number of bytes written to disks or block devices. Doesn't include bytes that are in page cache dirty pages. May not include data that was written by OS asynchronously.") \ M(OSReadChars, "Number of bytes read from filesystem, including page cache.") \ M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \ + \ + M(PERF_COUNT_SW_CPU_CLOCK, "") \ + M(PERF_COUNT_SW_TASK_CLOCK, "") \ + M(PERF_COUNT_SW_PAGE_FAULTS, "") \ + M(PERF_COUNT_SW_CONTEXT_SWITCHES, "") \ + M(PERF_COUNT_SW_CPU_MIGRATIONS, "") \ + M(PERF_COUNT_SW_PAGE_FAULTS_MIN, "") \ + M(PERF_COUNT_SW_PAGE_FAULTS_MAJ, "") \ + M(PERF_COUNT_SW_ALIGNMENT_FAULTS, "") \ + M(PERF_COUNT_SW_EMULATION_FAULTS, "") \ + M(PERF_COUNT_SW_DUMMY, "") \ + M(PERF_COUNT_SW_BPF_OUTPUT, "") \ + \ M(CreatedHTTPConnections, "Total amount of created HTTP connections (closed or opened).") \ \ M(CannotWriteToWriteBufferDiscard, "Number of stack traces dropped by query profiler or signal handler because pipe is full or cannot write to pipe.") \ diff --git a/dbms/src/Common/ThreadProfileEvents.cpp b/dbms/src/Common/ThreadProfileEvents.cpp new file mode 100644 index 00000000000..b35b4531dc5 --- /dev/null +++ b/dbms/src/Common/ThreadProfileEvents.cpp @@ -0,0 +1,171 @@ +#include + +#if defined(__linux__) +#include +#include +#include +#include +#endif + +namespace DB { + +#if defined(__linux__) + + static constexpr int perf_event_configs[] = { + perf_sw_ids::PERF_COUNT_SW_CPU_CLOCK, + perf_sw_ids::PERF_COUNT_SW_TASK_CLOCK, + perf_sw_ids::PERF_COUNT_SW_PAGE_FAULTS, + perf_sw_ids::PERF_COUNT_SW_CONTEXT_SWITCHES, + perf_sw_ids::PERF_COUNT_SW_CPU_MIGRATIONS, + perf_sw_ids::PERF_COUNT_SW_PAGE_FAULTS_MIN, + perf_sw_ids::PERF_COUNT_SW_PAGE_FAULTS_MAJ, + perf_sw_ids::PERF_COUNT_SW_ALIGNMENT_FAULTS, + perf_sw_ids::PERF_COUNT_SW_EMULATION_FAULTS, + perf_sw_ids::PERF_COUNT_SW_DUMMY, + perf_sw_ids::PERF_COUNT_SW_BPF_OUTPUT + }; + + static const std::string perf_event_names[] = { + "PERF_COUNT_SW_CPU_CLOCK", + "PERF_COUNT_SW_TASK_CLOCK", + "PERF_COUNT_SW_PAGE_FAULTS", + "PERF_COUNT_SW_CONTEXT_SWITCHES", + "PERF_COUNT_SW_CPU_MIGRATIONS", + "PERF_COUNT_SW_PAGE_FAULTS_MIN", + "PERF_COUNT_SW_PAGE_FAULTS_MAJ", + "PERF_COUNT_SW_ALIGNMENT_FAULTS", + "PERF_COUNT_SW_EMULATION_FAULTS", + "PERF_COUNT_SW_DUMMY", + "PERF_COUNT_SW_BPF_OUTPUT" + }; + + static const ProfileEvents::Event perf_events[] = { + ProfileEvents::PERF_COUNT_SW_CPU_CLOCK, + ProfileEvents::PERF_COUNT_SW_TASK_CLOCK, + ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS, + ProfileEvents::PERF_COUNT_SW_CONTEXT_SWITCHES, + ProfileEvents::PERF_COUNT_SW_CPU_MIGRATIONS, + ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MIN, + ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MAJ, + ProfileEvents::PERF_COUNT_SW_ALIGNMENT_FAULTS, + ProfileEvents::PERF_COUNT_SW_EMULATION_FAULTS, + ProfileEvents::PERF_COUNT_SW_DUMMY, + ProfileEvents::PERF_COUNT_SW_BPF_OUTPUT + }; + + constexpr size_t NUMBER_OF_EVENTS = std::size(perf_event_configs); + + static_assert(std::size(perf_event_names) == NUMBER_OF_EVENTS); + static_assert(std::size(perf_events) == NUMBER_OF_EVENTS); + + static int events_descriptors[NUMBER_OF_EVENTS]; + static bool perf_events_opened = false; + + static int openPerfEvent(perf_event_attr *hw_event, pid_t pid, int cpu, int group_fd, unsigned long flags) { + return static_cast(syscall(SYS_perf_event_open, hw_event, pid, cpu, group_fd, flags)); + } + + static bool getPerfEventParanoid(int &result) { + // the longest possible variant: "-1\0" + constexpr int MAX_LENGTH = 3; + FILE *fp; + char str[MAX_LENGTH]; + + fp = fopen("/proc/sys/kernel/perf_event_paranoid", "r"); + if (fp == nullptr) + return false; + + char *res = fgets(str, MAX_LENGTH, fp); + fclose(fp); + + if (res == nullptr) + return false; + + str[MAX_LENGTH - 1] = '\0'; + // todo: change to `strtol` + result = atoi(str); + return true; + } + + static void perf_event_start(int perf_event_paranoid, int perf_event_type, int perf_event_config, int &event_file_descriptor) { + perf_event_attr pe = perf_event_attr(); + pe.type = perf_event_type; + pe.size = sizeof(struct perf_event_attr); + pe.config = perf_event_config; + pe.disabled = 1; + // can record kernel only when `perf_event_paranoid` <= 1 + pe.exclude_kernel = perf_event_paranoid >= 2; + pe.exclude_hv = 1; + + event_file_descriptor = openPerfEvent(&pe, 0, -1, -1, 0); + } + +// static void disable_events() { +// if (!perf_events_opened) +// return; +// +// for (int & fd : events_descriptors) { +// if (fd == -1) +// continue; +// +// ioctl(fd, PERF_EVENT_IOC_DISABLE, 0); +// close(fd); +// fd = -1; +// } +// +// perf_events_opened = false; +// } + + void PerfEventsCounters::updateProfileEvents(ProfileEvents::Counters &profile_events) { + if (perf_events_opened) { + for (size_t i = 0; i < NUMBER_OF_EVENTS; ++i) { + int fd = events_descriptors[i]; + if (fd == -1) + continue; + + long long count; + read(fd, &count, sizeof(count)); + + profile_events.increment(perf_events[i], static_cast(count)); +// printf("%s: %lld\n", perf_event_names[i].c_str(), count); + + ioctl(fd, PERF_EVENT_IOC_RESET, 0); + } + + return; + } + + int perf_event_paranoid = 0; + bool is_pref_available = getPerfEventParanoid(perf_event_paranoid); +// printf("is_perf_available: %s, perf_event_paranoid: %d\n", is_pref_available ? "true" : "false", perf_event_paranoid); + + if (!is_pref_available) + return; + + for (size_t i = 0; i < NUMBER_OF_EVENTS; ++i) + { + int eventConfig = perf_event_configs[i]; + std::string eventName = perf_event_names[i]; + + perf_event_start(perf_event_paranoid, perf_type_id::PERF_TYPE_SOFTWARE, eventConfig, events_descriptors[i]); + } + + for (size_t i = 0; i < NUMBER_OF_EVENTS; ++i) + { + int fd = events_descriptors[i]; + if (fd == -1) + fprintf(stderr, "Event config %d is unsupported\n", perf_event_configs[i]); + else + ioctl(fd, PERF_EVENT_IOC_ENABLE, 0); + } + + perf_events_opened = true; + } + +#else + + void PerfEventsCounters::updateProfileEvents(ProfileEvents::Counters &) {} + +#endif + +} \ No newline at end of file diff --git a/dbms/src/Common/ThreadProfileEvents.h b/dbms/src/Common/ThreadProfileEvents.h index e17a4066fe6..85236f5a176 100644 --- a/dbms/src/Common/ThreadProfileEvents.h +++ b/dbms/src/Common/ThreadProfileEvents.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include @@ -34,6 +34,18 @@ namespace ProfileEvents extern const Event OSWriteChars; extern const Event OSReadBytes; extern const Event OSWriteBytes; + + extern const Event PERF_COUNT_SW_CPU_CLOCK; + extern const Event PERF_COUNT_SW_TASK_CLOCK; + extern const Event PERF_COUNT_SW_PAGE_FAULTS; + extern const Event PERF_COUNT_SW_CONTEXT_SWITCHES; + extern const Event PERF_COUNT_SW_CPU_MIGRATIONS; + extern const Event PERF_COUNT_SW_PAGE_FAULTS_MIN; + extern const Event PERF_COUNT_SW_PAGE_FAULTS_MAJ; + extern const Event PERF_COUNT_SW_ALIGNMENT_FAULTS; + extern const Event PERF_COUNT_SW_EMULATION_FAULTS; + extern const Event PERF_COUNT_SW_DUMMY; + extern const Event PERF_COUNT_SW_BPF_OUTPUT; #endif } @@ -117,6 +129,23 @@ struct RUsageCounters } }; +struct PerfEventsCounters +{ + // cat /proc/sys/kernel/perf_event_paranoid - if perf_event_paranoid is set to 3, all calls to `perf_event_open` are rejected (even for the current process) + // https://lwn.net/Articles/696234/ + // -1: Allow use of (almost) all events by all users + // >=0: Disallow raw tracepoint access by users without CAP_IOC_LOCK + // >=1: Disallow CPU event access by users without CAP_SYS_ADMIN + // >=2: Disallow kernel profiling by users without CAP_SYS_ADMIN + // >=3: Disallow all event access by users without CAP_SYS_ADMIN + + // https://lwn.net/Articles/696216/ + // It adds a another value that can be set for the sysctl parameter (i.e. kernel.perf_event_paranoid=3) + // that restricts perf_event_open() to processes with the CAP_SYS_ADMIN capability + // todo: check whether perf_event_open() is available with CAP_SYS_ADMIN + + static void updateProfileEvents(ProfileEvents::Counters & profile_events); +}; #if defined(__linux__) diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index 9bed96552ea..7ceb848a7c8 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -105,6 +105,7 @@ void ThreadStatus::updatePerformanceCounters() try { RUsageCounters::updateProfileEvents(*last_rusage, performance_counters); + PerfEventsCounters::updateProfileEvents(performance_counters); if (taskstats_getter) TasksStatsCounters::updateProfileEvents(*last_taskstats, performance_counters); } diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index 58af6d4efff..fe45431e4d8 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -33,6 +33,7 @@ class QueryProfilerCpu; class QueryThreadLog; struct TasksStatsCounters; struct RUsageCounters; +struct PerfEventsCounters; class TaskStatsInfoGetter; class InternalTextLogsQueue; using InternalTextLogsQueuePtr = std::shared_ptr; @@ -195,6 +196,7 @@ protected: /// Use ptr not to add extra dependencies in the header std::unique_ptr last_rusage; + std::unique_ptr last_perf_events; std::unique_ptr last_taskstats; /// Set to non-nullptr only if we have enough capabilities. From ddbb061c369f7d486f40d58703315ae192668b88 Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Tue, 4 Feb 2020 23:50:05 +0300 Subject: [PATCH 005/183] Removed unused code --- dbms/src/Common/ThreadStatus.h | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index fe45431e4d8..58af6d4efff 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -33,7 +33,6 @@ class QueryProfilerCpu; class QueryThreadLog; struct TasksStatsCounters; struct RUsageCounters; -struct PerfEventsCounters; class TaskStatsInfoGetter; class InternalTextLogsQueue; using InternalTextLogsQueuePtr = std::shared_ptr; @@ -196,7 +195,6 @@ protected: /// Use ptr not to add extra dependencies in the header std::unique_ptr last_rusage; - std::unique_ptr last_perf_events; std::unique_ptr last_taskstats; /// Set to non-nullptr only if we have enough capabilities. From f87b062f9f07d76a5b5c0df99b227b54df3c95a0 Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Wed, 19 Feb 2020 19:35:01 +0300 Subject: [PATCH 006/183] Correct multithread metrics recording --- dbms/src/Common/ProfileEvents.cpp | 19 ++- dbms/src/Common/ThreadProfileEvents.cpp | 152 +++++++++++----------- dbms/src/Common/ThreadProfileEvents.h | 18 ++- dbms/src/Common/ThreadStatus.cpp | 11 +- dbms/src/Common/ThreadStatus.h | 2 + dbms/src/Interpreters/ThreadStatusExt.cpp | 9 ++ 6 files changed, 118 insertions(+), 93 deletions(-) diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index 22e4c7460fe..cda51739c93 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -178,17 +178,14 @@ M(OSReadChars, "Number of bytes read from filesystem, including page cache.") \ M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \ \ - M(PERF_COUNT_SW_CPU_CLOCK, "") \ - M(PERF_COUNT_SW_TASK_CLOCK, "") \ - M(PERF_COUNT_SW_PAGE_FAULTS, "") \ - M(PERF_COUNT_SW_CONTEXT_SWITCHES, "") \ - M(PERF_COUNT_SW_CPU_MIGRATIONS, "") \ - M(PERF_COUNT_SW_PAGE_FAULTS_MIN, "") \ - M(PERF_COUNT_SW_PAGE_FAULTS_MAJ, "") \ - M(PERF_COUNT_SW_ALIGNMENT_FAULTS, "") \ - M(PERF_COUNT_SW_EMULATION_FAULTS, "") \ - M(PERF_COUNT_SW_DUMMY, "") \ - M(PERF_COUNT_SW_BPF_OUTPUT, "") \ + M(PERF_COUNT_SW_TASK_CLOCK, "A clock count specific to the task that is running") \ + M(PERF_COUNT_SW_PAGE_FAULTS, "Number of page faults") \ + M(PERF_COUNT_SW_CONTEXT_SWITCHES, "Number of context switches") \ + M(PERF_COUNT_SW_CPU_MIGRATIONS, "Number of times the process has migrated to a new CPU") \ + M(PERF_COUNT_SW_PAGE_FAULTS_MIN, "Number of minor page faults. These did not require disk I/O to handle") \ + M(PERF_COUNT_SW_PAGE_FAULTS_MAJ, "Number of major page faults. These required disk I/O to handle") \ + M(PERF_COUNT_SW_ALIGNMENT_FAULTS, "Number of alignment faults. These happen when unaligned memory accesses happen; the kernel can handle these but it reduces performance. This happens only on some architectures (never on x86).") \ + M(PERF_COUNT_SW_EMULATION_FAULTS, "Number of emulation faults. The kernel sometimes traps on unimplemented instructions and emulates them for user space. This can negatively impact performance.") \ \ M(CreatedHTTPConnections, "Total amount of created HTTP connections (closed or opened).") \ \ diff --git a/dbms/src/Common/ThreadProfileEvents.cpp b/dbms/src/Common/ThreadProfileEvents.cpp index b35b4531dc5..959bda87b06 100644 --- a/dbms/src/Common/ThreadProfileEvents.cpp +++ b/dbms/src/Common/ThreadProfileEvents.cpp @@ -5,42 +5,50 @@ #include #include #include +#include #endif namespace DB { #if defined(__linux__) - static constexpr int perf_event_configs[] = { - perf_sw_ids::PERF_COUNT_SW_CPU_CLOCK, + // todo: think about event counters' overflow + // todo: ask about the usual error reporting (whether stderr is an accepted way) + + // descriptions' source: http://man7.org/linux/man-pages/man2/perf_event_open.2.html + const int PerfEventsCounters::perf_event_configs[] = { + // This reports the CPU clock, a high-resolution per-CPU timer. +// perf_sw_ids::PERF_COUNT_SW_CPU_CLOCK, + // This reports a clock count specific to the task that is running. perf_sw_ids::PERF_COUNT_SW_TASK_CLOCK, + // This reports the number of page faults. perf_sw_ids::PERF_COUNT_SW_PAGE_FAULTS, + // This counts context switches. + // Until Linux 2.6.34, these were all reported as user-space events, + // after that they are reported as happening in the kernel perf_sw_ids::PERF_COUNT_SW_CONTEXT_SWITCHES, + // This reports the number of times the process has migrated to a new CPU. perf_sw_ids::PERF_COUNT_SW_CPU_MIGRATIONS, + // This counts the number of minor page faults. These did not require disk I/O to handle. perf_sw_ids::PERF_COUNT_SW_PAGE_FAULTS_MIN, + // This counts the number of major page faults. These required disk I/O to handle. perf_sw_ids::PERF_COUNT_SW_PAGE_FAULTS_MAJ, + // This counts the number of alignment faults. These happen when unaligned memory accesses happen; + // the kernel can handle these but it reduces performance. + // This happens only on some architectures (never on x86). perf_sw_ids::PERF_COUNT_SW_ALIGNMENT_FAULTS, - perf_sw_ids::PERF_COUNT_SW_EMULATION_FAULTS, - perf_sw_ids::PERF_COUNT_SW_DUMMY, - perf_sw_ids::PERF_COUNT_SW_BPF_OUTPUT + // This counts the number of emulation faults. The kernel sometimes traps on unimplemented instructions and + // emulates them for user space. This can negatively impact performance. + perf_sw_ids::PERF_COUNT_SW_EMULATION_FAULTS + // This is a placeholder event that counts nothing. Informational sample record types such as mmap or + // comm must be associated with an active event. This dummy event allows gathering such records + // without requiring a counting event. +// perf_sw_ids::PERF_COUNT_SW_DUMMY }; - static const std::string perf_event_names[] = { - "PERF_COUNT_SW_CPU_CLOCK", - "PERF_COUNT_SW_TASK_CLOCK", - "PERF_COUNT_SW_PAGE_FAULTS", - "PERF_COUNT_SW_CONTEXT_SWITCHES", - "PERF_COUNT_SW_CPU_MIGRATIONS", - "PERF_COUNT_SW_PAGE_FAULTS_MIN", - "PERF_COUNT_SW_PAGE_FAULTS_MAJ", - "PERF_COUNT_SW_ALIGNMENT_FAULTS", - "PERF_COUNT_SW_EMULATION_FAULTS", - "PERF_COUNT_SW_DUMMY", - "PERF_COUNT_SW_BPF_OUTPUT" - }; - - static const ProfileEvents::Event perf_events[] = { - ProfileEvents::PERF_COUNT_SW_CPU_CLOCK, + const ProfileEvents::Event PerfEventsCounters::perf_events[] = { + // a bit broken according to this: https://stackoverflow.com/a/56967896 +// ProfileEvents::PERF_COUNT_SW_CPU_CLOCK, ProfileEvents::PERF_COUNT_SW_TASK_CLOCK, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS, ProfileEvents::PERF_COUNT_SW_CONTEXT_SWITCHES, @@ -48,18 +56,12 @@ namespace DB { ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MIN, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MAJ, ProfileEvents::PERF_COUNT_SW_ALIGNMENT_FAULTS, - ProfileEvents::PERF_COUNT_SW_EMULATION_FAULTS, - ProfileEvents::PERF_COUNT_SW_DUMMY, - ProfileEvents::PERF_COUNT_SW_BPF_OUTPUT + ProfileEvents::PERF_COUNT_SW_EMULATION_FAULTS +// ProfileEvents::PERF_COUNT_SW_DUMMY, }; - constexpr size_t NUMBER_OF_EVENTS = std::size(perf_event_configs); - - static_assert(std::size(perf_event_names) == NUMBER_OF_EVENTS); - static_assert(std::size(perf_events) == NUMBER_OF_EVENTS); - - static int events_descriptors[NUMBER_OF_EVENTS]; - static bool perf_events_opened = false; + static_assert(std::size(PerfEventsCounters::perf_event_configs) == PerfEventsCounters::NUMBER_OF_EVENTS); + static_assert(std::size(PerfEventsCounters::perf_events) == PerfEventsCounters::NUMBER_OF_EVENTS); static int openPerfEvent(perf_event_attr *hw_event, pid_t pid, int cpu, int group_fd, unsigned long flags) { return static_cast(syscall(SYS_perf_event_open, hw_event, pid, cpu, group_fd, flags)); @@ -87,53 +89,22 @@ namespace DB { return true; } - static void perf_event_start(int perf_event_paranoid, int perf_event_type, int perf_event_config, int &event_file_descriptor) { + static void perfEventStart(int perf_event_paranoid, int perf_event_type, int perf_event_config, int &event_file_descriptor) { perf_event_attr pe = perf_event_attr(); pe.type = perf_event_type; pe.size = sizeof(struct perf_event_attr); pe.config = perf_event_config; + // disable by default to add as little extra time as possible pe.disabled = 1; // can record kernel only when `perf_event_paranoid` <= 1 pe.exclude_kernel = perf_event_paranoid >= 2; - pe.exclude_hv = 1; - event_file_descriptor = openPerfEvent(&pe, 0, -1, -1, 0); + event_file_descriptor = openPerfEvent(&pe, /* measure the calling thread */ 0, /* on any cpu */ -1, -1, 0); } -// static void disable_events() { -// if (!perf_events_opened) -// return; -// -// for (int & fd : events_descriptors) { -// if (fd == -1) -// continue; -// -// ioctl(fd, PERF_EVENT_IOC_DISABLE, 0); -// close(fd); -// fd = -1; -// } -// -// perf_events_opened = false; -// } - - void PerfEventsCounters::updateProfileEvents(ProfileEvents::Counters &profile_events) { - if (perf_events_opened) { - for (size_t i = 0; i < NUMBER_OF_EVENTS; ++i) { - int fd = events_descriptors[i]; - if (fd == -1) - continue; - - long long count; - read(fd, &count, sizeof(count)); - - profile_events.increment(perf_events[i], static_cast(count)); -// printf("%s: %lld\n", perf_event_names[i].c_str(), count); - - ioctl(fd, PERF_EVENT_IOC_RESET, 0); - } - + void PerfEventsCounters::initializeProfileEvents(PerfEventsCounters & counters) { + if (counters.perf_events_recording) return; - } int perf_event_paranoid = 0; bool is_pref_available = getPerfEventParanoid(perf_event_paranoid); @@ -144,27 +115,54 @@ namespace DB { for (size_t i = 0; i < NUMBER_OF_EVENTS; ++i) { - int eventConfig = perf_event_configs[i]; - std::string eventName = perf_event_names[i]; - - perf_event_start(perf_event_paranoid, perf_type_id::PERF_TYPE_SOFTWARE, eventConfig, events_descriptors[i]); + perfEventStart(perf_event_paranoid, perf_type_id::PERF_TYPE_SOFTWARE, perf_event_configs[i], counters.events_descriptors[i]); } for (size_t i = 0; i < NUMBER_OF_EVENTS; ++i) { - int fd = events_descriptors[i]; - if (fd == -1) + int fd = counters.events_descriptors[i]; + if (fd == -1) { fprintf(stderr, "Event config %d is unsupported\n", perf_event_configs[i]); - else - ioctl(fd, PERF_EVENT_IOC_ENABLE, 0); + continue; + } + + ioctl(fd, PERF_EVENT_IOC_ENABLE, 0); } - perf_events_opened = true; + counters.perf_events_recording = true; + } + + void PerfEventsCounters::finalizeProfileEvents(PerfEventsCounters & counters, ProfileEvents::Counters & profile_events) { + if (!counters.perf_events_recording) + return; + + for (size_t i = 0; i < NUMBER_OF_EVENTS; ++i) { + int & fd = counters.events_descriptors[i]; + if (fd == -1) + continue; + + long long count; + read(fd, &count, sizeof(count)); + + profile_events.increment(perf_events[i], static_cast(count)); +// printf("%s: %lld\n", perf_event_names[i].c_str(), count); + + if (ioctl(fd, PERF_EVENT_IOC_DISABLE, 0)) + fprintf(stderr, "Can't disable perf event with file descriptor: %d\n", fd); + + if (close(fd)) + fprintf(stderr, "Can't close perf event file descriptor: %d; error: %d - %s\n", fd, errno, strerror(errno)); + + fd = -1; + } + + counters.perf_events_recording = false; } #else - void PerfEventsCounters::updateProfileEvents(ProfileEvents::Counters &) {} + void PerfEventsCounters::initializeProfileEvents(PerfEventsCounters &) {} + void PerfEventsCounters::finalizeProfileEvents(PerfEventsCounters &, ProfileEvents::Counters &) {} #endif diff --git a/dbms/src/Common/ThreadProfileEvents.h b/dbms/src/Common/ThreadProfileEvents.h index 85236f5a176..767df93c43e 100644 --- a/dbms/src/Common/ThreadProfileEvents.h +++ b/dbms/src/Common/ThreadProfileEvents.h @@ -35,7 +35,7 @@ namespace ProfileEvents extern const Event OSReadBytes; extern const Event OSWriteBytes; - extern const Event PERF_COUNT_SW_CPU_CLOCK; +// extern const Event PERF_COUNT_SW_CPU_CLOCK; extern const Event PERF_COUNT_SW_TASK_CLOCK; extern const Event PERF_COUNT_SW_PAGE_FAULTS; extern const Event PERF_COUNT_SW_CONTEXT_SWITCHES; @@ -44,8 +44,6 @@ namespace ProfileEvents extern const Event PERF_COUNT_SW_PAGE_FAULTS_MAJ; extern const Event PERF_COUNT_SW_ALIGNMENT_FAULTS; extern const Event PERF_COUNT_SW_EMULATION_FAULTS; - extern const Event PERF_COUNT_SW_DUMMY; - extern const Event PERF_COUNT_SW_BPF_OUTPUT; #endif } @@ -144,7 +142,19 @@ struct PerfEventsCounters // that restricts perf_event_open() to processes with the CAP_SYS_ADMIN capability // todo: check whether perf_event_open() is available with CAP_SYS_ADMIN - static void updateProfileEvents(ProfileEvents::Counters & profile_events); +#if defined(__linux__) + static constexpr size_t NUMBER_OF_EVENTS = 8; + + static const int perf_event_configs[]; + static const ProfileEvents::Event perf_events[]; + + int events_descriptors[NUMBER_OF_EVENTS]; + bool perf_events_recording = false; +#endif + + static void initializeProfileEvents(PerfEventsCounters & counters); + + static void finalizeProfileEvents(PerfEventsCounters & counters, ProfileEvents::Counters & profile_events); }; #if defined(__linux__) diff --git a/dbms/src/Common/ThreadStatus.cpp b/dbms/src/Common/ThreadStatus.cpp index 7ceb848a7c8..90970a12c56 100644 --- a/dbms/src/Common/ThreadStatus.cpp +++ b/dbms/src/Common/ThreadStatus.cpp @@ -37,6 +37,7 @@ ThreadStatus::ThreadStatus() last_rusage = std::make_unique(); last_taskstats = std::make_unique(); + perf_events = std::make_unique(); memory_tracker.setDescription("(for thread)"); log = &Poco::Logger::get("ThreadStatus"); @@ -83,6 +84,15 @@ void ThreadStatus::initPerformanceCounters() *last_rusage = RUsageCounters::current(query_start_time_nanoseconds); + try + { + PerfEventsCounters::initializeProfileEvents(*perf_events); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + try { if (TaskStatsInfoGetter::checkPermissions()) @@ -105,7 +115,6 @@ void ThreadStatus::updatePerformanceCounters() try { RUsageCounters::updateProfileEvents(*last_rusage, performance_counters); - PerfEventsCounters::updateProfileEvents(performance_counters); if (taskstats_getter) TasksStatsCounters::updateProfileEvents(*last_taskstats, performance_counters); } diff --git a/dbms/src/Common/ThreadStatus.h b/dbms/src/Common/ThreadStatus.h index 58af6d4efff..1dacad5ab02 100644 --- a/dbms/src/Common/ThreadStatus.h +++ b/dbms/src/Common/ThreadStatus.h @@ -33,6 +33,7 @@ class QueryProfilerCpu; class QueryThreadLog; struct TasksStatsCounters; struct RUsageCounters; +struct PerfEventsCounters; class TaskStatsInfoGetter; class InternalTextLogsQueue; using InternalTextLogsQueuePtr = std::shared_ptr; @@ -196,6 +197,7 @@ protected: /// Use ptr not to add extra dependencies in the header std::unique_ptr last_rusage; std::unique_ptr last_taskstats; + std::unique_ptr perf_events; /// Set to non-nullptr only if we have enough capabilities. std::unique_ptr taskstats_getter; diff --git a/dbms/src/Interpreters/ThreadStatusExt.cpp b/dbms/src/Interpreters/ThreadStatusExt.cpp index 42b0721859f..692d43141ee 100644 --- a/dbms/src/Interpreters/ThreadStatusExt.cpp +++ b/dbms/src/Interpreters/ThreadStatusExt.cpp @@ -136,6 +136,15 @@ void ThreadStatus::finalizePerformanceCounters() performance_counters_finalized = true; updatePerformanceCounters(); + try + { + PerfEventsCounters::finalizeProfileEvents(*perf_events, performance_counters); + } + catch (...) + { + tryLogCurrentException(log); + } + try { if (global_context && query_context) From f769a51331113a1ce44cdae3884c9596c6da7dd3 Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Fri, 21 Feb 2020 21:43:23 +0300 Subject: [PATCH 007/183] Fixed logging for events + added hardware event and a couple of custom ones --- dbms/src/Common/ProfileEvents.cpp | 14 ++ dbms/src/Common/ThreadProfileEvents.cpp | 171 +++++++++++++++++------- dbms/src/Common/ThreadProfileEvents.h | 46 ++++++- 3 files changed, 180 insertions(+), 51 deletions(-) diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index cda51739c93..36f9e5c9392 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -178,6 +178,17 @@ M(OSReadChars, "Number of bytes read from filesystem, including page cache.") \ M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \ \ + M(PERF_COUNT_HW_CPU_CYCLES, "") \ + M(PERF_COUNT_HW_INSTRUCTIONS, "") \ + M(PERF_COUNT_HW_CACHE_REFERENCES, "") \ + M(PERF_COUNT_HW_CACHE_MISSES, "") \ + M(PERF_COUNT_HW_BRANCH_INSTRUCTIONS, "") \ + M(PERF_COUNT_HW_BRANCH_MISSES, "") \ + M(PERF_COUNT_HW_BUS_CYCLES, "") \ + M(PERF_COUNT_HW_STALLED_CYCLES_FRONTEND, "") \ + M(PERF_COUNT_HW_STALLED_CYCLES_BACKEND, "") \ + M(PERF_COUNT_HW_REF_CPU_CYCLES, "") \ + \ M(PERF_COUNT_SW_TASK_CLOCK, "A clock count specific to the task that is running") \ M(PERF_COUNT_SW_PAGE_FAULTS, "Number of page faults") \ M(PERF_COUNT_SW_CONTEXT_SWITCHES, "Number of context switches") \ @@ -187,6 +198,9 @@ M(PERF_COUNT_SW_ALIGNMENT_FAULTS, "Number of alignment faults. These happen when unaligned memory accesses happen; the kernel can handle these but it reduces performance. This happens only on some architectures (never on x86).") \ M(PERF_COUNT_SW_EMULATION_FAULTS, "Number of emulation faults. The kernel sometimes traps on unimplemented instructions and emulates them for user space. This can negatively impact performance.") \ \ + M(PERF_CUSTOM_INSTRUCTIONS_PER_CPU_CYCLE_SCALED, "") \ + M(PERF_CUSTOM_INSTRUCTIONS_PER_CPU_CYCLE, "") \ + \ M(CreatedHTTPConnections, "Total amount of created HTTP connections (closed or opened).") \ \ M(CannotWriteToWriteBufferDiscard, "Number of stack traces dropped by query profiler or signal handler because pipe is full or cannot write to pipe.") \ diff --git a/dbms/src/Common/ThreadProfileEvents.cpp b/dbms/src/Common/ThreadProfileEvents.cpp index 959bda87b06..67fe1bcc6cd 100644 --- a/dbms/src/Common/ThreadProfileEvents.cpp +++ b/dbms/src/Common/ThreadProfileEvents.cpp @@ -12,56 +12,103 @@ namespace DB { #if defined(__linux__) - // todo: think about event counters' overflow - // todo: ask about the usual error reporting (whether stderr is an accepted way) + static PerfEventInfo softwareEvent(int event_config, ProfileEvents::Event profile_event) + { + return PerfEventInfo + { + .event_type = perf_type_id::PERF_TYPE_SOFTWARE, + .event_config = event_config, + .profile_event = profile_event + }; + } + + static PerfEventInfo hardwareEvent(int event_config, ProfileEvents::Event profile_event) + { + return PerfEventInfo + { + .event_type = perf_type_id::PERF_TYPE_HARDWARE, + .event_config = event_config, + .profile_event = profile_event + }; + } // descriptions' source: http://man7.org/linux/man-pages/man2/perf_event_open.2.html - const int PerfEventsCounters::perf_event_configs[] = { + const PerfEventInfo PerfEventsCounters::perf_raw_events_info[] = { + + // Total cycles. Be wary of what happens during CPU frequency scaling. + hardwareEvent(perf_hw_id::PERF_COUNT_HW_CPU_CYCLES, ProfileEvents::PERF_COUNT_HW_CPU_CYCLES), + // Retired instructions. Be careful, these can be affected by various issues, most notably hardware + // interrupt counts. + hardwareEvent(perf_hw_id::PERF_COUNT_HW_INSTRUCTIONS, ProfileEvents::PERF_COUNT_HW_INSTRUCTIONS), + // Cache accesses. Usually this indicates Last Level Cache accesses but this may vary depending on your CPU. + // This may include prefetches and coherency messages; again this depends on the design of your CPU. + hardwareEvent(perf_hw_id::PERF_COUNT_HW_CACHE_REFERENCES, ProfileEvents::PERF_COUNT_HW_CACHE_REFERENCES), + // Cache misses. Usually this indicates Last Level Cache misses; this is intended to be used in con‐junction + // with the PERF_COUNT_HW_CACHE_REFERENCES event to calculate cache miss rates. + hardwareEvent(perf_hw_id::PERF_COUNT_HW_CACHE_MISSES, ProfileEvents::PERF_COUNT_HW_CACHE_MISSES), + // Retired branch instructions. Prior to Linux 2.6.35, this used the wrong event on AMD processors. + hardwareEvent(perf_hw_id::PERF_COUNT_HW_BRANCH_INSTRUCTIONS, ProfileEvents::PERF_COUNT_HW_BRANCH_INSTRUCTIONS), + // Mispredicted branch instructions. + hardwareEvent(perf_hw_id::PERF_COUNT_HW_BRANCH_MISSES, ProfileEvents::PERF_COUNT_HW_BRANCH_MISSES), + // Bus cycles, which can be different from total cycles. + hardwareEvent(perf_hw_id::PERF_COUNT_HW_BUS_CYCLES, ProfileEvents::PERF_COUNT_HW_BUS_CYCLES), + // Stalled cycles during issue. + hardwareEvent(perf_hw_id::PERF_COUNT_HW_STALLED_CYCLES_FRONTEND, ProfileEvents::PERF_COUNT_HW_STALLED_CYCLES_FRONTEND), + // Stalled cycles during retirement. + hardwareEvent(perf_hw_id::PERF_COUNT_HW_STALLED_CYCLES_BACKEND, ProfileEvents::PERF_COUNT_HW_STALLED_CYCLES_BACKEND), + // Total cycles; not affected by CPU frequency scaling. + hardwareEvent(perf_hw_id::PERF_COUNT_HW_REF_CPU_CYCLES, ProfileEvents::PERF_COUNT_HW_REF_CPU_CYCLES), + // This reports the CPU clock, a high-resolution per-CPU timer. -// perf_sw_ids::PERF_COUNT_SW_CPU_CLOCK, + // a bit broken according to this: https://stackoverflow.com/a/56967896 +// makeInfo(perf_type_id::PERF_TYPE_SOFTWARE, perf_sw_ids::PERF_COUNT_SW_CPU_CLOCK, ProfileEvents::PERF_COUNT_SW_CPU_CLOCK), // This reports a clock count specific to the task that is running. - perf_sw_ids::PERF_COUNT_SW_TASK_CLOCK, + softwareEvent(perf_sw_ids::PERF_COUNT_SW_TASK_CLOCK, ProfileEvents::PERF_COUNT_SW_TASK_CLOCK), // This reports the number of page faults. - perf_sw_ids::PERF_COUNT_SW_PAGE_FAULTS, + softwareEvent(perf_sw_ids::PERF_COUNT_SW_PAGE_FAULTS, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS), // This counts context switches. // Until Linux 2.6.34, these were all reported as user-space events, // after that they are reported as happening in the kernel - perf_sw_ids::PERF_COUNT_SW_CONTEXT_SWITCHES, + softwareEvent(perf_sw_ids::PERF_COUNT_SW_CONTEXT_SWITCHES, ProfileEvents::PERF_COUNT_SW_CONTEXT_SWITCHES), // This reports the number of times the process has migrated to a new CPU. - perf_sw_ids::PERF_COUNT_SW_CPU_MIGRATIONS, + softwareEvent(perf_sw_ids::PERF_COUNT_SW_CPU_MIGRATIONS, ProfileEvents::PERF_COUNT_SW_CPU_MIGRATIONS), // This counts the number of minor page faults. These did not require disk I/O to handle. - perf_sw_ids::PERF_COUNT_SW_PAGE_FAULTS_MIN, + softwareEvent(perf_sw_ids::PERF_COUNT_SW_PAGE_FAULTS_MIN, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MIN), // This counts the number of major page faults. These required disk I/O to handle. - perf_sw_ids::PERF_COUNT_SW_PAGE_FAULTS_MAJ, + softwareEvent(perf_sw_ids::PERF_COUNT_SW_PAGE_FAULTS_MAJ, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MAJ), // This counts the number of alignment faults. These happen when unaligned memory accesses happen; // the kernel can handle these but it reduces performance. // This happens only on some architectures (never on x86). - perf_sw_ids::PERF_COUNT_SW_ALIGNMENT_FAULTS, + softwareEvent(perf_sw_ids::PERF_COUNT_SW_ALIGNMENT_FAULTS, ProfileEvents::PERF_COUNT_SW_ALIGNMENT_FAULTS), // This counts the number of emulation faults. The kernel sometimes traps on unimplemented instructions and // emulates them for user space. This can negatively impact performance. - perf_sw_ids::PERF_COUNT_SW_EMULATION_FAULTS + softwareEvent(perf_sw_ids::PERF_COUNT_SW_EMULATION_FAULTS, ProfileEvents::PERF_COUNT_SW_EMULATION_FAULTS) // This is a placeholder event that counts nothing. Informational sample record types such as mmap or // comm must be associated with an active event. This dummy event allows gathering such records // without requiring a counting event. -// perf_sw_ids::PERF_COUNT_SW_DUMMY +// softwareEventInfo(perf_sw_ids::PERF_COUNT_SW_DUMMY, ProfileEvents::PERF_COUNT_SW_DUMMY) }; - const ProfileEvents::Event PerfEventsCounters::perf_events[] = { - // a bit broken according to this: https://stackoverflow.com/a/56967896 -// ProfileEvents::PERF_COUNT_SW_CPU_CLOCK, - ProfileEvents::PERF_COUNT_SW_TASK_CLOCK, - ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS, - ProfileEvents::PERF_COUNT_SW_CONTEXT_SWITCHES, - ProfileEvents::PERF_COUNT_SW_CPU_MIGRATIONS, - ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MIN, - ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MAJ, - ProfileEvents::PERF_COUNT_SW_ALIGNMENT_FAULTS, - ProfileEvents::PERF_COUNT_SW_EMULATION_FAULTS -// ProfileEvents::PERF_COUNT_SW_DUMMY, - }; + static_assert(std::size(PerfEventsCounters::perf_raw_events_info) == PerfEventsCounters::NUMBER_OF_RAW_EVENTS); - static_assert(std::size(PerfEventsCounters::perf_event_configs) == PerfEventsCounters::NUMBER_OF_EVENTS); - static_assert(std::size(PerfEventsCounters::perf_events) == PerfEventsCounters::NUMBER_OF_EVENTS); + const std::map PerfEventsCounters::event_config_to_info_index = [] { + std::map map; + for (size_t i = 0; i < PerfEventsCounters::NUMBER_OF_RAW_EVENTS; ++i) + map.emplace(PerfEventsCounters::perf_raw_events_info[i].event_config, i); + return map; + } (); + + std::atomic PerfEventsCounters::events_availability_logged = false; + + Logger * PerfEventsCounters::getLogger() + { + return &Logger::get("PerfEventsCounters"); + } + + long long PerfEventsCounters::getRawValue(int event_config) + { + return raw_event_values[event_config_to_info_index.find(event_config)->second]; + } static int openPerfEvent(perf_event_attr *hw_event, pid_t pid, int cpu, int group_fd, unsigned long flags) { return static_cast(syscall(SYS_perf_event_open, hw_event, pid, cpu, group_fd, flags)); @@ -89,7 +136,7 @@ namespace DB { return true; } - static void perfEventStart(int perf_event_paranoid, int perf_event_type, int perf_event_config, int &event_file_descriptor) { + static void perfEventOpenDisabled(int perf_event_paranoid, int perf_event_type, int perf_event_config, int &event_file_descriptor) { perf_event_attr pe = perf_event_attr(); pe.type = perf_event_type; pe.size = sizeof(struct perf_event_attr); @@ -113,20 +160,28 @@ namespace DB { if (!is_pref_available) return; - for (size_t i = 0; i < NUMBER_OF_EVENTS; ++i) + bool expected = false; + bool log_unsupported_event = events_availability_logged.compare_exchange_strong(expected, true); + for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) { - perfEventStart(perf_event_paranoid, perf_type_id::PERF_TYPE_SOFTWARE, perf_event_configs[i], counters.events_descriptors[i]); + counters.raw_event_values[i] = 0; + const PerfEventInfo & event_info = perf_raw_events_info[i]; + int & fd = counters.events_descriptors[i]; + perfEventOpenDisabled(perf_event_paranoid, event_info.event_type, event_info.event_config, fd); + + if (fd == -1 && log_unsupported_event) + { + LOG_WARNING( + getLogger(), + "Perf event is unsupported: event_type=" << event_info.event_type + << ", event_config=" << event_info.event_config); + } } - for (size_t i = 0; i < NUMBER_OF_EVENTS; ++i) + for (int fd : counters.events_descriptors) { - int fd = counters.events_descriptors[i]; - if (fd == -1) { - fprintf(stderr, "Event config %d is unsupported\n", perf_event_configs[i]); - continue; - } - - ioctl(fd, PERF_EVENT_IOC_ENABLE, 0); + if (fd != -1) + ioctl(fd, PERF_EVENT_IOC_ENABLE, 0); } counters.perf_events_recording = true; @@ -136,26 +191,48 @@ namespace DB { if (!counters.perf_events_recording) return; - for (size_t i = 0; i < NUMBER_OF_EVENTS; ++i) { + // process raw events + + // only read counters here to have as little overhead for processing as possible + for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) + { + int fd = counters.events_descriptors[i]; + if (fd != -1) + read(fd, &counters.raw_event_values[i], sizeof(long long)); + } + + // actually process counters' values and release resources + for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) + { int & fd = counters.events_descriptors[i]; if (fd == -1) continue; - long long count; - read(fd, &count, sizeof(count)); - - profile_events.increment(perf_events[i], static_cast(count)); -// printf("%s: %lld\n", perf_event_names[i].c_str(), count); + profile_events.increment(perf_raw_events_info[i].profile_event, counters.raw_event_values[i]); if (ioctl(fd, PERF_EVENT_IOC_DISABLE, 0)) - fprintf(stderr, "Can't disable perf event with file descriptor: %d\n", fd); + LOG_WARNING(getLogger(), "Can't disable perf event with file descriptor: " << fd); if (close(fd)) - fprintf(stderr, "Can't close perf event file descriptor: %d; error: %d - %s\n", fd, errno, strerror(errno)); + LOG_WARNING(getLogger(), "Can't close perf event file descriptor: " << fd << "; error: " << errno << " - " << strerror(errno)); fd = -1; } + // process custom events which depend on the raw ones + long long hw_cpu_cycles = counters.getRawValue(perf_hw_id::PERF_COUNT_HW_CPU_CYCLES); + long long hw_ref_cpu_cycles = counters.getRawValue(perf_hw_id::PERF_COUNT_HW_REF_CPU_CYCLES); + + long long instructions_per_cpu_scaled = hw_cpu_cycles != 0 + ? counters.getRawValue(perf_hw_id::PERF_COUNT_HW_INSTRUCTIONS) / hw_cpu_cycles + : 0; + long long instructions_per_cpu = hw_ref_cpu_cycles != 0 + ? counters.getRawValue(perf_hw_id::PERF_COUNT_HW_INSTRUCTIONS) / hw_ref_cpu_cycles + : 0; + + profile_events.increment(ProfileEvents::PERF_CUSTOM_INSTRUCTIONS_PER_CPU_CYCLE_SCALED, instructions_per_cpu_scaled); + profile_events.increment(ProfileEvents::PERF_CUSTOM_INSTRUCTIONS_PER_CPU_CYCLE, instructions_per_cpu); + counters.perf_events_recording = false; } diff --git a/dbms/src/Common/ThreadProfileEvents.h b/dbms/src/Common/ThreadProfileEvents.h index 767df93c43e..2ca262bfd60 100644 --- a/dbms/src/Common/ThreadProfileEvents.h +++ b/dbms/src/Common/ThreadProfileEvents.h @@ -5,6 +5,8 @@ #include #include #include +#include +#include #if defined(__linux__) #include @@ -35,6 +37,17 @@ namespace ProfileEvents extern const Event OSReadBytes; extern const Event OSWriteBytes; + extern const Event PERF_COUNT_HW_CPU_CYCLES; + extern const Event PERF_COUNT_HW_INSTRUCTIONS; + extern const Event PERF_COUNT_HW_CACHE_REFERENCES; + extern const Event PERF_COUNT_HW_CACHE_MISSES; + extern const Event PERF_COUNT_HW_BRANCH_INSTRUCTIONS; + extern const Event PERF_COUNT_HW_BRANCH_MISSES; + extern const Event PERF_COUNT_HW_BUS_CYCLES; + extern const Event PERF_COUNT_HW_STALLED_CYCLES_FRONTEND; + extern const Event PERF_COUNT_HW_STALLED_CYCLES_BACKEND; + extern const Event PERF_COUNT_HW_REF_CPU_CYCLES; + // extern const Event PERF_COUNT_SW_CPU_CLOCK; extern const Event PERF_COUNT_SW_TASK_CLOCK; extern const Event PERF_COUNT_SW_PAGE_FAULTS; @@ -44,6 +57,9 @@ namespace ProfileEvents extern const Event PERF_COUNT_SW_PAGE_FAULTS_MAJ; extern const Event PERF_COUNT_SW_ALIGNMENT_FAULTS; extern const Event PERF_COUNT_SW_EMULATION_FAULTS; + + extern const Event PERF_CUSTOM_INSTRUCTIONS_PER_CPU_CYCLE_SCALED; + extern const Event PERF_CUSTOM_INSTRUCTIONS_PER_CPU_CYCLE; #endif } @@ -127,6 +143,18 @@ struct RUsageCounters } }; +#if defined(__linux__) + + struct PerfEventInfo { + // see perf_event.h/perf_type_id enum + int event_type; + // see configs in perf_event.h + int event_config; + ProfileEvents::Event profile_event; + }; + +#endif + struct PerfEventsCounters { // cat /proc/sys/kernel/perf_event_paranoid - if perf_event_paranoid is set to 3, all calls to `perf_event_open` are rejected (even for the current process) @@ -143,18 +171,28 @@ struct PerfEventsCounters // todo: check whether perf_event_open() is available with CAP_SYS_ADMIN #if defined(__linux__) - static constexpr size_t NUMBER_OF_EVENTS = 8; + static constexpr size_t NUMBER_OF_RAW_EVENTS = 18; - static const int perf_event_configs[]; - static const ProfileEvents::Event perf_events[]; + static const PerfEventInfo perf_raw_events_info[]; + static const std::map event_config_to_info_index; - int events_descriptors[NUMBER_OF_EVENTS]; + int events_descriptors[NUMBER_OF_RAW_EVENTS]; + // temp array just to not create it each time event processing finishes + long long raw_event_values[NUMBER_OF_RAW_EVENTS]; bool perf_events_recording = false; #endif static void initializeProfileEvents(PerfEventsCounters & counters); static void finalizeProfileEvents(PerfEventsCounters & counters, ProfileEvents::Counters & profile_events); + +private: + // used to write information about perf event availability only once for all threads + static std::atomic events_availability_logged; + + static Logger * getLogger(); + + long long getRawValue(int event_config); }; #if defined(__linux__) From ce49b1aff77852a29bbb654141c95f3e37ac9231 Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Fri, 21 Feb 2020 21:59:08 +0300 Subject: [PATCH 008/183] Removed map from the perf events --- dbms/src/Common/ThreadProfileEvents.cpp | 63 +++++++++++++------------ dbms/src/Common/ThreadProfileEvents.h | 6 +-- 2 files changed, 35 insertions(+), 34 deletions(-) diff --git a/dbms/src/Common/ThreadProfileEvents.cpp b/dbms/src/Common/ThreadProfileEvents.cpp index 67fe1bcc6cd..0d307447922 100644 --- a/dbms/src/Common/ThreadProfileEvents.cpp +++ b/dbms/src/Common/ThreadProfileEvents.cpp @@ -36,53 +36,53 @@ namespace DB { const PerfEventInfo PerfEventsCounters::perf_raw_events_info[] = { // Total cycles. Be wary of what happens during CPU frequency scaling. - hardwareEvent(perf_hw_id::PERF_COUNT_HW_CPU_CYCLES, ProfileEvents::PERF_COUNT_HW_CPU_CYCLES), + hardwareEvent(PERF_COUNT_HW_CPU_CYCLES, ProfileEvents::PERF_COUNT_HW_CPU_CYCLES), // Retired instructions. Be careful, these can be affected by various issues, most notably hardware // interrupt counts. - hardwareEvent(perf_hw_id::PERF_COUNT_HW_INSTRUCTIONS, ProfileEvents::PERF_COUNT_HW_INSTRUCTIONS), + hardwareEvent(PERF_COUNT_HW_INSTRUCTIONS, ProfileEvents::PERF_COUNT_HW_INSTRUCTIONS), // Cache accesses. Usually this indicates Last Level Cache accesses but this may vary depending on your CPU. // This may include prefetches and coherency messages; again this depends on the design of your CPU. - hardwareEvent(perf_hw_id::PERF_COUNT_HW_CACHE_REFERENCES, ProfileEvents::PERF_COUNT_HW_CACHE_REFERENCES), + hardwareEvent(PERF_COUNT_HW_CACHE_REFERENCES, ProfileEvents::PERF_COUNT_HW_CACHE_REFERENCES), // Cache misses. Usually this indicates Last Level Cache misses; this is intended to be used in con‐junction // with the PERF_COUNT_HW_CACHE_REFERENCES event to calculate cache miss rates. - hardwareEvent(perf_hw_id::PERF_COUNT_HW_CACHE_MISSES, ProfileEvents::PERF_COUNT_HW_CACHE_MISSES), + hardwareEvent(PERF_COUNT_HW_CACHE_MISSES, ProfileEvents::PERF_COUNT_HW_CACHE_MISSES), // Retired branch instructions. Prior to Linux 2.6.35, this used the wrong event on AMD processors. - hardwareEvent(perf_hw_id::PERF_COUNT_HW_BRANCH_INSTRUCTIONS, ProfileEvents::PERF_COUNT_HW_BRANCH_INSTRUCTIONS), + hardwareEvent(PERF_COUNT_HW_BRANCH_INSTRUCTIONS, ProfileEvents::PERF_COUNT_HW_BRANCH_INSTRUCTIONS), // Mispredicted branch instructions. - hardwareEvent(perf_hw_id::PERF_COUNT_HW_BRANCH_MISSES, ProfileEvents::PERF_COUNT_HW_BRANCH_MISSES), + hardwareEvent(PERF_COUNT_HW_BRANCH_MISSES, ProfileEvents::PERF_COUNT_HW_BRANCH_MISSES), // Bus cycles, which can be different from total cycles. - hardwareEvent(perf_hw_id::PERF_COUNT_HW_BUS_CYCLES, ProfileEvents::PERF_COUNT_HW_BUS_CYCLES), + hardwareEvent(PERF_COUNT_HW_BUS_CYCLES, ProfileEvents::PERF_COUNT_HW_BUS_CYCLES), // Stalled cycles during issue. - hardwareEvent(perf_hw_id::PERF_COUNT_HW_STALLED_CYCLES_FRONTEND, ProfileEvents::PERF_COUNT_HW_STALLED_CYCLES_FRONTEND), + hardwareEvent(PERF_COUNT_HW_STALLED_CYCLES_FRONTEND, ProfileEvents::PERF_COUNT_HW_STALLED_CYCLES_FRONTEND), // Stalled cycles during retirement. - hardwareEvent(perf_hw_id::PERF_COUNT_HW_STALLED_CYCLES_BACKEND, ProfileEvents::PERF_COUNT_HW_STALLED_CYCLES_BACKEND), + hardwareEvent(PERF_COUNT_HW_STALLED_CYCLES_BACKEND, ProfileEvents::PERF_COUNT_HW_STALLED_CYCLES_BACKEND), // Total cycles; not affected by CPU frequency scaling. - hardwareEvent(perf_hw_id::PERF_COUNT_HW_REF_CPU_CYCLES, ProfileEvents::PERF_COUNT_HW_REF_CPU_CYCLES), + hardwareEvent(PERF_COUNT_HW_REF_CPU_CYCLES, ProfileEvents::PERF_COUNT_HW_REF_CPU_CYCLES), // This reports the CPU clock, a high-resolution per-CPU timer. // a bit broken according to this: https://stackoverflow.com/a/56967896 // makeInfo(perf_type_id::PERF_TYPE_SOFTWARE, perf_sw_ids::PERF_COUNT_SW_CPU_CLOCK, ProfileEvents::PERF_COUNT_SW_CPU_CLOCK), // This reports a clock count specific to the task that is running. - softwareEvent(perf_sw_ids::PERF_COUNT_SW_TASK_CLOCK, ProfileEvents::PERF_COUNT_SW_TASK_CLOCK), + softwareEvent(PERF_COUNT_SW_TASK_CLOCK, ProfileEvents::PERF_COUNT_SW_TASK_CLOCK), // This reports the number of page faults. - softwareEvent(perf_sw_ids::PERF_COUNT_SW_PAGE_FAULTS, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS), + softwareEvent(PERF_COUNT_SW_PAGE_FAULTS, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS), // This counts context switches. // Until Linux 2.6.34, these were all reported as user-space events, // after that they are reported as happening in the kernel - softwareEvent(perf_sw_ids::PERF_COUNT_SW_CONTEXT_SWITCHES, ProfileEvents::PERF_COUNT_SW_CONTEXT_SWITCHES), + softwareEvent(PERF_COUNT_SW_CONTEXT_SWITCHES, ProfileEvents::PERF_COUNT_SW_CONTEXT_SWITCHES), // This reports the number of times the process has migrated to a new CPU. - softwareEvent(perf_sw_ids::PERF_COUNT_SW_CPU_MIGRATIONS, ProfileEvents::PERF_COUNT_SW_CPU_MIGRATIONS), + softwareEvent(PERF_COUNT_SW_CPU_MIGRATIONS, ProfileEvents::PERF_COUNT_SW_CPU_MIGRATIONS), // This counts the number of minor page faults. These did not require disk I/O to handle. - softwareEvent(perf_sw_ids::PERF_COUNT_SW_PAGE_FAULTS_MIN, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MIN), + softwareEvent(PERF_COUNT_SW_PAGE_FAULTS_MIN, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MIN), // This counts the number of major page faults. These required disk I/O to handle. - softwareEvent(perf_sw_ids::PERF_COUNT_SW_PAGE_FAULTS_MAJ, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MAJ), + softwareEvent(PERF_COUNT_SW_PAGE_FAULTS_MAJ, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MAJ), // This counts the number of alignment faults. These happen when unaligned memory accesses happen; // the kernel can handle these but it reduces performance. // This happens only on some architectures (never on x86). - softwareEvent(perf_sw_ids::PERF_COUNT_SW_ALIGNMENT_FAULTS, ProfileEvents::PERF_COUNT_SW_ALIGNMENT_FAULTS), + softwareEvent(PERF_COUNT_SW_ALIGNMENT_FAULTS, ProfileEvents::PERF_COUNT_SW_ALIGNMENT_FAULTS), // This counts the number of emulation faults. The kernel sometimes traps on unimplemented instructions and // emulates them for user space. This can negatively impact performance. - softwareEvent(perf_sw_ids::PERF_COUNT_SW_EMULATION_FAULTS, ProfileEvents::PERF_COUNT_SW_EMULATION_FAULTS) + softwareEvent(PERF_COUNT_SW_EMULATION_FAULTS, ProfileEvents::PERF_COUNT_SW_EMULATION_FAULTS) // This is a placeholder event that counts nothing. Informational sample record types such as mmap or // comm must be associated with an active event. This dummy event allows gathering such records // without requiring a counting event. @@ -91,13 +91,6 @@ namespace DB { static_assert(std::size(PerfEventsCounters::perf_raw_events_info) == PerfEventsCounters::NUMBER_OF_RAW_EVENTS); - const std::map PerfEventsCounters::event_config_to_info_index = [] { - std::map map; - for (size_t i = 0; i < PerfEventsCounters::NUMBER_OF_RAW_EVENTS; ++i) - map.emplace(PerfEventsCounters::perf_raw_events_info[i].event_config, i); - return map; - } (); - std::atomic PerfEventsCounters::events_availability_logged = false; Logger * PerfEventsCounters::getLogger() @@ -105,9 +98,17 @@ namespace DB { return &Logger::get("PerfEventsCounters"); } - long long PerfEventsCounters::getRawValue(int event_config) + long long PerfEventsCounters::getRawValue(int event_type, int event_config) const { - return raw_event_values[event_config_to_info_index.find(event_config)->second]; + for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) + { + const PerfEventInfo & event_info = perf_raw_events_info[i]; + if (event_info.event_type == event_type && event_info.event_config == event_config) + return raw_event_values[i]; + } + + LOG_WARNING(getLogger(), "Can't find perf event info for event_type=" << event_type << ", event_config=" << event_config); + return 0; } static int openPerfEvent(perf_event_attr *hw_event, pid_t pid, int cpu, int group_fd, unsigned long flags) { @@ -220,14 +221,14 @@ namespace DB { } // process custom events which depend on the raw ones - long long hw_cpu_cycles = counters.getRawValue(perf_hw_id::PERF_COUNT_HW_CPU_CYCLES); - long long hw_ref_cpu_cycles = counters.getRawValue(perf_hw_id::PERF_COUNT_HW_REF_CPU_CYCLES); + long long hw_cpu_cycles = counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_CPU_CYCLES); + long long hw_ref_cpu_cycles = counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_REF_CPU_CYCLES); long long instructions_per_cpu_scaled = hw_cpu_cycles != 0 - ? counters.getRawValue(perf_hw_id::PERF_COUNT_HW_INSTRUCTIONS) / hw_cpu_cycles + ? counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_INSTRUCTIONS) / hw_cpu_cycles : 0; long long instructions_per_cpu = hw_ref_cpu_cycles != 0 - ? counters.getRawValue(perf_hw_id::PERF_COUNT_HW_INSTRUCTIONS) / hw_ref_cpu_cycles + ? counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_INSTRUCTIONS) / hw_ref_cpu_cycles : 0; profile_events.increment(ProfileEvents::PERF_CUSTOM_INSTRUCTIONS_PER_CPU_CYCLE_SCALED, instructions_per_cpu_scaled); diff --git a/dbms/src/Common/ThreadProfileEvents.h b/dbms/src/Common/ThreadProfileEvents.h index 2ca262bfd60..bf11a382997 100644 --- a/dbms/src/Common/ThreadProfileEvents.h +++ b/dbms/src/Common/ThreadProfileEvents.h @@ -5,7 +5,6 @@ #include #include #include -#include #include #if defined(__linux__) @@ -174,7 +173,6 @@ struct PerfEventsCounters static constexpr size_t NUMBER_OF_RAW_EVENTS = 18; static const PerfEventInfo perf_raw_events_info[]; - static const std::map event_config_to_info_index; int events_descriptors[NUMBER_OF_RAW_EVENTS]; // temp array just to not create it each time event processing finishes @@ -186,13 +184,15 @@ struct PerfEventsCounters static void finalizeProfileEvents(PerfEventsCounters & counters, ProfileEvents::Counters & profile_events); +#if defined(__linux__) private: // used to write information about perf event availability only once for all threads static std::atomic events_availability_logged; static Logger * getLogger(); - long long getRawValue(int event_config); + [[nodiscard]] long long getRawValue(int event_type, int event_config) const; +#endif }; #if defined(__linux__) From ca7b5a3a056bfb7269477a444046e441eac60cbd Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Sat, 22 Feb 2020 00:34:33 +0300 Subject: [PATCH 009/183] Moved events' comments to the definitions file --- dbms/src/Common/ProfileEvents.cpp | 20 +++++++++--------- dbms/src/Common/ThreadProfileEvents.cpp | 27 ------------------------- 2 files changed, 10 insertions(+), 37 deletions(-) diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index 36f9e5c9392..970eb89ff71 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -178,16 +178,16 @@ M(OSReadChars, "Number of bytes read from filesystem, including page cache.") \ M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \ \ - M(PERF_COUNT_HW_CPU_CYCLES, "") \ - M(PERF_COUNT_HW_INSTRUCTIONS, "") \ - M(PERF_COUNT_HW_CACHE_REFERENCES, "") \ - M(PERF_COUNT_HW_CACHE_MISSES, "") \ - M(PERF_COUNT_HW_BRANCH_INSTRUCTIONS, "") \ - M(PERF_COUNT_HW_BRANCH_MISSES, "") \ - M(PERF_COUNT_HW_BUS_CYCLES, "") \ - M(PERF_COUNT_HW_STALLED_CYCLES_FRONTEND, "") \ - M(PERF_COUNT_HW_STALLED_CYCLES_BACKEND, "") \ - M(PERF_COUNT_HW_REF_CPU_CYCLES, "") \ + M(PERF_COUNT_HW_CPU_CYCLES, "Total cycles. Be wary of what happens during CPU frequency scaling.") \ + M(PERF_COUNT_HW_INSTRUCTIONS, "Retired instructions. Be careful, these can be affected by various issues, most notably hardware interrupt counts.") \ + M(PERF_COUNT_HW_CACHE_REFERENCES, "Cache accesses. Usually this indicates Last Level Cache accesses but this may vary depending on your CPU. This may include prefetches and coherency messages; again this depends on the design of your CPU.") \ + M(PERF_COUNT_HW_CACHE_MISSES, "Cache misses. Usually this indicates Last Level Cache misses; this is intended to be used in con‐junction with the PERF_COUNT_HW_CACHE_REFERENCES event to calculate cache miss rates.") \ + M(PERF_COUNT_HW_BRANCH_INSTRUCTIONS, "Retired branch instructions. Prior to Linux 2.6.35, this used the wrong event on AMD processors.") \ + M(PERF_COUNT_HW_BRANCH_MISSES, "Mispredicted branch instructions.") \ + M(PERF_COUNT_HW_BUS_CYCLES, "Bus cycles, which can be different from total cycles.") \ + M(PERF_COUNT_HW_STALLED_CYCLES_FRONTEND, "Stalled cycles during issue.") \ + M(PERF_COUNT_HW_STALLED_CYCLES_BACKEND, "Stalled cycles during retirement.") \ + M(PERF_COUNT_HW_REF_CPU_CYCLES, "Total cycles; not affected by CPU frequency scaling.") \ \ M(PERF_COUNT_SW_TASK_CLOCK, "A clock count specific to the task that is running") \ M(PERF_COUNT_SW_PAGE_FAULTS, "Number of page faults") \ diff --git a/dbms/src/Common/ThreadProfileEvents.cpp b/dbms/src/Common/ThreadProfileEvents.cpp index 0d307447922..981380831de 100644 --- a/dbms/src/Common/ThreadProfileEvents.cpp +++ b/dbms/src/Common/ThreadProfileEvents.cpp @@ -34,54 +34,27 @@ namespace DB { // descriptions' source: http://man7.org/linux/man-pages/man2/perf_event_open.2.html const PerfEventInfo PerfEventsCounters::perf_raw_events_info[] = { - - // Total cycles. Be wary of what happens during CPU frequency scaling. hardwareEvent(PERF_COUNT_HW_CPU_CYCLES, ProfileEvents::PERF_COUNT_HW_CPU_CYCLES), - // Retired instructions. Be careful, these can be affected by various issues, most notably hardware - // interrupt counts. hardwareEvent(PERF_COUNT_HW_INSTRUCTIONS, ProfileEvents::PERF_COUNT_HW_INSTRUCTIONS), - // Cache accesses. Usually this indicates Last Level Cache accesses but this may vary depending on your CPU. - // This may include prefetches and coherency messages; again this depends on the design of your CPU. hardwareEvent(PERF_COUNT_HW_CACHE_REFERENCES, ProfileEvents::PERF_COUNT_HW_CACHE_REFERENCES), - // Cache misses. Usually this indicates Last Level Cache misses; this is intended to be used in con‐junction - // with the PERF_COUNT_HW_CACHE_REFERENCES event to calculate cache miss rates. hardwareEvent(PERF_COUNT_HW_CACHE_MISSES, ProfileEvents::PERF_COUNT_HW_CACHE_MISSES), - // Retired branch instructions. Prior to Linux 2.6.35, this used the wrong event on AMD processors. hardwareEvent(PERF_COUNT_HW_BRANCH_INSTRUCTIONS, ProfileEvents::PERF_COUNT_HW_BRANCH_INSTRUCTIONS), - // Mispredicted branch instructions. hardwareEvent(PERF_COUNT_HW_BRANCH_MISSES, ProfileEvents::PERF_COUNT_HW_BRANCH_MISSES), - // Bus cycles, which can be different from total cycles. hardwareEvent(PERF_COUNT_HW_BUS_CYCLES, ProfileEvents::PERF_COUNT_HW_BUS_CYCLES), - // Stalled cycles during issue. hardwareEvent(PERF_COUNT_HW_STALLED_CYCLES_FRONTEND, ProfileEvents::PERF_COUNT_HW_STALLED_CYCLES_FRONTEND), - // Stalled cycles during retirement. hardwareEvent(PERF_COUNT_HW_STALLED_CYCLES_BACKEND, ProfileEvents::PERF_COUNT_HW_STALLED_CYCLES_BACKEND), - // Total cycles; not affected by CPU frequency scaling. hardwareEvent(PERF_COUNT_HW_REF_CPU_CYCLES, ProfileEvents::PERF_COUNT_HW_REF_CPU_CYCLES), // This reports the CPU clock, a high-resolution per-CPU timer. // a bit broken according to this: https://stackoverflow.com/a/56967896 // makeInfo(perf_type_id::PERF_TYPE_SOFTWARE, perf_sw_ids::PERF_COUNT_SW_CPU_CLOCK, ProfileEvents::PERF_COUNT_SW_CPU_CLOCK), - // This reports a clock count specific to the task that is running. softwareEvent(PERF_COUNT_SW_TASK_CLOCK, ProfileEvents::PERF_COUNT_SW_TASK_CLOCK), - // This reports the number of page faults. softwareEvent(PERF_COUNT_SW_PAGE_FAULTS, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS), - // This counts context switches. - // Until Linux 2.6.34, these were all reported as user-space events, - // after that they are reported as happening in the kernel softwareEvent(PERF_COUNT_SW_CONTEXT_SWITCHES, ProfileEvents::PERF_COUNT_SW_CONTEXT_SWITCHES), - // This reports the number of times the process has migrated to a new CPU. softwareEvent(PERF_COUNT_SW_CPU_MIGRATIONS, ProfileEvents::PERF_COUNT_SW_CPU_MIGRATIONS), - // This counts the number of minor page faults. These did not require disk I/O to handle. softwareEvent(PERF_COUNT_SW_PAGE_FAULTS_MIN, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MIN), - // This counts the number of major page faults. These required disk I/O to handle. softwareEvent(PERF_COUNT_SW_PAGE_FAULTS_MAJ, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MAJ), - // This counts the number of alignment faults. These happen when unaligned memory accesses happen; - // the kernel can handle these but it reduces performance. - // This happens only on some architectures (never on x86). softwareEvent(PERF_COUNT_SW_ALIGNMENT_FAULTS, ProfileEvents::PERF_COUNT_SW_ALIGNMENT_FAULTS), - // This counts the number of emulation faults. The kernel sometimes traps on unimplemented instructions and - // emulates them for user space. This can negatively impact performance. softwareEvent(PERF_COUNT_SW_EMULATION_FAULTS, ProfileEvents::PERF_COUNT_SW_EMULATION_FAULTS) // This is a placeholder event that counts nothing. Informational sample record types such as mmap or // comm must be associated with an active event. This dummy event allows gathering such records From 50c603a74c1a5fa2a74809ba040232d97541500c Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Fri, 6 Mar 2020 19:30:50 +0300 Subject: [PATCH 010/183] Paranoid str to int check --- dbms/src/Common/ThreadProfileEvents.cpp | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/dbms/src/Common/ThreadProfileEvents.cpp b/dbms/src/Common/ThreadProfileEvents.cpp index 981380831de..7a964797be8 100644 --- a/dbms/src/Common/ThreadProfileEvents.cpp +++ b/dbms/src/Common/ThreadProfileEvents.cpp @@ -105,8 +105,12 @@ namespace DB { return false; str[MAX_LENGTH - 1] = '\0'; - // todo: change to `strtol` - result = atoi(str); + long value = strtol(str, nullptr, 10); + // the only way to be incorrect is to not be a number + if (value == 0 && errno != 0) + return false; + + result = static_cast(value); return true; } From 0f1dff21b2e85e0f49f14f1f0c2c14f6918b7d3e Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Fri, 6 Mar 2020 19:31:31 +0300 Subject: [PATCH 011/183] Reformatting code + static initializers --- dbms/src/Common/ThreadProfileEvents.cpp | 26 ++++++++++--------------- dbms/src/Common/ThreadProfileEvents.h | 4 ++-- 2 files changed, 12 insertions(+), 18 deletions(-) diff --git a/dbms/src/Common/ThreadProfileEvents.cpp b/dbms/src/Common/ThreadProfileEvents.cpp index 7a964797be8..c636781528d 100644 --- a/dbms/src/Common/ThreadProfileEvents.cpp +++ b/dbms/src/Common/ThreadProfileEvents.cpp @@ -47,7 +47,7 @@ namespace DB { // This reports the CPU clock, a high-resolution per-CPU timer. // a bit broken according to this: https://stackoverflow.com/a/56967896 -// makeInfo(perf_type_id::PERF_TYPE_SOFTWARE, perf_sw_ids::PERF_COUNT_SW_CPU_CLOCK, ProfileEvents::PERF_COUNT_SW_CPU_CLOCK), +// softwareEvent(PERF_COUNT_SW_CPU_CLOCK, ProfileEvents::PERF_COUNT_SW_CPU_CLOCK), softwareEvent(PERF_COUNT_SW_TASK_CLOCK, ProfileEvents::PERF_COUNT_SW_TASK_CLOCK), softwareEvent(PERF_COUNT_SW_PAGE_FAULTS, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS), softwareEvent(PERF_COUNT_SW_CONTEXT_SWITCHES, ProfileEvents::PERF_COUNT_SW_CONTEXT_SWITCHES), @@ -59,7 +59,7 @@ namespace DB { // This is a placeholder event that counts nothing. Informational sample record types such as mmap or // comm must be associated with an active event. This dummy event allows gathering such records // without requiring a counting event. -// softwareEventInfo(perf_sw_ids::PERF_COUNT_SW_DUMMY, ProfileEvents::PERF_COUNT_SW_DUMMY) +// softwareEventInfo(PERF_COUNT_SW_DUMMY, ProfileEvents::PERF_COUNT_SW_DUMMY) }; static_assert(std::size(PerfEventsCounters::perf_raw_events_info) == PerfEventsCounters::NUMBER_OF_RAW_EVENTS); @@ -88,19 +88,17 @@ namespace DB { return static_cast(syscall(SYS_perf_event_open, hw_event, pid, cpu, group_fd, flags)); } - static bool getPerfEventParanoid(int &result) { + static bool getPerfEventParanoid(int & result) { // the longest possible variant: "-1\0" constexpr int MAX_LENGTH = 3; - FILE *fp; - char str[MAX_LENGTH]; - fp = fopen("/proc/sys/kernel/perf_event_paranoid", "r"); + FILE * fp = fopen("/proc/sys/kernel/perf_event_paranoid", "r"); if (fp == nullptr) return false; - char *res = fgets(str, MAX_LENGTH, fp); + char str[MAX_LENGTH]; + char * res = fgets(str, MAX_LENGTH, fp); fclose(fp); - if (res == nullptr) return false; @@ -114,7 +112,7 @@ namespace DB { return true; } - static void perfEventOpenDisabled(int perf_event_paranoid, int perf_event_type, int perf_event_config, int &event_file_descriptor) { + static void perfEventOpenDisabled(int perf_event_paranoid, int perf_event_type, int perf_event_config, int & event_file_descriptor) { perf_event_attr pe = perf_event_attr(); pe.type = perf_event_type; pe.size = sizeof(struct perf_event_attr); @@ -133,8 +131,6 @@ namespace DB { int perf_event_paranoid = 0; bool is_pref_available = getPerfEventParanoid(perf_event_paranoid); -// printf("is_perf_available: %s, perf_event_paranoid: %d\n", is_pref_available ? "true" : "false", perf_event_paranoid); - if (!is_pref_available) return; @@ -149,9 +145,7 @@ namespace DB { if (fd == -1 && log_unsupported_event) { - LOG_WARNING( - getLogger(), - "Perf event is unsupported: event_type=" << event_info.event_type + LOG_WARNING(getLogger(), "Perf event is unsupported: event_type=" << event_info.event_type << ", event_config=" << event_info.event_config); } } @@ -190,9 +184,9 @@ namespace DB { if (ioctl(fd, PERF_EVENT_IOC_DISABLE, 0)) LOG_WARNING(getLogger(), "Can't disable perf event with file descriptor: " << fd); - if (close(fd)) - LOG_WARNING(getLogger(), "Can't close perf event file descriptor: " << fd << "; error: " << errno << " - " << strerror(errno)); + LOG_WARNING(getLogger(),"Can't close perf event file descriptor: " << fd + << "; error: " << errno << " - " << strerror(errno)); fd = -1; } diff --git a/dbms/src/Common/ThreadProfileEvents.h b/dbms/src/Common/ThreadProfileEvents.h index bf11a382997..047c13d304c 100644 --- a/dbms/src/Common/ThreadProfileEvents.h +++ b/dbms/src/Common/ThreadProfileEvents.h @@ -174,9 +174,9 @@ struct PerfEventsCounters static const PerfEventInfo perf_raw_events_info[]; - int events_descriptors[NUMBER_OF_RAW_EVENTS]; + int events_descriptors[NUMBER_OF_RAW_EVENTS]{}; // temp array just to not create it each time event processing finishes - long long raw_event_values[NUMBER_OF_RAW_EVENTS]; + long long raw_event_values[NUMBER_OF_RAW_EVENTS]{}; bool perf_events_recording = false; #endif From 8650c1996763c989d5c404a764391b24f54df54b Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Tue, 14 Apr 2020 00:10:31 +0300 Subject: [PATCH 012/183] Fixed styling --- base/common/ThreadProfileEvents.cpp | 19 ++++++++++++------- src/Common/ThreadProfileEvents.h | 15 ++++++++------- 2 files changed, 20 insertions(+), 14 deletions(-) diff --git a/base/common/ThreadProfileEvents.cpp b/base/common/ThreadProfileEvents.cpp index c636781528d..4f9f98686a5 100644 --- a/base/common/ThreadProfileEvents.cpp +++ b/base/common/ThreadProfileEvents.cpp @@ -8,7 +8,8 @@ #include #endif -namespace DB { +namespace DB +{ #if defined(__linux__) @@ -44,7 +45,6 @@ namespace DB { hardwareEvent(PERF_COUNT_HW_STALLED_CYCLES_FRONTEND, ProfileEvents::PERF_COUNT_HW_STALLED_CYCLES_FRONTEND), hardwareEvent(PERF_COUNT_HW_STALLED_CYCLES_BACKEND, ProfileEvents::PERF_COUNT_HW_STALLED_CYCLES_BACKEND), hardwareEvent(PERF_COUNT_HW_REF_CPU_CYCLES, ProfileEvents::PERF_COUNT_HW_REF_CPU_CYCLES), - // This reports the CPU clock, a high-resolution per-CPU timer. // a bit broken according to this: https://stackoverflow.com/a/56967896 // softwareEvent(PERF_COUNT_SW_CPU_CLOCK, ProfileEvents::PERF_COUNT_SW_CPU_CLOCK), @@ -84,11 +84,13 @@ namespace DB { return 0; } - static int openPerfEvent(perf_event_attr *hw_event, pid_t pid, int cpu, int group_fd, unsigned long flags) { + static int openPerfEvent(perf_event_attr *hw_event, pid_t pid, int cpu, int group_fd, unsigned long flags) + { return static_cast(syscall(SYS_perf_event_open, hw_event, pid, cpu, group_fd, flags)); } - static bool getPerfEventParanoid(int & result) { + static bool getPerfEventParanoid(int & result) + { // the longest possible variant: "-1\0" constexpr int MAX_LENGTH = 3; @@ -112,7 +114,8 @@ namespace DB { return true; } - static void perfEventOpenDisabled(int perf_event_paranoid, int perf_event_type, int perf_event_config, int & event_file_descriptor) { + static void perfEventOpenDisabled(int perf_event_paranoid, int perf_event_type, int perf_event_config, int & event_file_descriptor) + { perf_event_attr pe = perf_event_attr(); pe.type = perf_event_type; pe.size = sizeof(struct perf_event_attr); @@ -125,7 +128,8 @@ namespace DB { event_file_descriptor = openPerfEvent(&pe, /* measure the calling thread */ 0, /* on any cpu */ -1, -1, 0); } - void PerfEventsCounters::initializeProfileEvents(PerfEventsCounters & counters) { + void PerfEventsCounters::initializeProfileEvents(PerfEventsCounters & counters) + { if (counters.perf_events_recording) return; @@ -159,7 +163,8 @@ namespace DB { counters.perf_events_recording = true; } - void PerfEventsCounters::finalizeProfileEvents(PerfEventsCounters & counters, ProfileEvents::Counters & profile_events) { + void PerfEventsCounters::finalizeProfileEvents(PerfEventsCounters & counters, ProfileEvents::Counters & profile_events) + { if (!counters.perf_events_recording) return; diff --git a/src/Common/ThreadProfileEvents.h b/src/Common/ThreadProfileEvents.h index 047c13d304c..f59b0622da0 100644 --- a/src/Common/ThreadProfileEvents.h +++ b/src/Common/ThreadProfileEvents.h @@ -144,13 +144,14 @@ struct RUsageCounters #if defined(__linux__) - struct PerfEventInfo { - // see perf_event.h/perf_type_id enum - int event_type; - // see configs in perf_event.h - int event_config; - ProfileEvents::Event profile_event; - }; +struct PerfEventInfo +{ + // see perf_event.h/perf_type_id enum + int event_type; + // see configs in perf_event.h + int event_config; + ProfileEvents::Event profile_event; +}; #endif From 2f708fec03ec859e5e13fd2ad39b76352490a113 Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Tue, 14 Apr 2020 15:17:59 +0300 Subject: [PATCH 013/183] Fixed compilation issues --- {base/common => src/Common}/ThreadProfileEvents.cpp | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) rename {base/common => src/Common}/ThreadProfileEvents.cpp (95%) diff --git a/base/common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp similarity index 95% rename from base/common/ThreadProfileEvents.cpp rename to src/Common/ThreadProfileEvents.cpp index 4f9f98686a5..e1b7d0859fc 100644 --- a/base/common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -174,8 +174,15 @@ namespace DB for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) { int fd = counters.events_descriptors[i]; - if (fd != -1) - read(fd, &counters.raw_event_values[i], sizeof(long long)); + if (fd == -1) + continue; + + constexpr ssize_t bytesToRead = sizeof(counters.raw_event_values[0]); + if (read(fd, &counters.raw_event_values[i], bytesToRead) != bytesToRead) + { + LOG_WARNING(getLogger(), "Can't read event value from file descriptor: " << fd); + counters.raw_event_values[i] = 0; + } } // actually process counters' values and release resources @@ -220,4 +227,4 @@ namespace DB #endif -} \ No newline at end of file +} From 578faa01a7694f059cf83f535d512e8c62212bf3 Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Tue, 14 Apr 2020 16:55:46 +0300 Subject: [PATCH 014/183] Handle `CAP_SYS_ADMIN` permission accordingly --- src/Common/ThreadProfileEvents.cpp | 28 ++++++++++++++++++++++------ src/Common/ThreadProfileEvents.h | 6 ++++-- 2 files changed, 26 insertions(+), 8 deletions(-) diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index e1b7d0859fc..3658e424283 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -6,6 +6,7 @@ #include #include #include +#include "hasLinuxCapability.h" #endif namespace DB @@ -64,7 +65,8 @@ namespace DB static_assert(std::size(PerfEventsCounters::perf_raw_events_info) == PerfEventsCounters::NUMBER_OF_RAW_EVENTS); - std::atomic PerfEventsCounters::events_availability_logged = false; + std::atomic PerfEventsCounters::perf_unavailability_logged = false; + std::atomic PerfEventsCounters::particular_events_unavailability_logged = false; Logger * PerfEventsCounters::getLogger() { @@ -114,7 +116,7 @@ namespace DB return true; } - static void perfEventOpenDisabled(int perf_event_paranoid, int perf_event_type, int perf_event_config, int & event_file_descriptor) + static void perfEventOpenDisabled(int perf_event_paranoid, bool has_cap_sys_admin, int perf_event_type, int perf_event_config, int & event_file_descriptor) { perf_event_attr pe = perf_event_attr(); pe.type = perf_event_type; @@ -122,8 +124,8 @@ namespace DB pe.config = perf_event_config; // disable by default to add as little extra time as possible pe.disabled = 1; - // can record kernel only when `perf_event_paranoid` <= 1 - pe.exclude_kernel = perf_event_paranoid >= 2; + // can record kernel only when `perf_event_paranoid` <= 1 or have CAP_SYS_ADMIN + pe.exclude_kernel = perf_event_paranoid >= 2 && !has_cap_sys_admin; event_file_descriptor = openPerfEvent(&pe, /* measure the calling thread */ 0, /* on any cpu */ -1, -1, 0); } @@ -136,16 +138,30 @@ namespace DB int perf_event_paranoid = 0; bool is_pref_available = getPerfEventParanoid(perf_event_paranoid); if (!is_pref_available) + { + bool expected_value = false; + if (perf_unavailability_logged.compare_exchange_strong(expected_value, true)) + LOG_WARNING(getLogger(), "Perf events are unsupported"); return; + } + + bool has_cap_sys_admin = hasLinuxCapability(CAP_SYS_ADMIN); + if (perf_event_paranoid >= 3 && !has_cap_sys_admin) + { + bool expected_value = false; + if (perf_unavailability_logged.compare_exchange_strong(expected_value, true)) + LOG_WARNING(getLogger(), "Not enough permissions to record perf events"); + return; + } bool expected = false; - bool log_unsupported_event = events_availability_logged.compare_exchange_strong(expected, true); + bool log_unsupported_event = particular_events_unavailability_logged.compare_exchange_strong(expected, true); for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) { counters.raw_event_values[i] = 0; const PerfEventInfo & event_info = perf_raw_events_info[i]; int & fd = counters.events_descriptors[i]; - perfEventOpenDisabled(perf_event_paranoid, event_info.event_type, event_info.event_config, fd); + perfEventOpenDisabled(perf_event_paranoid, has_cap_sys_admin, event_info.event_type, event_info.event_config, fd); if (fd == -1 && log_unsupported_event) { diff --git a/src/Common/ThreadProfileEvents.h b/src/Common/ThreadProfileEvents.h index f59b0622da0..b52e25a995e 100644 --- a/src/Common/ThreadProfileEvents.h +++ b/src/Common/ThreadProfileEvents.h @@ -187,8 +187,10 @@ struct PerfEventsCounters #if defined(__linux__) private: - // used to write information about perf event availability only once for all threads - static std::atomic events_availability_logged; + // used to write information about perf unavailability only once for all threads + static std::atomic perf_unavailability_logged; + // used to write information about particular perf events unavailability only once for all threads + static std::atomic particular_events_unavailability_logged; static Logger * getLogger(); From f4f43ee8ab8b1bd4b10f233707ff509d8f88edd4 Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Tue, 14 Apr 2020 16:58:32 +0300 Subject: [PATCH 015/183] Replaced some warnings with infos as perf events measurements are optional --- src/Common/ThreadProfileEvents.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index 3658e424283..7cb555bad42 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -141,7 +141,7 @@ namespace DB { bool expected_value = false; if (perf_unavailability_logged.compare_exchange_strong(expected_value, true)) - LOG_WARNING(getLogger(), "Perf events are unsupported"); + LOG_INFO(getLogger(), "Perf events are unsupported"); return; } @@ -150,7 +150,7 @@ namespace DB { bool expected_value = false; if (perf_unavailability_logged.compare_exchange_strong(expected_value, true)) - LOG_WARNING(getLogger(), "Not enough permissions to record perf events"); + LOG_INFO(getLogger(), "Not enough permissions to record perf events"); return; } @@ -165,7 +165,7 @@ namespace DB if (fd == -1 && log_unsupported_event) { - LOG_WARNING(getLogger(), "Perf event is unsupported: event_type=" << event_info.event_type + LOG_INFO(getLogger(), "Perf event is unsupported: event_type=" << event_info.event_type << ", event_config=" << event_info.event_config); } } From 396c9e427944e50933f59c154743d34b4f70e8da Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Tue, 14 Apr 2020 19:23:33 +0300 Subject: [PATCH 016/183] Fixed styling v2 --- src/Common/ThreadProfileEvents.cpp | 34 +++++++++++++++--------------- src/Common/ThreadProfileEvents.h | 4 ++-- 2 files changed, 19 insertions(+), 19 deletions(-) diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index 7cb555bad42..a032e98b076 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -73,7 +73,7 @@ namespace DB return &Logger::get("PerfEventsCounters"); } - long long PerfEventsCounters::getRawValue(int event_type, int event_config) const + Int64 PerfEventsCounters::getRawValue(int event_type, int event_config) const { for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) { @@ -86,37 +86,37 @@ namespace DB return 0; } - static int openPerfEvent(perf_event_attr *hw_event, pid_t pid, int cpu, int group_fd, unsigned long flags) + static int openPerfEvent(perf_event_attr *hw_event, pid_t pid, int cpu, int group_fd, UInt64 flags) { return static_cast(syscall(SYS_perf_event_open, hw_event, pid, cpu, group_fd, flags)); } - static bool getPerfEventParanoid(int & result) + static bool getPerfEventParanoid(Int32 & result) { // the longest possible variant: "-1\0" - constexpr int MAX_LENGTH = 3; + constexpr Int32 max_length = 3; FILE * fp = fopen("/proc/sys/kernel/perf_event_paranoid", "r"); if (fp == nullptr) return false; - char str[MAX_LENGTH]; - char * res = fgets(str, MAX_LENGTH, fp); + char str[max_length]; + char * res = fgets(str, max_length, fp); fclose(fp); if (res == nullptr) return false; - str[MAX_LENGTH - 1] = '\0'; - long value = strtol(str, nullptr, 10); + str[max_length - 1] = '\0'; + Int64 value = strtol(str, nullptr, 10); // the only way to be incorrect is to not be a number if (value == 0 && errno != 0) return false; - result = static_cast(value); + result = static_cast(value); return true; } - static void perfEventOpenDisabled(int perf_event_paranoid, bool has_cap_sys_admin, int perf_event_type, int perf_event_config, int & event_file_descriptor) + static void perfEventOpenDisabled(Int32 perf_event_paranoid, bool has_cap_sys_admin, int perf_event_type, int perf_event_config, int & event_file_descriptor) { perf_event_attr pe = perf_event_attr(); pe.type = perf_event_type; @@ -135,7 +135,7 @@ namespace DB if (counters.perf_events_recording) return; - int perf_event_paranoid = 0; + Int32 perf_event_paranoid = 0; bool is_pref_available = getPerfEventParanoid(perf_event_paranoid); if (!is_pref_available) { @@ -193,8 +193,8 @@ namespace DB if (fd == -1) continue; - constexpr ssize_t bytesToRead = sizeof(counters.raw_event_values[0]); - if (read(fd, &counters.raw_event_values[i], bytesToRead) != bytesToRead) + constexpr ssize_t bytes_to_read = sizeof(counters.raw_event_values[0]); + if (read(fd, &counters.raw_event_values[i], bytes_to_read) != bytes_to_read) { LOG_WARNING(getLogger(), "Can't read event value from file descriptor: " << fd); counters.raw_event_values[i] = 0; @@ -220,13 +220,13 @@ namespace DB } // process custom events which depend on the raw ones - long long hw_cpu_cycles = counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_CPU_CYCLES); - long long hw_ref_cpu_cycles = counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_REF_CPU_CYCLES); + Int64 hw_cpu_cycles = counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_CPU_CYCLES); + Int64 hw_ref_cpu_cycles = counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_REF_CPU_CYCLES); - long long instructions_per_cpu_scaled = hw_cpu_cycles != 0 + Int64 instructions_per_cpu_scaled = hw_cpu_cycles != 0 ? counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_INSTRUCTIONS) / hw_cpu_cycles : 0; - long long instructions_per_cpu = hw_ref_cpu_cycles != 0 + Int64 instructions_per_cpu = hw_ref_cpu_cycles != 0 ? counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_INSTRUCTIONS) / hw_ref_cpu_cycles : 0; diff --git a/src/Common/ThreadProfileEvents.h b/src/Common/ThreadProfileEvents.h index b52e25a995e..bfd923175a6 100644 --- a/src/Common/ThreadProfileEvents.h +++ b/src/Common/ThreadProfileEvents.h @@ -177,7 +177,7 @@ struct PerfEventsCounters int events_descriptors[NUMBER_OF_RAW_EVENTS]{}; // temp array just to not create it each time event processing finishes - long long raw_event_values[NUMBER_OF_RAW_EVENTS]{}; + Int64 raw_event_values[NUMBER_OF_RAW_EVENTS]{}; bool perf_events_recording = false; #endif @@ -194,7 +194,7 @@ private: static Logger * getLogger(); - [[nodiscard]] long long getRawValue(int event_type, int event_config) const; + [[nodiscard]] Int64 getRawValue(int event_type, int event_config) const; #endif }; From 11f94baf4bfc8bb42bfb250ca5cd18bc64c6837a Mon Sep 17 00:00:00 2001 From: Dmitry Date: Sat, 18 Apr 2020 12:51:21 +0300 Subject: [PATCH 017/183] First realization, many drowbacks --- .../AggregateFunctions/IAggregateFunction.h | 8 + dbms/src/Common/Arena.h | 2 +- dbms/src/Interpreters/Aggregator.cpp | 146 ++++++++++- dbms/src/Interpreters/Aggregator.h | 64 +++-- .../Interpreters/InterpreterSelectQuery.cpp | 116 +++++---- .../src/Interpreters/InterpreterSelectQuery.h | 1 + dbms/src/Processors/IProcessor.h | 4 +- dbms/src/Processors/ISource.cpp | 2 +- .../AggregatingInOrderTransform.cpp | 236 ++++++++++++++++++ .../Transforms/AggregatingInOrderTransform.h | 50 ++++ .../Transforms/FinishSortingTransform.cpp | 6 +- 11 files changed, 559 insertions(+), 76 deletions(-) create mode 100644 dbms/src/Processors/Transforms/AggregatingInOrderTransform.cpp create mode 100644 dbms/src/Processors/Transforms/AggregatingInOrderTransform.h diff --git a/dbms/src/AggregateFunctions/IAggregateFunction.h b/dbms/src/AggregateFunctions/IAggregateFunction.h index d7ccd4c206a..0fd1814371a 100644 --- a/dbms/src/AggregateFunctions/IAggregateFunction.h +++ b/dbms/src/AggregateFunctions/IAggregateFunction.h @@ -141,6 +141,8 @@ public: */ virtual void addBatchSinglePlace(size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const = 0; + virtual void addBatchSinglePlaceFromInterval(size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const = 0; + /** In addition to addBatch, this method collects multiple rows of arguments into array "places" * as long as they are between offsets[i-1] and offsets[i]. This is used for arrayReduce and * -Array combinator. It might also be used generally to break data dependency when array @@ -186,6 +188,12 @@ public: static_cast(this)->add(place, columns, i, arena); } + void addBatchSinglePlaceFromInterval(size_t batch_begin, size_t batch_end, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const override + { + for (size_t i = batch_begin; i < batch_end; ++i) + static_cast(this)->add(place, columns, i, arena); + } + void addBatchArray( size_t batch_size, AggregateDataPtr * places, size_t place_offset, const IColumn ** columns, const UInt64 * offsets, Arena * arena) const override diff --git a/dbms/src/Common/Arena.h b/dbms/src/Common/Arena.h index e1556ef73c5..4fd3afe4355 100644 --- a/dbms/src/Common/Arena.h +++ b/dbms/src/Common/Arena.h @@ -150,7 +150,7 @@ public: return res; } - /// Get peice of memory with alignment + /// Get piece of memory with alignment char * alignedAlloc(size_t size, size_t alignment) { do diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 07c1d7476ad..04d675a35fa 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -522,13 +522,29 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl( for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) { if (inst->offsets) - inst->batch_that->addBatchSinglePlace( - inst->offsets[static_cast(rows - 1)], res + inst->state_offset, inst->batch_arguments, arena); + inst->batch_that->addBatchSinglePlace(inst->offsets[static_cast(rows - 1)], res + inst->state_offset, inst->batch_arguments, arena); else inst->batch_that->addBatchSinglePlace(rows, res + inst->state_offset, inst->batch_arguments, arena); } } +void NO_INLINE Aggregator::executeOnIntervalWithoutKeyImpl( + AggregatedDataWithoutKey & res, + size_t row_begin, + size_t row_end, + AggregateFunctionInstruction * aggregate_instructions, + Arena * arena) +{ + /// Adding values + for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) + { + if (inst->offsets) + inst->batch_that->addBatchSinglePlaceFromInterval(inst->offsets[row_begin], inst->offsets[static_cast(row_end - 1)], res + inst->state_offset, inst->batch_arguments, arena); + else + inst->batch_that->addBatchSinglePlaceFromInterval(row_begin, row_end, res + inst->state_offset, inst->batch_arguments, arena); + } +} + bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & result, ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, bool & no_more_keys) @@ -537,6 +553,99 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re return executeOnBlock(block.getColumns(), num_rows, result, key_columns, aggregate_columns, no_more_keys); } +AggregateFunctionInstructions NO_INLINE Aggregator::prepareBlockForAggregation(Columns & materialized_columns, Columns columns, AggregatedDataVariants & result, + ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns) +{ + /// TODO remove code duplication + + /// `result` will destroy the states of aggregate functions in the destructor + result.aggregator = this; + + /// How to perform the aggregation? + if (result.empty()) + { + result.init(method_chosen); + result.keys_size = params.keys_size; + result.key_sizes = key_sizes; + LOG_TRACE(log, "Aggregation method: " << result.getMethodName()); + } + + for (size_t i = 0; i < params.aggregates_size; ++i) + aggregate_columns[i].resize(params.aggregates[i].arguments.size()); + + /** Constant columns are not supported directly during aggregation. + * To make them work anyway, we materialize them. + */ +// Columns materialized_columns; + + /// Remember the columns we will work with + for (size_t i = 0; i < params.keys_size; ++i) + { + materialized_columns.push_back(columns.at(params.keys[i])->convertToFullColumnIfConst()); + key_columns[i] = materialized_columns.back().get(); + + if (!result.isLowCardinality()) + { + auto column_no_lc = recursiveRemoveLowCardinality(key_columns[i]->getPtr()); + if (column_no_lc.get() != key_columns[i]) + { + materialized_columns.emplace_back(std::move(column_no_lc)); + key_columns[i] = materialized_columns.back().get(); + } + } + } + + AggregateFunctionInstructions aggregate_functions_instructions(params.aggregates_size + 1); + aggregate_functions_instructions[params.aggregates_size].that = nullptr; + + std::vector> nested_columns_holder; + for (size_t i = 0; i < params.aggregates_size; ++i) + { + for (size_t j = 0; j < aggregate_columns[i].size(); ++j) + { + materialized_columns.push_back(columns.at(params.aggregates[i].arguments[j])->convertToFullColumnIfConst()); + aggregate_columns[i][j] = materialized_columns.back().get(); + + auto column_no_lc = recursiveRemoveLowCardinality(aggregate_columns[i][j]->getPtr()); + if (column_no_lc.get() != aggregate_columns[i][j]) + { + materialized_columns.emplace_back(std::move(column_no_lc)); + aggregate_columns[i][j] = materialized_columns.back().get(); + } + } + + aggregate_functions_instructions[i].arguments = aggregate_columns[i].data(); + aggregate_functions_instructions[i].state_offset = offsets_of_aggregate_states[i]; + auto that = aggregate_functions[i]; + + /// Unnest consecutive trailing -State combinators + while (auto func = typeid_cast(that)) + that = func->getNestedFunction().get(); + + aggregate_functions_instructions[i].that = that; + aggregate_functions_instructions[i].func = that->getAddressOfAddFunction(); + + if (auto func = typeid_cast(that)) + { + /// Unnest consecutive -State combinators before -Array + that = func->getNestedFunction().get(); + while (auto nested_func = typeid_cast(that)) + that = nested_func->getNestedFunction().get(); + + auto [nested_columns, offsets] = checkAndGetNestedArrayOffset(aggregate_columns[i].data(), that->getArgumentTypes().size()); + nested_columns_holder.push_back(std::move(nested_columns)); + aggregate_functions_instructions[i].batch_arguments = nested_columns_holder.back().data(); + aggregate_functions_instructions[i].offsets = offsets; + } + else + aggregate_functions_instructions[i].batch_arguments = aggregate_columns[i].data(); + + aggregate_functions_instructions[i].batch_that = that; + } + + return aggregate_functions_instructions; +} + bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedDataVariants & result, ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, bool & no_more_keys) { @@ -605,9 +714,11 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData aggregate_functions_instructions[i].arguments = aggregate_columns[i].data(); aggregate_functions_instructions[i].state_offset = offsets_of_aggregate_states[i]; auto that = aggregate_functions[i]; + /// Unnest consecutive trailing -State combinators while (auto func = typeid_cast(that)) that = func->getNestedFunction().get(); + aggregate_functions_instructions[i].that = that; aggregate_functions_instructions[i].func = that->getAddressOfAddFunction(); @@ -617,6 +728,7 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData that = func->getNestedFunction().get(); while (auto nested_func = typeid_cast(that)) that = nested_func->getNestedFunction().get(); + auto [nested_columns, offsets] = checkAndGetNestedArrayOffset(aggregate_columns[i].data(), that->getArgumentTypes().size()); nested_columns_holder.push_back(std::move(nested_columns)); aggregate_functions_instructions[i].batch_arguments = nested_columns_holder.back().data(); @@ -1052,7 +1164,7 @@ Block Aggregator::prepareBlockAndFill( aggregate_columns[i] = header.safeGetByPosition(i + params.keys_size).type->createColumn(); /// The ColumnAggregateFunction column captures the shared ownership of the arena with the aggregate function states. - ColumnAggregateFunction & column_aggregate_func = assert_cast(*aggregate_columns[i]); + auto & column_aggregate_func = assert_cast(*aggregate_columns[i]); for (auto & pool : data_variants.aggregates_pools) column_aggregate_func.addArena(pool); @@ -1078,6 +1190,7 @@ Block Aggregator::prepareBlockAndFill( filler(key_columns, aggregate_columns_data, final_aggregate_columns, final); +//CREATING LAST BLOCK Block res = header.cloneEmpty(); for (size_t i = 0; i < params.keys_size; ++i) @@ -1099,7 +1212,34 @@ Block Aggregator::prepareBlockAndFill( return res; } +void Aggregator::fillAggregateColumnsWithSingleKey( + AggregatedDataVariants & data_variants, + MutableColumns & final_aggregate_columns) +{ + AggregatedDataWithoutKey & data = data_variants.without_key; + for (size_t i = 0; i < params.aggregates_size; ++i) + { + aggregate_functions[i]->insertResultInto(data + offsets_of_aggregate_states[i], *final_aggregate_columns[i]); + } + destroyWithoutKey(data_variants); +} + +void Aggregator::createStatesAndFillKeyColumnsWithSingleKey( + AggregatedDataVariants & data_variants, + ColumnRawPtrs key_columns, + size_t key_row, + MutableColumns & final_key_columns) +{ + AggregateDataPtr place = data_variants.aggregates_pool->alignedAlloc(total_size_of_aggregate_states, align_aggregate_states); + createAggregateStates(place); + data_variants.without_key = place; + + for (size_t i = 0; i < params.keys_size; ++i) + { + final_key_columns[i]->insertFrom(*key_columns[i], key_row); + } +} Block Aggregator::prepareBlockAndFillWithoutKey(AggregatedDataVariants & data_variants, bool final, bool is_overflows) const { diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index dc833456e14..fa1bb6e2a85 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -828,6 +828,28 @@ using AggregatedDataVariantsPtr = std::shared_ptr; using ManyAggregatedDataVariants = std::vector; using ManyAggregatedDataVariantsPtr = std::shared_ptr; +/** This array serves two purposes. + * + * 1. Function arguments are collected side by side, and they do not need to be collected from different places. Also the array is made zero-terminated. + * The inner loop (for the case without_key) is almost twice as compact; performance gain of about 30%. + * + * 2. Calling a function by pointer is better than a virtual call, because in the case of a virtual call, + * GCC 5.1.2 generates code that, at each iteration of the loop, reloads the function address from memory into the register + * (the offset value in the virtual function table). + */ +struct AggregateFunctionInstruction +{ + const IAggregateFunction * that; + IAggregateFunction::AddFunc func; + size_t state_offset; + const IColumn ** arguments; + const IAggregateFunction * batch_that; + const IColumn ** batch_arguments; + const UInt64 * offsets = nullptr; +}; + +using AggregateFunctionInstructions = std::vector; + /** How are "total" values calculated with WITH TOTALS? * (For more details, see TotalsHavingBlockInputStream.) * @@ -932,6 +954,9 @@ public: ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block bool & no_more_keys); + AggregateFunctionInstructions prepareBlockForAggregation(Columns & materialized_columns, Columns columns, AggregatedDataVariants & result, + ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns); + /** Convert the aggregation data structure into a block. * If overflow_row = true, then aggregates for rows that are not included in max_rows_to_group_by are put in the first block. * @@ -1002,6 +1027,7 @@ protected: friend class MergingAndConvertingBlockInputStream; friend class ConvertingAggregatedToChunksTransform; friend class ConvertingAggregatedToChunksSource; + friend class AggregatingInOrderTransform; Params params; @@ -1012,28 +1038,6 @@ protected: AggregateFunctionsPlainPtrs aggregate_functions; - /** This array serves two purposes. - * - * 1. Function arguments are collected side by side, and they do not need to be collected from different places. Also the array is made zero-terminated. - * The inner loop (for the case without_key) is almost twice as compact; performance gain of about 30%. - * - * 2. Calling a function by pointer is better than a virtual call, because in the case of a virtual call, - * GCC 5.1.2 generates code that, at each iteration of the loop, reloads the function address from memory into the register - * (the offset value in the virtual function table). - */ - struct AggregateFunctionInstruction - { - const IAggregateFunction * that; - IAggregateFunction::AddFunc func; - size_t state_offset; - const IColumn ** arguments; - const IAggregateFunction * batch_that; - const IColumn ** batch_arguments; - const UInt64 * offsets = nullptr; - }; - - using AggregateFunctionInstructions = std::vector; - Sizes offsets_of_aggregate_states; /// The offset to the n-th aggregate function in a row of aggregate functions. size_t total_size_of_aggregate_states = 0; /// The total size of the row from the aggregate functions. @@ -1105,6 +1109,13 @@ protected: AggregateFunctionInstruction * aggregate_instructions, Arena * arena); + static void executeOnIntervalWithoutKeyImpl( + AggregatedDataWithoutKey & res, + size_t row_begin, + size_t row_end, + AggregateFunctionInstruction * aggregate_instructions, + Arena * arena); + template void writeToTemporaryFileImpl( AggregatedDataVariants & data_variants, @@ -1250,6 +1261,15 @@ protected: * - sets the variable no_more_keys to true. */ bool checkLimits(size_t result_size, bool & no_more_keys) const; + + void fillAggregateColumnsWithSingleKey( + AggregatedDataVariants & data_variants, + MutableColumns & final_aggregate_columns); + + void createStatesAndFillKeyColumnsWithSingleKey( + AggregatedDataVariants & data_variants, + ColumnRawPtrs key_columns, size_t key_row, + MutableColumns & final_key_columns); }; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 9478d25b61a..a02cb97c8c9 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -98,6 +98,7 @@ #include #include #include +#include namespace DB @@ -1615,7 +1616,7 @@ void InterpreterSelectQuery::executeWhere(QueryPipeline & pipeline, const Expres }); } -void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr group_by_info) +void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr /*group_by_info*/) { pipeline.transform([&](auto & stream) { @@ -1635,15 +1636,6 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre const Settings & settings = context->getSettingsRef(); - if (group_by_info) - { - /// TODO optimization :) -// std::cerr << "\n"; -// for (const auto & elem : group_by_info->order_key_prefix_descr) -// std::cerr << elem.column_name << " "; -// std::cerr << "\n"; - } - /** Two-level aggregation is useful in two cases: * 1. Parallel aggregation is done, and the results should be merged in parallel. * 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. @@ -1688,7 +1680,7 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre } -void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr /*group_by_info*/) +void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr group_by_info) { pipeline.addSimpleTransform([&](const Block & header) { @@ -1725,6 +1717,32 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const pipeline.dropTotalsIfHas(); + /// TODO better case determination + if (group_by_info && settings.optimize_aggregation_in_order) + { +// std::cerr << "\n\n"; +// for (const auto & elem : group_by_info->order_key_prefix_descr) +// std::cerr << elem.column_name << " "; +// std::cerr << "\n\n"; + + auto & query = getSelectQuery(); + SortDescription group_by_descr = getSortDescriptionFromGroupBy(query, *context); + + ///TODO Finish sorting first +// UInt64 limit = getLimitForSorting(query, *context); +// executeOrderOptimized(pipeline, group_by_info, limit, group_by_descr); + + pipeline.resize(1); + + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, transform_params, group_by_descr, group_by_descr); + }); + + pipeline.enableQuotaForCurrentStreams(); + return; + } + /// If there are several sources, then we perform parallel aggregation if (pipeline.getNumStreams() > 1) { @@ -2052,6 +2070,45 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, InputSortingInfoP } } +void InterpreterSelectQuery::executeOrderOptimized(QueryPipeline & pipeline, InputSortingInfoPtr input_sorting_info, UInt64 limit, SortDescription & output_order_descr) +{ + const Settings & settings = context->getSettingsRef(); + + bool need_finish_sorting = (input_sorting_info->order_key_prefix_descr.size() < output_order_descr.size()); + std::cerr << "\n Need finish: " << need_finish_sorting << "\n"; + if (pipeline.getNumStreams() > 1) + { + UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit); + auto transform = std::make_shared( + pipeline.getHeader(), + pipeline.getNumStreams(), + input_sorting_info->order_key_prefix_descr, + settings.max_block_size, limit_for_merging); + + pipeline.addPipe({ std::move(transform) }); + } + + pipeline.enableQuotaForCurrentStreams(); + + if (need_finish_sorting) + { + pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr + { + if (stream_type != QueryPipeline::StreamType::Main) + return nullptr; + + return std::make_shared(header, output_order_descr, limit); + }); + + pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr + { + return std::make_shared( + header, input_sorting_info->order_key_prefix_descr, + output_order_descr, settings.max_block_size, limit); + }); + } +} + void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSortingInfoPtr input_sorting_info) { auto & query = getSelectQuery(); @@ -2073,41 +2130,8 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting * and then merge them into one sorted stream. * At this stage we merge per-thread streams into one. */ - - bool need_finish_sorting = (input_sorting_info->order_key_prefix_descr.size() < output_order_descr.size()); - - if (pipeline.getNumStreams() > 1) - { - UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit); - auto transform = std::make_shared( - pipeline.getHeader(), - pipeline.getNumStreams(), - input_sorting_info->order_key_prefix_descr, - settings.max_block_size, limit_for_merging); - - pipeline.addPipe({ std::move(transform) }); - } - - pipeline.enableQuotaForCurrentStreams(); - - if (need_finish_sorting) - { - pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr - { - if (stream_type != QueryPipeline::StreamType::Main) - return nullptr; - - return std::make_shared(header, output_order_descr, limit); - }); - - pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr - { - return std::make_shared( - header, input_sorting_info->order_key_prefix_descr, - output_order_descr, settings.max_block_size, limit); - }); - } - + std::cerr << "\nHello optimized order here!\n"; + executeOrderOptimized(pipeline, input_sorting_info, limit, output_order_descr); return; } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 5954b70cf0f..f97ca42e6a3 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -198,6 +198,7 @@ private: void executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); static void executeExpression(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); void executeOrder(QueryPipeline & pipeline, InputSortingInfoPtr sorting_info); + void executeOrderOptimized(QueryPipeline & pipeline, InputSortingInfoPtr sorting_info, UInt64 limit, SortDescription & sort_description); void executeWithFill(QueryPipeline & pipeline); void executeMergeSorted(QueryPipeline & pipeline); void executePreLimit(QueryPipeline & pipeline, bool do_not_skip_offset); diff --git a/dbms/src/Processors/IProcessor.h b/dbms/src/Processors/IProcessor.h index a613e8008d0..e98ce8723e6 100644 --- a/dbms/src/Processors/IProcessor.h +++ b/dbms/src/Processors/IProcessor.h @@ -158,11 +158,11 @@ public: static std::string statusToName(Status status); - /** Method 'prepare' is responsible for all cheap ("instantenous": O(1) of data volume, no wait) calculations. + /** Method 'prepare' is responsible for all cheap ("instantaneous": O(1) of data volume, no wait) calculations. * * It may access input and output ports, * indicate the need for work by another processor by returning NeedData or PortFull, - * or indicate the absense of work by returning Finished or Unneeded, + * or indicate the absence of work by returning Finished or Unneeded, * it may pull data from input ports and push data to output ports. * * The method is not thread-safe and must be called from a single thread in one moment of time, diff --git a/dbms/src/Processors/ISource.cpp b/dbms/src/Processors/ISource.cpp index 7c620a98a74..e2093c99223 100644 --- a/dbms/src/Processors/ISource.cpp +++ b/dbms/src/Processors/ISource.cpp @@ -58,7 +58,7 @@ void ISource::work() } // { // current_chunk = std::current_exception(); -// has_input = true; +// ready_to_push = true; // got_exception = true; // } } diff --git a/dbms/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/dbms/src/Processors/Transforms/AggregatingInOrderTransform.cpp new file mode 100644 index 00000000000..a70376074ad --- /dev/null +++ b/dbms/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -0,0 +1,236 @@ +#include + +#include + +namespace DB +{ +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +AggregatingInOrderTransform::AggregatingInOrderTransform( + Block header, AggregatingTransformParamsPtr params_, SortDescription & sort_description_, + SortDescription & group_by_description_) + : IProcessor({std::move(header)}, {params_->getHeader()}) + , params(std::move(params_)) + , sort_description(sort_description_) + , group_by_description(group_by_description_) + , key_columns(params->params.keys_size) + , aggregate_columns(params->params.aggregates_size) + , many_data(std::make_shared(1)) + , variants(*many_data->variants[0]) +{ + Block res_header = params->getHeader(); + + /// Replace column names to column position in description_sorted. + for (auto & column_description : group_by_description) + { + if (!column_description.column_name.empty()) + { + column_description.column_number = res_header.getPositionByName(column_description.column_name); + column_description.column_name.clear(); + } + } + + res_key_columns.resize(params->params.keys_size); + res_aggregate_columns.resize(params->params.aggregates_size); + + for (size_t i = 0; i < params->params.keys_size; ++i) + { + /// TODO key_columns have low cardinality removed but res_key_columns not + res_key_columns[i] = res_header.safeGetByPosition(i).type->createColumn(); + } + + for (size_t i = 0; i < params->params.aggregates_size; ++i) + { + res_aggregate_columns[i] = params->aggregator.aggregate_functions[i]->getReturnType()->createColumn(); + } +} + +AggregatingInOrderTransform::~AggregatingInOrderTransform() = default; + +static bool less(const MutableColumns & lhs, const ColumnRawPtrs & rhs, size_t i, size_t j, const SortDescription & descr) +{ + for (const auto & elem : descr) + { + size_t ind = elem.column_number; + int res = elem.direction * lhs[ind]->compareAt(i, j, *rhs[ind], elem.nulls_direction); + if (res < 0) + return true; + else if (res > 0) + return false; + } + return false; +} +/// TODO something broken when there are 10'000'000 rows od data need to investigate +/// TODO maybe move all things inside the Aggregator? + +void AggregatingInOrderTransform::consume(Chunk chunk) +{ + /// Find the position of last already read key in current chunk. + size_t rows = chunk.getNumRows(); + + if (rows == 0) + return; + + size_t mid = 0; + size_t high = 0; + size_t low = -1; + + size_t key_end = 0; + size_t key_begin = 0; + + /// So that key_columns could live longer xD + /// Need a better construction probably + Columns materialized_columns; + + AggregateFunctionInstructions aggregate_function_instructions = + params->aggregator.prepareBlockForAggregation(materialized_columns, chunk.detachColumns(), variants, key_columns, aggregate_columns); + +// std::cerr << "\nPrepared block of size " << rows << "\n"; + + if (!res_block_size) + { +// std::cerr << "\nCreating first state with key " << key_begin << "\n"; + params->aggregator.createStatesAndFillKeyColumnsWithSingleKey(variants, key_columns, key_begin, res_key_columns); + ++res_block_size; + } + + while (key_end != rows) + { + high = rows; + + /// Find the first position of new key in current chunk + while (high - low > 1) + { + mid = (low + high) / 2; +// std::cerr << "Comparing last key and row " << mid << "\n"; + if (!less(res_key_columns, key_columns, res_block_size - 1, mid, group_by_description)) + { + low = mid; + } + else + { + high = mid; + } + } + + key_end = high; + + if (key_begin != key_end) + { +// std::cerr << "Executing from " << key_begin << " to " << key_end << "\n"; + /// Add data to the state if segment is not empty (Empty when we were looking for last key in new block and haven't found it) + params->aggregator.executeOnIntervalWithoutKeyImpl(variants.without_key, key_begin, key_end, aggregate_function_instructions.data(), variants.aggregates_pool); + } + + low = key_begin = key_end; + + if (key_begin != rows) + { +// std::cerr << "\nFinalizing the last state.\n"; + /// We finalize last key aggregation states if a new key found (Not found if high == rows) + params->aggregator.fillAggregateColumnsWithSingleKey(variants, res_aggregate_columns); + +// std::cerr << "\nCreating state with key " << key_begin << "\n"; + /// We create a new state for the new key and update res_key_columns + params->aggregator.createStatesAndFillKeyColumnsWithSingleKey(variants, key_columns, key_begin, res_key_columns); + ++res_block_size; + } + } + +} + +/// Convert block to chunk. +/// Adds additional info about aggregation. +Chunk convertToChunk(const Block & block) +{ + auto info = std::make_shared(); + info->bucket_num = block.info.bucket_num; + info->is_overflows = block.info.is_overflows; + + UInt64 num_rows = block.rows(); + Chunk chunk(block.getColumns(), num_rows); + chunk.setChunkInfo(std::move(info)); + + return chunk; +} + +void AggregatingInOrderTransform::work() +{ + if (is_consume_finished) + { + generate(); + } + else + { + consume(std::move(current_chunk)); + } +} + + +IProcessor::Status AggregatingInOrderTransform::prepare() +{ + auto & output = outputs.front(); + + /// Last output is current. All other outputs should already be closed. + auto & input = inputs.back(); + + /// Check can output. + if (output.isFinished()) + { + input.close(); + return Status::Finished; + } + + if (!output.canPush()) + { + input.setNotNeeded(); + return Status::PortFull; + } + + /// Get chunk from input. + if (input.isFinished() && !is_consume_finished) + { + is_consume_finished = true; + return Status::Ready; + } + + if (is_consume_finished) + { + /// TODO many blocks + output.push(std::move(current_chunk)); + output.finish(); + return Status::Finished; + } + + if (!input.hasData()) + { + input.setNeeded(); + return Status::NeedData; + } + + current_chunk = input.pull(); + return Status::Ready; +} + + +void AggregatingInOrderTransform::generate() +{ +// std::cerr << "\nFinalizing the last state in generate().\n"; + params->aggregator.fillAggregateColumnsWithSingleKey(variants, res_aggregate_columns); + + Block res = params->getHeader().cloneEmpty(); + + for (size_t i = 0; i < res_key_columns.size(); ++i) + res.getByPosition(i).column = std::move(res_key_columns[i]); + + for (size_t i = 0; i < res_aggregate_columns.size(); ++i) + { + res.getByPosition(i + res_key_columns.size()).column = std::move(res_aggregate_columns[i]); + } + current_chunk = convertToChunk(res); +} + +} diff --git a/dbms/src/Processors/Transforms/AggregatingInOrderTransform.h b/dbms/src/Processors/Transforms/AggregatingInOrderTransform.h new file mode 100644 index 00000000000..204091cd867 --- /dev/null +++ b/dbms/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -0,0 +1,50 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +class AggregatingInOrderTransform : public IProcessor +{ + +public: + AggregatingInOrderTransform(Block header, AggregatingTransformParamsPtr params, + SortDescription & sort_description, SortDescription & group_by_description); + + ~AggregatingInOrderTransform() override; + + String getName() const override { return "AggregatingInOrderTransform"; } + + Status prepare() override; + + void work() override; + + void consume(Chunk chunk); + +private: + void generate(); + + size_t res_block_size{}; + MutableColumns res_key_columns; + MutableColumns res_aggregate_columns; + + AggregatingTransformParamsPtr params; + + SortDescription sort_description; + SortDescription group_by_description; + + ColumnRawPtrs key_columns; + Aggregator::AggregateColumns aggregate_columns; + + ManyAggregatedDataPtr many_data; + AggregatedDataVariants & variants; + + bool is_consume_finished = false; + + Chunk current_chunk; +}; + +} diff --git a/dbms/src/Processors/Transforms/FinishSortingTransform.cpp b/dbms/src/Processors/Transforms/FinishSortingTransform.cpp index 4c904eb95a1..6df795de314 100644 --- a/dbms/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/dbms/src/Processors/Transforms/FinishSortingTransform.cpp @@ -27,6 +27,7 @@ FinishSortingTransform::FinishSortingTransform( : SortingTransform(header, description_to_sort_, max_merged_block_size_, limit_) , description_sorted(description_sorted_) { + std::cerr << "Finishing created.\n"; const auto & sample = inputs.front().getHeader(); /// Replace column names to column position in description_sorted. @@ -48,6 +49,8 @@ static bool less(const Columns & lhs, const Columns & rhs, size_t i, size_t j, c { for (const auto & elem : descr) { + std::cerr << elem.column_name << ":" << elem.column_number << " "; + size_t ind = elem.column_number; int res = elem.direction * lhs[ind]->compareAt(i, j, *rhs[ind], elem.nulls_direction); if (res < 0) @@ -55,6 +58,7 @@ static bool less(const Columns & lhs, const Columns & rhs, size_t i, size_t j, c else if (res > 0) return false; } + std::cerr << " ----> equal!"; return false; } @@ -112,7 +116,7 @@ void FinishSortingTransform::consume(Chunk chunk) } } - /// If we reach here, that means that current cunk is first in portion + /// If we reach here, that means that current chunk is first in portion /// or it all consists of rows with the same key as tail of a previous chunk. chunks.push_back(std::move(chunk)); } From 3a18982e2bb1da4bb51386eaaf401582b9715445 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Sat, 18 Apr 2020 13:04:49 +0300 Subject: [PATCH 018/183] removed debug cerr --- src/Interpreters/InterpreterSelectQuery.cpp | 1 - src/Processors/ISource.cpp | 2 +- src/Processors/Transforms/FinishSortingTransform.cpp | 4 ---- 3 files changed, 1 insertion(+), 6 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 7f37bb6cc7e..365ca4699b0 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -2097,7 +2097,6 @@ void InterpreterSelectQuery::executeOrderOptimized(QueryPipeline & pipeline, Inp const Settings & settings = context->getSettingsRef(); bool need_finish_sorting = (input_sorting_info->order_key_prefix_descr.size() < output_order_descr.size()); - std::cerr << "\n Need finish: " << need_finish_sorting << "\n"; if (pipeline.getNumStreams() > 1) { UInt64 limit_for_merging = (need_finish_sorting ? 0 : limit); diff --git a/src/Processors/ISource.cpp b/src/Processors/ISource.cpp index e2093c99223..7c620a98a74 100644 --- a/src/Processors/ISource.cpp +++ b/src/Processors/ISource.cpp @@ -58,7 +58,7 @@ void ISource::work() } // { // current_chunk = std::current_exception(); -// ready_to_push = true; +// has_input = true; // got_exception = true; // } } diff --git a/src/Processors/Transforms/FinishSortingTransform.cpp b/src/Processors/Transforms/FinishSortingTransform.cpp index 6df795de314..b58b008339d 100644 --- a/src/Processors/Transforms/FinishSortingTransform.cpp +++ b/src/Processors/Transforms/FinishSortingTransform.cpp @@ -27,7 +27,6 @@ FinishSortingTransform::FinishSortingTransform( : SortingTransform(header, description_to_sort_, max_merged_block_size_, limit_) , description_sorted(description_sorted_) { - std::cerr << "Finishing created.\n"; const auto & sample = inputs.front().getHeader(); /// Replace column names to column position in description_sorted. @@ -49,8 +48,6 @@ static bool less(const Columns & lhs, const Columns & rhs, size_t i, size_t j, c { for (const auto & elem : descr) { - std::cerr << elem.column_name << ":" << elem.column_number << " "; - size_t ind = elem.column_number; int res = elem.direction * lhs[ind]->compareAt(i, j, *rhs[ind], elem.nulls_direction); if (res < 0) @@ -58,7 +55,6 @@ static bool less(const Columns & lhs, const Columns & rhs, size_t i, size_t j, c else if (res > 0) return false; } - std::cerr << " ----> equal!"; return false; } From 9ed9475e46d1934ed34a25954a5db67b1dbaf5e5 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Thu, 23 Apr 2020 03:55:03 +0300 Subject: [PATCH 019/183] boop the CI --- src/Common/ProfileEvents.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index c7b691977ba..4c4d6e457f1 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -178,7 +178,7 @@ M(OSReadChars, "Number of bytes read from filesystem, including page cache.") \ M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \ \ - M(PERF_COUNT_HW_CPU_CYCLES, "Total cycles. Be wary of what happens during CPU frequency scaling.") \ + M(PERF_COUNT_HW_CPU_CYCLES, "Total cycles. Be wary of what happens during CPU frequency scaling.") \ M(PERF_COUNT_HW_INSTRUCTIONS, "Retired instructions. Be careful, these can be affected by various issues, most notably hardware interrupt counts.") \ M(PERF_COUNT_HW_CACHE_REFERENCES, "Cache accesses. Usually this indicates Last Level Cache accesses but this may vary depending on your CPU. This may include prefetches and coherency messages; again this depends on the design of your CPU.") \ M(PERF_COUNT_HW_CACHE_MISSES, "Cache misses. Usually this indicates Last Level Cache misses; this is intended to be used in con‐junction with the PERF_COUNT_HW_CACHE_REFERENCES event to calculate cache miss rates.") \ From 30e19c3abb9d1a1743536ca6087ba42c12a63931 Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Thu, 23 Apr 2020 21:09:34 +0300 Subject: [PATCH 020/183] Using the same file descriptors for all counters on the current thread (only one instance of `PerfEventsCounters` can be active at a given time for a thread) --- src/Common/ThreadProfileEvents.cpp | 97 ++++++++++++++++++++++-------- src/Common/ThreadProfileEvents.h | 28 +++++++-- 2 files changed, 95 insertions(+), 30 deletions(-) diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index a032e98b076..489b8c91ba6 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -35,7 +35,7 @@ namespace DB } // descriptions' source: http://man7.org/linux/man-pages/man2/perf_event_open.2.html - const PerfEventInfo PerfEventsCounters::perf_raw_events_info[] = { + const PerfEventInfo PerfEventsCounters::raw_events_info[] = { hardwareEvent(PERF_COUNT_HW_CPU_CYCLES, ProfileEvents::PERF_COUNT_HW_CPU_CYCLES), hardwareEvent(PERF_COUNT_HW_INSTRUCTIONS, ProfileEvents::PERF_COUNT_HW_INSTRUCTIONS), hardwareEvent(PERF_COUNT_HW_CACHE_REFERENCES, ProfileEvents::PERF_COUNT_HW_CACHE_REFERENCES), @@ -62,8 +62,11 @@ namespace DB // without requiring a counting event. // softwareEventInfo(PERF_COUNT_SW_DUMMY, ProfileEvents::PERF_COUNT_SW_DUMMY) }; + static_assert(std::size(PerfEventsCounters::raw_events_info) == PerfEventsCounters::NUMBER_OF_RAW_EVENTS); - static_assert(std::size(PerfEventsCounters::perf_raw_events_info) == PerfEventsCounters::NUMBER_OF_RAW_EVENTS); + thread_local PerfDescriptorsHolder PerfEventsCounters::thread_events_descriptors_holder{}; + thread_local bool PerfEventsCounters::thread_events_descriptors_opened = false; + thread_local PerfEventsCounters * PerfEventsCounters::current_thread_counters = nullptr; std::atomic PerfEventsCounters::perf_unavailability_logged = false; std::atomic PerfEventsCounters::particular_events_unavailability_logged = false; @@ -77,7 +80,7 @@ namespace DB { for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) { - const PerfEventInfo & event_info = perf_raw_events_info[i]; + const PerfEventInfo & event_info = raw_events_info[i]; if (event_info.event_type == event_type && event_info.event_config == event_config) return raw_event_values[i]; } @@ -130,10 +133,10 @@ namespace DB event_file_descriptor = openPerfEvent(&pe, /* measure the calling thread */ 0, /* on any cpu */ -1, -1, 0); } - void PerfEventsCounters::initializeProfileEvents(PerfEventsCounters & counters) + bool PerfEventsCounters::initializeThreadLocalEvents(PerfEventsCounters & counters) { - if (counters.perf_events_recording) - return; + if (thread_events_descriptors_opened) + return true; Int32 perf_event_paranoid = 0; bool is_pref_available = getPerfEventParanoid(perf_event_paranoid); @@ -142,7 +145,7 @@ namespace DB bool expected_value = false; if (perf_unavailability_logged.compare_exchange_strong(expected_value, true)) LOG_INFO(getLogger(), "Perf events are unsupported"); - return; + return false; } bool has_cap_sys_admin = hasLinuxCapability(CAP_SYS_ADMIN); @@ -151,7 +154,7 @@ namespace DB bool expected_value = false; if (perf_unavailability_logged.compare_exchange_strong(expected_value, true)) LOG_INFO(getLogger(), "Not enough permissions to record perf events"); - return; + return false; } bool expected = false; @@ -159,29 +162,51 @@ namespace DB for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) { counters.raw_event_values[i] = 0; - const PerfEventInfo & event_info = perf_raw_events_info[i]; - int & fd = counters.events_descriptors[i]; + const PerfEventInfo & event_info = raw_events_info[i]; + int & fd = thread_events_descriptors_holder.descriptors[i]; perfEventOpenDisabled(perf_event_paranoid, has_cap_sys_admin, event_info.event_type, event_info.event_config, fd); if (fd == -1 && log_unsupported_event) { LOG_INFO(getLogger(), "Perf event is unsupported: event_type=" << event_info.event_type - << ", event_config=" << event_info.event_config); + << ", event_config=" << event_info.event_config); } } - for (int fd : counters.events_descriptors) + thread_events_descriptors_opened = true; + return true; + } + + void PerfEventsCounters::initializeProfileEvents(PerfEventsCounters & counters) + { + if (current_thread_counters == &counters) + return; + if (current_thread_counters != nullptr) + { + LOG_WARNING(getLogger(), "Only one instance of `PerfEventsCounters` can be used on the thread"); + return; + } + + if (!initializeThreadLocalEvents(counters)) + return; + + for (Int64 & raw_value : counters.raw_event_values) + raw_value = 0; + + for (int fd : thread_events_descriptors_holder.descriptors) { if (fd != -1) ioctl(fd, PERF_EVENT_IOC_ENABLE, 0); } - counters.perf_events_recording = true; + current_thread_counters = &counters; } void PerfEventsCounters::finalizeProfileEvents(PerfEventsCounters & counters, ProfileEvents::Counters & profile_events) { - if (!counters.perf_events_recording) + if (current_thread_counters != &counters) + return; + if (!thread_events_descriptors_opened) return; // process raw events @@ -189,7 +214,7 @@ namespace DB // only read counters here to have as little overhead for processing as possible for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) { - int fd = counters.events_descriptors[i]; + int fd = counters.thread_events_descriptors_holder.descriptors[i]; if (fd == -1) continue; @@ -201,22 +226,19 @@ namespace DB } } - // actually process counters' values and release resources + // actually process counters' values and stop measuring for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) { - int & fd = counters.events_descriptors[i]; + int fd = counters.thread_events_descriptors_holder.descriptors[i]; if (fd == -1) continue; - profile_events.increment(perf_raw_events_info[i].profile_event, counters.raw_event_values[i]); + profile_events.increment(raw_events_info[i].profile_event, counters.raw_event_values[i]); if (ioctl(fd, PERF_EVENT_IOC_DISABLE, 0)) LOG_WARNING(getLogger(), "Can't disable perf event with file descriptor: " << fd); - if (close(fd)) - LOG_WARNING(getLogger(),"Can't close perf event file descriptor: " << fd - << "; error: " << errno << " - " << strerror(errno)); - - fd = -1; + if (ioctl(fd, PERF_EVENT_IOC_RESET, 0)) + LOG_WARNING(getLogger(), "Can't reset perf event with file descriptor: " << fd); } // process custom events which depend on the raw ones @@ -233,9 +255,36 @@ namespace DB profile_events.increment(ProfileEvents::PERF_CUSTOM_INSTRUCTIONS_PER_CPU_CYCLE_SCALED, instructions_per_cpu_scaled); profile_events.increment(ProfileEvents::PERF_CUSTOM_INSTRUCTIONS_PER_CPU_CYCLE, instructions_per_cpu); - counters.perf_events_recording = false; + current_thread_counters = nullptr; } + Logger * PerfDescriptorsHolder::getLogger() + { + return &Logger::get("PerfDescriptorsHolder"); + } + + PerfDescriptorsHolder::PerfDescriptorsHolder() + { + for (int & descriptor : descriptors) + descriptor = -1; + } + + PerfDescriptorsHolder::~PerfDescriptorsHolder() + { + for (int & descriptor : descriptors) + { + if (descriptor == -1) + continue; + + if (ioctl(descriptor, PERF_EVENT_IOC_DISABLE, 0)) + LOG_WARNING(getLogger(), "Can't disable perf event with file descriptor: " << descriptor); + if (close(descriptor)) + LOG_WARNING(getLogger(),"Can't close perf event file descriptor: " << descriptor + << "; error: " << errno << " - " << strerror(errno)); + + descriptor = -1; + } + } #else void PerfEventsCounters::initializeProfileEvents(PerfEventsCounters &) {} diff --git a/src/Common/ThreadProfileEvents.h b/src/Common/ThreadProfileEvents.h index bfd923175a6..f75218dfa57 100644 --- a/src/Common/ThreadProfileEvents.h +++ b/src/Common/ThreadProfileEvents.h @@ -155,6 +155,8 @@ struct PerfEventInfo #endif +struct PerfDescriptorsHolder; + struct PerfEventsCounters { // cat /proc/sys/kernel/perf_event_paranoid - if perf_event_paranoid is set to 3, all calls to `perf_event_open` are rejected (even for the current process) @@ -173,12 +175,7 @@ struct PerfEventsCounters #if defined(__linux__) static constexpr size_t NUMBER_OF_RAW_EVENTS = 18; - static const PerfEventInfo perf_raw_events_info[]; - - int events_descriptors[NUMBER_OF_RAW_EVENTS]{}; - // temp array just to not create it each time event processing finishes - Int64 raw_event_values[NUMBER_OF_RAW_EVENTS]{}; - bool perf_events_recording = false; + static const PerfEventInfo raw_events_info[]; #endif static void initializeProfileEvents(PerfEventsCounters & counters); @@ -192,14 +189,33 @@ private: // used to write information about particular perf events unavailability only once for all threads static std::atomic particular_events_unavailability_logged; + static thread_local PerfDescriptorsHolder thread_events_descriptors_holder; + static thread_local bool thread_events_descriptors_opened; + static thread_local PerfEventsCounters * current_thread_counters; + + // temp array just to not create it each time event processing finishes + Int64 raw_event_values[NUMBER_OF_RAW_EVENTS]{}; + static Logger * getLogger(); + static bool initializeThreadLocalEvents(PerfEventsCounters & counters); + [[nodiscard]] Int64 getRawValue(int event_type, int event_config) const; #endif }; #if defined(__linux__) +struct PerfDescriptorsHolder { + static Logger * getLogger(); + + int descriptors[PerfEventsCounters::NUMBER_OF_RAW_EVENTS]{}; + + PerfDescriptorsHolder(); + + ~PerfDescriptorsHolder(); +}; + struct TasksStatsCounters { ::taskstats stat; From 30a87a8a58a0a4448564aaaa2f417b6c4828d536 Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Thu, 23 Apr 2020 21:32:24 +0300 Subject: [PATCH 021/183] perf events' values are 64 bit unsigned; so, read them instead of the signed ones --- src/Common/ThreadProfileEvents.cpp | 12 ++++++------ src/Common/ThreadProfileEvents.h | 4 ++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index 489b8c91ba6..97350905bf7 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -76,7 +76,7 @@ namespace DB return &Logger::get("PerfEventsCounters"); } - Int64 PerfEventsCounters::getRawValue(int event_type, int event_config) const + UInt64 PerfEventsCounters::getRawValue(int event_type, int event_config) const { for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) { @@ -190,7 +190,7 @@ namespace DB if (!initializeThreadLocalEvents(counters)) return; - for (Int64 & raw_value : counters.raw_event_values) + for (UInt64 & raw_value : counters.raw_event_values) raw_value = 0; for (int fd : thread_events_descriptors_holder.descriptors) @@ -242,13 +242,13 @@ namespace DB } // process custom events which depend on the raw ones - Int64 hw_cpu_cycles = counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_CPU_CYCLES); - Int64 hw_ref_cpu_cycles = counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_REF_CPU_CYCLES); + UInt64 hw_cpu_cycles = counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_CPU_CYCLES); + UInt64 hw_ref_cpu_cycles = counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_REF_CPU_CYCLES); - Int64 instructions_per_cpu_scaled = hw_cpu_cycles != 0 + UInt64 instructions_per_cpu_scaled = hw_cpu_cycles != 0 ? counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_INSTRUCTIONS) / hw_cpu_cycles : 0; - Int64 instructions_per_cpu = hw_ref_cpu_cycles != 0 + UInt64 instructions_per_cpu = hw_ref_cpu_cycles != 0 ? counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_INSTRUCTIONS) / hw_ref_cpu_cycles : 0; diff --git a/src/Common/ThreadProfileEvents.h b/src/Common/ThreadProfileEvents.h index f75218dfa57..218969668cf 100644 --- a/src/Common/ThreadProfileEvents.h +++ b/src/Common/ThreadProfileEvents.h @@ -194,13 +194,13 @@ private: static thread_local PerfEventsCounters * current_thread_counters; // temp array just to not create it each time event processing finishes - Int64 raw_event_values[NUMBER_OF_RAW_EVENTS]{}; + UInt64 raw_event_values[NUMBER_OF_RAW_EVENTS]{}; static Logger * getLogger(); static bool initializeThreadLocalEvents(PerfEventsCounters & counters); - [[nodiscard]] Int64 getRawValue(int event_type, int event_config) const; + [[nodiscard]] UInt64 getRawValue(int event_type, int event_config) const; #endif }; From 0cf949f1b5f40bea93cb93baf9aa1b25c78b6997 Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Thu, 23 Apr 2020 21:46:19 +0300 Subject: [PATCH 022/183] A bit of reformatting code --- src/Common/ThreadProfileEvents.cpp | 11 ++++++----- src/Common/ThreadProfileEvents.h | 24 ++++++++++++++---------- 2 files changed, 20 insertions(+), 15 deletions(-) diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index 97350905bf7..49ad13d751f 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -258,11 +258,6 @@ namespace DB current_thread_counters = nullptr; } - Logger * PerfDescriptorsHolder::getLogger() - { - return &Logger::get("PerfDescriptorsHolder"); - } - PerfDescriptorsHolder::PerfDescriptorsHolder() { for (int & descriptor : descriptors) @@ -285,6 +280,12 @@ namespace DB descriptor = -1; } } + + Logger * PerfDescriptorsHolder::getLogger() + { + return &Logger::get("PerfDescriptorsHolder"); + } + #else void PerfEventsCounters::initializeProfileEvents(PerfEventsCounters &) {} diff --git a/src/Common/ThreadProfileEvents.h b/src/Common/ThreadProfileEvents.h index 218969668cf..fb2a5ee4b2c 100644 --- a/src/Common/ThreadProfileEvents.h +++ b/src/Common/ThreadProfileEvents.h @@ -153,8 +153,6 @@ struct PerfEventInfo ProfileEvents::Event profile_event; }; -#endif - struct PerfDescriptorsHolder; struct PerfEventsCounters @@ -172,17 +170,14 @@ struct PerfEventsCounters // that restricts perf_event_open() to processes with the CAP_SYS_ADMIN capability // todo: check whether perf_event_open() is available with CAP_SYS_ADMIN -#if defined(__linux__) static constexpr size_t NUMBER_OF_RAW_EVENTS = 18; static const PerfEventInfo raw_events_info[]; -#endif static void initializeProfileEvents(PerfEventsCounters & counters); static void finalizeProfileEvents(PerfEventsCounters & counters, ProfileEvents::Counters & profile_events); -#if defined(__linux__) private: // used to write information about perf unavailability only once for all threads static std::atomic perf_unavailability_logged; @@ -201,21 +196,30 @@ private: static bool initializeThreadLocalEvents(PerfEventsCounters & counters); [[nodiscard]] UInt64 getRawValue(int event_type, int event_config) const; -#endif }; -#if defined(__linux__) - struct PerfDescriptorsHolder { - static Logger * getLogger(); - int descriptors[PerfEventsCounters::NUMBER_OF_RAW_EVENTS]{}; PerfDescriptorsHolder(); ~PerfDescriptorsHolder(); + + static Logger * getLogger(); }; +#else + +struct PerfEventsCounters +{ + static void initializeProfileEvents(PerfEventsCounters & counters); + static void finalizeProfileEvents(PerfEventsCounters & counters, ProfileEvents::Counters & profile_events); +}; + +#endif + +#if defined(__linux__) + struct TasksStatsCounters { ::taskstats stat; From b6a5b1b12fb3829b34ba6d750ed79f1fc06348c4 Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Thu, 23 Apr 2020 22:38:15 +0300 Subject: [PATCH 023/183] Fix styling --- src/Common/ThreadProfileEvents.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Common/ThreadProfileEvents.h b/src/Common/ThreadProfileEvents.h index fb2a5ee4b2c..f08ca6de971 100644 --- a/src/Common/ThreadProfileEvents.h +++ b/src/Common/ThreadProfileEvents.h @@ -198,7 +198,8 @@ private: [[nodiscard]] UInt64 getRawValue(int event_type, int event_config) const; }; -struct PerfDescriptorsHolder { +struct PerfDescriptorsHolder +{ int descriptors[PerfEventsCounters::NUMBER_OF_RAW_EVENTS]{}; PerfDescriptorsHolder(); From e98c23a8cfe4506839083535ec3d5e8f691b53cc Mon Sep 17 00:00:00 2001 From: Dmitry Date: Tue, 5 May 2020 17:35:23 +0300 Subject: [PATCH 024/183] read in order enabled --- src/Core/SortDescription.h | 6 +++ src/Interpreters/Aggregator.h | 2 +- src/Interpreters/ExpressionAnalyzer.cpp | 9 +++- src/Interpreters/InterpreterSelectQuery.cpp | 13 +----- .../AggregatingInOrderTransform.cpp | 16 ++++--- .../Transforms/AggregatingInOrderTransform.h | 2 + .../MergeTree/MergeTreeDataSelectExecutor.cpp | 46 ++++++++++++++++--- src/Storages/ReadInOrderOptimizer.cpp | 12 +++-- src/Storages/SelectQueryInfo.h | 2 +- 9 files changed, 77 insertions(+), 31 deletions(-) diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index e1ec142f645..a16f32b628a 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -57,6 +57,12 @@ struct SortColumnDescription { return !(*this == other); } + + std::string dump() const { + std::stringstream ss; + ss << column_name << ":" << column_number << ":dir " << direction; + return ss.str(); + } }; /// Description of the sorting rule for several columns. diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index fa1bb6e2a85..b69202b3f2d 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1042,7 +1042,7 @@ protected: size_t total_size_of_aggregate_states = 0; /// The total size of the row from the aggregate functions. // add info to track alignment requirement - // If there are states whose alignmentment are v1, ..vn, align_aggregate_states will be max(v1, ... vn) + // If there are states whose alignment are v1, ..vn, align_aggregate_states will be max(v1, ... vn) size_t align_aggregate_states = 1; bool all_aggregates_has_trivial_destructor = false; diff --git a/src/Interpreters/ExpressionAnalyzer.cpp b/src/Interpreters/ExpressionAnalyzer.cpp index 535dc6becdf..4566ea1ea4e 100644 --- a/src/Interpreters/ExpressionAnalyzer.cpp +++ b/src/Interpreters/ExpressionAnalyzer.cpp @@ -747,6 +747,10 @@ bool SelectQueryExpressionAnalyzer::appendGroupBy(ExpressionActionsChain & chain group_by_elements_actions.emplace_back(std::make_shared(all_columns, context)); getRootActions(child, only_types, group_by_elements_actions.back()); } +// std::cerr << "group_by_elements_actions\n"; +// for (const auto & elem : group_by_elements_actions) { +// std::cerr << elem->dumpActions() << "\n"; +// } } return true; @@ -840,8 +844,11 @@ bool SelectQueryExpressionAnalyzer::appendOrderBy(ExpressionActionsChain & chain order_by_elements_actions.emplace_back(std::make_shared(all_columns, context)); getRootActions(child, only_types, order_by_elements_actions.back()); } +// std::cerr << "order_by_elements_actions\n"; +// for (const auto & elem : order_by_elements_actions) { +// std::cerr << elem->dumpActions() << "\n"; +// } } - return true; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a2f530e7be8..5376b1e4d5c 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1744,19 +1744,11 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const /// TODO better case determination if (group_by_info && settings.optimize_aggregation_in_order) { -// std::cerr << "\n\n"; -// for (const auto & elem : group_by_info->order_key_prefix_descr) -// std::cerr << elem.column_name << " "; -// std::cerr << "\n\n"; - auto & query = getSelectQuery(); SortDescription group_by_descr = getSortDescriptionFromGroupBy(query, *context); + UInt64 limit = getLimitForSorting(query, *context); - ///TODO Finish sorting first -// UInt64 limit = getLimitForSorting(query, *context); -// executeOrderOptimized(pipeline, group_by_info, limit, group_by_descr); - - pipeline.resize(1); + executeOrderOptimized(pipeline, group_by_info, limit, group_by_descr); pipeline.addSimpleTransform([&](const Block & header) { @@ -2153,7 +2145,6 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting * and then merge them into one sorted stream. * At this stage we merge per-thread streams into one. */ - std::cerr << "\nHello optimized order here!\n"; executeOrderOptimized(pipeline, input_sorting_info, limit, output_order_descr); return; } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index a70376074ad..e2695457a0c 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -32,7 +32,6 @@ AggregatingInOrderTransform::AggregatingInOrderTransform( column_description.column_name.clear(); } } - res_key_columns.resize(params->params.keys_size); res_aggregate_columns.resize(params->params.aggregates_size); @@ -63,11 +62,14 @@ static bool less(const MutableColumns & lhs, const ColumnRawPtrs & rhs, size_t i } return false; } -/// TODO something broken when there are 10'000'000 rows od data need to investigate + /// TODO maybe move all things inside the Aggregator? void AggregatingInOrderTransform::consume(Chunk chunk) { +// std::cerr << "\nchunk " << x++ << " of size " << chunk.getNumRows() << "\n"; +// sz += chunk.getNumRows(); + /// Find the position of last already read key in current chunk. size_t rows = chunk.getNumRows(); @@ -92,7 +94,7 @@ void AggregatingInOrderTransform::consume(Chunk chunk) if (!res_block_size) { -// std::cerr << "\nCreating first state with key " << key_begin << "\n"; +// std::cerr << "Creating first state with key " << key_begin << "\n"; params->aggregator.createStatesAndFillKeyColumnsWithSingleKey(variants, key_columns, key_begin, res_key_columns); ++res_block_size; } @@ -129,11 +131,11 @@ void AggregatingInOrderTransform::consume(Chunk chunk) if (key_begin != rows) { -// std::cerr << "\nFinalizing the last state.\n"; +// std::cerr << "Finalizing the last state.\n"; /// We finalize last key aggregation states if a new key found (Not found if high == rows) params->aggregator.fillAggregateColumnsWithSingleKey(variants, res_aggregate_columns); -// std::cerr << "\nCreating state with key " << key_begin << "\n"; +// std::cerr << "Creating state with key " << key_begin << "\n"; /// We create a new state for the new key and update res_key_columns params->aggregator.createStatesAndFillKeyColumnsWithSingleKey(variants, key_columns, key_begin, res_key_columns); ++res_block_size; @@ -218,14 +220,16 @@ IProcessor::Status AggregatingInOrderTransform::prepare() void AggregatingInOrderTransform::generate() { +// std::cerr << sz << "\n"; // std::cerr << "\nFinalizing the last state in generate().\n"; params->aggregator.fillAggregateColumnsWithSingleKey(variants, res_aggregate_columns); Block res = params->getHeader().cloneEmpty(); for (size_t i = 0; i < res_key_columns.size(); ++i) + { res.getByPosition(i).column = std::move(res_key_columns[i]); - + } for (size_t i = 0; i < res_aggregate_columns.size(); ++i) { res.getByPosition(i + res_key_columns.size()).column = std::move(res_aggregate_columns[i]); diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index 204091cd867..9a7f8c23133 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -26,6 +26,8 @@ public: private: void generate(); +// size_t x = 1; +// size_t sz = 0; size_t res_block_size{}; MutableColumns res_key_columns; diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 580c95b34dd..76314e823f9 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -646,6 +646,27 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( settings, reader_settings); } + else if (settings.optimize_aggregation_in_order && query_info.group_by_info) + { + size_t prefix_size = query_info.group_by_info->order_key_prefix_descr.size(); + auto order_key_prefix_ast = data.sorting_key_expr_ast->clone(); + order_key_prefix_ast->children.resize(prefix_size); + + auto syntax_result = SyntaxAnalyzer(context).analyze(order_key_prefix_ast, data.getColumns().getAllPhysical()); + auto sorting_key_prefix_expr = ExpressionAnalyzer(order_key_prefix_ast, syntax_result, context).getActions(false); + + res = spreadMarkRangesAmongStreamsWithOrder( + std::move(parts_with_ranges), + num_streams, + column_names_to_read, + max_block_size, + settings.use_uncompressed_cache, + query_info, + sorting_key_prefix_expr, + virt_column_names, + settings, + reader_settings); + } else { res = spreadMarkRangesAmongStreams( @@ -827,6 +848,8 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( { size_t sum_marks = 0; const InputSortingInfoPtr & input_sorting_info = query_info.input_sorting_info; + const InputSortingInfoPtr & group_by_info = query_info.group_by_info; + size_t adaptive_parts = 0; std::vector sum_marks_in_parts(parts.size()); const auto data_settings = data.getSettings(); @@ -969,10 +992,13 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( } parts.emplace_back(part); } + /// TODO Better code + if (group_by_info) + ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, group_by_info->direction); + else + ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_sorting_info->direction); - ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_sorting_info->direction); - - if (input_sorting_info->direction == 1) + if (group_by_info || input_sorting_info->direction == 1) { pipes.emplace_back(std::make_shared( data, part.data_part, max_block_size, settings.preferred_block_size_bytes, @@ -995,9 +1021,17 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( if (pipes.size() > 1) { SortDescription sort_description; - for (size_t j = 0; j < input_sorting_info->order_key_prefix_descr.size(); ++j) - sort_description.emplace_back(data.sorting_key_columns[j], - input_sorting_info->direction, 1); + /// TODO Better code + if (group_by_info) + { + for (size_t j = 0; j < group_by_info->order_key_prefix_descr.size(); ++j) + sort_description.emplace_back(data.sorting_key_columns[j], group_by_info->direction, 1); + } + else + { + for (size_t j = 0; j < input_sorting_info->order_key_prefix_descr.size(); ++j) + sort_description.emplace_back(data.sorting_key_columns[j], input_sorting_info->direction, 1); + } for (auto & pipe : pipes) pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), sorting_key_prefix_expr)); diff --git a/src/Storages/ReadInOrderOptimizer.cpp b/src/Storages/ReadInOrderOptimizer.cpp index 5bbe5be9928..ece90c97ce6 100644 --- a/src/Storages/ReadInOrderOptimizer.cpp +++ b/src/Storages/ReadInOrderOptimizer.cpp @@ -55,7 +55,7 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora int read_direction = required_sort_description.at(0).direction; size_t prefix_size = std::min(required_sort_description.size(), sorting_key_columns.size()); - + std::cerr << "Looking for common prefix\n"; for (size_t i = 0; i < prefix_size; ++i) { if (forbidden_columns.count(required_sort_description[i].column_name)) @@ -72,6 +72,7 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora bool found_function = false; for (const auto & action : elements_actions[i]->getActions()) { + std::cerr << action.toString() << "\n"; if (action.type != ExpressionAction::APPLY_FUNCTION) continue; @@ -82,7 +83,7 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora } else found_function = true; - + std::cerr << "Function was found\n"; if (action.argument_names.size() != 1 || action.argument_names.at(0) != sorting_key_columns[i]) { current_direction = 0; @@ -95,7 +96,7 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora current_direction = 0; break; } - + std::cerr << "Function has info about monotonicity\n"; auto monotonicity = func.getMonotonicityForRange(*func.getArgumentTypes().at(0), {}, {}); if (!monotonicity.is_monotonic) { @@ -104,14 +105,15 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora } else if (!monotonicity.is_positive) current_direction *= -1; + std::cerr << "Function is monotonic\n"; } if (!found_function) current_direction = 0; - + std::cerr << current_direction << " " << read_direction << "\n"; if (!current_direction || (i > 0 && current_direction != read_direction)) break; - + std::cerr << "Adding function\n"; if (i == 0) read_direction = current_direction; diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 3aae218defe..1b08489b2ee 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -80,7 +80,7 @@ struct SelectQueryInfo /// We can modify it while reading from storage mutable InputSortingInfoPtr input_sorting_info; - mutable InputSortingInfoPtr group_by_info; + InputSortingInfoPtr group_by_info; /// Prepared sets are used for indices by storage engine. /// Example: x IN (1, 2, 3) From 0883dd67d3d19681f2fc43c2e425238688ee822a Mon Sep 17 00:00:00 2001 From: Dmitry Date: Thu, 7 May 2020 17:54:15 +0300 Subject: [PATCH 025/183] removed some code duplication --- src/Interpreters/Aggregator.cpp | 102 ++---------------- src/Interpreters/Aggregator.h | 56 +++++----- .../AggregatingInOrderTransform.cpp | 6 +- 3 files changed, 44 insertions(+), 120 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 63d9c11654b..e6b0e0edfe0 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -557,31 +557,13 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re return executeOnBlock(block.getColumns(), num_rows, result, key_columns, aggregate_columns, no_more_keys); } -AggregateFunctionInstructions NO_INLINE Aggregator::prepareBlockForAggregation(Columns & materialized_columns, Columns columns, AggregatedDataVariants & result, - ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns) +void Aggregator::prepareKeysAndInstructions(Columns columns, AggregatedDataVariants & result, ColumnRawPtrs & key_columns, + AggregateColumns & aggregate_columns, Columns & materialized_columns, + AggregateFunctionInstructions & aggregate_functions_instructions) { - /// TODO remove code duplication - - /// `result` will destroy the states of aggregate functions in the destructor - result.aggregator = this; - - /// How to perform the aggregation? - if (result.empty()) - { - result.init(method_chosen); - result.keys_size = params.keys_size; - result.key_sizes = key_sizes; - LOG_TRACE(log, "Aggregation method: " << result.getMethodName()); - } - for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_columns[i].resize(params.aggregates[i].arguments.size()); - /** Constant columns are not supported directly during aggregation. - * To make them work anyway, we materialize them. - */ -// Columns materialized_columns; - /// Remember the columns we will work with for (size_t i = 0; i < params.keys_size; ++i) { @@ -599,7 +581,7 @@ AggregateFunctionInstructions NO_INLINE Aggregator::prepareBlockForAggregation(C } } - AggregateFunctionInstructions aggregate_functions_instructions(params.aggregates_size + 1); + aggregate_functions_instructions.resize(params.aggregates_size + 1); aggregate_functions_instructions[params.aggregates_size].that = nullptr; std::vector> nested_columns_holder; @@ -620,20 +602,20 @@ AggregateFunctionInstructions NO_INLINE Aggregator::prepareBlockForAggregation(C aggregate_functions_instructions[i].arguments = aggregate_columns[i].data(); aggregate_functions_instructions[i].state_offset = offsets_of_aggregate_states[i]; - auto that = aggregate_functions[i]; + auto * that = aggregate_functions[i]; /// Unnest consecutive trailing -State combinators - while (auto func = typeid_cast(that)) + while (const auto * func = typeid_cast(that)) that = func->getNestedFunction().get(); aggregate_functions_instructions[i].that = that; aggregate_functions_instructions[i].func = that->getAddressOfAddFunction(); - if (auto func = typeid_cast(that)) + if (const auto * func = typeid_cast(that)) { /// Unnest consecutive -State combinators before -Array that = func->getNestedFunction().get(); - while (auto nested_func = typeid_cast(that)) + while (const auto * nested_func = typeid_cast(that)) that = nested_func->getNestedFunction().get(); auto [nested_columns, offsets] = checkAndGetNestedArrayOffset(aggregate_columns[i].data(), that->getArgumentTypes().size()); @@ -646,8 +628,6 @@ AggregateFunctionInstructions NO_INLINE Aggregator::prepareBlockForAggregation(C aggregate_functions_instructions[i].batch_that = that; } - - return aggregate_functions_instructions; } bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedDataVariants & result, @@ -671,75 +651,13 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData if (isCancelled()) return true; - for (size_t i = 0; i < params.aggregates_size; ++i) - aggregate_columns[i].resize(params.aggregates[i].arguments.size()); - /** Constant columns are not supported directly during aggregation. * To make them work anyway, we materialize them. */ Columns materialized_columns; + AggregateFunctionInstructions aggregate_functions_instructions; - /// Remember the columns we will work with - for (size_t i = 0; i < params.keys_size; ++i) - { - materialized_columns.push_back(columns.at(params.keys[i])->convertToFullColumnIfConst()); - key_columns[i] = materialized_columns.back().get(); - - if (!result.isLowCardinality()) - { - auto column_no_lc = recursiveRemoveLowCardinality(key_columns[i]->getPtr()); - if (column_no_lc.get() != key_columns[i]) - { - materialized_columns.emplace_back(std::move(column_no_lc)); - key_columns[i] = materialized_columns.back().get(); - } - } - } - - AggregateFunctionInstructions aggregate_functions_instructions(params.aggregates_size + 1); - aggregate_functions_instructions[params.aggregates_size].that = nullptr; - - std::vector> nested_columns_holder; - for (size_t i = 0; i < params.aggregates_size; ++i) - { - for (size_t j = 0; j < aggregate_columns[i].size(); ++j) - { - materialized_columns.push_back(columns.at(params.aggregates[i].arguments[j])->convertToFullColumnIfConst()); - aggregate_columns[i][j] = materialized_columns.back().get(); - - auto column_no_lc = recursiveRemoveLowCardinality(aggregate_columns[i][j]->getPtr()); - if (column_no_lc.get() != aggregate_columns[i][j]) - { - materialized_columns.emplace_back(std::move(column_no_lc)); - aggregate_columns[i][j] = materialized_columns.back().get(); - } - } - - aggregate_functions_instructions[i].arguments = aggregate_columns[i].data(); - aggregate_functions_instructions[i].state_offset = offsets_of_aggregate_states[i]; - auto * that = aggregate_functions[i]; - /// Unnest consecutive trailing -State combinators - while (const auto * func = typeid_cast(that)) - that = func->getNestedFunction().get(); - aggregate_functions_instructions[i].that = that; - aggregate_functions_instructions[i].func = that->getAddressOfAddFunction(); - - if (const auto * func = typeid_cast(that)) - { - /// Unnest consecutive -State combinators before -Array - that = func->getNestedFunction().get(); - while (const auto * nested_func = typeid_cast(that)) - that = nested_func->getNestedFunction().get(); - auto [nested_columns, offsets] = checkAndGetNestedArrayOffset(aggregate_columns[i].data(), that->getArgumentTypes().size()); - nested_columns_holder.push_back(std::move(nested_columns)); - aggregate_functions_instructions[i].batch_arguments = nested_columns_holder.back().data(); - aggregate_functions_instructions[i].offsets = offsets; - } - else - aggregate_functions_instructions[i].batch_arguments = aggregate_columns[i].data(); - - aggregate_functions_instructions[i].batch_that = that; - } + prepareKeysAndInstructions(columns, result, key_columns, aggregate_columns, materialized_columns, aggregate_functions_instructions); if (isCancelled()) return true; diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index b69202b3f2d..083958772f3 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -828,28 +828,6 @@ using AggregatedDataVariantsPtr = std::shared_ptr; using ManyAggregatedDataVariants = std::vector; using ManyAggregatedDataVariantsPtr = std::shared_ptr; -/** This array serves two purposes. - * - * 1. Function arguments are collected side by side, and they do not need to be collected from different places. Also the array is made zero-terminated. - * The inner loop (for the case without_key) is almost twice as compact; performance gain of about 30%. - * - * 2. Calling a function by pointer is better than a virtual call, because in the case of a virtual call, - * GCC 5.1.2 generates code that, at each iteration of the loop, reloads the function address from memory into the register - * (the offset value in the virtual function table). - */ -struct AggregateFunctionInstruction -{ - const IAggregateFunction * that; - IAggregateFunction::AddFunc func; - size_t state_offset; - const IColumn ** arguments; - const IAggregateFunction * batch_that; - const IColumn ** batch_arguments; - const UInt64 * offsets = nullptr; -}; - -using AggregateFunctionInstructions = std::vector; - /** How are "total" values calculated with WITH TOTALS? * (For more details, see TotalsHavingBlockInputStream.) * @@ -954,9 +932,6 @@ public: ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block bool & no_more_keys); - AggregateFunctionInstructions prepareBlockForAggregation(Columns & materialized_columns, Columns columns, AggregatedDataVariants & result, - ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns); - /** Convert the aggregation data structure into a block. * If overflow_row = true, then aggregates for rows that are not included in max_rows_to_group_by are put in the first block. * @@ -1028,7 +1003,6 @@ protected: friend class ConvertingAggregatedToChunksTransform; friend class ConvertingAggregatedToChunksSource; friend class AggregatingInOrderTransform; - Params params; AggregatedDataVariants::Type method_chosen; @@ -1038,6 +1012,28 @@ protected: AggregateFunctionsPlainPtrs aggregate_functions; + /** This array serves two purposes. + * + * 1. Function arguments are collected side by side, and they do not need to be collected from different places. Also the array is made zero-terminated. + * The inner loop (for the case without_key) is almost twice as compact; performance gain of about 30%. + * + * 2. Calling a function by pointer is better than a virtual call, because in the case of a virtual call, + * GCC 5.1.2 generates code that, at each iteration of the loop, reloads the function address from memory into the register + * (the offset value in the virtual function table). + */ + struct AggregateFunctionInstruction + { + const IAggregateFunction * that; + IAggregateFunction::AddFunc func; + size_t state_offset; + const IColumn ** arguments; + const IAggregateFunction * batch_that; + const IColumn ** batch_arguments; + const UInt64 * offsets = nullptr; + }; + + using AggregateFunctionInstructions = std::vector; + Sizes offsets_of_aggregate_states; /// The offset to the n-th aggregate function in a row of aggregate functions. size_t total_size_of_aggregate_states = 0; /// The total size of the row from the aggregate functions. @@ -1262,6 +1258,14 @@ protected: */ bool checkLimits(size_t result_size, bool & no_more_keys) const; + void prepareKeysAndInstructions( + Columns columns, + AggregatedDataVariants & result, + ColumnRawPtrs & key_columns, + AggregateColumns & aggregate_columns, + Columns & materialized_columns, + AggregateFunctionInstructions & instructions); + void fillAggregateColumnsWithSingleKey( AggregatedDataVariants & data_variants, MutableColumns & final_aggregate_columns); diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index e2695457a0c..0a93380f5e0 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -21,6 +21,8 @@ AggregatingInOrderTransform::AggregatingInOrderTransform( , many_data(std::make_shared(1)) , variants(*many_data->variants[0]) { +// std::cerr << "AggregatingInOrderTransform\n"; + Block res_header = params->getHeader(); /// Replace column names to column position in description_sorted. @@ -86,9 +88,9 @@ void AggregatingInOrderTransform::consume(Chunk chunk) /// So that key_columns could live longer xD /// Need a better construction probably Columns materialized_columns; + Aggregator::AggregateFunctionInstructions aggregate_function_instructions; - AggregateFunctionInstructions aggregate_function_instructions = - params->aggregator.prepareBlockForAggregation(materialized_columns, chunk.detachColumns(), variants, key_columns, aggregate_columns); + params->aggregator.prepareKeysAndInstructions(chunk.detachColumns(), variants, key_columns, aggregate_columns, materialized_columns, aggregate_function_instructions); // std::cerr << "\nPrepared block of size " << rows << "\n"; From 7bbb85dbe53da541d94dbd2ccfa75de29ffa528a Mon Sep 17 00:00:00 2001 From: Dmitry Date: Thu, 7 May 2020 18:37:19 +0300 Subject: [PATCH 026/183] small fixes --- src/Core/SortDescription.h | 3 ++- src/Processors/Transforms/AggregatingInOrderTransform.cpp | 8 +++----- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index a16f32b628a..6f42ad48f82 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -58,7 +58,8 @@ struct SortColumnDescription return !(*this == other); } - std::string dump() const { + std::string dump() const + { std::stringstream ss; ss << column_name << ":" << column_number << ":dir " << direction; return ss.str(); diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 0a93380f5e0..a50413fad17 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -4,10 +4,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} AggregatingInOrderTransform::AggregatingInOrderTransform( Block header, AggregatingTransformParamsPtr params_, SortDescription & sort_description_, @@ -224,7 +220,9 @@ void AggregatingInOrderTransform::generate() { // std::cerr << sz << "\n"; // std::cerr << "\nFinalizing the last state in generate().\n"; - params->aggregator.fillAggregateColumnsWithSingleKey(variants, res_aggregate_columns); + + if (res_block_size) + params->aggregator.fillAggregateColumnsWithSingleKey(variants, res_aggregate_columns); Block res = params->getHeader().cloneEmpty(); From 465dfe47fc1095334abcf75bca6388fa0c26721f Mon Sep 17 00:00:00 2001 From: Dmitry Date: Thu, 7 May 2020 23:13:51 +0300 Subject: [PATCH 027/183] fixed faults on LC --- src/Interpreters/Aggregator.cpp | 47 +++++++++---------- src/Interpreters/Aggregator.h | 6 +-- .../AggregatingInOrderTransform.cpp | 18 ++++--- .../Transforms/AggregatingInOrderTransform.h | 2 +- 4 files changed, 37 insertions(+), 36 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index e6b0e0edfe0..d1bb411eb70 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -557,30 +557,12 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re return executeOnBlock(block.getColumns(), num_rows, result, key_columns, aggregate_columns, no_more_keys); } -void Aggregator::prepareKeysAndInstructions(Columns columns, AggregatedDataVariants & result, ColumnRawPtrs & key_columns, - AggregateColumns & aggregate_columns, Columns & materialized_columns, - AggregateFunctionInstructions & aggregate_functions_instructions) +void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns & aggregate_columns, Columns & materialized_columns, + AggregateFunctionInstructions & aggregate_functions_instructions) { for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_columns[i].resize(params.aggregates[i].arguments.size()); - /// Remember the columns we will work with - for (size_t i = 0; i < params.keys_size; ++i) - { - materialized_columns.push_back(columns.at(params.keys[i])->convertToFullColumnIfConst()); - key_columns[i] = materialized_columns.back().get(); - - if (!result.isLowCardinality()) - { - auto column_no_lc = recursiveRemoveLowCardinality(key_columns[i]->getPtr()); - if (column_no_lc.get() != key_columns[i]) - { - materialized_columns.emplace_back(std::move(column_no_lc)); - key_columns[i] = materialized_columns.back().get(); - } - } - } - aggregate_functions_instructions.resize(params.aggregates_size + 1); aggregate_functions_instructions[params.aggregates_size].that = nullptr; @@ -655,9 +637,26 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData * To make them work anyway, we materialize them. */ Columns materialized_columns; - AggregateFunctionInstructions aggregate_functions_instructions; - prepareKeysAndInstructions(columns, result, key_columns, aggregate_columns, materialized_columns, aggregate_functions_instructions); + /// Remember the columns we will work with + for (size_t i = 0; i < params.keys_size; ++i) + { + materialized_columns.push_back(columns.at(params.keys[i])->convertToFullColumnIfConst()); + key_columns[i] = materialized_columns.back().get(); + + if (!result.isLowCardinality()) + { + auto column_no_lc = recursiveRemoveLowCardinality(key_columns[i]->getPtr()); + if (column_no_lc.get() != key_columns[i]) + { + materialized_columns.emplace_back(std::move(column_no_lc)); + key_columns[i] = materialized_columns.back().get(); + } + } + } + + AggregateFunctionInstructions aggregate_functions_instructions; + prepareAggregateInstructions(columns, aggregate_columns, materialized_columns, aggregate_functions_instructions); if (isCancelled()) return true; @@ -1154,7 +1153,7 @@ void Aggregator::fillAggregateColumnsWithSingleKey( void Aggregator::createStatesAndFillKeyColumnsWithSingleKey( AggregatedDataVariants & data_variants, - ColumnRawPtrs key_columns, + Columns key_columns, size_t key_row, MutableColumns & final_key_columns) { @@ -1164,7 +1163,7 @@ void Aggregator::createStatesAndFillKeyColumnsWithSingleKey( for (size_t i = 0; i < params.keys_size; ++i) { - final_key_columns[i]->insertFrom(*key_columns[i], key_row); + final_key_columns[i]->insertFrom(*key_columns[i].get(), key_row); } } diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 083958772f3..117298a749a 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1258,10 +1258,8 @@ protected: */ bool checkLimits(size_t result_size, bool & no_more_keys) const; - void prepareKeysAndInstructions( + void prepareAggregateInstructions( Columns columns, - AggregatedDataVariants & result, - ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, Columns & materialized_columns, AggregateFunctionInstructions & instructions); @@ -1272,7 +1270,7 @@ protected: void createStatesAndFillKeyColumnsWithSingleKey( AggregatedDataVariants & data_variants, - ColumnRawPtrs key_columns, size_t key_row, + Columns key_columns, size_t key_row, MutableColumns & final_key_columns); }; diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index a50413fad17..fc9473bfd6c 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -1,6 +1,5 @@ #include - -#include +#include namespace DB { @@ -12,7 +11,6 @@ AggregatingInOrderTransform::AggregatingInOrderTransform( , params(std::move(params_)) , sort_description(sort_description_) , group_by_description(group_by_description_) - , key_columns(params->params.keys_size) , aggregate_columns(params->params.aggregates_size) , many_data(std::make_shared(1)) , variants(*many_data->variants[0]) @@ -35,7 +33,6 @@ AggregatingInOrderTransform::AggregatingInOrderTransform( for (size_t i = 0; i < params->params.keys_size; ++i) { - /// TODO key_columns have low cardinality removed but res_key_columns not res_key_columns[i] = res_header.safeGetByPosition(i).type->createColumn(); } @@ -47,7 +44,7 @@ AggregatingInOrderTransform::AggregatingInOrderTransform( AggregatingInOrderTransform::~AggregatingInOrderTransform() = default; -static bool less(const MutableColumns & lhs, const ColumnRawPtrs & rhs, size_t i, size_t j, const SortDescription & descr) +static bool less(const MutableColumns & lhs, const Columns & rhs, size_t i, size_t j, const SortDescription & descr) { for (const auto & elem : descr) { @@ -84,9 +81,16 @@ void AggregatingInOrderTransform::consume(Chunk chunk) /// So that key_columns could live longer xD /// Need a better construction probably Columns materialized_columns; - Aggregator::AggregateFunctionInstructions aggregate_function_instructions; - params->aggregator.prepareKeysAndInstructions(chunk.detachColumns(), variants, key_columns, aggregate_columns, materialized_columns, aggregate_function_instructions); + Columns key_columns(params->params.keys_size); + for (size_t i = 0; i < params->params.keys_size; ++i) + { + materialized_columns.push_back(chunk.getColumns().at(params->params.keys[i])->convertToFullColumnIfConst()); + key_columns[i] = materialized_columns.back(); + } + + Aggregator::AggregateFunctionInstructions aggregate_function_instructions; + params->aggregator.prepareAggregateInstructions(chunk.detachColumns(), aggregate_columns, materialized_columns, aggregate_function_instructions); // std::cerr << "\nPrepared block of size " << rows << "\n"; diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index 9a7f8c23133..8afb83232db 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -30,6 +30,7 @@ private: // size_t sz = 0; size_t res_block_size{}; + MutableColumns res_key_columns; MutableColumns res_aggregate_columns; @@ -38,7 +39,6 @@ private: SortDescription sort_description; SortDescription group_by_description; - ColumnRawPtrs key_columns; Aggregator::AggregateColumns aggregate_columns; ManyAggregatedDataPtr many_data; From e7b747b0b71983ea6e807b420d47805728eb4f96 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 8 May 2020 16:13:50 +0300 Subject: [PATCH 028/183] limit fixes + func(primary_key) group by works --- src/Interpreters/InterpreterSelectQuery.cpp | 4 +-- .../AggregatingInOrderTransform.cpp | 2 ++ .../Transforms/AggregatingInOrderTransform.h | 2 ++ .../MergeTree/MergeTreeDataSelectExecutor.cpp | 25 ++++++++++++++++++- src/Storages/ReadInOrderOptimizer.cpp | 6 ----- 5 files changed, 30 insertions(+), 9 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 5376b1e4d5c..a5a409d0f1d 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1746,10 +1746,10 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const { auto & query = getSelectQuery(); SortDescription group_by_descr = getSortDescriptionFromGroupBy(query, *context); - UInt64 limit = getLimitForSorting(query, *context); - executeOrderOptimized(pipeline, group_by_info, limit, group_by_descr); + executeOrderOptimized(pipeline, group_by_info, 0, group_by_descr); + pipeline.resize(1); pipeline.addSimpleTransform([&](const Block & header) { return std::make_shared(header, transform_params, group_by_descr, group_by_descr); diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index fc9473bfd6c..20e623010d4 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -97,6 +97,7 @@ void AggregatingInOrderTransform::consume(Chunk chunk) if (!res_block_size) { // std::cerr << "Creating first state with key " << key_begin << "\n"; + LOG_TRACE(log, "AggregatingInOrder"); params->aggregator.createStatesAndFillKeyColumnsWithSingleKey(variants, key_columns, key_begin, res_key_columns); ++res_block_size; } @@ -228,6 +229,7 @@ void AggregatingInOrderTransform::generate() if (res_block_size) params->aggregator.fillAggregateColumnsWithSingleKey(variants, res_aggregate_columns); + LOG_TRACE(log, "Aggregated"); Block res = params->getHeader().cloneEmpty(); for (size_t i = 0; i < res_key_columns.size(); ++i) diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index 8afb83232db..5928ab97972 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -47,6 +47,8 @@ private: bool is_consume_finished = false; Chunk current_chunk; + + Logger * log = &Logger::get("AggregatingInOrderTransform"); }; } diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 76314e823f9..61c81c9f2b4 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -834,6 +834,14 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreams( return res; } +static ExpressionActionsPtr createProjection(const Pipe & pipe, const MergeTreeData & data) +{ + const auto & header = pipe.getHeader(); + auto projection = std::make_shared(header.getNamesAndTypesList(), data.global_context); + projection->add(ExpressionAction::project(header.getNames())); + return projection; +} + Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( RangesInDataParts && parts, size_t num_streams, @@ -1033,13 +1041,20 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( sort_description.emplace_back(data.sorting_key_columns[j], input_sorting_info->direction, 1); } + /// Project input columns to drop columns from sorting_key_prefix_expr + /// to allow execute the same expression later. + /// NOTE: It may lead to double computation of expression. + auto projection = createProjection(pipes.back(), data); + for (auto & pipe : pipes) pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), sorting_key_prefix_expr)); auto merging_sorted = std::make_shared( pipes.back().getHeader(), pipes.size(), sort_description, max_block_size); - res.emplace_back(std::move(pipes), std::move(merging_sorted)); + Pipe merged(std::move(pipes), std::move(merging_sorted)); + merged.addSimpleTransform(std::make_shared(merged.getHeader(), projection)); + res.emplace_back(std::move(merged)); } else res.emplace_back(std::move(pipes.front())); @@ -1085,6 +1100,10 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( use_uncompressed_cache = false; Pipes pipes; + /// Project input columns to drop columns from sorting_key_expr + /// to allow execute the same expression later. + /// NOTE: It may lead to double computation of expression. + ExpressionActionsPtr projection; for (const auto & part : parts) { @@ -1095,6 +1114,9 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( virt_columns, part.part_index_in_query); Pipe pipe(std::move(source_processor)); + if (!projection) + projection = createProjection(pipe, data); + pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), data.sorting_key_expr)); pipes.emplace_back(std::move(pipe)); } @@ -1167,6 +1189,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal( if (merged_processor) { Pipe pipe(std::move(pipes), std::move(merged_processor)); + pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), projection)); pipes = Pipes(); pipes.emplace_back(std::move(pipe)); } diff --git a/src/Storages/ReadInOrderOptimizer.cpp b/src/Storages/ReadInOrderOptimizer.cpp index ece90c97ce6..e164f1928cf 100644 --- a/src/Storages/ReadInOrderOptimizer.cpp +++ b/src/Storages/ReadInOrderOptimizer.cpp @@ -55,7 +55,6 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora int read_direction = required_sort_description.at(0).direction; size_t prefix_size = std::min(required_sort_description.size(), sorting_key_columns.size()); - std::cerr << "Looking for common prefix\n"; for (size_t i = 0; i < prefix_size; ++i) { if (forbidden_columns.count(required_sort_description[i].column_name)) @@ -83,7 +82,6 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora } else found_function = true; - std::cerr << "Function was found\n"; if (action.argument_names.size() != 1 || action.argument_names.at(0) != sorting_key_columns[i]) { current_direction = 0; @@ -96,7 +94,6 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora current_direction = 0; break; } - std::cerr << "Function has info about monotonicity\n"; auto monotonicity = func.getMonotonicityForRange(*func.getArgumentTypes().at(0), {}, {}); if (!monotonicity.is_monotonic) { @@ -105,15 +102,12 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora } else if (!monotonicity.is_positive) current_direction *= -1; - std::cerr << "Function is monotonic\n"; } if (!found_function) current_direction = 0; - std::cerr << current_direction << " " << read_direction << "\n"; if (!current_direction || (i > 0 && current_direction != read_direction)) break; - std::cerr << "Adding function\n"; if (i == 0) read_direction = current_direction; From 015a3555c6502e5c08f721b9a4c677a058405b5b Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 8 May 2020 16:28:18 +0300 Subject: [PATCH 029/183] small fixes --- src/Interpreters/Aggregator.cpp | 22 +++++++++---------- src/Interpreters/Aggregator.h | 2 +- .../AggregatingInOrderTransform.cpp | 17 +++++--------- 3 files changed, 17 insertions(+), 24 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index d1bb411eb70..bab369a4b15 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -532,6 +532,7 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl( } } + void NO_INLINE Aggregator::executeOnIntervalWithoutKeyImpl( AggregatedDataWithoutKey & res, size_t row_begin, @@ -550,13 +551,6 @@ void NO_INLINE Aggregator::executeOnIntervalWithoutKeyImpl( } -bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & result, - ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, bool & no_more_keys) -{ - UInt64 num_rows = block.rows(); - return executeOnBlock(block.getColumns(), num_rows, result, key_columns, aggregate_columns, no_more_keys); -} - void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns & aggregate_columns, Columns & materialized_columns, AggregateFunctionInstructions & aggregate_functions_instructions) { @@ -585,11 +579,9 @@ void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns aggregate_functions_instructions[i].arguments = aggregate_columns[i].data(); aggregate_functions_instructions[i].state_offset = offsets_of_aggregate_states[i]; auto * that = aggregate_functions[i]; - /// Unnest consecutive trailing -State combinators while (const auto * func = typeid_cast(that)) that = func->getNestedFunction().get(); - aggregate_functions_instructions[i].that = that; aggregate_functions_instructions[i].func = that->getAddressOfAddFunction(); @@ -599,7 +591,6 @@ void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns that = func->getNestedFunction().get(); while (const auto * nested_func = typeid_cast(that)) that = nested_func->getNestedFunction().get(); - auto [nested_columns, offsets] = checkAndGetNestedArrayOffset(aggregate_columns[i].data(), that->getArgumentTypes().size()); nested_columns_holder.push_back(std::move(nested_columns)); aggregate_functions_instructions[i].batch_arguments = nested_columns_holder.back().data(); @@ -612,6 +603,15 @@ void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns } } + +bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & result, + ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, bool & no_more_keys) +{ + UInt64 num_rows = block.rows(); + return executeOnBlock(block.getColumns(), num_rows, result, key_columns, aggregate_columns, no_more_keys); +} + + bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedDataVariants & result, ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, bool & no_more_keys) { @@ -1153,7 +1153,7 @@ void Aggregator::fillAggregateColumnsWithSingleKey( void Aggregator::createStatesAndFillKeyColumnsWithSingleKey( AggregatedDataVariants & data_variants, - Columns key_columns, + Columns & key_columns, size_t key_row, MutableColumns & final_key_columns) { diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index 117298a749a..a7ec9ed11fd 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1270,7 +1270,7 @@ protected: void createStatesAndFillKeyColumnsWithSingleKey( AggregatedDataVariants & data_variants, - Columns key_columns, size_t key_row, + Columns & key_columns, size_t key_row, MutableColumns & final_key_columns); }; diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 20e623010d4..c75aff97938 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -64,24 +64,13 @@ void AggregatingInOrderTransform::consume(Chunk chunk) { // std::cerr << "\nchunk " << x++ << " of size " << chunk.getNumRows() << "\n"; // sz += chunk.getNumRows(); - /// Find the position of last already read key in current chunk. size_t rows = chunk.getNumRows(); if (rows == 0) return; - size_t mid = 0; - size_t high = 0; - size_t low = -1; - - size_t key_end = 0; - size_t key_begin = 0; - - /// So that key_columns could live longer xD - /// Need a better construction probably Columns materialized_columns; - Columns key_columns(params->params.keys_size); for (size_t i = 0; i < params->params.keys_size; ++i) { @@ -92,7 +81,8 @@ void AggregatingInOrderTransform::consume(Chunk chunk) Aggregator::AggregateFunctionInstructions aggregate_function_instructions; params->aggregator.prepareAggregateInstructions(chunk.detachColumns(), aggregate_columns, materialized_columns, aggregate_function_instructions); -// std::cerr << "\nPrepared block of size " << rows << "\n"; + size_t key_end = 0; + size_t key_begin = 0; if (!res_block_size) { @@ -101,6 +91,9 @@ void AggregatingInOrderTransform::consume(Chunk chunk) params->aggregator.createStatesAndFillKeyColumnsWithSingleKey(variants, key_columns, key_begin, res_key_columns); ++res_block_size; } + size_t mid = 0; + size_t high = 0; + size_t low = -1; while (key_end != rows) { From 0286b60ed6ac26fb8caa5509ee7802d77ecd7526 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 8 May 2020 22:46:52 +0300 Subject: [PATCH 030/183] return multiple blocks --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- .../AggregatingInOrderTransform.cpp | 195 +++++++++--------- .../Transforms/AggregatingInOrderTransform.h | 10 +- src/Storages/ReadInOrderOptimizer.cpp | 6 +- 4 files changed, 113 insertions(+), 100 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index a5a409d0f1d..8066a4e4c4a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -1752,7 +1752,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const pipeline.resize(1); pipeline.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, transform_params, group_by_descr, group_by_descr); + return std::make_shared(header, transform_params, group_by_descr, group_by_descr, settings.max_block_size); }); pipeline.enableQuotaForCurrentStreams(); diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index c75aff97938..3030fccc431 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -6,8 +6,9 @@ namespace DB AggregatingInOrderTransform::AggregatingInOrderTransform( Block header, AggregatingTransformParamsPtr params_, SortDescription & sort_description_, - SortDescription & group_by_description_) + SortDescription & group_by_description_, size_t max_block_size_) : IProcessor({std::move(header)}, {params_->getHeader()}) + , max_block_size(max_block_size_) , params(std::move(params_)) , sort_description(sort_description_) , group_by_description(group_by_description_) @@ -15,8 +16,6 @@ AggregatingInOrderTransform::AggregatingInOrderTransform( , many_data(std::make_shared(1)) , variants(*many_data->variants[0]) { -// std::cerr << "AggregatingInOrderTransform\n"; - Block res_header = params->getHeader(); /// Replace column names to column position in description_sorted. @@ -28,18 +27,6 @@ AggregatingInOrderTransform::AggregatingInOrderTransform( column_description.column_name.clear(); } } - res_key_columns.resize(params->params.keys_size); - res_aggregate_columns.resize(params->params.aggregates_size); - - for (size_t i = 0; i < params->params.keys_size; ++i) - { - res_key_columns[i] = res_header.safeGetByPosition(i).type->createColumn(); - } - - for (size_t i = 0; i < params->params.aggregates_size; ++i) - { - res_aggregate_columns[i] = params->aggregator.aggregate_functions[i]->getReturnType()->createColumn(); - } } AggregatingInOrderTransform::~AggregatingInOrderTransform() = default; @@ -58,12 +45,9 @@ static bool less(const MutableColumns & lhs, const Columns & rhs, size_t i, size return false; } -/// TODO maybe move all things inside the Aggregator? void AggregatingInOrderTransform::consume(Chunk chunk) { -// std::cerr << "\nchunk " << x++ << " of size " << chunk.getNumRows() << "\n"; -// sz += chunk.getNumRows(); /// Find the position of last already read key in current chunk. size_t rows = chunk.getNumRows(); @@ -79,15 +63,25 @@ void AggregatingInOrderTransform::consume(Chunk chunk) } Aggregator::AggregateFunctionInstructions aggregate_function_instructions; - params->aggregator.prepareAggregateInstructions(chunk.detachColumns(), aggregate_columns, materialized_columns, aggregate_function_instructions); + params->aggregator.prepareAggregateInstructions(chunk.getColumns(), aggregate_columns, materialized_columns, aggregate_function_instructions); size_t key_end = 0; size_t key_begin = 0; if (!res_block_size) { -// std::cerr << "Creating first state with key " << key_begin << "\n"; - LOG_TRACE(log, "AggregatingInOrder"); + res_key_columns.resize(params->params.keys_size); + res_aggregate_columns.resize(params->params.aggregates_size); + + for (size_t i = 0; i < params->params.keys_size; ++i) + { + res_key_columns[i] = params->getHeader().safeGetByPosition(i).type->createColumn(); + } + + for (size_t i = 0; i < params->params.aggregates_size; ++i) + { + res_aggregate_columns[i] = params->aggregator.aggregate_functions[i]->getReturnType()->createColumn(); + } params->aggregator.createStatesAndFillKeyColumnsWithSingleKey(variants, key_columns, key_begin, res_key_columns); ++res_block_size; } @@ -98,27 +92,19 @@ void AggregatingInOrderTransform::consume(Chunk chunk) while (key_end != rows) { high = rows; - /// Find the first position of new key in current chunk while (high - low > 1) { mid = (low + high) / 2; -// std::cerr << "Comparing last key and row " << mid << "\n"; if (!less(res_key_columns, key_columns, res_block_size - 1, mid, group_by_description)) - { low = mid; - } else - { high = mid; - } } - key_end = high; if (key_begin != key_end) { -// std::cerr << "Executing from " << key_begin << " to " << key_end << "\n"; /// Add data to the state if segment is not empty (Empty when we were looking for last key in new block and haven't found it) params->aggregator.executeOnIntervalWithoutKeyImpl(variants.without_key, key_begin, key_end, aggregate_function_instructions.data(), variants.aggregates_pool); } @@ -127,19 +113,98 @@ void AggregatingInOrderTransform::consume(Chunk chunk) if (key_begin != rows) { -// std::cerr << "Finalizing the last state.\n"; /// We finalize last key aggregation states if a new key found (Not found if high == rows) params->aggregator.fillAggregateColumnsWithSingleKey(variants, res_aggregate_columns); -// std::cerr << "Creating state with key " << key_begin << "\n"; + if (res_block_size == max_block_size) { + Columns source_columns = chunk.detachColumns(); + + for (auto & source_column : source_columns) + source_column = source_column->cut(key_begin, rows - key_begin); + + current_chunk = Chunk(source_columns, rows - key_begin); + block_end_reached = true; + need_generate = true; + res_block_size = 0; + return; + } + /// We create a new state for the new key and update res_key_columns params->aggregator.createStatesAndFillKeyColumnsWithSingleKey(variants, key_columns, key_begin, res_key_columns); ++res_block_size; } } - + block_end_reached = false; } + +void AggregatingInOrderTransform::work() +{ + if (is_consume_finished || need_generate) + { + generate(); + } + else + { + consume(std::move(current_chunk)); + } +} + +/// TODO less complicated +IProcessor::Status AggregatingInOrderTransform::prepare() +{ + auto & output = outputs.front(); + auto & input = inputs.back(); + + /// Check can output. + if (output.isFinished()) + { + input.close(); + return Status::Finished; + } + + if (!output.canPush()) + { + input.setNotNeeded(); + return Status::PortFull; + } + + if (block_end_reached) + { + if (need_generate) + { + return Status::Ready; + } + else + { + output.push(std::move(to_push_chunk)); + return Status::Ready; + } + } + if (!block_end_reached) + { + if (is_consume_finished) + { + output.push(std::move(to_push_chunk)); + output.finish(); + return Status::Finished; + } + if (input.isFinished()) + { + is_consume_finished = true; + return Status::Ready; + } + } + if (!input.hasData()) + { + input.setNeeded(); + return Status::NeedData; + } + current_chunk = input.pull(!is_consume_finished); + return Status::Ready; +} + + /// Convert block to chunk. /// Adds additional info about aggregation. Chunk convertToChunk(const Block & block) @@ -155,71 +220,10 @@ Chunk convertToChunk(const Block & block) return chunk; } -void AggregatingInOrderTransform::work() -{ - if (is_consume_finished) - { - generate(); - } - else - { - consume(std::move(current_chunk)); - } -} - - -IProcessor::Status AggregatingInOrderTransform::prepare() -{ - auto & output = outputs.front(); - - /// Last output is current. All other outputs should already be closed. - auto & input = inputs.back(); - - /// Check can output. - if (output.isFinished()) - { - input.close(); - return Status::Finished; - } - - if (!output.canPush()) - { - input.setNotNeeded(); - return Status::PortFull; - } - - /// Get chunk from input. - if (input.isFinished() && !is_consume_finished) - { - is_consume_finished = true; - return Status::Ready; - } - - if (is_consume_finished) - { - /// TODO many blocks - output.push(std::move(current_chunk)); - output.finish(); - return Status::Finished; - } - - if (!input.hasData()) - { - input.setNeeded(); - return Status::NeedData; - } - - current_chunk = input.pull(); - return Status::Ready; -} - void AggregatingInOrderTransform::generate() { -// std::cerr << sz << "\n"; -// std::cerr << "\nFinalizing the last state in generate().\n"; - - if (res_block_size) + if (res_block_size && is_consume_finished) params->aggregator.fillAggregateColumnsWithSingleKey(variants, res_aggregate_columns); LOG_TRACE(log, "Aggregated"); @@ -233,7 +237,8 @@ void AggregatingInOrderTransform::generate() { res.getByPosition(i + res_key_columns.size()).column = std::move(res_aggregate_columns[i]); } - current_chunk = convertToChunk(res); + to_push_chunk = convertToChunk(res); + need_generate = false; } } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index 5928ab97972..9b919c00bd8 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -11,8 +11,8 @@ class AggregatingInOrderTransform : public IProcessor { public: - AggregatingInOrderTransform(Block header, AggregatingTransformParamsPtr params, - SortDescription & sort_description, SortDescription & group_by_description); + AggregatingInOrderTransform(Block header, AggregatingTransformParamsPtr params, SortDescription & sort_description, + SortDescription & group_by_description, size_t max_block_size); ~AggregatingInOrderTransform() override; @@ -29,7 +29,8 @@ private: // size_t x = 1; // size_t sz = 0; - size_t res_block_size{}; + size_t max_block_size; + size_t res_block_size = 0; MutableColumns res_key_columns; MutableColumns res_aggregate_columns; @@ -44,9 +45,12 @@ private: ManyAggregatedDataPtr many_data; AggregatedDataVariants & variants; + bool need_generate = false; + bool block_end_reached = false; bool is_consume_finished = false; Chunk current_chunk; + Chunk to_push_chunk; Logger * log = &Logger::get("AggregatingInOrderTransform"); }; diff --git a/src/Storages/ReadInOrderOptimizer.cpp b/src/Storages/ReadInOrderOptimizer.cpp index e164f1928cf..5bbe5be9928 100644 --- a/src/Storages/ReadInOrderOptimizer.cpp +++ b/src/Storages/ReadInOrderOptimizer.cpp @@ -55,6 +55,7 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora int read_direction = required_sort_description.at(0).direction; size_t prefix_size = std::min(required_sort_description.size(), sorting_key_columns.size()); + for (size_t i = 0; i < prefix_size; ++i) { if (forbidden_columns.count(required_sort_description[i].column_name)) @@ -71,7 +72,6 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora bool found_function = false; for (const auto & action : elements_actions[i]->getActions()) { - std::cerr << action.toString() << "\n"; if (action.type != ExpressionAction::APPLY_FUNCTION) continue; @@ -82,6 +82,7 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora } else found_function = true; + if (action.argument_names.size() != 1 || action.argument_names.at(0) != sorting_key_columns[i]) { current_direction = 0; @@ -94,6 +95,7 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora current_direction = 0; break; } + auto monotonicity = func.getMonotonicityForRange(*func.getArgumentTypes().at(0), {}, {}); if (!monotonicity.is_monotonic) { @@ -106,8 +108,10 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora if (!found_function) current_direction = 0; + if (!current_direction || (i > 0 && current_direction != read_direction)) break; + if (i == 0) read_direction = current_direction; From 3ec80b5531b76bc3690f29e22afc7f1afcc0c132 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Tue, 12 May 2020 17:50:13 +0300 Subject: [PATCH 031/183] now aggregates in parallel --- src/Core/SortCursor.h | 2 +- src/Core/SortDescription.h | 2 +- src/Interpreters/Aggregator.cpp | 11 +- src/Interpreters/Aggregator.h | 1 + src/Interpreters/InterpreterSelectQuery.cpp | 60 ++++++++-- src/Interpreters/InterpreterSelectQuery.h | 9 +- .../AggregatingInOrderTransform.cpp | 112 ++++++++++++++---- .../Transforms/AggregatingInOrderTransform.h | 39 ++++-- .../Transforms/AggregatingTransform.h | 3 + 9 files changed, 193 insertions(+), 46 deletions(-) diff --git a/src/Core/SortCursor.h b/src/Core/SortCursor.h index edf507f8a1d..4c90cc723bf 100644 --- a/src/Core/SortCursor.h +++ b/src/Core/SortCursor.h @@ -63,7 +63,7 @@ struct SortCursorImpl for (auto & column_desc : desc) { if (!column_desc.column_name.empty()) - throw Exception("SortDesctiption should contain column position if SortCursor was used without header.", + throw Exception("SortDescription should contain column position if SortCursor was used without header.", ErrorCodes::LOGICAL_ERROR); } reset(columns, {}); diff --git a/src/Core/SortDescription.h b/src/Core/SortDescription.h index 6f42ad48f82..935a933008b 100644 --- a/src/Core/SortDescription.h +++ b/src/Core/SortDescription.h @@ -61,7 +61,7 @@ struct SortColumnDescription std::string dump() const { std::stringstream ss; - ss << column_name << ":" << column_number << ":dir " << direction; + ss << column_name << ":" << column_number << ":dir " << direction << "nulls " << nulls_direction; return ss.str(); } }; diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index bab369a4b15..ea6b111287f 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1143,12 +1143,19 @@ void Aggregator::fillAggregateColumnsWithSingleKey( MutableColumns & final_aggregate_columns) { AggregatedDataWithoutKey & data = data_variants.without_key; + AggregateColumnsData aggregate_columns_data(params.aggregates_size); for (size_t i = 0; i < params.aggregates_size; ++i) { - aggregate_functions[i]->insertResultInto(data + offsets_of_aggregate_states[i], *final_aggregate_columns[i]); + ColumnAggregateFunction & column_aggregate_func = assert_cast(*final_aggregate_columns[i]); + for (auto & pool : data_variants.aggregates_pools) + { + column_aggregate_func.addArena(pool); + } + aggregate_columns_data[i] = &column_aggregate_func.getData(); + aggregate_columns_data[i]->push_back(data + offsets_of_aggregate_states[i]); } - destroyWithoutKey(data_variants); + data = nullptr; } void Aggregator::createStatesAndFillKeyColumnsWithSingleKey( diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index a7ec9ed11fd..b28ebe11f39 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1003,6 +1003,7 @@ protected: friend class ConvertingAggregatedToChunksTransform; friend class ConvertingAggregatedToChunksSource; friend class AggregatingInOrderTransform; + Params params; AggregatedDataVariants::Type method_chosen; diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8066a4e4c4a..7a9f1755a78 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -98,6 +98,7 @@ #include #include #include +#include namespace DB @@ -1746,17 +1747,60 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const { auto & query = getSelectQuery(); SortDescription group_by_descr = getSortDescriptionFromGroupBy(query, *context); + bool need_finish_sorting = (group_by_info->order_key_prefix_descr.size() < group_by_descr.size()); - executeOrderOptimized(pipeline, group_by_info, 0, group_by_descr); - - pipeline.resize(1); - pipeline.addSimpleTransform([&](const Block & header) + if (need_finish_sorting) { - return std::make_shared(header, transform_params, group_by_descr, group_by_descr, settings.max_block_size); - }); + /// TOO SLOW + } + else + { + if (pipeline.getNumStreams() > 1) + { + auto many_data = std::make_shared(pipeline.getNumStreams()); + size_t counter = 0; + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, transform_params, group_by_descr, settings.max_block_size, many_data, counter++); + }); - pipeline.enableQuotaForCurrentStreams(); - return; + /// TODO remove code duplication + for (auto & column_description : group_by_descr) + { + if (!column_description.column_name.empty()) + { + column_description.column_number = pipeline.getHeader().getPositionByName(column_description.column_name); + column_description.column_name.clear(); + } + } + + auto transform = std::make_shared( + pipeline.getHeader(), + pipeline.getNumStreams(), + group_by_descr, + settings.max_block_size); + + pipeline.addPipe({ std::move(transform) }); + } + else + { + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, transform_params, group_by_descr, settings.max_block_size); + }); + } + + if (final) + { + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, transform_params); + }); + } + + pipeline.enableQuotaForCurrentStreams(); + return; + } } /// If there are several sources, then we perform parallel aggregation diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index f97ea669561..d1b99c6fbf5 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -173,7 +173,7 @@ private: QueryPipeline & save_context_and_storage); void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter); - void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr sorting_info); + void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr group_by_info); void executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final); void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final); void executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression); @@ -190,15 +190,14 @@ private: void executeExtremes(Pipeline & pipeline); void executeSubqueriesInSetsAndJoins(Pipeline & pipeline, const std::unordered_map & subqueries_for_sets); void executeMergeSorted(Pipeline & pipeline, const SortDescription & sort_description, UInt64 limit); - - void executeWhere(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_fiter); - void executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr sorting_info); + void executeWhere(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter); + void executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr group_by_info); void executeMergeAggregated(QueryPipeline & pipeline, bool overflow_row, bool final); void executeTotalsAndHaving(QueryPipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final); void executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); static void executeExpression(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); void executeOrder(QueryPipeline & pipeline, InputSortingInfoPtr sorting_info); - void executeOrderOptimized(QueryPipeline & pipeline, InputSortingInfoPtr sorting_info, UInt64 limit, SortDescription & sort_description); + void executeOrderOptimized(QueryPipeline & pipeline, InputSortingInfoPtr sorting_info, UInt64 limit, SortDescription & output_order_descr); void executeWithFill(QueryPipeline & pipeline); void executeMergeSorted(QueryPipeline & pipeline); void executePreLimit(QueryPipeline & pipeline, bool do_not_skip_offset); diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 3030fccc431..6ad882b0a54 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -1,22 +1,32 @@ #include +#include #include + namespace DB { AggregatingInOrderTransform::AggregatingInOrderTransform( - Block header, AggregatingTransformParamsPtr params_, SortDescription & sort_description_, - SortDescription & group_by_description_, size_t max_block_size_) - : IProcessor({std::move(header)}, {params_->getHeader()}) - , max_block_size(max_block_size_) + Block header, AggregatingTransformParamsPtr params_, + const SortDescription & group_by_description_, size_t res_block_size_) + : AggregatingInOrderTransform(std::move(header), std::move(params_) + , group_by_description_, res_block_size_, std::make_unique(1), 0) +{ +} + +AggregatingInOrderTransform::AggregatingInOrderTransform( + Block header, AggregatingTransformParamsPtr params_, + const SortDescription & group_by_description_, size_t res_block_size_, + ManyAggregatedDataPtr many_data_, size_t current_variant) + : IProcessor({std::move(header)}, {params_->getHeader(false)}) + , res_block_size(res_block_size_) , params(std::move(params_)) - , sort_description(sort_description_) , group_by_description(group_by_description_) , aggregate_columns(params->params.aggregates_size) - , many_data(std::make_shared(1)) - , variants(*many_data->variants[0]) + , many_data(std::move(many_data_)) + , variants(*many_data->variants[current_variant]) { - Block res_header = params->getHeader(); + res_header = params->getHeader(false); /// Replace column names to column position in description_sorted. for (auto & column_description : group_by_description) @@ -50,7 +60,6 @@ void AggregatingInOrderTransform::consume(Chunk chunk) { /// Find the position of last already read key in current chunk. size_t rows = chunk.getNumRows(); - if (rows == 0) return; @@ -68,22 +77,22 @@ void AggregatingInOrderTransform::consume(Chunk chunk) size_t key_end = 0; size_t key_begin = 0; - if (!res_block_size) + if (!cur_block_size) { res_key_columns.resize(params->params.keys_size); res_aggregate_columns.resize(params->params.aggregates_size); for (size_t i = 0; i < params->params.keys_size; ++i) { - res_key_columns[i] = params->getHeader().safeGetByPosition(i).type->createColumn(); + res_key_columns[i] = res_header.safeGetByPosition(i).type->createColumn(); } for (size_t i = 0; i < params->params.aggregates_size; ++i) { - res_aggregate_columns[i] = params->aggregator.aggregate_functions[i]->getReturnType()->createColumn(); + res_aggregate_columns[i] = res_header.safeGetByPosition(i + params->params.keys_size).type->createColumn(); } params->aggregator.createStatesAndFillKeyColumnsWithSingleKey(variants, key_columns, key_begin, res_key_columns); - ++res_block_size; + ++cur_block_size; } size_t mid = 0; size_t high = 0; @@ -96,7 +105,7 @@ void AggregatingInOrderTransform::consume(Chunk chunk) while (high - low > 1) { mid = (low + high) / 2; - if (!less(res_key_columns, key_columns, res_block_size - 1, mid, group_by_description)) + if (!less(res_key_columns, key_columns, cur_block_size - 1, mid, group_by_description)) low = mid; else high = mid; @@ -116,7 +125,8 @@ void AggregatingInOrderTransform::consume(Chunk chunk) /// We finalize last key aggregation states if a new key found (Not found if high == rows) params->aggregator.fillAggregateColumnsWithSingleKey(variants, res_aggregate_columns); - if (res_block_size == max_block_size) { + if (cur_block_size == res_block_size) + { Columns source_columns = chunk.detachColumns(); for (auto & source_column : source_columns) @@ -125,13 +135,13 @@ void AggregatingInOrderTransform::consume(Chunk chunk) current_chunk = Chunk(source_columns, rows - key_begin); block_end_reached = true; need_generate = true; - res_block_size = 0; + cur_block_size = 0; return; } /// We create a new state for the new key and update res_key_columns params->aggregator.createStatesAndFillKeyColumnsWithSingleKey(variants, key_columns, key_begin, res_key_columns); - ++res_block_size; + ++cur_block_size; } } block_end_reached = false; @@ -150,7 +160,7 @@ void AggregatingInOrderTransform::work() } } -/// TODO less complicated +/// TODO simplify prepare IProcessor::Status AggregatingInOrderTransform::prepare() { auto & output = outputs.front(); @@ -223,11 +233,10 @@ Chunk convertToChunk(const Block & block) void AggregatingInOrderTransform::generate() { - if (res_block_size && is_consume_finished) + if (cur_block_size && is_consume_finished) params->aggregator.fillAggregateColumnsWithSingleKey(variants, res_aggregate_columns); - LOG_TRACE(log, "Aggregated"); - Block res = params->getHeader().cloneEmpty(); + Block res = res_header.cloneEmpty(); for (size_t i = 0; i < res_key_columns.size(); ++i) { @@ -241,4 +250,65 @@ void AggregatingInOrderTransform::generate() need_generate = false; } +FinalizingInOrderTransform::FinalizingInOrderTransform(Block header, AggregatingTransformParamsPtr params_) + : IProcessor({std::move(header)}, {params_->getHeader(true)}) +{ +} + + +FinalizingInOrderTransform::~FinalizingInOrderTransform() = default; + + +void FinalizingInOrderTransform::consume(Chunk chunk) +{ + finalizeChunk(chunk); + current_chunk = std::move(chunk); +} + +void FinalizingInOrderTransform::work() +{ + consume(std::move(current_chunk)); +} + +IProcessor::Status FinalizingInOrderTransform::prepare() +{ + auto & output = outputs.front(); + auto & input = inputs.back(); + + /// Check can output. + if (output.isFinished()) + { + input.close(); + return Status::Finished; + } + + if (!output.canPush()) + { + input.setNotNeeded(); + return Status::PortFull; + } + + if (input.isFinished()) + { + output.push(std::move(current_chunk)); + output.finish(); + return Status::Finished; + } + + if (!current_chunk.empty()) + { + output.push(std::move(current_chunk)); + current_chunk.clear(); + return Status::Ready; + } + + if (!input.hasData()) + { + input.setNeeded(); + return Status::NeedData; + } + current_chunk = input.pull(true); + return Status::Ready; +} + } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index 9b919c00bd8..e26d67b40e6 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -1,8 +1,8 @@ #pragma once +#include #include #include -#include namespace DB { @@ -11,8 +11,12 @@ class AggregatingInOrderTransform : public IProcessor { public: - AggregatingInOrderTransform(Block header, AggregatingTransformParamsPtr params, SortDescription & sort_description, - SortDescription & group_by_description, size_t max_block_size); + AggregatingInOrderTransform(Block header, AggregatingTransformParamsPtr params, + const SortDescription & group_by_description, size_t res_block_size, + ManyAggregatedDataPtr many_data, size_t current_variant); + + AggregatingInOrderTransform(Block header, AggregatingTransformParamsPtr params, + const SortDescription & group_by_description, size_t res_block_size); ~AggregatingInOrderTransform() override; @@ -26,18 +30,15 @@ public: private: void generate(); -// size_t x = 1; -// size_t sz = 0; - size_t max_block_size; - size_t res_block_size = 0; + size_t res_block_size; + size_t cur_block_size = 0; MutableColumns res_key_columns; MutableColumns res_aggregate_columns; AggregatingTransformParamsPtr params; - SortDescription sort_description; SortDescription group_by_description; Aggregator::AggregateColumns aggregate_columns; @@ -49,10 +50,32 @@ private: bool block_end_reached = false; bool is_consume_finished = false; + Block res_header; Chunk current_chunk; Chunk to_push_chunk; Logger * log = &Logger::get("AggregatingInOrderTransform"); }; + +class FinalizingInOrderTransform : public IProcessor +{ +public: + FinalizingInOrderTransform(Block header, AggregatingTransformParamsPtr params); + + ~FinalizingInOrderTransform() override; + + String getName() const override { return "FinalizingInOrderTransform"; } + + /// TODO Simplify prepare + Status prepare() override; + + void work() override; + + void consume(Chunk chunk); + +private: + Chunk current_chunk; + Logger * log = &Logger::get("FinalizingInOrderTransform"); +}; } diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index 9c1e9d4e2db..13cc6944bcc 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -28,6 +28,9 @@ struct AggregatingTransformParams : params(params_), aggregator(params), final(final_) {} Block getHeader() const { return aggregator.getHeader(final); } + + /// TODO remove that logic + Block getHeader(bool final_) const { return aggregator.getHeader(final_); } }; struct ManyAggregatedData From 152a636c232272b28fbae6389dbeab353ee3236a Mon Sep 17 00:00:00 2001 From: Dmitry Date: Wed, 13 May 2020 03:13:01 +0300 Subject: [PATCH 032/183] fix bad merge --- src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 1f0dfe4e1f1..2ef48f180f2 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -1041,11 +1041,6 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( sort_description.emplace_back(data.sorting_key_columns[j], input_sorting_info->direction, 1); } - /// Project input columns to drop columns from sorting_key_prefix_expr - /// to allow execute the same expression later. - /// NOTE: It may lead to double computation of expression. - auto projection = createProjection(pipes.back(), data); - /// Project input columns to drop columns from sorting_key_prefix_expr /// to allow execute the same expression later. /// NOTE: It may lead to double computation of expression. From bbe0245b9d70ab0f596098620635cbb6e37bf1be Mon Sep 17 00:00:00 2001 From: Dmitry Date: Wed, 13 May 2020 16:49:10 +0300 Subject: [PATCH 033/183] changes after review #1 --- src/Interpreters/Aggregator.cpp | 2 +- src/Interpreters/InterpreterSelectQuery.cpp | 48 +++++++------- src/Interpreters/InterpreterSelectQuery.h | 10 +-- .../AggregatingInOrderTransform.cpp | 62 ------------------- .../Transforms/AggregatingInOrderTransform.h | 27 ++++---- .../Transforms/TotalsHavingTransform.h | 3 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 49 +++------------ src/Storages/ReadInOrderOptimizer.cpp | 4 +- src/Storages/ReadInOrderOptimizer.h | 2 +- src/Storages/SelectQueryInfo.h | 17 +++-- src/Storages/StorageBuffer.cpp | 4 +- src/Storages/StorageMaterializedView.cpp | 4 +- src/Storages/StorageMerge.cpp | 8 +-- 13 files changed, 67 insertions(+), 173 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 0cd3f81591f..9bc2d304f32 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -540,7 +540,7 @@ void NO_INLINE Aggregator::executeOnIntervalWithoutKeyImpl( for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) { if (inst->offsets) - inst->batch_that->addBatchSinglePlaceFromInterval(inst->offsets[row_begin], inst->offsets[static_cast(row_end - 1)], res + inst->state_offset, inst->batch_arguments, arena); + inst->batch_that->addBatchSinglePlaceFromInterval(inst->offsets[row_begin], inst->offsets[row_end - 1], res + inst->state_offset, inst->batch_arguments, arena); else inst->batch_that->addBatchSinglePlaceFromInterval(row_begin, row_end, res + inst->state_offset, inst->batch_arguments, arena); } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 816a458c00f..8b4d6e69326 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -831,7 +831,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if (!expressions.second_stage && !expressions.need_aggregate && !expressions.hasHaving()) { if (expressions.has_order_by) - executeOrder(pipeline, query_info.input_sorting_info); + executeOrder(pipeline, query_info.input_order_info); if (expressions.has_order_by && query.limitLength()) executeDistinct(pipeline, false, expressions.selected_columns); @@ -1025,7 +1025,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if (!expressions.first_stage && !expressions.need_aggregate && !(query.group_by_with_totals && !aggregate_final)) executeMergeSorted(pipeline); else /// Otherwise, just sort. - executeOrder(pipeline, query_info.input_sorting_info); + executeOrder(pipeline, query_info.input_order_info); } /** Optimization - if there are several sources and there is LIMIT, then first apply the preliminary LIMIT, @@ -1424,25 +1424,21 @@ void InterpreterSelectQuery::executeFetchColumns( query_info.prewhere_info = prewhere_info; /// Create optimizer with prepared actions. - /// Maybe we will need to calc input_sorting_info later, e.g. while reading from StorageMerge. - if (analysis_result.optimize_read_in_order) + /// Maybe we will need to calc input_order_info later, e.g. while reading from StorageMerge. + if (analysis_result.optimize_read_in_order || analysis_result.optimize_aggregation_in_order) { - query_info.order_by_optimizer = std::make_shared( - analysis_result.order_by_elements_actions, - getSortDescription(query, *context), - query_info.syntax_analyzer_result); - - query_info.input_sorting_info = query_info.order_by_optimizer->getInputOrder(storage); - } - - if (analysis_result.optimize_aggregation_in_order) - { - query_info.group_by_optimizer = std::make_shared( + if (analysis_result.optimize_read_in_order) + query_info.order_optimizer = std::make_shared( + analysis_result.order_by_elements_actions, + getSortDescription(query, *context), + query_info.syntax_analyzer_result); + else + query_info.order_optimizer = std::make_shared( analysis_result.group_by_elements_actions, getSortDescriptionFromGroupBy(query, *context), query_info.syntax_analyzer_result); - query_info.group_by_info = query_info.group_by_optimizer->getInputOrder(storage); + query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage); } @@ -1647,7 +1643,7 @@ void InterpreterSelectQuery::executeWhere(QueryPipeline & pipeline, const Expres }); } -void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr /*group_by_info*/) +void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr /*group_by_info*/) { pipeline.transform([&](auto & stream) { @@ -1711,7 +1707,7 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre } -void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr group_by_info) +void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info) { pipeline.addSimpleTransform([&](const Block & header) { @@ -1801,7 +1797,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const { pipeline.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, transform_params); + return std::make_shared(header, transform_params); }); } @@ -2075,7 +2071,7 @@ void InterpreterSelectQuery::executeExpression(QueryPipeline & pipeline, const E }); } -void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, InputSortingInfoPtr input_sorting_info) +void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, InputOrderInfoPtr input_sorting_info) { auto & query = getSelectQuery(); SortDescription output_order_descr = getSortDescription(query, *context); @@ -2138,7 +2134,7 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, InputSortingInfoP } } -void InterpreterSelectQuery::executeOrderOptimized(QueryPipeline & pipeline, InputSortingInfoPtr input_sorting_info, UInt64 limit, SortDescription & output_order_descr) +void InterpreterSelectQuery::executeOrderOptimized(QueryPipeline & pipeline, InputOrderInfoPtr input_sorting_info, UInt64 limit, SortDescription & output_order_descr) { const Settings & settings = context->getSettingsRef(); @@ -2176,7 +2172,7 @@ void InterpreterSelectQuery::executeOrderOptimized(QueryPipeline & pipeline, Inp } } -void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSortingInfoPtr input_sorting_info) +void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputOrderInfoPtr input_sorting_info) { auto & query = getSelectQuery(); SortDescription output_order_descr = getSortDescription(query, *context); @@ -2649,11 +2645,11 @@ void InterpreterSelectQuery::executeExtremes(QueryPipeline & pipeline) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline, const SubqueriesForSets & subqueries_for_sets) { /// Merge streams to one. Use MergeSorting if data was read in sorted order, Union otherwise. - if (query_info.input_sorting_info) + if (query_info.input_order_info) { if (pipeline.stream_with_non_joined_data) throw Exception("Using read in order optimization, but has stream with non-joined data in pipeline", ErrorCodes::LOGICAL_ERROR); - executeMergeSorted(pipeline, query_info.input_sorting_info->order_key_prefix_descr, 0); + executeMergeSorted(pipeline, query_info.input_order_info->order_key_prefix_descr, 0); } else executeUnion(pipeline, {}); @@ -2664,8 +2660,8 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, const SubqueriesForSets & subqueries_for_sets) { - if (query_info.input_sorting_info) - executeMergeSorted(pipeline, query_info.input_sorting_info->order_key_prefix_descr, 0); + if (query_info.input_order_info) + executeMergeSorted(pipeline, query_info.input_order_info->order_key_prefix_descr, 0); const Settings & settings = context->getSettingsRef(); diff --git a/src/Interpreters/InterpreterSelectQuery.h b/src/Interpreters/InterpreterSelectQuery.h index d1b99c6fbf5..aa4ff0fac12 100644 --- a/src/Interpreters/InterpreterSelectQuery.h +++ b/src/Interpreters/InterpreterSelectQuery.h @@ -173,12 +173,12 @@ private: QueryPipeline & save_context_and_storage); void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter); - void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr group_by_info); + void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info); void executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final); void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final); void executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression); static void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression); - void executeOrder(Pipeline & pipeline, InputSortingInfoPtr sorting_info); + void executeOrder(Pipeline & pipeline, InputOrderInfoPtr sorting_info); void executeWithFill(Pipeline & pipeline); void executeMergeSorted(Pipeline & pipeline); void executePreLimit(Pipeline & pipeline); @@ -191,13 +191,13 @@ private: void executeSubqueriesInSetsAndJoins(Pipeline & pipeline, const std::unordered_map & subqueries_for_sets); void executeMergeSorted(Pipeline & pipeline, const SortDescription & sort_description, UInt64 limit); void executeWhere(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter); - void executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputSortingInfoPtr group_by_info); + void executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info); void executeMergeAggregated(QueryPipeline & pipeline, bool overflow_row, bool final); void executeTotalsAndHaving(QueryPipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final); void executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); static void executeExpression(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); - void executeOrder(QueryPipeline & pipeline, InputSortingInfoPtr sorting_info); - void executeOrderOptimized(QueryPipeline & pipeline, InputSortingInfoPtr sorting_info, UInt64 limit, SortDescription & output_order_descr); + void executeOrder(QueryPipeline & pipeline, InputOrderInfoPtr sorting_info); + void executeOrderOptimized(QueryPipeline & pipeline, InputOrderInfoPtr sorting_info, UInt64 limit, SortDescription & output_order_descr); void executeWithFill(QueryPipeline & pipeline); void executeMergeSorted(QueryPipeline & pipeline); void executePreLimit(QueryPipeline & pipeline, bool do_not_skip_offset); diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index 6ad882b0a54..a7680326fba 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -1,8 +1,6 @@ #include -#include #include - namespace DB { @@ -250,65 +248,5 @@ void AggregatingInOrderTransform::generate() need_generate = false; } -FinalizingInOrderTransform::FinalizingInOrderTransform(Block header, AggregatingTransformParamsPtr params_) - : IProcessor({std::move(header)}, {params_->getHeader(true)}) -{ -} - - -FinalizingInOrderTransform::~FinalizingInOrderTransform() = default; - - -void FinalizingInOrderTransform::consume(Chunk chunk) -{ - finalizeChunk(chunk); - current_chunk = std::move(chunk); -} - -void FinalizingInOrderTransform::work() -{ - consume(std::move(current_chunk)); -} - -IProcessor::Status FinalizingInOrderTransform::prepare() -{ - auto & output = outputs.front(); - auto & input = inputs.back(); - - /// Check can output. - if (output.isFinished()) - { - input.close(); - return Status::Finished; - } - - if (!output.canPush()) - { - input.setNotNeeded(); - return Status::PortFull; - } - - if (input.isFinished()) - { - output.push(std::move(current_chunk)); - output.finish(); - return Status::Finished; - } - - if (!current_chunk.empty()) - { - output.push(std::move(current_chunk)); - current_chunk.clear(); - return Status::Ready; - } - - if (!input.hasData()) - { - input.setNeeded(); - return Status::NeedData; - } - current_chunk = input.pull(true); - return Status::Ready; -} } diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.h b/src/Processors/Transforms/AggregatingInOrderTransform.h index e26d67b40e6..00e6f666ed7 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.h +++ b/src/Processors/Transforms/AggregatingInOrderTransform.h @@ -2,7 +2,9 @@ #include #include +#include #include +#include namespace DB { @@ -58,24 +60,19 @@ private: }; -class FinalizingInOrderTransform : public IProcessor +class FinalizingSimpleTransform : public ISimpleTransform { public: - FinalizingInOrderTransform(Block header, AggregatingTransformParamsPtr params); + FinalizingSimpleTransform(Block header, AggregatingTransformParamsPtr params) + : ISimpleTransform({std::move(header)}, {params->getHeader(true)}, true) {} - ~FinalizingInOrderTransform() override; + void transform(Chunk & chunk) override + { + finalizeChunk(chunk); + } - String getName() const override { return "FinalizingInOrderTransform"; } - - /// TODO Simplify prepare - Status prepare() override; - - void work() override; - - void consume(Chunk chunk); - -private: - Chunk current_chunk; - Logger * log = &Logger::get("FinalizingInOrderTransform"); + String getName() const override { return "FinalizingSimpleTransform"; } }; + + } diff --git a/src/Processors/Transforms/TotalsHavingTransform.h b/src/Processors/Transforms/TotalsHavingTransform.h index b6069da66f3..f16b333ffd4 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.h +++ b/src/Processors/Transforms/TotalsHavingTransform.h @@ -1,5 +1,6 @@ -#include +#pragma once +#include #include namespace DB diff --git a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 2ef48f180f2..6d5164b1d9b 100644 --- a/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -625,30 +625,9 @@ Pipes MergeTreeDataSelectExecutor::readFromParts( settings, reader_settings); } - else if (settings.optimize_read_in_order && query_info.input_sorting_info) + else if ((settings.optimize_read_in_order || settings.optimize_aggregation_in_order) && query_info.input_order_info) { - size_t prefix_size = query_info.input_sorting_info->order_key_prefix_descr.size(); - auto order_key_prefix_ast = data.sorting_key_expr_ast->clone(); - order_key_prefix_ast->children.resize(prefix_size); - - auto syntax_result = SyntaxAnalyzer(context).analyze(order_key_prefix_ast, data.getColumns().getAllPhysical()); - auto sorting_key_prefix_expr = ExpressionAnalyzer(order_key_prefix_ast, syntax_result, context).getActions(false); - - res = spreadMarkRangesAmongStreamsWithOrder( - std::move(parts_with_ranges), - num_streams, - column_names_to_read, - max_block_size, - settings.use_uncompressed_cache, - query_info, - sorting_key_prefix_expr, - virt_column_names, - settings, - reader_settings); - } - else if (settings.optimize_aggregation_in_order && query_info.group_by_info) - { - size_t prefix_size = query_info.group_by_info->order_key_prefix_descr.size(); + size_t prefix_size = query_info.input_order_info->order_key_prefix_descr.size(); auto order_key_prefix_ast = data.sorting_key_expr_ast->clone(); order_key_prefix_ast->children.resize(prefix_size); @@ -855,8 +834,7 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( const MergeTreeReaderSettings & reader_settings) const { size_t sum_marks = 0; - const InputSortingInfoPtr & input_sorting_info = query_info.input_sorting_info; - const InputSortingInfoPtr & group_by_info = query_info.group_by_info; + const InputOrderInfoPtr & input_order_info = query_info.input_order_info; size_t adaptive_parts = 0; std::vector sum_marks_in_parts(parts.size()); @@ -1000,13 +978,9 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( } parts.emplace_back(part); } - /// TODO Better code - if (group_by_info) - ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, group_by_info->direction); - else - ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_sorting_info->direction); + ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction); - if (group_by_info || input_sorting_info->direction == 1) + if (input_order_info->direction == 1) { pipes.emplace_back(std::make_shared( data, part.data_part, max_block_size, settings.preferred_block_size_bytes, @@ -1029,17 +1003,8 @@ Pipes MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsWithOrder( if (pipes.size() > 1) { SortDescription sort_description; - /// TODO Better code - if (group_by_info) - { - for (size_t j = 0; j < group_by_info->order_key_prefix_descr.size(); ++j) - sort_description.emplace_back(data.sorting_key_columns[j], group_by_info->direction, 1); - } - else - { - for (size_t j = 0; j < input_sorting_info->order_key_prefix_descr.size(); ++j) - sort_description.emplace_back(data.sorting_key_columns[j], input_sorting_info->direction, 1); - } + for (size_t j = 0; j < input_order_info->order_key_prefix_descr.size(); ++j) + sort_description.emplace_back(data.sorting_key_columns[j], input_order_info->direction, 1); /// Project input columns to drop columns from sorting_key_prefix_expr /// to allow execute the same expression later. diff --git a/src/Storages/ReadInOrderOptimizer.cpp b/src/Storages/ReadInOrderOptimizer.cpp index 5bbe5be9928..bfdbd7ef557 100644 --- a/src/Storages/ReadInOrderOptimizer.cpp +++ b/src/Storages/ReadInOrderOptimizer.cpp @@ -30,7 +30,7 @@ ReadInOrderOptimizer::ReadInOrderOptimizer( forbidden_columns.insert(elem.first); } -InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & storage) const +InputOrderInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & storage) const { Names sorting_key_columns; if (const auto * merge_tree = dynamic_cast(storage.get())) @@ -122,7 +122,7 @@ InputSortingInfoPtr ReadInOrderOptimizer::getInputOrder(const StoragePtr & stora if (order_key_prefix_descr.empty()) return {}; - return std::make_shared(std::move(order_key_prefix_descr), read_direction); + return std::make_shared(std::move(order_key_prefix_descr), read_direction); } } diff --git a/src/Storages/ReadInOrderOptimizer.h b/src/Storages/ReadInOrderOptimizer.h index 4f69831c49f..de858e8fd92 100644 --- a/src/Storages/ReadInOrderOptimizer.h +++ b/src/Storages/ReadInOrderOptimizer.h @@ -20,7 +20,7 @@ public: const SortDescription & required_sort_description, const SyntaxAnalyzerResultPtr & syntax_result); - InputSortingInfoPtr getInputOrder(const StoragePtr & storage) const; + InputOrderInfoPtr getInputOrder(const StoragePtr & storage) const; private: /// Actions for every element of order expression to analyze functions for monotonicity diff --git a/src/Storages/SelectQueryInfo.h b/src/Storages/SelectQueryInfo.h index 1b08489b2ee..911b04aff8f 100644 --- a/src/Storages/SelectQueryInfo.h +++ b/src/Storages/SelectQueryInfo.h @@ -36,25 +36,25 @@ struct FilterInfo bool do_remove_column = false; }; -struct InputSortingInfo +struct InputOrderInfo { SortDescription order_key_prefix_descr; int direction; - InputSortingInfo(const SortDescription & order_key_prefix_descr_, int direction_) + InputOrderInfo(const SortDescription & order_key_prefix_descr_, int direction_) : order_key_prefix_descr(order_key_prefix_descr_), direction(direction_) {} - bool operator ==(const InputSortingInfo & other) const + bool operator ==(const InputOrderInfo & other) const { return order_key_prefix_descr == other.order_key_prefix_descr && direction == other.direction; } - bool operator !=(const InputSortingInfo & other) const { return !(*this == other); } + bool operator !=(const InputOrderInfo & other) const { return !(*this == other); } }; using PrewhereInfoPtr = std::shared_ptr; using FilterInfoPtr = std::shared_ptr; -using InputSortingInfoPtr = std::shared_ptr; +using InputOrderInfoPtr = std::shared_ptr; struct SyntaxAnalyzerResult; using SyntaxAnalyzerResultPtr = std::shared_ptr; @@ -75,12 +75,9 @@ struct SelectQueryInfo PrewhereInfoPtr prewhere_info; - ReadInOrderOptimizerPtr order_by_optimizer; - ReadInOrderOptimizerPtr group_by_optimizer; - + ReadInOrderOptimizerPtr order_optimizer; /// We can modify it while reading from storage - mutable InputSortingInfoPtr input_sorting_info; - InputSortingInfoPtr group_by_info; + mutable InputOrderInfoPtr input_order_info; /// Prepared sets are used for indices by storage engine. /// Example: x IN (1, 2, 3) diff --git a/src/Storages/StorageBuffer.cpp b/src/Storages/StorageBuffer.cpp index f7563df318a..e1f42e106f8 100644 --- a/src/Storages/StorageBuffer.cpp +++ b/src/Storages/StorageBuffer.cpp @@ -171,8 +171,8 @@ Pipes StorageBuffer::read( if (dst_has_same_structure) { - if (query_info.order_by_optimizer) - query_info.input_sorting_info = query_info.order_by_optimizer->getInputOrder(destination); + if (query_info.order_optimizer) + query_info.input_order_info = query_info.order_optimizer->getInputOrder(destination); /// The destination table has the same structure of the requested columns and we can simply read blocks from there. pipes_from_dst = destination->read(column_names, query_info, context, processed_stage, max_block_size, num_streams); diff --git a/src/Storages/StorageMaterializedView.cpp b/src/Storages/StorageMaterializedView.cpp index a565c8c6260..2439ebbbe0a 100644 --- a/src/Storages/StorageMaterializedView.cpp +++ b/src/Storages/StorageMaterializedView.cpp @@ -180,8 +180,8 @@ Pipes StorageMaterializedView::read( auto lock = storage->lockStructureForShare( false, context.getCurrentQueryId(), context.getSettingsRef().lock_acquire_timeout); - if (query_info.order_by_optimizer) - query_info.input_sorting_info = query_info.order_by_optimizer->getInputOrder(storage); + if (query_info.order_optimizer) + query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage); Pipes pipes = storage->read(column_names, query_info, context, processed_stage, max_block_size, num_streams); diff --git a/src/Storages/StorageMerge.cpp b/src/Storages/StorageMerge.cpp index fb6d88c8d33..4712c7f8f33 100644 --- a/src/Storages/StorageMerge.cpp +++ b/src/Storages/StorageMerge.cpp @@ -177,12 +177,12 @@ Pipes StorageMerge::read( num_streams *= num_streams_multiplier; size_t remaining_streams = num_streams; - InputSortingInfoPtr input_sorting_info; - if (query_info.order_by_optimizer) + InputOrderInfoPtr input_sorting_info; + if (query_info.order_optimizer) { for (auto it = selected_tables.begin(); it != selected_tables.end(); ++it) { - auto current_info = query_info.order_by_optimizer->getInputOrder(std::get<0>(*it)); + auto current_info = query_info.order_optimizer->getInputOrder(std::get<0>(*it)); if (it == selected_tables.begin()) input_sorting_info = current_info; else if (!current_info || (input_sorting_info && *current_info != *input_sorting_info)) @@ -192,7 +192,7 @@ Pipes StorageMerge::read( break; } - query_info.input_sorting_info = input_sorting_info; + query_info.input_order_info = input_sorting_info; } for (const auto & table : selected_tables) From 84be0fe8470ce8e5a194cd1141f6d9ad7ac6ed80 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Wed, 13 May 2020 17:21:22 +0300 Subject: [PATCH 034/183] fixes previous changes --- src/Interpreters/InterpreterSelectQuery.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 8b4d6e69326..82f34e6b2a9 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -959,7 +959,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS executeWhere(pipeline, expressions.before_where, expressions.remove_where_filter); if (expressions.need_aggregate) - executeAggregation(pipeline, expressions.before_aggregation, aggregate_overflow_row, aggregate_final, query_info.group_by_info); + executeAggregation(pipeline, expressions.before_aggregation, aggregate_overflow_row, aggregate_final, query_info.input_order_info); else { executeExpression(pipeline, expressions.before_order_and_select); From 3221c7de85857bdcce2dd1c72ba84148ae161978 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Thu, 14 May 2020 16:56:17 +0300 Subject: [PATCH 035/183] fixes previous changes 2 --- src/Interpreters/InterpreterSelectQuery.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 82f34e6b2a9..f7f502dd4aa 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -959,7 +959,11 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS executeWhere(pipeline, expressions.before_where, expressions.remove_where_filter); if (expressions.need_aggregate) + { executeAggregation(pipeline, expressions.before_aggregation, aggregate_overflow_row, aggregate_final, query_info.input_order_info); + /// We need to reset input order info, so that executeOrder can't use it + query_info.input_order_info.reset(); + } else { executeExpression(pipeline, expressions.before_order_and_select); From bef37517b8b4961b4fdc2844284b43b8ae3d54df Mon Sep 17 00:00:00 2001 From: Dmitry Date: Thu, 14 May 2020 17:20:49 +0300 Subject: [PATCH 036/183] fixes after reivew 2 --- src/Interpreters/Aggregator.cpp | 4 +-- src/Interpreters/InterpreterSelectQuery.cpp | 6 ++-- .../AggregatingInOrderTransform.cpp | 19 +----------- .../Transforms/AggregatingTransform.cpp | 30 +++++++++---------- .../Transforms/AggregatingTransform.h | 2 ++ 5 files changed, 22 insertions(+), 39 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 9bc2d304f32..85b31c033f2 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -1139,7 +1139,6 @@ void Aggregator::fillAggregateColumnsWithSingleKey( MutableColumns & final_aggregate_columns) { AggregatedDataWithoutKey & data = data_variants.without_key; - AggregateColumnsData aggregate_columns_data(params.aggregates_size); for (size_t i = 0; i < params.aggregates_size; ++i) { @@ -1148,8 +1147,7 @@ void Aggregator::fillAggregateColumnsWithSingleKey( { column_aggregate_func.addArena(pool); } - aggregate_columns_data[i] = &column_aggregate_func.getData(); - aggregate_columns_data[i]->push_back(data + offsets_of_aggregate_states[i]); + column_aggregate_func.getData().push_back(data + offsets_of_aggregate_states[i]); } data = nullptr; } diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index f7f502dd4aa..a2ccf7d3a3a 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -657,7 +657,7 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, const Co return order_descr; } -static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query, const Context & /*context*/) +static SortDescription getSortDescriptionFromGroupBy(const ASTSelectQuery & query) { SortDescription order_descr; order_descr.reserve(query.groupBy()->children.size()); @@ -1439,7 +1439,7 @@ void InterpreterSelectQuery::executeFetchColumns( else query_info.order_optimizer = std::make_shared( analysis_result.group_by_elements_actions, - getSortDescriptionFromGroupBy(query, *context), + getSortDescriptionFromGroupBy(query), query_info.syntax_analyzer_result); query_info.input_order_info = query_info.order_optimizer->getInputOrder(storage); @@ -1753,7 +1753,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const if (group_by_info && settings.optimize_aggregation_in_order) { auto & query = getSelectQuery(); - SortDescription group_by_descr = getSortDescriptionFromGroupBy(query, *context); + SortDescription group_by_descr = getSortDescriptionFromGroupBy(query); bool need_finish_sorting = (group_by_info->order_key_prefix_descr.size() < group_by_descr.size()); if (need_finish_sorting) diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index a7680326fba..e8c4029e222 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -189,7 +189,7 @@ IProcessor::Status AggregatingInOrderTransform::prepare() return Status::Ready; } } - if (!block_end_reached) + else { if (is_consume_finished) { @@ -212,23 +212,6 @@ IProcessor::Status AggregatingInOrderTransform::prepare() return Status::Ready; } - -/// Convert block to chunk. -/// Adds additional info about aggregation. -Chunk convertToChunk(const Block & block) -{ - auto info = std::make_shared(); - info->bucket_num = block.info.bucket_num; - info->is_overflows = block.info.is_overflows; - - UInt64 num_rows = block.rows(); - Chunk chunk(block.getColumns(), num_rows); - chunk.setChunkInfo(std::move(info)); - - return chunk; -} - - void AggregatingInOrderTransform::generate() { if (cur_block_size && is_consume_finished) diff --git a/src/Processors/Transforms/AggregatingTransform.cpp b/src/Processors/Transforms/AggregatingTransform.cpp index 134894232e3..f64e4c7b54c 100644 --- a/src/Processors/Transforms/AggregatingTransform.cpp +++ b/src/Processors/Transforms/AggregatingTransform.cpp @@ -20,23 +20,23 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +/// Convert block to chunk. +/// Adds additional info about aggregation. +Chunk convertToChunk(const Block & block) +{ + auto info = std::make_shared(); + info->bucket_num = block.info.bucket_num; + info->is_overflows = block.info.is_overflows; + + UInt64 num_rows = block.rows(); + Chunk chunk(block.getColumns(), num_rows); + chunk.setChunkInfo(std::move(info)); + + return chunk; +} + namespace { - /// Convert block to chunk. - /// Adds additional info about aggregation. - Chunk convertToChunk(const Block & block) - { - auto info = std::make_shared(); - info->bucket_num = block.info.bucket_num; - info->is_overflows = block.info.is_overflows; - - UInt64 num_rows = block.rows(); - Chunk chunk(block.getColumns(), num_rows); - chunk.setChunkInfo(std::move(info)); - - return chunk; - } - const AggregatedChunkInfo * getInfoFromChunk(const Chunk & chunk) { const auto & info = chunk.getChunkInfo(); diff --git a/src/Processors/Transforms/AggregatingTransform.h b/src/Processors/Transforms/AggregatingTransform.h index afda0010c80..a14067a8e18 100644 --- a/src/Processors/Transforms/AggregatingTransform.h +++ b/src/Processors/Transforms/AggregatingTransform.h @@ -120,4 +120,6 @@ private: void initGenerate(); }; +Chunk convertToChunk(const Block & block); + } From a597e62d3b7bfacfbac815734c0bc90748683736 Mon Sep 17 00:00:00 2001 From: Dmitry Date: Fri, 15 May 2020 22:27:18 +0300 Subject: [PATCH 037/183] nested_columns_holder is not destoyed --- src/Interpreters/Aggregator.cpp | 7 +++---- src/Interpreters/Aggregator.h | 4 +++- src/Processors/Transforms/AggregatingInOrderTransform.cpp | 3 ++- 3 files changed, 8 insertions(+), 6 deletions(-) diff --git a/src/Interpreters/Aggregator.cpp b/src/Interpreters/Aggregator.cpp index 85b31c033f2..d44c171d639 100644 --- a/src/Interpreters/Aggregator.cpp +++ b/src/Interpreters/Aggregator.cpp @@ -548,7 +548,7 @@ void NO_INLINE Aggregator::executeOnIntervalWithoutKeyImpl( void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns & aggregate_columns, Columns & materialized_columns, - AggregateFunctionInstructions & aggregate_functions_instructions) + AggregateFunctionInstructions & aggregate_functions_instructions, NestedColumnsHolder & nested_columns_holder) { for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_columns[i].resize(params.aggregates[i].arguments.size()); @@ -556,7 +556,6 @@ void Aggregator::prepareAggregateInstructions(Columns columns, AggregateColumns aggregate_functions_instructions.resize(params.aggregates_size + 1); aggregate_functions_instructions[params.aggregates_size].that = nullptr; - std::vector> nested_columns_holder; for (size_t i = 0; i < params.aggregates_size; ++i) { for (size_t j = 0; j < aggregate_columns[i].size(); ++j) @@ -650,9 +649,9 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData } } } - + NestedColumnsHolder nested_columns_holder; AggregateFunctionInstructions aggregate_functions_instructions; - prepareAggregateInstructions(columns, aggregate_columns, materialized_columns, aggregate_functions_instructions); + prepareAggregateInstructions(columns, aggregate_columns, materialized_columns, aggregate_functions_instructions, nested_columns_holder); if (isCancelled()) return true; diff --git a/src/Interpreters/Aggregator.h b/src/Interpreters/Aggregator.h index b43ed911b5d..6962e019a00 100644 --- a/src/Interpreters/Aggregator.h +++ b/src/Interpreters/Aggregator.h @@ -1034,6 +1034,7 @@ protected: }; using AggregateFunctionInstructions = std::vector; + using NestedColumnsHolder = std::vector>; Sizes offsets_of_aggregate_states; /// The offset to the n-th aggregate function in a row of aggregate functions. size_t total_size_of_aggregate_states = 0; /// The total size of the row from the aggregate functions. @@ -1263,7 +1264,8 @@ protected: Columns columns, AggregateColumns & aggregate_columns, Columns & materialized_columns, - AggregateFunctionInstructions & instructions); + AggregateFunctionInstructions & instructions, + NestedColumnsHolder & nested_columns_holder); void fillAggregateColumnsWithSingleKey( AggregatedDataVariants & data_variants, diff --git a/src/Processors/Transforms/AggregatingInOrderTransform.cpp b/src/Processors/Transforms/AggregatingInOrderTransform.cpp index e8c4029e222..5ebfb740c6b 100644 --- a/src/Processors/Transforms/AggregatingInOrderTransform.cpp +++ b/src/Processors/Transforms/AggregatingInOrderTransform.cpp @@ -69,8 +69,9 @@ void AggregatingInOrderTransform::consume(Chunk chunk) key_columns[i] = materialized_columns.back(); } + Aggregator::NestedColumnsHolder nested_columns_holder; Aggregator::AggregateFunctionInstructions aggregate_function_instructions; - params->aggregator.prepareAggregateInstructions(chunk.getColumns(), aggregate_columns, materialized_columns, aggregate_function_instructions); + params->aggregator.prepareAggregateInstructions(chunk.getColumns(), aggregate_columns, materialized_columns, aggregate_function_instructions, nested_columns_holder); size_t key_end = 0; size_t key_begin = 0; From 619492b1cb877f6c2eb621f513aeddead2658b6c Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Sun, 17 May 2020 15:36:31 +0300 Subject: [PATCH 038/183] Updated modified submodules --- contrib/aws | 2 +- contrib/boost | 2 +- contrib/cctz | 2 +- contrib/libgsasl | 2 +- contrib/libunwind | 2 +- contrib/llvm | 2 +- contrib/lz4 | 2 +- contrib/poco | 2 +- contrib/rapidjson | 2 +- contrib/zstd | 2 +- 10 files changed, 10 insertions(+), 10 deletions(-) diff --git a/contrib/aws b/contrib/aws index 45dd8552d3c..fb5c604525f 160000 --- a/contrib/aws +++ b/contrib/aws @@ -1 +1 @@ -Subproject commit 45dd8552d3c492defca79d2720bcc809e35654da +Subproject commit fb5c604525f5151d75a856462653e7e38b559b79 diff --git a/contrib/boost b/contrib/boost index 86be2aef20b..a04e72c0464 160000 --- a/contrib/boost +++ b/contrib/boost @@ -1 +1 @@ -Subproject commit 86be2aef20bee2356b744e5569eed6eaded85dbe +Subproject commit a04e72c0464f0c31d3384f18f0c0db36a05538e0 diff --git a/contrib/cctz b/contrib/cctz index 4f9776a310f..7a2db4ece6e 160000 --- a/contrib/cctz +++ b/contrib/cctz @@ -1 +1 @@ -Subproject commit 4f9776a310f4952454636363def82c2bf6641d5f +Subproject commit 7a2db4ece6e0f1b246173cbdb62711ae258ee841 diff --git a/contrib/libgsasl b/contrib/libgsasl index 42ef2068704..140fb582505 160000 --- a/contrib/libgsasl +++ b/contrib/libgsasl @@ -1 +1 @@ -Subproject commit 42ef20687042637252e64df1934b6d47771486d1 +Subproject commit 140fb58250588c8323285b75fcf127c4adc33dfa diff --git a/contrib/libunwind b/contrib/libunwind index ede00622ff8..27026ef4a9c 160000 --- a/contrib/libunwind +++ b/contrib/libunwind @@ -1 +1 @@ -Subproject commit ede00622ff8ecb1848ed22187eabbfaf8b4e9307 +Subproject commit 27026ef4a9c6c8cc956d1d131c4d794e24096981 diff --git a/contrib/llvm b/contrib/llvm index 5dab18f4861..3d6c7e91676 160000 --- a/contrib/llvm +++ b/contrib/llvm @@ -1 +1 @@ -Subproject commit 5dab18f4861677548b8f7f6815f49384480ecead +Subproject commit 3d6c7e916760b395908f28a1c885c8334d4fa98b diff --git a/contrib/lz4 b/contrib/lz4 index 3d67671559b..f39b79fb029 160000 --- a/contrib/lz4 +++ b/contrib/lz4 @@ -1 +1 @@ -Subproject commit 3d67671559be723b0912bbee2fcd2eb14783a721 +Subproject commit f39b79fb02962a1cd880bbdecb6dffba4f754a11 diff --git a/contrib/poco b/contrib/poco index ddca76ba495..be2ab90ba5d 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit ddca76ba4956cb57150082394536cc43ff28f6fa +Subproject commit be2ab90ba5dccd46919a116e3fe4fa77bb85063b diff --git a/contrib/rapidjson b/contrib/rapidjson index 01950eb7ace..8f4c021fa2f 160000 --- a/contrib/rapidjson +++ b/contrib/rapidjson @@ -1 +1 @@ -Subproject commit 01950eb7acec78818d68b762efc869bba2420d82 +Subproject commit 8f4c021fa2f1e001d2376095928fc0532adf2ae6 diff --git a/contrib/zstd b/contrib/zstd index 255597502c3..10f0e6993f9 160000 --- a/contrib/zstd +++ b/contrib/zstd @@ -1 +1 @@ -Subproject commit 255597502c3a4ef150abc964e376d4202a8c2929 +Subproject commit 10f0e6993f9d2f682da6d04aa2385b7d53cbb4ee From 1ea5e97d7ae002d24e78ef611b2ec7ce7e4d5a8f Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Sun, 17 May 2020 15:38:42 +0300 Subject: [PATCH 039/183] Removed website/images/feathericons --- website/images/feathericons | 1 - 1 file changed, 1 deletion(-) delete mode 160000 website/images/feathericons diff --git a/website/images/feathericons b/website/images/feathericons deleted file mode 160000 index dca4f121b86..00000000000 --- a/website/images/feathericons +++ /dev/null @@ -1 +0,0 @@ -Subproject commit dca4f121b86577616e90d46ffcd9771942311f71 From d935da98b9d6b0efc3570531fcaa0cda642f22df Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Sun, 17 May 2020 15:40:25 +0300 Subject: [PATCH 040/183] Added contrib/libcpuid --- contrib/libcpuid | 1 + 1 file changed, 1 insertion(+) create mode 160000 contrib/libcpuid diff --git a/contrib/libcpuid b/contrib/libcpuid new file mode 160000 index 00000000000..8db3b8d2d32 --- /dev/null +++ b/contrib/libcpuid @@ -0,0 +1 @@ +Subproject commit 8db3b8d2d32d22437f063ce692a1b9bb15e42d18 From 40ad18be8ec214f63e1745405da530a17bea2264 Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Sun, 17 May 2020 16:56:17 +0300 Subject: [PATCH 041/183] Fixed compilation namespace issue and formatting --- src/Common/ThreadProfileEvents.cpp | 473 +++++++++++++++-------------- 1 file changed, 238 insertions(+), 235 deletions(-) diff --git a/src/Common/ThreadProfileEvents.cpp b/src/Common/ThreadProfileEvents.cpp index 3f55a60ec4a..9c21387e187 100644 --- a/src/Common/ThreadProfileEvents.cpp +++ b/src/Common/ThreadProfileEvents.cpp @@ -55,24 +55,24 @@ TasksStatsCounters::TasksStatsCounters(const UInt64 tid, const MetricsProvider p { switch (provider) { - case MetricsProvider::Netlink: - stats_getter = [metrics_provider = std::make_shared(), tid]() - { - ::taskstats result; - metrics_provider->getStat(result, tid); - return result; - }; - break; - case MetricsProvider::Procfs: - stats_getter = [metrics_provider = std::make_shared(tid)]() - { - ::taskstats result; - metrics_provider->getTaskStats(result); - return result; - }; - break; - case MetricsProvider::None: - ; + case MetricsProvider::Netlink: + stats_getter = [metrics_provider = std::make_shared(), tid]() + { + ::taskstats result; + metrics_provider->getStat(result, tid); + return result; + }; + break; + case MetricsProvider::Procfs: + stats_getter = [metrics_provider = std::make_shared(tid)]() + { + ::taskstats result; + metrics_provider->getTaskStats(result); + return result; + }; + break; + case MetricsProvider::None: + ; } } @@ -110,283 +110,286 @@ void TasksStatsCounters::incrementProfileEvents(const ::taskstats & prev, const profile_events.increment(ProfileEvents::OSReadBytes, safeDiff(prev.read_bytes, curr.read_bytes)); profile_events.increment(ProfileEvents::OSWriteBytes, safeDiff(prev.write_bytes, curr.write_bytes)); } -} static PerfEventInfo softwareEvent(int event_config, ProfileEvents::Event profile_event) - { - return PerfEventInfo +{ + return PerfEventInfo { .event_type = perf_type_id::PERF_TYPE_SOFTWARE, .event_config = event_config, .profile_event = profile_event }; - } +} - static PerfEventInfo hardwareEvent(int event_config, ProfileEvents::Event profile_event) - { - return PerfEventInfo +static PerfEventInfo hardwareEvent(int event_config, ProfileEvents::Event profile_event) +{ + return PerfEventInfo { .event_type = perf_type_id::PERF_TYPE_HARDWARE, .event_config = event_config, .profile_event = profile_event }; - } +} - // descriptions' source: http://man7.org/linux/man-pages/man2/perf_event_open.2.html - const PerfEventInfo PerfEventsCounters::raw_events_info[] = { - hardwareEvent(PERF_COUNT_HW_CPU_CYCLES, ProfileEvents::PERF_COUNT_HW_CPU_CYCLES), - hardwareEvent(PERF_COUNT_HW_INSTRUCTIONS, ProfileEvents::PERF_COUNT_HW_INSTRUCTIONS), - hardwareEvent(PERF_COUNT_HW_CACHE_REFERENCES, ProfileEvents::PERF_COUNT_HW_CACHE_REFERENCES), - hardwareEvent(PERF_COUNT_HW_CACHE_MISSES, ProfileEvents::PERF_COUNT_HW_CACHE_MISSES), - hardwareEvent(PERF_COUNT_HW_BRANCH_INSTRUCTIONS, ProfileEvents::PERF_COUNT_HW_BRANCH_INSTRUCTIONS), - hardwareEvent(PERF_COUNT_HW_BRANCH_MISSES, ProfileEvents::PERF_COUNT_HW_BRANCH_MISSES), - hardwareEvent(PERF_COUNT_HW_BUS_CYCLES, ProfileEvents::PERF_COUNT_HW_BUS_CYCLES), - hardwareEvent(PERF_COUNT_HW_STALLED_CYCLES_FRONTEND, ProfileEvents::PERF_COUNT_HW_STALLED_CYCLES_FRONTEND), - hardwareEvent(PERF_COUNT_HW_STALLED_CYCLES_BACKEND, ProfileEvents::PERF_COUNT_HW_STALLED_CYCLES_BACKEND), - hardwareEvent(PERF_COUNT_HW_REF_CPU_CYCLES, ProfileEvents::PERF_COUNT_HW_REF_CPU_CYCLES), - // This reports the CPU clock, a high-resolution per-CPU timer. - // a bit broken according to this: https://stackoverflow.com/a/56967896 +// descriptions' source: http://man7.org/linux/man-pages/man2/perf_event_open.2.html +const PerfEventInfo PerfEventsCounters::raw_events_info[] = { + hardwareEvent(PERF_COUNT_HW_CPU_CYCLES, ProfileEvents::PERF_COUNT_HW_CPU_CYCLES), + hardwareEvent(PERF_COUNT_HW_INSTRUCTIONS, ProfileEvents::PERF_COUNT_HW_INSTRUCTIONS), + hardwareEvent(PERF_COUNT_HW_CACHE_REFERENCES, ProfileEvents::PERF_COUNT_HW_CACHE_REFERENCES), + hardwareEvent(PERF_COUNT_HW_CACHE_MISSES, ProfileEvents::PERF_COUNT_HW_CACHE_MISSES), + hardwareEvent(PERF_COUNT_HW_BRANCH_INSTRUCTIONS, ProfileEvents::PERF_COUNT_HW_BRANCH_INSTRUCTIONS), + hardwareEvent(PERF_COUNT_HW_BRANCH_MISSES, ProfileEvents::PERF_COUNT_HW_BRANCH_MISSES), + hardwareEvent(PERF_COUNT_HW_BUS_CYCLES, ProfileEvents::PERF_COUNT_HW_BUS_CYCLES), + hardwareEvent(PERF_COUNT_HW_STALLED_CYCLES_FRONTEND, ProfileEvents::PERF_COUNT_HW_STALLED_CYCLES_FRONTEND), + hardwareEvent(PERF_COUNT_HW_STALLED_CYCLES_BACKEND, ProfileEvents::PERF_COUNT_HW_STALLED_CYCLES_BACKEND), + hardwareEvent(PERF_COUNT_HW_REF_CPU_CYCLES, ProfileEvents::PERF_COUNT_HW_REF_CPU_CYCLES), + // This reports the CPU clock, a high-resolution per-CPU timer. + // a bit broken according to this: https://stackoverflow.com/a/56967896 // softwareEvent(PERF_COUNT_SW_CPU_CLOCK, ProfileEvents::PERF_COUNT_SW_CPU_CLOCK), - softwareEvent(PERF_COUNT_SW_TASK_CLOCK, ProfileEvents::PERF_COUNT_SW_TASK_CLOCK), - softwareEvent(PERF_COUNT_SW_PAGE_FAULTS, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS), - softwareEvent(PERF_COUNT_SW_CONTEXT_SWITCHES, ProfileEvents::PERF_COUNT_SW_CONTEXT_SWITCHES), - softwareEvent(PERF_COUNT_SW_CPU_MIGRATIONS, ProfileEvents::PERF_COUNT_SW_CPU_MIGRATIONS), - softwareEvent(PERF_COUNT_SW_PAGE_FAULTS_MIN, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MIN), - softwareEvent(PERF_COUNT_SW_PAGE_FAULTS_MAJ, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MAJ), - softwareEvent(PERF_COUNT_SW_ALIGNMENT_FAULTS, ProfileEvents::PERF_COUNT_SW_ALIGNMENT_FAULTS), - softwareEvent(PERF_COUNT_SW_EMULATION_FAULTS, ProfileEvents::PERF_COUNT_SW_EMULATION_FAULTS) - // This is a placeholder event that counts nothing. Informational sample record types such as mmap or - // comm must be associated with an active event. This dummy event allows gathering such records - // without requiring a counting event. + softwareEvent(PERF_COUNT_SW_TASK_CLOCK, ProfileEvents::PERF_COUNT_SW_TASK_CLOCK), + softwareEvent(PERF_COUNT_SW_PAGE_FAULTS, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS), + softwareEvent(PERF_COUNT_SW_CONTEXT_SWITCHES, ProfileEvents::PERF_COUNT_SW_CONTEXT_SWITCHES), + softwareEvent(PERF_COUNT_SW_CPU_MIGRATIONS, ProfileEvents::PERF_COUNT_SW_CPU_MIGRATIONS), + softwareEvent(PERF_COUNT_SW_PAGE_FAULTS_MIN, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MIN), + softwareEvent(PERF_COUNT_SW_PAGE_FAULTS_MAJ, ProfileEvents::PERF_COUNT_SW_PAGE_FAULTS_MAJ), + softwareEvent(PERF_COUNT_SW_ALIGNMENT_FAULTS, ProfileEvents::PERF_COUNT_SW_ALIGNMENT_FAULTS), + softwareEvent(PERF_COUNT_SW_EMULATION_FAULTS, ProfileEvents::PERF_COUNT_SW_EMULATION_FAULTS) + // This is a placeholder event that counts nothing. Informational sample record types such as mmap or + // comm must be associated with an active event. This dummy event allows gathering such records + // without requiring a counting event. // softwareEventInfo(PERF_COUNT_SW_DUMMY, ProfileEvents::PERF_COUNT_SW_DUMMY) - }; - static_assert(std::size(PerfEventsCounters::raw_events_info) == PerfEventsCounters::NUMBER_OF_RAW_EVENTS); +}; +static_assert(std::size(PerfEventsCounters::raw_events_info) == PerfEventsCounters::NUMBER_OF_RAW_EVENTS); - thread_local PerfDescriptorsHolder PerfEventsCounters::thread_events_descriptors_holder{}; - thread_local bool PerfEventsCounters::thread_events_descriptors_opened = false; - thread_local PerfEventsCounters * PerfEventsCounters::current_thread_counters = nullptr; +thread_local PerfDescriptorsHolder PerfEventsCounters::thread_events_descriptors_holder{}; +thread_local bool PerfEventsCounters::thread_events_descriptors_opened = false; +thread_local PerfEventsCounters * PerfEventsCounters::current_thread_counters = nullptr; - std::atomic PerfEventsCounters::perf_unavailability_logged = false; - std::atomic PerfEventsCounters::particular_events_unavailability_logged = false; +std::atomic PerfEventsCounters::perf_unavailability_logged = false; +std::atomic PerfEventsCounters::particular_events_unavailability_logged = false; - Logger * PerfEventsCounters::getLogger() +Logger * PerfEventsCounters::getLogger() +{ + return &Logger::get("PerfEventsCounters"); +} + +UInt64 PerfEventsCounters::getRawValue(int event_type, int event_config) const +{ + for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) { - return &Logger::get("PerfEventsCounters"); + const PerfEventInfo & event_info = raw_events_info[i]; + if (event_info.event_type == event_type && event_info.event_config == event_config) + return raw_event_values[i]; } - UInt64 PerfEventsCounters::getRawValue(int event_type, int event_config) const - { - for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) - { - const PerfEventInfo & event_info = raw_events_info[i]; - if (event_info.event_type == event_type && event_info.event_config == event_config) - return raw_event_values[i]; - } + LOG_WARNING(getLogger(), "Can't find perf event info for event_type=" << event_type << ", event_config=" << event_config); + return 0; +} - LOG_WARNING(getLogger(), "Can't find perf event info for event_type=" << event_type << ", event_config=" << event_config); - return 0; - } +static int openPerfEvent(perf_event_attr *hw_event, pid_t pid, int cpu, int group_fd, UInt64 flags) +{ + return static_cast(syscall(SYS_perf_event_open, hw_event, pid, cpu, group_fd, flags)); +} - static int openPerfEvent(perf_event_attr *hw_event, pid_t pid, int cpu, int group_fd, UInt64 flags) - { - return static_cast(syscall(SYS_perf_event_open, hw_event, pid, cpu, group_fd, flags)); - } +static bool getPerfEventParanoid(Int32 & result) +{ + // the longest possible variant: "-1\0" + constexpr Int32 max_length = 3; - static bool getPerfEventParanoid(Int32 & result) - { - // the longest possible variant: "-1\0" - constexpr Int32 max_length = 3; + FILE * fp = fopen("/proc/sys/kernel/perf_event_paranoid", "r"); + if (fp == nullptr) + return false; - FILE * fp = fopen("/proc/sys/kernel/perf_event_paranoid", "r"); - if (fp == nullptr) - return false; + char str[max_length]; + char * res = fgets(str, max_length, fp); + fclose(fp); + if (res == nullptr) + return false; - char str[max_length]; - char * res = fgets(str, max_length, fp); - fclose(fp); - if (res == nullptr) - return false; + str[max_length - 1] = '\0'; + Int64 value = strtol(str, nullptr, 10); + // the only way to be incorrect is to not be a number + if (value == 0 && errno != 0) + return false; - str[max_length - 1] = '\0'; - Int64 value = strtol(str, nullptr, 10); - // the only way to be incorrect is to not be a number - if (value == 0 && errno != 0) - return false; + result = static_cast(value); + return true; +} - result = static_cast(value); +static void perfEventOpenDisabled(Int32 perf_event_paranoid, bool has_cap_sys_admin, int perf_event_type, int perf_event_config, int & event_file_descriptor) +{ + perf_event_attr pe = perf_event_attr(); + pe.type = perf_event_type; + pe.size = sizeof(struct perf_event_attr); + pe.config = perf_event_config; + // disable by default to add as little extra time as possible + pe.disabled = 1; + // can record kernel only when `perf_event_paranoid` <= 1 or have CAP_SYS_ADMIN + pe.exclude_kernel = perf_event_paranoid >= 2 && !has_cap_sys_admin; + + event_file_descriptor = openPerfEvent(&pe, /* measure the calling thread */ 0, /* on any cpu */ -1, -1, 0); +} + +bool PerfEventsCounters::initializeThreadLocalEvents(PerfEventsCounters & counters) +{ + if (thread_events_descriptors_opened) return true; + + Int32 perf_event_paranoid = 0; + bool is_pref_available = getPerfEventParanoid(perf_event_paranoid); + if (!is_pref_available) + { + bool expected_value = false; + if (perf_unavailability_logged.compare_exchange_strong(expected_value, true)) + LOG_INFO(getLogger(), "Perf events are unsupported"); + return false; } - static void perfEventOpenDisabled(Int32 perf_event_paranoid, bool has_cap_sys_admin, int perf_event_type, int perf_event_config, int & event_file_descriptor) + bool has_cap_sys_admin = hasLinuxCapability(CAP_SYS_ADMIN); + if (perf_event_paranoid >= 3 && !has_cap_sys_admin) { - perf_event_attr pe = perf_event_attr(); - pe.type = perf_event_type; - pe.size = sizeof(struct perf_event_attr); - pe.config = perf_event_config; - // disable by default to add as little extra time as possible - pe.disabled = 1; - // can record kernel only when `perf_event_paranoid` <= 1 or have CAP_SYS_ADMIN - pe.exclude_kernel = perf_event_paranoid >= 2 && !has_cap_sys_admin; - - event_file_descriptor = openPerfEvent(&pe, /* measure the calling thread */ 0, /* on any cpu */ -1, -1, 0); + bool expected_value = false; + if (perf_unavailability_logged.compare_exchange_strong(expected_value, true)) + LOG_INFO(getLogger(), "Not enough permissions to record perf events"); + return false; } - bool PerfEventsCounters::initializeThreadLocalEvents(PerfEventsCounters & counters) + bool expected = false; + bool log_unsupported_event = particular_events_unavailability_logged.compare_exchange_strong(expected, true); + for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) { - if (thread_events_descriptors_opened) - return true; + counters.raw_event_values[i] = 0; + const PerfEventInfo & event_info = raw_events_info[i]; + int & fd = thread_events_descriptors_holder.descriptors[i]; + perfEventOpenDisabled(perf_event_paranoid, has_cap_sys_admin, event_info.event_type, event_info.event_config, fd); - Int32 perf_event_paranoid = 0; - bool is_pref_available = getPerfEventParanoid(perf_event_paranoid); - if (!is_pref_available) + if (fd == -1 && log_unsupported_event) { - bool expected_value = false; - if (perf_unavailability_logged.compare_exchange_strong(expected_value, true)) - LOG_INFO(getLogger(), "Perf events are unsupported"); - return false; + LOG_INFO(getLogger(), "Perf event is unsupported: event_type=" << event_info.event_type + << ", event_config=" << event_info.event_config); } + } - bool has_cap_sys_admin = hasLinuxCapability(CAP_SYS_ADMIN); - if (perf_event_paranoid >= 3 && !has_cap_sys_admin) - { - bool expected_value = false; - if (perf_unavailability_logged.compare_exchange_strong(expected_value, true)) - LOG_INFO(getLogger(), "Not enough permissions to record perf events"); - return false; - } + thread_events_descriptors_opened = true; + return true; +} - bool expected = false; - bool log_unsupported_event = particular_events_unavailability_logged.compare_exchange_strong(expected, true); - for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) +void PerfEventsCounters::initializeProfileEvents(PerfEventsCounters & counters) +{ + if (current_thread_counters == &counters) + return; + if (current_thread_counters != nullptr) + { + LOG_WARNING(getLogger(), "Only one instance of `PerfEventsCounters` can be used on the thread"); + return; + } + + if (!initializeThreadLocalEvents(counters)) + return; + + for (UInt64 & raw_value : counters.raw_event_values) + raw_value = 0; + + for (int fd : thread_events_descriptors_holder.descriptors) + { + if (fd != -1) + ioctl(fd, PERF_EVENT_IOC_ENABLE, 0); + } + + current_thread_counters = &counters; +} + +void PerfEventsCounters::finalizeProfileEvents(PerfEventsCounters & counters, ProfileEvents::Counters & profile_events) +{ + if (current_thread_counters != &counters) + return; + if (!thread_events_descriptors_opened) + return; + + // process raw events + + // only read counters here to have as little overhead for processing as possible + for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) + { + int fd = counters.thread_events_descriptors_holder.descriptors[i]; + if (fd == -1) + continue; + + constexpr ssize_t bytes_to_read = sizeof(counters.raw_event_values[0]); + if (read(fd, &counters.raw_event_values[i], bytes_to_read) != bytes_to_read) { + LOG_WARNING(getLogger(), "Can't read event value from file descriptor: " << fd); counters.raw_event_values[i] = 0; - const PerfEventInfo & event_info = raw_events_info[i]; - int & fd = thread_events_descriptors_holder.descriptors[i]; - perfEventOpenDisabled(perf_event_paranoid, has_cap_sys_admin, event_info.event_type, event_info.event_config, fd); - - if (fd == -1 && log_unsupported_event) - { - LOG_INFO(getLogger(), "Perf event is unsupported: event_type=" << event_info.event_type - << ", event_config=" << event_info.event_config); - } } - - thread_events_descriptors_opened = true; - return true; } - void PerfEventsCounters::initializeProfileEvents(PerfEventsCounters & counters) + // actually process counters' values and stop measuring + for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) { - if (current_thread_counters == &counters) - return; - if (current_thread_counters != nullptr) - { - LOG_WARNING(getLogger(), "Only one instance of `PerfEventsCounters` can be used on the thread"); - return; - } + int fd = counters.thread_events_descriptors_holder.descriptors[i]; + if (fd == -1) + continue; - if (!initializeThreadLocalEvents(counters)) - return; + profile_events.increment(raw_events_info[i].profile_event, counters.raw_event_values[i]); - for (UInt64 & raw_value : counters.raw_event_values) - raw_value = 0; - - for (int fd : thread_events_descriptors_holder.descriptors) - { - if (fd != -1) - ioctl(fd, PERF_EVENT_IOC_ENABLE, 0); - } - - current_thread_counters = &counters; + if (ioctl(fd, PERF_EVENT_IOC_DISABLE, 0)) + LOG_WARNING(getLogger(), "Can't disable perf event with file descriptor: " << fd); + if (ioctl(fd, PERF_EVENT_IOC_RESET, 0)) + LOG_WARNING(getLogger(), "Can't reset perf event with file descriptor: " << fd); } - void PerfEventsCounters::finalizeProfileEvents(PerfEventsCounters & counters, ProfileEvents::Counters & profile_events) + // process custom events which depend on the raw ones + UInt64 hw_cpu_cycles = counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_CPU_CYCLES); + UInt64 hw_ref_cpu_cycles = counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_REF_CPU_CYCLES); + + UInt64 instructions_per_cpu_scaled = hw_cpu_cycles != 0 + ? counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_INSTRUCTIONS) / hw_cpu_cycles + : 0; + UInt64 instructions_per_cpu = hw_ref_cpu_cycles != 0 + ? counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_INSTRUCTIONS) / hw_ref_cpu_cycles + : 0; + + profile_events.increment(ProfileEvents::PERF_CUSTOM_INSTRUCTIONS_PER_CPU_CYCLE_SCALED, instructions_per_cpu_scaled); + profile_events.increment(ProfileEvents::PERF_CUSTOM_INSTRUCTIONS_PER_CPU_CYCLE, instructions_per_cpu); + + current_thread_counters = nullptr; +} + +PerfDescriptorsHolder::PerfDescriptorsHolder() +{ + for (int & descriptor : descriptors) + descriptor = -1; +} + +PerfDescriptorsHolder::~PerfDescriptorsHolder() +{ + for (int & descriptor : descriptors) { - if (current_thread_counters != &counters) - return; - if (!thread_events_descriptors_opened) - return; + if (descriptor == -1) + continue; - // process raw events + if (ioctl(descriptor, PERF_EVENT_IOC_DISABLE, 0)) + LOG_WARNING(getLogger(), "Can't disable perf event with file descriptor: " << descriptor); + if (close(descriptor)) + LOG_WARNING(getLogger(),"Can't close perf event file descriptor: " << descriptor + << "; error: " << errno << " - " << strerror(errno)); - // only read counters here to have as little overhead for processing as possible - for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) - { - int fd = counters.thread_events_descriptors_holder.descriptors[i]; - if (fd == -1) - continue; - - constexpr ssize_t bytes_to_read = sizeof(counters.raw_event_values[0]); - if (read(fd, &counters.raw_event_values[i], bytes_to_read) != bytes_to_read) - { - LOG_WARNING(getLogger(), "Can't read event value from file descriptor: " << fd); - counters.raw_event_values[i] = 0; - } - } - - // actually process counters' values and stop measuring - for (size_t i = 0; i < NUMBER_OF_RAW_EVENTS; ++i) - { - int fd = counters.thread_events_descriptors_holder.descriptors[i]; - if (fd == -1) - continue; - - profile_events.increment(raw_events_info[i].profile_event, counters.raw_event_values[i]); - - if (ioctl(fd, PERF_EVENT_IOC_DISABLE, 0)) - LOG_WARNING(getLogger(), "Can't disable perf event with file descriptor: " << fd); - if (ioctl(fd, PERF_EVENT_IOC_RESET, 0)) - LOG_WARNING(getLogger(), "Can't reset perf event with file descriptor: " << fd); - } - - // process custom events which depend on the raw ones - UInt64 hw_cpu_cycles = counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_CPU_CYCLES); - UInt64 hw_ref_cpu_cycles = counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_REF_CPU_CYCLES); - - UInt64 instructions_per_cpu_scaled = hw_cpu_cycles != 0 - ? counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_INSTRUCTIONS) / hw_cpu_cycles - : 0; - UInt64 instructions_per_cpu = hw_ref_cpu_cycles != 0 - ? counters.getRawValue(PERF_TYPE_HARDWARE, PERF_COUNT_HW_INSTRUCTIONS) / hw_ref_cpu_cycles - : 0; - - profile_events.increment(ProfileEvents::PERF_CUSTOM_INSTRUCTIONS_PER_CPU_CYCLE_SCALED, instructions_per_cpu_scaled); - profile_events.increment(ProfileEvents::PERF_CUSTOM_INSTRUCTIONS_PER_CPU_CYCLE, instructions_per_cpu); - - current_thread_counters = nullptr; + descriptor = -1; } +} - PerfDescriptorsHolder::PerfDescriptorsHolder() - { - for (int & descriptor : descriptors) - descriptor = -1; - } - - PerfDescriptorsHolder::~PerfDescriptorsHolder() - { - for (int & descriptor : descriptors) - { - if (descriptor == -1) - continue; - - if (ioctl(descriptor, PERF_EVENT_IOC_DISABLE, 0)) - LOG_WARNING(getLogger(), "Can't disable perf event with file descriptor: " << descriptor); - if (close(descriptor)) - LOG_WARNING(getLogger(),"Can't close perf event file descriptor: " << descriptor - << "; error: " << errno << " - " << strerror(errno)); - - descriptor = -1; - } - } - - Logger * PerfDescriptorsHolder::getLogger() - { - return &Logger::get("PerfDescriptorsHolder"); - } +Logger * PerfDescriptorsHolder::getLogger() +{ + return &Logger::get("PerfDescriptorsHolder"); +} +} #else +namespace DB +{ void PerfEventsCounters::initializeProfileEvents(PerfEventsCounters &) {} void PerfEventsCounters::finalizeProfileEvents(PerfEventsCounters &, ProfileEvents::Counters &) {} +} #endif From 182e4f2c0b650b2838bfe6d955c904f687994120 Mon Sep 17 00:00:00 2001 From: Andrey Skobtsov Date: Mon, 18 May 2020 13:40:01 +0300 Subject: [PATCH 042/183] Added running and enabled time for perf metrics --- src/Common/ProfileEvents.cpp | 20 ++++++++++ src/Common/ThreadProfileEvents.cpp | 64 ++++++++++++++++++------------ src/Common/ThreadProfileEvents.h | 35 ++++++++++++++-- 3 files changed, 91 insertions(+), 28 deletions(-) diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 4c4d6e457f1..57ca0c606f5 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -179,15 +179,35 @@ M(OSWriteChars, "Number of bytes written to filesystem, including page cache.") \ \ M(PERF_COUNT_HW_CPU_CYCLES, "Total cycles. Be wary of what happens during CPU frequency scaling.") \ + M(PERF_COUNT_HW_CPU_CYCLES_RUNNING, "Total cycles (