mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
fix in operator with constant arrays
This commit is contained in:
parent
5162ece940
commit
d8331d9dcb
@ -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,58 @@ static NamesAndTypesList::iterator findColumn(const String & name, NamesAndTypes
|
||||
[&](const NamesAndTypesList::value_type & val) { return val.name == name; });
|
||||
}
|
||||
|
||||
static Block createBlockFromArray(const Array & array, 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 : array)
|
||||
{
|
||||
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)
|
||||
@ -87,80 +142,42 @@ SetPtr makeExplicitSet(
|
||||
|
||||
const auto right_arg_evaluated = evaluateConstantExpression(right_arg, context);
|
||||
|
||||
std::function<size_t(const DataTypePtr &)> getTupleDepth;
|
||||
getTupleDepth = [&getTupleDepth](const DataTypePtr & type) -> size_t
|
||||
std::function<size_t(const DataTypePtr &)> getTypeDepth;
|
||||
getTypeDepth = [&getTypeDepth](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;
|
||||
};
|
||||
|
||||
const auto& right_arg_type = right_arg_evaluated.second;
|
||||
const auto& right_arg_value = right_arg_evaluated.first;
|
||||
const auto & right_arg_type = right_arg_evaluated.second;
|
||||
const auto & right_arg_value = right_arg_evaluated.first;
|
||||
|
||||
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)
|
||||
{
|
||||
throw Exception("Invalid types for IN function: "
|
||||
+ left_arg_type->getName() + " and " + right_arg_type->getName() + ".",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
const size_t left_type_depth = getTypeDepth(left_arg_type);
|
||||
const size_t right_type_depth = getTypeDepth(right_arg_type);
|
||||
|
||||
Block block;
|
||||
auto col = left_arg_type->createColumn();
|
||||
switch (right_arg_type->getTypeId())
|
||||
/// 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)
|
||||
{
|
||||
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);
|
||||
Array array{right_arg_value};
|
||||
block = createBlockFromArray(array, set_element_types);
|
||||
}
|
||||
/// 1 in (1, 2); (1, 2) in ((1, 2), (3, 4)); etc.
|
||||
else if (left_type_depth + 1 == right_type_depth)
|
||||
{
|
||||
const Array & array = DB::safeGet<const Array &>(right_arg_value);
|
||||
block = createBlockFromArray(array, set_element_types);
|
||||
}
|
||||
else
|
||||
{
|
||||
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);
|
||||
|
||||
|
@ -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);
|
||||
|
||||
|
@ -23,7 +23,7 @@ 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 }
|
||||
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 }
|
||||
|
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