diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index 4bef41f3c3e..92078040ee9 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include @@ -11,6 +12,7 @@ #include #include #include +#include #include #include @@ -50,6 +52,7 @@ namespace ErrorCodes extern const int UNEXPECTED_EXPRESSION; extern const int TYPE_MISMATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int INCORRECT_ELEMENT_OF_SET; } static NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypesList & cols) @@ -58,6 +61,59 @@ static NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypes [&](const NamesAndTypesList::value_type & val) { return val.name == name; }); } +template +static Block createBlockFromCollection(const Collection & collection, const DataTypes & types) +{ + size_t columns_num = types.size(); + MutableColumns columns(columns_num); + for (size_t i = 0; i < columns_num; ++i) + columns[i] = types[i]->createColumn(); + + Row tuple_values; + for (const auto & value : collection) + { + if (columns_num == 1) + { + auto field = convertFieldToType(value, *types[0]); + if (!field.isNull()) + columns[0]->insert(std::move(field)); + } + else + { + if (value.getType() != Field::Types::Tuple) + throw Exception("Invalid type in set. Expected tuple, got " + + String(value.getTypeName()), ErrorCodes::INCORRECT_ELEMENT_OF_SET); + + const auto & tuple = DB::get(value); + size_t tuple_size = tuple.size(); + + if (tuple_size != columns_num) + throw Exception("Incorrect size of tuple in set: " + toString(tuple_size) + + " instead of " + toString(columns_num), ErrorCodes::INCORRECT_ELEMENT_OF_SET); + + if (tuple_values.empty()) + tuple_values.resize(tuple_size); + + size_t i = 0; + for (; i < tuple_size; ++i) + { + tuple_values[i] = convertFieldToType(tuple[i], *types[i]); + if (tuple_values[i].isNull()) + break; + } + + if (i == tuple_size) + for (i = 0; i < tuple_size; ++i) + columns[i]->insert(std::move(tuple_values[i])); + } + } + + Block res; + for (size_t i = 0; i < columns_num; ++i) + res.insert(ColumnWithTypeAndName{std::move(columns[i]), types[i], "_" + toString(i)}); + return res; +} + SetPtr makeExplicitSet( const ASTFunction * node, const Block & sample_block, bool create_ordered_set, const Context & context, const SizeLimits & size_limits, PreparedSets & prepared_sets) @@ -85,62 +141,54 @@ SetPtr makeExplicitSet( if (prepared_sets.count(set_key)) return prepared_sets.at(set_key); /// Already prepared. - auto getTupleTypeFromAst = [&context](const ASTPtr & tuple_ast) -> DataTypePtr + auto [right_arg_value, right_arg_type] = evaluateConstantExpression(right_arg, context); + + std::function getTypeDepth; + getTypeDepth = [&getTypeDepth](const DataTypePtr & type) -> size_t { - const auto * func = tuple_ast->as(); - if (func && func->name == "tuple" && !func->arguments->children.empty()) - { - /// Won't parse all values of outer tuple. - auto element = func->arguments->children.at(0); - std::pair value_raw = evaluateConstantExpression(element, context); - return std::make_shared(DataTypes({value_raw.second})); - } - - return evaluateConstantExpression(tuple_ast, context).second; - }; - - const DataTypePtr & right_arg_type = getTupleTypeFromAst(right_arg); - - std::function getTupleDepth; - getTupleDepth = [&getTupleDepth](const DataTypePtr & type) -> size_t - { - if (auto tuple_type = typeid_cast(type.get())) - return 1 + (tuple_type->getElements().empty() ? 0 : getTupleDepth(tuple_type->getElements().at(0))); + if (auto array_type = typeid_cast(type.get())) + return 1 + getTypeDepth(array_type->getNestedType()); + else if (auto tuple_type = typeid_cast(type.get())) + return 1 + (tuple_type->getElements().empty() ? 0 : getTypeDepth(tuple_type->getElements().at(0))); return 0; }; - size_t left_tuple_depth = getTupleDepth(left_arg_type); - size_t right_tuple_depth = getTupleDepth(right_arg_type); + const size_t left_type_depth = getTypeDepth(left_arg_type); + const size_t right_type_depth = getTypeDepth(right_arg_type); - ASTPtr elements_ast = nullptr; - - /// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc. - if (left_tuple_depth == right_tuple_depth) + auto throw_unsupported_type = [](const auto & type) { - ASTPtr exp_list = std::make_shared(); - exp_list->children.push_back(right_arg); - elements_ast = exp_list; + throw Exception("Unsupported value type at the right-side of IN: " + + type->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + }; + + Block block; + /// 1 in 1; (1, 2) in (1, 2); identity(tuple(tuple(tuple(1)))) in tuple(tuple(tuple(1))); etc. + if (left_type_depth == right_type_depth) + { + Array array{right_arg_value}; + block = createBlockFromCollection(array, set_element_types); } /// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)); etc. - else if (left_tuple_depth + 1 == right_tuple_depth) + else if (left_type_depth + 1 == right_type_depth) { - const auto * set_func = right_arg->as(); - - if (!set_func || set_func->name != "tuple") - throw Exception("Incorrect type of 2nd argument for function " + node->name - + ". Must be subquery or set of elements with type " + left_arg_type->getName() + ".", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - elements_ast = set_func->arguments; + auto type_index = right_arg_type->getTypeId(); + if (type_index == TypeIndex::Tuple) + block = createBlockFromCollection(DB::get(right_arg_value), set_element_types); + else if (type_index == TypeIndex::Array) + block = createBlockFromCollection(DB::get(right_arg_value), set_element_types); + else + throw_unsupported_type(right_arg_type); } else - throw Exception("Invalid types for IN function: " - + left_arg_type->getName() + " and " + right_arg_type->getName() + ".", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw_unsupported_type(right_arg_type); SetPtr set = std::make_shared(size_limits, create_ordered_set); - set->createFromAST(set_element_types, elements_ast, context); + + set->setHeader(block); + set->insertFromBlock(block); + prepared_sets[set_key] = set; return set; } diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index 6be4f4e8897..23bdcf10729 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -228,7 +228,6 @@ static Field extractValueFromNode(const ASTPtr & node, const IDataType & type, c throw Exception("Incorrect element of set. Must be literal or constant expression.", ErrorCodes::INCORRECT_ELEMENT_OF_SET); } - void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & context) { /// Will form a block with values from the set. diff --git a/dbms/src/Interpreters/Set.h b/dbms/src/Interpreters/Set.h index 3c2ab1742a8..9d79017adbd 100644 --- a/dbms/src/Interpreters/Set.h +++ b/dbms/src/Interpreters/Set.h @@ -45,7 +45,6 @@ public: /** Create a Set from expression (specified literally in the query). * 'types' - types of what are on the left hand side of IN. * 'node' - list of values: 1, 2, 3 or list of tuples: (1, 2), (3, 4), (5, 6). - * 'fill_set_elements' - if true, fill vector of elements. For primary key to work. */ void createFromAST(const DataTypes & types, ASTPtr node, const Context & context); diff --git a/dbms/src/Parsers/ASTFunction.cpp b/dbms/src/Parsers/ASTFunction.cpp index b550c7062d1..988ac66916d 100644 --- a/dbms/src/Parsers/ASTFunction.cpp +++ b/dbms/src/Parsers/ASTFunction.cpp @@ -193,8 +193,10 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format const auto * second_arg_func = arguments->children[1]->as(); const auto * second_arg_literal = arguments->children[1]->as(); bool extra_parents_around_in_rhs = (name == "in" || name == "notIn" || name == "globalIn" || name == "globalNotIn") - && !(second_arg_func && second_arg_func->name == "tuple") - && !(second_arg_literal && second_arg_literal->value.getType() == Field::Types::Tuple) + && !second_arg_func + && !(second_arg_literal + && (second_arg_literal->value.getType() == Field::Types::Tuple + || second_arg_literal->value.getType() == Field::Types::Array)) && !arguments->children[1]->as(); if (extra_parents_around_in_rhs) diff --git a/dbms/src/Parsers/ASTLiteral.h b/dbms/src/Parsers/ASTLiteral.h index 416bc118151..552f5da04a2 100644 --- a/dbms/src/Parsers/ASTLiteral.h +++ b/dbms/src/Parsers/ASTLiteral.h @@ -21,6 +21,7 @@ public: std::optional begin; std::optional end; + ASTLiteral(Field && value_) : value(value_) {} ASTLiteral(const Field & value_) : value(value_) {} /** Get the text that identifies this element. */ diff --git a/dbms/tests/queries/0_stateless/00936_function_result_with_operator_in.reference b/dbms/tests/queries/0_stateless/00936_function_result_with_operator_in.reference new file mode 100644 index 00000000000..7c650247cf4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00936_function_result_with_operator_in.reference @@ -0,0 +1,7 @@ +5 +2 +5 +empty: +0 +0 +errors: diff --git a/dbms/tests/queries/0_stateless/00936_function_result_with_operator_in.sql b/dbms/tests/queries/0_stateless/00936_function_result_with_operator_in.sql new file mode 100644 index 00000000000..dfb19c1f3ec --- /dev/null +++ b/dbms/tests/queries/0_stateless/00936_function_result_with_operator_in.sql @@ -0,0 +1,34 @@ +SET force_primary_key = 1; + +DROP TABLE IF EXISTS samples; +CREATE TABLE samples (key UInt32, value UInt32) ENGINE = MergeTree() ORDER BY key PRIMARY KEY key; +INSERT INTO samples VALUES (1, 1)(2, 2)(3, 3)(4, 4)(5, 5); + +-- all etries, verify that index is used +SELECT count() FROM samples WHERE key IN range(10); + +-- some entries: +SELECT count() FROM samples WHERE key IN arraySlice(range(100), 5, 10); + +-- different type +SELECT count() FROM samples WHERE toUInt64(key) IN range(100); + +SELECT 'empty:'; +-- should be empty +SELECT count() FROM samples WHERE key IN arraySlice(range(100), 10, 10); + +-- not only ints: +SELECT 'a' IN splitByChar('c', 'abcdef'); + +SELECT 'errors:'; +-- non-constant expressions in the right side of IN +SELECT count() FROM samples WHERE 1 IN range(samples.value); -- { serverError 47 } +SELECT count() FROM samples WHERE 1 IN range(rand() % 1000); -- { serverError 36 } + +-- index is not used +SELECT count() FROM samples WHERE value IN range(3); -- { serverError 277 } + +-- wrong type +SELECT 123 IN splitByChar('c', 'abcdef'); -- { serverError 53 } + +DROP TABLE samples; \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/00948_format_in_with_single_element.reference b/dbms/tests/queries/0_stateless/00948_format_in_with_single_element.reference index 91f53ffadb0..e843bb2b350 100644 --- a/dbms/tests/queries/0_stateless/00948_format_in_with_single_element.reference +++ b/dbms/tests/queries/0_stateless/00948_format_in_with_single_element.reference @@ -1,13 +1,13 @@ SELECT 1 IN (1) SELECT 1 IN (1) SELECT 1 IN (1, 2) -SELECT 1 IN (f(1)) -SELECT 1 IN (f(1)) +SELECT 1 IN f(1) +SELECT 1 IN f(1) SELECT 1 IN (f(1), f(2)) -SELECT 1 IN (f(1, 2)) +SELECT 1 IN f(1, 2) SELECT 1 IN (1 + 1) SELECT 1 IN ('hello') -SELECT 1 IN (f('hello')) +SELECT 1 IN f('hello') SELECT 1 IN ('hello', 'world') -SELECT 1 IN (f('hello', 'world')) +SELECT 1 IN f('hello', 'world') SELECT 1 IN (SELECT 1) diff --git a/dbms/tests/queries/0_stateless/01071_in_array.reference b/dbms/tests/queries/0_stateless/01071_in_array.reference new file mode 100644 index 00000000000..a561412c1f2 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01071_in_array.reference @@ -0,0 +1,8 @@ +1 +1 +1 +1 +\N +0 +1 +1 diff --git a/dbms/tests/queries/0_stateless/01071_in_array.sql b/dbms/tests/queries/0_stateless/01071_in_array.sql new file mode 100644 index 00000000000..1f2406605c8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01071_in_array.sql @@ -0,0 +1,8 @@ +select [1, 2] in [1, 2]; +select (1, 2) in (1, 2); +select (1, 2) in [(1, 3), (1, 2)]; +select [1] in [[1], [2, 3]]; +select NULL in NULL; +select ([1], [2]) in ([NULL], [NULL]); +select ([1], [2]) in (([NULL], [NULL]), ([1], [2])); +select ([1], [2]) in [([NULL], [NULL]), ([1], [2])];