mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-13 09:52:38 +00:00
Read-in-order over query plan (continuation)
This commit is contained in:
parent
79b30fe133
commit
5d41e7a6d5
@ -193,10 +193,24 @@ void enreachFixedColumns(ActionsDAGPtr & dag, FixedColumns & fixed_columns)
|
||||
}
|
||||
}
|
||||
|
||||
/// Here we try to find inner DAG inside outer DAG.
|
||||
/// Build a map: inner.nodes -> outer.nodes.
|
||||
// using NodesMap = std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *>;
|
||||
int isMonotonicSubtree(const ActionsDAG::Node * inner, const ActionsDAG::Node * outer)
|
||||
struct MatchedTrees
|
||||
{
|
||||
struct Monotonicity
|
||||
{
|
||||
int direction = 1;
|
||||
bool strict = true;
|
||||
};
|
||||
|
||||
struct Match
|
||||
{
|
||||
const ActionsDAG::Node * node = nullptr;
|
||||
std::optional<Monotonicity> monotonicity;
|
||||
};
|
||||
|
||||
using Matches = std::unordered_map<const ActionsDAG::Node *, Match>;
|
||||
};
|
||||
|
||||
MatchedTrees::Matches matchTrees(const ActionsDAG & inner_dag, const ActionsDAG & outer_dag)
|
||||
{
|
||||
using Parents = std::set<const ActionsDAG::Node *>;
|
||||
std::unordered_map<const ActionsDAG::Node *, Parents> inner_parents;
|
||||
@ -204,8 +218,13 @@ int isMonotonicSubtree(const ActionsDAG::Node * inner, const ActionsDAG::Node *
|
||||
|
||||
{
|
||||
std::stack<const ActionsDAG::Node *> stack;
|
||||
stack.push(inner);
|
||||
inner_parents.emplace(inner, Parents());
|
||||
for (const auto * out : inner_dag.getOutputs())
|
||||
{
|
||||
if (inner_parents.contains(out))
|
||||
continue;
|
||||
|
||||
stack.push(out);
|
||||
inner_parents.emplace(out, Parents());
|
||||
while (!stack.empty())
|
||||
{
|
||||
const auto * node = stack.top();
|
||||
@ -224,20 +243,20 @@ int isMonotonicSubtree(const ActionsDAG::Node * inner, const ActionsDAG::Node *
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> outer_to_inner;
|
||||
std::unordered_map<const ActionsDAG::Node *, int> direction;
|
||||
|
||||
{
|
||||
struct Frame
|
||||
{
|
||||
const ActionsDAG::Node * node;
|
||||
ActionsDAG::NodeRawConstPtrs mapped_children;
|
||||
int direction = 1;
|
||||
};
|
||||
|
||||
MatchedTrees::Matches matches;
|
||||
|
||||
for (const auto * out : outer_dag.getOutputs())
|
||||
{
|
||||
std::stack<Frame> stack;
|
||||
stack.push(Frame{outer, {}});
|
||||
stack.push(Frame{out, {}});
|
||||
while (!stack.empty())
|
||||
{
|
||||
auto & frame = stack.top();
|
||||
@ -246,13 +265,13 @@ int isMonotonicSubtree(const ActionsDAG::Node * inner, const ActionsDAG::Node *
|
||||
while (frame.mapped_children.size() < frame.node->children.size())
|
||||
{
|
||||
const auto * child = frame.node->children[frame.mapped_children.size()];
|
||||
auto it = outer_to_inner.find(child);
|
||||
if (it == outer_to_inner.end())
|
||||
auto it = matches.find(child);
|
||||
if (it == matches.end())
|
||||
{
|
||||
stack.push(Frame{child, {}});
|
||||
break;
|
||||
}
|
||||
frame.mapped_children.push_back(it->second);
|
||||
frame.mapped_children.push_back(it->second.node);
|
||||
}
|
||||
|
||||
if (frame.mapped_children.size() < frame.node->children.size())
|
||||
@ -264,59 +283,22 @@ int isMonotonicSubtree(const ActionsDAG::Node * inner, const ActionsDAG::Node *
|
||||
if (auto it = inner_inputs.find(frame.node->result_name); it != inner_inputs.end())
|
||||
mapped = it->second;
|
||||
|
||||
outer_to_inner.emplace(frame.node, mapped);
|
||||
matches.emplace(frame.node, MatchedTrees::Match{.node = mapped});
|
||||
}
|
||||
else if (frame.node->type == ActionsDAG::ActionType::ALIAS)
|
||||
{
|
||||
outer_to_inner.emplace(frame.node, frame.mapped_children.at(0));
|
||||
matches.emplace(frame.node, matches[frame.node->children.at(0)]);
|
||||
}
|
||||
else if (frame.node->type == ActionsDAG::ActionType::FUNCTION)
|
||||
{
|
||||
auto & match = matches[frame.node];
|
||||
|
||||
bool found_all_children = true;
|
||||
size_t num_found_inner_roots = 0;
|
||||
for (const auto * child : frame.mapped_children)
|
||||
{
|
||||
if (!child)
|
||||
found_all_children = false;
|
||||
else if (child == inner)
|
||||
++num_found_inner_roots;
|
||||
}
|
||||
|
||||
bool found_monotonic_wrapper = false;
|
||||
if (num_found_inner_roots == 1)
|
||||
{
|
||||
if (frame.node->function_base->hasInformationAboutMonotonicity())
|
||||
{
|
||||
size_t num_const_args = 0;
|
||||
const ActionsDAG::Node * monotonic_child = nullptr;
|
||||
for (const auto * child : frame.node->children)
|
||||
{
|
||||
if (child->column)
|
||||
++num_const_args;
|
||||
else
|
||||
monotonic_child = child;
|
||||
}
|
||||
|
||||
if (monotonic_child && num_const_args + 1 == frame.node->children.size())
|
||||
{
|
||||
auto info = frame.node->function_base->getMonotonicityForRange(*monotonic_child->result_type, {}, {});
|
||||
if (info.is_always_monotonic)
|
||||
{
|
||||
found_monotonic_wrapper = true;
|
||||
outer_to_inner[frame.node] = inner;
|
||||
|
||||
int cur_direction = info.is_positive ? 1 : -1;
|
||||
auto it = direction.find(monotonic_child);
|
||||
if (it != direction.end())
|
||||
cur_direction *= it->second;
|
||||
|
||||
direction[frame.node] = cur_direction;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!found_monotonic_wrapper && found_all_children && !frame.mapped_children.empty())
|
||||
if (found_all_children && !frame.mapped_children.empty())
|
||||
{
|
||||
Parents container;
|
||||
Parents * intersection = &inner_parents[frame.mapped_children[0]];
|
||||
@ -351,8 +333,52 @@ int isMonotonicSubtree(const ActionsDAG::Node * inner, const ActionsDAG::Node *
|
||||
{
|
||||
auto func_name = frame.node->function_base->getName();
|
||||
for (const auto * parent : *intersection)
|
||||
{
|
||||
if (parent->type == ActionsDAG::ActionType::FUNCTION && func_name == parent->function_base->getName())
|
||||
outer_to_inner[frame.node] = parent;
|
||||
{
|
||||
match.node = parent;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!match.node && frame.node->function_base->hasInformationAboutMonotonicity())
|
||||
{
|
||||
size_t num_const_args = 0;
|
||||
const ActionsDAG::Node * monotonic_child = nullptr;
|
||||
for (const auto * child : frame.node->children)
|
||||
{
|
||||
if (child->column)
|
||||
++num_const_args;
|
||||
else
|
||||
monotonic_child = child;
|
||||
}
|
||||
|
||||
if (monotonic_child && num_const_args + 1 == frame.node->children.size())
|
||||
{
|
||||
const auto & child_match = matches[monotonic_child];
|
||||
if (child_match.node)
|
||||
{
|
||||
auto info = frame.node->function_base->getMonotonicityForRange(*monotonic_child->result_type, {}, {});
|
||||
if (info.is_always_monotonic)
|
||||
{
|
||||
match.node = child_match.node;
|
||||
|
||||
MatchedTrees::Monotonicity monotonicity;
|
||||
monotonicity.direction *= info.is_positive ? 1 : -1;
|
||||
monotonicity.strict = info.is_strict;
|
||||
|
||||
if (child_match.monotonicity)
|
||||
{
|
||||
monotonicity.direction *= child_match.monotonicity->direction;
|
||||
if (!child_match.monotonicity->strict)
|
||||
monotonicity.strict = false;
|
||||
}
|
||||
|
||||
match.monotonicity = monotonicity;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -361,84 +387,92 @@ int isMonotonicSubtree(const ActionsDAG::Node * inner, const ActionsDAG::Node *
|
||||
}
|
||||
}
|
||||
|
||||
if (outer_to_inner[outer] != inner)
|
||||
return 0;
|
||||
|
||||
int res = 1;
|
||||
if (auto it = direction.find(outer); it != direction.end())
|
||||
res = it->second;
|
||||
|
||||
return res;
|
||||
return matches;
|
||||
}
|
||||
|
||||
SortDescription buildPrefixSortDescription(
|
||||
InputOrderInfoPtr buildInputOrderInfo(
|
||||
const FixedColumns & fixed_columns,
|
||||
const ActionsDAGPtr & dag,
|
||||
const SortDescription & description,
|
||||
const ActionsDAG & sorting_key_dag,
|
||||
const Names & sorting_key_columns,
|
||||
int & read_direction)
|
||||
size_t limit)
|
||||
{
|
||||
SortDescription order_key_prefix_descr;
|
||||
order_key_prefix_descr.reserve(description.size());
|
||||
|
||||
MatchedTrees::Matches matches;
|
||||
if (dag)
|
||||
matches = matchTrees(sorting_key_dag, *dag);
|
||||
|
||||
/// This is a result direction we will read from MergeTree
|
||||
/// 1 - in order,
|
||||
/// -1 - in reverse order,
|
||||
/// 0 - usual read, don't apply optimization
|
||||
///
|
||||
/// So far, 0 means any direction is possible. It is ok for constant prefix.
|
||||
read_direction = 0;
|
||||
int read_direction = 0;
|
||||
size_t next_descr_column = 0;
|
||||
size_t next_sort_key = 0;
|
||||
|
||||
for (size_t i = 0, next_sort_key = 0; i < description.size() && next_sort_key < sorting_key_columns.size(); ++i)
|
||||
for (; next_descr_column < description.size() && next_sort_key < sorting_key_columns.size(); ++next_sort_key)
|
||||
{
|
||||
const auto & sort_column = description[i];
|
||||
const auto & sorting_key_column = sorting_key_columns[next_sort_key];
|
||||
const auto & descr = description[next_descr_column];
|
||||
|
||||
/// If required order depend on collation, it cannot be matched with primary key order.
|
||||
/// Because primary keys cannot have collations.
|
||||
if (sort_column.collator)
|
||||
return order_key_prefix_descr;
|
||||
if (descr.collator)
|
||||
break;
|
||||
|
||||
/// Direction for current sort key.
|
||||
int current_direction = 0;
|
||||
|
||||
if (!dag)
|
||||
{
|
||||
if (sort_column.column_name != sorting_key_column)
|
||||
return order_key_prefix_descr;
|
||||
|
||||
current_direction = sort_column.direction;
|
||||
++next_sort_key;
|
||||
}
|
||||
else
|
||||
{
|
||||
const ActionsDAG::Node * sort_node = dag->tryFindInOutputs(sort_column.column_name);
|
||||
/// It is possible when e.g. sort by array joined column.
|
||||
if (!sort_node)
|
||||
return order_key_prefix_descr;
|
||||
bool strict_monotonic = true;
|
||||
|
||||
const ActionsDAG::Node * sort_column_node = sorting_key_dag.tryFindInOutputs(sorting_key_column);
|
||||
/// This should not happen.
|
||||
if (!sort_column_node)
|
||||
return order_key_prefix_descr;
|
||||
break;
|
||||
|
||||
bool is_fixed_column = sort_node->column || fixed_columns.contains(sort_node);
|
||||
if (!dag)
|
||||
{
|
||||
if (sort_column_node->type != ActionsDAG::ActionType::INPUT)
|
||||
break;
|
||||
|
||||
/// We try to find the match even if column is fixed. In this case, potentially more keys will match.
|
||||
if (descr.column_name != sorting_key_column)
|
||||
break;
|
||||
|
||||
current_direction = descr.direction;
|
||||
}
|
||||
else
|
||||
{
|
||||
const ActionsDAG::Node * sort_node = dag->tryFindInOutputs(descr.column_name);
|
||||
/// It is possible when e.g. sort by array joined column.
|
||||
if (!sort_node)
|
||||
break;
|
||||
|
||||
const auto & match = matches[sort_node];
|
||||
|
||||
if (match.node)
|
||||
{
|
||||
/// We try to find the match first even if column is fixed. In this case, potentially more keys will match.
|
||||
/// Example: 'table (x Int32, y Int32) ORDER BY x + 1, y + 1'
|
||||
/// 'SELECT x, y FROM table WHERE x = 42 ORDER BY x + 1, y + 1'
|
||||
/// Here, 'x + 1' would be a fixed point. But it is reasonable to read-in-order.
|
||||
current_direction = isMonotonicSubtree(sort_column_node, sort_node) * sort_column.direction;
|
||||
|
||||
if (current_direction == 0 || !is_fixed_column)
|
||||
return order_key_prefix_descr;
|
||||
|
||||
if (current_direction)
|
||||
++next_sort_key;
|
||||
|
||||
if (is_fixed_column)
|
||||
current_direction = 0;
|
||||
current_direction = 1;
|
||||
if (match.monotonicity)
|
||||
{
|
||||
current_direction *= match.monotonicity->direction;
|
||||
strict_monotonic = match.monotonicity->strict;
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
bool is_fixed_column = sort_node->column || fixed_columns.contains(sort_node);
|
||||
if (!is_fixed_column)
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
/// read_direction == 0 means we can choose any global direction.
|
||||
@ -446,11 +480,23 @@ SortDescription buildPrefixSortDescription(
|
||||
if (current_direction && read_direction && current_direction != read_direction)
|
||||
break;
|
||||
|
||||
if (read_direction == 0)
|
||||
read_direction = current_direction;
|
||||
order_key_prefix_descr.push_back(description[i]);
|
||||
|
||||
if (current_direction)
|
||||
{
|
||||
order_key_prefix_descr.push_back(description[next_descr_column]);
|
||||
++next_descr_column;
|
||||
|
||||
if (!strict_monotonic)
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return order_key_prefix_descr;
|
||||
if (read_direction == 0 || order_key_prefix_descr.empty())
|
||||
return nullptr;
|
||||
|
||||
return std::make_shared<InputOrderInfo>(order_key_prefix_descr, next_sort_key, read_direction, limit);
|
||||
}
|
||||
|
||||
void optimizeReadInOrder(QueryPlan::Node & node)
|
||||
@ -475,30 +521,20 @@ void optimizeReadInOrder(QueryPlan::Node & node)
|
||||
buildSortingDAG(node.children.front(), dag, fixed_columns);
|
||||
|
||||
const auto & description = sorting->getSortDescription();
|
||||
auto limit = sorting->getLimit();
|
||||
const auto & sorting_key_columns = sorting_key.column_names;
|
||||
|
||||
int read_direction = 0;
|
||||
auto prefix_description = buildPrefixSortDescription(
|
||||
auto order_info = buildInputOrderInfo(
|
||||
fixed_columns,
|
||||
dag, description,
|
||||
sorting_key.expression->getActionsDAG(), sorting_key_columns,
|
||||
read_direction);
|
||||
limit);
|
||||
|
||||
/// It is possible that prefix_description is not empty, but read_direction is 0.
|
||||
/// It means that some prefix of sorting key matched, but it was constant.
|
||||
/// In this case, read-in-order is useless.
|
||||
if (read_direction == 0 || prefix_description.empty())
|
||||
if (!order_info)
|
||||
return;
|
||||
|
||||
auto limit = sorting->getLimit();
|
||||
|
||||
auto order_info = std::make_shared<InputOrderInfo>(
|
||||
SortDescription{},
|
||||
std::move(prefix_description),
|
||||
read_direction, limit);
|
||||
|
||||
reading->setQueryInfoInputOrderInfo(order_info);
|
||||
sorting->convertToFinishSorting(order_info->order_key_prefix_descr);
|
||||
reading->requestReadingInOrder(order_info->used_prefix_of_sorting_key_size, order_info->direction, order_info->limit);
|
||||
sorting->convertToFinishSorting(order_info->sort_description_for_merging);
|
||||
}
|
||||
|
||||
size_t tryReuseStorageOrderingForWindowFunctions(QueryPlan::Node * parent_node, QueryPlan::Nodes & /*nodes*/)
|
||||
|
Loading…
Reference in New Issue
Block a user