mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
fix order by optimization with monotonous functions
This commit is contained in:
parent
6e62108606
commit
a03b6df897
@ -26,6 +26,7 @@
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Storages/StorageInMemoryMetadata.h>
|
||||
|
||||
|
||||
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<ASTOrderByElement>();
|
||||
auto * order_by_element = order_by->children[i]->as<ASTOrderByElement>();
|
||||
auto & ast_func = order_by_element->children[0];
|
||||
if (!ast_func->as<ASTFunction>())
|
||||
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<TableWithColumnNamesAndTypes> & 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)
|
||||
|
@ -8,6 +8,8 @@ namespace DB
|
||||
{
|
||||
|
||||
class Context;
|
||||
struct StorageInMemoryMetadata;
|
||||
using StorageMetadataPtr = std::shared_ptr<const StorageInMemoryMetadata>;
|
||||
|
||||
/// 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<TableWithColumnNamesAndTypes> & 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);
|
||||
};
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
|
@ -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
|
@ -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;
|
Loading…
Reference in New Issue
Block a user