This commit is contained in:
Nikita Taranov 2024-06-10 23:54:19 +01:00
parent fbb36fc738
commit 6d48962ca0
4 changed files with 62 additions and 0 deletions

View File

@ -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());

View File

@ -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;
}

View File

@ -0,0 +1 @@
2

View 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'