mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-12 10:34:21 +00:00
Merge branch 'function_result_with_operator_in' of https://github.com/Enmk/ClickHouse into merging-in-arrays
This commit is contained in:
commit
5162ece940
@ -85,21 +85,7 @@ SetPtr makeExplicitSet(
|
|||||||
if (prepared_sets.count(set_key))
|
if (prepared_sets.count(set_key))
|
||||||
return prepared_sets.at(set_key); /// Already prepared.
|
return prepared_sets.at(set_key); /// Already prepared.
|
||||||
|
|
||||||
auto getTupleTypeFromAst = [&context](const ASTPtr & tuple_ast) -> DataTypePtr
|
const auto right_arg_evaluated = evaluateConstantExpression(right_arg, context);
|
||||||
{
|
|
||||||
const auto * func = tuple_ast->as<ASTFunction>();
|
|
||||||
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<Field, DataTypePtr> value_raw = evaluateConstantExpression(element, context);
|
|
||||||
return std::make_shared<DataTypeTuple>(DataTypes({value_raw.second}));
|
|
||||||
}
|
|
||||||
|
|
||||||
return evaluateConstantExpression(tuple_ast, context).second;
|
|
||||||
};
|
|
||||||
|
|
||||||
const DataTypePtr & right_arg_type = getTupleTypeFromAst(right_arg);
|
|
||||||
|
|
||||||
std::function<size_t(const DataTypePtr &)> getTupleDepth;
|
std::function<size_t(const DataTypePtr &)> getTupleDepth;
|
||||||
getTupleDepth = [&getTupleDepth](const DataTypePtr & type) -> size_t
|
getTupleDepth = [&getTupleDepth](const DataTypePtr & type) -> size_t
|
||||||
@ -110,37 +96,77 @@ SetPtr makeExplicitSet(
|
|||||||
return 0;
|
return 0;
|
||||||
};
|
};
|
||||||
|
|
||||||
size_t left_tuple_depth = getTupleDepth(left_arg_type);
|
const auto& right_arg_type = right_arg_evaluated.second;
|
||||||
size_t right_tuple_depth = getTupleDepth(right_arg_type);
|
const auto& right_arg_value = right_arg_evaluated.first;
|
||||||
|
|
||||||
ASTPtr elements_ast = nullptr;
|
const size_t left_tuple_depth = getTupleDepth(left_arg_type);
|
||||||
|
const size_t right_tuple_depth = getTupleDepth(right_arg_type);
|
||||||
/// 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 && left_tuple_depth + 1 != right_tuple_depth)
|
||||||
if (left_tuple_depth == right_tuple_depth)
|
|
||||||
{
|
{
|
||||||
ASTPtr exp_list = std::make_shared<ASTExpressionList>();
|
|
||||||
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<ASTFunction>();
|
|
||||||
|
|
||||||
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: "
|
throw Exception("Invalid types for IN function: "
|
||||||
+ left_arg_type->getName() + " and " + right_arg_type->getName() + ".",
|
+ left_arg_type->getName() + " and " + right_arg_type->getName() + ".",
|
||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
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<const Array &>(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<Set>(size_limits, create_ordered_set);
|
SetPtr set = std::make_shared<Set>(size_limits, create_ordered_set);
|
||||||
set->createFromAST(set_element_types, elements_ast, context);
|
|
||||||
|
set->setHeader(block);
|
||||||
|
set->insertFromBlock(block);
|
||||||
|
|
||||||
prepared_sets[set_key] = set;
|
prepared_sets[set_key] = set;
|
||||||
return set;
|
return set;
|
||||||
}
|
}
|
||||||
|
@ -193,8 +193,10 @@ void ASTFunction::formatImplWithoutAlias(const FormatSettings & settings, Format
|
|||||||
const auto * second_arg_func = arguments->children[1]->as<ASTFunction>();
|
const auto * second_arg_func = arguments->children[1]->as<ASTFunction>();
|
||||||
const auto * second_arg_literal = arguments->children[1]->as<ASTLiteral>();
|
const auto * second_arg_literal = arguments->children[1]->as<ASTLiteral>();
|
||||||
bool extra_parents_around_in_rhs = (name == "in" || name == "notIn" || name == "globalIn" || name == "globalNotIn")
|
bool extra_parents_around_in_rhs = (name == "in" || name == "notIn" || name == "globalIn" || name == "globalNotIn")
|
||||||
&& !(second_arg_func && second_arg_func->name == "tuple")
|
&& !second_arg_func
|
||||||
&& !(second_arg_literal && second_arg_literal->value.getType() == Field::Types::Tuple)
|
&& !(second_arg_literal
|
||||||
|
&& (second_arg_literal->value.getType() == Field::Types::Tuple
|
||||||
|
|| second_arg_literal->value.getType() == Field::Types::Array))
|
||||||
&& !arguments->children[1]->as<ASTSubquery>();
|
&& !arguments->children[1]->as<ASTSubquery>();
|
||||||
|
|
||||||
if (extra_parents_around_in_rhs)
|
if (extra_parents_around_in_rhs)
|
||||||
|
@ -21,6 +21,7 @@ public:
|
|||||||
std::optional<TokenIterator> begin;
|
std::optional<TokenIterator> begin;
|
||||||
std::optional<TokenIterator> end;
|
std::optional<TokenIterator> end;
|
||||||
|
|
||||||
|
ASTLiteral(Field && value_) : value(value_) {}
|
||||||
ASTLiteral(const Field & value_) : value(value_) {}
|
ASTLiteral(const Field & value_) : value(value_) {}
|
||||||
|
|
||||||
/** Get the text that identifies this element. */
|
/** Get the text that identifies this element. */
|
||||||
|
@ -0,0 +1,7 @@
|
|||||||
|
5
|
||||||
|
2
|
||||||
|
5
|
||||||
|
empty:
|
||||||
|
0
|
||||||
|
0
|
||||||
|
errors:
|
@ -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;
|
@ -1,13 +1,13 @@
|
|||||||
SELECT 1 IN (1)
|
SELECT 1 IN (1)
|
||||||
SELECT 1 IN (1)
|
SELECT 1 IN (1)
|
||||||
SELECT 1 IN (1, 2)
|
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), f(2))
|
||||||
SELECT 1 IN (f(1, 2))
|
SELECT 1 IN f(1, 2)
|
||||||
SELECT 1 IN (1 + 1)
|
SELECT 1 IN (1 + 1)
|
||||||
SELECT 1 IN ('hello')
|
SELECT 1 IN ('hello')
|
||||||
SELECT 1 IN (f('hello'))
|
SELECT 1 IN f('hello')
|
||||||
SELECT 1 IN ('hello', 'world')
|
SELECT 1 IN ('hello', 'world')
|
||||||
SELECT 1 IN (f('hello', 'world'))
|
SELECT 1 IN f('hello', 'world')
|
||||||
SELECT 1 IN (SELECT 1)
|
SELECT 1 IN (SELECT 1)
|
||||||
|
Loading…
Reference in New Issue
Block a user