Revert "Enable merge filters optimization."

This commit is contained in:
Raúl Marín 2024-11-13 17:39:17 +01:00 committed by GitHub
parent 9f62518846
commit 475989b311
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
20 changed files with 132 additions and 358 deletions

View File

@ -4560,7 +4560,7 @@ Possible values:
- 0 - Disable
- 1 - Enable
)", 0) \
DECLARE(Bool, query_plan_merge_filters, true, R"(
DECLARE(Bool, query_plan_merge_filters, false, R"(
Allow to merge filters in the query plan
)", 0) \
DECLARE(Bool, query_plan_filter_push_down, true, R"(

View File

@ -76,7 +76,6 @@ static std::initializer_list<std::pair<ClickHouseVersion, SettingsChangesHistory
{"backup_restore_keeper_max_retries_while_initializing", 0, 20, "New setting."},
{"backup_restore_keeper_max_retries_while_handling_error", 0, 20, "New setting."},
{"backup_restore_finish_timeout_after_error_sec", 0, 180, "New setting."},
{"query_plan_merge_filters", false, true, "Allow to merge filters in the query plan. This is required to properly support filter-push-down with a new analyzer."},
{"parallel_replicas_local_plan", false, true, "Use local plan for local replica in a query with parallel replicas"},
{"filesystem_cache_skip_download_if_exceeds_per_query_cache_write_limit", 1, 1, "Rename of setting skip_download_if_exceeds_query_cache_limit"},
{"filesystem_cache_prefer_bigger_buffer_size", true, true, "New setting"},

View File

@ -6,23 +6,12 @@
namespace DB
{
namespace Setting
{
extern const SettingsBool query_plan_merge_filters;
}
BuildQueryPipelineSettings BuildQueryPipelineSettings::fromContext(ContextPtr from)
{
const auto & query_settings = from->getSettingsRef();
BuildQueryPipelineSettings settings;
settings.actions_settings = ExpressionActionsSettings::fromSettings(query_settings, CompileExpressions::yes);
settings.actions_settings = ExpressionActionsSettings::fromSettings(from->getSettingsRef(), CompileExpressions::yes);
settings.process_list_element = from->getProcessListElement();
settings.progress_callback = from->getProgressCallback();
/// Setting query_plan_merge_filters is enabled by default.
/// But it can brake short-circuit without splitting filter step into smaller steps.
/// So, enable and disable this optimizations together.
settings.enable_multiple_filters_transforms_for_and_chain = query_settings[Setting::query_plan_merge_filters];
return settings;
}

View File

@ -17,8 +17,6 @@ using TemporaryFileLookupPtr = std::shared_ptr<ITemporaryFileLookup>;
struct BuildQueryPipelineSettings
{
bool enable_multiple_filters_transforms_for_and_chain = true;
ExpressionActionsSettings actions_settings;
QueryStatusPtr process_list_element;
ProgressCallback progress_callback = nullptr;

View File

@ -5,11 +5,6 @@
#include <Interpreters/ExpressionActions.h>
#include <IO/Operators.h>
#include <Common/JSONBuilder.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypesNumber.h>
#include <Functions/IFunction.h>
#include <stack>
#include <ranges>
namespace DB
{
@ -29,92 +24,6 @@ static ITransformingStep::Traits getTraits()
};
}
static bool isTrivialSubtree(const ActionsDAG::Node * node)
{
while (node->type == ActionsDAG::ActionType::ALIAS)
node = node->children.at(0);
return node->type != ActionsDAG::ActionType::FUNCTION && node->type != ActionsDAG::ActionType::ARRAY_JOIN;
}
struct ActionsAndName
{
ActionsDAG dag;
std::string name;
};
static ActionsAndName splitSingleAndFilter(ActionsDAG & dag, const ActionsDAG::Node * filter_node)
{
auto split_result = dag.split({filter_node}, true);
dag = std::move(split_result.second);
const auto * split_filter_node = split_result.split_nodes_mapping[filter_node];
auto filter_type = removeLowCardinality(split_filter_node->result_type);
if (!filter_type->onlyNull() && !isUInt8(removeNullable(filter_type)))
{
DataTypePtr cast_type = std::make_shared<DataTypeUInt8>();
if (filter_type->isNullable())
cast_type = std::make_shared<DataTypeNullable>(std::move(cast_type));
split_filter_node = &split_result.first.addCast(*split_filter_node, cast_type, {});
}
split_result.first.getOutputs().emplace(split_result.first.getOutputs().begin(), split_filter_node);
auto name = split_filter_node->result_name;
return ActionsAndName{std::move(split_result.first), std::move(name)};
}
/// Try to split the left most AND atom to a separate DAG.
static std::optional<ActionsAndName> trySplitSingleAndFilter(ActionsDAG & dag, const std::string & filter_name)
{
const auto * filter = &dag.findInOutputs(filter_name);
while (filter->type == ActionsDAG::ActionType::ALIAS)
filter = filter->children.at(0);
if (filter->type != ActionsDAG::ActionType::FUNCTION || filter->function_base->getName() != "and")
return {};
const ActionsDAG::Node * condition_to_split = nullptr;
std::stack<const ActionsDAG::Node *> nodes;
nodes.push(filter);
while (!nodes.empty())
{
const auto * node = nodes.top();
nodes.pop();
if (node->type == ActionsDAG::ActionType::FUNCTION && node->function_base->getName() == "and")
{
/// The order is important. We should take the left-most atom, so put conditions on stack in reverse order.
for (const auto * child : node->children | std::ranges::views::reverse)
nodes.push(child);
continue;
}
if (isTrivialSubtree(node))
continue;
/// Do not split subtree if it's the last non-trivial one.
/// So, split the first found condition only when there is a another one found.
if (condition_to_split)
return splitSingleAndFilter(dag, condition_to_split);
condition_to_split = node;
}
return {};
}
std::vector<ActionsAndName> splitAndChainIntoMultipleFilters(ActionsDAG & dag, const std::string & filter_name)
{
std::vector<ActionsAndName> res;
while (auto condition = trySplitSingleAndFilter(dag, filter_name))
res.push_back(std::move(*condition));
return res;
}
FilterStep::FilterStep(
const Header & input_header_,
ActionsDAG actions_dag_,
@ -141,23 +50,6 @@ FilterStep::FilterStep(
void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings)
{
std::vector<ActionsAndName> and_atoms;
/// Splitting AND filter condition to steps under the setting, which is enabled with merge_filters optimization.
/// This is needed to support short-circuit properly.
if (settings.enable_multiple_filters_transforms_for_and_chain && !actions_dag.hasStatefulFunctions())
and_atoms = splitAndChainIntoMultipleFilters(actions_dag, filter_column_name);
for (auto & and_atom : and_atoms)
{
auto expression = std::make_shared<ExpressionActions>(std::move(and_atom.dag), settings.getActionsSettings());
pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type)
{
bool on_totals = stream_type == QueryPipelineBuilder::StreamType::Totals;
return std::make_shared<FilterTransform>(header, expression, and_atom.name, true, on_totals);
});
}
auto expression = std::make_shared<ExpressionActions>(std::move(actions_dag), settings.getActionsSettings());
pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type)
@ -184,45 +76,18 @@ void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQ
void FilterStep::describeActions(FormatSettings & settings) const
{
String prefix(settings.offset, settings.indent_char);
auto cloned_dag = actions_dag.clone();
std::vector<ActionsAndName> and_atoms;
if (!actions_dag.hasStatefulFunctions())
and_atoms = splitAndChainIntoMultipleFilters(cloned_dag, filter_column_name);
for (auto & and_atom : and_atoms)
{
auto expression = std::make_shared<ExpressionActions>(std::move(and_atom.dag));
settings.out << prefix << "AND column: " << and_atom.name << '\n';
expression->describeActions(settings.out, prefix);
}
settings.out << prefix << "Filter column: " << filter_column_name;
if (remove_filter_column)
settings.out << " (removed)";
settings.out << '\n';
auto expression = std::make_shared<ExpressionActions>(std::move(cloned_dag));
auto expression = std::make_shared<ExpressionActions>(actions_dag.clone());
expression->describeActions(settings.out, prefix);
}
void FilterStep::describeActions(JSONBuilder::JSONMap & map) const
{
auto cloned_dag = actions_dag.clone();
std::vector<ActionsAndName> and_atoms;
if (!actions_dag.hasStatefulFunctions())
and_atoms = splitAndChainIntoMultipleFilters(cloned_dag, filter_column_name);
for (auto & and_atom : and_atoms)
{
auto expression = std::make_shared<ExpressionActions>(std::move(and_atom.dag));
map.add("AND column", and_atom.name);
map.add("Expression", expression->toTree());
}
map.add("Filter Column", filter_column_name);
map.add("Removes Filter", remove_filter_column);

View File

@ -32,7 +32,7 @@ struct QueryPlanOptimizationSettings
bool merge_expressions = true;
/// If merge-filters optimization is enabled.
bool merge_filters = true;
bool merge_filters = false;
/// If filter push down optimization is enabled.
bool filter_push_down = true;

View File

@ -176,7 +176,6 @@ namespace Setting
extern const SettingsBool use_skip_indexes;
extern const SettingsBool use_skip_indexes_if_final;
extern const SettingsBool use_uncompressed_cache;
extern const SettingsBool query_plan_merge_filters;
extern const SettingsUInt64 merge_tree_min_read_task_size;
extern const SettingsBool read_in_order_use_virtual_row;
}
@ -209,7 +208,6 @@ static MergeTreeReaderSettings getMergeTreeReaderSettings(
.use_asynchronous_read_from_pool = settings[Setting::allow_asynchronous_read_from_io_pool_for_merge_tree]
&& (settings[Setting::max_streams_to_max_threads_ratio] > 1 || settings[Setting::max_streams_for_merge_tree_reading] > 1),
.enable_multiple_prewhere_read_steps = settings[Setting::enable_multiple_prewhere_read_steps],
.force_short_circuit_execution = settings[Setting::query_plan_merge_filters]
};
}

View File

@ -330,7 +330,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(
auto prewhere_actions = MergeTreeSelectProcessor::getPrewhereActions(
prewhere_info,
actions_settings,
reader_settings.enable_multiple_prewhere_read_steps, reader_settings.force_short_circuit_execution);
reader_settings.enable_multiple_prewhere_read_steps);
for (const auto & step : prewhere_actions.steps)
add_step(*step);

View File

@ -45,8 +45,6 @@ struct MergeTreeReaderSettings
bool use_asynchronous_read_from_pool = false;
/// If PREWHERE has multiple conditions combined with AND, execute them in separate read/filtering steps.
bool enable_multiple_prewhere_read_steps = false;
/// In case of multiple prewhere steps, execute filtering earlier to support short-circuit properly.
bool force_short_circuit_execution = false;
/// If true, try to lower size of read buffer according to granule size and compressed block size.
bool adjust_read_buffer_size = true;
/// If true, it's allowed to read the whole part without reading marks.

View File

@ -91,7 +91,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor(
, algorithm(std::move(algorithm_))
, prewhere_info(prewhere_info_)
, actions_settings(actions_settings_)
, prewhere_actions(getPrewhereActions(prewhere_info, actions_settings, reader_settings_.enable_multiple_prewhere_read_steps, reader_settings_.force_short_circuit_execution))
, prewhere_actions(getPrewhereActions(prewhere_info, actions_settings, reader_settings_.enable_multiple_prewhere_read_steps))
, reader_settings(reader_settings_)
, result_header(transformHeader(pool->getHeader(), prewhere_info))
{
@ -124,9 +124,9 @@ String MergeTreeSelectProcessor::getName() const
return fmt::format("MergeTreeSelect(pool: {}, algorithm: {})", pool->getName(), algorithm->getName());
}
bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere, bool force_short_circuit_execution);
bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere);
PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps, bool force_short_circuit_execution)
PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, bool enable_multiple_prewhere_read_steps)
{
PrewhereExprInfo prewhere_actions;
if (prewhere_info)
@ -147,7 +147,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr
}
if (!enable_multiple_prewhere_read_steps ||
!tryBuildPrewhereSteps(prewhere_info, actions_settings, prewhere_actions, force_short_circuit_execution))
!tryBuildPrewhereSteps(prewhere_info, actions_settings, prewhere_actions))
{
PrewhereExprStep prewhere_step
{

View File

@ -73,8 +73,7 @@ public:
static PrewhereExprInfo getPrewhereActions(
PrewhereInfoPtr prewhere_info,
const ExpressionActionsSettings & actions_settings,
bool enable_multiple_prewhere_read_steps,
bool force_short_circuit_execution);
bool enable_multiple_prewhere_read_steps);
void addPartLevelToChunk(bool add_part_level_) { add_part_level = add_part_level_; }

View File

@ -4,7 +4,6 @@
#include <Storages/SelectQueryInfo.h>
#include <Storages/MergeTree/MergeTreeRangeReader.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Interpreters/ExpressionActions.h>
@ -58,9 +57,9 @@ struct DAGNodeRef
const ActionsDAG::Node * node;
};
/// ResultNode -> DAGNodeRef
using OriginalToNewNodeMap = std::unordered_map<const ActionsDAG::Node *, DAGNodeRef>;
using NodeNameToLastUsedStepMap = std::unordered_map<const ActionsDAG::Node *, size_t>;
/// Result name -> DAGNodeRef
using OriginalToNewNodeMap = std::unordered_map<String, DAGNodeRef>;
using NodeNameToLastUsedStepMap = std::unordered_map<String, size_t>;
/// Clones the part of original DAG responsible for computing the original_dag_node and adds it to the new DAG.
const ActionsDAG::Node & addClonedDAGToDAG(
@ -70,28 +69,25 @@ const ActionsDAG::Node & addClonedDAGToDAG(
OriginalToNewNodeMap & node_remap,
NodeNameToLastUsedStepMap & node_to_step_map)
{
const String & node_name = original_dag_node->result_name;
/// Look for the node in the map of already known nodes
if (node_remap.contains(original_dag_node))
if (node_remap.contains(node_name))
{
/// If the node is already in the new DAG, return it
const auto & node_ref = node_remap.at(original_dag_node);
const auto & node_ref = node_remap.at(node_name);
if (node_ref.dag == new_dag.get())
return *node_ref.node;
/// If the node is known from the previous steps, add it as an input, except for constants
if (original_dag_node->type != ActionsDAG::ActionType::COLUMN)
{
/// If the node was found in node_remap, it was not added to outputs yet.
/// The only exception is the filter node, which is always the first one.
if (node_ref.dag->getOutputs().at(0) != node_ref.node)
node_ref.dag->getOutputs().push_back(node_ref.node);
node_ref.dag->addOrReplaceInOutputs(*node_ref.node);
const auto & new_node = new_dag->addInput(node_ref.node->result_name, node_ref.node->result_type);
node_remap[original_dag_node] = {new_dag.get(), &new_node};
node_remap[node_name] = {new_dag.get(), &new_node}; /// TODO: here we update the node reference. Is it always correct?
/// Remember the index of the last step which reuses this node.
/// We cannot remove this node from the outputs before that step.
node_to_step_map[original_dag_node] = step;
node_to_step_map[node_name] = step;
return new_node;
}
}
@ -100,7 +96,7 @@ const ActionsDAG::Node & addClonedDAGToDAG(
if (original_dag_node->type == ActionsDAG::ActionType::INPUT)
{
const auto & new_node = new_dag->addInput(original_dag_node->result_name, original_dag_node->result_type);
node_remap[original_dag_node] = {new_dag.get(), &new_node};
node_remap[node_name] = {new_dag.get(), &new_node};
return new_node;
}
@ -109,7 +105,7 @@ const ActionsDAG::Node & addClonedDAGToDAG(
{
const auto & new_node = new_dag->addColumn(
ColumnWithTypeAndName(original_dag_node->column, original_dag_node->result_type, original_dag_node->result_name));
node_remap[original_dag_node] = {new_dag.get(), &new_node};
node_remap[node_name] = {new_dag.get(), &new_node};
return new_node;
}
@ -117,7 +113,7 @@ const ActionsDAG::Node & addClonedDAGToDAG(
{
const auto & alias_child = addClonedDAGToDAG(step, original_dag_node->children[0], new_dag, node_remap, node_to_step_map);
const auto & new_node = new_dag->addAlias(alias_child, original_dag_node->result_name);
node_remap[original_dag_node] = {new_dag.get(), &new_node};
node_remap[node_name] = {new_dag.get(), &new_node};
return new_node;
}
@ -132,7 +128,7 @@ const ActionsDAG::Node & addClonedDAGToDAG(
}
const auto & new_node = new_dag->addFunction(original_dag_node->function_base, new_children, original_dag_node->result_name);
node_remap[original_dag_node] = {new_dag.get(), &new_node};
node_remap[node_name] = {new_dag.get(), &new_node};
return new_node;
}
@ -142,9 +138,11 @@ const ActionsDAG::Node & addClonedDAGToDAG(
const ActionsDAG::Node & addFunction(
const ActionsDAGPtr & new_dag,
const FunctionOverloadResolverPtr & function,
ActionsDAG::NodeRawConstPtrs children)
ActionsDAG::NodeRawConstPtrs children,
OriginalToNewNodeMap & node_remap)
{
const auto & new_node = new_dag->addFunction(function, children, "");
node_remap[new_node.result_name] = {new_dag.get(), &new_node};
return new_node;
}
@ -154,12 +152,14 @@ const ActionsDAG::Node & addFunction(
const ActionsDAG::Node & addCast(
const ActionsDAGPtr & dag,
const ActionsDAG::Node & node_to_cast,
const DataTypePtr & to_type)
const DataTypePtr & to_type,
OriginalToNewNodeMap & node_remap)
{
if (!node_to_cast.result_type->equals(*to_type))
return node_to_cast;
const auto & new_node = dag->addCast(node_to_cast, to_type, {});
node_remap[new_node.result_name] = {dag.get(), &new_node};
return new_node;
}
@ -169,7 +169,8 @@ const ActionsDAG::Node & addCast(
/// 2. makes sure that the result contains only 0 or 1 values even if the source column contains non-boolean values.
const ActionsDAG::Node & addAndTrue(
const ActionsDAGPtr & dag,
const ActionsDAG::Node & filter_node_to_normalize)
const ActionsDAG::Node & filter_node_to_normalize,
OriginalToNewNodeMap & node_remap)
{
Field const_true_value(true);
@ -180,7 +181,7 @@ const ActionsDAG::Node & addAndTrue(
const auto * const_true_node = &dag->addColumn(std::move(const_true_column));
ActionsDAG::NodeRawConstPtrs children = {&filter_node_to_normalize, const_true_node};
FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
return addFunction(dag, func_builder_and, children);
return addFunction(dag, func_builder_and, children, node_remap);
}
}
@ -205,11 +206,7 @@ const ActionsDAG::Node & addAndTrue(
/// 6. Find all outputs of the original DAG
/// 7. Find all outputs that were computed in the already built DAGs, mark these nodes as outputs in the steps where they were computed
/// 8. Add computation of the remaining outputs to the last step with the procedure similar to 4
bool tryBuildPrewhereSteps(
PrewhereInfoPtr prewhere_info,
const ExpressionActionsSettings & actions_settings,
PrewhereExprInfo & prewhere,
bool force_short_circuit_execution)
bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere)
{
if (!prewhere_info)
return true;
@ -246,10 +243,7 @@ bool tryBuildPrewhereSteps(
struct Step
{
ActionsDAGPtr actions;
/// Original condition, in case if we have only one condition, and it was not casted
const ActionsDAG::Node * original_node;
/// Result condition node
const ActionsDAG::Node * result_node;
String column_name;
};
std::vector<Step> steps;
@ -260,8 +254,7 @@ bool tryBuildPrewhereSteps(
{
const auto & condition_group = condition_groups[step_index];
ActionsDAGPtr step_dag = std::make_unique<ActionsDAG>();
const ActionsDAG::Node * original_node = nullptr;
const ActionsDAG::Node * result_node;
String result_name;
std::vector<const ActionsDAG::Node *> new_condition_nodes;
for (const auto * node : condition_group)
@ -274,37 +267,48 @@ bool tryBuildPrewhereSteps(
{
/// Add AND function to combine the conditions
FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
const auto & and_function_node = addFunction(step_dag, func_builder_and, new_condition_nodes);
result_node = &and_function_node;
const auto & and_function_node = addFunction(step_dag, func_builder_and, new_condition_nodes, node_remap);
step_dag->addOrReplaceInOutputs(and_function_node);
result_name = and_function_node.result_name;
}
else
{
result_node = new_condition_nodes.front();
const auto & result_node = *new_condition_nodes.front();
/// Check if explicit cast is needed for the condition to serve as a filter.
if (!isUInt8(removeNullable(removeLowCardinality(result_node->result_type))))
const auto result_type_name = result_node.result_type->getName();
if (result_type_name == "UInt8" ||
result_type_name == "Nullable(UInt8)" ||
result_type_name == "LowCardinality(UInt8)" ||
result_type_name == "LowCardinality(Nullable(UInt8))")
{
/// No need to cast
step_dag->addOrReplaceInOutputs(result_node);
result_name = result_node.result_name;
}
else
{
/// Build "condition AND True" expression to "cast" the condition to UInt8 or Nullable(UInt8) depending on its type.
result_node = &addAndTrue(step_dag, *result_node);
const auto & cast_node = addAndTrue(step_dag, result_node, node_remap);
step_dag->addOrReplaceInOutputs(cast_node);
result_name = cast_node.result_name;
}
}
step_dag->getOutputs().insert(step_dag->getOutputs().begin(), result_node);
steps.push_back({std::move(step_dag), original_node, result_node});
steps.push_back({std::move(step_dag), result_name});
}
/// 6. Find all outputs of the original DAG
auto original_outputs = prewhere_info->prewhere_actions.getOutputs();
steps.back().actions->getOutputs().clear();
/// 7. Find all outputs that were computed in the already built DAGs, mark these nodes as outputs in the steps where they were computed
/// 8. Add computation of the remaining outputs to the last step with the procedure similar to 4
std::unordered_set<const ActionsDAG::Node *> all_outputs;
NameSet all_output_names;
for (const auto * output : original_outputs)
{
all_outputs.insert(output);
if (node_remap.contains(output))
all_output_names.insert(output->result_name);
if (node_remap.contains(output->result_name))
{
const auto & new_node_info = node_remap[output];
new_node_info.dag->getOutputs().push_back(new_node_info.node);
const auto & new_node_info = node_remap[output->result_name];
new_node_info.dag->addOrReplaceInOutputs(*new_node_info.node);
}
else if (output->result_name == prewhere_info->prewhere_column_name)
{
@ -315,21 +319,20 @@ bool tryBuildPrewhereSteps(
/// 1. AND the last condition with constant True. This is needed to make sure that in the last step filter has UInt8 type
/// but contains values other than 0 and 1 (e.g. if it is (number%5) it contains 2,3,4)
/// 2. CAST the result to the exact type of the PREWHERE column from the original DAG
const auto & last_step_result_node_info = node_remap[steps.back().column_name];
auto & last_step_dag = steps.back().actions;
auto & last_step_result_node = steps.back().result_node;
/// Build AND(last_step_result_node, true)
const auto & and_node = addAndTrue(last_step_dag, *last_step_result_node);
const auto & and_node = addAndTrue(last_step_dag, *last_step_result_node_info.node, node_remap);
/// Build CAST(and_node, type of PREWHERE column)
const auto & cast_node = addCast(last_step_dag, and_node, output->result_type);
const auto & cast_node = addCast(last_step_dag, and_node, output->result_type, node_remap);
/// Add alias for the result with the name of the PREWHERE column
const auto & prewhere_result_node = last_step_dag->addAlias(cast_node, output->result_name);
last_step_dag->getOutputs().push_back(&prewhere_result_node);
steps.back().result_node = &prewhere_result_node;
last_step_dag->addOrReplaceInOutputs(prewhere_result_node);
}
else
{
const auto & node_in_new_dag = addClonedDAGToDAG(steps.size() - 1, output, steps.back().actions, node_remap, node_to_step);
steps.back().actions->getOutputs().push_back(&node_in_new_dag);
steps.back().actions->addOrReplaceInOutputs(node_in_new_dag);
}
}
@ -342,18 +345,17 @@ bool tryBuildPrewhereSteps(
{
.type = PrewhereExprStep::Filter,
.actions = std::make_shared<ExpressionActions>(std::move(*step.actions), actions_settings),
.filter_column_name = step.result_node->result_name,
.filter_column_name = step.column_name,
/// Don't remove if it's in the list of original outputs
.remove_filter_column =
step.original_node && !all_outputs.contains(step.original_node) && node_to_step[step.original_node] <= step_index,
.need_filter = force_short_circuit_execution,
!all_output_names.contains(step.column_name) && node_to_step[step.column_name] <= step_index,
.need_filter = false,
.perform_alter_conversions = true,
};
prewhere.steps.push_back(std::make_shared<PrewhereExprStep>(std::move(new_step)));
}
prewhere.steps.back()->remove_filter_column = prewhere_info->remove_prewhere_column;
prewhere.steps.back()->need_filter = prewhere_info->need_filter;
}

View File

@ -82,12 +82,12 @@ Filter column: notEquals(__table1.y, 0_UInt8)
9 10
> one condition of filter should be pushed down after aggregating, other two conditions are ANDed
Filter column
FUNCTION and(minus(s, 8) :: 3, minus(s, 4) :: 5) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))
FUNCTION and(minus(s, 8) :: 5, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))
Aggregating
Filter column: notEquals(y, 0)
> (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed
Filter column
FUNCTION and(minus(__table1.s, 8_UInt8) :: 3, minus(__table1.s, 4_UInt8) :: 5) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))
FUNCTION and(minus(__table1.s, 8_UInt8) :: 1, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))
Aggregating
Filter column: notEquals(__table1.y, 0_UInt8)
0 1
@ -163,6 +163,7 @@ Filter column: notEquals(__table1.y, 2_UInt8)
> filter is pushed down before CreatingSets
CreatingSets
Filter
Filter
1
3
> one condition of filter is pushed down before LEFT JOIN

View File

@ -89,14 +89,14 @@ $CLICKHOUSE_CLIENT --enable_analyzer=0 --convert_query_to_cnf=0 -q "
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0 and s - 8 and s - 4
settings enable_optimize_predicate_expression=0" |
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 8) :: 3, minus(s, 4) :: 5) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))"
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 8) :: 5, minus(s, 4) :: 2) -> and(notEquals(y, 0), minus(s, 8), minus(s, 4))"
echo "> (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed"
$CLICKHOUSE_CLIENT --enable_analyzer=1 --convert_query_to_cnf=0 -q "
explain actions = 1 select s, y from (
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
) where y != 0 and s - 8 and s - 4
settings enable_optimize_predicate_expression=0" |
grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 8_UInt8) :: 3, minus(__table1.s, 4_UInt8) :: 5) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))"
grep -o "Aggregating\|Filter column\|Filter column: notEquals(__table1.y, 0_UInt8)\|FUNCTION and(minus(__table1.s, 8_UInt8) :: 1, minus(__table1.s, 4_UInt8) :: 2) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 8_UInt8), minus(__table1.s, 4_UInt8))"
$CLICKHOUSE_CLIENT -q "
select s, y from (
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y

View File

@ -332,7 +332,8 @@ SETTINGS optimize_aggregators_of_group_by_keys=0 -- avoid removing any() as it d
Expression (Projection)
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
Filter (((WHERE + (Projection + Before ORDER BY)) + HAVING))
Filter ((WHERE + (Projection + Before ORDER BY)))
Filter (HAVING)
Aggregating
Expression ((Before GROUP BY + Projection))
Sorting (Sorting for ORDER BY)

View File

@ -28,7 +28,11 @@ WHERE type_1 = \'all\'
(Expression)
ExpressionTransform × 2
(Filter)
FilterTransform × 6
FilterTransform × 2
(Filter)
FilterTransform × 2
(Filter)
FilterTransform × 2
(Aggregating)
ExpressionTransform × 2
AggregatingTransform × 2
@ -64,6 +68,10 @@ ExpressionTransform × 2
ExpressionTransform × 2
AggregatingTransform × 2
Copy 1 → 2
(Filter)
FilterTransform
(Filter)
FilterTransform
(Expression)
ExpressionTransform
(ReadFromMergeTree)

View File

@ -163,21 +163,17 @@ Positions: 4 2 0 1
Filter (( + (JOIN actions + Change column names to column identifiers)))
Header: __table1.id UInt64
__table1.value String
AND column: equals(__table1.id, 5_UInt8)
Actions: INPUT : 0 -> id UInt64 : 0
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 1
FUNCTION equals(id : 0, 5_UInt8 :: 1) -> equals(__table1.id, 5_UInt8) UInt8 : 2
Positions: 2 0 2
Filter column: and(equals(__table1.id, 5_UInt8), equals(__table1.id, 6_UInt8)) (removed)
Actions: INPUT : 2 -> value String : 0
INPUT : 1 -> id UInt64 : 1
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 2
INPUT : 0 -> equals(__table1.id, 5_UInt8) UInt8 : 3
ALIAS value :: 0 -> __table1.value String : 4
ALIAS id : 1 -> __table1.id UInt64 : 0
FUNCTION equals(id :: 1, 6_UInt8 :: 2) -> equals(__table1.id, 6_UInt8) UInt8 : 5
FUNCTION and(equals(__table1.id, 5_UInt8) :: 3, equals(__table1.id, 6_UInt8) :: 5) -> and(equals(__table1.id, 5_UInt8), equals(__table1.id, 6_UInt8)) UInt8 : 2
Positions: 2 0 4
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 3
ALIAS id : 0 -> __table1.id UInt64 : 4
ALIAS value :: 1 -> __table1.value String : 5
FUNCTION equals(id : 0, 6_UInt8 :: 2) -> equals(__table1.id, 6_UInt8) UInt8 : 1
FUNCTION equals(id :: 0, 5_UInt8 :: 3) -> equals(__table1.id, 5_UInt8) UInt8 : 2
FUNCTION and(equals(__table1.id, 5_UInt8) :: 2, equals(__table1.id, 6_UInt8) :: 1) -> and(equals(__table1.id, 5_UInt8), equals(__table1.id, 6_UInt8)) UInt8 : 3
Positions: 3 4 5
ReadFromMergeTree (default.test_table_1)
Header: id UInt64
value String
@ -187,21 +183,17 @@ Positions: 4 2 0 1
Filter (( + (JOIN actions + Change column names to column identifiers)))
Header: __table2.id UInt64
__table2.value String
AND column: equals(__table2.id, 6_UInt8)
Actions: INPUT : 0 -> id UInt64 : 0
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 1
FUNCTION equals(id : 0, 6_UInt8 :: 1) -> equals(__table2.id, 6_UInt8) UInt8 : 2
Positions: 2 0 2
Filter column: and(equals(__table2.id, 6_UInt8), equals(__table2.id, 5_UInt8)) (removed)
Actions: INPUT : 2 -> value String : 0
INPUT : 1 -> id UInt64 : 1
Actions: INPUT : 0 -> id UInt64 : 0
INPUT : 1 -> value String : 1
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2
INPUT : 0 -> equals(__table2.id, 6_UInt8) UInt8 : 3
ALIAS value :: 0 -> __table2.value String : 4
ALIAS id : 1 -> __table2.id UInt64 : 0
FUNCTION equals(id :: 1, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 5
FUNCTION and(equals(__table2.id, 6_UInt8) :: 3, equals(__table2.id, 5_UInt8) :: 5) -> and(equals(__table2.id, 6_UInt8), equals(__table2.id, 5_UInt8)) UInt8 : 2
Positions: 2 0 4
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 3
ALIAS id : 0 -> __table2.id UInt64 : 4
ALIAS value :: 1 -> __table2.value String : 5
FUNCTION equals(id : 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1
FUNCTION equals(id :: 0, 6_UInt8 :: 3) -> equals(__table2.id, 6_UInt8) UInt8 : 2
FUNCTION and(equals(__table2.id, 6_UInt8) :: 2, equals(__table2.id, 5_UInt8) :: 1) -> and(equals(__table2.id, 6_UInt8), equals(__table2.id, 5_UInt8)) UInt8 : 3
Positions: 3 4 5
ReadFromMergeTree (default.test_table_2)
Header: id UInt64
value String
@ -664,21 +656,17 @@ Positions: 4 2 0 1
__table1.value String
__table2.value String
__table2.id UInt64
AND column: equals(__table1.id, 5_UInt8)
Actions: INPUT : 0 -> __table1.id UInt64 : 0
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 1
FUNCTION equals(__table1.id : 0, 5_UInt8 :: 1) -> equals(__table1.id, 5_UInt8) UInt8 : 2
Positions: 2 0 2
Filter column: and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) (removed)
Actions: INPUT :: 1 -> __table1.id UInt64 : 0
INPUT :: 2 -> __table1.value String : 1
INPUT :: 3 -> __table2.value String : 2
INPUT : 4 -> __table2.id UInt64 : 3
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 4
INPUT : 0 -> equals(__table1.id, 5_UInt8) UInt8 : 5
FUNCTION equals(__table2.id : 3, 6_UInt8 :: 4) -> equals(__table2.id, 6_UInt8) UInt8 : 6
FUNCTION and(equals(__table1.id, 5_UInt8) :: 5, equals(__table2.id, 6_UInt8) :: 6) -> and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) UInt8 : 4
Positions: 4 0 1 2 3
Actions: INPUT : 0 -> __table1.id UInt64 : 0
INPUT :: 1 -> __table1.value String : 1
INPUT :: 2 -> __table2.value String : 2
INPUT : 3 -> __table2.id UInt64 : 3
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 5
FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_UInt8) UInt8 : 6
FUNCTION equals(__table2.id : 3, 6_UInt8 :: 5) -> equals(__table2.id, 6_UInt8) UInt8 : 4
FUNCTION and(equals(__table1.id, 5_UInt8) :: 6, equals(__table2.id, 6_UInt8) :: 4) -> and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) UInt8 : 5
Positions: 5 0 1 2 3
Join (JOIN FillRightFirst)
Header: __table1.id UInt64
__table1.value String

View File

@ -1,5 +1,3 @@
set allow_reorder_prewhere_conditions=0;
drop table if exists t1;
drop table if exists t2;
@ -51,23 +49,7 @@ tmp1 AS
fs1
FROM t2
LEFT JOIN tmp1 USING (fs1)
WHERE (fs1 IN ('test')) SETTINGS enable_multiple_prewhere_read_steps = 0, query_plan_merge_filters=0;
WITH
tmp1 AS
(
SELECT
CAST(s1, 'FixedString(10)') AS fs1,
s2 AS sector,
s3
FROM t1
WHERE (s3 != 'test')
)
SELECT
fs1
FROM t2
LEFT JOIN tmp1 USING (fs1)
WHERE (fs1 IN ('test')) SETTINGS enable_multiple_prewhere_read_steps = 1, query_plan_merge_filters=1;
WHERE (fs1 IN ('test')) SETTINGS enable_multiple_prewhere_read_steps = 0;
optimize table t1 final;
@ -85,20 +67,4 @@ tmp1 AS
fs1
FROM t2
LEFT JOIN tmp1 USING (fs1)
WHERE (fs1 IN ('test')) SETTINGS enable_multiple_prewhere_read_steps = 0, query_plan_merge_filters=0;
WITH
tmp1 AS
(
SELECT
CAST(s1, 'FixedString(10)') AS fs1,
s2 AS sector,
s3
FROM t1
WHERE (s3 != 'test')
)
SELECT
fs1
FROM t2
LEFT JOIN tmp1 USING (fs1)
WHERE (fs1 IN ('test')) SETTINGS enable_multiple_prewhere_read_steps = 1, query_plan_merge_filters=1;
WHERE (fs1 IN ('test'));

View File

@ -1,2 +0,0 @@
Condition: and((materialize(auid) in [1, 1]), (_CAST(toDate(ts)) in (-Inf, 1703980800]))
Granules: 1/3

View File

@ -1,36 +0,0 @@
DROP TABLE IF EXISTS alpha;
DROP TABLE IF EXISTS alpha__day;
SET session_timezone = 'Etc/UTC';
CREATE TABLE alpha
(
`ts` DateTime64(6),
`auid` Int64,
)
ENGINE = MergeTree
ORDER BY (auid, ts)
SETTINGS index_granularity = 1;
CREATE VIEW alpha__day
(
`ts_date` Date,
`auid` Int64,
)
AS SELECT
ts_date,
auid,
FROM
(
SELECT
toDate(ts) AS ts_date,
auid
FROM alpha
)
WHERE ts_date <= toDateTime('2024-01-01 00:00:00') - INTERVAL 1 DAY;
INSERT INTO alpha VALUES (toDateTime64('2024-01-01 00:00:00.000', 3) - INTERVAL 3 DAY, 1);
INSERT INTO alpha VALUES (toDateTime64('2024-01-01 00:00:00.000', 3) - INTERVAL 3 DAY, 2);
INSERT INTO alpha VALUES (toDateTime64('2024-01-01 00:00:00.000', 3) - INTERVAL 3 DAY, 3);
select trimLeft(explain) from (EXPLAIN indexes = 1 SELECT auid FROM alpha__day WHERE auid = 1) where explain like '%Condition:%' or explain like '%Granules:%' settings allow_experimental_analyzer = 1;