Getting rid of measure action, other small fixes applied.

This commit is contained in:
Tsarkova Anastasia 2018-04-30 21:33:32 +02:00
parent a99ff6f380
commit 210a389592
10 changed files with 135 additions and 83 deletions

View File

@ -91,9 +91,9 @@ public:
return data->getInt(0);
}
UInt8 getUInt8(size_t) const override
UInt8 getBoolRepresentation(size_t) const override
{
return data->getUInt8(0);
return data->getBoolRepresentation(0);
}
bool isNullAt(size_t) const override

View File

@ -46,7 +46,7 @@ public:
bool isNullAt(size_t n) const override { return static_cast<const ColumnUInt8 &>(*null_map).getData()[n] != 0;}
Field operator[](size_t n) const override;
void get(size_t n, Field & res) const override;
UInt8 getUInt8(size_t n) const override { return isNullAt(n) ? 0 : nested_column->getUInt8(n); }
UInt8 getBoolRepresentation(size_t n) const override { return isNullAt(n) ? 0 : nested_column->getBoolRepresentation(n); }
UInt64 get64(size_t n) const override { return nested_column->get64(n); }
StringRef getDataAt(size_t n) const override;
void insertData(const char * pos, size_t length) override;

View File

@ -231,7 +231,7 @@ public:
return UInt64(data[n]);
}
UInt8 getUInt8(size_t n) const override
UInt8 getBoolRepresentation(size_t n) const override
{
return UInt8(!!data[n]);
}

View File

@ -95,9 +95,9 @@ public:
throw Exception("Method getUInt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
virtual UInt8 getUInt8(size_t /*n*/) const
virtual UInt8 getBoolRepresentation(size_t /*n*/) const
{
throw Exception("Method getUInt8 is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
throw Exception("Method getBoolRepresentation is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
}
virtual Int64 getInt(size_t /*n*/) const

View File

@ -31,8 +31,9 @@ void FunctionOneOrZero::executeImpl(Block & block, const ColumnNumbers & argumen
auto col_res = ColumnUInt8::create();
auto & vec_res = col_res->getData();
vec_res.resize(data_column->size());
for (size_t i = 0; i < data_column->size(); ++i) {
if (data_column->getUInt8(i)) {
for (size_t i = 0; i < data_column->size(); ++i)
{
if (data_column->getBoolRepresentation(i)) {
vec_res[i] = 1;
} else {
vec_res[i] = 0;
@ -58,6 +59,11 @@ size_t FunctionProject::getNumberOfArguments() const
DataTypePtr FunctionProject::getReturnTypeImpl(const DataTypes & arguments) const
{
if (!checkAndGetDataType<DataTypeUInt8>(arguments[1].get()))
{
throw Exception("Illegal type " + arguments[1]->getName() + " of 2nd argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
return arguments[0];
}
@ -71,7 +77,7 @@ void FunctionProject::executeImpl(Block & block, const ColumnNumbers & arguments
}
else if (const auto projection_column_uint8_const = checkAndGetColumnConst<ColumnUInt8>(projection_column.get()))
{
if (projection_column_uint8_const->getUInt8(0)) {
if (projection_column_uint8_const->getBoolRepresentation(0)) {
block.getByPosition(result).column = std::move(data_column->cloneResized(data_column->size()));
} else {
block.getByPosition(result).column = std::move(data_column->cloneEmpty());
@ -98,8 +104,16 @@ size_t FunctionBuildProjectionComposition::getNumberOfArguments() const
return 2;
}
DataTypePtr FunctionBuildProjectionComposition::getReturnTypeImpl(const DataTypes & /*arguments*/) const
DataTypePtr FunctionBuildProjectionComposition::getReturnTypeImpl(const DataTypes & arguments) const
{
for (size_t i = 0; i < 2; ++i)
{
if (!checkAndGetDataType<DataTypeUInt8>(arguments[i].get()))
{
throw Exception("Illegal type " + arguments[i]->getName() + " of " + std::to_string(i + 1) + " argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
}
return std::make_shared<DataTypeUInt8>();
}
@ -110,14 +124,19 @@ void FunctionBuildProjectionComposition::executeImpl(Block & block, const Column
auto col_res = ColumnUInt8::create();
auto & vec_res = col_res->getData();
vec_res.resize(first_projection_column->size());
size_t current_reverse_index = 0;
size_t current_reserve_index = 0;
for (size_t i = 0; i < first_projection_column->size(); ++i) {
if (first_projection_column->getUInt8(i) == 0) {
if (first_projection_column->getBoolRepresentation(i) == 0) {
vec_res[i] = 0;
} else {
vec_res[i] = second_projection_column->getUInt8(current_reverse_index++);
vec_res[i] = second_projection_column->getBoolRepresentation(current_reserve_index++);
}
}
if (current_reserve_index != second_projection_column->size()) {
throw Exception("Second argument size is not appropriate: " + std::to_string(second_projection_column->size())
+ " instead of " + std::to_string(current_reserve_index),
ErrorCodes::BAD_ARGUMENTS);
}
block.getByPosition(result).column = std::move(col_res);
}
@ -157,7 +176,7 @@ void FunctionRestoreProjection::executeImpl(Block & block, const ColumnNumbers &
auto col_res = block.getByPosition(arguments[1]).column->cloneEmpty();
std::vector<size_t> override_indices(arguments.size() - 1, 0);
for (size_t i = 0; i < projection_column->size(); ++i) {
size_t argument_index = projection_column->getUInt8(i);
size_t argument_index = projection_column->getBoolRepresentation(i);
col_res->insertFrom(*block.getByPosition(arguments[argument_index + 1]).column, override_indices[argument_index]++);
}
block.getByPosition(result).column = std::move(col_res);

View File

@ -44,6 +44,11 @@ Names ExpressionAction::getNeededColumns() const
if (!source_name.empty())
res.push_back(source_name);
if (!row_projection_column.empty())
{
res.push_back(row_projection_column);
}
return res;
}
@ -51,7 +56,7 @@ Names ExpressionAction::getNeededColumns() const
ExpressionAction ExpressionAction::applyFunction(const FunctionBuilderPtr & function_,
const std::vector<std::string> & argument_names_,
std::string result_name_,
const std::string & input_row_projection_expression)
const std::string & row_projection_column)
{
if (result_name_ == "")
{
@ -70,12 +75,12 @@ ExpressionAction ExpressionAction::applyFunction(const FunctionBuilderPtr & func
a.result_name = result_name_;
a.function_builder = function_;
a.argument_names = argument_names_;
a.input_row_projection_expression = input_row_projection_expression;
a.row_projection_column = row_projection_column;
return a;
}
ExpressionAction ExpressionAction::addColumn(const ColumnWithTypeAndName & added_column_,
const std::string & input_row_projection_expression,
const std::string & row_projection_column,
bool is_row_projection_complementary)
{
ExpressionAction a;
@ -83,7 +88,7 @@ ExpressionAction ExpressionAction::addColumn(const ColumnWithTypeAndName & added
a.result_name = added_column_.name;
a.result_type = added_column_.type;
a.added_column = added_column_.column;
a.input_row_projection_expression = input_row_projection_expression;
a.row_projection_column = row_projection_column;
a.is_row_projection_complementary = is_row_projection_complementary;
return a;
}
@ -123,16 +128,6 @@ ExpressionAction ExpressionAction::project(const Names & projected_columns_)
return a;
}
ExpressionAction ExpressionAction::measureInputRowsCount(const std::string & source_name,
const std::string & output_row_projection_expression)
{
ExpressionAction a;
a.type = MEASURE_INPUT_ROWS_COUNT;
a.source_name = source_name;
a.output_row_projection_expression = output_row_projection_expression;
return a;
}
ExpressionAction ExpressionAction::arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left, const Context & context)
{
if (array_joined_columns.empty())
@ -221,12 +216,6 @@ void ExpressionAction::prepare(Block & sample_block)
break;
}
case MEASURE_INPUT_ROWS_COUNT:
{
// Do nothing
break;
}
case ARRAY_JOIN:
{
for (const auto & name : array_joined_columns)
@ -297,15 +286,39 @@ void ExpressionAction::prepare(Block & sample_block)
}
}
size_t ExpressionAction::getInputRowsCount(Block & block, std::unordered_map<std::string, size_t> & input_rows_counts) const {
auto it = input_rows_counts.find(row_projection_column);
size_t projection_space_dimention;
if (it == input_rows_counts.end())
{
const auto & projection_column = block.getByName(row_projection_column).column;
projection_space_dimention = 0;
for (size_t i = 0; i < projection_column->size(); ++i) {
if (projection_column->getBoolRepresentation(i) > 0) {
++projection_space_dimention;
}
}
input_rows_counts[row_projection_column] = projection_space_dimention;
}
else
{
projection_space_dimention = it->second;
}
size_t parent_space_dimention;
if (row_projection_column.empty()) {
parent_space_dimention = input_rows_counts[""];
} else {
parent_space_dimention = block.getByName(row_projection_column).column->size();
}
return is_row_projection_complementary ? parent_space_dimention - projection_space_dimention : projection_space_dimention;
}
void ExpressionAction::execute(Block & block, std::unordered_map<std::string, size_t> & input_rows_counts) const
{
// std::cerr << "executing: " << toString() << std::endl;
size_t input_rows_count = input_rows_counts[input_row_projection_expression];
if (is_row_projection_complementary) {
input_rows_count = input_rows_counts[""] - input_rows_count;
}
size_t input_rows_count = getInputRowsCount(block, input_rows_counts);
if (type == REMOVE_COLUMN || type == COPY_COLUMN)
if (!block.has(source_name))
@ -336,22 +349,6 @@ void ExpressionAction::execute(Block & block, std::unordered_map<std::string, si
break;
}
case MEASURE_INPUT_ROWS_COUNT:
{
const auto & projection_column = block.getByName(source_name).column;
size_t projection_size = 0;
for (size_t i = 0; i < projection_column->size(); ++i) {
if (projection_column->getUInt8(i) > 0) {
++projection_size;
}
}
input_rows_counts[output_row_projection_expression] = projection_size;
break;
}
case ARRAY_JOIN:
{
if (array_joined_columns.empty())
@ -887,6 +884,9 @@ void ExpressionActions::finalize(const Names & output_columns)
if (!action.source_name.empty())
++columns_refcount[action.source_name];
if (!action.row_projection_column.empty())
++columns_refcount[action.row_projection_column];
for (const auto & name : action.argument_names)
++columns_refcount[name];
@ -915,6 +915,9 @@ void ExpressionActions::finalize(const Names & output_columns)
if (!action.source_name.empty())
process(action.source_name);
if (!action.row_projection_column.empty())
process(action.row_projection_column);
for (const auto & name : action.argument_names)
process(name);

View File

@ -58,8 +58,6 @@ public:
/// Reorder and rename the columns, delete the extra ones. The same column names are allowed in the result.
PROJECT,
MEASURE_INPUT_ROWS_COUNT,
};
Type type;
@ -70,9 +68,8 @@ public:
DataTypePtr result_type;
/// For conditional projections (projections on subset of rows)
std::string input_row_projection_expression;
std::string row_projection_column;
bool is_row_projection_complementary = false;
std::string output_row_projection_expression;
/// For ADD_COLUMN.
ColumnPtr added_column;
@ -96,17 +93,15 @@ public:
/// If result_name_ == "", as name "function_name(arguments separated by commas) is used".
static ExpressionAction applyFunction(
const FunctionBuilderPtr & function_, const std::vector<std::string> & argument_names_, std::string result_name_ = "",
const std::string & input_row_projection_expression = "");
const std::string & row_projection_column = "");
static ExpressionAction addColumn(const ColumnWithTypeAndName & added_column_,
const std::string & input_row_projection_expression,
const std::string & row_projection_column,
bool is_row_projection_complementary);
static ExpressionAction removeColumn(const std::string & removed_name);
static ExpressionAction copyColumn(const std::string & from_name, const std::string & to_name);
static ExpressionAction project(const NamesWithAliases & projected_columns_);
static ExpressionAction project(const Names & projected_columns_);
static ExpressionAction measureInputRowsCount(const std::string & source_name,
const std::string & output_row_projection_expression);
static ExpressionAction arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left, const Context & context);
static ExpressionAction ordinaryJoin(std::shared_ptr<const Join> join_, const NamesAndTypesList & columns_added_by_join_);
@ -119,6 +114,7 @@ private:
friend class ExpressionActions;
void prepare(Block & sample_block);
size_t getInputRowsCount(Block & block, std::unordered_map<std::string, size_t> & input_rows_counts) const;
void execute(Block & block, std::unordered_map<std::string, size_t> & input_rows_counts) const;
void executeOnTotals(Block & block) const;
};

View File

@ -2055,7 +2055,7 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
ColumnWithTypeAndName fake_column;
fake_column.name = projection_manipulator->getColumnName(node->getColumnName());
fake_column.type = std::make_shared<DataTypeUInt8>();
actions_stack.addAction(ExpressionAction::addColumn(fake_column, projection_manipulator->getProjectionExpression(), false));
actions_stack.addAction(ExpressionAction::addColumn(fake_column, projection_manipulator->getProjectionSourceColumn(), false));
getActionsImpl(node->arguments->children.at(0), no_subqueries, only_consts, actions_stack,
projection_manipulator);
}
@ -2069,7 +2069,7 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
{
actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName(
ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared<DataTypeUInt8>(),
projection_manipulator->getColumnName(node->getColumnName())), projection_manipulator->getProjectionExpression(), false));
projection_manipulator->getColumnName(node->getColumnName())), projection_manipulator->getProjectionSourceColumn(), false));
return;
}
@ -2125,7 +2125,7 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
{
column.column = ColumnSet::create(1, set);
actions_stack.addAction(ExpressionAction::addColumn(column, projection_manipulator->getProjectionExpression(), false));
actions_stack.addAction(ExpressionAction::addColumn(column, projection_manipulator->getProjectionSourceColumn(), false));
}
argument_types.push_back(column.type);
@ -2212,7 +2212,7 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
auto function_capture = std::make_shared<FunctionCapture>(
lambda_actions, captured, lambda_arguments, result_type, result_name);
actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name,
projection_manipulator->getProjectionExpression()));
projection_manipulator->getProjectionSourceColumn()));
argument_types[i] = std::make_shared<DataTypeFunction>(lambda_type->getArgumentTypes(), result_type);
argument_names[i] = lambda_name;
@ -2242,7 +2242,7 @@ void ExpressionAnalyzer::getActionsImpl(const ASTPtr & ast, bool no_subqueries,
ExpressionAction::applyFunction(function_builder,
argument_names,
projection_manipulator->getColumnName(node->getColumnName()),
projection_manipulator->getProjectionExpression()));
projection_manipulator->getProjectionSourceColumn()));
}
}
}

View File

@ -33,6 +33,9 @@ std::string DefaultProjectionManipulator::getProjectionExpression() {
return "";
}
std::string DefaultProjectionManipulator::getProjectionSourceColumn() const {
return "";
}
ConditionalTree::Node::Node()
: projection_expression_string(),
@ -72,12 +75,12 @@ std::string ConditionalTree::getColumnName(const std::string & col_name) const
}
std::string ConditionalTree::getProjectionColumnName(const std::string & first_projection_expr,
const std::string & second_projection_expr)
const std::string & second_projection_expr) const
{
return std::string("P<") + first_projection_expr + "><" + second_projection_expr + ">";
}
std::string ConditionalTree::getProjectionColumnName(const size_t first_index, const size_t second_index)
std::string ConditionalTree::getProjectionColumnName(const size_t first_index, const size_t second_index) const
{
return getProjectionColumnName(
nodes[first_index].projection_expression_string,
@ -102,7 +105,7 @@ void ConditionalTree::buildProjectionCompositionRecursive(const std::vector<size
getProjectionColumnName(path[parent_index], path[middle_index]),
getProjectionColumnName(path[middle_index], path[child_index])
},
projection_name, getProjectionExpression()));
projection_name, getProjectionSourceColumn()));
}
}
@ -120,6 +123,16 @@ void ConditionalTree::buildProjectionComposition(const size_t child_node, const
buildProjectionCompositionRecursive(path, 0, path.size() - 1);
}
std::string ConditionalTree::getProjectionSourceColumn(size_t node) const {
if (nodes[node].is_root)
{
return "";
}
else
{
return ConditionalTree::getProjectionColumnName(nodes[node].getParentNode(), node);
}
}
ConditionalTree::ConditionalTree(ScopeStack & scopes, const Context & context)
: current_node(0),
@ -139,8 +152,7 @@ void ConditionalTree::goToProjection(const std::string & field_name)
if (!scopes.getSampleBlock().has(projection_column_name)) {
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("one_or_zero", context);
scopes.addAction(ExpressionAction::applyFunction(function_builder, {getColumnName(field_name)}, projection_column_name,
getProjectionExpression()));
scopes.addAction(ExpressionAction::measureInputRowsCount(projection_column_name, new_projection_name));
getProjectionSourceColumn()));
nodes.emplace_back(Node());
nodes.back().projection_expression_string = new_projection_name;
nodes.back().parent_node = current_node;
@ -151,6 +163,15 @@ void ConditionalTree::goToProjection(const std::string & field_name)
}
}
std::string ConditionalTree::buildRestoreProjectionAndGetName(const size_t levels_up) {
size_t target_node = current_node;
for (size_t i = 0; i < levels_up; ++i) {
target_node = nodes[target_node].getParentNode();
}
buildProjectionComposition(current_node, target_node);
return getProjectionColumnName(target_node, current_node);
}
void ConditionalTree::restoreColumn(
const std::string & default_values_name,
const std::string & new_values_name,
@ -162,7 +183,6 @@ void ConditionalTree::restoreColumn(
for (size_t i = 0; i < levels_up; ++i) {
target_node = nodes[target_node].getParentNode();
}
buildProjectionComposition(current_node, target_node);
const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("__inner_restore_projection__",
context);
scopes.addAction(ExpressionAction::applyFunction(
@ -172,7 +192,7 @@ void ConditionalTree::restoreColumn(
getColumnNameByIndex(default_values_name, current_node),
getColumnNameByIndex(new_values_name, current_node)
},
getColumnNameByIndex(result_name, target_node), getProjectionExpression()));
getColumnNameByIndex(result_name, target_node), getProjectionSourceColumn()));
}
void ConditionalTree::goUp(const size_t levels_up)
@ -197,7 +217,7 @@ bool ConditionalTree::isAlreadyComputed(const std::string & column_name)
getColumnNameByIndex(column_name, node),
getProjectionColumnName(node, current_node)
},
getColumnName(column_name), nodes[node].projection_expression_string));
getColumnName(column_name), getProjectionSourceColumn(node)));
}
return true;
}
@ -213,6 +233,10 @@ std::string ConditionalTree::getProjectionExpression() {
return nodes[current_node].projection_expression_string;
}
std::string ConditionalTree::getProjectionSourceColumn() const {
return getProjectionSourceColumn(current_node);
}
void DefaultProjectionAction::preArgumentAction()
{}
@ -250,14 +274,14 @@ std::string AndOperatorProjectionAction::getFinalColumnName()
return "__inner_final_column__" + expression_name;
}
void AndOperatorProjectionAction::createZerosColumn()
void AndOperatorProjectionAction::createZerosColumn(const std::string & restore_projection_name)
{
auto zeros_column_name = projection_manipulator->getColumnName(getZerosColumnName());
if (!scopes.getSampleBlock().has(zeros_column_name))
{
scopes.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName(
ColumnUInt8::create(0, 1), std::make_shared<DataTypeUInt8>(), zeros_column_name),
projection_manipulator->getProjectionExpression(), true));
restore_projection_name, true));
}
}
@ -291,8 +315,9 @@ void AndOperatorProjectionAction::preCalculation()
projection_manipulator->getColumnName(previous_argument_name)
},
projection_manipulator->getColumnName(final_column),
projection_manipulator->getProjectionExpression()));
createZerosColumn();
projection_manipulator->getProjectionSourceColumn()));
std::string restore_projection_name = conditional_tree->buildRestoreProjectionAndGetName(projection_levels_count);
createZerosColumn(restore_projection_name);
conditional_tree->restoreColumn(getZerosColumnName(), final_column,
projection_levels_count, expression_name);
conditional_tree->goUp(projection_levels_count);

View File

@ -25,6 +25,8 @@ public:
virtual std::string getProjectionExpression() = 0;
virtual std::string getProjectionSourceColumn() const = 0;
virtual ~ProjectionManipulatorBase();
};
@ -46,6 +48,8 @@ public:
std::string getColumnName(const std::string & col_name) const final;
std::string getProjectionExpression() final;
std::string getProjectionSourceColumn() const final;
};
/*
@ -103,9 +107,9 @@ private:
std::string getColumnNameByIndex(const std::string & col_name, size_t node) const;
std::string getProjectionColumnName(const std::string & first_projection_expr,
const std::string & second_projection_expr);
const std::string & second_projection_expr) const;
std::string getProjectionColumnName(size_t first_index, size_t second_index);
std::string getProjectionColumnName(size_t first_index, size_t second_index) const;
void buildProjectionCompositionRecursive(const std::vector<size_t> & path,
size_t child_index,
@ -113,6 +117,7 @@ private:
void buildProjectionComposition(size_t child_node, size_t parent_node);
std::string getProjectionSourceColumn(size_t node) const;
public:
ConditionalTree(ScopeStack & scopes, const Context & context);
@ -120,10 +125,12 @@ public:
void goToProjection(const std::string & field_name);
std::string buildRestoreProjectionAndGetName(size_t levels_up);
void restoreColumn(
const std::string & default_values_name,
const std::string & new_values_name,
const size_t levels_up,
size_t levels_up,
const std::string & result_name
);
@ -132,6 +139,8 @@ public:
bool isAlreadyComputed(const std::string & column_name) final;
std::string getProjectionExpression() final;
std::string getProjectionSourceColumn() const final;
};
using ConditionalTreePtr = std::shared_ptr<ConditionalTree>;
@ -195,7 +204,7 @@ private:
std::string getFinalColumnName();
void createZerosColumn();
void createZerosColumn(const std::string & restore_projection_name);
public:
AndOperatorProjectionAction(ScopeStack & scopes,
ProjectionManipulatorPtr projection_manipulator,