diff --git a/src/Interpreters/TreeOptimizer.cpp b/src/Interpreters/TreeOptimizer.cpp index 61ca933dd53..603733129f4 100644 --- a/src/Interpreters/TreeOptimizer.cpp +++ b/src/Interpreters/TreeOptimizer.cpp @@ -26,6 +26,7 @@ #include #include +#include namespace DB @@ -438,7 +439,8 @@ void optimizeDuplicateDistinct(ASTSelectQuery & select) /// Replace monotonous functions in ORDER BY if they don't participate in GROUP BY expression, /// has a single argument and not an aggregate functions. void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, const Context & context, - const TablesWithColumns & tables_with_columns) + const TablesWithColumns & tables_with_columns, + const Names & sorting_key_columns) { auto order_by = select_query->orderBy(); if (!order_by) @@ -455,13 +457,22 @@ void optimizeMonotonousFunctionsInOrderBy(ASTSelectQuery * select_query, const C } } - for (auto & child : order_by->children) + bool is_sorting_key_prefix = true; + for (size_t i = 0; i < order_by->children.size(); ++i) { - auto * order_by_element = child->as(); + auto * order_by_element = order_by->children[i]->as(); auto & ast_func = order_by_element->children[0]; if (!ast_func->as()) continue; + if (i >= sorting_key_columns.size() || ast_func->getColumnName() != sorting_key_columns[i]) + is_sorting_key_prefix = false; + + /// If order by expression matches the sorting key, do not remove + /// functions to allow execute reading in order of key. + if (is_sorting_key_prefix) + continue; + MonotonicityCheckVisitor::Data data{tables_with_columns, context, group_by_hashes}; MonotonicityCheckVisitor(data).visit(ast_func); @@ -611,7 +622,8 @@ void TreeOptimizer::optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, const std::vector & tables_with_columns, - const Context & context, bool & rewrite_subqueries) + const Context & context, const StorageMetadataPtr & metadata_snapshot, + bool & rewrite_subqueries) { const auto & settings = context.getSettingsRef(); @@ -652,9 +664,6 @@ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & sou optimizeAggregateFunctionsOfGroupByKeys(select_query, query); } - /// Remove duplicate items from ORDER BY. - optimizeDuplicatesInOrderBy(select_query); - /// Remove duplicate ORDER BY and DISTINCT from subqueries. if (settings.optimize_duplicate_order_by_and_distinct) { @@ -672,7 +681,13 @@ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & sou /// Replace monotonous functions with its argument if (settings.optimize_monotonous_functions_in_order_by) - optimizeMonotonousFunctionsInOrderBy(select_query, context, tables_with_columns); + optimizeMonotonousFunctionsInOrderBy(select_query, context, tables_with_columns, + metadata_snapshot ? metadata_snapshot->getSortingKeyColumns() : Names{}); + + /// Remove duplicate items from ORDER BY. + /// Execute it after all order by optimizations, + /// because they can produce duplicated columns. + optimizeDuplicatesInOrderBy(select_query); /// If function "if" has String-type arguments, transform them into enum if (settings.optimize_if_transform_strings_to_enum) diff --git a/src/Interpreters/TreeOptimizer.h b/src/Interpreters/TreeOptimizer.h index a81264184c1..a10dfc57451 100644 --- a/src/Interpreters/TreeOptimizer.h +++ b/src/Interpreters/TreeOptimizer.h @@ -8,6 +8,8 @@ namespace DB { class Context; +struct StorageInMemoryMetadata; +using StorageMetadataPtr = std::shared_ptr; /// Part of of Tree Rewriter (SyntaxAnalyzer) that optimizes AST. /// Query should be ready to execute either before either after it. But resulting query could be faster. @@ -16,7 +18,8 @@ class TreeOptimizer public: static void apply(ASTPtr & query, Aliases & aliases, const NameSet & source_columns_set, const std::vector & tables_with_columns, - const Context & context, bool & rewrite_subqueries); + const Context & context, const StorageMetadataPtr & metadata_snapshot, + bool & rewrite_subqueries); static void optimizeIf(ASTPtr & query, Aliases & aliases, bool if_chain_to_multiif); }; diff --git a/src/Interpreters/TreeRewriter.cpp b/src/Interpreters/TreeRewriter.cpp index b2dbd027191..c02d54435eb 100644 --- a/src/Interpreters/TreeRewriter.cpp +++ b/src/Interpreters/TreeRewriter.cpp @@ -645,7 +645,7 @@ TreeRewriterResultPtr TreeRewriter::analyzeSelect( /// Executing scalar subqueries - replacing them with constant values. executeScalarSubqueries(query, context, subquery_depth, result.scalars, select_options.only_analyze); - TreeOptimizer::apply(query, result.aliases, source_columns_set, tables_with_columns, context, result.rewrite_subqueries); + TreeOptimizer::apply(query, result.aliases, source_columns_set, tables_with_columns, context, result.metadata_snapshot, result.rewrite_subqueries); /// array_join_alias_to_name, array_join_result_to_source. getArrayJoinedColumns(query, result, select_query, result.source_columns, source_columns_set); diff --git a/tests/queries/0_stateless/01321_monotonous_functions_in_order_by.reference b/tests/queries/0_stateless/01321_monotonous_functions_in_order_by.reference index ffa91586f35..0a7746e1832 100644 --- a/tests/queries/0_stateless/01321_monotonous_functions_in_order_by.reference +++ b/tests/queries/0_stateless/01321_monotonous_functions_in_order_by.reference @@ -54,14 +54,10 @@ FROM numbers(3) ORDER BY exp(number) ASC SELECT roundToExp2(number) AS x FROM numbers(3) -ORDER BY - number ASC, - number ASC +ORDER BY number ASC SELECT number AS x FROM numbers(3) -ORDER BY - number ASC, - number ASC +ORDER BY number ASC SELECT number FROM numbers(3) ORDER BY number DESC @@ -79,9 +75,7 @@ FROM numbers(3) ORDER BY exp(number) DESC SELECT roundToExp2(number) AS x FROM numbers(3) -ORDER BY - number DESC, - number DESC +ORDER BY number DESC 0 1 2 diff --git a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference new file mode 100644 index 00000000000..203efdab299 --- /dev/null +++ b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.reference @@ -0,0 +1,46 @@ +SELECT + timestamp, + key +FROM test_order_by +ORDER BY timestamp ASC +LIMIT 10 +Expression (Projection) + Limit (preliminary LIMIT) + MergingSorted (Merge sorted streams for ORDER BY) + MergeSorting (Merge sorted blocks for ORDER BY) + PartialSorting (Sort each block for ORDER BY) + Expression (Before ORDER BY and SELECT) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (MergeTree) +SELECT + timestamp, + key +FROM test_order_by +ORDER BY toDate(timestamp) ASC +LIMIT 10 +Expression (Projection) + Limit (preliminary LIMIT) + FinishSorting + Expression (Before ORDER BY and SELECT) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (MergeTree with order) +SELECT + timestamp, + key +FROM test_order_by +ORDER BY + toDate(timestamp) ASC, + timestamp ASC +LIMIT 10 +Expression (Projection) + Limit (preliminary LIMIT) + FinishSorting + Expression (Before ORDER BY and SELECT) + SettingQuotaAndLimits (Set limits and quota after reading from storage) + ReadFromStorage (MergeTree with order) +SELECT + timestamp, + key +FROM test_order_by +ORDER BY timestamp ASC +LIMIT 10 diff --git a/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.sql b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.sql new file mode 100644 index 00000000000..b31457d8f68 --- /dev/null +++ b/tests/queries/0_stateless/01562_optimize_monotonous_functions_in_order_by.sql @@ -0,0 +1,26 @@ +SET optimize_monotonous_functions_in_order_by = 1; + +DROP TABLE IF EXISTS test_order_by; + +CREATE TABLE test_order_by (timestamp DateTime, key UInt32) ENGINE=MergeTree() ORDER BY (toDate(timestamp), key); +INSERT INTO test_order_by SELECT now() + toIntervalSecond(number), number % 4 FROM numbers(10000); +OPTIMIZE TABLE test_order_by FINAL; + +EXPLAIN SYNTAX SELECT * FROM test_order_by ORDER BY timestamp LIMIT 10; +EXPLAIN PLAN SELECT * FROM test_order_by ORDER BY timestamp LIMIT 10; + +EXPLAIN SYNTAX SELECT * FROM test_order_by ORDER BY toDate(timestamp) LIMIT 10; +EXPLAIN PLAN SELECT * FROM test_order_by ORDER BY toDate(timestamp) LIMIT 10; + +EXPLAIN SYNTAX SELECT * FROM test_order_by ORDER BY toDate(timestamp), timestamp LIMIT 10; +EXPLAIN PLAN SELECT * FROM test_order_by ORDER BY toDate(timestamp), timestamp LIMIT 10; + +DROP TABLE IF EXISTS test_order_by; + +CREATE TABLE test_order_by (timestamp DateTime, key UInt32) ENGINE=MergeTree() ORDER BY tuple(); +INSERT INTO test_order_by SELECT now() + toIntervalSecond(number), number % 4 FROM numbers(10000); +OPTIMIZE TABLE test_order_by FINAL; + +EXPLAIN SYNTAX SELECT * FROM test_order_by ORDER BY toDate(timestamp), timestamp LIMIT 10; + +DROP TABLE IF EXISTS test_order_by;