ClickHouse/dbms/src/DataTypes/DataTypeEnum.cpp

400 lines
14 KiB
C++
Raw Normal View History

#include <IO/WriteBufferFromString.h>
#include <Formats/FormatSettings.h>
#include <Formats/ProtobufReader.h>
#include <Formats/ProtobufWriter.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeFactory.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
2017-07-13 20:58:19 +00:00
#include <Common/typeid_cast.h>
#include <Common/UTF8Helpers.h>
#include <Poco/UTF8Encoding.h>
#include <limits>
namespace DB
{
namespace ErrorCodes
{
extern const int SYNTAX_ERROR;
extern const int EMPTY_DATA_PASSED;
extern const int UNEXPECTED_AST_STRUCTURE;
extern const int ARGUMENT_OUT_OF_BOUND;
}
template <typename FieldType> struct EnumName;
template <> struct EnumName<Int8> { static constexpr auto value = "Enum8"; };
template <> struct EnumName<Int16> { static constexpr auto value = "Enum16"; };
template <typename Type>
const char * DataTypeEnum<Type>::getFamilyName() const
{
return EnumName<FieldType>::value;
}
template <typename Type>
std::string DataTypeEnum<Type>::generateName(const Values & values)
{
2017-07-31 21:39:24 +00:00
WriteBufferFromOwnString out;
2017-07-31 21:39:24 +00:00
writeString(EnumName<FieldType>::value, out);
writeChar('(', out);
2017-07-31 21:39:24 +00:00
auto first = true;
for (const auto & name_and_value : values)
{
if (!first)
writeString(", ", out);
2017-07-31 21:39:24 +00:00
first = false;
2017-07-31 21:39:24 +00:00
writeQuotedString(name_and_value.first, out);
writeString(" = ", out);
writeText(name_and_value.second, out);
}
2017-07-31 21:39:24 +00:00
writeChar(')', out);
return out.str();
}
template <typename Type>
void DataTypeEnum<Type>::fillMaps()
{
2017-07-03 18:29:56 +00:00
for (const auto & name_and_value : values)
{
const auto name_to_value_pair = name_to_value_map.insert(
{ StringRef{name_and_value.first}, name_and_value.second });
2018-02-05 17:56:50 +00:00
if (!name_to_value_pair.second)
2018-02-05 17:56:50 +00:00
throw Exception{"Duplicate names in enum: '" + name_and_value.first + "' = " + toString(name_and_value.second)
A Proper lookup table that uses HashTable's API This is the first step of allowing heterogeneous cells in hash tables. performance test results are ``` 1. HashMap<UInt16, UInt8, TrivialHash, HashTableFixedGrower<16>>; 2. NewLookupMap<UInt16, UInt8> ResolutionWidth 30000 1 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................223550276.46 ResolutionWidth 30000 2 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................248772721.24 Best: 2 - 24877272124 ResolutionWidth 100000 1 ..........................................................................................................................................................................................................................................................238498413.99 ResolutionWidth 100000 2 ..........................................................................................................................................................................................................................................................261808889.98 Best: 2 - 26180888998 ResolutionWidth 300000 1 ...................................................................................239307348.81 ResolutionWidth 300000 2 ...................................................................................257592761.30 Best: 2 - 25759276130 ResolutionWidth 1000000 1 .........................240144759.26 ResolutionWidth 1000000 2 .........................257093531.91 Best: 2 - 25709353191 ResolutionWidth 5000000 1 .....241573260.35 ResolutionWidth 5000000 2 .....259314162.79 Best: 2 - 25931416279 ResolutionDepth 30000 1 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................217108119.84 ResolutionDepth 30000 2 .................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................................249459504.41 Best: 2 - 24945950441 ResolutionDepth 100000 1 ..........................................................................................................................................................................................................................................................229065162.17 ResolutionDepth 100000 2 ..........................................................................................................................................................................................................................................................253769105.64 Best: 2 - 25376910564 ResolutionDepth 300000 1 ...................................................................................233079225.18 ResolutionDepth 300000 2 ...................................................................................256316273.78 Best: 2 - 25631627378 ResolutionDepth 1000000 1 .........................234184633.51 ResolutionDepth 1000000 2 .........................261100491.57 Best: 2 - 26110049157 ResolutionDepth 5000000 1 .....233118795.66 ResolutionDepth 5000000 2 .....252436160.41 Best: 2 - 25243616041 ```
2019-02-28 09:35:38 +00:00
+ " and '" + name_to_value_pair.first->getFirst().toString() + "' = " + toString(name_to_value_pair.first->getSecond()),
2017-07-03 18:29:56 +00:00
ErrorCodes::SYNTAX_ERROR};
const auto value_to_name_pair = value_to_name_map.insert(
{ name_and_value.second, StringRef{name_and_value.first} });
2018-02-05 17:56:50 +00:00
if (!value_to_name_pair.second)
2018-02-05 17:56:50 +00:00
throw Exception{"Duplicate values in enum: '" + name_and_value.first + "' = " + toString(name_and_value.second)
2017-07-03 18:29:56 +00:00
+ " and '" + value_to_name_pair.first->second.toString() + "' = " + toString(value_to_name_pair.first->first),
ErrorCodes::SYNTAX_ERROR};
}
}
template <typename Type>
DataTypeEnum<Type>::DataTypeEnum(const Values & values_) : values{values_}
{
if (values.empty())
2018-02-05 17:56:50 +00:00
throw Exception{"DataTypeEnum enumeration cannot be empty", ErrorCodes::EMPTY_DATA_PASSED};
std::sort(std::begin(values), std::end(values), [] (auto & left, auto & right)
{
return left.second < right.second;
});
2018-02-05 17:57:38 +00:00
fillMaps();
type_name = generateName(values);
}
template <typename Type>
void DataTypeEnum<Type>::serializeBinary(const Field & field, WriteBuffer & ostr) const
{
2018-11-20 20:09:20 +00:00
const FieldType x = get<NearestFieldType<FieldType>>(field);
writeBinary(x, ostr);
}
template <typename Type>
void DataTypeEnum<Type>::deserializeBinary(Field & field, ReadBuffer & istr) const
{
FieldType x;
readBinary(x, istr);
field = nearestFieldType(x);
}
template <typename Type>
void DataTypeEnum<Type>::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
writeBinary(static_cast<const ColumnType &>(column).getData()[row_num], ostr);
}
template <typename Type>
void DataTypeEnum<Type>::deserializeBinary(IColumn & column, ReadBuffer & istr) const
{
typename ColumnType::value_type x;
readBinary(x, istr);
static_cast<ColumnType &>(column).getData().push_back(x);
}
template <typename Type>
void DataTypeEnum<Type>::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{
writeString(getNameForValue(static_cast<const ColumnType &>(column).getData()[row_num]), ostr);
}
template <typename Type>
void DataTypeEnum<Type>::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{
writeEscapedString(getNameForValue(static_cast<const ColumnType &>(column).getData()[row_num]), ostr);
}
template <typename Type>
void DataTypeEnum<Type>::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
/// NOTE It would be nice to do without creating a temporary object - at least extract std::string out.
std::string field_name;
readEscapedString(field_name, istr);
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(field_name)));
}
template <typename Type>
void DataTypeEnum<Type>::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{
writeQuotedString(getNameForValue(static_cast<const ColumnType &>(column).getData()[row_num]), ostr);
}
template <typename Type>
void DataTypeEnum<Type>::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
std::string field_name;
readQuotedStringWithSQLStyle(field_name, istr);
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(field_name)));
}
template <typename Type>
void DataTypeEnum<Type>::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
{
writeJSONString(getNameForValue(static_cast<const ColumnType &>(column).getData()[row_num]), ostr, settings);
}
template <typename Type>
void DataTypeEnum<Type>::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{
writeXMLString(getNameForValue(static_cast<const ColumnType &>(column).getData()[row_num]), ostr);
}
template <typename Type>
void DataTypeEnum<Type>::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
{
std::string field_name;
readJSONString(field_name, istr);
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(field_name)));
}
template <typename Type>
void DataTypeEnum<Type>::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
{
writeCSVString(getNameForValue(static_cast<const ColumnType &>(column).getData()[row_num]), ostr);
}
template <typename Type>
void DataTypeEnum<Type>::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
{
std::string field_name;
readCSVString(field_name, istr, settings.csv);
static_cast<ColumnType &>(column).getData().push_back(getValue(StringRef(field_name)));
}
template <typename Type>
void DataTypeEnum<Type>::serializeBinaryBulk(
const IColumn & column, WriteBuffer & ostr, const size_t offset, size_t limit) const
{
const auto & x = typeid_cast<const ColumnType &>(column).getData();
const auto size = x.size();
if (limit == 0 || offset + limit > size)
limit = size - offset;
ostr.write(reinterpret_cast<const char *>(&x[offset]), sizeof(FieldType) * limit);
}
template <typename Type>
void DataTypeEnum<Type>::deserializeBinaryBulk(
IColumn & column, ReadBuffer & istr, const size_t limit, const double /*avg_value_size_hint*/) const
{
auto & x = typeid_cast<ColumnType &>(column).getData();
const auto initial_size = x.size();
x.resize(initial_size + limit);
const auto size = istr.readBig(reinterpret_cast<char*>(&x[initial_size]), sizeof(FieldType) * limit);
x.resize(initial_size + size / sizeof(FieldType));
}
template <typename Type>
void DataTypeEnum<Type>::serializeProtobuf(const IColumn & column, size_t row_num, ProtobufWriter & protobuf, size_t & value_index) const
{
if (value_index)
return;
protobuf.prepareEnumMapping(values);
value_index = static_cast<bool>(protobuf.writeEnum(static_cast<const ColumnType &>(column).getData()[row_num]));
}
template<typename Type>
void DataTypeEnum<Type>::deserializeProtobuf(IColumn & column, ProtobufReader & protobuf, bool allow_add_row, bool & row_added) const
{
protobuf.prepareEnumMapping(values);
row_added = false;
Type value;
if (!protobuf.readEnum(value))
return;
auto & container = static_cast<ColumnType &>(column).getData();
if (allow_add_row)
{
container.emplace_back(value);
row_added = true;
}
else
container.back() = value;
}
template <typename Type>
Field DataTypeEnum<Type>::getDefault() const
{
return values.front().second;
}
template <typename Type>
void DataTypeEnum<Type>::insertDefaultInto(IColumn & column) const
{
static_cast<ColumnType &>(column).getData().push_back(values.front().second);
}
template <typename Type>
bool DataTypeEnum<Type>::equals(const IDataType & rhs) const
{
return typeid(rhs) == typeid(*this) && type_name == static_cast<const DataTypeEnum<Type> &>(rhs).type_name;
}
template <typename Type>
bool DataTypeEnum<Type>::textCanContainOnlyValidUTF8() const
{
for (const auto & elem : values)
{
const char * pos = elem.first.data();
const char * end = pos + elem.first.size();
while (pos < end)
{
size_t length = UTF8::seqLength(*pos);
if (pos + length > end)
return false;
if (Poco::UTF8Encoding::isLegal(reinterpret_cast<const unsigned char *>(pos), length))
pos += length;
else
return false;
}
}
return true;
}
template <typename Type>
static void checkOverflow(Int64 value)
{
if (!(std::numeric_limits<Type>::min() <= value && value <= std::numeric_limits<Type>::max()))
throw Exception("DataTypeEnum: Unexpected value " + toString(value), ErrorCodes::BAD_TYPE_OF_FIELD);
}
template <typename Type>
Field DataTypeEnum<Type>::castToName(const Field & value_or_name) const
{
if (value_or_name.getType() == Field::Types::String)
{
getValue(value_or_name.get<String>()); /// Check correctness
return value_or_name.get<String>();
}
else if (value_or_name.getType() == Field::Types::Int64)
{
Int64 value = value_or_name.get<Int64>();
checkOverflow<Type>(value);
return getNameForValue(static_cast<Type>(value)).toString();
}
else
throw Exception(String("DataTypeEnum: Unsupported type of field ") + value_or_name.getTypeName(), ErrorCodes::BAD_TYPE_OF_FIELD);
}
template <typename Type>
Field DataTypeEnum<Type>::castToValue(const Field & value_or_name) const
{
if (value_or_name.getType() == Field::Types::String)
{
return getValue(value_or_name.get<String>());
}
else if (value_or_name.getType() == Field::Types::Int64
|| value_or_name.getType() == Field::Types::UInt64)
{
Int64 value = value_or_name.get<Int64>();
checkOverflow<Type>(value);
getNameForValue(static_cast<Type>(value)); /// Check correctness
return value;
}
else
throw Exception(String("DataTypeEnum: Unsupported type of field ") + value_or_name.getTypeName(), ErrorCodes::BAD_TYPE_OF_FIELD);
}
2017-03-25 20:12:56 +00:00
/// Explicit instantiations.
template class DataTypeEnum<Int8>;
template class DataTypeEnum<Int16>;
template <typename DataTypeEnum>
static DataTypePtr create(const ASTPtr & arguments)
{
if (!arguments || arguments->children.empty())
throw Exception("Enum data type cannot be empty", ErrorCodes::EMPTY_DATA_PASSED);
typename DataTypeEnum::Values values;
values.reserve(arguments->children.size());
using FieldType = typename DataTypeEnum::FieldType;
/// Children must be functions 'equals' with string literal as left argument and numeric literal as right argument.
for (const ASTPtr & child : arguments->children)
{
2019-03-11 13:22:51 +00:00
const auto * func = child->as<ASTFunction>();
if (!func
|| func->name != "equals"
|| func->parameters
|| !func->arguments
|| func->arguments->children.size() != 2)
throw Exception("Elements of Enum data type must be of form: 'name' = number, where name is string literal and number is an integer",
ErrorCodes::UNEXPECTED_AST_STRUCTURE);
2019-03-11 13:22:51 +00:00
const auto * name_literal = func->arguments->children[0]->as<ASTLiteral>();
const auto * value_literal = func->arguments->children[1]->as<ASTLiteral>();
if (!name_literal
|| !value_literal
|| name_literal->value.getType() != Field::Types::String
|| (value_literal->value.getType() != Field::Types::UInt64 && value_literal->value.getType() != Field::Types::Int64))
throw Exception("Elements of Enum data type must be of form: 'name' = number, where name is string literal and number is an integer",
ErrorCodes::UNEXPECTED_AST_STRUCTURE);
const String & field_name = name_literal->value.get<String>();
2018-11-20 20:09:20 +00:00
const auto value = value_literal->value.get<NearestFieldType<FieldType>>();
if (value > std::numeric_limits<FieldType>::max() || value < std::numeric_limits<FieldType>::min())
throw Exception{"Value " + toString(value) + " for element '" + field_name + "' exceeds range of " + EnumName<FieldType>::value,
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
values.emplace_back(field_name, value);
}
return std::make_shared<DataTypeEnum>(values);
}
void registerDataTypeEnum(DataTypeFactory & factory)
{
factory.registerDataType("Enum8", create<DataTypeEnum<Int8>>);
factory.registerDataType("Enum16", create<DataTypeEnum<Int16>>);
}
}