2018-05-07 02:09:29 +00:00
|
|
|
#include <memory>
|
|
|
|
#include <string>
|
|
|
|
#include <vector>
|
|
|
|
#include <Columns/ColumnsNumber.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
2018-04-24 07:16:39 +00:00
|
|
|
#include <Functions/FunctionFactory.h>
|
2018-05-07 02:09:29 +00:00
|
|
|
#include <Interpreters/ExpressionActions.h>
|
2018-04-24 07:16:39 +00:00
|
|
|
#include <Interpreters/ExpressionAnalyzer.h>
|
|
|
|
#include <Interpreters/ProjectionManipulation.h>
|
2018-05-07 02:09:29 +00:00
|
|
|
#include <Common/Exception.h>
|
2018-04-24 07:16:39 +00:00
|
|
|
#include <Common/typeid_cast.h>
|
|
|
|
|
2018-05-07 02:09:29 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
ProjectionManipulatorBase::~ProjectionManipulatorBase() {}
|
2018-04-24 07:16:39 +00:00
|
|
|
|
2018-05-07 02:09:29 +00:00
|
|
|
DefaultProjectionManipulator::DefaultProjectionManipulator(ScopeStack & scopes) : scopes(scopes) {}
|
2018-04-24 07:16:39 +00:00
|
|
|
|
2018-05-01 13:24:43 +00:00
|
|
|
bool DefaultProjectionManipulator::tryToGetFromUpperProjection(const std::string & column_name)
|
2018-04-24 07:16:39 +00:00
|
|
|
{
|
|
|
|
return scopes.getSampleBlock().has(column_name);
|
|
|
|
}
|
|
|
|
|
|
|
|
std::string DefaultProjectionManipulator::getColumnName(const std::string & column_name) const
|
|
|
|
{
|
|
|
|
return column_name;
|
|
|
|
}
|
|
|
|
|
2018-05-01 13:24:43 +00:00
|
|
|
std::string DefaultProjectionManipulator::getProjectionExpression()
|
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
return "";
|
|
|
|
}
|
|
|
|
|
2018-05-07 02:09:29 +00:00
|
|
|
std::string DefaultProjectionManipulator::getProjectionSourceColumn() const
|
|
|
|
{
|
2018-04-30 19:33:32 +00:00
|
|
|
return "";
|
|
|
|
}
|
2018-04-24 07:16:39 +00:00
|
|
|
|
2018-05-07 02:09:29 +00:00
|
|
|
ConditionalTree::Node::Node() : projection_expression_string(), parent_node(0), is_root(false) {}
|
2018-04-24 07:16:39 +00:00
|
|
|
|
|
|
|
size_t ConditionalTree::Node::getParentNode() const
|
|
|
|
{
|
|
|
|
if (is_root)
|
|
|
|
{
|
2018-05-07 02:09:29 +00:00
|
|
|
throw Exception(
|
|
|
|
"Failed to get parent projection node of node " + projection_expression_string, ErrorCodes::CONDITIONAL_TREE_PARENT_NOT_FOUND);
|
2018-04-24 07:16:39 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
return parent_node;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
std::string ConditionalTree::getColumnNameByIndex(const std::string & col_name, const size_t node) const
|
|
|
|
{
|
|
|
|
std::string projection_name = nodes[node].projection_expression_string;
|
|
|
|
if (projection_name.empty())
|
|
|
|
{
|
|
|
|
return col_name;
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
return col_name + '<' + projection_name + '>';
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
std::string ConditionalTree::getColumnName(const std::string & col_name) const
|
|
|
|
{
|
|
|
|
return getColumnNameByIndex(col_name, current_node);
|
|
|
|
}
|
|
|
|
|
2018-05-07 02:09:29 +00:00
|
|
|
std::string ConditionalTree::getProjectionColumnName(
|
|
|
|
const std::string & first_projection_expr, const std::string & second_projection_expr) const
|
2018-04-24 07:16:39 +00:00
|
|
|
{
|
|
|
|
return std::string("P<") + first_projection_expr + "><" + second_projection_expr + ">";
|
|
|
|
}
|
|
|
|
|
2018-04-30 19:33:32 +00:00
|
|
|
std::string ConditionalTree::getProjectionColumnName(const size_t first_index, const size_t second_index) const
|
2018-04-24 07:16:39 +00:00
|
|
|
{
|
2018-05-07 02:09:29 +00:00
|
|
|
return getProjectionColumnName(nodes[first_index].projection_expression_string, nodes[second_index].projection_expression_string);
|
2018-04-24 07:16:39 +00:00
|
|
|
}
|
|
|
|
|
2018-05-07 02:09:29 +00:00
|
|
|
void ConditionalTree::buildProjectionCompositionRecursive(
|
|
|
|
const std::vector<size_t> & path, const size_t child_index, const size_t parent_index)
|
2018-04-24 07:16:39 +00:00
|
|
|
{
|
|
|
|
std::string projection_name = getProjectionColumnName(path[parent_index], path[child_index]);
|
|
|
|
if (parent_index - child_index >= 2 && !scopes.getSampleBlock().has(projection_name))
|
|
|
|
{
|
|
|
|
size_t middle_index = (child_index + parent_index) / 2;
|
|
|
|
buildProjectionCompositionRecursive(path, child_index, middle_index);
|
|
|
|
buildProjectionCompositionRecursive(path, middle_index, parent_index);
|
2018-05-07 02:09:29 +00:00
|
|
|
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("__inner_build_projection_composition__", context);
|
|
|
|
scopes.addAction(ExpressionAction::applyFunction(function_builder,
|
|
|
|
{getProjectionColumnName(path[parent_index], path[middle_index]),
|
|
|
|
getProjectionColumnName(path[middle_index], path[child_index])},
|
|
|
|
projection_name,
|
|
|
|
getProjectionSourceColumn()));
|
2018-04-24 07:16:39 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void ConditionalTree::buildProjectionComposition(const size_t child_node, const size_t parent_node)
|
|
|
|
{
|
|
|
|
std::vector<size_t> path;
|
|
|
|
size_t node = child_node;
|
2018-05-01 13:24:43 +00:00
|
|
|
while (true)
|
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
path.push_back(node);
|
2018-05-01 13:24:43 +00:00
|
|
|
if (node == parent_node)
|
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
node = nodes[node].getParentNode();
|
|
|
|
}
|
|
|
|
buildProjectionCompositionRecursive(path, 0, path.size() - 1);
|
|
|
|
}
|
|
|
|
|
2018-05-07 02:09:29 +00:00
|
|
|
std::string ConditionalTree::getProjectionSourceColumn(size_t node) const
|
|
|
|
{
|
2018-04-30 19:33:32 +00:00
|
|
|
if (nodes[node].is_root)
|
|
|
|
{
|
|
|
|
return "";
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
return ConditionalTree::getProjectionColumnName(nodes[node].getParentNode(), node);
|
|
|
|
}
|
|
|
|
}
|
2018-04-24 07:16:39 +00:00
|
|
|
|
|
|
|
ConditionalTree::ConditionalTree(ScopeStack & scopes, const Context & context)
|
2018-05-07 02:09:29 +00:00
|
|
|
: current_node(0), nodes(1), scopes(scopes), context(context), projection_expression_index()
|
2018-04-24 07:16:39 +00:00
|
|
|
{
|
|
|
|
nodes[0].is_root = true;
|
|
|
|
}
|
|
|
|
|
|
|
|
void ConditionalTree::goToProjection(const std::string & field_name)
|
|
|
|
{
|
|
|
|
std::string current_projection_name = nodes[current_node].projection_expression_string;
|
|
|
|
std::string new_projection_name = current_projection_name.empty() ? field_name : current_projection_name + ";" + field_name;
|
|
|
|
std::string projection_column_name = getProjectionColumnName(current_projection_name, new_projection_name);
|
2018-05-01 13:24:43 +00:00
|
|
|
if (!scopes.getSampleBlock().has(projection_column_name))
|
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("one_or_zero", context);
|
2018-05-07 02:09:29 +00:00
|
|
|
scopes.addAction(ExpressionAction::applyFunction(
|
|
|
|
function_builder, {getColumnName(field_name)}, projection_column_name, getProjectionSourceColumn()));
|
2018-04-24 07:16:39 +00:00
|
|
|
nodes.emplace_back(Node());
|
|
|
|
nodes.back().projection_expression_string = new_projection_name;
|
|
|
|
nodes.back().parent_node = current_node;
|
|
|
|
current_node = nodes.size() - 1;
|
|
|
|
projection_expression_index[projection_column_name] = current_node;
|
2018-05-07 02:09:29 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
current_node = projection_expression_index[projection_column_name];
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-05-01 13:24:43 +00:00
|
|
|
std::string ConditionalTree::buildRestoreProjectionAndGetName(const size_t levels_up)
|
|
|
|
{
|
2018-04-30 19:33:32 +00:00
|
|
|
size_t target_node = current_node;
|
2018-05-01 13:24:43 +00:00
|
|
|
for (size_t i = 0; i < levels_up; ++i)
|
|
|
|
{
|
2018-04-30 19:33:32 +00:00
|
|
|
target_node = nodes[target_node].getParentNode();
|
|
|
|
}
|
|
|
|
buildProjectionComposition(current_node, target_node);
|
|
|
|
return getProjectionColumnName(target_node, current_node);
|
|
|
|
}
|
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
void ConditionalTree::restoreColumn(
|
2018-05-07 02:09:29 +00:00
|
|
|
const std::string & default_values_name, const std::string & new_values_name, const size_t levels_up, const std::string & result_name)
|
2018-04-24 07:16:39 +00:00
|
|
|
{
|
|
|
|
size_t target_node = current_node;
|
2018-05-01 13:24:43 +00:00
|
|
|
for (size_t i = 0; i < levels_up; ++i)
|
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
target_node = nodes[target_node].getParentNode();
|
|
|
|
}
|
2018-05-07 02:09:29 +00:00
|
|
|
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("__inner_restore_projection__", context);
|
|
|
|
scopes.addAction(ExpressionAction::applyFunction(function_builder,
|
|
|
|
{getProjectionColumnName(target_node, current_node),
|
2018-04-29 21:48:33 +00:00
|
|
|
getColumnNameByIndex(default_values_name, current_node),
|
2018-05-07 02:09:29 +00:00
|
|
|
getColumnNameByIndex(new_values_name, current_node)},
|
|
|
|
getColumnNameByIndex(result_name, target_node),
|
|
|
|
getProjectionSourceColumn()));
|
2018-04-24 07:16:39 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
void ConditionalTree::goUp(const size_t levels_up)
|
|
|
|
{
|
2018-05-01 13:24:43 +00:00
|
|
|
for (size_t i = 0; i < levels_up; ++i)
|
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
current_node = nodes[current_node].getParentNode();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-05-01 13:24:43 +00:00
|
|
|
bool ConditionalTree::tryToGetFromUpperProjection(const std::string & column_name)
|
2018-04-24 07:16:39 +00:00
|
|
|
{
|
|
|
|
size_t node = current_node;
|
2018-05-01 13:24:43 +00:00
|
|
|
while (true)
|
|
|
|
{
|
|
|
|
if (scopes.getSampleBlock().has(getColumnNameByIndex(column_name, node)))
|
|
|
|
{
|
|
|
|
if (node != current_node)
|
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
buildProjectionComposition(current_node, node);
|
2018-05-07 02:09:29 +00:00
|
|
|
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("__inner_project__", context);
|
|
|
|
scopes.addAction(ExpressionAction::applyFunction(function_builder,
|
|
|
|
{getColumnNameByIndex(column_name, node), getProjectionColumnName(node, current_node)},
|
|
|
|
getColumnName(column_name),
|
|
|
|
getProjectionSourceColumn(node)));
|
2018-04-24 07:16:39 +00:00
|
|
|
}
|
|
|
|
return true;
|
|
|
|
}
|
2018-05-01 13:24:43 +00:00
|
|
|
if (nodes[node].is_root)
|
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
node = nodes[node].getParentNode();
|
|
|
|
}
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2018-05-01 13:24:43 +00:00
|
|
|
std::string ConditionalTree::getProjectionExpression()
|
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
return nodes[current_node].projection_expression_string;
|
|
|
|
}
|
|
|
|
|
2018-05-07 02:09:29 +00:00
|
|
|
std::string ConditionalTree::getProjectionSourceColumn() const
|
|
|
|
{
|
2018-04-30 19:33:32 +00:00
|
|
|
return getProjectionSourceColumn(current_node);
|
|
|
|
}
|
|
|
|
|
2018-05-07 02:09:29 +00:00
|
|
|
void DefaultProjectionAction::preArgumentAction() {}
|
2018-04-24 07:16:39 +00:00
|
|
|
|
2018-05-07 02:09:29 +00:00
|
|
|
void DefaultProjectionAction::postArgumentAction(const std::string & /*argument_name*/) {}
|
2018-04-24 07:16:39 +00:00
|
|
|
|
2018-05-07 02:09:29 +00:00
|
|
|
void DefaultProjectionAction::preCalculation() {}
|
2018-04-24 07:16:39 +00:00
|
|
|
|
|
|
|
bool DefaultProjectionAction::isCalculationRequired()
|
|
|
|
{
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2018-05-07 02:09:29 +00:00
|
|
|
AndOperatorProjectionAction::AndOperatorProjectionAction(
|
|
|
|
ScopeStack & scopes, ProjectionManipulatorPtr projection_manipulator, const std::string & expression_name, const Context & context)
|
|
|
|
: scopes(scopes)
|
|
|
|
, projection_manipulator(projection_manipulator)
|
|
|
|
, previous_argument_name()
|
|
|
|
, projection_levels_count(0)
|
|
|
|
, expression_name(expression_name)
|
|
|
|
, context(context)
|
|
|
|
{
|
|
|
|
}
|
2018-04-24 07:16:39 +00:00
|
|
|
|
|
|
|
std::string AndOperatorProjectionAction::getZerosColumnName()
|
|
|
|
{
|
|
|
|
return "__inner_zeroes_column__" + expression_name;
|
|
|
|
}
|
|
|
|
|
|
|
|
std::string AndOperatorProjectionAction::getFinalColumnName()
|
|
|
|
{
|
|
|
|
return "__inner_final_column__" + expression_name;
|
|
|
|
}
|
|
|
|
|
2018-04-30 19:33:32 +00:00
|
|
|
void AndOperatorProjectionAction::createZerosColumn(const std::string & restore_projection_name)
|
2018-04-24 07:16:39 +00:00
|
|
|
{
|
|
|
|
auto zeros_column_name = projection_manipulator->getColumnName(getZerosColumnName());
|
|
|
|
if (!scopes.getSampleBlock().has(zeros_column_name))
|
|
|
|
{
|
2018-05-07 02:09:29 +00:00
|
|
|
scopes.addAction(ExpressionAction::addColumn(
|
|
|
|
ColumnWithTypeAndName(ColumnUInt8::create(0, 1), std::make_shared<DataTypeUInt8>(), zeros_column_name),
|
|
|
|
restore_projection_name,
|
|
|
|
true));
|
2018-04-24 07:16:39 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void AndOperatorProjectionAction::preArgumentAction()
|
|
|
|
{
|
2018-04-29 13:49:58 +00:00
|
|
|
if (!previous_argument_name.empty())
|
2018-04-24 07:16:39 +00:00
|
|
|
{
|
|
|
|
// Before processing arguments starting from second to last
|
2018-05-01 13:24:43 +00:00
|
|
|
if (auto * conditional_tree = typeid_cast<ConditionalTree *>(projection_manipulator.get()))
|
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
conditional_tree->goToProjection(previous_argument_name);
|
2018-05-01 13:24:43 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2018-05-07 02:09:29 +00:00
|
|
|
throw Exception(
|
|
|
|
"Illegal projection manipulator used in AndOperatorProjectionAction", ErrorCodes::ILLEGAL_PROJECTION_MANIPULATOR);
|
2018-04-24 07:16:39 +00:00
|
|
|
}
|
|
|
|
++projection_levels_count;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void AndOperatorProjectionAction::postArgumentAction(const std::string & argument_name)
|
|
|
|
{
|
|
|
|
previous_argument_name = argument_name;
|
|
|
|
}
|
|
|
|
|
|
|
|
void AndOperatorProjectionAction::preCalculation()
|
|
|
|
{
|
2018-05-01 13:24:43 +00:00
|
|
|
if (auto * conditional_tree = typeid_cast<ConditionalTree *>(projection_manipulator.get()))
|
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
auto final_column = getFinalColumnName();
|
|
|
|
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("one_or_zero", context);
|
2018-05-07 02:09:29 +00:00
|
|
|
scopes.addAction(ExpressionAction::applyFunction(function_builder,
|
|
|
|
{projection_manipulator->getColumnName(previous_argument_name)},
|
|
|
|
projection_manipulator->getColumnName(final_column),
|
|
|
|
projection_manipulator->getProjectionSourceColumn()));
|
2018-04-30 19:33:32 +00:00
|
|
|
std::string restore_projection_name = conditional_tree->buildRestoreProjectionAndGetName(projection_levels_count);
|
|
|
|
createZerosColumn(restore_projection_name);
|
2018-05-07 02:09:29 +00:00
|
|
|
conditional_tree->restoreColumn(getZerosColumnName(), final_column, projection_levels_count, expression_name);
|
2018-04-24 07:16:39 +00:00
|
|
|
conditional_tree->goUp(projection_levels_count);
|
2018-05-01 13:24:43 +00:00
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
2018-04-24 07:16:39 +00:00
|
|
|
throw Exception("Illegal projection manipulator used in AndOperatorProjectionAction", ErrorCodes::ILLEGAL_PROJECTION_MANIPULATOR);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
bool AndOperatorProjectionAction::isCalculationRequired()
|
|
|
|
{
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
2018-05-07 02:09:29 +00:00
|
|
|
ProjectionActionBase::~ProjectionActionBase() {}
|
2018-04-24 07:16:39 +00:00
|
|
|
|
|
|
|
ProjectionActionPtr getProjectionAction(const std::string & node_name,
|
2018-05-07 02:09:29 +00:00
|
|
|
ScopeStack & scopes,
|
|
|
|
ProjectionManipulatorPtr projection_manipulator,
|
|
|
|
const std::string & expression_name,
|
|
|
|
const Context & context)
|
2018-04-24 07:16:39 +00:00
|
|
|
{
|
|
|
|
if (typeid_cast<ConditionalTree *>(projection_manipulator.get()) && node_name == "and")
|
|
|
|
{
|
|
|
|
return std::make_shared<AndOperatorProjectionAction>(scopes, projection_manipulator, expression_name, context);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
{
|
|
|
|
return std::make_shared<DefaultProjectionAction>();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
}
|