mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
rewrite duplicate distinct optimization
This commit is contained in:
parent
7445ab94bf
commit
911e6efe3e
@ -1,72 +0,0 @@
|
|||||||
#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>;
|
|
||||||
|
|
||||||
}
|
|
@ -4,7 +4,6 @@
|
|||||||
#include <Interpreters/OptimizeIfChains.h>
|
#include <Interpreters/OptimizeIfChains.h>
|
||||||
#include <Interpreters/OptimizeIfWithConstantConditionVisitor.h>
|
#include <Interpreters/OptimizeIfWithConstantConditionVisitor.h>
|
||||||
#include <Interpreters/ArithmeticOperationsInAgrFuncOptimize.h>
|
#include <Interpreters/ArithmeticOperationsInAgrFuncOptimize.h>
|
||||||
#include <Interpreters/DuplicateDistinctVisitor.h>
|
|
||||||
#include <Interpreters/DuplicateOrderByVisitor.h>
|
#include <Interpreters/DuplicateOrderByVisitor.h>
|
||||||
#include <Interpreters/GroupByFunctionKeysVisitor.h>
|
#include <Interpreters/GroupByFunctionKeysVisitor.h>
|
||||||
#include <Interpreters/AggregateFunctionOfGroupByKeysVisitor.h>
|
#include <Interpreters/AggregateFunctionOfGroupByKeysVisitor.h>
|
||||||
@ -22,6 +21,8 @@
|
|||||||
#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/ASTSubquery.h>
|
||||||
|
#include <Parsers/ASTSelectWithUnionQuery.h>
|
||||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||||
|
|
||||||
#include <Functions/FunctionFactory.h>
|
#include <Functions/FunctionFactory.h>
|
||||||
@ -311,13 +312,118 @@ void optimizeDuplicatesInOrderBy(const ASTSelectQuery * select_query)
|
|||||||
elems = std::move(unique_elems);
|
elems = std::move(unique_elems);
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Optimize duplicate ORDER BY and DISTINCT
|
/// Optimize duplicate ORDER BY
|
||||||
void optimizeDuplicateOrderByAndDistinct(ASTPtr & query, const Context & context)
|
void optimizeDuplicateOrderBy(ASTPtr & query, const Context & context)
|
||||||
{
|
{
|
||||||
DuplicateOrderByVisitor::Data order_by_data{context};
|
DuplicateOrderByVisitor::Data order_by_data{context};
|
||||||
DuplicateOrderByVisitor(order_by_data).visit(query);
|
DuplicateOrderByVisitor(order_by_data).visit(query);
|
||||||
DuplicateDistinctVisitor::Data distinct_data{};
|
}
|
||||||
DuplicateDistinctVisitor(distinct_data).visit(query);
|
|
||||||
|
/// Return simple subselect (without UNIONs or JOINs) if any
|
||||||
|
const ASTSelectQuery * getSimpleSubselect(const ASTSelectQuery & select)
|
||||||
|
{
|
||||||
|
const auto & tables = select.tables()->children;
|
||||||
|
if (tables.empty() || tables.size() != 1)
|
||||||
|
return nullptr;
|
||||||
|
|
||||||
|
const auto & ast_table_expression = tables[0]->as<ASTTablesInSelectQueryElement>()->table_expression;
|
||||||
|
if (!ast_table_expression)
|
||||||
|
return nullptr;
|
||||||
|
|
||||||
|
const auto & table_expression = ast_table_expression->as<ASTTableExpression>();
|
||||||
|
if (!table_expression->subquery)
|
||||||
|
return nullptr;
|
||||||
|
|
||||||
|
const auto & subquery = table_expression->subquery->as<ASTSubquery>();
|
||||||
|
if (!subquery || subquery->children.size() != 1)
|
||||||
|
return nullptr;
|
||||||
|
|
||||||
|
const auto & subselect_union = subquery->children[0]->as<ASTSelectWithUnionQuery>();
|
||||||
|
if (!subselect_union || !subselect_union->list_of_selects ||
|
||||||
|
subselect_union->list_of_selects->children.size() != 1)
|
||||||
|
return nullptr;
|
||||||
|
|
||||||
|
return subselect_union->list_of_selects->children[0]->as<ASTSelectQuery>();
|
||||||
|
}
|
||||||
|
|
||||||
|
std::unordered_set<String> getDistinctNames(const ASTSelectQuery & select)
|
||||||
|
{
|
||||||
|
if (!select.select() || select.select()->children.empty())
|
||||||
|
return {};
|
||||||
|
|
||||||
|
std::unordered_set<String> names;
|
||||||
|
std::unordered_set<String> implicit_distinct;
|
||||||
|
|
||||||
|
if (!select.distinct)
|
||||||
|
{
|
||||||
|
/// SELECT a, b FROM (SELECT DISTINCT a FROM ...)
|
||||||
|
if (const ASTSelectQuery * subselect = getSimpleSubselect(select))
|
||||||
|
implicit_distinct = getDistinctNames(*subselect);
|
||||||
|
|
||||||
|
if (implicit_distinct.empty())
|
||||||
|
return {};
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Extract result column names (prefer aliases, ignore table name)
|
||||||
|
for (const auto & id : select.select()->children)
|
||||||
|
{
|
||||||
|
String alias = id->tryGetAlias();
|
||||||
|
|
||||||
|
if (const auto * identifier = id->as<ASTIdentifier>())
|
||||||
|
{
|
||||||
|
String name = identifier->shortName();
|
||||||
|
|
||||||
|
if (select.distinct || implicit_distinct.count(name))
|
||||||
|
{
|
||||||
|
if (alias.empty())
|
||||||
|
names.insert(name);
|
||||||
|
else
|
||||||
|
names.insert(alias);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else if (select.distinct && !alias.empty())
|
||||||
|
{
|
||||||
|
/// It's not possible to use getAliasOrColumnName() cause name is context specific (function arguments)
|
||||||
|
names.insert(alias);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return names;
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Remove DISTINCT from query if columns are known as DISTINCT from subquery
|
||||||
|
void optimizeDuplicateDistinct(ASTSelectQuery & select)
|
||||||
|
{
|
||||||
|
if (!select.select() || select.select()->children.empty())
|
||||||
|
return;
|
||||||
|
|
||||||
|
const ASTSelectQuery * subselect = getSimpleSubselect(select);
|
||||||
|
if (!subselect)
|
||||||
|
return;
|
||||||
|
|
||||||
|
std::unordered_set<String> distinct_names = getDistinctNames(*subselect);
|
||||||
|
std::unordered_set<String> selected_names;
|
||||||
|
|
||||||
|
/// Check source column names from select list (ignore aliases and table names)
|
||||||
|
for (const auto & id : select.select()->children)
|
||||||
|
{
|
||||||
|
const auto * identifier = id->as<ASTIdentifier>();
|
||||||
|
if (!identifier)
|
||||||
|
return;
|
||||||
|
|
||||||
|
String name = identifier->shortName();
|
||||||
|
if (!distinct_names.count(name))
|
||||||
|
return; /// Not a distinct column, keep DISTINCT for it.
|
||||||
|
|
||||||
|
selected_names.insert(name);
|
||||||
|
}
|
||||||
|
|
||||||
|
/// select columns list != distinct columns list
|
||||||
|
/// SELECT DISTINCT a FROM (SELECT DISTINCT a, b FROM ...)) -- cannot remove DISTINCT
|
||||||
|
if (selected_names.size() != distinct_names.size())
|
||||||
|
return;
|
||||||
|
|
||||||
|
select.distinct = false;
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Replace monotonous functions in ORDER BY if they don't participate in GROUP BY expression,
|
/// Replace monotonous functions in ORDER BY if they don't participate in GROUP BY expression,
|
||||||
@ -537,7 +643,10 @@ void TreeOptimizer::apply(ASTPtr & query, Aliases & aliases, const NameSet & sou
|
|||||||
|
|
||||||
/// Remove duplicate ORDER BY and DISTINCT from subqueries.
|
/// Remove duplicate ORDER BY and DISTINCT from subqueries.
|
||||||
if (settings.optimize_duplicate_order_by_and_distinct)
|
if (settings.optimize_duplicate_order_by_and_distinct)
|
||||||
optimizeDuplicateOrderByAndDistinct(query, context);
|
{
|
||||||
|
optimizeDuplicateOrderBy(query, context);
|
||||||
|
optimizeDuplicateDistinct(*select_query);
|
||||||
|
}
|
||||||
|
|
||||||
/// Remove functions from ORDER BY if its argument is also in ORDER BY
|
/// Remove functions from ORDER BY if its argument is also in ORDER BY
|
||||||
if (settings.optimize_redundant_functions_in_order_by)
|
if (settings.optimize_redundant_functions_in_order_by)
|
||||||
|
@ -0,0 +1,124 @@
|
|||||||
|
SELECT DISTINCT number
|
||||||
|
FROM numbers(1)
|
||||||
|
SELECT number
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT number
|
||||||
|
FROM numbers(1)
|
||||||
|
)
|
||||||
|
SELECT DISTINCT number * 2
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT
|
||||||
|
number * 2,
|
||||||
|
number
|
||||||
|
FROM numbers(1)
|
||||||
|
)
|
||||||
|
SELECT number
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT number * 2 AS number
|
||||||
|
FROM numbers(1)
|
||||||
|
)
|
||||||
|
SELECT
|
||||||
|
b,
|
||||||
|
a
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT
|
||||||
|
number % 2 AS a,
|
||||||
|
number % 3 AS b
|
||||||
|
FROM numbers(100)
|
||||||
|
)
|
||||||
|
SELECT DISTINCT a
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT
|
||||||
|
number % 2 AS a,
|
||||||
|
number % 3 AS b
|
||||||
|
FROM numbers(100)
|
||||||
|
)
|
||||||
|
SELECT a
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT a
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT
|
||||||
|
number % 2 AS a,
|
||||||
|
number % 3 AS b
|
||||||
|
FROM numbers(100)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
SELECT DISTINCT a
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
a,
|
||||||
|
b
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT
|
||||||
|
number % 2 AS a,
|
||||||
|
number % 3 AS b
|
||||||
|
FROM numbers(100)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
SELECT
|
||||||
|
a,
|
||||||
|
b
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
b,
|
||||||
|
a
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT
|
||||||
|
number AS a,
|
||||||
|
number AS b
|
||||||
|
FROM numbers(1)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
SELECT
|
||||||
|
a,
|
||||||
|
b
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT
|
||||||
|
b,
|
||||||
|
a,
|
||||||
|
a + b
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT
|
||||||
|
number % 2 AS a,
|
||||||
|
number % 3 AS b
|
||||||
|
FROM numbers(100)
|
||||||
|
)
|
||||||
|
)
|
||||||
|
SELECT DISTINCT number
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT number
|
||||||
|
FROM numbers(1)
|
||||||
|
) AS t1
|
||||||
|
CROSS JOIN numbers(2) AS t2
|
||||||
|
SELECT number
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT number
|
||||||
|
FROM numbers(1) AS t1
|
||||||
|
CROSS JOIN numbers(2) AS t2
|
||||||
|
)
|
||||||
|
SELECT DISTINCT number
|
||||||
|
FROM
|
||||||
|
(
|
||||||
|
SELECT DISTINCT number
|
||||||
|
FROM numbers(1)
|
||||||
|
UNION ALL
|
||||||
|
SELECT DISTINCT number
|
||||||
|
FROM numbers(2)
|
||||||
|
)
|
||||||
|
0
|
||||||
|
1
|
@ -0,0 +1,32 @@
|
|||||||
|
SET enable_debug_queries = 1;
|
||||||
|
SET optimize_duplicate_order_by_and_distinct = 1;
|
||||||
|
|
||||||
|
ANALYZE SELECT DISTINCT number FROM numbers(1);
|
||||||
|
ANALYZE SELECT DISTINCT number FROM (SELECT DISTINCT number FROM numbers(1));
|
||||||
|
ANALYZE SELECT DISTINCT number * 2 FROM (SELECT DISTINCT number * 2, number FROM numbers(1));
|
||||||
|
ANALYZE SELECT DISTINCT number FROM (SELECT DISTINCT number * 2 AS number FROM numbers(1));
|
||||||
|
ANALYZE SELECT DISTINCT b, a FROM (SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100));
|
||||||
|
ANALYZE SELECT DISTINCT a FROM (SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100));
|
||||||
|
ANALYZE SELECT DISTINCT a FROM (SELECT DISTINCT a FROM (SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100)));
|
||||||
|
ANALYZE SELECT DISTINCT a FROM (SELECT DISTINCT a, b FROM (SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100)));
|
||||||
|
ANALYZE SELECT DISTINCT a, b FROM (SELECT DISTINCT b, a FROM (SELECT DISTINCT number a, number b FROM numbers(1)));
|
||||||
|
ANALYZE SELECT DISTINCT a, b FROM (SELECT b, a, a + b FROM (SELECT DISTINCT number % 2 AS a, number % 3 AS b FROM numbers(100)));
|
||||||
|
ANALYZE SELECT DISTINCT number FROM (SELECT DISTINCT number FROM numbers(1)) t1 CROSS JOIN numbers(2) t2;
|
||||||
|
ANALYZE SELECT DISTINCT number FROM (SELECT DISTINCT number FROM numbers(1) t1 CROSS JOIN numbers(2) t2);
|
||||||
|
|
||||||
|
ANALYZE SELECT DISTINCT number FROM
|
||||||
|
(
|
||||||
|
(SELECT DISTINCT number FROM numbers(1))
|
||||||
|
UNION ALL
|
||||||
|
(SELECT DISTINCT number FROM numbers(2))
|
||||||
|
);
|
||||||
|
|
||||||
|
--
|
||||||
|
|
||||||
|
SELECT DISTINCT number FROM
|
||||||
|
(
|
||||||
|
(SELECT DISTINCT number FROM numbers(1))
|
||||||
|
UNION ALL
|
||||||
|
(SELECT DISTINCT number FROM numbers(2))
|
||||||
|
)
|
||||||
|
ORDER BY number;
|
Loading…
Reference in New Issue
Block a user