mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-18 04:12:19 +00:00
Optimize monotonous ORDER BY (#12467)
This commit is contained in:
parent
f8229e0bf7
commit
5f01a587f6
@ -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_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_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_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, 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) \
|
||||||
\
|
\
|
||||||
|
142
src/Interpreters/MonotonicityCheckVisitor.h
Normal file
142
src/Interpreters/MonotonicityCheckVisitor.h
Normal 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>;
|
||||||
|
|
||||||
|
}
|
@ -30,6 +30,7 @@
|
|||||||
#include <Interpreters/AnyInputOptimize.h>
|
#include <Interpreters/AnyInputOptimize.h>
|
||||||
#include <Interpreters/RemoveInjectiveFunctionsVisitor.h>
|
#include <Interpreters/RemoveInjectiveFunctionsVisitor.h>
|
||||||
#include <Interpreters/RedundantFunctionsInOrderByVisitor.h>
|
#include <Interpreters/RedundantFunctionsInOrderByVisitor.h>
|
||||||
|
#include <Interpreters/MonotonicityCheckVisitor.h>
|
||||||
|
|
||||||
#include <Parsers/ASTExpressionList.h>
|
#include <Parsers/ASTExpressionList.h>
|
||||||
#include <Parsers/ASTFunction.h>
|
#include <Parsers/ASTFunction.h>
|
||||||
@ -529,6 +530,46 @@ void optimizeDuplicateOrderByAndDistinct(ASTPtr & query, const Context & context
|
|||||||
DuplicateDistinctVisitor(distinct_data).visit(query);
|
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.
|
/// 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
|
/// 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).
|
/// 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)
|
if (settings.optimize_redundant_functions_in_order_by)
|
||||||
optimizeRedundantFunctionsInOrderBy(select_query, context);
|
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.
|
/// Remove duplicated elements from LIMIT BY clause.
|
||||||
optimizeLimitBy(select_query);
|
optimizeLimitBy(select_query);
|
||||||
|
|
||||||
|
9
tests/performance/monotonous_order_by.xml
Normal file
9
tests/performance/monotonous_order_by.xml
Normal 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>
|
@ -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
|
@ -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
|
@ -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
|
@ -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;
|
Loading…
Reference in New Issue
Block a user