fix type map with integer keys

This commit is contained in:
Anton Popov 2021-02-18 01:05:31 +03:00
parent 3429925338
commit f1fa110d48
7 changed files with 61 additions and 25 deletions

View File

@ -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
{

View File

@ -98,6 +98,8 @@ private:
template <typename Reader>
void deserializeTextImpl(IColumn & column, ReadBuffer & istr, bool need_safe_get_int_key, Reader && reader) const;
void assertKeyType() const;
};
}

View File

@ -872,7 +872,7 @@ bool FunctionArrayElement::matchKeyToIndexNumberConst(
if (!data_numeric)
return false;
if (index.getType() != Field::Types::UInt64 && index.getType() != Field::Types::Int64)
if (index.getType() != Field::Types::UInt64 && index.getType() != Field::Types::Int64 && index.getType() != Field::Types::Int128)
return false;
MatcherNumberConst<DataType> matcher{data_numeric->getData(), get<DataType>(index)};
@ -910,6 +910,7 @@ 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)
|| matchKeyToIndexString(data, offsets, arguments, matched_idxs);
}
@ -925,6 +926,7 @@ 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)
|| matchKeyToIndexStringConst(data, offsets, index, matched_idxs);
}
@ -945,11 +947,14 @@ ColumnPtr FunctionArrayElement::executeMap(
indices_column->reserve(input_rows_count);
auto & indices_data = assert_cast<ColumnVector<UInt64> &>(*indices_column).getData();
std::cerr << "types: " << arguments[0].type->getName() << " " << arguments[1].type->getName() << "\n";
std::cerr << "columns: " << arguments[0].column->dumpStructure() << " " << arguments[1].column->dumpStructure() << "\n";
if (!isColumnConst(*arguments[1].column))
{
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

View File

@ -15,6 +15,7 @@
#include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeMap.h>
#include <Core/AccurateComparison.h>
#include <Common/FieldVisitors.h>
@ -276,6 +277,40 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID
return have_unconvertible_element ? Field(Null()) : Field(res);
}
}
else if (const DataTypeMap * type_map = typeid_cast<const DataTypeMap *>(&type))
{
if (src.getType() == Field::Types::Map)
{
const auto & src_map = src.get<Map>();
const auto & key_type = *type_map->getKeyType();
const auto & value_type = *type_map->getValueType();
bool have_unconvertible_element = false;
Map res(src_map.size());
for (size_t i = 0; i < src_map.size(); ++i)
{
const auto & src_tuple = src_map[i].safeGet<const Tuple &>();
assert(src_tuple.size() == 2);
Tuple res_tuple(2);
res_tuple[0] = convertFieldToType(src_tuple[0], key_type);
res_tuple[1] = convertFieldToType(src_tuple[1], value_type);
if ((res_tuple[0].isNull() && !key_type.isNullable())
|| (res_tuple[1].isNull() && !value_type.isNullable()))
{
// See the comment for Tuples above.
have_unconvertible_element = true;
}
res[i] = std::move(res_tuple);
}
return have_unconvertible_element ? Field(Null()) : Field(res);
}
}
else if (const DataTypeAggregateFunction * agg_func_type = typeid_cast<const DataTypeAggregateFunction *>(&type))
{
if (src.getType() != Field::Types::AggregateFunctionState)

View File

@ -1979,7 +1979,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)

View File

@ -324,18 +324,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:

View File

@ -203,12 +203,6 @@ private:
if (not_null == array.end())
return true;
}
else if (literal->value.getType() == Field::Types::Map)
{
const Map & map = literal->value.get<Map>();
if (map.size() % 2)
return false;
}
String column_name = "_dummy_" + std::to_string(replaced_literals.size());
replaced_literals.emplace_back(literal, column_name, force_nullable);
@ -481,14 +475,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);