diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index 47144516fb0f..a6109c709349 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -32,13 +32,6 @@ generate_function_register(Arithmetic FunctionIntExp10 ) -generate_function_register(Projection - FunctionOneOrZero - FunctionProject - FunctionBuildProjectionComposition - FunctionRestoreProjection -) - add_headers_and_sources(clickhouse_functions .) add_headers_and_sources(clickhouse_functions ./GatherUtils) diff --git a/dbms/src/Functions/FunctionsProjection.cpp b/dbms/src/Functions/FunctionsProjection.cpp deleted file mode 100644 index c526d20b5589..000000000000 --- a/dbms/src/Functions/FunctionsProjection.cpp +++ /dev/null @@ -1,203 +0,0 @@ -#include -#include -#include -#include - -namespace DB -{ -FunctionPtr FunctionOneOrZero::create(const Context &) -{ - return std::make_shared(); -} - -String FunctionOneOrZero::getName() const -{ - return name; -} - -size_t FunctionOneOrZero::getNumberOfArguments() const -{ - return 1; -} - -DataTypePtr FunctionOneOrZero::getReturnTypeImpl(const DataTypes & /*arguments*/) const -{ - return std::make_shared(); -} - -void FunctionOneOrZero::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - const auto & data_column = block.getByPosition(arguments[0]).column; - 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->getBool(i)) - { - vec_res[i] = 1; - } - else - { - vec_res[i] = 0; - } - } - block.getByPosition(result).column = std::move(col_res); -} - -FunctionPtr FunctionProject::create(const Context &) -{ - return std::make_shared(); -} - -String FunctionProject::getName() const -{ - return name; -} - -size_t FunctionProject::getNumberOfArguments() const -{ - return 2; -} - -DataTypePtr FunctionProject::getReturnTypeImpl(const DataTypes & arguments) const -{ - if (!checkAndGetDataType(arguments[1].get())) - { - throw Exception( - "Illegal type " + arguments[1]->getName() + " of 2nd argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - } - return arguments[0]; -} - -void FunctionProject::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - const auto & data_column = block.getByPosition(arguments[0]).column; - const auto & projection_column = block.getByPosition(arguments[1]).column; - if (const auto projection_column_uint8 = checkAndGetColumn(projection_column.get())) - { - block.getByPosition(result).column = data_column->filter(projection_column_uint8->getData(), -1); - } - else if (const auto projection_column_uint8_const = checkAndGetColumnConst(projection_column.get())) - { - if (projection_column_uint8_const->getBool(0)) - { - block.getByPosition(result).column = data_column->cloneResized(data_column->size()); - } - else - { - block.getByPosition(result).column = data_column->cloneEmpty(); - } - } - else - { - throw Exception("Unexpected column: " + projection_column->getName(), ErrorCodes::ILLEGAL_COLUMN); - } -} - -FunctionPtr FunctionBuildProjectionComposition::create(const Context &) -{ - return std::make_shared(); -} - -String FunctionBuildProjectionComposition::getName() const -{ - return name; -} - -size_t FunctionBuildProjectionComposition::getNumberOfArguments() const -{ - return 2; -} - -DataTypePtr FunctionBuildProjectionComposition::getReturnTypeImpl(const DataTypes & arguments) const -{ - for (size_t i = 0; i < 2; ++i) - { - if (!checkAndGetDataType(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(); -} - -void FunctionBuildProjectionComposition::executeImpl( - Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - const auto & first_projection_column = block.getByPosition(arguments[0]).column; - const auto & second_projection_column = block.getByPosition(arguments[1]).column; - auto col_res = ColumnUInt8::create(); - auto & vec_res = col_res->getData(); - vec_res.resize(first_projection_column->size()); - size_t current_reserve_index = 0; - for (size_t i = 0; i < first_projection_column->size(); ++i) - { - if (!first_projection_column->getBool(i)) - { - vec_res[i] = 0; - } - else - { - vec_res[i] = second_projection_column->getBool(current_reserve_index); - ++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); -} - -FunctionPtr FunctionRestoreProjection::create(const Context &) -{ - return std::make_shared(); -} - -String FunctionRestoreProjection::getName() const -{ - return name; -} - -bool FunctionRestoreProjection::isVariadic() const -{ - return true; -} - -size_t FunctionRestoreProjection::getNumberOfArguments() const -{ - return 0; -} - -DataTypePtr FunctionRestoreProjection::getReturnTypeImpl(const DataTypes & arguments) const -{ - if (arguments.size() < 2) - { - throw Exception("Wrong argument count: " + std::to_string(arguments.size()), ErrorCodes::BAD_ARGUMENTS); - } - return arguments[1]; -} - -void FunctionRestoreProjection::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) -{ - if (arguments.size() < 2) - { - throw Exception("Wrong argument count: " + std::to_string(arguments.size()), ErrorCodes::BAD_ARGUMENTS); - } - const auto & projection_column = block.getByPosition(arguments[0]).column; - auto col_res = block.getByPosition(arguments[1]).column->cloneEmpty(); - std::vector override_indices(arguments.size() - 1, 0); - for (size_t i = 0; i < projection_column->size(); ++i) - { - size_t argument_index = projection_column->getBool(i); - col_res->insertFrom(*block.getByPosition(arguments[argument_index + 1]).column, override_indices[argument_index]++); - } - block.getByPosition(result).column = std::move(col_res); -} - -} diff --git a/dbms/src/Functions/FunctionsProjection.h b/dbms/src/Functions/FunctionsProjection.h deleted file mode 100644 index bbb1951fc5be..000000000000 --- a/dbms/src/Functions/FunctionsProjection.h +++ /dev/null @@ -1,71 +0,0 @@ -#pragma once - -#include -#include "FunctionsConversion.h" - -namespace DB -{ -/* - * This function accepts one column and converts it to UInt8, replacing values, which evaluate to true, with 1, and values, - * which evaluate to false with 0 - */ -class FunctionOneOrZero final : public IFunction -{ -public: - static constexpr auto name = "one_or_zero"; - static FunctionPtr create(const Context &); - String getName() const override; - size_t getNumberOfArguments() const override; - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - -/* - * FunctionProject accepts two columns: data column and projection column. - * Projection column is a column of UInt8 values 0 and 1, which indicate the binary mask of rows, where to project. - * This function builds a column of a smaller, which contains values of the data column at the positions where - * the projection column contained 1. The size of result column equals the count of ones in the projection column. - */ -class FunctionProject final : public IFunction -{ -public: - static constexpr auto name = "__inner_project__"; - static FunctionPtr create(const Context &); - String getName() const override; - size_t getNumberOfArguments() const override; - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - -/* - * FunctionBuildProjectionComposition constructs the composition of two projection columns. The size of - * second projection column should equal the count of ones in the first input projection column. - */ -class FunctionBuildProjectionComposition final : public IFunction -{ -public: - static constexpr auto name = "__inner_build_projection_composition__"; - static FunctionPtr create(const Context &); - String getName() const override; - size_t getNumberOfArguments() const override; - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - -/* - * Accepts mapping column with values from range [0, N) and N more columns as arguments. - * Forms a column by taking value from column, which number is in the mapping column. - */ -class FunctionRestoreProjection final : public IFunction -{ -public: - static constexpr auto name = "__inner_restore_projection__"; - static FunctionPtr create(const Context &); - String getName() const override; - bool isVariadic() const override; - size_t getNumberOfArguments() const override; - DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override; -}; - -} diff --git a/dbms/src/Functions/registerFunctions.cpp b/dbms/src/Functions/registerFunctions.cpp index c7f1ca82cabc..292f6f608df1 100644 --- a/dbms/src/Functions/registerFunctions.cpp +++ b/dbms/src/Functions/registerFunctions.cpp @@ -40,7 +40,6 @@ void registerFunctionsTransform(FunctionFactory &); void registerFunctionsGeo(FunctionFactory &); void registerFunctionsNull(FunctionFactory &); void registerFunctionsFindCluster(FunctionFactory &); -void registerFunctionsProjection(FunctionFactory &); #if USE_ICU void registerFunctionConvertCharset(FunctionFactory &); @@ -80,7 +79,6 @@ void registerFunctions() registerFunctionsGeo(factory); registerFunctionsNull(factory); registerFunctionsFindCluster(factory); - registerFunctionsProjection(factory); #if USE_ICU registerFunctionConvertCharset(factory); diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index c781da56bccb..ef9f73ac1949 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -27,7 +27,6 @@ #include #include -#include #include #include #include @@ -223,7 +222,7 @@ const Block & ScopeStack::getSampleBlock() const ActionsVisitor::ActionsVisitor( - const Context & context_, SizeLimits set_size_limit_, bool is_conditional_tree, size_t subquery_depth_, + const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_) @@ -240,10 +239,6 @@ ActionsVisitor::ActionsVisitor( ostr(ostr_), actions_stack(actions, context) { - if (is_conditional_tree) - projection_manipulator = std::make_shared(actions_stack, context); - else - projection_manipulator = std::make_shared(actions_stack); } void ActionsVisitor::visit(const ASTPtr & ast) @@ -261,12 +256,12 @@ void ActionsVisitor::visit(const ASTPtr & ast) /// If the result of the calculation already exists in the block. if ((typeid_cast(ast.get()) || typeid_cast(ast.get())) - && projection_manipulator->tryToGetFromUpperProjection(getColumnName())) + && actions_stack.getSampleBlock().has(getColumnName())) return; if (auto * identifier = typeid_cast(ast.get())) { - if (!only_consts && !projection_manipulator->tryToGetFromUpperProjection(getColumnName())) + if (!only_consts && !actions_stack.getSampleBlock().has(getColumnName())) { /// The requested column is not in the block. /// If such a column exists in the table, then the user probably forgot to surround it with an aggregate function or add it to GROUP BY. @@ -300,8 +295,8 @@ void ActionsVisitor::visit(const ASTPtr & ast) visit(arg); if (!only_consts) { - String result_name = projection_manipulator->getColumnName(getColumnName()); - actions_stack.addAction(ExpressionAction::copyColumn(projection_manipulator->getColumnName(arg->getColumnName()), result_name)); + String result_name = getColumnName(); + actions_stack.addAction(ExpressionAction::copyColumn(arg->getColumnName(), result_name)); NameSet joined_columns; joined_columns.insert(result_name); actions_stack.addAction(ExpressionAction::arrayJoin(joined_columns, false, context)); @@ -330,8 +325,7 @@ void ActionsVisitor::visit(const ASTPtr & ast) actions_stack.addAction(ExpressionAction::applyFunction( FunctionFactory::instance().get("ignore", context), { node->arguments->children.at(0)->getColumnName() }, - projection_manipulator->getColumnName(getColumnName()), - projection_manipulator->getProjectionSourceColumn())); + getColumnName())); } return; } @@ -343,7 +337,7 @@ void ActionsVisitor::visit(const ASTPtr & ast) { actions_stack.addAction(ExpressionAction::addColumn(ColumnWithTypeAndName( ColumnConst::create(ColumnUInt8::create(1, 1), 1), std::make_shared(), - projection_manipulator->getColumnName(getColumnName())), projection_manipulator->getProjectionSourceColumn(), false)); + getColumnName()))); return; } @@ -356,7 +350,6 @@ void ActionsVisitor::visit(const ASTPtr & ast) : context; const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get(node->name, function_context); - auto projection_action = getProjectionAction(node->name, actions_stack, projection_manipulator, getColumnName(), function_context); Names argument_names; DataTypes argument_types; @@ -401,13 +394,11 @@ void ActionsVisitor::visit(const ASTPtr & ast) else column.name = child_column_name; - column.name = projection_manipulator->getColumnName(column.name); - if (!actions_stack.getSampleBlock().has(column.name)) { column.column = ColumnSet::create(1, set); - actions_stack.addAction(ExpressionAction::addColumn(column, projection_manipulator->getProjectionSourceColumn(), false)); + actions_stack.addAction(ExpressionAction::addColumn(column)); } argument_types.push_back(column.type); @@ -416,10 +407,8 @@ void ActionsVisitor::visit(const ASTPtr & ast) else { /// If the argument is not a lambda expression, call it recursively and find out its type. - projection_action->preArgumentAction(); visit(child); - std::string name = projection_manipulator->getColumnName(child_column_name); - projection_action->postArgumentAction(child_column_name); + std::string name = child_column_name; if (actions_stack.getSampleBlock().has(name)) { argument_types.push_back(actions_stack.getSampleBlock().getByName(name).type); @@ -428,13 +417,9 @@ void ActionsVisitor::visit(const ASTPtr & ast) else { if (only_consts) - { arguments_present = false; - } else - { - throw Exception("Unknown identifier: " + name + ", projection layer " + projection_manipulator->getProjectionExpression() , ErrorCodes::UNKNOWN_IDENTIFIER); - } + throw Exception("Unknown identifier: " + name, ErrorCodes::UNKNOWN_IDENTIFIER); } } } @@ -470,12 +455,11 @@ void ActionsVisitor::visit(const ASTPtr & ast) lambda_arguments.emplace_back(arg_name, lambda_type->getArgumentTypes()[j]); } - projection_action->preArgumentAction(); actions_stack.pushLevel(lambda_arguments); visit(lambda->arguments->children.at(1)); ExpressionActionsPtr lambda_actions = actions_stack.popLevel(); - String result_name = projection_manipulator->getColumnName(lambda->arguments->children.at(1)->getColumnName()); + String result_name = lambda->arguments->children.at(1)->getColumnName(); lambda_actions->finalize(Names(1, result_name)); DataTypePtr result_type = lambda_actions->getSampleBlock().getByName(result_name).type; @@ -491,12 +475,10 @@ void ActionsVisitor::visit(const ASTPtr & ast) auto function_capture = std::make_shared( lambda_actions, captured, lambda_arguments, result_type, result_name); - actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name, - projection_manipulator->getProjectionSourceColumn())); + actions_stack.addAction(ExpressionAction::applyFunction(function_capture, captured, lambda_name)); argument_types[i] = std::make_shared(lambda_type->getArgumentTypes(), result_type); argument_names[i] = lambda_name; - projection_action->postArgumentAction(lambda_name); } } } @@ -515,15 +497,8 @@ void ActionsVisitor::visit(const ASTPtr & ast) if (arguments_present) { - projection_action->preCalculation(); - if (projection_action->isCalculationRequired()) - { - actions_stack.addAction( - ExpressionAction::applyFunction(function_builder, - argument_names, - projection_manipulator->getColumnName(getColumnName()), - projection_manipulator->getProjectionSourceColumn())); - } + actions_stack.addAction( + ExpressionAction::applyFunction(function_builder, argument_names, getColumnName())); } } else if (ASTLiteral * literal = typeid_cast(ast.get())) @@ -535,8 +510,7 @@ void ActionsVisitor::visit(const ASTPtr & ast) column.type = type; column.name = getColumnName(); - actions_stack.addAction(ExpressionAction::addColumn(column, "", false)); - projection_manipulator->tryToGetFromUpperProjection(column.name); + actions_stack.addAction(ExpressionAction::addColumn(column)); } else { diff --git a/dbms/src/Interpreters/ActionsVisitor.h b/dbms/src/Interpreters/ActionsVisitor.h index 0344934455d0..805b0ec02c44 100644 --- a/dbms/src/Interpreters/ActionsVisitor.h +++ b/dbms/src/Interpreters/ActionsVisitor.h @@ -9,7 +9,6 @@ namespace DB class Context; class ASTFunction; -struct ProjectionManipulatorBase; class Set; @@ -89,7 +88,7 @@ struct ScopeStack class ActionsVisitor { public: - ActionsVisitor(const Context & context_, SizeLimits set_size_limit_, bool is_conditional_tree, size_t subquery_depth_, + ActionsVisitor(const Context & context_, SizeLimits set_size_limit_, size_t subquery_depth_, const NamesAndTypesList & source_columns_, const ExpressionActionsPtr & actions, PreparedSets & prepared_sets_, SubqueriesForSets & subqueries_for_sets_, bool no_subqueries_, bool only_consts_, bool no_storage_or_local_, std::ostream * ostr_ = nullptr); @@ -111,7 +110,6 @@ class ActionsVisitor mutable size_t visit_depth; std::ostream * ostr; ScopeStack actions_stack; - std::shared_ptr projection_manipulator; void makeSet(const ASTFunction * node, const Block & sample_block); }; diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 11a7ecfb1a40..b1fab40a6545 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -51,11 +51,6 @@ 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; } @@ -63,8 +58,7 @@ Names ExpressionAction::getNeededColumns() const ExpressionAction ExpressionAction::applyFunction( const FunctionBuilderPtr & function_, const std::vector & argument_names_, - std::string result_name_, - const std::string & row_projection_column) + std::string result_name_) { if (result_name_ == "") { @@ -83,22 +77,17 @@ ExpressionAction ExpressionAction::applyFunction( a.result_name = result_name_; a.function_builder = function_; a.argument_names = argument_names_; - a.row_projection_column = row_projection_column; return a; } ExpressionAction ExpressionAction::addColumn( - const ColumnWithTypeAndName & added_column_, - const std::string & row_projection_column, - bool is_row_projection_complementary) + const ColumnWithTypeAndName & added_column_) { ExpressionAction a; a.type = ADD_COLUMN; a.result_name = added_column_.name; a.result_type = added_column_.type; a.added_column = added_column_.column; - a.row_projection_column = row_projection_column; - a.is_row_projection_complementary = is_row_projection_complementary; return a; } @@ -335,40 +324,10 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings) } } -size_t ExpressionAction::getInputRowsCount(Block & block, std::unordered_map & input_rows_counts) const -{ - auto it = input_rows_counts.find(row_projection_column); - size_t projection_space_dimension; - if (it == input_rows_counts.end()) - { - const auto & projection_column = block.getByName(row_projection_column).column; - projection_space_dimension = 0; - for (size_t i = 0; i < projection_column->size(); ++i) - if (projection_column->getBool(i)) - ++projection_space_dimension; - input_rows_counts[row_projection_column] = projection_space_dimension; - } - else - { - projection_space_dimension = it->second; - } - size_t parent_space_dimension; - if (row_projection_column.empty()) - { - parent_space_dimension = input_rows_counts[""]; - } - else - { - parent_space_dimension = block.getByName(row_projection_column).column->size(); - } - - return is_row_projection_complementary ? parent_space_dimension - projection_space_dimension : projection_space_dimension; -} - -void ExpressionAction::execute(Block & block, std::unordered_map & input_rows_counts) const +void ExpressionAction::execute(Block & block) const { - size_t input_rows_count = getInputRowsCount(block, input_rows_counts); + size_t input_rows_count = block.rows(); if (type == REMOVE_COLUMN || type == COPY_COLUMN) if (!block.has(source_name)) @@ -463,8 +422,6 @@ void ExpressionAction::execute(Block & block, std::unordered_map input_rows_counts; - input_rows_counts[""] = block.rows(); if (type != JOIN) - execute(block, input_rows_counts); + execute(block); else join->joinTotals(block); } @@ -751,11 +706,9 @@ bool ExpressionActions::popUnusedArrayJoin(const Names & required_columns, Expre void ExpressionActions::execute(Block & block) const { - std::unordered_map input_rows_counts; - input_rows_counts[""] = block.rows(); for (const auto & action : actions) { - action.execute(block, input_rows_counts); + action.execute(block); checkLimits(block); } } @@ -980,9 +933,6 @@ 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]; @@ -1011,9 +961,6 @@ 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); @@ -1240,8 +1187,6 @@ bool ExpressionAction::operator==(const ExpressionAction & other) const return source_name == other.source_name && result_name == other.result_name - && row_projection_column == other.row_projection_column - && is_row_projection_complementary == other.is_row_projection_complementary && argument_names == other.argument_names && array_joined_columns == other.array_joined_columns && array_join_is_left == other.array_join_is_left diff --git a/dbms/src/Interpreters/ExpressionActions.h b/dbms/src/Interpreters/ExpressionActions.h index 4dc81c7d9385..781134dbeb26 100644 --- a/dbms/src/Interpreters/ExpressionActions.h +++ b/dbms/src/Interpreters/ExpressionActions.h @@ -81,10 +81,6 @@ struct ExpressionAction /// If COPY_COLUMN can replace the result column. bool can_replace = false; - /// For conditional projections (projections on subset of rows) - std::string row_projection_column; - bool is_row_projection_complementary = false; - /// For ADD_COLUMN. ColumnPtr added_column; @@ -112,12 +108,9 @@ struct ExpressionAction /// If result_name_ == "", as name "function_name(arguments separated by commas) is used". static ExpressionAction applyFunction( - const FunctionBuilderPtr & function_, const std::vector & argument_names_, std::string result_name_ = "", - const std::string & row_projection_column = ""); + const FunctionBuilderPtr & function_, const std::vector & argument_names_, std::string result_name_ = ""); - static ExpressionAction addColumn(const ColumnWithTypeAndName & added_column_, - const std::string & row_projection_column, - bool is_row_projection_complementary); + static ExpressionAction addColumn(const ColumnWithTypeAndName & added_column_); static ExpressionAction removeColumn(const std::string & removed_name); static ExpressionAction copyColumn(const std::string & from_name, const std::string & to_name, bool can_replace = false); static ExpressionAction project(const NamesWithAliases & projected_columns_); @@ -143,8 +136,7 @@ struct ExpressionAction friend class ExpressionActions; void prepare(Block & sample_block, const Settings & settings); - size_t getInputRowsCount(Block & block, std::unordered_map & input_rows_counts) const; - void execute(Block & block, std::unordered_map & input_rows_counts) const; + void execute(Block & block) const; void executeOnTotals(Block & block) const; }; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 8e00a77e1091..d9f2406ef4ba 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -392,10 +392,8 @@ bool ExpressionAnalyzer::isThereArrayJoin(const ASTPtr & ast) void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts) { - bool is_conditional_tree = !isThereArrayJoin(ast) && settings.enable_conditional_computation && !only_consts; - LogAST log; - ActionsVisitor actions_visitor(context, settings.size_limits_for_set, is_conditional_tree, subquery_depth, + ActionsVisitor actions_visitor(context, settings.size_limits_for_set, subquery_depth, source_columns, actions, prepared_sets, subqueries_for_sets, no_subqueries, only_consts, !isRemoteStorage(), log.stream()); actions_visitor.visit(ast); @@ -406,10 +404,9 @@ void ExpressionAnalyzer::getRootActions(const ASTPtr & ast, bool no_subqueries, void ExpressionAnalyzer::getActionsFromJoinKeys(const ASTTableJoin & table_join, bool no_subqueries, ExpressionActionsPtr & actions) { bool only_consts = false; - bool is_conditional_tree = !isThereArrayJoin(query) && settings.enable_conditional_computation && !only_consts; LogAST log; - ActionsVisitor actions_visitor(context, settings.size_limits_for_set, is_conditional_tree, subquery_depth, + ActionsVisitor actions_visitor(context, settings.size_limits_for_set, subquery_depth, source_columns, actions, prepared_sets, subqueries_for_sets, no_subqueries, only_consts, !isRemoteStorage(), log.stream()); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 0c714e78af00..ba895b416502 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -113,7 +113,6 @@ class ExpressionAnalyzer : private ExpressionAnalyzerData, private boost::noncop /// for ExpressionAnalyzer const bool asterisk_left_columns_only; const bool use_index_for_in_with_subqueries; - const bool enable_conditional_computation; const bool join_use_nulls; const SizeLimits size_limits_for_set; const SizeLimits size_limits_for_join; @@ -127,7 +126,6 @@ class ExpressionAnalyzer : private ExpressionAnalyzerData, private boost::noncop enable_optimize_predicate_expression(settings.enable_optimize_predicate_expression), asterisk_left_columns_only(settings.asterisk_left_columns_only), use_index_for_in_with_subqueries(settings.use_index_for_in_with_subqueries), - enable_conditional_computation(settings.enable_conditional_computation), join_use_nulls(settings.join_use_nulls), size_limits_for_set(settings.max_rows_in_set, settings.max_bytes_in_set, settings.set_overflow_mode), size_limits_for_join(settings.max_rows_in_join, settings.max_bytes_in_join, settings.join_overflow_mode), diff --git a/dbms/src/Interpreters/ProjectionManipulation.cpp b/dbms/src/Interpreters/ProjectionManipulation.cpp deleted file mode 100644 index afdc289d2fa2..000000000000 --- a/dbms/src/Interpreters/ProjectionManipulation.cpp +++ /dev/null @@ -1,341 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -ProjectionManipulatorBase::~ProjectionManipulatorBase() {} - -DefaultProjectionManipulator::DefaultProjectionManipulator(ScopeStack & scopes) : scopes(scopes) {} - -bool DefaultProjectionManipulator::tryToGetFromUpperProjection(const std::string & column_name) -{ - return scopes.getSampleBlock().has(column_name); -} - -std::string DefaultProjectionManipulator::getColumnName(const std::string & column_name) const -{ - return column_name; -} - -std::string DefaultProjectionManipulator::getProjectionExpression() -{ - return ""; -} - -std::string DefaultProjectionManipulator::getProjectionSourceColumn() const -{ - return ""; -} - -ConditionalTree::Node::Node() : projection_expression_string(), parent_node(0), is_root(false) {} - -size_t ConditionalTree::Node::getParentNode() const -{ - if (is_root) - { - throw Exception( - "Failed to get parent projection node of node " + projection_expression_string, ErrorCodes::CONDITIONAL_TREE_PARENT_NOT_FOUND); - } - 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); -} - -std::string ConditionalTree::getProjectionColumnName( - const std::string & first_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) const -{ - return getProjectionColumnName(nodes[first_index].projection_expression_string, nodes[second_index].projection_expression_string); -} - -void ConditionalTree::buildProjectionCompositionRecursive( - const std::vector & path, const size_t child_index, const size_t parent_index) -{ - 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); - 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())); - } -} - -void ConditionalTree::buildProjectionComposition(const size_t child_node, const size_t parent_node) -{ - std::vector path; - size_t node = child_node; - while (true) - { - path.push_back(node); - if (node == parent_node) - { - break; - } - node = nodes[node].getParentNode(); - } - 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), nodes(1), scopes(scopes), context(context), projection_expression_index() -{ - 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); - 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, getProjectionSourceColumn())); - 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; - } - else - { - current_node = projection_expression_index[projection_column_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, const size_t levels_up, const std::string & result_name) -{ - size_t target_node = current_node; - for (size_t i = 0; i < levels_up; ++i) - { - target_node = nodes[target_node].getParentNode(); - } - const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("__inner_restore_projection__", context); - scopes.addAction(ExpressionAction::applyFunction(function_builder, - {getProjectionColumnName(target_node, current_node), - getColumnNameByIndex(default_values_name, current_node), - getColumnNameByIndex(new_values_name, current_node)}, - getColumnNameByIndex(result_name, target_node), - getProjectionSourceColumn())); -} - -void ConditionalTree::goUp(const size_t levels_up) -{ - for (size_t i = 0; i < levels_up; ++i) - { - current_node = nodes[current_node].getParentNode(); - } -} - -bool ConditionalTree::tryToGetFromUpperProjection(const std::string & column_name) -{ - size_t node = current_node; - while (true) - { - if (scopes.getSampleBlock().has(getColumnNameByIndex(column_name, node))) - { - if (node != current_node) - { - buildProjectionComposition(current_node, node); - 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))); - } - return true; - } - if (nodes[node].is_root) - { - break; - } - node = nodes[node].getParentNode(); - } - return false; -} - -std::string ConditionalTree::getProjectionExpression() -{ - return nodes[current_node].projection_expression_string; -} - -std::string ConditionalTree::getProjectionSourceColumn() const -{ - return getProjectionSourceColumn(current_node); -} - -void DefaultProjectionAction::preArgumentAction() {} - -void DefaultProjectionAction::postArgumentAction(const std::string & /*argument_name*/) {} - -void DefaultProjectionAction::preCalculation() {} - -bool DefaultProjectionAction::isCalculationRequired() -{ - return true; -} - -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) -{ -} - -std::string AndOperatorProjectionAction::getZerosColumnName() -{ - return "__inner_zeroes_column__" + expression_name; -} - -std::string AndOperatorProjectionAction::getFinalColumnName() -{ - return "__inner_final_column__" + expression_name; -} - -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(), zeros_column_name), - restore_projection_name, - true)); - } -} - -void AndOperatorProjectionAction::preArgumentAction() -{ - if (!previous_argument_name.empty()) - { - // Before processing arguments starting from second to last - if (auto * conditional_tree = typeid_cast(projection_manipulator.get())) - { - conditional_tree->goToProjection(previous_argument_name); - } - else - { - throw Exception( - "Illegal projection manipulator used in AndOperatorProjectionAction", ErrorCodes::ILLEGAL_PROJECTION_MANIPULATOR); - } - ++projection_levels_count; - } -} - -void AndOperatorProjectionAction::postArgumentAction(const std::string & argument_name) -{ - previous_argument_name = argument_name; -} - -void AndOperatorProjectionAction::preCalculation() -{ - if (auto * conditional_tree = typeid_cast(projection_manipulator.get())) - { - auto final_column = getFinalColumnName(); - const FunctionBuilderPtr & function_builder = FunctionFactory::instance().get("one_or_zero", context); - scopes.addAction(ExpressionAction::applyFunction(function_builder, - {projection_manipulator->getColumnName(previous_argument_name)}, - projection_manipulator->getColumnName(final_column), - 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); - } - else - { - throw Exception("Illegal projection manipulator used in AndOperatorProjectionAction", ErrorCodes::ILLEGAL_PROJECTION_MANIPULATOR); - } -} - -bool AndOperatorProjectionAction::isCalculationRequired() -{ - return false; -} - -ProjectionActionBase::~ProjectionActionBase() {} - -ProjectionActionPtr getProjectionAction(const std::string & node_name, - ScopeStack & scopes, - ProjectionManipulatorPtr projection_manipulator, - const std::string & expression_name, - const Context & context) -{ - if (typeid_cast(projection_manipulator.get()) && node_name == "and") - { - return std::make_shared(scopes, projection_manipulator, expression_name, context); - } - else - { - return std::make_shared(); - } -} - -} diff --git a/dbms/src/Interpreters/ProjectionManipulation.h b/dbms/src/Interpreters/ProjectionManipulation.h deleted file mode 100644 index b5529b5ee5c9..000000000000 --- a/dbms/src/Interpreters/ProjectionManipulation.h +++ /dev/null @@ -1,256 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - -class ExpressionAnalyzer; -class Context; -struct ScopeStack; - - -namespace ErrorCodes -{ - extern const int CONDITIONAL_TREE_PARENT_NOT_FOUND; - extern const int ILLEGAL_PROJECTION_MANIPULATOR; -} - -/* - * This is a base class for the ConditionalTree. Look at the description of ConditionalTree. - */ -struct ProjectionManipulatorBase -{ -public: - virtual bool tryToGetFromUpperProjection(const std::string & column_name) = 0; - - virtual std::string getColumnName(const std::string & col_name) const = 0; - - virtual std::string getProjectionExpression() = 0; - - virtual std::string getProjectionSourceColumn() const = 0; - - virtual ~ProjectionManipulatorBase(); -}; - -using ProjectionManipulatorPtr = std::shared_ptr; - -/* - * This is the default ProjectionManipulator. It is needed for backwards compatibility. - * For the better understanding of what ProjectionManipulator does, - * look at the description of ConditionalTree. - */ -struct DefaultProjectionManipulator : public ProjectionManipulatorBase -{ -private: - ScopeStack & scopes; - -public: - explicit DefaultProjectionManipulator(ScopeStack & scopes); - - bool tryToGetFromUpperProjection(const std::string & column_name) final; - - std::string getColumnName(const std::string & col_name) const final; - - std::string getProjectionExpression() final; - - std::string getProjectionSourceColumn() const final; -}; - -/* - * ConditionalTree is a projection manipulator. It is used in ExpressionAnalyzer::getActionsImpl. - * It is a helper class, which helps to build sequence of ExpressionAction instances -- actions, needed for - * computation of expression. It represents the current state of a projection layer. That is, if we have an expression - * f and g, we need to calculate f, afterwards we need to calculate g on the projection layer . - * This projection layer is stored in the ConditionalTree. Also, it stores the tree of all projection layers, which - * was seen before. If we have seen the projection layer and , conditional tree will put - * the second layer as a child to the first one. - * - * The description of what methods do: - * 1) getColumnName -- constructs the name of expression. which contains the information of the projection layer. - * It is needed to make computed column name unique. That is, if we have an expression g and conditional layer - * , it forms the name g - * - * 2) goToProjection -- accepts field name f and builds child projection layer with the additional condition - * . For instance, if we are on the projection layer a != 0 and the function accepts the expression b != 0, - * it will build a projection layer , and remember that this layer is a child to a previous one. - * Moreover, the function will store the actions to build projection between this two layers in the corresponding - * ScopeStack - * - * 3) restoreColumn(default_values_name, new_values_name, levels, result_name) -- stores action to restore calculated - * 'new_values_name' column, to insert its values to the projection layer, which is 'levels' number of levels higher. - * - * 4) goUp -- goes several levels up in the conditional tree, raises the exception if we hit the root of the tree and - * there are still remained some levels up to go. - * - * 5) tryToGetFromUpperProjection -- goes up to the root projection level and checks whether the expression is - * already calculated somewhere in the higher projection level. If it is, we may just project it to the current - * layer to have it computed in the current layer. In this case, the function stores all actions needed to compute - * the projection: computes composition of projections and uses it to project the column. In the other case, if - * the column is not computed on the higher level, the function returns false. It is used in getActinosImpl to - * understand whether we need to scan the expression deeply, or can it be easily computed just with the projection - * from one of the higher projection layers. - */ -struct ConditionalTree : public ProjectionManipulatorBase -{ -private: - struct Node - { - Node(); - - size_t getParentNode() const; - - std::string projection_expression_string; - size_t parent_node; - bool is_root; - }; - - size_t current_node; - std::vector nodes; - ScopeStack & scopes; - const Context & context; - std::unordered_map projection_expression_index; - -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 getProjectionColumnName(size_t first_index, size_t second_index) const; - - void buildProjectionCompositionRecursive(const std::vector & path, size_t child_index, size_t parent_index); - - void buildProjectionComposition(size_t child_node, size_t parent_node); - - std::string getProjectionSourceColumn(size_t node) const; - -public: - ConditionalTree(ScopeStack & scopes, const Context & context); - - std::string getColumnName(const std::string & col_name) const final; - - 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, size_t levels_up, const std::string & result_name); - - void goUp(size_t levels_up); - - bool tryToGetFromUpperProjection(const std::string & column_name) final; - - std::string getProjectionExpression() final; - - std::string getProjectionSourceColumn() const final; -}; - -using ConditionalTreePtr = std::shared_ptr; - -/* - * ProjectionAction describes in what way should some specific function use the projection manipulator. - * This class has two inherited classes: DefaultProjectionAction, which does nothing, and AndOperatorProjectionAction, - * which represents how function "and" uses projection manipulator. - */ -class ProjectionActionBase -{ -public: - /* - * What to do before scanning the function argument (each of it) - */ - virtual void preArgumentAction() = 0; - - /* - * What to do after scanning each argument - */ - virtual void postArgumentAction(const std::string & argument_name) = 0; - - /* - * What to do after scanning all the arguments, before the computation - */ - virtual void preCalculation() = 0; - - /* - * Should default computation procedure be run or not - */ - virtual bool isCalculationRequired() = 0; - - virtual ~ProjectionActionBase(); -}; - -using ProjectionActionPtr = std::shared_ptr; - -class DefaultProjectionAction : public ProjectionActionBase -{ -public: - void preArgumentAction() final; - - void postArgumentAction(const std::string & argument_name) final; - - void preCalculation() final; - - bool isCalculationRequired() final; -}; - -/* - * This is a specification of ProjectionAction specifically for the 'and' operation - */ -class AndOperatorProjectionAction : public ProjectionActionBase -{ -private: - ScopeStack & scopes; - ProjectionManipulatorPtr projection_manipulator; - std::string previous_argument_name; - size_t projection_levels_count; - std::string expression_name; - const Context & context; - - std::string getZerosColumnName(); - - std::string getFinalColumnName(); - - void createZerosColumn(const std::string & restore_projection_name); - -public: - AndOperatorProjectionAction( - ScopeStack & scopes, ProjectionManipulatorPtr projection_manipulator, const std::string & expression_name, const Context & context); - - /* - * Before scanning each argument, we should go to the next projection layer. For example, if the expression is - * f and g and h, then before computing g we should project to and before computing h we should project to - * - */ - void preArgumentAction() final; - - /* - * Stores the previous argument name - */ - void postArgumentAction(const std::string & argument_name) final; - - /* - * Restores the result column to the uppermost projection level. For example, if the expression is f and g and h, - * we should restore h to the main projection layer - */ - void preCalculation() final; - - /* - * After what is done in preCalculation, we do not need to run default calculation of 'and' operator. So, the - * function returns false. - */ - bool isCalculationRequired() final; -}; - -/* - * This function accepts the operator name and returns its projection action. For example, for 'and' operator, - * it returns the pointer to AndOperatorProjectionAction. - */ -ProjectionActionPtr getProjectionAction(const std::string & node_name, - ScopeStack & scopes, - ProjectionManipulatorPtr projection_manipulator, - const std::string & expression_name, - const Context & context); - -} diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index c1436267ffcb..77ba93ec1748 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -272,7 +272,6 @@ struct Settings M(SettingBool, format_csv_allow_single_quotes, 1, "If it is set to true, allow strings in single quotes.") \ M(SettingBool, format_csv_allow_double_quotes, 1, "If it is set to true, allow strings in double quotes.") \ \ - M(SettingUInt64, enable_conditional_computation, 0, "Enable conditional computations") \ M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \ M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.") \ M(SettingBool, log_query_settings, true, "Log query settings into the query_log.") \ diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 05eb90611e05..af8b78c7a015 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -1021,7 +1021,7 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name /// This is temporary name for expression. TODO Invent the name more safely. const String new_type_name_column = '#' + new_type_name + "_column"; out_expression->add(ExpressionAction::addColumn( - { DataTypeString().createColumnConst(1, new_type_name), std::make_shared(), new_type_name_column }, "", false)); + { DataTypeString().createColumnConst(1, new_type_name), std::make_shared(), new_type_name_column })); const auto & function = FunctionFactory::instance().get("CAST", context); out_expression->add(ExpressionAction::applyFunction( diff --git a/dbms/tests/queries/0_stateless/00631_conditional_computation.reference b/dbms/tests/queries/0_stateless/00631_conditional_computation.reference deleted file mode 100644 index ea763551e39a..000000000000 --- a/dbms/tests/queries/0_stateless/00631_conditional_computation.reference +++ /dev/null @@ -1,4 +0,0 @@ -0 0 0 0 0 0 0 -0 5 0 0 0 0 0 -6 3 1 1 1 1 0 -7 0 0 0 0 0 0 diff --git a/dbms/tests/queries/0_stateless/00631_conditional_computation.sql b/dbms/tests/queries/0_stateless/00631_conditional_computation.sql deleted file mode 100644 index 70438c766edf..000000000000 --- a/dbms/tests/queries/0_stateless/00631_conditional_computation.sql +++ /dev/null @@ -1,23 +0,0 @@ -USE test; - -DROP TABLE IF EXISTS test; -CREATE TABLE test (d Date DEFAULT '2000-01-01', x UInt64, y UInt64) ENGINE = MergeTree(d, x, 1); -INSERT INTO test(x,y) VALUES (6, 3); -INSERT INTO test(x,y) VALUES (0, 5); -INSERT INTO test(x,y) VALUES (7, 0); -INSERT INTO test(x,y) VALUES (0, 0); -SET enable_conditional_computation=1; -SELECT - x, - y, - x and y, - y and x, - x and 1 and x and y, - x and modulo(y, x), - y and modulo(x,y) -FROM - test -ORDER BY - x, y -; -SET enable_conditional_computation=0; diff --git a/dbms/tests/queries/bugs/fuzzy.sql b/dbms/tests/queries/bugs/fuzzy.sql index 5712498a555c..f81140ba8c93 100644 --- a/dbms/tests/queries/bugs/fuzzy.sql +++ b/dbms/tests/queries/bugs/fuzzy.sql @@ -1,6 +1,3 @@ -SELECT __inner_restore_projection__(2.0885, -66.72488); -SELECT __inner_restore_projection__(-4, ''); -SELECT __inner_restore_projection__(067274, 'vb\s'); SELECT sequenceCount((CAST((( SELECT NULL ) AS rg, ( SELECT ( SELECT [], '