mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-29 02:52:13 +00:00
Merge pull request #55836 from azat/dist/limit-by-fix
RFC: Fix "Cannot find column X in source stream" for Distributed queries with LIMIT BY
This commit is contained in:
commit
f8ebe5134d
@ -3,6 +3,7 @@
|
|||||||
from argparse import ArgumentParser
|
from argparse import ArgumentParser
|
||||||
import os
|
import os
|
||||||
import jinja2
|
import jinja2
|
||||||
|
import itertools
|
||||||
|
|
||||||
|
|
||||||
def removesuffix(text, suffix):
|
def removesuffix(text, suffix):
|
||||||
@ -47,6 +48,7 @@ def main(args):
|
|||||||
loader=jinja2.FileSystemLoader(suite_dir),
|
loader=jinja2.FileSystemLoader(suite_dir),
|
||||||
keep_trailing_newline=True,
|
keep_trailing_newline=True,
|
||||||
)
|
)
|
||||||
|
j2env.globals.update(product=itertools.product)
|
||||||
|
|
||||||
test_names = os.listdir(suite_dir)
|
test_names = os.listdir(suite_dir)
|
||||||
for test_name in test_names:
|
for test_name in test_names:
|
||||||
|
@ -1050,6 +1050,9 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
|
|||||||
if (analysis_result.before_window)
|
if (analysis_result.before_window)
|
||||||
return analysis_result.before_window->getResultColumns();
|
return analysis_result.before_window->getResultColumns();
|
||||||
|
|
||||||
|
// NOTE: should not handle before_limit_by specially since
|
||||||
|
// WithMergeableState does not process LIMIT BY
|
||||||
|
|
||||||
return analysis_result.before_order_by->getResultColumns();
|
return analysis_result.before_order_by->getResultColumns();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1093,6 +1096,12 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
|
|||||||
if (analysis_result.before_window)
|
if (analysis_result.before_window)
|
||||||
return analysis_result.before_window->getResultColumns();
|
return analysis_result.before_window->getResultColumns();
|
||||||
|
|
||||||
|
// In case of query on remote shards executed up to
|
||||||
|
// WithMergeableStateAfterAggregation*, they can process LIMIT BY,
|
||||||
|
// since the initiator will not apply LIMIT BY again.
|
||||||
|
if (analysis_result.before_limit_by)
|
||||||
|
return analysis_result.before_limit_by->getResultColumns();
|
||||||
|
|
||||||
return analysis_result.before_order_by->getResultColumns();
|
return analysis_result.before_order_by->getResultColumns();
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -1539,7 +1548,11 @@ void InterpreterSelectQuery::executeImpl(QueryPlan & query_plan, std::optional<P
|
|||||||
if (query.limitLength())
|
if (query.limitLength())
|
||||||
executeDistinct(query_plan, false, expressions.selected_columns, false);
|
executeDistinct(query_plan, false, expressions.selected_columns, false);
|
||||||
|
|
||||||
if (expressions.hasLimitBy())
|
/// In case of query executed on remote shards (up to
|
||||||
|
/// WithMergeableState*) LIMIT BY cannot be applied, since it
|
||||||
|
/// will be applied on the initiator as well, and the header
|
||||||
|
/// may not match in some obscure cases.
|
||||||
|
if (options.to_stage == QueryProcessingStage::FetchColumns && expressions.hasLimitBy())
|
||||||
{
|
{
|
||||||
executeExpression(query_plan, expressions.before_limit_by, "Before LIMIT BY");
|
executeExpression(query_plan, expressions.before_limit_by, "Before LIMIT BY");
|
||||||
executeLimitBy(query_plan);
|
executeLimitBy(query_plan);
|
||||||
|
@ -8,6 +8,7 @@
|
|||||||
import enum
|
import enum
|
||||||
from queue import Full
|
from queue import Full
|
||||||
import shutil
|
import shutil
|
||||||
|
import itertools
|
||||||
import sys
|
import sys
|
||||||
import os
|
import os
|
||||||
import os.path
|
import os.path
|
||||||
@ -1640,6 +1641,8 @@ class TestSuite:
|
|||||||
if USE_JINJA
|
if USE_JINJA
|
||||||
else None
|
else None
|
||||||
)
|
)
|
||||||
|
if j2env is not None:
|
||||||
|
j2env.globals.update(product=itertools.product)
|
||||||
|
|
||||||
for test_name in os.listdir(self.suite_path):
|
for test_name in os.listdir(self.suite_path):
|
||||||
if not is_test_from_dir(self.suite_path, test_name):
|
if not is_test_from_dir(self.suite_path, test_name):
|
||||||
|
@ -19,10 +19,8 @@ explain select distinct on (k1) k2 from remote('127.{1,2}', view(select 1 k1, 2
|
|||||||
Expression (Projection)
|
Expression (Projection)
|
||||||
LimitBy
|
LimitBy
|
||||||
Union
|
Union
|
||||||
Expression (Before LIMIT BY)
|
Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))))
|
||||||
LimitBy
|
ReadFromStorage (SystemNumbers)
|
||||||
Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY))))))
|
|
||||||
ReadFromStorage (SystemNumbers)
|
|
||||||
Expression
|
Expression
|
||||||
ReadFromRemote (Read from remote replica)
|
ReadFromRemote (Read from remote replica)
|
||||||
explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized
|
explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)); -- optimized
|
||||||
@ -58,11 +56,10 @@ Expression (Projection)
|
|||||||
Expression (Before LIMIT BY)
|
Expression (Before LIMIT BY)
|
||||||
Sorting (Merge sorted streams for ORDER BY, without aggregation)
|
Sorting (Merge sorted streams for ORDER BY, without aggregation)
|
||||||
Union
|
Union
|
||||||
LimitBy
|
Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) [lifted up part])
|
||||||
Expression ((Before LIMIT BY + (Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))) [lifted up part]))
|
Sorting (Sorting for ORDER BY)
|
||||||
Sorting (Sorting for ORDER BY)
|
Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))
|
||||||
Expression ((Before ORDER BY + (Convert VIEW subquery result to VIEW table structure + (Materialize constants after VIEW subquery + (Projection + Before ORDER BY)))))
|
ReadFromStorage (SystemNumbers)
|
||||||
ReadFromStorage (SystemNumbers)
|
|
||||||
ReadFromRemote (Read from remote replica)
|
ReadFromRemote (Read from remote replica)
|
||||||
explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized
|
explain select distinct on (k1, k2) v from remote('127.{1,2}', view(select 1 k1, 2 k2, 3 v from numbers(2)), cityHash64(k1, k2)) order by v; -- optimized
|
||||||
Expression (Projection)
|
Expression (Projection)
|
||||||
|
@ -0,0 +1,52 @@
|
|||||||
|
Used settings: prefer_localhost_replica=0,distributed_group_by_no_merge=0,distributed_push_down_limit=1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Used settings: prefer_localhost_replica=0,distributed_group_by_no_merge=0,distributed_push_down_limit=0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Used settings: prefer_localhost_replica=0,distributed_group_by_no_merge=1,distributed_push_down_limit=1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Used settings: prefer_localhost_replica=0,distributed_group_by_no_merge=1,distributed_push_down_limit=0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Used settings: prefer_localhost_replica=0,distributed_group_by_no_merge=2,distributed_push_down_limit=1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Used settings: prefer_localhost_replica=0,distributed_group_by_no_merge=2,distributed_push_down_limit=0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Used settings: prefer_localhost_replica=1,distributed_group_by_no_merge=0,distributed_push_down_limit=1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Used settings: prefer_localhost_replica=1,distributed_group_by_no_merge=0,distributed_push_down_limit=0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Used settings: prefer_localhost_replica=1,distributed_group_by_no_merge=1,distributed_push_down_limit=1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Used settings: prefer_localhost_replica=1,distributed_group_by_no_merge=1,distributed_push_down_limit=0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Used settings: prefer_localhost_replica=1,distributed_group_by_no_merge=2,distributed_push_down_limit=1
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
Used settings: prefer_localhost_replica=1,distributed_group_by_no_merge=2,distributed_push_down_limit=0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
||||||
|
0
|
26
tests/queries/0_stateless/02899_distributed_limit_by.sql.j2
Normal file
26
tests/queries/0_stateless/02899_distributed_limit_by.sql.j2
Normal file
@ -0,0 +1,26 @@
|
|||||||
|
{#
|
||||||
|
Randomize settings:
|
||||||
|
- prefer_localhost_replica
|
||||||
|
- distributed_group_by_no_merge (0 = WithMergeableState, 1 = Complete, 2 = WithMergeableStateAfterAggregation/WithMergeableStateAfterAggregationAndLimit)
|
||||||
|
- distributed_push_down_limit (0/1 = dis/allows WithMergeableStateAfterAggregationAndLimit
|
||||||
|
#}
|
||||||
|
{% for settings in product(
|
||||||
|
[
|
||||||
|
'prefer_localhost_replica=0',
|
||||||
|
'prefer_localhost_replica=1',
|
||||||
|
],
|
||||||
|
[
|
||||||
|
'distributed_group_by_no_merge=0',
|
||||||
|
'distributed_group_by_no_merge=1',
|
||||||
|
'distributed_group_by_no_merge=2',
|
||||||
|
],
|
||||||
|
[
|
||||||
|
'distributed_push_down_limit=1',
|
||||||
|
'distributed_push_down_limit=0',
|
||||||
|
],
|
||||||
|
) %}
|
||||||
|
{% set settings = settings | join(',') %}
|
||||||
|
select 'Used settings: {{ settings }}';
|
||||||
|
select dummy from remote('127.{1,1}', system.one) where dummy + dummy >= 0 limit 1 by dummy + dummy + 0 as l settings {{ settings }};
|
||||||
|
select dummy from (select dummy + dummy + 0 as l, dummy from remote('127.{1,1}', system.one) where dummy + dummy >= 0 limit 1 by l) settings {{ settings }};
|
||||||
|
{% endfor %}
|
@ -0,0 +1,3 @@
|
|||||||
|
0 0
|
||||||
|
0 0
|
||||||
|
0 0
|
9
tests/queries/0_stateless/02900_limit_by_query_stage.sh
Executable file
9
tests/queries/0_stateless/02900_limit_by_query_stage.sh
Executable file
@ -0,0 +1,9 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
|
||||||
|
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
# shellcheck source=../shell_config.sh
|
||||||
|
. "$CUR_DIR"/../shell_config.sh
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT --stage with_mergeable_state --query 'SELECT dummy FROM system.one WHERE (dummy + dummy) >= 0 LIMIT 1 BY (dummy + dummy) + 0 AS l'
|
||||||
|
$CLICKHOUSE_CLIENT --stage with_mergeable_state_after_aggregation --query 'SELECT dummy FROM system.one WHERE (dummy + dummy) >= 0 LIMIT 1 BY (dummy + dummy) + 0 AS l'
|
||||||
|
$CLICKHOUSE_CLIENT --stage with_mergeable_state_after_aggregation_and_limit --query 'SELECT dummy FROM system.one WHERE (dummy + dummy) >= 0 LIMIT 1 BY (dummy + dummy) + 0 AS l'
|
Loading…
Reference in New Issue
Block a user