mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #8813 from CurtizJ/merging-in-arrays
Merging #5342 (IN with function result).
This commit is contained in:
commit
c1535e10d4
@ -1,5 +1,6 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/PODArray.h>
|
||||
#include <Core/Row.h>
|
||||
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/FunctionsMiscellaneous.h>
|
||||
@ -11,6 +12,7 @@
|
||||
#include <DataTypes/DataTypeFunction.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/FieldToDataType.h>
|
||||
|
||||
@ -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<typename Collection>
|
||||
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<const Tuple &>(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<size_t(const DataTypePtr &)> getTypeDepth;
|
||||
getTypeDepth = [&getTypeDepth](const DataTypePtr & type) -> size_t
|
||||
{
|
||||
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;
|
||||
getTupleDepth = [&getTupleDepth](const DataTypePtr & type) -> size_t
|
||||
{
|
||||
if (auto tuple_type = typeid_cast<const DataTypeTuple *>(type.get()))
|
||||
return 1 + (tuple_type->getElements().empty() ? 0 : getTupleDepth(tuple_type->getElements().at(0)));
|
||||
if (auto array_type = typeid_cast<const DataTypeArray *>(type.get()))
|
||||
return 1 + getTypeDepth(array_type->getNestedType());
|
||||
else if (auto tuple_type = typeid_cast<const DataTypeTuple *>(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<ASTExpressionList>();
|
||||
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<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;
|
||||
auto type_index = right_arg_type->getTypeId();
|
||||
if (type_index == TypeIndex::Tuple)
|
||||
block = createBlockFromCollection(DB::get<const Tuple &>(right_arg_value), set_element_types);
|
||||
else if (type_index == TypeIndex::Array)
|
||||
block = createBlockFromCollection(DB::get<const Array &>(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<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;
|
||||
return set;
|
||||
}
|
||||
|
@ -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.
|
||||
|
@ -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);
|
||||
|
||||
|
@ -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_literal = arguments->children[1]->as<ASTLiteral>();
|
||||
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<ASTSubquery>();
|
||||
|
||||
if (extra_parents_around_in_rhs)
|
||||
|
@ -21,6 +21,7 @@ public:
|
||||
std::optional<TokenIterator> begin;
|
||||
std::optional<TokenIterator> end;
|
||||
|
||||
ASTLiteral(Field && value_) : value(value_) {}
|
||||
ASTLiteral(const Field & value_) : value(value_) {}
|
||||
|
||||
/** 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() % 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;
|
@ -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)
|
||||
|
8
dbms/tests/queries/0_stateless/01071_in_array.reference
Normal file
8
dbms/tests/queries/0_stateless/01071_in_array.reference
Normal file
@ -0,0 +1,8 @@
|
||||
1
|
||||
1
|
||||
1
|
||||
1
|
||||
\N
|
||||
0
|
||||
1
|
||||
1
|
8
dbms/tests/queries/0_stateless/01071_in_array.sql
Normal file
8
dbms/tests/queries/0_stateless/01071_in_array.sql
Normal file
@ -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])];
|
Loading…
Reference in New Issue
Block a user