Optimize monotonous ORDER BY (#12467)

This commit is contained in:
Artem Zuikov 2020-07-15 13:10:21 +03:00 committed by GitHub
parent f8229e0bf7
commit 5f01a587f6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 477 additions and 0 deletions

View File

@ -372,6 +372,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, optimize_duplicate_order_by_and_distinct, true, "Remove duplicate ORDER BY and DISTINCT if it's possible", 0) \
M(SettingBool, optimize_redundant_functions_in_order_by, true, "Remove functions from ORDER BY if its argument is also in ORDER BY", 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) \
M(SettingBool, optimize_monotonous_functions_in_order_by, true, "Replace monotonous function with its argument in ORDER BY", 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) \
\

View File

@ -0,0 +1,142 @@
#pragma once
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <DataTypes/DataTypeFactory.h>
#include <Functions/FunctionFactory.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/InDepthNodeVisitor.h>
#include <Interpreters/IdentifierSemantic.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Parsers/ASTOrderByElement.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Parsers/IAST.h>
#include <Common/typeid_cast.h>
namespace DB
{
using Monotonicity = IFunctionBase::Monotonicity;
/// Checks from bottom to top if function composition is monotonous
class MonotonicityCheckMatcher
{
public:
struct Data
{
const TablesWithColumns & tables;
const Context & context;
const std::unordered_set<String> & group_by_function_hashes;
Monotonicity monotonicity{true, true, true};
ASTIdentifier * identifier = nullptr;
DataTypePtr arg_data_type = {};
void reject() { monotonicity.is_monotonic = false; }
bool isRejected() const { return !monotonicity.is_monotonic; }
bool canOptimize(const ASTFunction & ast_function) const
{
/// if GROUP BY contains the same function ORDER BY shouldn't be optimized
auto hash = ast_function.getTreeHash();
String key = toString(hash.first) + '_' + toString(hash.second);
if (group_by_function_hashes.count(key))
return false;
/// if ORDER BY contains aggregate function it shouldn't be optimized
if (AggregateFunctionFactory::instance().isAggregateFunctionName(ast_function.name))
return false;
return true;
}
bool extractIdentifierAndType(const ASTFunction & ast_function)
{
if (identifier)
return true;
identifier = ast_function.arguments->children[0]->as<ASTIdentifier>();
if (!identifier)
return false;
auto pos = IdentifierSemantic::getMembership(*identifier);
if (!pos)
pos = IdentifierSemantic::chooseTableColumnMatch(*identifier, tables, true);
if (!pos)
return false;
if (auto data_type_and_name = tables[*pos].columns.tryGetByName(identifier->shortName()))
{
arg_data_type = data_type_and_name->type;
return true;
}
return false;
}
};
static void visit(const ASTPtr & ast, Data & data)
{
if (const auto * ast_function = ast->as<ASTFunction>())
visit(*ast_function, data);
}
static void visit(const ASTFunction & ast_function, Data & data)
{
if (data.isRejected())
return;
/// TODO: monotonicity for fucntions of several arguments
auto arguments = ast_function.arguments;
if (arguments->children.size() != 1)
{
data.reject();
return;
}
if (!data.canOptimize(ast_function))
{
data.reject();
return;
}
const auto & function = FunctionFactory::instance().tryGet(ast_function.name, data.context);
if (!function)
{
data.reject();
return;
}
/// First time extract the most enclosed identifier and its data type
if (!data.arg_data_type && !data.extractIdentifierAndType(ast_function))
{
data.reject();
return;
}
ColumnsWithTypeAndName args;
args.emplace_back(data.arg_data_type, "tmp");
auto function_base = function->build(args);
if (function_base && function_base->hasInformationAboutMonotonicity())
{
bool is_positive = data.monotonicity.is_positive;
data.monotonicity = function_base->getMonotonicityForRange(*data.arg_data_type, Field(), Field());
if (!is_positive)
data.monotonicity.is_positive = !data.monotonicity.is_positive;
data.arg_data_type = function_base->getReturnType();
}
else
data.reject();
}
static bool needChildVisit(const ASTPtr &, const ASTPtr &)
{
return true;
}
};
using MonotonicityCheckVisitor = ConstInDepthNodeVisitor<MonotonicityCheckMatcher, false>;
}

View File

@ -30,6 +30,7 @@
#include <Interpreters/AnyInputOptimize.h>
#include <Interpreters/RemoveInjectiveFunctionsVisitor.h>
#include <Interpreters/RedundantFunctionsInOrderByVisitor.h>
#include <Interpreters/MonotonicityCheckVisitor.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
@ -529,6 +530,46 @@ void optimizeDuplicateOrderByAndDistinct(ASTPtr & query, const Context & context
DuplicateDistinctVisitor(distinct_data).visit(query);
}
/// 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)
{
auto order_by = select_query->orderBy();
if (!order_by)
return;
std::unordered_set<String> group_by_hashes;
if (auto group_by = select_query->groupBy())
{
for (auto & elem : group_by->children)
{
auto hash = elem->getTreeHash();
String key = toString(hash.first) + '_' + toString(hash.second);
group_by_hashes.insert(key);
}
}
for (auto & child : order_by->children)
{
auto * order_by_element = child->as<ASTOrderByElement>();
auto & ast_func = order_by_element->children[0];
if (!ast_func->as<ASTFunction>())
continue;
MonotonicityCheckVisitor::Data data{tables_with_columns, context, group_by_hashes};
MonotonicityCheckVisitor(data).visit(ast_func);
if (!data.isRejected())
{
ast_func = data.identifier->clone();
ast_func->setAlias("");
if (!data.monotonicity.is_positive)
order_by_element->direction *= -1;
}
}
}
/// If ORDER BY has argument x followed by f(x) transfroms it to ORDER BY x.
/// Optimize ORDER BY x, y, f(x), g(x, y), f(h(x)), t(f(x), g(x)) into ORDER BY x, y
/// in case if f(), g(), h(), t() are deterministic (in scope of query).
@ -1066,6 +1107,10 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyzeSelect(
if (settings.optimize_redundant_functions_in_order_by)
optimizeRedundantFunctionsInOrderBy(select_query, context);
/// Replace monotonous functions with its argument
if (settings.optimize_monotonous_functions_in_order_by)
optimizeMonotonousFunctionsInOrderBy(select_query, context, tables_with_columns);
/// Remove duplicated elements from LIMIT BY clause.
optimizeLimitBy(select_query);

View File

@ -0,0 +1,9 @@
<test>
<preconditions>
<table_exists>hits_10m_single</table_exists>
</preconditions>
<query>SELECT * FROM (SELECT CounterID, EventDate FROM hits_10m_single) ORDER BY toFloat32(toFloat64(toFloat32(toFloat64(CounterID)))) FORMAT Null</query>
<query>SELECT * FROM (SELECT CounterID, EventDate FROM hits_10m_single) ORDER BY toFloat32(toFloat64(toFloat32(toFloat64(CounterID)))) DESC, toFloat32(toFloat64(toFloat32(toFloat64(EventDate)))) ASC FORMAT Null</query>
</test>

View File

@ -0,0 +1,168 @@
0
1
2
0
1
2
0
1
2
2
1
0
0
1
2
0
1
2
0
1
2
2
1
0
2
1
0
2
1
0
0
1
2
2
1
0
2
1
0
SELECT number
FROM numbers(3)
ORDER BY number ASC
SELECT number
FROM numbers(3)
ORDER BY abs(toFloat32(number)) ASC
SELECT number
FROM numbers(3)
ORDER BY toFloat32(abs(number)) ASC
SELECT number
FROM numbers(3)
ORDER BY number DESC
SELECT number
FROM numbers(3)
ORDER BY exp(number) ASC
SELECT roundToExp2(number) AS x
FROM numbers(3)
ORDER BY
number ASC,
number ASC
SELECT number AS x
FROM numbers(3)
ORDER BY
number ASC,
number ASC
SELECT number
FROM numbers(3)
ORDER BY number DESC
SELECT number
FROM numbers(3)
ORDER BY abs(toFloat32(number)) DESC
SELECT number
FROM numbers(3)
ORDER BY toFloat32(abs(number)) DESC
SELECT number
FROM numbers(3)
ORDER BY number ASC
SELECT number
FROM numbers(3)
ORDER BY exp(number) DESC
SELECT roundToExp2(number) AS x
FROM numbers(3)
ORDER BY
number DESC,
number DESC
0
1
2
0
1
2
0
1
2
2
1
0
0
1
2
0
1
2
0
1
2
2
1
0
2
1
0
2
1
0
0
1
2
2
1
0
2
1
0
SELECT number
FROM numbers(3)
ORDER BY toFloat32(toFloat64(number)) ASC
SELECT number
FROM numbers(3)
ORDER BY abs(toFloat32(number)) ASC
SELECT number
FROM numbers(3)
ORDER BY toFloat32(abs(number)) ASC
SELECT number
FROM numbers(3)
ORDER BY -number ASC
SELECT number
FROM numbers(3)
ORDER BY exp(number) ASC
SELECT roundToExp2(number) AS x
FROM numbers(3)
ORDER BY
x ASC,
toFloat32(x) ASC
SELECT number AS x
FROM numbers(3)
ORDER BY
toFloat32(x) AS k ASC,
toFloat64(k) ASC
SELECT number
FROM numbers(3)
ORDER BY toFloat32(toFloat64(number)) DESC
SELECT number
FROM numbers(3)
ORDER BY abs(toFloat32(number)) DESC
SELECT number
FROM numbers(3)
ORDER BY toFloat32(abs(number)) DESC
SELECT number
FROM numbers(3)
ORDER BY -number DESC
SELECT number
FROM numbers(3)
ORDER BY exp(number) DESC
SELECT roundToExp2(number) AS x
FROM numbers(3)
ORDER BY
x DESC,
toFloat32(x) DESC

View File

@ -0,0 +1,59 @@
SET enable_debug_queries = 1;
SET optimize_monotonous_functions_in_order_by = 1;
SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number));
SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number));
SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number));
SELECT number FROM numbers(3) ORDER BY -number;
SELECT number FROM numbers(3) ORDER BY exp(number);
SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x, toFloat32(x);
SELECT number AS x FROM numbers(3) ORDER BY toFloat32(x) as k, toFloat64(k);
SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number)) DESC;
SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number)) DESC;
SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number)) DESC;
SELECT number FROM numbers(3) ORDER BY -number DESC;
SELECT number FROM numbers(3) ORDER BY exp(number) DESC;
SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x DESC, toFloat32(x) DESC;
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number));
analyze SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number));
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number));
analyze SELECT number FROM numbers(3) ORDER BY -number;
analyze SELECT number FROM numbers(3) ORDER BY exp(number);
analyze SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x, toFloat32(x);
analyze SELECT number AS x FROM numbers(3) ORDER BY toFloat32(x) as k, toFloat64(k);
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number)) DESC;
analyze SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number)) DESC;
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number)) DESC;
analyze SELECT number FROM numbers(3) ORDER BY -number DESC;
analyze SELECT number FROM numbers(3) ORDER BY exp(number) DESC;
analyze SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x DESC, toFloat32(x) DESC;
SET optimize_monotonous_functions_in_order_by = 0;
SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number));
SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number));
SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number));
SELECT number FROM numbers(3) ORDER BY -number;
SELECT number FROM numbers(3) ORDER BY exp(number);
SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x, toFloat32(x);
SELECT number AS x FROM numbers(3) ORDER BY toFloat32(x) as k, toFloat64(k);
SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number)) DESC;
SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number)) DESC;
SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number)) DESC;
SELECT number FROM numbers(3) ORDER BY -number DESC;
SELECT number FROM numbers(3) ORDER BY exp(number) DESC;
SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x DESC, toFloat32(x) DESC;
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number));
analyze SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number));
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number));
analyze SELECT number FROM numbers(3) ORDER BY -number;
analyze SELECT number FROM numbers(3) ORDER BY exp(number);
analyze SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x, toFloat32(x);
analyze SELECT number AS x FROM numbers(3) ORDER BY toFloat32(x) as k, toFloat64(k);
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(toFloat64(number)) DESC;
analyze SELECT number FROM numbers(3) ORDER BY abs(toFloat32(number)) DESC;
analyze SELECT number FROM numbers(3) ORDER BY toFloat32(abs(number)) DESC;
analyze SELECT number FROM numbers(3) ORDER BY -number DESC;
analyze SELECT number FROM numbers(3) ORDER BY exp(number) DESC;
analyze SELECT roundToExp2(number) AS x FROM numbers(3) ORDER BY x DESC, toFloat32(x) DESC;
-- TODO: exp() should be monotonous function

View File

@ -0,0 +1,32 @@
1 4 3
1 3 3
2 5 4
2 2 4
1 3 3
1 4 3
2 2 4
2 5 4
2
1
2
1 3 3
1 4 3
2 5 4
2 2 4
2
1 4 3
1 3 3
2 5 4
2 2 4
1 3 3
1 4 3
2 2 4
2 5 4
2
1
2
1 3 3
1 4 3
2 5 4
2 2 4
2

View File

@ -0,0 +1,21 @@
DROP TABLE IF EXISTS test;
CREATE TABLE test (x Int8, y Int8, z Int8) ENGINE = MergeTree ORDER BY tuple();
INSERT INTO test VALUES (1, 3, 3), (1, 4, 3), (2, 5, 4), (2, 2, 4);
SET optimize_monotonous_functions_in_order_by = 1;
SELECT * FROM test ORDER BY toFloat32(x), -y, -z DESC;
SELECT * FROM test ORDER BY toFloat32(x), -(-y), -z DESC;
SELECT max(x) as k FROM test ORDER BY k;
SELECT roundToExp2(x) as k FROM test GROUP BY k ORDER BY k;
SELECT roundToExp2(x) as k, y, z FROM test WHERE k >= 1 ORDER BY k;
SELECT max(x) as k FROM test HAVING k > 0 ORDER BY k;
SET optimize_monotonous_functions_in_order_by = 0;
SELECT * FROM test ORDER BY toFloat32(x), -y, -z DESC;
SELECT * FROM test ORDER BY toFloat32(x), -(-y), -z DESC;
SELECT max(x) as k FROM test ORDER BY k;
SELECT roundToExp2(x) as k From test GROUP BY k ORDER BY k;
SELECT roundToExp2(x) as k, y, z FROM test WHERE k >= 1 ORDER BY k;
SELECT max(x) as k FROM test HAVING k > 0 ORDER BY k;
DROP TABLE test;