mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Remove duplicate ORDER BY and DISTINCT from subqueries (#10067)
This commit is contained in:
parent
20b8e1e9ec
commit
10566e2b43
@ -360,6 +360,7 @@ struct Settings : public SettingsCollection<Settings>
|
|||||||
M(SettingBool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \
|
M(SettingBool, optimize_trivial_count_query, true, "Process trivial 'SELECT count() FROM table' query from metadata.", 0) \
|
||||||
M(SettingUInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \
|
M(SettingUInt64, mutations_sync, 0, "Wait for synchronous execution of ALTER TABLE UPDATE/DELETE queries (mutations). 0 - execute asynchronously. 1 - wait current server. 2 - wait all replicas if they exist.", 0) \
|
||||||
M(SettingBool, optimize_arithmetic_operations_in_aggregate_functions, true, "Move arithmetic operations out of aggregation functions", 0) \
|
M(SettingBool, optimize_arithmetic_operations_in_aggregate_functions, true, "Move arithmetic operations out of aggregation functions", 0) \
|
||||||
|
M(SettingBool, optimize_duplicate_order_by_and_distinct, true, "Remove duplicate ORDER BY and DISTINCT if it's possible", 0) \
|
||||||
M(SettingBool, optimize_if_chain_to_miltiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \
|
M(SettingBool, optimize_if_chain_to_miltiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \
|
||||||
M(SettingBool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \
|
M(SettingBool, allow_experimental_alter_materialized_view_structure, false, "Allow atomic alter on Materialized views. Work in progress.", 0) \
|
||||||
M(SettingBool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \
|
M(SettingBool, enable_early_constant_folding, true, "Enable query optimization where we analyze function and subqueries results and rewrite query if there're constants there", 0) \
|
||||||
|
72
src/Interpreters/DuplicateDistinctVisitor.h
Normal file
72
src/Interpreters/DuplicateDistinctVisitor.h
Normal file
@ -0,0 +1,72 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Functions/FunctionFactory.h>
|
||||||
|
#include <IO/WriteHelpers.h>
|
||||||
|
#include <Interpreters/InDepthNodeVisitor.h>
|
||||||
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <Parsers/ASTLiteral.h>
|
||||||
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
|
#include <Parsers/ASTSetQuery.h>
|
||||||
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
|
#include <Parsers/IAST.h>
|
||||||
|
#include <Common/typeid_cast.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/// Removes duplicate DISTINCT from queries.
|
||||||
|
class DuplicateDistinctMatcher
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
struct Data
|
||||||
|
{
|
||||||
|
bool is_distinct;
|
||||||
|
std::vector<String> last_ids;
|
||||||
|
};
|
||||||
|
|
||||||
|
static void visit(const ASTPtr & ast, Data & data)
|
||||||
|
{
|
||||||
|
auto * select_query = ast->as<ASTSelectQuery>();
|
||||||
|
if (select_query)
|
||||||
|
visit(*select_query, data);
|
||||||
|
}
|
||||||
|
|
||||||
|
static void visit(ASTSelectQuery & select_query, Data & data)
|
||||||
|
{
|
||||||
|
if (!select_query.distinct || !select_query.select())
|
||||||
|
return;
|
||||||
|
|
||||||
|
/// Optimize shouldn't work for distributed tables
|
||||||
|
for (const auto & elem : select_query.children)
|
||||||
|
{
|
||||||
|
if (elem->as<ASTSetQuery>() && !elem->as<ASTSetQuery>()->is_standalone)
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
auto expression_list = select_query.select();
|
||||||
|
std::vector<String> current_ids;
|
||||||
|
|
||||||
|
if (expression_list->children.empty())
|
||||||
|
return;
|
||||||
|
|
||||||
|
current_ids.reserve(expression_list->children.size());
|
||||||
|
for (const auto & id : expression_list->children)
|
||||||
|
current_ids.push_back(id->getColumnName());
|
||||||
|
|
||||||
|
if (data.is_distinct && current_ids == data.last_ids)
|
||||||
|
select_query.distinct = false;
|
||||||
|
|
||||||
|
data.is_distinct = true;
|
||||||
|
data.last_ids = std::move(current_ids);
|
||||||
|
}
|
||||||
|
|
||||||
|
static bool needChildVisit(const ASTPtr &, const ASTPtr &)
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
};
|
||||||
|
|
||||||
|
using DuplicateDistinctVisitor = InDepthNodeVisitor<DuplicateDistinctMatcher, false>;
|
||||||
|
|
||||||
|
}
|
127
src/Interpreters/DuplicateOrderByVisitor.h
Normal file
127
src/Interpreters/DuplicateOrderByVisitor.h
Normal file
@ -0,0 +1,127 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Functions/FunctionFactory.h>
|
||||||
|
#include <IO/WriteHelpers.h>
|
||||||
|
#include <Interpreters/InDepthNodeVisitor.h>
|
||||||
|
#include <Parsers/ASTFunction.h>
|
||||||
|
#include <Parsers/ASTLiteral.h>
|
||||||
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
|
#include <Parsers/ASTSetQuery.h>
|
||||||
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
|
#include <Parsers/IAST.h>
|
||||||
|
#include <Common/typeid_cast.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
/// Checks if SELECT has stateful functions
|
||||||
|
class ASTFunctionStatefulData
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
using TypeToVisit = ASTFunction;
|
||||||
|
|
||||||
|
const Context & context;
|
||||||
|
bool & is_stateful;
|
||||||
|
void visit(ASTFunction & ast_function, ASTPtr &)
|
||||||
|
{
|
||||||
|
if (ast_function.name == "any" || ast_function.name == "groupArray")
|
||||||
|
{
|
||||||
|
is_stateful = true;
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
const auto & function = FunctionFactory::instance().tryGet(ast_function.name, context);
|
||||||
|
|
||||||
|
if (function && function->isStateful())
|
||||||
|
{
|
||||||
|
is_stateful = true;
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
using ASTFunctionStatefulMatcher = OneTypeMatcher<ASTFunctionStatefulData>;
|
||||||
|
using ASTFunctionStatefulVisitor = InDepthNodeVisitor<ASTFunctionStatefulMatcher, true>;
|
||||||
|
|
||||||
|
|
||||||
|
/// Erases unnecessary ORDER BY from subquery
|
||||||
|
class DuplicateOrderByFromSubqueriesData
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
using TypeToVisit = ASTSelectQuery;
|
||||||
|
|
||||||
|
bool done = false;
|
||||||
|
|
||||||
|
void visit(ASTSelectQuery & select_query, ASTPtr &)
|
||||||
|
{
|
||||||
|
if (done)
|
||||||
|
return;
|
||||||
|
|
||||||
|
if (select_query.orderBy() && !select_query.limitBy() && !select_query.limitByOffset() &&
|
||||||
|
!select_query.limitByLength() && !select_query.limitLength() && !select_query.limitOffset())
|
||||||
|
{
|
||||||
|
select_query.setExpression(ASTSelectQuery::Expression::ORDER_BY, nullptr);
|
||||||
|
}
|
||||||
|
|
||||||
|
done = true;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
using DuplicateOrderByFromSubqueriesMatcher = OneTypeMatcher<DuplicateOrderByFromSubqueriesData>;
|
||||||
|
using DuplicateOrderByFromSubqueriesVisitor = InDepthNodeVisitor<DuplicateOrderByFromSubqueriesMatcher, true>;
|
||||||
|
|
||||||
|
|
||||||
|
/// Finds SELECT that can be optimized
|
||||||
|
class DuplicateOrderByData
|
||||||
|
{
|
||||||
|
public:
|
||||||
|
using TypeToVisit = ASTSelectQuery;
|
||||||
|
|
||||||
|
const Context & context;
|
||||||
|
bool done = false;
|
||||||
|
|
||||||
|
void visit(ASTSelectQuery & select_query, ASTPtr &)
|
||||||
|
{
|
||||||
|
if (done)
|
||||||
|
return;
|
||||||
|
|
||||||
|
/// Disable optimization for distributed tables
|
||||||
|
for (const auto & elem : select_query.children)
|
||||||
|
{
|
||||||
|
if (elem->as<ASTSetQuery>() && !elem->as<ASTSetQuery>()->is_standalone)
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (select_query.orderBy() || select_query.groupBy())
|
||||||
|
{
|
||||||
|
for (auto & elem : select_query.children)
|
||||||
|
{
|
||||||
|
if (elem->as<ASTExpressionList>())
|
||||||
|
{
|
||||||
|
bool is_stateful = false;
|
||||||
|
ASTFunctionStatefulVisitor::Data data{context, is_stateful};
|
||||||
|
ASTFunctionStatefulVisitor(data).visit(elem);
|
||||||
|
if (is_stateful)
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
if (auto select_table_ptr = select_query.tables())
|
||||||
|
{
|
||||||
|
if (auto * select_table = select_table_ptr->as<ASTTablesInSelectQuery>())
|
||||||
|
{
|
||||||
|
if (!select_table->children.empty())
|
||||||
|
{
|
||||||
|
DuplicateOrderByFromSubqueriesVisitor::Data data{false};
|
||||||
|
DuplicateOrderByFromSubqueriesVisitor(data).visit(select_table->children[0]);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
using DuplicateOrderByMatcher = OneTypeMatcher<DuplicateOrderByData>;
|
||||||
|
using DuplicateOrderByVisitor = InDepthNodeVisitor<DuplicateOrderByMatcher, true>;
|
||||||
|
|
||||||
|
}
|
@ -23,12 +23,15 @@
|
|||||||
#include <Interpreters/getTableExpressions.h>
|
#include <Interpreters/getTableExpressions.h>
|
||||||
#include <Interpreters/OptimizeIfChains.h>
|
#include <Interpreters/OptimizeIfChains.h>
|
||||||
#include <Interpreters/ArithmeticOperationsInAgrFuncOptimize.h>
|
#include <Interpreters/ArithmeticOperationsInAgrFuncOptimize.h>
|
||||||
|
#include <Interpreters/DuplicateDistinctVisitor.h>
|
||||||
|
#include <Interpreters/DuplicateOrderByVisitor.h>
|
||||||
|
|
||||||
#include <Parsers/ASTExpressionList.h>
|
#include <Parsers/ASTExpressionList.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
#include <Parsers/ASTLiteral.h>
|
#include <Parsers/ASTLiteral.h>
|
||||||
#include <Parsers/ASTOrderByElement.h>
|
#include <Parsers/ASTOrderByElement.h>
|
||||||
#include <Parsers/ASTSelectQuery.h>
|
#include <Parsers/ASTSelectQuery.h>
|
||||||
|
#include <Parsers/ASTSetQuery.h>
|
||||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
#include <Parsers/queryToString.h>
|
#include <Parsers/queryToString.h>
|
||||||
|
|
||||||
@ -370,6 +373,18 @@ void optimizeOrderBy(const ASTSelectQuery * select_query)
|
|||||||
elems = std::move(unique_elems);
|
elems = std::move(unique_elems);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/// Optimize duplicate ORDER BY and DISTINCT
|
||||||
|
void optimizeDuplicateOrderByAndDistinct(ASTPtr & query, bool optimize_duplicate_order_by_and_distinct, const Context & context)
|
||||||
|
{
|
||||||
|
if (optimize_duplicate_order_by_and_distinct)
|
||||||
|
{
|
||||||
|
DuplicateOrderByVisitor::Data order_by_data{context, false};
|
||||||
|
DuplicateOrderByVisitor(order_by_data).visit(query);
|
||||||
|
DuplicateDistinctVisitor::Data distinct_data{};
|
||||||
|
DuplicateDistinctVisitor(distinct_data).visit(query);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/// Remove duplicate items from LIMIT BY.
|
/// Remove duplicate items from LIMIT BY.
|
||||||
void optimizeLimitBy(const ASTSelectQuery * select_query)
|
void optimizeLimitBy(const ASTSelectQuery * select_query)
|
||||||
{
|
{
|
||||||
@ -831,6 +846,9 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyzeSelect(
|
|||||||
/// Remove duplicate items from ORDER BY.
|
/// Remove duplicate items from ORDER BY.
|
||||||
optimizeOrderBy(select_query);
|
optimizeOrderBy(select_query);
|
||||||
|
|
||||||
|
/// Remove duplicate ORDER BY and DISTINCT from subqueries.
|
||||||
|
optimizeDuplicateOrderByAndDistinct(query, settings.optimize_duplicate_order_by_and_distinct, context);
|
||||||
|
|
||||||
/// Remove duplicated elements from LIMIT BY clause.
|
/// Remove duplicated elements from LIMIT BY clause.
|
||||||
optimizeLimitBy(select_query);
|
optimizeLimitBy(select_query);
|
||||||
|
|
||||||
|
10
tests/performance/duplicate_order_by_and_distinct.xml
Normal file
10
tests/performance/duplicate_order_by_and_distinct.xml
Normal file
@ -0,0 +1,10 @@
|
|||||||
|
<test>
|
||||||
|
<preconditions>
|
||||||
|
<table_exists>hits_10m_single</table_exists>
|
||||||
|
</preconditions>
|
||||||
|
|
||||||
|
<query>SELECT * FROM (SELECT CounterID, EventDate FROM hits_10m_single ORDER BY CounterID DESC) ORDER BY EventDate, CounterID FORMAT Null</query>
|
||||||
|
<query>SELECT DISTINCT * FROM (SELECT DISTINCT CounterID, EventDate FROM hits_10m_single) FORMAT Null</query>
|
||||||
|
<query>SELECT DISTINCT * FROM (SELECT DISTINCT CounterID, EventDate FROM hits_10m_single ORDER BY CounterID DESC) ORDER BY toStartOfWeek(EventDate) FORMAT Null</query>
|
||||||
|
|
||||||
|
</test>
|
@ -0,0 +1,14 @@
|
|||||||
|
SELECT number\nFROM \n(\n SELECT number\n FROM \n (\n SELECT DISTINCT number\n FROM numbers(3)\n )\n)\nORDER BY number ASC
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
SELECT DISTINCT number\nFROM \n(\n SELECT DISTINCT number\n FROM \n (\n SELECT DISTINCT number\n FROM numbers(3)\n ORDER BY number ASC\n )\n ORDER BY number ASC\n)\nORDER BY number ASC
|
||||||
|
0
|
||||||
|
1
|
||||||
|
2
|
||||||
|
SELECT number\nFROM \n(\n SELECT DISTINCT number\n FROM \n (\n SELECT DISTINCT number % 2 AS number\n FROM numbers(3)\n )\n)\nORDER BY number ASC
|
||||||
|
0
|
||||||
|
1
|
||||||
|
SELECT DISTINCT number\nFROM \n(\n SELECT DISTINCT number\n FROM \n (\n SELECT DISTINCT number % 2 AS number\n FROM numbers(3)\n ORDER BY number ASC\n )\n ORDER BY number ASC\n)\nORDER BY number ASC
|
||||||
|
0
|
||||||
|
1
|
@ -0,0 +1,124 @@
|
|||||||
|
set enable_debug_queries = 1;
|
||||||
|
set optimize_duplicate_order_by_and_distinct = 1;
|
||||||
|
|
||||||
|
analyze SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM numbers(3)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number;
|
||||||
|
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM numbers(3)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number;
|
||||||
|
|
||||||
|
set optimize_duplicate_order_by_and_distinct = 0;
|
||||||
|
|
||||||
|
analyze SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM numbers(3)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number;
|
||||||
|
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM numbers(3)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number;
|
||||||
|
|
||||||
|
set optimize_duplicate_order_by_and_distinct = 1;
|
||||||
|
|
||||||
|
analyze SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT number % 2
|
||||||
|
AS number
|
||||||
|
FROM numbers(3)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number;
|
||||||
|
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT number % 2
|
||||||
|
AS number
|
||||||
|
FROM numbers(3)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number;
|
||||||
|
|
||||||
|
set optimize_duplicate_order_by_and_distinct = 0;
|
||||||
|
|
||||||
|
analyze SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT number % 2
|
||||||
|
AS number
|
||||||
|
FROM numbers(3)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number;
|
||||||
|
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT *
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT number % 2
|
||||||
|
AS number
|
||||||
|
FROM numbers(3)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number
|
||||||
|
)
|
||||||
|
ORDER BY number;
|
@ -0,0 +1,2 @@
|
|||||||
|
0
|
||||||
|
0
|
@ -0,0 +1,20 @@
|
|||||||
|
set optimize_duplicate_order_by_and_distinct = 1;
|
||||||
|
SELECT DISTINCT number
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT number
|
||||||
|
FROM remote('127.0.0.{1,2}', system.numbers)
|
||||||
|
LIMIT 1
|
||||||
|
SETTINGS distributed_group_by_no_merge = 1
|
||||||
|
);
|
||||||
|
|
||||||
|
set optimize_duplicate_order_by_and_distinct = 0;
|
||||||
|
SELECT DISTINCT number
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT number
|
||||||
|
FROM remote('127.0.0.{1,2}', system.numbers)
|
||||||
|
LIMIT 1
|
||||||
|
SETTINGS distributed_group_by_no_merge = 1
|
||||||
|
);
|
||||||
|
|
Loading…
Reference in New Issue
Block a user