mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Revert "Enable merge filters optimization."
This commit is contained in:
parent
9f62518846
commit
475989b311
@ -4560,7 +4560,7 @@ Possible values:
|
|||||||
- 0 - Disable
|
- 0 - Disable
|
||||||
- 1 - Enable
|
- 1 - Enable
|
||||||
)", 0) \
|
)", 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
|
Allow to merge filters in the query plan
|
||||||
)", 0) \
|
)", 0) \
|
||||||
DECLARE(Bool, query_plan_filter_push_down, true, R"(
|
DECLARE(Bool, query_plan_filter_push_down, true, R"(
|
||||||
|
@ -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_initializing", 0, 20, "New setting."},
|
||||||
{"backup_restore_keeper_max_retries_while_handling_error", 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."},
|
{"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"},
|
{"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_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"},
|
{"filesystem_cache_prefer_bigger_buffer_size", true, true, "New setting"},
|
||||||
|
@ -6,23 +6,12 @@
|
|||||||
namespace DB
|
namespace DB
|
||||||
{
|
{
|
||||||
|
|
||||||
namespace Setting
|
|
||||||
{
|
|
||||||
extern const SettingsBool query_plan_merge_filters;
|
|
||||||
}
|
|
||||||
|
|
||||||
BuildQueryPipelineSettings BuildQueryPipelineSettings::fromContext(ContextPtr from)
|
BuildQueryPipelineSettings BuildQueryPipelineSettings::fromContext(ContextPtr from)
|
||||||
{
|
{
|
||||||
const auto & query_settings = from->getSettingsRef();
|
|
||||||
BuildQueryPipelineSettings settings;
|
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.process_list_element = from->getProcessListElement();
|
||||||
settings.progress_callback = from->getProgressCallback();
|
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;
|
return settings;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -17,8 +17,6 @@ using TemporaryFileLookupPtr = std::shared_ptr<ITemporaryFileLookup>;
|
|||||||
|
|
||||||
struct BuildQueryPipelineSettings
|
struct BuildQueryPipelineSettings
|
||||||
{
|
{
|
||||||
bool enable_multiple_filters_transforms_for_and_chain = true;
|
|
||||||
|
|
||||||
ExpressionActionsSettings actions_settings;
|
ExpressionActionsSettings actions_settings;
|
||||||
QueryStatusPtr process_list_element;
|
QueryStatusPtr process_list_element;
|
||||||
ProgressCallback progress_callback = nullptr;
|
ProgressCallback progress_callback = nullptr;
|
||||||
|
@ -5,11 +5,6 @@
|
|||||||
#include <Interpreters/ExpressionActions.h>
|
#include <Interpreters/ExpressionActions.h>
|
||||||
#include <IO/Operators.h>
|
#include <IO/Operators.h>
|
||||||
#include <Common/JSONBuilder.h>
|
#include <Common/JSONBuilder.h>
|
||||||
#include <DataTypes/DataTypeLowCardinality.h>
|
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
|
||||||
#include <Functions/IFunction.h>
|
|
||||||
#include <stack>
|
|
||||||
#include <ranges>
|
|
||||||
|
|
||||||
namespace DB
|
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(
|
FilterStep::FilterStep(
|
||||||
const Header & input_header_,
|
const Header & input_header_,
|
||||||
ActionsDAG actions_dag_,
|
ActionsDAG actions_dag_,
|
||||||
@ -141,23 +50,6 @@ FilterStep::FilterStep(
|
|||||||
|
|
||||||
void FilterStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings)
|
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());
|
auto expression = std::make_shared<ExpressionActions>(std::move(actions_dag), settings.getActionsSettings());
|
||||||
|
|
||||||
pipeline.addSimpleTransform([&](const Block & header, QueryPipelineBuilder::StreamType stream_type)
|
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
|
void FilterStep::describeActions(FormatSettings & settings) const
|
||||||
{
|
{
|
||||||
String prefix(settings.offset, settings.indent_char);
|
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;
|
settings.out << prefix << "Filter column: " << filter_column_name;
|
||||||
|
|
||||||
if (remove_filter_column)
|
if (remove_filter_column)
|
||||||
settings.out << " (removed)";
|
settings.out << " (removed)";
|
||||||
settings.out << '\n';
|
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);
|
expression->describeActions(settings.out, prefix);
|
||||||
}
|
}
|
||||||
|
|
||||||
void FilterStep::describeActions(JSONBuilder::JSONMap & map) const
|
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("Filter Column", filter_column_name);
|
||||||
map.add("Removes Filter", remove_filter_column);
|
map.add("Removes Filter", remove_filter_column);
|
||||||
|
|
||||||
|
@ -32,7 +32,7 @@ struct QueryPlanOptimizationSettings
|
|||||||
bool merge_expressions = true;
|
bool merge_expressions = true;
|
||||||
|
|
||||||
/// If merge-filters optimization is enabled.
|
/// If merge-filters optimization is enabled.
|
||||||
bool merge_filters = true;
|
bool merge_filters = false;
|
||||||
|
|
||||||
/// If filter push down optimization is enabled.
|
/// If filter push down optimization is enabled.
|
||||||
bool filter_push_down = true;
|
bool filter_push_down = true;
|
||||||
|
@ -176,7 +176,6 @@ namespace Setting
|
|||||||
extern const SettingsBool use_skip_indexes;
|
extern const SettingsBool use_skip_indexes;
|
||||||
extern const SettingsBool use_skip_indexes_if_final;
|
extern const SettingsBool use_skip_indexes_if_final;
|
||||||
extern const SettingsBool use_uncompressed_cache;
|
extern const SettingsBool use_uncompressed_cache;
|
||||||
extern const SettingsBool query_plan_merge_filters;
|
|
||||||
extern const SettingsUInt64 merge_tree_min_read_task_size;
|
extern const SettingsUInt64 merge_tree_min_read_task_size;
|
||||||
extern const SettingsBool read_in_order_use_virtual_row;
|
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]
|
.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),
|
&& (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],
|
.enable_multiple_prewhere_read_steps = settings[Setting::enable_multiple_prewhere_read_steps],
|
||||||
.force_short_circuit_execution = settings[Setting::query_plan_merge_filters]
|
|
||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -330,7 +330,7 @@ MergeTreeReadTaskColumns getReadTaskColumns(
|
|||||||
auto prewhere_actions = MergeTreeSelectProcessor::getPrewhereActions(
|
auto prewhere_actions = MergeTreeSelectProcessor::getPrewhereActions(
|
||||||
prewhere_info,
|
prewhere_info,
|
||||||
actions_settings,
|
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)
|
for (const auto & step : prewhere_actions.steps)
|
||||||
add_step(*step);
|
add_step(*step);
|
||||||
|
@ -45,8 +45,6 @@ struct MergeTreeReaderSettings
|
|||||||
bool use_asynchronous_read_from_pool = false;
|
bool use_asynchronous_read_from_pool = false;
|
||||||
/// If PREWHERE has multiple conditions combined with AND, execute them in separate read/filtering steps.
|
/// If PREWHERE has multiple conditions combined with AND, execute them in separate read/filtering steps.
|
||||||
bool enable_multiple_prewhere_read_steps = false;
|
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.
|
/// If true, try to lower size of read buffer according to granule size and compressed block size.
|
||||||
bool adjust_read_buffer_size = true;
|
bool adjust_read_buffer_size = true;
|
||||||
/// If true, it's allowed to read the whole part without reading marks.
|
/// If true, it's allowed to read the whole part without reading marks.
|
||||||
|
@ -91,7 +91,7 @@ MergeTreeSelectProcessor::MergeTreeSelectProcessor(
|
|||||||
, algorithm(std::move(algorithm_))
|
, algorithm(std::move(algorithm_))
|
||||||
, prewhere_info(prewhere_info_)
|
, prewhere_info(prewhere_info_)
|
||||||
, actions_settings(actions_settings_)
|
, 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_)
|
, reader_settings(reader_settings_)
|
||||||
, result_header(transformHeader(pool->getHeader(), prewhere_info))
|
, 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());
|
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;
|
PrewhereExprInfo prewhere_actions;
|
||||||
if (prewhere_info)
|
if (prewhere_info)
|
||||||
@ -147,7 +147,7 @@ PrewhereExprInfo MergeTreeSelectProcessor::getPrewhereActions(PrewhereInfoPtr pr
|
|||||||
}
|
}
|
||||||
|
|
||||||
if (!enable_multiple_prewhere_read_steps ||
|
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
|
PrewhereExprStep prewhere_step
|
||||||
{
|
{
|
||||||
|
@ -73,8 +73,7 @@ public:
|
|||||||
static PrewhereExprInfo getPrewhereActions(
|
static PrewhereExprInfo getPrewhereActions(
|
||||||
PrewhereInfoPtr prewhere_info,
|
PrewhereInfoPtr prewhere_info,
|
||||||
const ExpressionActionsSettings & actions_settings,
|
const ExpressionActionsSettings & actions_settings,
|
||||||
bool enable_multiple_prewhere_read_steps,
|
bool enable_multiple_prewhere_read_steps);
|
||||||
bool force_short_circuit_execution);
|
|
||||||
|
|
||||||
void addPartLevelToChunk(bool add_part_level_) { add_part_level = add_part_level_; }
|
void addPartLevelToChunk(bool add_part_level_) { add_part_level = add_part_level_; }
|
||||||
|
|
||||||
|
@ -4,7 +4,6 @@
|
|||||||
#include <Storages/SelectQueryInfo.h>
|
#include <Storages/SelectQueryInfo.h>
|
||||||
#include <Storages/MergeTree/MergeTreeRangeReader.h>
|
#include <Storages/MergeTree/MergeTreeRangeReader.h>
|
||||||
#include <DataTypes/DataTypeString.h>
|
#include <DataTypes/DataTypeString.h>
|
||||||
#include <DataTypes/DataTypeLowCardinality.h>
|
|
||||||
#include <Interpreters/ExpressionActions.h>
|
#include <Interpreters/ExpressionActions.h>
|
||||||
|
|
||||||
|
|
||||||
@ -58,9 +57,9 @@ struct DAGNodeRef
|
|||||||
const ActionsDAG::Node * node;
|
const ActionsDAG::Node * node;
|
||||||
};
|
};
|
||||||
|
|
||||||
/// ResultNode -> DAGNodeRef
|
/// Result name -> DAGNodeRef
|
||||||
using OriginalToNewNodeMap = std::unordered_map<const ActionsDAG::Node *, DAGNodeRef>;
|
using OriginalToNewNodeMap = std::unordered_map<String, DAGNodeRef>;
|
||||||
using NodeNameToLastUsedStepMap = std::unordered_map<const ActionsDAG::Node *, size_t>;
|
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.
|
/// Clones the part of original DAG responsible for computing the original_dag_node and adds it to the new DAG.
|
||||||
const ActionsDAG::Node & addClonedDAGToDAG(
|
const ActionsDAG::Node & addClonedDAGToDAG(
|
||||||
@ -70,28 +69,25 @@ const ActionsDAG::Node & addClonedDAGToDAG(
|
|||||||
OriginalToNewNodeMap & node_remap,
|
OriginalToNewNodeMap & node_remap,
|
||||||
NodeNameToLastUsedStepMap & node_to_step_map)
|
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
|
/// 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
|
/// 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())
|
if (node_ref.dag == new_dag.get())
|
||||||
return *node_ref.node;
|
return *node_ref.node;
|
||||||
|
|
||||||
/// If the node is known from the previous steps, add it as an input, except for constants
|
/// 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 (original_dag_node->type != ActionsDAG::ActionType::COLUMN)
|
||||||
{
|
{
|
||||||
/// If the node was found in node_remap, it was not added to outputs yet.
|
node_ref.dag->addOrReplaceInOutputs(*node_ref.node);
|
||||||
/// 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);
|
|
||||||
|
|
||||||
const auto & new_node = new_dag->addInput(node_ref.node->result_name, node_ref.node->result_type);
|
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.
|
/// Remember the index of the last step which reuses this node.
|
||||||
/// We cannot remove this node from the outputs before that step.
|
/// 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;
|
return new_node;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -100,7 +96,7 @@ const ActionsDAG::Node & addClonedDAGToDAG(
|
|||||||
if (original_dag_node->type == ActionsDAG::ActionType::INPUT)
|
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);
|
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;
|
return new_node;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -109,7 +105,7 @@ const ActionsDAG::Node & addClonedDAGToDAG(
|
|||||||
{
|
{
|
||||||
const auto & new_node = new_dag->addColumn(
|
const auto & new_node = new_dag->addColumn(
|
||||||
ColumnWithTypeAndName(original_dag_node->column, original_dag_node->result_type, original_dag_node->result_name));
|
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;
|
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 & 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);
|
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;
|
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);
|
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;
|
return new_node;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -142,9 +138,11 @@ const ActionsDAG::Node & addClonedDAGToDAG(
|
|||||||
const ActionsDAG::Node & addFunction(
|
const ActionsDAG::Node & addFunction(
|
||||||
const ActionsDAGPtr & new_dag,
|
const ActionsDAGPtr & new_dag,
|
||||||
const FunctionOverloadResolverPtr & function,
|
const FunctionOverloadResolverPtr & function,
|
||||||
ActionsDAG::NodeRawConstPtrs children)
|
ActionsDAG::NodeRawConstPtrs children,
|
||||||
|
OriginalToNewNodeMap & node_remap)
|
||||||
{
|
{
|
||||||
const auto & new_node = new_dag->addFunction(function, children, "");
|
const auto & new_node = new_dag->addFunction(function, children, "");
|
||||||
|
node_remap[new_node.result_name] = {new_dag.get(), &new_node};
|
||||||
return new_node;
|
return new_node;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -154,12 +152,14 @@ const ActionsDAG::Node & addFunction(
|
|||||||
const ActionsDAG::Node & addCast(
|
const ActionsDAG::Node & addCast(
|
||||||
const ActionsDAGPtr & dag,
|
const ActionsDAGPtr & dag,
|
||||||
const ActionsDAG::Node & node_to_cast,
|
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))
|
if (!node_to_cast.result_type->equals(*to_type))
|
||||||
return node_to_cast;
|
return node_to_cast;
|
||||||
|
|
||||||
const auto & new_node = dag->addCast(node_to_cast, to_type, {});
|
const auto & new_node = dag->addCast(node_to_cast, to_type, {});
|
||||||
|
node_remap[new_node.result_name] = {dag.get(), &new_node};
|
||||||
return 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.
|
/// 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 ActionsDAG::Node & addAndTrue(
|
||||||
const ActionsDAGPtr & dag,
|
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);
|
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));
|
const auto * const_true_node = &dag->addColumn(std::move(const_true_column));
|
||||||
ActionsDAG::NodeRawConstPtrs children = {&filter_node_to_normalize, const_true_node};
|
ActionsDAG::NodeRawConstPtrs children = {&filter_node_to_normalize, const_true_node};
|
||||||
FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
|
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
|
/// 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
|
/// 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
|
/// 8. Add computation of the remaining outputs to the last step with the procedure similar to 4
|
||||||
bool tryBuildPrewhereSteps(
|
bool tryBuildPrewhereSteps(PrewhereInfoPtr prewhere_info, const ExpressionActionsSettings & actions_settings, PrewhereExprInfo & prewhere)
|
||||||
PrewhereInfoPtr prewhere_info,
|
|
||||||
const ExpressionActionsSettings & actions_settings,
|
|
||||||
PrewhereExprInfo & prewhere,
|
|
||||||
bool force_short_circuit_execution)
|
|
||||||
{
|
{
|
||||||
if (!prewhere_info)
|
if (!prewhere_info)
|
||||||
return true;
|
return true;
|
||||||
@ -246,10 +243,7 @@ bool tryBuildPrewhereSteps(
|
|||||||
struct Step
|
struct Step
|
||||||
{
|
{
|
||||||
ActionsDAGPtr actions;
|
ActionsDAGPtr actions;
|
||||||
/// Original condition, in case if we have only one condition, and it was not casted
|
String column_name;
|
||||||
const ActionsDAG::Node * original_node;
|
|
||||||
/// Result condition node
|
|
||||||
const ActionsDAG::Node * result_node;
|
|
||||||
};
|
};
|
||||||
std::vector<Step> steps;
|
std::vector<Step> steps;
|
||||||
|
|
||||||
@ -260,8 +254,7 @@ bool tryBuildPrewhereSteps(
|
|||||||
{
|
{
|
||||||
const auto & condition_group = condition_groups[step_index];
|
const auto & condition_group = condition_groups[step_index];
|
||||||
ActionsDAGPtr step_dag = std::make_unique<ActionsDAG>();
|
ActionsDAGPtr step_dag = std::make_unique<ActionsDAG>();
|
||||||
const ActionsDAG::Node * original_node = nullptr;
|
String result_name;
|
||||||
const ActionsDAG::Node * result_node;
|
|
||||||
|
|
||||||
std::vector<const ActionsDAG::Node *> new_condition_nodes;
|
std::vector<const ActionsDAG::Node *> new_condition_nodes;
|
||||||
for (const auto * node : condition_group)
|
for (const auto * node : condition_group)
|
||||||
@ -274,37 +267,48 @@ bool tryBuildPrewhereSteps(
|
|||||||
{
|
{
|
||||||
/// Add AND function to combine the conditions
|
/// Add AND function to combine the conditions
|
||||||
FunctionOverloadResolverPtr func_builder_and = std::make_unique<FunctionToOverloadResolverAdaptor>(std::make_shared<FunctionAnd>());
|
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);
|
const auto & and_function_node = addFunction(step_dag, func_builder_and, new_condition_nodes, node_remap);
|
||||||
result_node = &and_function_node;
|
step_dag->addOrReplaceInOutputs(and_function_node);
|
||||||
|
result_name = and_function_node.result_name;
|
||||||
}
|
}
|
||||||
else
|
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.
|
/// 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.
|
/// 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), result_name});
|
||||||
steps.push_back({std::move(step_dag), original_node, result_node});
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// 6. Find all outputs of the original DAG
|
/// 6. Find all outputs of the original DAG
|
||||||
auto original_outputs = prewhere_info->prewhere_actions.getOutputs();
|
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
|
/// 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
|
/// 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)
|
for (const auto * output : original_outputs)
|
||||||
{
|
{
|
||||||
all_outputs.insert(output);
|
all_output_names.insert(output->result_name);
|
||||||
if (node_remap.contains(output))
|
if (node_remap.contains(output->result_name))
|
||||||
{
|
{
|
||||||
const auto & new_node_info = node_remap[output];
|
const auto & new_node_info = node_remap[output->result_name];
|
||||||
new_node_info.dag->getOutputs().push_back(new_node_info.node);
|
new_node_info.dag->addOrReplaceInOutputs(*new_node_info.node);
|
||||||
}
|
}
|
||||||
else if (output->result_name == prewhere_info->prewhere_column_name)
|
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
|
/// 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)
|
/// 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
|
/// 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_dag = steps.back().actions;
|
||||||
auto & last_step_result_node = steps.back().result_node;
|
|
||||||
/// Build AND(last_step_result_node, true)
|
/// 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)
|
/// 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
|
/// 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);
|
const auto & prewhere_result_node = last_step_dag->addAlias(cast_node, output->result_name);
|
||||||
last_step_dag->getOutputs().push_back(&prewhere_result_node);
|
last_step_dag->addOrReplaceInOutputs(prewhere_result_node);
|
||||||
steps.back().result_node = &prewhere_result_node;
|
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
const auto & node_in_new_dag = addClonedDAGToDAG(steps.size() - 1, output, steps.back().actions, node_remap, node_to_step);
|
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,
|
.type = PrewhereExprStep::Filter,
|
||||||
.actions = std::make_shared<ExpressionActions>(std::move(*step.actions), actions_settings),
|
.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
|
/// Don't remove if it's in the list of original outputs
|
||||||
.remove_filter_column =
|
.remove_filter_column =
|
||||||
step.original_node && !all_outputs.contains(step.original_node) && node_to_step[step.original_node] <= step_index,
|
!all_output_names.contains(step.column_name) && node_to_step[step.column_name] <= step_index,
|
||||||
.need_filter = force_short_circuit_execution,
|
.need_filter = false,
|
||||||
.perform_alter_conversions = true,
|
.perform_alter_conversions = true,
|
||||||
};
|
};
|
||||||
|
|
||||||
prewhere.steps.push_back(std::make_shared<PrewhereExprStep>(std::move(new_step)));
|
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;
|
prewhere.steps.back()->need_filter = prewhere_info->need_filter;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -82,12 +82,12 @@ Filter column: notEquals(__table1.y, 0_UInt8)
|
|||||||
9 10
|
9 10
|
||||||
> one condition of filter should be pushed down after aggregating, other two conditions are ANDed
|
> one condition of filter should be pushed down after aggregating, other two conditions are ANDed
|
||||||
Filter column
|
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
|
Aggregating
|
||||||
Filter column: notEquals(y, 0)
|
Filter column: notEquals(y, 0)
|
||||||
> (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed
|
> (analyzer) one condition of filter should be pushed down after aggregating, other two conditions are ANDed
|
||||||
Filter column
|
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
|
Aggregating
|
||||||
Filter column: notEquals(__table1.y, 0_UInt8)
|
Filter column: notEquals(__table1.y, 0_UInt8)
|
||||||
0 1
|
0 1
|
||||||
@ -163,6 +163,7 @@ Filter column: notEquals(__table1.y, 2_UInt8)
|
|||||||
> filter is pushed down before CreatingSets
|
> filter is pushed down before CreatingSets
|
||||||
CreatingSets
|
CreatingSets
|
||||||
Filter
|
Filter
|
||||||
|
Filter
|
||||||
1
|
1
|
||||||
3
|
3
|
||||||
> one condition of filter is pushed down before LEFT JOIN
|
> one condition of filter is pushed down before LEFT JOIN
|
||||||
|
@ -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
|
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
|
) where y != 0 and s - 8 and s - 4
|
||||||
settings enable_optimize_predicate_expression=0" |
|
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"
|
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 "
|
$CLICKHOUSE_CLIENT --enable_analyzer=1 --convert_query_to_cnf=0 -q "
|
||||||
explain actions = 1 select s, y from (
|
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
|
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
|
) where y != 0 and s - 8 and s - 4
|
||||||
settings enable_optimize_predicate_expression=0" |
|
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 "
|
$CLICKHOUSE_CLIENT -q "
|
||||||
select s, y from (
|
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
|
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
|
||||||
|
@ -332,7 +332,8 @@ SETTINGS optimize_aggregators_of_group_by_keys=0 -- avoid removing any() as it d
|
|||||||
Expression (Projection)
|
Expression (Projection)
|
||||||
Sorting (Sorting for ORDER BY)
|
Sorting (Sorting for ORDER BY)
|
||||||
Expression (Before ORDER BY)
|
Expression (Before ORDER BY)
|
||||||
Filter (((WHERE + (Projection + Before ORDER BY)) + HAVING))
|
Filter ((WHERE + (Projection + Before ORDER BY)))
|
||||||
|
Filter (HAVING)
|
||||||
Aggregating
|
Aggregating
|
||||||
Expression ((Before GROUP BY + Projection))
|
Expression ((Before GROUP BY + Projection))
|
||||||
Sorting (Sorting for ORDER BY)
|
Sorting (Sorting for ORDER BY)
|
||||||
|
@ -28,7 +28,11 @@ WHERE type_1 = \'all\'
|
|||||||
(Expression)
|
(Expression)
|
||||||
ExpressionTransform × 2
|
ExpressionTransform × 2
|
||||||
(Filter)
|
(Filter)
|
||||||
FilterTransform × 6
|
FilterTransform × 2
|
||||||
|
(Filter)
|
||||||
|
FilterTransform × 2
|
||||||
|
(Filter)
|
||||||
|
FilterTransform × 2
|
||||||
(Aggregating)
|
(Aggregating)
|
||||||
ExpressionTransform × 2
|
ExpressionTransform × 2
|
||||||
AggregatingTransform × 2
|
AggregatingTransform × 2
|
||||||
@ -64,6 +68,10 @@ ExpressionTransform × 2
|
|||||||
ExpressionTransform × 2
|
ExpressionTransform × 2
|
||||||
AggregatingTransform × 2
|
AggregatingTransform × 2
|
||||||
Copy 1 → 2
|
Copy 1 → 2
|
||||||
|
(Filter)
|
||||||
|
FilterTransform
|
||||||
|
(Filter)
|
||||||
|
FilterTransform
|
||||||
(Expression)
|
(Expression)
|
||||||
ExpressionTransform
|
ExpressionTransform
|
||||||
(ReadFromMergeTree)
|
(ReadFromMergeTree)
|
||||||
|
@ -163,21 +163,17 @@ Positions: 4 2 0 1
|
|||||||
Filter (( + (JOIN actions + Change column names to column identifiers)))
|
Filter (( + (JOIN actions + Change column names to column identifiers)))
|
||||||
Header: __table1.id UInt64
|
Header: __table1.id UInt64
|
||||||
__table1.value String
|
__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)
|
Filter column: and(equals(__table1.id, 5_UInt8), equals(__table1.id, 6_UInt8)) (removed)
|
||||||
Actions: INPUT : 2 -> value String : 0
|
Actions: INPUT : 0 -> id UInt64 : 0
|
||||||
INPUT : 1 -> id UInt64 : 1
|
INPUT : 1 -> value String : 1
|
||||||
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 2
|
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 2
|
||||||
INPUT : 0 -> equals(__table1.id, 5_UInt8) UInt8 : 3
|
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 3
|
||||||
ALIAS value :: 0 -> __table1.value String : 4
|
ALIAS id : 0 -> __table1.id UInt64 : 4
|
||||||
ALIAS id : 1 -> __table1.id UInt64 : 0
|
ALIAS value :: 1 -> __table1.value String : 5
|
||||||
FUNCTION equals(id :: 1, 6_UInt8 :: 2) -> equals(__table1.id, 6_UInt8) UInt8 : 5
|
FUNCTION equals(id : 0, 6_UInt8 :: 2) -> equals(__table1.id, 6_UInt8) UInt8 : 1
|
||||||
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
|
FUNCTION equals(id :: 0, 5_UInt8 :: 3) -> equals(__table1.id, 5_UInt8) UInt8 : 2
|
||||||
Positions: 2 0 4
|
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)
|
ReadFromMergeTree (default.test_table_1)
|
||||||
Header: id UInt64
|
Header: id UInt64
|
||||||
value String
|
value String
|
||||||
@ -187,21 +183,17 @@ Positions: 4 2 0 1
|
|||||||
Filter (( + (JOIN actions + Change column names to column identifiers)))
|
Filter (( + (JOIN actions + Change column names to column identifiers)))
|
||||||
Header: __table2.id UInt64
|
Header: __table2.id UInt64
|
||||||
__table2.value String
|
__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)
|
Filter column: and(equals(__table2.id, 6_UInt8), equals(__table2.id, 5_UInt8)) (removed)
|
||||||
Actions: INPUT : 2 -> value String : 0
|
Actions: INPUT : 0 -> id UInt64 : 0
|
||||||
INPUT : 1 -> id UInt64 : 1
|
INPUT : 1 -> value String : 1
|
||||||
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2
|
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 2
|
||||||
INPUT : 0 -> equals(__table2.id, 6_UInt8) UInt8 : 3
|
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 3
|
||||||
ALIAS value :: 0 -> __table2.value String : 4
|
ALIAS id : 0 -> __table2.id UInt64 : 4
|
||||||
ALIAS id : 1 -> __table2.id UInt64 : 0
|
ALIAS value :: 1 -> __table2.value String : 5
|
||||||
FUNCTION equals(id :: 1, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 5
|
FUNCTION equals(id : 0, 5_UInt8 :: 2) -> equals(__table2.id, 5_UInt8) UInt8 : 1
|
||||||
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
|
FUNCTION equals(id :: 0, 6_UInt8 :: 3) -> equals(__table2.id, 6_UInt8) UInt8 : 2
|
||||||
Positions: 2 0 4
|
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)
|
ReadFromMergeTree (default.test_table_2)
|
||||||
Header: id UInt64
|
Header: id UInt64
|
||||||
value String
|
value String
|
||||||
@ -664,21 +656,17 @@ Positions: 4 2 0 1
|
|||||||
__table1.value String
|
__table1.value String
|
||||||
__table2.value String
|
__table2.value String
|
||||||
__table2.id UInt64
|
__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)
|
Filter column: and(equals(__table1.id, 5_UInt8), equals(__table2.id, 6_UInt8)) (removed)
|
||||||
Actions: INPUT :: 1 -> __table1.id UInt64 : 0
|
Actions: INPUT : 0 -> __table1.id UInt64 : 0
|
||||||
INPUT :: 2 -> __table1.value String : 1
|
INPUT :: 1 -> __table1.value String : 1
|
||||||
INPUT :: 3 -> __table2.value String : 2
|
INPUT :: 2 -> __table2.value String : 2
|
||||||
INPUT : 4 -> __table2.id UInt64 : 3
|
INPUT : 3 -> __table2.id UInt64 : 3
|
||||||
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 4
|
COLUMN Const(UInt8) -> 5_UInt8 UInt8 : 4
|
||||||
INPUT : 0 -> equals(__table1.id, 5_UInt8) UInt8 : 5
|
COLUMN Const(UInt8) -> 6_UInt8 UInt8 : 5
|
||||||
FUNCTION equals(__table2.id : 3, 6_UInt8 :: 4) -> equals(__table2.id, 6_UInt8) UInt8 : 6
|
FUNCTION equals(__table1.id : 0, 5_UInt8 :: 4) -> equals(__table1.id, 5_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
|
FUNCTION equals(__table2.id : 3, 6_UInt8 :: 5) -> equals(__table2.id, 6_UInt8) UInt8 : 4
|
||||||
Positions: 4 0 1 2 3
|
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)
|
Join (JOIN FillRightFirst)
|
||||||
Header: __table1.id UInt64
|
Header: __table1.id UInt64
|
||||||
__table1.value String
|
__table1.value String
|
||||||
|
@ -1,5 +1,3 @@
|
|||||||
set allow_reorder_prewhere_conditions=0;
|
|
||||||
|
|
||||||
drop table if exists t1;
|
drop table if exists t1;
|
||||||
drop table if exists t2;
|
drop table if exists t2;
|
||||||
|
|
||||||
@ -51,23 +49,7 @@ tmp1 AS
|
|||||||
fs1
|
fs1
|
||||||
FROM t2
|
FROM t2
|
||||||
LEFT JOIN tmp1 USING (fs1)
|
LEFT JOIN tmp1 USING (fs1)
|
||||||
WHERE (fs1 IN ('test')) SETTINGS enable_multiple_prewhere_read_steps = 0, query_plan_merge_filters=0;
|
WHERE (fs1 IN ('test')) SETTINGS enable_multiple_prewhere_read_steps = 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;
|
|
||||||
|
|
||||||
optimize table t1 final;
|
optimize table t1 final;
|
||||||
|
|
||||||
@ -85,20 +67,4 @@ tmp1 AS
|
|||||||
fs1
|
fs1
|
||||||
FROM t2
|
FROM t2
|
||||||
LEFT JOIN tmp1 USING (fs1)
|
LEFT JOIN tmp1 USING (fs1)
|
||||||
WHERE (fs1 IN ('test')) SETTINGS enable_multiple_prewhere_read_steps = 0, query_plan_merge_filters=0;
|
WHERE (fs1 IN ('test'));
|
||||||
|
|
||||||
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;
|
|
||||||
|
@ -1,2 +0,0 @@
|
|||||||
Condition: and((materialize(auid) in [1, 1]), (_CAST(toDate(ts)) in (-Inf, 1703980800]))
|
|
||||||
Granules: 1/3
|
|
@ -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;
|
|
Loading…
Reference in New Issue
Block a user