mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 08:02:02 +00:00
impl
This commit is contained in:
parent
fbb36fc738
commit
6d48962ca0
@ -1515,6 +1515,15 @@ static void buildIndexes(
|
||||
|
||||
void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes)
|
||||
{
|
||||
/// Sometimes a really dumb problem may happen.
|
||||
/// For ReadFromMerge for example we may recursively call `applyFilters` for child reading steps (with no filters added so far).
|
||||
/// Then later `optimizePrimaryKeyCondition` will try to apply filters to those child reading steps, but with no luck,
|
||||
/// because we already made an `applyFilters` call that could lead to indexes initialization few lines below.
|
||||
/// So effectively the right set of filters will be just ignored.
|
||||
/// This is not an ultimate solution, of course, we're better to have more structured way of applying filters.
|
||||
if (added_filter_nodes.nodes.empty())
|
||||
return;
|
||||
|
||||
if (!indexes)
|
||||
{
|
||||
filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes, query_info.buildNodeNameToInputNodeColumn());
|
||||
|
@ -1573,8 +1573,14 @@ QueryPlanRawPtrs ReadFromMerge::getChildPlans()
|
||||
|
||||
QueryPlanRawPtrs plans;
|
||||
for (auto & child_plan : *child_plans)
|
||||
{
|
||||
if (child_plan.plan.isInitialized())
|
||||
{
|
||||
/// So we will see the optimized plan in EXPLAIN output
|
||||
child_plan.plan.optimize(QueryPlanOptimizationSettings::fromContext(context));
|
||||
plans.push_back(&child_plan.plan);
|
||||
}
|
||||
}
|
||||
|
||||
return plans;
|
||||
}
|
||||
|
@ -0,0 +1 @@
|
||||
2
|
46
tests/queries/0_stateless/03155_test_move_to_prewhere.sh
Executable file
46
tests/queries/0_stateless/03155_test_move_to_prewhere.sh
Executable file
@ -0,0 +1,46 @@
|
||||
#!/usr/bin/env bash
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
${CLICKHOUSE_CLIENT} -nq "
|
||||
CREATE TABLE event_envoy
|
||||
(
|
||||
timestamp_interval DateTime CODEC(DoubleDelta),
|
||||
region LowCardinality(String),
|
||||
cluster LowCardinality(String)
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
ORDER BY (timestamp_interval)
|
||||
SETTINGS index_granularity = 8192;
|
||||
|
||||
INSERT INTO event_envoy SELECT now() - number, 'us-east-1', 'ch_super_fast' FROM numbers_mt(1e5);
|
||||
"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -nq "
|
||||
CREATE TABLE event_envoy_remote
|
||||
(
|
||||
timestamp_interval DateTime CODEC(DoubleDelta),
|
||||
region LowCardinality(String),
|
||||
cluster LowCardinality(String)
|
||||
) AS remote('127.0.0.1', '${CLICKHOUSE_DATABASE}', event_envoy);
|
||||
"
|
||||
|
||||
${CLICKHOUSE_CLIENT} -q "
|
||||
CREATE TABLE global_event_envoy
|
||||
(
|
||||
timestamp_interval DateTime,
|
||||
region LowCardinality(String),
|
||||
cluster LowCardinality(String)
|
||||
)
|
||||
ENGINE = Merge('${CLICKHOUSE_DATABASE}', 'event_envoy.*');
|
||||
"
|
||||
|
||||
${CLICKHOUSE_CLIENT} --prefer_localhost_replica 1 -q "
|
||||
EXPLAIN indexes=1
|
||||
SELECT timestamp_interval
|
||||
FROM global_event_envoy
|
||||
WHERE timestamp_interval <= now() - 54321 AND region = 'us-east-1'
|
||||
" | grep -c 'Condition.*timestamp_interval'
|
||||
|
Loading…
Reference in New Issue
Block a user