mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-27 01:51:59 +00:00
Merge pull request #38950 from hexiaoting/dev-prewhere
Support optimize `where` clause with sorting key expression move to `prewhere` for query with` final `
This commit is contained in:
commit
b1524196c6
@ -77,8 +77,17 @@ public:
|
|||||||
|
|
||||||
return std::make_shared<DataTypeUInt8>();
|
return std::make_shared<DataTypeUInt8>();
|
||||||
}
|
}
|
||||||
|
ColumnPtr executeImplDryRun(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
|
||||||
|
{
|
||||||
|
return execute(arguments, result_type, true);
|
||||||
|
}
|
||||||
|
|
||||||
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
|
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t /*input_rows_count*/) const override
|
||||||
|
{
|
||||||
|
return execute(arguments, result_type, false);
|
||||||
|
}
|
||||||
|
|
||||||
|
ColumnPtr execute(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, bool dry_run) const
|
||||||
{
|
{
|
||||||
const IColumn * col = arguments[0].column.get();
|
const IColumn * col = arguments[0].column.get();
|
||||||
|
|
||||||
@ -99,11 +108,14 @@ public:
|
|||||||
if (seconds > 3.0) /// The choice is arbitrary
|
if (seconds > 3.0) /// The choice is arbitrary
|
||||||
throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is 3 seconds. Requested: {}", toString(seconds));
|
throw Exception(ErrorCodes::TOO_SLOW, "The maximum sleep time is 3 seconds. Requested: {}", toString(seconds));
|
||||||
|
|
||||||
UInt64 count = (variant == FunctionSleepVariant::PerBlock ? 1 : size);
|
if (!dry_run)
|
||||||
UInt64 microseconds = static_cast<UInt64>(seconds * count * 1e6);
|
{
|
||||||
sleepForMicroseconds(microseconds);
|
UInt64 count = (variant == FunctionSleepVariant::PerBlock ? 1 : size);
|
||||||
ProfileEvents::increment(ProfileEvents::SleepFunctionCalls, count);
|
UInt64 microseconds = static_cast<UInt64>(seconds * count * 1e6);
|
||||||
ProfileEvents::increment(ProfileEvents::SleepFunctionMicroseconds, microseconds);
|
sleepForMicroseconds(microseconds);
|
||||||
|
ProfileEvents::increment(ProfileEvents::SleepFunctionCalls, count);
|
||||||
|
ProfileEvents::increment(ProfileEvents::SleepFunctionMicroseconds, microseconds);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
/// convertToFullColumn needed, because otherwise (constant expression case) function will not get called on each columns.
|
/// convertToFullColumn needed, because otherwise (constant expression case) function will not get called on each columns.
|
||||||
|
@ -42,10 +42,6 @@ MergeTreeWhereOptimizer::MergeTreeWhereOptimizer(
|
|||||||
, log{log_}
|
, log{log_}
|
||||||
, column_sizes{std::move(column_sizes_)}
|
, column_sizes{std::move(column_sizes_)}
|
||||||
{
|
{
|
||||||
const auto & primary_key = metadata_snapshot->getPrimaryKey();
|
|
||||||
if (!primary_key.column_names.empty())
|
|
||||||
first_primary_key_column = primary_key.column_names[0];
|
|
||||||
|
|
||||||
for (const auto & name : queried_columns)
|
for (const auto & name : queried_columns)
|
||||||
{
|
{
|
||||||
auto it = column_sizes.find(name);
|
auto it = column_sizes.find(name);
|
||||||
@ -193,8 +189,9 @@ void MergeTreeWhereOptimizer::analyzeImpl(Conditions & res, const ASTPtr & node,
|
|||||||
/// Condition depend on some column. Constant expressions are not moved.
|
/// Condition depend on some column. Constant expressions are not moved.
|
||||||
!cond.identifiers.empty()
|
!cond.identifiers.empty()
|
||||||
&& !cannotBeMoved(node, is_final)
|
&& !cannotBeMoved(node, is_final)
|
||||||
/// Do not take into consideration the conditions consisting only of the first primary key column
|
/// When use final, do not take into consideration the conditions with non-sorting keys. Because final select
|
||||||
&& !hasPrimaryKeyAtoms(node)
|
/// need to use all sorting keys, it will cause correctness issues if we filter other columns before final merge.
|
||||||
|
&& (!is_final || isExpressionOverSortingKey(node))
|
||||||
/// Only table columns are considered. Not array joined columns. NOTE We're assuming that aliases was expanded.
|
/// Only table columns are considered. Not array joined columns. NOTE We're assuming that aliases was expanded.
|
||||||
&& isSubsetOfTableColumns(cond.identifiers)
|
&& isSubsetOfTableColumns(cond.identifiers)
|
||||||
/// Do not move conditions involving all queried columns.
|
/// Do not move conditions involving all queried columns.
|
||||||
@ -320,48 +317,22 @@ UInt64 MergeTreeWhereOptimizer::getIdentifiersColumnSize(const NameSet & identif
|
|||||||
return size;
|
return size;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
bool MergeTreeWhereOptimizer::isExpressionOverSortingKey(const ASTPtr & ast) const
|
||||||
bool MergeTreeWhereOptimizer::hasPrimaryKeyAtoms(const ASTPtr & ast) const
|
|
||||||
{
|
{
|
||||||
if (const auto * func = ast->as<ASTFunction>())
|
if (const auto * func = ast->as<ASTFunction>())
|
||||||
{
|
{
|
||||||
const auto & args = func->arguments->children;
|
const auto & args = func->arguments->children;
|
||||||
|
for (const auto & arg : args)
|
||||||
if ((func->name == "not" && 1 == args.size()) || func->name == "and" || func->name == "or")
|
|
||||||
{
|
{
|
||||||
for (const auto & arg : args)
|
if (isConstant(ast) || sorting_key_names.contains(arg->getColumnName()))
|
||||||
if (hasPrimaryKeyAtoms(arg))
|
continue;
|
||||||
return true;
|
if (!isExpressionOverSortingKey(arg))
|
||||||
|
return false;
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
return isPrimaryKeyAtom(ast);
|
return isConstant(ast) || sorting_key_names.contains(ast->getColumnName());
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const ASTPtr & ast) const
|
|
||||||
{
|
|
||||||
if (const auto * func = ast->as<ASTFunction>())
|
|
||||||
{
|
|
||||||
if (!KeyCondition::atom_map.contains(func->name))
|
|
||||||
return false;
|
|
||||||
|
|
||||||
const auto & args = func->arguments->children;
|
|
||||||
if (args.size() != 2)
|
|
||||||
return false;
|
|
||||||
|
|
||||||
const auto & first_arg_name = args.front()->getColumnName();
|
|
||||||
const auto & second_arg_name = args.back()->getColumnName();
|
|
||||||
|
|
||||||
if ((first_primary_key_column == first_arg_name && isConstant(args[1]))
|
|
||||||
|| (first_primary_key_column == second_arg_name && isConstant(args[0]))
|
|
||||||
|| (first_primary_key_column == first_arg_name && functionIsInOrGlobalInOperator(func->name)))
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
return false;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -83,9 +83,7 @@ private:
|
|||||||
|
|
||||||
UInt64 getIdentifiersColumnSize(const NameSet & identifiers) const;
|
UInt64 getIdentifiersColumnSize(const NameSet & identifiers) const;
|
||||||
|
|
||||||
bool hasPrimaryKeyAtoms(const ASTPtr & ast) const;
|
bool isExpressionOverSortingKey(const ASTPtr & ast) const;
|
||||||
|
|
||||||
bool isPrimaryKeyAtom(const ASTPtr & ast) const;
|
|
||||||
|
|
||||||
bool isSortingKey(const String & column_name) const;
|
bool isSortingKey(const String & column_name) const;
|
||||||
|
|
||||||
@ -105,7 +103,6 @@ private:
|
|||||||
|
|
||||||
using StringSet = std::unordered_set<std::string>;
|
using StringSet = std::unordered_set<std::string>;
|
||||||
|
|
||||||
String first_primary_key_column;
|
|
||||||
const StringSet table_columns;
|
const StringSet table_columns;
|
||||||
const Names queried_columns;
|
const Names queried_columns;
|
||||||
const NameSet sorting_key_names;
|
const NameSet sorting_key_names;
|
||||||
|
@ -179,7 +179,7 @@ FROM
|
|||||||
name,
|
name,
|
||||||
value
|
value
|
||||||
FROM test_00597
|
FROM test_00597
|
||||||
WHERE id = 1
|
PREWHERE id = 1
|
||||||
)
|
)
|
||||||
WHERE id = 1
|
WHERE id = 1
|
||||||
2000-01-01 1 test string 1 1
|
2000-01-01 1 test string 1 1
|
||||||
@ -203,7 +203,7 @@ FROM
|
|||||||
name,
|
name,
|
||||||
value
|
value
|
||||||
FROM test_00597
|
FROM test_00597
|
||||||
WHERE id = 1
|
PREWHERE id = 1
|
||||||
)
|
)
|
||||||
WHERE id = 1
|
WHERE id = 1
|
||||||
)
|
)
|
||||||
@ -229,7 +229,7 @@ FROM
|
|||||||
name,
|
name,
|
||||||
value
|
value
|
||||||
FROM test_00597
|
FROM test_00597
|
||||||
WHERE id = 1
|
PREWHERE id = 1
|
||||||
) AS b
|
) AS b
|
||||||
WHERE id = 1
|
WHERE id = 1
|
||||||
)
|
)
|
||||||
@ -248,7 +248,7 @@ FROM
|
|||||||
name,
|
name,
|
||||||
value
|
value
|
||||||
FROM test_00597
|
FROM test_00597
|
||||||
WHERE id = 1
|
PREWHERE id = 1
|
||||||
)
|
)
|
||||||
WHERE id = 1
|
WHERE id = 1
|
||||||
2000-01-01 1 test string 1 1
|
2000-01-01 1 test string 1 1
|
||||||
@ -272,7 +272,7 @@ FROM
|
|||||||
name,
|
name,
|
||||||
value
|
value
|
||||||
FROM test_00597
|
FROM test_00597
|
||||||
WHERE id = 1
|
PREWHERE id = 1
|
||||||
)
|
)
|
||||||
WHERE id = 1
|
WHERE id = 1
|
||||||
)
|
)
|
||||||
@ -291,7 +291,7 @@ FROM
|
|||||||
name,
|
name,
|
||||||
value
|
value
|
||||||
FROM test_00597
|
FROM test_00597
|
||||||
WHERE id = 1
|
PREWHERE id = 1
|
||||||
) AS b
|
) AS b
|
||||||
WHERE id = 1
|
WHERE id = 1
|
||||||
2000-01-01 1 test string 1 1
|
2000-01-01 1 test string 1 1
|
||||||
@ -315,7 +315,7 @@ FROM
|
|||||||
name,
|
name,
|
||||||
value
|
value
|
||||||
FROM test_00597
|
FROM test_00597
|
||||||
WHERE id = 1
|
PREWHERE id = 1
|
||||||
) AS a
|
) AS a
|
||||||
WHERE id = 1
|
WHERE id = 1
|
||||||
) AS b
|
) AS b
|
||||||
@ -332,7 +332,7 @@ FROM
|
|||||||
date,
|
date,
|
||||||
min(value) AS value
|
min(value) AS value
|
||||||
FROM test_00597
|
FROM test_00597
|
||||||
WHERE id = 1
|
PREWHERE id = 1
|
||||||
GROUP BY
|
GROUP BY
|
||||||
id,
|
id,
|
||||||
date
|
date
|
||||||
@ -352,7 +352,7 @@ FROM
|
|||||||
name,
|
name,
|
||||||
value
|
value
|
||||||
FROM test_00597
|
FROM test_00597
|
||||||
WHERE id = 1
|
PREWHERE id = 1
|
||||||
UNION ALL
|
UNION ALL
|
||||||
SELECT
|
SELECT
|
||||||
date,
|
date,
|
||||||
@ -360,7 +360,7 @@ FROM
|
|||||||
name,
|
name,
|
||||||
value
|
value
|
||||||
FROM test_00597
|
FROM test_00597
|
||||||
WHERE id = 1
|
PREWHERE id = 1
|
||||||
)
|
)
|
||||||
WHERE id = 1
|
WHERE id = 1
|
||||||
2000-01-01 1 test string 1 1
|
2000-01-01 1 test string 1 1
|
||||||
@ -381,7 +381,7 @@ FROM
|
|||||||
name,
|
name,
|
||||||
value
|
value
|
||||||
FROM test_00597
|
FROM test_00597
|
||||||
WHERE id = 1
|
PREWHERE id = 1
|
||||||
)
|
)
|
||||||
ANY LEFT JOIN
|
ANY LEFT JOIN
|
||||||
(
|
(
|
||||||
@ -441,7 +441,7 @@ FROM
|
|||||||
name,
|
name,
|
||||||
value
|
value
|
||||||
FROM test_00597
|
FROM test_00597
|
||||||
WHERE id = 1
|
PREWHERE id = 1
|
||||||
)
|
)
|
||||||
ANY LEFT JOIN
|
ANY LEFT JOIN
|
||||||
(
|
(
|
||||||
@ -532,7 +532,7 @@ FROM
|
|||||||
name,
|
name,
|
||||||
value
|
value
|
||||||
FROM test_00597
|
FROM test_00597
|
||||||
WHERE id = 1
|
PREWHERE id = 1
|
||||||
) AS a
|
) AS a
|
||||||
ANY LEFT JOIN
|
ANY LEFT JOIN
|
||||||
(
|
(
|
||||||
@ -579,7 +579,7 @@ SEMI LEFT JOIN
|
|||||||
name,
|
name,
|
||||||
value
|
value
|
||||||
FROM test_00597
|
FROM test_00597
|
||||||
WHERE id = 1
|
PREWHERE id = 1
|
||||||
)
|
)
|
||||||
WHERE id = 1
|
WHERE id = 1
|
||||||
) AS r USING (id)
|
) AS r USING (id)
|
||||||
|
@ -24,6 +24,6 @@ FROM
|
|||||||
n,
|
n,
|
||||||
finalizeAggregation(s)
|
finalizeAggregation(s)
|
||||||
FROM test_00808_push_down_with_finalizeAggregation
|
FROM test_00808_push_down_with_finalizeAggregation
|
||||||
WHERE (n <= 5) AND (n >= 2)
|
PREWHERE (n <= 5) AND (n >= 2)
|
||||||
)
|
)
|
||||||
WHERE (n >= 2) AND (n <= 5)
|
WHERE (n >= 2) AND (n <= 5)
|
||||||
|
@ -293,8 +293,8 @@ select * from (select * from tab where (a + b) * c = 8 union all select * from t
|
|||||||
select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab3 where (a + b) * c = 18) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%';
|
select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab3 where (a + b) * c = 18) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%';
|
||||||
Prefix sort description: sin(divide(a, b)) ASC
|
Prefix sort description: sin(divide(a, b)) ASC
|
||||||
Result sort description: sin(divide(a, b)) ASC
|
Result sort description: sin(divide(a, b)) ASC
|
||||||
ReadType: InOrder
|
ReadType: InOrder
|
||||||
ReadType: InOrder
|
ReadType: InOrder
|
||||||
select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b);
|
select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b);
|
||||||
2 2 2 2
|
2 2 2 2
|
||||||
2 2 2 2
|
2 2 2 2
|
||||||
@ -311,7 +311,7 @@ select * from (select * from tab where (a + b) * c = 8 union all select * from t
|
|||||||
select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%';
|
select * from (explain plan actions = 1 select * from (select * from tab where (a + b) * c = 8 union all select * from tab4) order by sin(a / b)) where explain like '%sort description%' or explain like '%ReadType%';
|
||||||
Prefix sort description: sin(divide(a, b)) ASC
|
Prefix sort description: sin(divide(a, b)) ASC
|
||||||
Result sort description: sin(divide(a, b)) ASC
|
Result sort description: sin(divide(a, b)) ASC
|
||||||
ReadType: InOrder
|
ReadType: InOrder
|
||||||
ReadType: InOrder
|
ReadType: InOrder
|
||||||
select * from (select * from tab union all select * from tab5) order by (a + b) * c;
|
select * from (select * from tab union all select * from tab5) order by (a + b) * c;
|
||||||
0 0 0 0
|
0 0 0 0
|
||||||
@ -403,3 +403,8 @@ select * from (explain plan actions = 1 select * from (select * from tab union a
|
|||||||
Sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC, d ASC
|
Sort description: multiply(plus(a, b), c) ASC, sin(divide(a, b)) ASC, d ASC
|
||||||
Limit 3
|
Limit 3
|
||||||
ReadType: Default
|
ReadType: Default
|
||||||
|
drop table if exists tab;
|
||||||
|
drop table if exists tab2;
|
||||||
|
drop table if exists tab3;
|
||||||
|
drop table if exists tab4;
|
||||||
|
drop table if exists tab5;
|
||||||
|
@ -1,5 +1,11 @@
|
|||||||
SET optimize_read_in_order = 1, query_plan_read_in_order=1;
|
SET optimize_read_in_order = 1, query_plan_read_in_order=1;
|
||||||
|
|
||||||
|
drop table if exists tab;
|
||||||
|
drop table if exists tab2;
|
||||||
|
drop table if exists tab3;
|
||||||
|
drop table if exists tab4;
|
||||||
|
drop table if exists tab5;
|
||||||
|
|
||||||
create table tab (a UInt32, b UInt32, c UInt32, d UInt32) engine = MergeTree order by ((a + b) * c, sin(a / b));
|
create table tab (a UInt32, b UInt32, c UInt32, d UInt32) engine = MergeTree order by ((a + b) * c, sin(a / b));
|
||||||
insert into tab select number, number, number, number from numbers(5);
|
insert into tab select number, number, number, number from numbers(5);
|
||||||
insert into tab select number, number, number, number from numbers(5);
|
insert into tab select number, number, number, number from numbers(5);
|
||||||
@ -142,3 +148,9 @@ select * from (explain plan actions = 1 select * from (select * from tab union a
|
|||||||
-- In case of tab4, we do full sorting by ((a + b) * c, sin(a / b), d) with LIMIT. We can replace it to sorting by ((a + b) * c, sin(a / b)) and LIMIT WITH TIES, when sorting alog support it.
|
-- In case of tab4, we do full sorting by ((a + b) * c, sin(a / b), d) with LIMIT. We can replace it to sorting by ((a + b) * c, sin(a / b)) and LIMIT WITH TIES, when sorting alog support it.
|
||||||
select * from (select * from tab union all select * from tab5 union all select * from tab4) order by (a + b) * c, sin(a / b), d limit 3;
|
select * from (select * from tab union all select * from tab5 union all select * from tab4) order by (a + b) * c, sin(a / b), d limit 3;
|
||||||
select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5 union all select * from tab4) order by (a + b) * c, sin(a / b), d limit 3) where explain ilike '%sort description%' or explain like '%ReadType%' or explain like '%Limit%';
|
select * from (explain plan actions = 1 select * from (select * from tab union all select * from tab5 union all select * from tab4) order by (a + b) * c, sin(a / b), d limit 3) where explain ilike '%sort description%' or explain like '%ReadType%' or explain like '%Limit%';
|
||||||
|
|
||||||
|
drop table if exists tab;
|
||||||
|
drop table if exists tab2;
|
||||||
|
drop table if exists tab3;
|
||||||
|
drop table if exists tab4;
|
||||||
|
drop table if exists tab5;
|
||||||
|
@ -1,5 +1,20 @@
|
|||||||
optimize_move_to_prewhere_if_final = 1
|
optimize_move_to_prewhere_if_final = 1
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
x,
|
||||||
|
y,
|
||||||
|
z
|
||||||
|
FROM prewhere_move_select_final
|
||||||
|
PREWHERE x > 100
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
x,
|
||||||
|
y,
|
||||||
|
z
|
||||||
|
FROM prewhere_move_select_final
|
||||||
|
FINAL
|
||||||
|
PREWHERE x > 100
|
||||||
|
|
||||||
SELECT
|
SELECT
|
||||||
x,
|
x,
|
||||||
y,
|
y,
|
||||||
@ -15,6 +30,21 @@ FROM prewhere_move_select_final
|
|||||||
FINAL
|
FINAL
|
||||||
PREWHERE y > 100
|
PREWHERE y > 100
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
x,
|
||||||
|
y,
|
||||||
|
z
|
||||||
|
FROM prewhere_move_select_final
|
||||||
|
PREWHERE (x + y) > 100
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
x,
|
||||||
|
y,
|
||||||
|
z
|
||||||
|
FROM prewhere_move_select_final
|
||||||
|
FINAL
|
||||||
|
PREWHERE (x + y) > 100
|
||||||
|
|
||||||
SELECT
|
SELECT
|
||||||
x,
|
x,
|
||||||
y,
|
y,
|
||||||
@ -32,6 +62,24 @@ FINAL
|
|||||||
PREWHERE y > 100
|
PREWHERE y > 100
|
||||||
WHERE (y > 100) AND (z > 400)
|
WHERE (y > 100) AND (z > 400)
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
x,
|
||||||
|
y,
|
||||||
|
z
|
||||||
|
FROM prewhere_move_select_final
|
||||||
|
FINAL
|
||||||
|
PREWHERE x > 50
|
||||||
|
WHERE (x > 50) AND (z > 400)
|
||||||
|
|
||||||
|
SELECT
|
||||||
|
x,
|
||||||
|
y,
|
||||||
|
z
|
||||||
|
FROM prewhere_move_select_final
|
||||||
|
FINAL
|
||||||
|
PREWHERE (x + y) > 50
|
||||||
|
WHERE ((x + y) > 50) AND (z > 400)
|
||||||
|
|
||||||
optimize_move_to_prewhere_if_final = 0
|
optimize_move_to_prewhere_if_final = 0
|
||||||
|
|
||||||
SELECT
|
SELECT
|
||||||
|
@ -11,17 +11,29 @@ SET optimize_move_to_prewhere_if_final = 1;
|
|||||||
|
|
||||||
-- order key can be pushed down with final
|
-- order key can be pushed down with final
|
||||||
select '';
|
select '';
|
||||||
|
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final WHERE x > 100;
|
||||||
|
select '';
|
||||||
|
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final FINAL WHERE x > 100;
|
||||||
|
select '';
|
||||||
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final WHERE y > 100;
|
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final WHERE y > 100;
|
||||||
select '';
|
select '';
|
||||||
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final FINAL WHERE y > 100;
|
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final FINAL WHERE y > 100;
|
||||||
|
select '';
|
||||||
|
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final WHERE x + y > 100;
|
||||||
|
select '';
|
||||||
|
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final FINAL WHERE x + y > 100;
|
||||||
|
|
||||||
-- can not be pushed down
|
-- can not be pushed down
|
||||||
select '';
|
select '';
|
||||||
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final FINAL WHERE z > 400;
|
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final FINAL WHERE z > 400;
|
||||||
|
|
||||||
-- only y can be pushed down
|
-- only condition with x/y can be pushed down
|
||||||
select '';
|
select '';
|
||||||
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final FINAL WHERE y > 100 and z > 400;
|
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final FINAL WHERE y > 100 and z > 400;
|
||||||
|
select '';
|
||||||
|
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final FINAL WHERE x > 50 and z > 400;
|
||||||
|
select '';
|
||||||
|
EXPLAIN SYNTAX SELECT * FROM prewhere_move_select_final FINAL WHERE x + y > 50 and z > 400;
|
||||||
|
|
||||||
select '';
|
select '';
|
||||||
select 'optimize_move_to_prewhere_if_final = 0';
|
select 'optimize_move_to_prewhere_if_final = 0';
|
||||||
|
@ -97,12 +97,12 @@
|
|||||||
ReadType: InReverseOrder
|
ReadType: InReverseOrder
|
||||||
Parts: 1
|
Parts: 1
|
||||||
Granules: 3
|
Granules: 3
|
||||||
ReadFromMergeTree (default.idx)
|
ReadFromMergeTree (default.idx)
|
||||||
Indexes:
|
Indexes:
|
||||||
PrimaryKey
|
PrimaryKey
|
||||||
Keys:
|
Keys:
|
||||||
x
|
x
|
||||||
plus(x, y)
|
plus(x, y)
|
||||||
Condition: or((x in 2-element set), (plus(plus(x, y), 1) in (-Inf, 2]))
|
Condition: or((x in 2-element set), (plus(plus(x, y), 1) in (-Inf, 2]))
|
||||||
Parts: 1/1
|
Parts: 1/1
|
||||||
Granules: 1/1
|
Granules: 1/1
|
||||||
|
@ -64,10 +64,8 @@ ExpressionTransform
|
|||||||
(Sorting)
|
(Sorting)
|
||||||
(Expression)
|
(Expression)
|
||||||
ExpressionTransform
|
ExpressionTransform
|
||||||
(Filter)
|
(ReadFromMergeTree)
|
||||||
FilterTransform
|
MergeTreeInOrder 0 → 1
|
||||||
(ReadFromMergeTree)
|
|
||||||
MergeTreeInOrder 0 → 1
|
|
||||||
2020-10-11 0 0
|
2020-10-11 0 0
|
||||||
2020-10-11 0 10
|
2020-10-11 0 10
|
||||||
2020-10-11 0 20
|
2020-10-11 0 20
|
||||||
@ -82,15 +80,20 @@ ExpressionTransform
|
|||||||
PartialSortingTransform
|
PartialSortingTransform
|
||||||
(Expression)
|
(Expression)
|
||||||
ExpressionTransform
|
ExpressionTransform
|
||||||
(Filter)
|
(ReadFromMergeTree)
|
||||||
FilterTransform
|
MergeTreeInOrder 0 → 1
|
||||||
(ReadFromMergeTree)
|
|
||||||
MergeTreeInOrder 0 → 1
|
|
||||||
2020-10-12 0
|
2020-10-12 0
|
||||||
2020-10-12 1
|
2020-10-12 1
|
||||||
2020-10-12 2
|
2020-10-12 2
|
||||||
2020-10-12 3
|
2020-10-12 3
|
||||||
2020-10-12 4
|
2020-10-12 4
|
||||||
|
SELECT
|
||||||
|
date,
|
||||||
|
i
|
||||||
|
FROM t_read_in_order
|
||||||
|
PREWHERE date = \'2020-10-12\'
|
||||||
|
ORDER BY i DESC
|
||||||
|
LIMIT 5
|
||||||
(Expression)
|
(Expression)
|
||||||
ExpressionTransform
|
ExpressionTransform
|
||||||
(Limit)
|
(Limit)
|
||||||
@ -98,11 +101,9 @@ ExpressionTransform
|
|||||||
(Sorting)
|
(Sorting)
|
||||||
(Expression)
|
(Expression)
|
||||||
ExpressionTransform
|
ExpressionTransform
|
||||||
(Filter)
|
(ReadFromMergeTree)
|
||||||
FilterTransform
|
ReverseTransform
|
||||||
(ReadFromMergeTree)
|
MergeTreeReverse 0 → 1
|
||||||
ReverseTransform
|
|
||||||
MergeTreeReverse 0 → 1
|
|
||||||
2020-10-12 99999
|
2020-10-12 99999
|
||||||
2020-10-12 99998
|
2020-10-12 99998
|
||||||
2020-10-12 99997
|
2020-10-12 99997
|
||||||
|
@ -30,6 +30,7 @@ INSERT INTO t_read_in_order SELECT '2020-10-12', number, number FROM numbers(100
|
|||||||
|
|
||||||
SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i LIMIT 5;
|
SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i LIMIT 5;
|
||||||
|
|
||||||
|
EXPLAIN SYNTAX SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i DESC LIMIT 5;
|
||||||
EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i DESC LIMIT 5;
|
EXPLAIN PIPELINE SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i DESC LIMIT 5;
|
||||||
SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i DESC LIMIT 5;
|
SELECT date, i FROM t_read_in_order WHERE date = '2020-10-12' ORDER BY i DESC LIMIT 5;
|
||||||
|
|
||||||
|
@ -55,7 +55,6 @@ MergeTreeThread
|
|||||||
Sorting (Stream): a ASC, b ASC
|
Sorting (Stream): a ASC, b ASC
|
||||||
Sorting (Stream): a ASC, b ASC
|
Sorting (Stream): a ASC, b ASC
|
||||||
Sorting (Stream): a ASC, b ASC
|
Sorting (Stream): a ASC, b ASC
|
||||||
Sorting (Stream): a ASC, b ASC
|
|
||||||
-- check that reading in order optimization for ORDER BY and DISTINCT applied correctly in the same query
|
-- check that reading in order optimization for ORDER BY and DISTINCT applied correctly in the same query
|
||||||
-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns
|
-- disabled, check that sorting description for ReadFromMergeTree match ORDER BY columns
|
||||||
Sorting (Stream): a ASC
|
Sorting (Stream): a ASC
|
||||||
|
Loading…
Reference in New Issue
Block a user