Update KeyCondition constructor

This commit is contained in:
Nikolai Kochetov 2021-06-22 13:28:56 +03:00
parent d15d16fee0
commit 21e39e10ea
4 changed files with 34 additions and 20 deletions

View File

@ -161,6 +161,9 @@ const ActionsDAG::Node & ActionsDAG::addArrayJoin(const Node & child, std::strin
if (!array_type)
throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH);
if (result_name.empty())
result_name = "arrayJoin(" + child.result_name + ")";
Node node;
node.type = ActionType::ARRAY_JOIN;
node.result_type = array_type->getNestedType();

View File

@ -279,4 +279,10 @@ private:
static ActionsDAGPtr cloneActionsForConjunction(NodeRawConstPtrs conjunction, const ColumnsWithTypeAndName & all_inputs);
};
/// This is an ugly way to bypass impossibility to forward declare ActionDAG::Node.
struct ActionDAGNodes
{
ActionsDAG::NodeRawConstPtrs nodes;
};
}

View File

@ -644,11 +644,14 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown(
switch (node.type)
{
case (ActionsDAG::ActionType::INPUT):
/// Should be already added
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot clone input in cloneASTWithInversionPushDown");
{
/// Note: inputs order is not important here. Will match columns by names.
res = &inverted_dag.addInput({node.column, node.result_type, node.result_name});
break;
}
case (ActionsDAG::ActionType::COLUMN):
{
res = &inverted_dag.addInput({node.column, node.result_type, node.result_name});
res = &inverted_dag.addColumn({node.column, node.result_type, node.result_name});
break;
}
case (ActionsDAG::ActionType::ALIAS):
@ -661,7 +664,7 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown(
case (ActionsDAG::ActionType::ARRAY_JOIN):
{
const auto & arg = cloneASTWithInversionPushDown(*node.children.front(), inverted_dag, to_inverted, context, false);
res = &inverted_dag.addArrayJoin(arg, node.result_name);
res = &inverted_dag.addArrayJoin(arg, "");
break;
}
case (ActionsDAG::ActionType::FUNCTION):
@ -691,6 +694,8 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown(
assert(function_builder);
/// We match columns by name, so it is important to fill name correctly.
/// So, use empty string to make it automatically.
const auto & func = inverted_dag.addFunction(function_builder, children, "");
to_inverted[&node] = &func;
return func;
@ -722,23 +727,22 @@ static const ActionsDAG::Node & cloneASTWithInversionPushDown(
return *res;
}
static ActionsDAGPtr cloneASTWithInversionPushDown(const ActionsDAG & dag, const ContextPtr & context)
static ActionsDAGPtr cloneASTWithInversionPushDown(ActionsDAG::NodeRawConstPtrs nodes, const ContextPtr & context)
{
auto res = std::make_shared<ActionsDAG>(dag.getRequiredColumns());
auto res = std::make_shared<ActionsDAG>();
std::unordered_map<const ActionsDAG::Node *, const ActionsDAG::Node *> to_inverted;
const auto & res_inputs = res->getInputs();
auto it = res_inputs.begin();
for (const auto * input : dag.getInputs())
for (auto & node : nodes)
node = &cloneASTWithInversionPushDown(*node, *res, to_inverted, context, false);
if (nodes.size() > 1)
{
to_inverted[input] = *it;
++it;
auto function_builder = FunctionFactory::instance().get("and", context);
nodes = {&res->addFunction(function_builder, std::move(nodes), "")};
}
ActionsDAG::NodeRawConstPtrs index;
index.reserve(dag.getIndex().size());
for (const auto * node : dag.getIndex())
index.push_back(&cloneASTWithInversionPushDown(*node, *res, to_inverted, context, false));
res->getIndex().swap(nodes);
return res;
}
@ -828,7 +832,7 @@ KeyCondition::KeyCondition(
}
KeyCondition::KeyCondition(
const ActionsDAG & dag,
ActionDAGNodes dag_nodes,
PreparedSets prepared_sets_,
ContextPtr context,
const Names & key_column_names,
@ -848,10 +852,10 @@ KeyCondition::KeyCondition(
key_columns[name] = i;
}
auto inverted_dag = cloneASTWithInversionPushDown(dag, context);
if (!inverted_dag->getIndex().empty())
if (!dag_nodes.nodes.empty())
{
auto inverted_dag = cloneASTWithInversionPushDown(std::move(dag_nodes.nodes), context);
Block empty;
for (const auto * node : inverted_dag->getIndex())
traverseAST(Tree(node), context, empty);

View File

@ -18,6 +18,7 @@ class IFunction;
using FunctionBasePtr = std::shared_ptr<IFunctionBase>;
class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
struct ActionDAGNodes;
/** A field, that can be stored in two representations:
* - A standalone field.
@ -238,7 +239,7 @@ public:
bool strict_ = false);
KeyCondition(
const ActionsDAG & dag,
ActionDAGNodes dag_nodes,
PreparedSets prepared_sets_,
ContextPtr context,
const Names & key_column_names,