diff --git a/dbms/src/Interpreters/ActionsVisitor.cpp b/dbms/src/Interpreters/ActionsVisitor.cpp index 9fe6e218746..39f2a2bb790 100644 --- a/dbms/src/Interpreters/ActionsVisitor.cpp +++ b/dbms/src/Interpreters/ActionsVisitor.cpp @@ -85,21 +85,7 @@ SetPtr makeExplicitSet( if (prepared_sets.count(set_key)) return prepared_sets.at(set_key); /// Already prepared. - auto getTupleTypeFromAst = [&context](const ASTPtr & tuple_ast) -> DataTypePtr - { - 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); + const auto right_arg_evaluated = evaluateConstantExpression(right_arg, context); std::function getTupleDepth; getTupleDepth = [&getTupleDepth](const DataTypePtr & type) -> size_t @@ -110,37 +96,77 @@ SetPtr makeExplicitSet( return 0; }; - size_t left_tuple_depth = getTupleDepth(left_arg_type); - size_t right_tuple_depth = getTupleDepth(right_arg_type); + const auto& right_arg_type = right_arg_evaluated.second; + const auto& right_arg_value = right_arg_evaluated.first; - 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) + const size_t left_tuple_depth = getTupleDepth(left_arg_type); + const size_t right_tuple_depth = getTupleDepth(right_arg_type); + if (left_tuple_depth != right_tuple_depth && left_tuple_depth + 1 != right_tuple_depth) { - ASTPtr exp_list = std::make_shared(); - exp_list->children.push_back(right_arg); - elements_ast = exp_list; - } - /// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)); etc. - else if (left_tuple_depth + 1 == right_tuple_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; - } - else throw Exception("Invalid types for IN function: " + left_arg_type->getName() + " and " + right_arg_type->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + + Block block; + auto col = left_arg_type->createColumn(); + switch (right_arg_type->getTypeId()) + { + case TypeIndex::UInt8: [[fallthrough]]; + case TypeIndex::UInt16: [[fallthrough]]; + case TypeIndex::UInt32: [[fallthrough]]; + case TypeIndex::UInt64: [[fallthrough]]; + case TypeIndex::UInt128: [[fallthrough]]; + case TypeIndex::Int8: [[fallthrough]]; + case TypeIndex::Int16: [[fallthrough]]; + case TypeIndex::Int32: [[fallthrough]]; + case TypeIndex::Int64: [[fallthrough]]; + case TypeIndex::Int128: [[fallthrough]]; + case TypeIndex::Float32: [[fallthrough]]; + case TypeIndex::Float64: [[fallthrough]]; + case TypeIndex::Date: [[fallthrough]]; + case TypeIndex::DateTime: [[fallthrough]]; + case TypeIndex::String: [[fallthrough]]; + case TypeIndex::FixedString: [[fallthrough]]; + case TypeIndex::Enum8: [[fallthrough]]; + case TypeIndex::Enum16: [[fallthrough]]; + case TypeIndex::Decimal32: [[fallthrough]]; + case TypeIndex::Decimal64: [[fallthrough]]; + case TypeIndex::Decimal128: [[fallthrough]]; + case TypeIndex::UUID: + { + col->insert(convertFieldToType(right_arg_value, *left_arg_type, right_arg_type.get())); + break; + } + // flatten compound values: + case TypeIndex::Array: [[fallthrough]]; + case TypeIndex::Tuple: [[fallthrough]]; + case TypeIndex::Set: + { + const Array & array = DB::get(right_arg_value); + if (array.size() == 0) + break; + + for (size_t i = 0 ; i < array.size(); ++i) + { + col->insert(convertFieldToType(array[i], *left_arg_type, nullptr)); + } + break; + } + default: + throw Exception("Unsupported value type at the right-side of IN:" + + right_arg_type->getName() + ".", + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + block.insert(ColumnWithTypeAndName{std::move(col), + left_arg_type, + "dummy_" + left_arg_type->getName()}); 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/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..fe0f0bc3a1f --- /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()); -- { 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)