ClickHouse/src/Interpreters/ExpressionActions.cpp

1444 lines
44 KiB
C++
Raw Normal View History

2019-10-11 17:27:54 +00:00
#include <Interpreters/Set.h>
#include <Common/ProfileEvents.h>
#include <Common/SipHash.h>
2020-08-13 20:17:18 +00:00
#include <Interpreters/ArrayJoinAction.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/ExpressionJIT.h>
#include <Interpreters/TableJoin.h>
#include <Interpreters/Context.h>
#include <Columns/ColumnsNumber.h>
2020-08-13 20:17:18 +00:00
#include <Columns/ColumnArray.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypesNumber.h>
2020-09-08 10:40:53 +00:00
#include <DataTypes/DataTypeNullable.h>
#include <Functions/IFunction.h>
2020-09-10 16:01:41 +00:00
#include <IO/Operators.h>
#include <optional>
2019-10-11 17:27:54 +00:00
#include <Columns/ColumnSet.h>
2020-09-13 13:51:31 +00:00
#include <queue>
2020-11-03 11:28:28 +00:00
#include <stack>
#if !defined(ARCADIA_BUILD)
# include "config_core.h"
#endif
#include <common/defines.h>
#if defined(MEMORY_SANITIZER)
#include <sanitizer/msan_interface.h>
#endif
#if defined(ADDRESS_SANITIZER)
#include <sanitizer/asan_interface.h>
#endif
namespace ProfileEvents
{
extern const Event FunctionExecute;
extern const Event CompiledFunctionExecute;
}
namespace DB
{
namespace ErrorCodes
{
2020-02-25 18:02:41 +00:00
extern const int LOGICAL_ERROR;
extern const int DUPLICATE_COLUMN;
extern const int UNKNOWN_IDENTIFIER;
extern const int NOT_FOUND_COLUMN_IN_BLOCK;
2018-03-09 23:23:15 +00:00
extern const int TOO_MANY_TEMPORARY_COLUMNS;
extern const int TOO_MANY_TEMPORARY_NON_CONST_COLUMNS;
2020-08-13 20:17:18 +00:00
extern const int TYPE_MISMATCH;
}
2019-07-31 10:22:56 +00:00
/// Read comment near usage
2020-11-03 11:28:28 +00:00
/// static constexpr auto DUMMY_COLUMN_NAME = "_dummy";
2020-11-03 11:28:28 +00:00
static std::ostream & operator << (std::ostream & out, const ExpressionActions::Argument & argument)
{
2020-11-09 15:01:08 +00:00
return out << (argument.needed_later ? ": " : ":: ") << argument.pos;
}
2020-11-03 11:28:28 +00:00
std::string ExpressionActions::Action::toString() const
{
2020-11-03 11:28:28 +00:00
std::stringstream out;
switch (node->type)
{
2020-11-03 11:28:28 +00:00
case ActionsDAG::Type::COLUMN:
out << "COLUMN "
<< (node->column ? node->column->getName() : "(no column)");
break;
2020-11-03 11:28:28 +00:00
case ActionsDAG::Type::ALIAS:
out << "ALIAS " << node->children.front()->result_name << " " << arguments.front();
break;
2020-11-03 11:28:28 +00:00
case ActionsDAG::Type::FUNCTION:
out << "FUNCTION " << (node->is_function_compiled ? "[compiled] " : "")
<< (node->function_base ? node->function_base->getName() : "(no function)") << "(";
for (size_t i = 0; i < node->children.size(); ++i)
{
if (i)
2020-11-03 11:28:28 +00:00
out << ", ";
out << node->children[i]->result_name << " " << arguments[i];
}
2020-11-03 11:28:28 +00:00
out << ")";
break;
2020-11-03 11:28:28 +00:00
case ActionsDAG::Type::ARRAY_JOIN:
out << "ARRAY JOIN " << node->children.front()->result_name << " " << arguments.front();
break;
2020-11-03 11:28:28 +00:00
case ActionsDAG::Type::INPUT:
out << "INPUT " << arguments.front();
break;
}
2020-11-03 11:28:28 +00:00
out << " -> " << node->result_name
2020-11-09 15:01:08 +00:00
<< " " << (node->result_type ? node->result_type->getName() : "(no type)") << " : " << result_position;
2020-11-03 11:28:28 +00:00
return out.str();
}
ExpressionActions::~ExpressionActions() = default;
2020-10-13 08:16:47 +00:00
void ExpressionActions::checkLimits(ExecutionContext & execution_context) const
{
2020-11-03 11:28:28 +00:00
if (max_temporary_non_const_columns)
{
size_t non_const_columns = 0;
2020-11-03 11:28:28 +00:00
for (const auto & column : execution_context.columns)
if (column.column && !isColumnConst(*column.column))
++non_const_columns;
2020-11-03 11:28:28 +00:00
if (non_const_columns > max_temporary_non_const_columns)
{
std::stringstream list_of_non_const_columns;
2020-11-03 11:28:28 +00:00
for (const auto & column : execution_context.columns)
if (column.column && !isColumnConst(*column.column))
list_of_non_const_columns << "\n" << column.name;
throw Exception("Too many temporary non-const columns:" + list_of_non_const_columns.str()
2020-11-03 11:28:28 +00:00
+ ". Maximum: " + std::to_string(max_temporary_non_const_columns),
2018-03-09 23:23:15 +00:00
ErrorCodes::TOO_MANY_TEMPORARY_NON_CONST_COLUMNS);
}
}
}
2020-11-03 11:28:28 +00:00
void ExpressionActions::execute(Block & block, size_t & num_rows, bool dry_run) const
{
2020-10-13 08:16:47 +00:00
ExecutionContext execution_context
{
2020-11-03 11:28:28 +00:00
.inputs = block.data,
.num_rows = num_rows,
2020-10-13 08:16:47 +00:00
};
2020-11-03 11:28:28 +00:00
execution_context.inputs_pos.reserve(required_columns.size());
2020-10-13 08:16:47 +00:00
for (const auto & column : required_columns)
{
2020-11-03 11:28:28 +00:00
ssize_t pos = -1;
if (block.has(column.name))
pos = block.getPositionByName(column.name);
execution_context.inputs_pos.push_back(pos);
}
2020-10-13 08:16:47 +00:00
execution_context.columns.resize(num_columns);
for (const auto & action : actions)
{
try
{
2020-10-13 08:16:47 +00:00
executeAction(action, execution_context, dry_run);
checkLimits(execution_context);
2020-11-03 11:28:28 +00:00
//std::cerr << "Action: " << action.toString() << std::endl;
//for (const auto & col : execution_context.columns)
// std::cerr << col.dumpStructure() << std::endl;
}
catch (Exception & e)
{
e.addMessage(fmt::format("while executing '{}'", action.toString()));
throw;
}
}
2020-10-13 08:16:47 +00:00
2020-11-03 11:28:28 +00:00
if (project_input)
{
block.clear();
}
else
2020-10-13 08:16:47 +00:00
{
2020-11-03 11:28:28 +00:00
std::sort(execution_context.inputs_pos.rbegin(), execution_context.inputs_pos.rend());
for (auto input : execution_context.inputs_pos)
if (input >= 0)
block.erase(input);
}
2020-10-13 08:16:47 +00:00
2020-11-03 11:28:28 +00:00
for (auto pos : result_positions)
if (execution_context.columns[pos].column)
block.insert(execution_context.columns[pos]);
2020-10-13 08:16:47 +00:00
2020-11-03 11:28:28 +00:00
num_rows = execution_context.num_rows;
}
void ExpressionActions::execute(Block & block, bool dry_run) const
{
size_t num_rows = block.rows();
execute(block, num_rows, dry_run);
if (!block)
block.insert({DataTypeUInt8().createColumnConst(num_rows, 0), std::make_shared<DataTypeUInt8>(), "_dummy"});
2013-05-28 12:05:47 +00:00
}
2020-10-07 18:37:27 +00:00
void ExpressionActions::executeAction(const Action & action, ExecutionContext & execution_context, bool dry_run)
{
2020-11-03 11:28:28 +00:00
auto & inputs = execution_context.inputs;
2020-10-13 08:16:47 +00:00
auto & columns = execution_context.columns;
auto & num_rows = execution_context.num_rows;
2020-10-07 18:37:27 +00:00
switch (action.node->type)
{
case ActionsDAG::Type::FUNCTION:
{
2020-10-13 08:16:47 +00:00
auto & res_column = columns[action.result_position];
if (res_column.type || res_column.column)
throw Exception("Result column is not empty", ErrorCodes::LOGICAL_ERROR);
2020-10-07 18:37:27 +00:00
2020-10-13 08:16:47 +00:00
res_column.type = action.node->result_type;
2020-11-03 11:28:28 +00:00
res_column.name = action.node->result_name;
ColumnsWithTypeAndName arguments(action.arguments.size());
for (size_t i = 0; i < arguments.size(); ++i)
{
2020-11-09 15:01:08 +00:00
if (!action.arguments[i].needed_later)
2020-11-03 11:28:28 +00:00
arguments[i] = std::move(columns[action.arguments[i].pos]);
else
arguments[i] = columns[action.arguments[i].pos];
}
2020-10-07 18:37:27 +00:00
ProfileEvents::increment(ProfileEvents::FunctionExecute);
2020-10-13 08:16:47 +00:00
if (action.node->is_function_compiled)
2020-10-07 18:37:27 +00:00
ProfileEvents::increment(ProfileEvents::CompiledFunctionExecute);
2020-11-03 11:28:28 +00:00
res_column.column = action.node->function->execute(arguments, res_column.type, num_rows, dry_run);
2020-10-07 18:37:27 +00:00
break;
}
2020-10-13 08:16:47 +00:00
case ActionsDAG::Type::ARRAY_JOIN:
2020-10-07 18:37:27 +00:00
{
2020-11-03 11:28:28 +00:00
size_t array_join_key_pos = action.arguments.front().pos;
2020-10-13 08:16:47 +00:00
auto array_join_key = columns[array_join_key_pos];
2020-10-07 18:37:27 +00:00
2020-10-13 08:16:47 +00:00
/// Remove array join argument in advance if it is not needed.
2020-11-09 15:01:08 +00:00
if (!action.arguments.front().needed_later)
2020-10-13 08:16:47 +00:00
columns[array_join_key_pos] = {};
2020-10-07 18:37:27 +00:00
2020-10-13 08:16:47 +00:00
array_join_key.column = array_join_key.column->convertToFullColumnIfConst();
2020-10-07 18:37:27 +00:00
2020-10-13 08:16:47 +00:00
const ColumnArray * array = typeid_cast<const ColumnArray *>(array_join_key.column.get());
if (!array)
throw Exception("ARRAY JOIN of not array: " + action.node->result_name, ErrorCodes::TYPE_MISMATCH);
2020-10-07 18:37:27 +00:00
2020-10-13 08:16:47 +00:00
for (auto & column : columns)
if (column.column)
column.column = column.column->replicate(array->getOffsets());
2020-10-07 18:37:27 +00:00
2020-11-03 11:28:28 +00:00
for (auto & column : inputs)
2020-10-13 08:16:47 +00:00
if (column.column)
column.column = column.column->replicate(array->getOffsets());
2020-10-07 18:37:27 +00:00
2020-10-13 08:16:47 +00:00
auto & res_column = columns[action.result_position];
2020-10-07 18:37:27 +00:00
2020-10-13 08:16:47 +00:00
res_column.column = array->getDataPtr();
res_column.type = assert_cast<const DataTypeArray &>(*array_join_key.type).getNestedType();
2020-11-03 11:28:28 +00:00
res_column.name = action.node->result_name;
2020-10-07 18:37:27 +00:00
2020-10-13 08:16:47 +00:00
num_rows = res_column.column->size();
2020-10-07 18:37:27 +00:00
break;
}
2020-10-13 08:16:47 +00:00
case ActionsDAG::Type::COLUMN:
2020-10-07 18:37:27 +00:00
{
2020-10-13 08:16:47 +00:00
auto & res_column = columns[action.result_position];
res_column.column = action.node->column->cloneResized(num_rows);
res_column.type = action.node->result_type;
2020-11-03 11:28:28 +00:00
res_column.name = action.node->result_name;
2020-10-07 18:37:27 +00:00
break;
}
2020-10-13 08:16:47 +00:00
case ActionsDAG::Type::ALIAS:
{
2020-11-03 11:28:28 +00:00
const auto & arg = action.arguments.front();
if (action.result_position != arg.pos)
{
columns[action.result_position].column = columns[arg.pos].column;
columns[action.result_position].type = columns[arg.pos].type;
2020-11-09 15:01:08 +00:00
if (!arg.needed_later)
2020-11-03 11:28:28 +00:00
columns[arg.pos] = {};
}
columns[action.result_position].name = action.node->result_name;
2020-10-07 18:37:27 +00:00
break;
2020-10-13 08:16:47 +00:00
}
2020-10-07 18:37:27 +00:00
2020-10-13 08:16:47 +00:00
case ActionsDAG::Type::INPUT:
{
2020-11-03 11:28:28 +00:00
auto pos = execution_context.inputs_pos[action.arguments.front().pos];
if (pos < 0)
{
2020-11-09 15:01:08 +00:00
if (action.arguments.front().needed_later)
2020-11-03 11:28:28 +00:00
throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK,
"Not found column {} in block",
action.node->result_name);
}
else
columns[action.result_position] = std::move(inputs[pos]);
break;
2020-10-13 08:16:47 +00:00
}
2020-10-07 18:37:27 +00:00
}
2020-11-03 11:28:28 +00:00
}
2020-10-13 08:16:47 +00:00
2020-11-03 11:28:28 +00:00
Names ExpressionActions::getRequiredColumns() const
{
Names names;
for (const auto & input : required_columns)
names.push_back(input.name);
return names;
2020-10-07 18:37:27 +00:00
}
2020-09-08 10:40:53 +00:00
bool ExpressionActions::hasArrayJoin() const
2020-06-18 13:00:16 +00:00
{
for (const auto & action : actions)
2020-10-13 08:16:47 +00:00
if (action.node->type == ActionsDAG::Type::ARRAY_JOIN)
2020-06-18 13:00:16 +00:00
return true;
return false;
}
std::string ExpressionActions::getSmallestColumn(const NamesAndTypesList & columns)
{
std::optional<size_t> min_size;
String res;
for (const auto & column : columns)
{
/// @todo resolve evil constant
size_t size = column.type->haveMaximumSizeOfValue() ? column.type->getMaximumSizeOfValueInMemory() : 100;
if (!min_size || size < *min_size)
{
min_size = size;
res = column.name;
}
}
if (!min_size)
throw Exception("No available columns", ErrorCodes::LOGICAL_ERROR);
return res;
}
std::string ExpressionActions::dumpActions() const
{
std::stringstream ss;
2020-11-07 00:14:53 +00:00
ss.exceptions(std::ios::failbit);
ss << "input:\n";
2020-11-03 11:28:28 +00:00
for (const auto & input_column : required_columns)
2020-03-08 23:48:08 +00:00
ss << input_column.name << " " << input_column.type->getName() << "\n";
ss << "\nactions:\n";
2020-03-08 23:48:08 +00:00
for (const auto & action : actions)
ss << action.toString() << '\n';
ss << "\noutput:\n";
NamesAndTypesList output_columns = sample_block.getNamesAndTypesList();
2020-03-08 23:48:08 +00:00
for (const auto & output_column : output_columns)
ss << output_column.name << " " << output_column.type->getName() << "\n";
2020-11-03 11:28:28 +00:00
ss << "\nproject input: " << project_input << "\noutput positions:";
for (auto pos : result_positions)
ss << " " << pos;
ss << "\n";
return ss.str();
}
2020-11-03 11:28:28 +00:00
//static std::string getUniqueNameForIndex(ActionsDAG::Index & index, std::string name)
//{
// if (index.contains(name))
// return name;
//
// size_t next_id = 0;
// std::string res;
// do
// res = name + "_" + std::to_string(next_id);
// while (index.contains(res));
//
// return res;
//}
bool ActionsDAG::hasArrayJoin() const
{
for (const auto & node : nodes)
if (node.type == Type::ARRAY_JOIN)
return true;
return false;
}
bool ActionsDAG::empty() const
2020-08-12 13:30:02 +00:00
{
2020-11-03 11:28:28 +00:00
for (const auto & node : nodes)
if (node.type != Type::INPUT)
return false;
return true;
}
ActionsDAGPtr ActionsDAG::splitActionsBeforeArrayJoin(const NameSet & array_joined_columns)
{
/// Split DAG into two parts.
/// (this_nodes, this_index) is a part which depends on ARRAY JOIN and stays here.
/// (split_nodes, split_index) is a part which will be moved before ARRAY JOIN.
std::list<Node> this_nodes;
std::list<Node> split_nodes;
Index this_index;
Index split_index;
struct Frame
2020-08-12 13:30:02 +00:00
{
2020-11-03 11:28:28 +00:00
Node * node;
size_t next_child_to_visit = 0;
};
2020-08-12 13:30:02 +00:00
2020-11-03 11:28:28 +00:00
struct Data
{
bool depend_on_array_join = false;
bool visited = false;
bool used_in_result = false;
/// Copies of node in one of the DAGs.
/// For COLUMN and INPUT both copies may exist.
Node * to_this = nullptr;
Node * to_split = nullptr;
};
2020-08-12 13:30:02 +00:00
2020-11-03 11:28:28 +00:00
std::stack<Frame> stack;
std::unordered_map<Node *, Data> data;
2020-08-12 13:30:02 +00:00
2020-11-03 11:28:28 +00:00
for (const auto & node : index)
data[node].used_in_result = true;
/// DFS. Decide if node depends on ARRAY JOIN and move it to one of the DAGs.
for (auto & node : nodes)
2020-08-12 13:30:02 +00:00
{
2020-11-03 11:28:28 +00:00
if (!data[&node].visited)
stack.push({.node = &node});
while (!stack.empty())
2020-08-12 13:30:02 +00:00
{
2020-11-03 11:28:28 +00:00
auto & cur = stack.top();
auto & cur_data = data[cur.node];
/// At first, visit all children. We depend on ARRAY JOIN if any child does.
while (cur.next_child_to_visit < cur.node->children.size())
2020-08-12 13:30:02 +00:00
{
2020-11-03 11:28:28 +00:00
auto * child = cur.node->children[cur.next_child_to_visit];
auto & child_data = data[child];
2020-08-13 18:40:21 +00:00
2020-11-03 11:28:28 +00:00
if (!child_data.visited)
2020-08-12 13:30:02 +00:00
{
2020-11-03 11:28:28 +00:00
stack.push({.node = child});
break;
2020-08-12 13:30:02 +00:00
}
2020-11-03 11:28:28 +00:00
++cur.next_child_to_visit;
if (child_data.depend_on_array_join)
cur_data.depend_on_array_join = true;
}
2020-08-12 13:30:02 +00:00
2020-11-03 11:28:28 +00:00
/// Make a copy part.
if (cur.next_child_to_visit == cur.node->children.size())
{
if (cur.node->type == Type::INPUT && array_joined_columns.count(cur.node->result_name))
cur_data.depend_on_array_join = true;
2020-08-12 13:30:02 +00:00
2020-11-03 11:28:28 +00:00
cur_data.visited = true;
stack.pop();
2020-08-12 13:30:02 +00:00
2020-11-03 11:28:28 +00:00
if (cur_data.depend_on_array_join)
{
auto & copy = this_nodes.emplace_back(*cur.node);
cur_data.to_this = &copy;
2020-08-12 13:30:02 +00:00
2020-11-03 11:28:28 +00:00
/// Replace children to newly created nodes.
for (auto & child : copy.children)
{
auto & child_data = data[child];
/// If children is not created, int may be from split part.
if (!child_data.to_this)
{
if (child->type == Type::COLUMN) /// Just create new node for COLUMN action.
{
child_data.to_this = &this_nodes.emplace_back(*child);
}
else
{
/// Node from split part is added as new input.
Node input_node;
input_node.type = Type::INPUT;
input_node.result_type = child->result_type;
input_node.result_name = child->result_name; // getUniqueNameForIndex(index, child->result_name);
child_data.to_this = &this_nodes.emplace_back(std::move(input_node));
/// This node is needed for current action, so put it to index also.
split_index[child_data.to_split->result_name] = child_data.to_split;
}
}
child = child_data.to_this;
}
}
else
{
auto & copy = split_nodes.emplace_back(*cur.node);
cur_data.to_split = &copy;
2020-08-12 13:30:02 +00:00
2020-11-03 11:28:28 +00:00
/// Replace children to newly created nodes.
for (auto & child : copy.children)
{
child = data[child].to_split;
assert(child != nullptr);
}
2020-08-13 18:40:21 +00:00
2020-11-03 11:28:28 +00:00
if (cur_data.used_in_result)
{
split_index[copy.result_name] = &copy;
/// If this node is needed in result, add it as input.
Node input_node;
input_node.type = Type::INPUT;
input_node.result_type = node.result_type;
input_node.result_name = node.result_name;
cur_data.to_this = &this_nodes.emplace_back(std::move(input_node));
}
}
}
2020-08-12 13:30:02 +00:00
}
}
2020-11-03 11:28:28 +00:00
for (auto * node : index)
this_index.insert(data[node].to_this);
2020-08-12 13:30:02 +00:00
2020-11-03 11:28:28 +00:00
/// Consider actions are empty if all nodes are constants or inputs.
bool split_actions_are_empty = true;
for (const auto & node : split_nodes)
if (!node.children.empty())
split_actions_are_empty = false;
2020-08-12 13:30:02 +00:00
2020-11-03 11:28:28 +00:00
if (split_actions_are_empty)
return {};
2020-08-12 13:30:02 +00:00
2020-11-03 11:28:28 +00:00
index.swap(this_index);
nodes.swap(this_nodes);
auto split_actions = cloneEmpty();
split_actions->nodes.swap(split_nodes);
split_actions->index.swap(split_index);
2020-08-12 13:30:02 +00:00
return split_actions;
}
2019-10-11 17:27:54 +00:00
bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) const
{
/// Check has column in (empty set).
String set_to_check;
2019-11-01 10:58:29 +00:00
for (auto it = actions.rbegin(); it != actions.rend(); ++it)
2019-10-11 17:27:54 +00:00
{
2020-04-22 06:01:33 +00:00
const auto & action = *it;
2020-11-03 11:28:28 +00:00
if (action.node->type == ActionsDAG::Type::FUNCTION && action.node->function_base)
2019-10-11 17:27:54 +00:00
{
2020-11-03 11:28:28 +00:00
if (action.node->result_name == column_name && action.node->children.size() > 1)
2019-10-11 17:55:33 +00:00
{
2020-11-03 11:28:28 +00:00
auto name = action.node->function_base->getName();
if ((name == "in" || name == "globalIn"))
{
set_to_check = action.node->children[1]->result_name;
break;
}
2019-10-11 17:55:33 +00:00
}
2019-10-11 17:27:54 +00:00
}
}
if (!set_to_check.empty())
{
2020-04-22 06:01:33 +00:00
for (const auto & action : actions)
2019-10-11 17:27:54 +00:00
{
2020-11-03 11:28:28 +00:00
if (action.node->type == ActionsDAG::Type::COLUMN && action.node->result_name == set_to_check)
2019-10-11 17:27:54 +00:00
{
2019-10-27 18:12:40 +00:00
// Constant ColumnSet cannot be empty, so we only need to check non-constant ones.
2020-11-03 11:28:28 +00:00
if (const auto * column_set = checkAndGetColumn<const ColumnSet>(action.node->column.get()))
2019-10-11 17:27:54 +00:00
{
2019-11-01 13:56:33 +00:00
if (column_set->getData()->isCreated() && column_set->getData()->getTotalRowCount() == 0)
2019-10-11 17:27:54 +00:00
return true;
}
}
}
}
return false;
}
2020-11-03 11:28:28 +00:00
void ExpressionActionsChain::addStep(NameSet non_constant_inputs)
{
if (steps.empty())
throw Exception("Cannot add action to empty ExpressionActionsChain", ErrorCodes::LOGICAL_ERROR);
2020-08-19 19:33:49 +00:00
ColumnsWithTypeAndName columns = steps.back()->getResultColumns();
2020-11-03 11:28:28 +00:00
for (auto & column : columns)
if (column.column && isColumnConst(*column.column) && non_constant_inputs.count(column.name))
column.column = nullptr;
2020-09-11 12:24:41 +00:00
steps.push_back(std::make_unique<ExpressionActionsStep>(std::make_shared<ActionsDAG>(columns)));
}
void ExpressionActionsChain::finalize()
{
2017-04-02 17:37:49 +00:00
/// Finalize all steps. Right to left to define unnecessary input columns.
for (int i = static_cast<int>(steps.size()) - 1; i >= 0; --i)
{
2020-08-19 19:33:49 +00:00
Names required_output = steps[i]->required_output;
2018-04-12 09:45:24 +00:00
std::unordered_map<String, size_t> required_output_indexes;
for (size_t j = 0; j < required_output.size(); ++j)
required_output_indexes[required_output[j]] = j;
2020-08-19 19:33:49 +00:00
auto & can_remove_required_output = steps[i]->can_remove_required_output;
2018-04-12 09:45:24 +00:00
if (i + 1 < static_cast<int>(steps.size()))
{
2020-08-19 19:33:49 +00:00
const NameSet & additional_input = steps[i + 1]->additional_input;
for (const auto & it : steps[i + 1]->getRequiredColumns())
2018-04-12 09:45:24 +00:00
{
if (additional_input.count(it.name) == 0)
{
auto iter = required_output_indexes.find(it.name);
if (iter == required_output_indexes.end())
required_output.push_back(it.name);
else if (!can_remove_required_output.empty())
2018-06-29 11:42:44 +00:00
can_remove_required_output[iter->second] = false;
2018-04-12 09:45:24 +00:00
}
}
}
2020-08-19 19:33:49 +00:00
steps[i]->finalize(required_output);
}
2017-04-02 17:37:49 +00:00
/// Adding the ejection of unnecessary columns to the beginning of each step.
for (size_t i = 1; i < steps.size(); ++i)
{
2020-08-19 19:33:49 +00:00
size_t columns_from_previous = steps[i - 1]->getResultColumns().size();
2017-04-02 17:37:49 +00:00
/// If unnecessary columns are formed at the output of the previous step, we'll add them to the beginning of this step.
/// Except when we drop all the columns and lose the number of rows in the block.
2020-08-19 19:33:49 +00:00
if (!steps[i]->getResultColumns().empty()
&& columns_from_previous > steps[i]->getRequiredColumns().size())
steps[i]->prependProjectInput();
}
}
2020-07-26 14:21:57 +00:00
std::string ExpressionActionsChain::dumpChain() const
{
std::stringstream ss;
2020-11-07 00:14:53 +00:00
ss.exceptions(std::ios::failbit);
for (size_t i = 0; i < steps.size(); ++i)
{
ss << "step " << i << "\n";
ss << "required output:\n";
2020-08-19 19:33:49 +00:00
for (const std::string & name : steps[i]->required_output)
ss << name << "\n";
2020-08-19 19:33:49 +00:00
ss << "\n" << steps[i]->dump() << "\n";
}
return ss.str();
}
2020-09-08 10:40:53 +00:00
ExpressionActionsChain::ArrayJoinStep::ArrayJoinStep(ArrayJoinActionPtr array_join_, ColumnsWithTypeAndName required_columns_)
: Step({})
2020-08-19 19:33:49 +00:00
, array_join(std::move(array_join_))
2020-08-13 20:17:18 +00:00
, result_columns(std::move(required_columns_))
2020-08-12 08:55:16 +00:00
{
2020-08-13 20:17:18 +00:00
for (auto & column : result_columns)
2020-08-12 08:55:16 +00:00
{
required_columns.emplace_back(NameAndTypePair(column.name, column.type));
if (array_join->columns.count(column.name) > 0)
{
const auto * array = typeid_cast<const DataTypeArray *>(column.type.get());
column.type = array->getNestedType();
/// Arrays are materialized
column.column = nullptr;
}
}
}
2020-08-19 19:33:49 +00:00
void ExpressionActionsChain::ArrayJoinStep::finalize(const Names & required_output_)
2020-08-12 08:55:16 +00:00
{
2020-08-13 20:17:18 +00:00
NamesAndTypesList new_required_columns;
ColumnsWithTypeAndName new_result_columns;
NameSet names(required_output_.begin(), required_output_.end());
for (const auto & column : result_columns)
2020-08-12 08:55:16 +00:00
{
2020-08-13 20:17:18 +00:00
if (array_join->columns.count(column.name) != 0 || names.count(column.name) != 0)
new_result_columns.emplace_back(column);
}
for (const auto & column : required_columns)
{
if (array_join->columns.count(column.name) != 0 || names.count(column.name) != 0)
new_required_columns.emplace_back(column);
}
2020-08-12 08:55:16 +00:00
2020-08-13 20:17:18 +00:00
std::swap(required_columns, new_required_columns);
std::swap(result_columns, new_result_columns);
}
2020-08-12 08:55:16 +00:00
2020-09-08 10:40:53 +00:00
ExpressionActionsChain::JoinStep::JoinStep(
std::shared_ptr<TableJoin> analyzed_join_,
JoinPtr join_,
ColumnsWithTypeAndName required_columns_)
: Step({})
, analyzed_join(std::move(analyzed_join_))
, join(std::move(join_))
, result_columns(std::move(required_columns_))
{
2020-09-08 12:31:36 +00:00
for (const auto & column : result_columns)
required_columns.emplace_back(column.name, column.type);
2020-09-08 10:40:53 +00:00
analyzed_join->addJoinedColumnsAndCorrectNullability(result_columns);
}
void ExpressionActionsChain::JoinStep::finalize(const Names & required_output_)
{
/// We need to update required and result columns by removing unused ones.
NamesAndTypesList new_required_columns;
ColumnsWithTypeAndName new_result_columns;
/// That's an input columns we need.
NameSet required_names(required_output_.begin(), required_output_.end());
for (const auto & name : analyzed_join->keyNamesLeft())
required_names.emplace(name);
for (const auto & column : required_columns)
{
if (required_names.count(column.name) != 0)
new_required_columns.emplace_back(column);
}
/// Result will also contain joined columns.
for (const auto & column : analyzed_join->columnsAddedByJoin())
required_names.emplace(column.name);
for (const auto & column : result_columns)
{
if (required_names.count(column.name) != 0)
new_result_columns.emplace_back(column);
}
std::swap(required_columns, new_required_columns);
std::swap(result_columns, new_result_columns);
}
2020-09-11 12:24:41 +00:00
ActionsDAGPtr & ExpressionActionsChain::Step::actions()
2020-08-12 08:55:16 +00:00
{
2020-11-03 11:28:28 +00:00
return typeid_cast<ExpressionActionsStep *>(this)->actions;
2020-08-13 20:17:18 +00:00
}
2020-09-11 12:24:41 +00:00
const ActionsDAGPtr & ExpressionActionsChain::Step::actions() const
2020-08-13 20:17:18 +00:00
{
2020-08-19 19:33:49 +00:00
return typeid_cast<const ExpressionActionsStep *>(this)->actions;
2020-08-12 08:55:16 +00:00
}
2020-09-10 16:01:41 +00:00
ActionsDAG::ActionsDAG(const NamesAndTypesList & inputs)
{
for (const auto & input : inputs)
addInput(input.name, input.type);
}
2020-09-10 18:36:51 +00:00
ActionsDAG::ActionsDAG(const ColumnsWithTypeAndName & inputs)
{
for (const auto & input : inputs)
2020-11-03 11:28:28 +00:00
{
if (input.column && isColumnConst(*input.column))
addInput(input);
else
addInput(input.name, input.type);
}
2020-09-10 18:36:51 +00:00
}
2020-09-11 12:24:41 +00:00
ActionsDAG::Node & ActionsDAG::addNode(Node node, bool can_replace)
2020-09-10 07:30:03 +00:00
{
2020-09-11 12:24:41 +00:00
auto it = index.find(node.result_name);
if (it != index.end() && !can_replace)
2020-09-10 07:30:03 +00:00
throw Exception("Column '" + node.result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN);
auto & res = nodes.emplace_back(std::move(node));
2020-09-11 12:24:41 +00:00
2020-09-13 13:51:31 +00:00
index[res.result_name] = &res;
2020-09-10 07:30:03 +00:00
return res;
}
ActionsDAG::Node & ActionsDAG::getNode(const std::string & name)
{
auto it = index.find(name);
if (it == index.end())
throw Exception("Unknown identifier: '" + name + "'", ErrorCodes::UNKNOWN_IDENTIFIER);
2020-11-03 11:28:28 +00:00
return **it;
2020-09-10 07:30:03 +00:00
}
const ActionsDAG::Node & ActionsDAG::addInput(std::string name, DataTypePtr type)
{
Node node;
node.type = Type::INPUT;
node.result_type = std::move(type);
node.result_name = std::move(name);
return addNode(std::move(node));
}
2020-09-13 13:51:31 +00:00
const ActionsDAG::Node & ActionsDAG::addInput(ColumnWithTypeAndName column)
{
Node node;
node.type = Type::INPUT;
node.result_type = std::move(column.type);
node.result_name = std::move(column.name);
node.column = std::move(column.column);
return addNode(std::move(node));
}
2020-09-10 16:01:41 +00:00
const ActionsDAG::Node & ActionsDAG::addColumn(ColumnWithTypeAndName column)
{
2020-09-13 13:51:31 +00:00
if (!column.column)
2020-11-03 11:28:28 +00:00
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot add column {} because it is nullptr", column.name);
2020-09-13 13:51:31 +00:00
2020-09-10 16:01:41 +00:00
Node node;
node.type = Type::COLUMN;
node.result_type = std::move(column.type);
node.result_name = std::move(column.name);
node.column = std::move(column.column);
return addNode(std::move(node));
}
2020-09-11 12:24:41 +00:00
const ActionsDAG::Node & ActionsDAG::addAlias(const std::string & name, std::string alias, bool can_replace)
2020-09-10 07:30:03 +00:00
{
auto & child = getNode(name);
Node node;
node.type = Type::ALIAS;
node.result_type = child.result_type;
node.result_name = std::move(alias);
node.column = child.column;
2020-09-10 16:01:41 +00:00
node.allow_constant_folding = child.allow_constant_folding;
2020-09-10 07:30:03 +00:00
node.children.emplace_back(&child);
2020-09-11 12:24:41 +00:00
return addNode(std::move(node), can_replace);
2020-09-10 07:30:03 +00:00
}
2020-11-03 11:28:28 +00:00
const ActionsDAG::Node & ActionsDAG::addArrayJoin(const std::string & source_name, std::string result_name)
2020-09-10 07:30:03 +00:00
{
auto & child = getNode(source_name);
const DataTypeArray * array_type = typeid_cast<const DataTypeArray *>(child.result_type.get());
if (!array_type)
throw Exception("ARRAY JOIN requires array argument", ErrorCodes::TYPE_MISMATCH);
Node node;
node.type = Type::ARRAY_JOIN;
node.result_type = array_type->getNestedType();
node.result_name = std::move(result_name);
node.children.emplace_back(&child);
return addNode(std::move(node));
}
2020-09-10 16:01:41 +00:00
const ActionsDAG::Node & ActionsDAG::addFunction(
const FunctionOverloadResolverPtr & function,
const Names & argument_names,
std::string result_name,
2020-11-03 11:28:28 +00:00
const Context & context [[maybe_unused]])
2020-09-10 07:30:03 +00:00
{
2020-11-03 11:28:28 +00:00
const auto & settings = context.getSettingsRef();
max_temporary_columns = settings.max_temporary_columns;
max_temporary_non_const_columns = settings.max_temporary_non_const_columns;
#if USE_EMBEDDED_COMPILER
2020-11-05 16:11:14 +00:00
compile_expressions = settings.compile_expressions;
min_count_to_compile_expression = settings.min_count_to_compile_expression;
2020-11-03 11:28:28 +00:00
if (!compilation_cache)
compilation_cache = context.getCompiledExpressionCache();
#endif
2020-09-10 16:01:41 +00:00
size_t num_arguments = argument_names.size();
2020-09-10 07:30:03 +00:00
Node node;
node.type = Type::FUNCTION;
2020-09-10 16:01:41 +00:00
node.function_builder = function;
node.children.reserve(num_arguments);
2020-09-10 07:30:03 +00:00
bool all_const = true;
2020-09-10 16:01:41 +00:00
ColumnsWithTypeAndName arguments(num_arguments);
2020-09-10 07:30:03 +00:00
2020-09-10 16:01:41 +00:00
for (size_t i = 0; i < num_arguments; ++i)
2020-09-10 07:30:03 +00:00
{
2020-09-10 16:01:41 +00:00
auto & child = getNode(argument_names[i]);
node.children.emplace_back(&child);
node.allow_constant_folding = node.allow_constant_folding && child.allow_constant_folding;
ColumnWithTypeAndName argument;
argument.column = child.column;
argument.type = child.result_type;
2020-10-23 14:28:55 +00:00
argument.name = child.result_name;
2020-09-10 16:01:41 +00:00
if (!argument.column || !isColumnConst(*argument.column))
2020-09-10 07:30:03 +00:00
all_const = false;
2020-09-10 16:01:41 +00:00
arguments[i] = std::move(argument);
2020-09-10 07:30:03 +00:00
}
2020-09-10 16:01:41 +00:00
node.function_base = function->build(arguments);
2020-10-15 16:52:25 +00:00
node.result_type = node.function_base->getResultType();
node.function = node.function_base->prepare(arguments);
2020-09-10 07:30:03 +00:00
/// If all arguments are constants, and function is suitable to be executed in 'prepare' stage - execute function.
/// But if we compile expressions compiled version of this function maybe placed in cache,
/// so we don't want to unfold non deterministic functions
2020-11-05 16:11:14 +00:00
if (all_const && node.function_base->isSuitableForConstantFolding() && (!compile_expressions || node.function_base->isDeterministic()))
2020-09-10 07:30:03 +00:00
{
2020-10-15 16:52:25 +00:00
size_t num_rows = arguments.empty() ? 0 : arguments.front().column->size();
auto col = node.function->execute(arguments, node.result_type, num_rows, true);
2020-09-10 07:30:03 +00:00
/// If the result is not a constant, just in case, we will consider the result as unknown.
2020-10-15 16:52:25 +00:00
if (isColumnConst(*col))
2020-09-10 07:30:03 +00:00
{
/// All constant (literal) columns in block are added with size 1.
/// But if there was no columns in block before executing a function, the result has size 0.
/// Change the size to 1.
2020-10-15 16:52:25 +00:00
if (col->empty())
col = col->cloneResized(1);
2020-09-10 07:30:03 +00:00
2020-10-15 16:52:25 +00:00
node.column = std::move(col);
2020-09-10 07:30:03 +00:00
}
}
/// Some functions like ignore() or getTypeName() always return constant result even if arguments are not constant.
/// We can't do constant folding, but can specify in sample block that function result is constant to avoid
/// unnecessary materialization.
2020-09-10 16:01:41 +00:00
if (!node.column && node.function_base->isSuitableForConstantFolding())
2020-09-10 07:30:03 +00:00
{
2020-10-15 16:52:25 +00:00
if (auto col = node.function_base->getResultIfAlwaysReturnsConstantAndHasArguments(arguments))
2020-09-10 07:30:03 +00:00
{
2020-09-10 16:01:41 +00:00
node.column = std::move(col);
node.allow_constant_folding = false;
2020-09-10 07:30:03 +00:00
}
}
2020-09-10 16:01:41 +00:00
if (result_name.empty())
{
result_name = function->getName() + "(";
for (size_t i = 0; i < argument_names.size(); ++i)
{
if (i)
result_name += ", ";
result_name += argument_names[i];
}
result_name += ")";
}
node.result_name = std::move(result_name);
return addNode(std::move(node));
}
2020-11-03 11:28:28 +00:00
NamesAndTypesList ActionsDAG::getRequiredColumns() const
{
NamesAndTypesList result;
for (const auto & node : nodes)
if (node.type == Type::INPUT)
result.push_back({node.result_name, node.result_type});
return result;
}
2020-09-10 18:36:51 +00:00
ColumnsWithTypeAndName ActionsDAG::getResultColumns() const
{
ColumnsWithTypeAndName result;
2020-09-11 12:24:41 +00:00
result.reserve(index.size());
2020-11-03 11:28:28 +00:00
for (const auto & node : index)
result.emplace_back(node->column, node->result_type, node->result_name);
2020-09-10 18:36:51 +00:00
return result;
}
2020-09-10 16:01:41 +00:00
NamesAndTypesList ActionsDAG::getNamesAndTypesList() const
{
NamesAndTypesList result;
2020-11-03 11:28:28 +00:00
for (const auto & node : index)
result.emplace_back(node->result_name, node->result_type);
2020-09-10 16:01:41 +00:00
return result;
}
2020-09-11 12:24:41 +00:00
Names ActionsDAG::getNames() const
{
Names names;
names.reserve(index.size());
2020-11-03 11:28:28 +00:00
for (const auto & node : index)
names.emplace_back(node->result_name);
2020-09-11 12:24:41 +00:00
return names;
}
2020-09-10 16:01:41 +00:00
std::string ActionsDAG::dumpNames() const
{
WriteBufferFromOwnString out;
for (auto it = nodes.begin(); it != nodes.end(); ++it)
{
if (it != nodes.begin())
out << ", ";
out << it->result_name;
}
return out.str();
}
2020-11-03 11:28:28 +00:00
void ActionsDAG::removeUnusedActions(const Names & required_names)
{
std::unordered_set<Node *> nodes_set;
std::vector<Node *> required_nodes;
required_nodes.reserve(required_names.size());
for (const auto & name : required_names)
{
auto it = index.find(name);
if (it == index.end())
throw Exception(ErrorCodes::UNKNOWN_IDENTIFIER,
"Unknown column: {}, there are only columns {}", name, dumpNames());
if (nodes_set.insert(*it).second)
required_nodes.push_back(*it);
}
removeUnusedActions(required_nodes);
}
void ActionsDAG::removeUnusedActions(const std::vector<Node *> & required_nodes)
{
{
Index new_index;
for (auto * node : required_nodes)
new_index.insert(node);
index.swap(new_index);
}
2020-11-05 16:11:14 +00:00
removeUnusedActions();
}
void ActionsDAG::removeUnusedActions()
{
std::unordered_set<const Node *> visited_nodes;
std::stack<Node *> stack;
for (auto * node : index)
{
visited_nodes.insert(node);
stack.push(node);
}
2020-11-03 11:28:28 +00:00
while (!stack.empty())
{
auto * node = stack.top();
stack.pop();
if (!node->children.empty() && node->column && isColumnConst(*node->column) && node->allow_constant_folding)
{
/// Constant folding.
node->type = ActionsDAG::Type::COLUMN;
node->children.clear();
}
for (auto * child : node->children)
{
if (visited_nodes.count(child) == 0)
{
stack.push(child);
visited_nodes.insert(child);
}
}
}
nodes.remove_if([&](const Node & node) { return visited_nodes.count(&node) == 0; });
}
void ActionsDAG::addAliases(const NamesWithAliases & aliases, std::vector<Node *> & result_nodes)
{
std::vector<Node *> required_nodes;
for (const auto & item : aliases)
{
auto & child = getNode(item.first);
required_nodes.push_back(&child);
}
result_nodes.reserve(aliases.size());
for (size_t i = 0; i < aliases.size(); ++i)
{
const auto & item = aliases[i];
auto * child = required_nodes[i];
if (!item.second.empty() && item.first != item.second)
{
Node node;
node.type = Type::ALIAS;
node.result_type = child->result_type;
node.result_name = std::move(item.second);
node.column = child->column;
node.allow_constant_folding = child->allow_constant_folding;
node.children.emplace_back(child);
auto & alias = addNode(std::move(node), true);
result_nodes.push_back(&alias);
}
else
result_nodes.push_back(child);
}
}
void ActionsDAG::addAliases(const NamesWithAliases & aliases)
{
std::vector<Node *> result_nodes;
addAliases(aliases, result_nodes);
}
void ActionsDAG::project(const NamesWithAliases & projection)
{
std::vector<Node *> result_nodes;
addAliases(projection, result_nodes);
removeUnusedActions(result_nodes);
projectInput();
projected_output = true;
}
void ActionsDAG::removeColumn(const std::string & column_name)
{
auto & node = getNode(column_name);
index.remove(&node);
}
bool ActionsDAG::tryRestoreColumn(const std::string & column_name)
{
if (index.contains(column_name))
return true;
for (auto it = nodes.rbegin(); it != nodes.rend(); ++it)
{
auto & node = *it;
if (node.result_name == column_name)
{
index[node.result_name] = &node;
return true;
}
}
return false;
}
ActionsDAGPtr ActionsDAG::clone() const
{
auto actions = cloneEmpty();
std::unordered_map<const Node *, Node *> copy_map;
for (const auto & node : nodes)
{
auto & copy_node = actions->nodes.emplace_back(node);
copy_map[&node] = &copy_node;
}
for (auto & node : actions->nodes)
for (auto & child : node.children)
child = copy_map[child];
for (const auto & node : index)
actions->index.insert(copy_map[node]);
return actions;
}
ExpressionActionsPtr ExpressionActions::clone() const
{
auto expressions = std::make_shared<ExpressionActions>();
expressions->actions = actions;
expressions->num_columns = num_columns;
expressions->required_columns = required_columns;
expressions->result_positions = result_positions;
expressions->sample_block = sample_block;
expressions->project_input = project_input;
expressions->max_temporary_non_const_columns = max_temporary_non_const_columns;
std::unordered_map<const Node *, Node *> copy_map;
for (const auto & node : nodes)
{
auto & copy_node = expressions->nodes.emplace_back(node);
copy_map[&node] = &copy_node;
}
for (auto & node : expressions->nodes)
for (auto & child : node.children)
child = copy_map[child];
for (auto & action : expressions->actions)
action.node = copy_map[action.node];
return expressions;
}
ExpressionActionsPtr ActionsDAG::linearizeActions() const
2020-09-10 16:01:41 +00:00
{
struct Data
2020-09-10 07:30:03 +00:00
{
2020-11-03 11:28:28 +00:00
const Node * node = nullptr;
2020-09-10 16:01:41 +00:00
size_t num_created_children = 0;
2020-11-03 11:28:28 +00:00
std::vector<const Node *> parents;
ssize_t position = -1;
size_t num_created_parents = 0;
bool used_in_result = false;
2020-09-10 16:01:41 +00:00
};
std::vector<Data> data(nodes.size());
2020-11-03 11:28:28 +00:00
std::unordered_map<const Node *, size_t> reverse_index;
2020-09-10 16:01:41 +00:00
2020-11-03 11:28:28 +00:00
for (const auto & node : nodes)
2020-09-10 16:01:41 +00:00
{
size_t id = reverse_index.size();
data[id].node = &node;
reverse_index[&node] = id;
2020-09-10 07:30:03 +00:00
}
2020-09-10 16:01:41 +00:00
2020-11-03 11:28:28 +00:00
std::queue<const Node *> ready_nodes;
std::queue<const Node *> ready_array_joins;
2020-09-10 16:01:41 +00:00
2020-11-03 11:28:28 +00:00
for (const auto * node : index)
data[reverse_index[node]].used_in_result = true;
2020-09-11 12:24:41 +00:00
2020-11-03 11:28:28 +00:00
for (const auto & node : nodes)
{
2020-09-10 16:01:41 +00:00
for (const auto & child : node.children)
data[reverse_index[child]].parents.emplace_back(&node);
2020-09-11 12:24:41 +00:00
}
2020-11-03 11:28:28 +00:00
for (const auto & node : nodes)
2020-09-11 12:24:41 +00:00
{
2020-11-03 11:28:28 +00:00
if (node.children.empty())
2020-09-10 16:01:41 +00:00
ready_nodes.emplace(&node);
}
2020-11-03 11:28:28 +00:00
auto expressions = std::make_shared<ExpressionActions>();
std::stack<size_t> free_positions;
2020-09-10 16:01:41 +00:00
while (!ready_nodes.empty() || !ready_array_joins.empty())
{
auto & stack = ready_nodes.empty() ? ready_array_joins : ready_nodes;
2020-11-03 11:28:28 +00:00
const Node * node = stack.front();
2020-09-10 16:01:41 +00:00
stack.pop();
Names argument_names;
for (const auto & child : node->children)
argument_names.emplace_back(child->result_name);
2020-09-11 12:24:41 +00:00
auto & cur = data[reverse_index[node]];
2020-11-03 11:28:28 +00:00
size_t free_position = expressions->num_columns;
if (free_positions.empty())
++expressions->num_columns;
else
{
free_position = free_positions.top();
free_positions.pop();
}
cur.position = free_position;
ExpressionActions::Arguments arguments;
arguments.reserve(cur.node->children.size());
for (auto * child : cur.node->children)
{
auto & arg = data[reverse_index[child]];
if (arg.position < 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Argument was not calculated for {}", child->result_name);
++arg.num_created_parents;
ExpressionActions::Argument argument;
argument.pos = arg.position;
2020-11-09 15:01:08 +00:00
argument.needed_later = arg.used_in_result || arg.num_created_parents != arg.parents.size();
2020-11-03 11:28:28 +00:00
2020-11-09 15:01:08 +00:00
if (!argument.needed_later)
2020-11-03 11:28:28 +00:00
free_positions.push(argument.pos);
arguments.emplace_back(argument);
}
if (node->type == Type::INPUT)
{
/// Argument for input is special. It contains the position from required columns.
ExpressionActions::Argument argument;
argument.pos = expressions->required_columns.size();
2020-11-09 15:01:08 +00:00
argument.needed_later = !cur.parents.empty();
2020-11-03 11:28:28 +00:00
arguments.emplace_back(argument);
expressions->required_columns.push_back({node->result_name, node->result_type});
}
expressions->actions.push_back({node, arguments, free_position});
for (const auto & parent : cur.parents)
{
auto & parent_data = data[reverse_index[parent]];
++parent_data.num_created_children;
if (parent_data.num_created_children == parent->children.size())
{
auto & push_stack = parent->type == Type::ARRAY_JOIN ? ready_array_joins : ready_nodes;
push_stack.push(parent);
}
}
}
expressions->result_positions.reserve(index.size());
for (const auto & node : index)
{
auto pos = data[reverse_index[node]].position;
if (pos < 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Action for {} was not calculated", node->result_name);
expressions->result_positions.push_back(pos);
ColumnWithTypeAndName col{node->column, node->result_type, node->result_name};
expressions->sample_block.insert(std::move(col));
}
return expressions;
}
2020-11-03 13:30:55 +00:00
ExpressionActionsPtr ActionsDAG::buildExpressions() const
2020-11-03 11:28:28 +00:00
{
auto cloned = clone();
2020-11-05 16:11:14 +00:00
#if USE_EMBEDDED_COMPILER
if (compile_expressions)
{
cloned->compileFunctions();
cloned->removeUnusedActions();
}
#endif
auto expressions = cloned->linearizeActions();
2020-11-03 11:28:28 +00:00
expressions->nodes.swap(cloned->nodes);
if (max_temporary_columns && expressions->num_columns > max_temporary_columns)
throw Exception(ErrorCodes::TOO_MANY_TEMPORARY_COLUMNS,
"Too many temporary columns: {}. Maximum: {}",
dumpNames(), std::to_string(max_temporary_columns));
expressions->max_temporary_non_const_columns = max_temporary_non_const_columns;
expressions->project_input = project_input;
return expressions;
}
std::string ActionsDAG::dump() const
{
return linearizeActions()->dumpActions();
}
std::string ActionsDAG::dumpDAG() const
{
std::unordered_map<const Node *, size_t> map;
for (const auto & node : nodes)
{
size_t idx = map.size();
map[&node] = idx;
}
std::stringstream out;
for (const auto & node : nodes)
{
out << map[&node] << " : ";
switch (node.type)
2020-09-10 16:01:41 +00:00
{
2020-11-03 11:28:28 +00:00
case ActionsDAG::Type::COLUMN:
out << "COLUMN ";
2020-09-10 16:01:41 +00:00
break;
2020-11-03 11:28:28 +00:00
case ActionsDAG::Type::ALIAS:
out << "ALIAS ";
2020-09-10 16:01:41 +00:00
break;
2020-11-03 11:28:28 +00:00
case ActionsDAG::Type::FUNCTION:
out << "FUNCTION ";
2020-09-10 16:01:41 +00:00
break;
2020-11-03 11:28:28 +00:00
case ActionsDAG::Type::ARRAY_JOIN:
out << "ARRAY JOIN ";
2020-09-10 16:01:41 +00:00
break;
2020-11-03 11:28:28 +00:00
case ActionsDAG::Type::INPUT:
out << "INPUT ";
2020-09-10 16:01:41 +00:00
break;
}
2020-11-03 11:28:28 +00:00
out << "(";
for (size_t i = 0; i < node.children.size(); ++i)
{
if (i)
out << ", ";
out << map[node.children[i]];
}
out << ")";
out << " " << (node.column ? node.column->getName() : "(no column)");
out << " " << (node.result_type ? node.result_type->getName() : "(no type)");
out << " " << (!node.result_name.empty() ? node.result_name : "(no name)");
if (node.function_base)
out << " [" << node.function_base->getName() << "]";
2020-09-10 16:01:41 +00:00
2020-11-03 11:28:28 +00:00
out << "\n";
2020-09-10 16:01:41 +00:00
}
2020-11-03 11:28:28 +00:00
return out.str();
2020-09-10 07:30:03 +00:00
}
}