mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-09 17:14:47 +00:00
Delete injective functions inside uniq (#12337)
This commit is contained in:
parent
5cae87e664
commit
01b5c2663c
@ -367,6 +367,7 @@ struct Settings : public SettingsCollection<Settings>
|
||||
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(SettingBool, optimize_move_functions_out_of_any, true, "Move functions out of aggregate functions 'any', 'anyLast'.", 0) \
|
||||
M(SettingBool, optimize_injective_functions_inside_uniq, true, "Delete injective functions of one argument inside uniq*() 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_multiif, false, "Replace if(cond1, then1, if(cond2, ...)) chains to multiIf. Currently it's not beneficial for numeric types.", 0) \
|
||||
|
63
src/Interpreters/RemoveInjectiveFunctionsVisitor.cpp
Normal file
63
src/Interpreters/RemoveInjectiveFunctionsVisitor.cpp
Normal file
@ -0,0 +1,63 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/ASTSubquery.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTTablesInSelectQuery.h>
|
||||
#include <Interpreters/RemoveInjectiveFunctionsVisitor.h>
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static bool isUniq(const ASTFunction & func)
|
||||
{
|
||||
return func.name == "uniq" || func.name == "uniqExact" || func.name == "uniqHLL12"
|
||||
|| func.name == "uniqCombined" || func.name == "uniqCombined64";
|
||||
}
|
||||
|
||||
/// Remove injective functions of one argument: replace with a child
|
||||
static bool removeInjectiveFunction(ASTPtr & ast, const Context & context, const FunctionFactory & function_factory)
|
||||
{
|
||||
const ASTFunction * func = ast->as<ASTFunction>();
|
||||
if (!func)
|
||||
return false;
|
||||
|
||||
if (func->arguments->children.size() != 1)
|
||||
return false;
|
||||
|
||||
if (!function_factory.get(func->name, context)->isInjective(Block{}))
|
||||
return false;
|
||||
|
||||
ast = func->arguments->children[0];
|
||||
return true;
|
||||
}
|
||||
|
||||
void RemoveInjectiveFunctionsMatcher::visit(ASTPtr & ast, const Data & data)
|
||||
{
|
||||
if (auto * func = ast->as<ASTFunction>())
|
||||
visit(*func, ast, data);
|
||||
}
|
||||
|
||||
void RemoveInjectiveFunctionsMatcher::visit(ASTFunction & func, ASTPtr &, const Data & data)
|
||||
{
|
||||
if (isUniq(func))
|
||||
{
|
||||
const FunctionFactory & function_factory = FunctionFactory::instance();
|
||||
|
||||
for (auto & arg : func.arguments->children)
|
||||
{
|
||||
while (removeInjectiveFunction(arg, data.context, function_factory))
|
||||
;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
bool RemoveInjectiveFunctionsMatcher::needChildVisit(const ASTPtr & node, const ASTPtr &)
|
||||
{
|
||||
if (node->as<ASTSubquery>() ||
|
||||
node->as<ASTTableExpression>())
|
||||
return false; // NOLINT
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
28
src/Interpreters/RemoveInjectiveFunctionsVisitor.h
Normal file
28
src/Interpreters/RemoveInjectiveFunctionsVisitor.h
Normal file
@ -0,0 +1,28 @@
|
||||
#pragma once
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Interpreters/InDepthNodeVisitor.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ASTFunction;
|
||||
|
||||
/// Removes unneded injective functions inside `uniq*()`.
|
||||
class RemoveInjectiveFunctionsMatcher
|
||||
{
|
||||
public:
|
||||
struct Data
|
||||
{
|
||||
const Context & context;
|
||||
};
|
||||
|
||||
static void visit(ASTPtr & ast, const Data & data);
|
||||
static void visit(ASTFunction &, ASTPtr & ast, const Data & data);
|
||||
|
||||
static bool needChildVisit(const ASTPtr & node, const ASTPtr & child);
|
||||
};
|
||||
|
||||
using RemoveInjectiveFunctionsVisitor = InDepthNodeVisitor<RemoveInjectiveFunctionsMatcher, true>;
|
||||
|
||||
}
|
@ -28,6 +28,7 @@
|
||||
#include <Interpreters/GroupByFunctionKeysVisitor.h>
|
||||
#include <Interpreters/AggregateFunctionOfGroupByKeysVisitor.h>
|
||||
#include <Interpreters/AnyInputOptimize.h>
|
||||
#include <Interpreters/RemoveInjectiveFunctionsVisitor.h>
|
||||
|
||||
#include <Parsers/ASTExpressionList.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
@ -601,6 +602,12 @@ void optimizeAnyInput(ASTPtr & query)
|
||||
AnyInputVisitor(data).visit(query);
|
||||
}
|
||||
|
||||
void optimizeInjectiveFunctionsInsideUniq(ASTPtr & query, const Context & context)
|
||||
{
|
||||
RemoveInjectiveFunctionsVisitor::Data data = {context};
|
||||
RemoveInjectiveFunctionsVisitor(data).visit(query);
|
||||
}
|
||||
|
||||
void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const ASTSelectQuery * select_query,
|
||||
const NamesAndTypesList & source_columns, const NameSet & source_columns_set)
|
||||
{
|
||||
@ -998,6 +1005,10 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyzeSelect(
|
||||
if (settings.optimize_move_functions_out_of_any)
|
||||
optimizeAnyInput(query);
|
||||
|
||||
/// Remove injective functions inside uniq
|
||||
if (settings.optimize_injective_functions_inside_uniq)
|
||||
optimizeInjectiveFunctionsInsideUniq(query, context);
|
||||
|
||||
/// Eliminate min/max/any aggregators of functions of GROUP BY keys
|
||||
if (settings.optimize_aggregators_of_group_by_keys)
|
||||
optimizeAggregateFunctionsOfGroupByKeys(select_query);
|
||||
|
@ -61,6 +61,7 @@ SRCS(
|
||||
HashJoin.cpp
|
||||
IdentifierSemantic.cpp
|
||||
IExternalLoadable.cpp
|
||||
RemoveInjectiveFunctionsVisitor.cpp
|
||||
InJoinSubqueriesPreprocessor.cpp
|
||||
inplaceBlockConversions.cpp
|
||||
InternalTextLogsQueue.cpp
|
||||
|
4
tests/performance/injective_functions_inside_uniq.xml
Normal file
4
tests/performance/injective_functions_inside_uniq.xml
Normal file
@ -0,0 +1,4 @@
|
||||
<test>
|
||||
<query>SELECT uniq(negate(negate(negate(number)))) FROM numbers(50000000)</query>
|
||||
<query>SELECT uniq(hex(hex(number))) FROM numbers(50000000)</query>
|
||||
</test>
|
@ -0,0 +1,152 @@
|
||||
SELECT
|
||||
uniq(x),
|
||||
uniqExact(x),
|
||||
uniqHLL12(x),
|
||||
uniqCombined(x),
|
||||
uniqCombined64(x)
|
||||
FROM
|
||||
(
|
||||
SELECT number % 2 AS x
|
||||
FROM numbers(10)
|
||||
)
|
||||
SELECT
|
||||
uniq(x + y),
|
||||
uniqExact(x + y),
|
||||
uniqHLL12(x + y),
|
||||
uniqCombined(x + y),
|
||||
uniqCombined64(x + y)
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
number % 2 AS x,
|
||||
number % 3 AS y
|
||||
FROM numbers(10)
|
||||
)
|
||||
SELECT
|
||||
uniq(x),
|
||||
uniqExact(x),
|
||||
uniqHLL12(x),
|
||||
uniqCombined(x),
|
||||
uniqCombined64(x)
|
||||
FROM
|
||||
(
|
||||
SELECT number % 2 AS x
|
||||
FROM numbers(10)
|
||||
)
|
||||
SELECT
|
||||
uniq(x),
|
||||
uniqExact(x),
|
||||
uniqHLL12(x),
|
||||
uniqCombined(x),
|
||||
uniqCombined64(x)
|
||||
FROM
|
||||
(
|
||||
SELECT number % 2 AS x
|
||||
FROM numbers(10)
|
||||
)
|
||||
SELECT
|
||||
uniq(x),
|
||||
uniqExact(x),
|
||||
uniqHLL12(x),
|
||||
uniqCombined(x),
|
||||
uniqCombined64(x)
|
||||
FROM
|
||||
(
|
||||
SELECT number % 2 AS x
|
||||
FROM numbers(10)
|
||||
)
|
||||
SELECT
|
||||
uniq(x),
|
||||
uniqExact(x),
|
||||
uniqHLL12(x),
|
||||
uniqCombined(x),
|
||||
uniqCombined64(x)
|
||||
FROM
|
||||
(
|
||||
SELECT number % 2 AS x
|
||||
FROM numbers(10)
|
||||
)
|
||||
SELECT uniqExact(x)
|
||||
FROM
|
||||
(
|
||||
SELECT number % 2 AS x
|
||||
FROM numbers(10)
|
||||
)
|
||||
SELECT uniq(concatAssumeInjective(\'x\', \'y\'))
|
||||
FROM numbers(10)
|
||||
SELECT
|
||||
uniq(x),
|
||||
uniqExact(x),
|
||||
uniqHLL12(x),
|
||||
uniqCombined(x),
|
||||
uniqCombined64(x)
|
||||
FROM
|
||||
(
|
||||
SELECT number % 2 AS x
|
||||
FROM numbers(10)
|
||||
)
|
||||
SELECT
|
||||
uniq(x + y),
|
||||
uniqExact(x + y),
|
||||
uniqHLL12(x + y),
|
||||
uniqCombined(x + y),
|
||||
uniqCombined64(x + y)
|
||||
FROM
|
||||
(
|
||||
SELECT
|
||||
number % 2 AS x,
|
||||
number % 3 AS y
|
||||
FROM numbers(10)
|
||||
)
|
||||
SELECT
|
||||
uniq(-x),
|
||||
uniqExact(-x),
|
||||
uniqHLL12(-x),
|
||||
uniqCombined(-x),
|
||||
uniqCombined64(-x)
|
||||
FROM
|
||||
(
|
||||
SELECT number % 2 AS x
|
||||
FROM numbers(10)
|
||||
)
|
||||
SELECT
|
||||
uniq(bitNot(x)),
|
||||
uniqExact(bitNot(x)),
|
||||
uniqHLL12(bitNot(x)),
|
||||
uniqCombined(bitNot(x)),
|
||||
uniqCombined64(bitNot(x))
|
||||
FROM
|
||||
(
|
||||
SELECT number % 2 AS x
|
||||
FROM numbers(10)
|
||||
)
|
||||
SELECT
|
||||
uniq(bitNot(-x)),
|
||||
uniqExact(bitNot(-x)),
|
||||
uniqHLL12(bitNot(-x)),
|
||||
uniqCombined(bitNot(-x)),
|
||||
uniqCombined64(bitNot(-x))
|
||||
FROM
|
||||
(
|
||||
SELECT number % 2 AS x
|
||||
FROM numbers(10)
|
||||
)
|
||||
SELECT
|
||||
uniq(-bitNot(-x)),
|
||||
uniqExact(-bitNot(-x)),
|
||||
uniqHLL12(-bitNot(-x)),
|
||||
uniqCombined(-bitNot(-x)),
|
||||
uniqCombined64(-bitNot(-x))
|
||||
FROM
|
||||
(
|
||||
SELECT number % 2 AS x
|
||||
FROM numbers(10)
|
||||
)
|
||||
SELECT uniqExact(-bitNot(-x))
|
||||
FROM
|
||||
(
|
||||
SELECT number % 2 AS x
|
||||
FROM numbers(10)
|
||||
)
|
||||
SELECT uniq(concatAssumeInjective(\'x\', \'y\'))
|
||||
FROM numbers(10)
|
47
tests/queries/0_stateless/01390_remove_injective_in_uniq.sql
Normal file
47
tests/queries/0_stateless/01390_remove_injective_in_uniq.sql
Normal file
@ -0,0 +1,47 @@
|
||||
set enable_debug_queries = 1;
|
||||
set optimize_injective_functions_inside_uniq = 1;
|
||||
|
||||
analyze select uniq(x), uniqExact(x), uniqHLL12(x), uniqCombined(x), uniqCombined64(x)
|
||||
from (select number % 2 as x from numbers(10));
|
||||
|
||||
analyze select uniq(x + y), uniqExact(x + y), uniqHLL12(x + y), uniqCombined(x + y), uniqCombined64(x + y)
|
||||
from (select number % 2 as x, number % 3 y from numbers(10));
|
||||
|
||||
analyze select uniq(-x), uniqExact(-x), uniqHLL12(-x), uniqCombined(-x), uniqCombined64(-x)
|
||||
from (select number % 2 as x from numbers(10));
|
||||
|
||||
analyze select uniq(bitNot(x)), uniqExact(bitNot(x)), uniqHLL12(bitNot(x)), uniqCombined(bitNot(x)), uniqCombined64(bitNot(x))
|
||||
from (select number % 2 as x from numbers(10));
|
||||
|
||||
analyze select uniq(bitNot(-x)), uniqExact(bitNot(-x)), uniqHLL12(bitNot(-x)), uniqCombined(bitNot(-x)), uniqCombined64(bitNot(-x))
|
||||
from (select number % 2 as x from numbers(10));
|
||||
|
||||
analyze select uniq(-bitNot(-x)), uniqExact(-bitNot(-x)), uniqHLL12(-bitNot(-x)), uniqCombined(-bitNot(-x)), uniqCombined64(-bitNot(-x))
|
||||
from (select number % 2 as x from numbers(10));
|
||||
|
||||
analyze select count(distinct -bitNot(-x)) from (select number % 2 as x from numbers(10));
|
||||
analyze select uniq(concatAssumeInjective('x', 'y')) from numbers(10);
|
||||
|
||||
|
||||
set optimize_injective_functions_inside_uniq = 0;
|
||||
|
||||
analyze select uniq(x), uniqExact(x), uniqHLL12(x), uniqCombined(x), uniqCombined64(x)
|
||||
from (select number % 2 as x from numbers(10));
|
||||
|
||||
analyze select uniq(x + y), uniqExact(x + y), uniqHLL12(x + y), uniqCombined(x + y), uniqCombined64(x + y)
|
||||
from (select number % 2 as x, number % 3 y from numbers(10));
|
||||
|
||||
analyze select uniq(-x), uniqExact(-x), uniqHLL12(-x), uniqCombined(-x), uniqCombined64(-x)
|
||||
from (select number % 2 as x from numbers(10));
|
||||
|
||||
analyze select uniq(bitNot(x)), uniqExact(bitNot(x)), uniqHLL12(bitNot(x)), uniqCombined(bitNot(x)), uniqCombined64(bitNot(x))
|
||||
from (select number % 2 as x from numbers(10));
|
||||
|
||||
analyze select uniq(bitNot(-x)), uniqExact(bitNot(-x)), uniqHLL12(bitNot(-x)), uniqCombined(bitNot(-x)), uniqCombined64(bitNot(-x))
|
||||
from (select number % 2 as x from numbers(10));
|
||||
|
||||
analyze select uniq(-bitNot(-x)), uniqExact(-bitNot(-x)), uniqHLL12(-bitNot(-x)), uniqCombined(-bitNot(-x)), uniqCombined64(-bitNot(-x))
|
||||
from (select number % 2 as x from numbers(10));
|
||||
|
||||
analyze select count(distinct -bitNot(-x)) from (select number % 2 as x from numbers(10));
|
||||
analyze select uniq(concatAssumeInjective('x', 'y')) from numbers(10);
|
Loading…
Reference in New Issue
Block a user