mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
Merge pull request #21157 from CurtizJ/fix-type-map
Improve support of integer keys in data type Map
This commit is contained in:
commit
5ac5ae3b27
@ -29,6 +29,7 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int CANNOT_READ_MAP_FROM_TEXT;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
|
||||
@ -38,6 +39,8 @@ DataTypeMap::DataTypeMap(const DataTypes & elems_)
|
||||
key_type = elems_[0];
|
||||
value_type = elems_[1];
|
||||
|
||||
assertKeyType();
|
||||
|
||||
nested = std::make_shared<DataTypeArray>(
|
||||
std::make_shared<DataTypeTuple>(DataTypes{key_type, value_type}, Names{"keys", "values"}));
|
||||
}
|
||||
@ -45,7 +48,19 @@ DataTypeMap::DataTypeMap(const DataTypes & elems_)
|
||||
DataTypeMap::DataTypeMap(const DataTypePtr & key_type_, const DataTypePtr & value_type_)
|
||||
: key_type(key_type_), value_type(value_type_)
|
||||
, nested(std::make_shared<DataTypeArray>(
|
||||
std::make_shared<DataTypeTuple>(DataTypes{key_type_, value_type_}, Names{"keys", "values"}))) {}
|
||||
std::make_shared<DataTypeTuple>(DataTypes{key_type_, value_type_}, Names{"keys", "values"})))
|
||||
{
|
||||
assertKeyType();
|
||||
}
|
||||
|
||||
void DataTypeMap::assertKeyType() const
|
||||
{
|
||||
if (!key_type->isValueRepresentedByInteger() && !isStringOrFixedString(*key_type) && !WhichDataType(key_type).isNothing())
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"Type of Map key must be a type, that can be represented by integer or string,"
|
||||
" but {} given", key_type->getName());
|
||||
}
|
||||
|
||||
|
||||
std::string DataTypeMap::doGetName() const
|
||||
{
|
||||
|
@ -97,6 +97,8 @@ private:
|
||||
|
||||
template <typename Reader>
|
||||
void deserializeTextImpl(IColumn & column, ReadBuffer & istr, bool need_safe_get_int_key, Reader && reader) const;
|
||||
|
||||
void assertKeyType() const;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -212,13 +212,13 @@ MutableColumnPtr DataTypeNumberBase<T>::createColumn() const
|
||||
template <typename T>
|
||||
bool DataTypeNumberBase<T>::isValueRepresentedByInteger() const
|
||||
{
|
||||
return std::is_integral_v<T>;
|
||||
return is_integer_v<T>;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool DataTypeNumberBase<T>::isValueRepresentedByUnsignedInteger() const
|
||||
{
|
||||
return std::is_integral_v<T> && is_unsigned_v<T>;
|
||||
return is_integer_v<T> && is_unsigned_v<T>;
|
||||
}
|
||||
|
||||
|
||||
|
@ -8,6 +8,7 @@
|
||||
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
@ -162,6 +163,36 @@ DataTypePtr getLeastSupertype(const DataTypes & types)
|
||||
}
|
||||
}
|
||||
|
||||
/// For maps
|
||||
{
|
||||
bool have_maps = false;
|
||||
bool all_maps = true;
|
||||
DataTypes key_types;
|
||||
DataTypes value_types;
|
||||
key_types.reserve(types.size());
|
||||
value_types.reserve(types.size());
|
||||
|
||||
for (const auto & type : types)
|
||||
{
|
||||
if (const DataTypeMap * type_map = typeid_cast<const DataTypeMap *>(type.get()))
|
||||
{
|
||||
have_maps = true;
|
||||
key_types.emplace_back(type_map->getKeyType());
|
||||
value_types.emplace_back(type_map->getValueType());
|
||||
}
|
||||
else
|
||||
all_maps = false;
|
||||
}
|
||||
|
||||
if (have_maps)
|
||||
{
|
||||
if (!all_maps)
|
||||
throw Exception(getExceptionMessagePrefix(types) + " because some of them are Maps and some of them are not", ErrorCodes::NO_COMMON_TYPE);
|
||||
|
||||
return std::make_shared<DataTypeMap>(getLeastSupertype(key_types), getLeastSupertype(value_types));
|
||||
}
|
||||
}
|
||||
|
||||
/// For LowCardinality. This is above Nullable, because LowCardinality can contain Nullable but cannot be inside Nullable.
|
||||
{
|
||||
bool have_low_cardinality = false;
|
||||
|
@ -872,7 +872,12 @@ bool FunctionArrayElement::matchKeyToIndexNumberConst(
|
||||
if (!data_numeric)
|
||||
return false;
|
||||
|
||||
if (index.getType() != Field::Types::UInt64 && index.getType() != Field::Types::Int64)
|
||||
bool is_integer_field = Field::dispatch([](const auto & value)
|
||||
{
|
||||
return is_integer_v<std::decay_t<decltype(value)>>;
|
||||
}, index);
|
||||
|
||||
if (!is_integer_field)
|
||||
return false;
|
||||
|
||||
MatcherNumberConst<DataType> matcher{data_numeric->getData(), get<DataType>(index)};
|
||||
@ -910,6 +915,10 @@ bool FunctionArrayElement::matchKeyToIndex(
|
||||
|| matchKeyToIndexNumber<Int16>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<Int32>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<Int64>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<Int128>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<UInt128>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<Int256>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexNumber<UInt256>(data, offsets, arguments, matched_idxs)
|
||||
|| matchKeyToIndexString(data, offsets, arguments, matched_idxs);
|
||||
}
|
||||
|
||||
@ -925,6 +934,10 @@ bool FunctionArrayElement::matchKeyToIndexConst(
|
||||
|| matchKeyToIndexNumberConst<Int16>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<Int32>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<Int64>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<Int128>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<UInt128>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<Int256>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexNumberConst<UInt256>(data, offsets, index, matched_idxs)
|
||||
|| matchKeyToIndexStringConst(data, offsets, index, matched_idxs);
|
||||
}
|
||||
|
||||
@ -949,7 +962,7 @@ ColumnPtr FunctionArrayElement::executeMap(
|
||||
{
|
||||
if (input_rows_count > 0 && !matchKeyToIndex(keys_data, offsets, arguments, indices_data))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
|
||||
"Illegal types of arguments: {}, {} for function ",
|
||||
"Illegal types of arguments: {}, {} for function {}",
|
||||
arguments[0].type->getName(), arguments[1].type->getName(), getName());
|
||||
}
|
||||
else
|
||||
|
@ -65,7 +65,8 @@ public:
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (arguments.size() % 2 != 0)
|
||||
throw Exception("Function " + getName() + " even number of arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
"Function {} requires even number of arguments, but {} given", getName(), arguments.size());
|
||||
|
||||
DataTypes keys, values;
|
||||
for (size_t i = 0; i < arguments.size(); i += 2)
|
||||
|
@ -1998,7 +1998,6 @@ bool ParserExpressionElement::parseImpl(Pos & pos, ASTPtr & node, Expected & exp
|
||||
{
|
||||
return ParserSubquery().parse(pos, node, expected)
|
||||
|| ParserTupleOfLiterals().parse(pos, node, expected)
|
||||
|| ParserMapOfLiterals().parse(pos, node, expected)
|
||||
|| ParserParenthesisExpression().parse(pos, node, expected)
|
||||
|| ParserArrayOfLiterals().parse(pos, node, expected)
|
||||
|| ParserArray().parse(pos, node, expected)
|
||||
|
@ -338,18 +338,6 @@ protected:
|
||||
}
|
||||
};
|
||||
|
||||
class ParserMapOfLiterals : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserCollectionOfLiterals<Map> map_parser{TokenType::OpeningCurlyBrace, TokenType::ClosingCurlyBrace};
|
||||
protected:
|
||||
const char * getName() const override { return "map"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
|
||||
{
|
||||
return map_parser.parse(pos, node, expected);
|
||||
}
|
||||
};
|
||||
|
||||
class ParserArrayOfLiterals : public IParserBase
|
||||
{
|
||||
public:
|
||||
|
@ -490,14 +490,12 @@ bool ConstantExpressionTemplate::parseLiteralAndAssertType(ReadBuffer & istr, co
|
||||
/// TODO faster way to check types without using Parsers
|
||||
ParserArrayOfLiterals parser_array;
|
||||
ParserTupleOfLiterals parser_tuple;
|
||||
ParserMapOfLiterals parser_map;
|
||||
|
||||
Tokens tokens_number(istr.position(), istr.buffer().end());
|
||||
IParser::Pos iterator(tokens_number, settings.max_parser_depth);
|
||||
Expected expected;
|
||||
ASTPtr ast;
|
||||
if (!parser_array.parse(iterator, ast, expected) && !parser_tuple.parse(iterator, ast, expected)
|
||||
&& !parser_map.parse(iterator, ast, expected))
|
||||
if (!parser_array.parse(iterator, ast, expected) && !parser_tuple.parse(iterator, ast, expected))
|
||||
return false;
|
||||
|
||||
istr.position() = const_cast<char *>(iterator->begin);
|
||||
|
40
tests/queries/0_stateless/01720_type_map_and_casts.reference
Normal file
40
tests/queries/0_stateless/01720_type_map_and_casts.reference
Normal file
@ -0,0 +1,40 @@
|
||||
Map(Int8, Int8)
|
||||
{127:1,0:1,-1:1}
|
||||
{}
|
||||
1 0 1 1
|
||||
0 0 0 0
|
||||
0
|
||||
1
|
||||
1
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
0
|
||||
1
|
||||
Map(Int32, UInt16)
|
||||
{-1:1,2147483647:2,-2147483648:3}
|
||||
1 2 3
|
||||
0
|
||||
1
|
||||
0
|
||||
0
|
||||
0
|
||||
Map(Date, Int32)
|
||||
{'2020-01-01':1,'2020-01-02':2,'1970-01-02':3}
|
||||
1 2 0
|
||||
0
|
||||
3
|
||||
0
|
||||
Map(UUID, UInt16)
|
||||
{'00001192-0000-4000-8000-000000000001':1,'00001192-0000-4000-7000-000000000001':2}
|
||||
0 2 1
|
||||
Map(Int128, Int32)
|
||||
{-1:'a',0:'b',1234567898765432193024000:'c',-1234567898765432193024000:'d'}
|
||||
a b c d
|
||||
|
||||
a
|
||||
b
|
||||
|
79
tests/queries/0_stateless/01720_type_map_and_casts.sql
Normal file
79
tests/queries/0_stateless/01720_type_map_and_casts.sql
Normal file
@ -0,0 +1,79 @@
|
||||
SET allow_experimental_map_type = 1;
|
||||
|
||||
DROP TABLE IF EXISTS table_map_with_key_integer;
|
||||
|
||||
CREATE TABLE table_map_with_key_integer (d DATE, m Map(Int8, Int8))
|
||||
ENGINE = MergeTree() ORDER BY d;
|
||||
|
||||
INSERT INTO table_map_with_key_integer VALUES ('2020-01-01', map(127, 1, 0, 1, -1, 1)) ('2020-01-01', map());
|
||||
|
||||
SELECT 'Map(Int8, Int8)';
|
||||
|
||||
SELECT m FROM table_map_with_key_integer;
|
||||
SELECT m[127], m[1], m[0], m[-1] FROM table_map_with_key_integer;
|
||||
SELECT m[toInt8(number - 2)] FROM table_map_with_key_integer ARRAY JOIN range(5) AS number;
|
||||
|
||||
SELECT count() FROM table_map_with_key_integer WHERE m = map();
|
||||
|
||||
DROP TABLE IF EXISTS table_map_with_key_integer;
|
||||
|
||||
CREATE TABLE table_map_with_key_integer (d DATE, m Map(Int32, UInt16))
|
||||
ENGINE = MergeTree() ORDER BY d;
|
||||
|
||||
INSERT INTO table_map_with_key_integer VALUES ('2020-01-01', map(-1, 1, 2147483647, 2, -2147483648, 3));
|
||||
|
||||
SELECT 'Map(Int32, UInt16)';
|
||||
|
||||
SELECT m FROM table_map_with_key_integer;
|
||||
SELECT m[-1], m[2147483647], m[-2147483648] FROM table_map_with_key_integer;
|
||||
SELECT m[toInt32(number - 2)] FROM table_map_with_key_integer ARRAY JOIN range(5) AS number;
|
||||
|
||||
DROP TABLE IF EXISTS table_map_with_key_integer;
|
||||
|
||||
CREATE TABLE table_map_with_key_integer (d DATE, m Map(Date, Int32))
|
||||
ENGINE = MergeTree() ORDER BY d;
|
||||
|
||||
INSERT INTO table_map_with_key_integer VALUES ('2020-01-01', map('2020-01-01', 1, '2020-01-02', 2, '1970-01-02', 3));
|
||||
|
||||
SELECT 'Map(Date, Int32)';
|
||||
|
||||
SELECT m FROM table_map_with_key_integer;
|
||||
SELECT m[toDate('2020-01-01')], m[toDate('2020-01-02')], m[toDate('2020-01-03')] FROM table_map_with_key_integer;
|
||||
SELECT m[toDate(number)] FROM table_map_with_key_integer ARRAY JOIN range(3) AS number;
|
||||
|
||||
DROP TABLE IF EXISTS table_map_with_key_integer;
|
||||
|
||||
CREATE TABLE table_map_with_key_integer (d DATE, m Map(UUID, UInt16))
|
||||
ENGINE = MergeTree() ORDER BY d;
|
||||
|
||||
INSERT INTO table_map_with_key_integer VALUES ('2020-01-01', map('00001192-0000-4000-8000-000000000001', 1, '00001192-0000-4000-7000-000000000001', 2));
|
||||
|
||||
SELECT 'Map(UUID, UInt16)';
|
||||
|
||||
SELECT m FROM table_map_with_key_integer;
|
||||
SELECT
|
||||
m[toUUID('00001192-0000-4000-6000-000000000001')],
|
||||
m[toUUID('00001192-0000-4000-7000-000000000001')],
|
||||
m[toUUID('00001192-0000-4000-8000-000000000001')]
|
||||
FROM table_map_with_key_integer;
|
||||
|
||||
DROP TABLE IF EXISTS table_map_with_key_integer;
|
||||
|
||||
CREATE TABLE table_map_with_key_integer (d DATE, m Map(Int128, String))
|
||||
ENGINE = MergeTree() ORDER BY d;
|
||||
|
||||
|
||||
INSERT INTO table_map_with_key_integer SELECT '2020-01-01', map(-1, 'a', 0, 'b', toInt128(1234567898765432123456789), 'c', toInt128(-1234567898765432123456789), 'd');
|
||||
|
||||
SELECT 'Map(Int128, Int32)';
|
||||
|
||||
SELECT m FROM table_map_with_key_integer;
|
||||
SELECT m[toInt128(-1)], m[toInt128(0)], m[toInt128(1234567898765432123456789)], m[toInt128(-1234567898765432123456789)] FROM table_map_with_key_integer;
|
||||
SELECT m[toInt128(number - 2)] FROM table_map_with_key_integer ARRAY JOIN range(4) AS number;
|
||||
|
||||
DROP TABLE IF EXISTS table_map_with_key_integer;
|
||||
|
||||
|
||||
CREATE TABLE table_map_with_key_integer (m Map(Float32, String)) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 36}
|
||||
CREATE TABLE table_map_with_key_integer (m Map(Nullable(String), String)) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 36}
|
||||
CREATE TABLE table_map_with_key_integer (m Map(Array(UInt32), String)) ENGINE = MergeTree() ORDER BY tuple(); -- { serverError 36}
|
@ -111,6 +111,7 @@
|
||||
"memory_tracking",
|
||||
"memory_usage",
|
||||
"live_view",
|
||||
"01720_type_map_and_casts",
|
||||
"01413_alter_update_supertype",
|
||||
"01149_zookeeper_mutation_stuck_after_replace_partition",
|
||||
"00836_indices_alter_replicated_zookeeper",
|
||||
|
Loading…
Reference in New Issue
Block a user