mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 08:32:02 +00:00
return back data types
This commit is contained in:
parent
1e61f64f95
commit
be540e442d
173
src/DataTypes/DataTypeAggregateFunction.cpp
Normal file
173
src/DataTypes/DataTypeAggregateFunction.cpp
Normal file
@ -0,0 +1,173 @@
|
||||
#include <Common/FieldVisitors.h>
|
||||
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/AlignedBuffer.h>
|
||||
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <DataTypes/Serializations/SerializationAggregateFunction.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <AggregateFunctions/AggregateFunctionFactory.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
std::string DataTypeAggregateFunction::doGetName() const
|
||||
{
|
||||
WriteBufferFromOwnString stream;
|
||||
stream << "AggregateFunction(" << function->getName();
|
||||
|
||||
if (!parameters.empty())
|
||||
{
|
||||
stream << '(';
|
||||
for (size_t i = 0; i < parameters.size(); ++i)
|
||||
{
|
||||
if (i)
|
||||
stream << ", ";
|
||||
stream << applyVisitor(DB::FieldVisitorToString(), parameters[i]);
|
||||
}
|
||||
stream << ')';
|
||||
}
|
||||
|
||||
for (const auto & argument_type : argument_types)
|
||||
stream << ", " << argument_type->getName();
|
||||
|
||||
stream << ')';
|
||||
return stream.str();
|
||||
}
|
||||
|
||||
MutableColumnPtr DataTypeAggregateFunction::createColumn() const
|
||||
{
|
||||
return ColumnAggregateFunction::create(function);
|
||||
}
|
||||
|
||||
|
||||
/// Create empty state
|
||||
Field DataTypeAggregateFunction::getDefault() const
|
||||
{
|
||||
Field field = AggregateFunctionStateData();
|
||||
field.get<AggregateFunctionStateData &>().name = getName();
|
||||
|
||||
AlignedBuffer place_buffer(function->sizeOfData(), function->alignOfData());
|
||||
AggregateDataPtr place = place_buffer.data();
|
||||
|
||||
function->create(place);
|
||||
|
||||
try
|
||||
{
|
||||
WriteBufferFromString buffer_from_field(field.get<AggregateFunctionStateData &>().data);
|
||||
function->serialize(place, buffer_from_field);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
function->destroy(place);
|
||||
throw;
|
||||
}
|
||||
|
||||
function->destroy(place);
|
||||
|
||||
return field;
|
||||
}
|
||||
|
||||
|
||||
bool DataTypeAggregateFunction::equals(const IDataType & rhs) const
|
||||
{
|
||||
return typeid(rhs) == typeid(*this) && getName() == rhs.getName();
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeAggregateFunction::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationAggregateFunction>(function);
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
String function_name;
|
||||
AggregateFunctionPtr function;
|
||||
DataTypes argument_types;
|
||||
Array params_row;
|
||||
|
||||
if (!arguments || arguments->children.empty())
|
||||
throw Exception("Data type AggregateFunction requires parameters: "
|
||||
"name of aggregate function and list of data types for arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (const auto * parametric = arguments->children[0]->as<ASTFunction>())
|
||||
{
|
||||
if (parametric->parameters)
|
||||
throw Exception("Unexpected level of parameters to aggregate function", ErrorCodes::SYNTAX_ERROR);
|
||||
function_name = parametric->name;
|
||||
|
||||
if (parametric->arguments)
|
||||
{
|
||||
const ASTs & parameters = parametric->arguments->children;
|
||||
params_row.resize(parameters.size());
|
||||
|
||||
for (size_t i = 0; i < parameters.size(); ++i)
|
||||
{
|
||||
const auto * literal = parameters[i]->as<ASTLiteral>();
|
||||
if (!literal)
|
||||
throw Exception(
|
||||
ErrorCodes::PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS,
|
||||
"Parameters to aggregate functions must be literals. "
|
||||
"Got parameter '{}' for function '{}'",
|
||||
parameters[i]->formatForErrorMessage(), function_name);
|
||||
|
||||
params_row[i] = literal->value;
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (auto opt_name = tryGetIdentifierName(arguments->children[0]))
|
||||
{
|
||||
function_name = *opt_name;
|
||||
}
|
||||
else if (arguments->children[0]->as<ASTLiteral>())
|
||||
{
|
||||
throw Exception("Aggregate function name for data type AggregateFunction must be passed as identifier (without quotes) or function",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
}
|
||||
else
|
||||
throw Exception("Unexpected AST element passed as aggregate function name for data type AggregateFunction. Must be identifier or function.",
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
for (size_t i = 1; i < arguments->children.size(); ++i)
|
||||
argument_types.push_back(DataTypeFactory::instance().get(arguments->children[i]));
|
||||
|
||||
if (function_name.empty())
|
||||
throw Exception("Logical error: empty name of aggregate function passed", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
AggregateFunctionProperties properties;
|
||||
function = AggregateFunctionFactory::instance().get(function_name, argument_types, params_row, properties);
|
||||
return std::make_shared<DataTypeAggregateFunction>(function, argument_types, params_row);
|
||||
}
|
||||
|
||||
void registerDataTypeAggregateFunction(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerDataType("AggregateFunction", create);
|
||||
}
|
||||
|
||||
|
||||
}
|
57
src/DataTypes/DataTypeAggregateFunction.h
Normal file
57
src/DataTypes/DataTypeAggregateFunction.h
Normal file
@ -0,0 +1,57 @@
|
||||
#pragma once
|
||||
|
||||
#include <AggregateFunctions/IAggregateFunction.h>
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Type - the state of the aggregate function.
|
||||
* Type parameters is an aggregate function, the types of its arguments, and its parameters (for parametric aggregate functions).
|
||||
*/
|
||||
class DataTypeAggregateFunction final : public IDataType
|
||||
{
|
||||
private:
|
||||
AggregateFunctionPtr function;
|
||||
DataTypes argument_types;
|
||||
Array parameters;
|
||||
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
DataTypeAggregateFunction(const AggregateFunctionPtr & function_, const DataTypes & argument_types_, const Array & parameters_)
|
||||
: function(function_), argument_types(argument_types_), parameters(parameters_)
|
||||
{
|
||||
}
|
||||
|
||||
std::string getFunctionName() const { return function->getName(); }
|
||||
AggregateFunctionPtr getFunction() const { return function; }
|
||||
|
||||
std::string doGetName() const override;
|
||||
const char * getFamilyName() const override { return "AggregateFunction"; }
|
||||
TypeIndex getTypeId() const override { return TypeIndex::AggregateFunction; }
|
||||
|
||||
bool canBeInsideNullable() const override { return false; }
|
||||
|
||||
DataTypePtr getReturnType() const { return function->getReturnType(); }
|
||||
DataTypePtr getReturnTypeToPredict() const { return function->getReturnTypeToPredict(); }
|
||||
DataTypes getArgumentsDataTypes() const { return argument_types; }
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool shouldAlignRightInPrettyFormats() const override { return false; }
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
};
|
||||
|
||||
|
||||
}
|
||||
|
147
src/DataTypes/DataTypeArray.cpp
Normal file
147
src/DataTypes/DataTypeArray.cpp
Normal file
@ -0,0 +1,147 @@
|
||||
#include <Columns/ColumnArray.h>
|
||||
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/Serializations/SerializationArray.h>
|
||||
#include <DataTypes/Serializations/SerializationTupleElement.h>
|
||||
#include <DataTypes/Serializations/SerializationNumber.h>
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
#include <Core/NamesAndTypes.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
|
||||
DataTypeArray::DataTypeArray(const DataTypePtr & nested_)
|
||||
: nested{nested_}
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr DataTypeArray::createColumn() const
|
||||
{
|
||||
return ColumnArray::create(nested->createColumn(), ColumnArray::ColumnOffsets::create());
|
||||
}
|
||||
|
||||
|
||||
Field DataTypeArray::getDefault() const
|
||||
{
|
||||
return Array();
|
||||
}
|
||||
|
||||
|
||||
bool DataTypeArray::equals(const IDataType & rhs) const
|
||||
{
|
||||
return typeid(rhs) == typeid(*this) && nested->equals(*static_cast<const DataTypeArray &>(rhs).nested);
|
||||
}
|
||||
|
||||
DataTypePtr DataTypeArray::tryGetSubcolumnType(const String & subcolumn_name) const
|
||||
{
|
||||
return tryGetSubcolumnTypeImpl(subcolumn_name, 0);
|
||||
}
|
||||
|
||||
DataTypePtr DataTypeArray::tryGetSubcolumnTypeImpl(const String & subcolumn_name, size_t level) const
|
||||
{
|
||||
if (subcolumn_name == "size" + std::to_string(level))
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
|
||||
DataTypePtr subcolumn;
|
||||
if (const auto * nested_array = typeid_cast<const DataTypeArray *>(nested.get()))
|
||||
subcolumn = nested_array->tryGetSubcolumnTypeImpl(subcolumn_name, level + 1);
|
||||
else
|
||||
subcolumn = nested->tryGetSubcolumnType(subcolumn_name);
|
||||
|
||||
if (subcolumn && subcolumn_name != MAIN_SUBCOLUMN_NAME)
|
||||
subcolumn = std::make_shared<DataTypeArray>(std::move(subcolumn));
|
||||
|
||||
return subcolumn;
|
||||
}
|
||||
|
||||
ColumnPtr DataTypeArray::getSubcolumn(const String & subcolumn_name, const IColumn & column) const
|
||||
{
|
||||
return getSubcolumnImpl(subcolumn_name, column, 0);
|
||||
}
|
||||
|
||||
ColumnPtr DataTypeArray::getSubcolumnImpl(const String & subcolumn_name, const IColumn & column, size_t level) const
|
||||
{
|
||||
const auto & column_array = assert_cast<const ColumnArray &>(column);
|
||||
if (subcolumn_name == "size" + std::to_string(level))
|
||||
return arrayOffsetsToSizes(column_array.getOffsetsColumn());
|
||||
|
||||
ColumnPtr subcolumn;
|
||||
if (const auto * nested_array = typeid_cast<const DataTypeArray *>(nested.get()))
|
||||
subcolumn = nested_array->getSubcolumnImpl(subcolumn_name, column_array.getData(), level + 1);
|
||||
else
|
||||
subcolumn = nested->getSubcolumn(subcolumn_name, column_array.getData());
|
||||
|
||||
return ColumnArray::create(subcolumn, column_array.getOffsetsPtr());
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeArray::getSubcolumnSerialization(
|
||||
const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const
|
||||
{
|
||||
return getSubcolumnSerializationImpl(subcolumn_name, base_serialization_getter, 0);
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeArray::getSubcolumnSerializationImpl(
|
||||
const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter, size_t level) const
|
||||
{
|
||||
if (subcolumn_name == "size" + std::to_string(level))
|
||||
return std::make_shared<SerializationTupleElement>(base_serialization_getter(DataTypeUInt64()), subcolumn_name, false);
|
||||
|
||||
SerializationPtr subcolumn;
|
||||
if (const auto * nested_array = typeid_cast<const DataTypeArray *>(nested.get()))
|
||||
subcolumn = nested_array->getSubcolumnSerializationImpl(subcolumn_name, base_serialization_getter, level + 1);
|
||||
else
|
||||
subcolumn = nested->getSubcolumnSerialization(subcolumn_name, base_serialization_getter);
|
||||
|
||||
return std::make_shared<SerializationArray>(subcolumn);
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeArray::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationArray>(nested->getDefaultSerialization());
|
||||
}
|
||||
|
||||
size_t DataTypeArray::getNumberOfDimensions() const
|
||||
{
|
||||
const DataTypeArray * nested_array = typeid_cast<const DataTypeArray *>(nested.get());
|
||||
if (!nested_array)
|
||||
return 1;
|
||||
return 1 + nested_array->getNumberOfDimensions(); /// Every modern C++ compiler optimizes tail recursion.
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->children.size() != 1)
|
||||
throw Exception("Array data type family must have exactly one argument - type of elements", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return std::make_shared<DataTypeArray>(DataTypeFactory::instance().get(arguments->children[0]));
|
||||
}
|
||||
|
||||
|
||||
void registerDataTypeArray(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerDataType("Array", create);
|
||||
}
|
||||
|
||||
}
|
76
src/DataTypes/DataTypeArray.h
Normal file
76
src/DataTypes/DataTypeArray.h
Normal file
@ -0,0 +1,76 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/Serializations/SerializationArray.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
class DataTypeArray final : public IDataType
|
||||
{
|
||||
private:
|
||||
/// The type of array elements.
|
||||
DataTypePtr nested;
|
||||
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
DataTypeArray(const DataTypePtr & nested_);
|
||||
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Array; }
|
||||
|
||||
std::string doGetName() const override
|
||||
{
|
||||
return "Array(" + nested->getName() + ")";
|
||||
}
|
||||
|
||||
const char * getFamilyName() const override
|
||||
{
|
||||
return "Array";
|
||||
}
|
||||
|
||||
bool canBeInsideNullable() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return true; }
|
||||
bool cannotBeStoredInTables() const override { return nested->cannotBeStoredInTables(); }
|
||||
bool textCanContainOnlyValidUTF8() const override { return nested->textCanContainOnlyValidUTF8(); }
|
||||
bool isComparable() const override { return nested->isComparable(); }
|
||||
bool canBeComparedWithCollation() const override { return nested->canBeComparedWithCollation(); }
|
||||
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override
|
||||
{
|
||||
return nested->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion();
|
||||
}
|
||||
|
||||
DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override;
|
||||
ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override;
|
||||
SerializationPtr getSubcolumnSerialization(
|
||||
const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const override;
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
|
||||
const DataTypePtr & getNestedType() const { return nested; }
|
||||
|
||||
/// 1 for plain array, 2 for array of arrays and so on.
|
||||
size_t getNumberOfDimensions() const;
|
||||
|
||||
private:
|
||||
ColumnPtr getSubcolumnImpl(const String & subcolumn_name, const IColumn & column, size_t level) const;
|
||||
DataTypePtr tryGetSubcolumnTypeImpl(const String & subcolumn_name, size_t level) const;
|
||||
SerializationPtr getSubcolumnSerializationImpl(
|
||||
const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter, size_t level) const;
|
||||
};
|
||||
|
||||
}
|
@ -3,7 +3,7 @@
|
||||
#include <memory>
|
||||
#include <cstddef>
|
||||
#include <Core/Types.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/Serializations/ISerialization.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -24,106 +24,20 @@ public:
|
||||
virtual String getName() const = 0;
|
||||
};
|
||||
|
||||
class IDataTypeCustomTextSerialization
|
||||
{
|
||||
public:
|
||||
virtual ~IDataTypeCustomTextSerialization() {}
|
||||
|
||||
/** Text serialization for displaying on a terminal or saving into a text file, and the like.
|
||||
* Without escaping or quoting.
|
||||
*/
|
||||
virtual void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
|
||||
/** Text deserialization without quoting or escaping.
|
||||
*/
|
||||
virtual void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
|
||||
/** Text serialization with escaping but without quoting.
|
||||
*/
|
||||
virtual void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
virtual void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
|
||||
/** Text serialization as a literal that may be inserted into a query.
|
||||
*/
|
||||
virtual void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
virtual void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
|
||||
/** Text serialization for the CSV format.
|
||||
*/
|
||||
virtual void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
virtual void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
|
||||
/** Text serialization intended for using in JSON format.
|
||||
*/
|
||||
virtual void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
virtual void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
|
||||
/** Text serialization for putting into the XML format.
|
||||
*/
|
||||
virtual void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const = 0;
|
||||
};
|
||||
|
||||
/** Allows to customize an existing data type by representation with custom substreams.
|
||||
* Customized data type will be serialized/deserialized to files with different names than base type,
|
||||
* but binary and text representation will be unchanged.
|
||||
* E.g it can be used for reading single subcolumns of complex types.
|
||||
*/
|
||||
class IDataTypeCustomStreams
|
||||
{
|
||||
public:
|
||||
virtual ~IDataTypeCustomStreams() = default;
|
||||
|
||||
virtual void enumerateStreams(
|
||||
const IDataType::StreamCallback & callback,
|
||||
IDataType::SubstreamPath & path) const = 0;
|
||||
|
||||
virtual void serializeBinaryBulkStatePrefix(
|
||||
IDataType::SerializeBinaryBulkSettings & settings,
|
||||
IDataType::SerializeBinaryBulkStatePtr & state) const = 0;
|
||||
|
||||
virtual void serializeBinaryBulkStateSuffix(
|
||||
IDataType::SerializeBinaryBulkSettings & settings,
|
||||
IDataType::SerializeBinaryBulkStatePtr & state) const = 0;
|
||||
|
||||
virtual void deserializeBinaryBulkStatePrefix(
|
||||
IDataType::DeserializeBinaryBulkSettings & settings,
|
||||
IDataType::DeserializeBinaryBulkStatePtr & state) const = 0;
|
||||
|
||||
virtual void serializeBinaryBulkWithMultipleStreams(
|
||||
const IColumn & column,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
IDataType::SerializeBinaryBulkSettings & settings,
|
||||
IDataType::SerializeBinaryBulkStatePtr & state) const = 0;
|
||||
|
||||
virtual void deserializeBinaryBulkWithMultipleStreams(
|
||||
ColumnPtr & column,
|
||||
size_t limit,
|
||||
IDataType::DeserializeBinaryBulkSettings & settings,
|
||||
IDataType::DeserializeBinaryBulkStatePtr & state,
|
||||
IDataType::SubstreamsCache * cache) const = 0;
|
||||
};
|
||||
|
||||
using DataTypeCustomNamePtr = std::unique_ptr<const IDataTypeCustomName>;
|
||||
using DataTypeCustomTextSerializationPtr = std::unique_ptr<const IDataTypeCustomTextSerialization>;
|
||||
using DataTypeCustomStreamsPtr = std::unique_ptr<const IDataTypeCustomStreams>;
|
||||
|
||||
|
||||
/** Describe a data type customization
|
||||
*/
|
||||
struct DataTypeCustomDesc
|
||||
{
|
||||
DataTypeCustomNamePtr name;
|
||||
DataTypeCustomTextSerializationPtr text_serialization;
|
||||
DataTypeCustomStreamsPtr streams;
|
||||
SerializationPtr serialization;
|
||||
|
||||
DataTypeCustomDesc(
|
||||
DataTypeCustomNamePtr name_,
|
||||
DataTypeCustomTextSerializationPtr text_serialization_ = nullptr,
|
||||
DataTypeCustomStreamsPtr streams_ = nullptr)
|
||||
SerializationPtr serialization_ = nullptr)
|
||||
: name(std::move(name_))
|
||||
, text_serialization(std::move(text_serialization_))
|
||||
, streams(std::move(streams_)) {}
|
||||
, serialization(std::move(serialization_)) {}
|
||||
};
|
||||
|
||||
using DataTypeCustomDescPtr = std::unique_ptr<DataTypeCustomDesc>;
|
||||
|
@ -1,7 +1,5 @@
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeCustom.h>
|
||||
#include <DataTypes/DataTypeCustomSimpleTextSerialization.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
@ -9,106 +7,20 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class DataTypeCustomPointSerialization : public DataTypeCustomSimpleTextSerialization
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->serializeAsText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->deserializeAsWholeText(column, istr, settings);
|
||||
}
|
||||
|
||||
static DataTypePtr nestedDataType()
|
||||
{
|
||||
static auto data_type = DataTypePtr(std::make_unique<DataTypeTuple>(
|
||||
DataTypes({std::make_unique<DataTypeFloat64>(), std::make_unique<DataTypeFloat64>()})));
|
||||
return data_type;
|
||||
}
|
||||
};
|
||||
|
||||
class DataTypeCustomRingSerialization : public DataTypeCustomSimpleTextSerialization
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->serializeAsText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->deserializeAsWholeText(column, istr, settings);
|
||||
}
|
||||
|
||||
static DataTypePtr nestedDataType()
|
||||
{
|
||||
static auto data_type = DataTypePtr(std::make_unique<DataTypeArray>(DataTypeCustomPointSerialization::nestedDataType()));
|
||||
return data_type;
|
||||
}
|
||||
};
|
||||
|
||||
class DataTypeCustomPolygonSerialization : public DataTypeCustomSimpleTextSerialization
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->serializeAsText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->deserializeAsWholeText(column, istr, settings);
|
||||
}
|
||||
|
||||
static DataTypePtr nestedDataType()
|
||||
{
|
||||
static auto data_type = DataTypePtr(std::make_unique<DataTypeArray>(DataTypeCustomRingSerialization::nestedDataType()));
|
||||
return data_type;
|
||||
}
|
||||
};
|
||||
|
||||
class DataTypeCustomMultiPolygonSerialization : public DataTypeCustomSimpleTextSerialization
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->serializeAsText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
nestedDataType()->deserializeAsWholeText(column, istr, settings);
|
||||
}
|
||||
|
||||
static DataTypePtr nestedDataType()
|
||||
{
|
||||
static auto data_type = DataTypePtr(std::make_unique<DataTypeArray>(DataTypeCustomPolygonSerialization::nestedDataType()));
|
||||
return data_type;
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
void registerDataTypeDomainGeo(DataTypeFactory & factory)
|
||||
{
|
||||
// Custom type for point represented as its coordinates stored as Tuple(Float64, Float64)
|
||||
factory.registerSimpleDataTypeCustom("Point", []
|
||||
{
|
||||
return std::make_pair(DataTypeFactory::instance().get("Tuple(Float64, Float64)"),
|
||||
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypeCustomFixedName>("Point"), std::make_unique<DataTypeCustomPointSerialization>()));
|
||||
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypeCustomFixedName>("Point")));
|
||||
});
|
||||
|
||||
// Custom type for simple polygon without holes stored as Array(Point)
|
||||
factory.registerSimpleDataTypeCustom("Ring", []
|
||||
{
|
||||
return std::make_pair(DataTypeFactory::instance().get("Array(Point)"),
|
||||
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypeCustomFixedName>("Ring"), std::make_unique<DataTypeCustomRingSerialization>()));
|
||||
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypeCustomFixedName>("Ring")));
|
||||
});
|
||||
|
||||
// Custom type for polygon with holes stored as Array(Ring)
|
||||
@ -116,14 +28,14 @@ void registerDataTypeDomainGeo(DataTypeFactory & factory)
|
||||
factory.registerSimpleDataTypeCustom("Polygon", []
|
||||
{
|
||||
return std::make_pair(DataTypeFactory::instance().get("Array(Ring)"),
|
||||
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypeCustomFixedName>("Polygon"), std::make_unique<DataTypeCustomPolygonSerialization>()));
|
||||
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypeCustomFixedName>("Polygon")));
|
||||
});
|
||||
|
||||
// Custom type for multiple polygons with holes stored as Array(Polygon)
|
||||
factory.registerSimpleDataTypeCustom("MultiPolygon", []
|
||||
{
|
||||
return std::make_pair(DataTypeFactory::instance().get("Array(Polygon)"),
|
||||
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypeCustomFixedName>("MultiPolygon"), std::make_unique<DataTypeCustomMultiPolygonSerialization>()));
|
||||
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypeCustomFixedName>("MultiPolygon")));
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -1,10 +1,6 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/formatIPv6.h>
|
||||
#include <DataTypes/DataTypeCustomSimpleTextSerialization.h>
|
||||
#include <DataTypes/Serializations/SerializationIP.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeCustom.h>
|
||||
#include <Functions/FunctionsCoding.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -15,101 +11,20 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
class DataTypeCustomIPv4Serialization : public DataTypeCustomSimpleTextSerialization
|
||||
{
|
||||
public:
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override
|
||||
{
|
||||
const auto * col = checkAndGetColumn<ColumnUInt32>(&column);
|
||||
if (!col)
|
||||
{
|
||||
throw Exception("IPv4 type can only serialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'};
|
||||
char * ptr = buffer;
|
||||
formatIPv4(reinterpret_cast<const unsigned char *>(&col->getData()[row_num]), ptr);
|
||||
|
||||
ostr.write(buffer, strlen(buffer));
|
||||
}
|
||||
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override
|
||||
{
|
||||
ColumnUInt32 * col = typeid_cast<ColumnUInt32 *>(&column);
|
||||
if (!col)
|
||||
{
|
||||
throw Exception("IPv4 type can only deserialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'};
|
||||
istr.read(buffer, sizeof(buffer) - 1);
|
||||
UInt32 ipv4_value = 0;
|
||||
if (!parseIPv4(buffer, reinterpret_cast<unsigned char *>(&ipv4_value)))
|
||||
{
|
||||
throw Exception("Invalid IPv4 value.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
|
||||
}
|
||||
|
||||
col->insert(ipv4_value);
|
||||
}
|
||||
};
|
||||
|
||||
class DataTypeCustomIPv6Serialization : public DataTypeCustomSimpleTextSerialization
|
||||
{
|
||||
public:
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override
|
||||
{
|
||||
const auto * col = checkAndGetColumn<ColumnFixedString>(&column);
|
||||
if (!col)
|
||||
{
|
||||
throw Exception("IPv6 type domain can only serialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
char buffer[IPV6_MAX_TEXT_LENGTH + 1] = {'\0'};
|
||||
char * ptr = buffer;
|
||||
formatIPv6(reinterpret_cast<const unsigned char *>(col->getDataAt(row_num).data), ptr);
|
||||
|
||||
ostr.write(buffer, strlen(buffer));
|
||||
}
|
||||
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override
|
||||
{
|
||||
ColumnFixedString * col = typeid_cast<ColumnFixedString *>(&column);
|
||||
if (!col)
|
||||
{
|
||||
throw Exception("IPv6 type domain can only deserialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
char buffer[IPV6_MAX_TEXT_LENGTH + 1] = {'\0'};
|
||||
istr.read(buffer, sizeof(buffer) - 1);
|
||||
|
||||
std::string ipv6_value(IPV6_BINARY_LENGTH, '\0');
|
||||
if (!parseIPv6(buffer, reinterpret_cast<unsigned char *>(ipv6_value.data())))
|
||||
{
|
||||
throw Exception("Invalid IPv6 value.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
|
||||
}
|
||||
|
||||
col->insertString(ipv6_value);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataTypeCustom("IPv4", []
|
||||
{
|
||||
return std::make_pair(DataTypeFactory::instance().get("UInt32"),
|
||||
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypeCustomFixedName>("IPv4"), std::make_unique<DataTypeCustomIPv4Serialization>()));
|
||||
auto type = DataTypeFactory::instance().get("UInt32");
|
||||
return std::make_pair(type, std::make_unique<DataTypeCustomDesc>(
|
||||
std::make_unique<DataTypeCustomFixedName>("IPv4"), std::make_unique<SerializationIPv4>(type->getDefaultSerialization())));
|
||||
});
|
||||
|
||||
factory.registerSimpleDataTypeCustom("IPv6", []
|
||||
{
|
||||
return std::make_pair(DataTypeFactory::instance().get("FixedString(16)"),
|
||||
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypeCustomFixedName>("IPv6"), std::make_unique<DataTypeCustomIPv6Serialization>()));
|
||||
auto type = DataTypeFactory::instance().get("FixedString(16)");
|
||||
return std::make_pair(type, std::make_unique<DataTypeCustomDesc>(
|
||||
std::make_unique<DataTypeCustomFixedName>("IPv6"), std::make_unique<SerializationIPv6>(type->getDefaultSerialization())));
|
||||
});
|
||||
|
||||
/// MySQL, MariaDB
|
||||
|
@ -1,18 +1,18 @@
|
||||
#pragma once
|
||||
// #pragma once
|
||||
|
||||
#include <memory>
|
||||
// #include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
// namespace DB
|
||||
// {
|
||||
|
||||
class IDataTypeCustomName;
|
||||
class IDataTypeCustomTextSerialization;
|
||||
class IDataTypeCustomStreams;
|
||||
struct DataTypeCustomDesc;
|
||||
// class IDataTypeCustomName;
|
||||
// class IDataTypeCustomTextSerialization;
|
||||
// class IDataTypeCustomStreams;
|
||||
// struct DataTypeCustomDesc;
|
||||
|
||||
using DataTypeCustomNamePtr = std::unique_ptr<const IDataTypeCustomName>;
|
||||
using DataTypeCustomTextSerializationPtr = std::unique_ptr<const IDataTypeCustomTextSerialization>;
|
||||
using DataTypeCustomStreamsPtr = std::unique_ptr<const IDataTypeCustomStreams>;
|
||||
using DataTypeCustomDescPtr = std::unique_ptr<DataTypeCustomDesc>;
|
||||
// using DataTypeCustomNamePtr = std::unique_ptr<const IDataTypeCustomName>;
|
||||
// using DataTypeCustomTextSerializationPtr = std::unique_ptr<const IDataTypeCustomTextSerialization>;
|
||||
// using DataTypeCustomStreamsPtr = std::unique_ptr<const IDataTypeCustomStreams>;
|
||||
// using DataTypeCustomDescPtr = std::unique_ptr<DataTypeCustomDesc>;
|
||||
|
||||
}
|
||||
// }
|
||||
|
30
src/DataTypes/DataTypeDate.cpp
Normal file
30
src/DataTypes/DataTypeDate.cpp
Normal file
@ -0,0 +1,30 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/Serializations/SerializationDate.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool DataTypeDate::equals(const IDataType & rhs) const
|
||||
{
|
||||
return typeid(rhs) == typeid(*this);
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeDate::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationDate>();
|
||||
}
|
||||
|
||||
void registerDataTypeDate(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataType("Date", [] { return DataTypePtr(std::make_shared<DataTypeDate>()); }, DataTypeFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
26
src/DataTypes/DataTypeDate.h
Normal file
26
src/DataTypes/DataTypeDate.h
Normal file
@ -0,0 +1,26 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class DataTypeDate final : public DataTypeNumberBase<UInt16>
|
||||
{
|
||||
public:
|
||||
static constexpr auto family_name = "Date";
|
||||
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Date; }
|
||||
const char * getFamilyName() const override { return family_name; }
|
||||
|
||||
bool canBeUsedAsVersion() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
protected:
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
};
|
||||
|
||||
}
|
56
src/DataTypes/DataTypeDateTime.cpp
Normal file
56
src/DataTypes/DataTypeDateTime.cpp
Normal file
@ -0,0 +1,56 @@
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/Serializations/SerializationDateTime.h>
|
||||
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <common/DateLUT.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/parseDateTimeBestEffort.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
TimezoneMixin::TimezoneMixin(const String & time_zone_name)
|
||||
: has_explicit_time_zone(!time_zone_name.empty()),
|
||||
time_zone(DateLUT::instance(time_zone_name)),
|
||||
utc_time_zone(DateLUT::instance("UTC"))
|
||||
{}
|
||||
|
||||
DataTypeDateTime::DataTypeDateTime(const String & time_zone_name)
|
||||
: TimezoneMixin(time_zone_name)
|
||||
{
|
||||
}
|
||||
|
||||
DataTypeDateTime::DataTypeDateTime(const TimezoneMixin & time_zone_)
|
||||
: TimezoneMixin(time_zone_)
|
||||
{}
|
||||
|
||||
String DataTypeDateTime::doGetName() const
|
||||
{
|
||||
if (!has_explicit_time_zone)
|
||||
return "DateTime";
|
||||
|
||||
WriteBufferFromOwnString out;
|
||||
out << "DateTime(" << quote << time_zone.getTimeZone() << ")";
|
||||
return out.str();
|
||||
}
|
||||
|
||||
bool DataTypeDateTime::equals(const IDataType & rhs) const
|
||||
{
|
||||
/// DateTime with different timezones are equal, because:
|
||||
/// "all types with different time zones are equivalent and may be used interchangingly."
|
||||
return typeid(rhs) == typeid(*this);
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeDateTime::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationDateTime>(time_zone, utc_time_zone);
|
||||
}
|
||||
|
||||
}
|
70
src/DataTypes/DataTypeDateTime.h
Normal file
70
src/DataTypes/DataTypeDateTime.h
Normal file
@ -0,0 +1,70 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
|
||||
class DateLUTImpl;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Mixin-class that manages timezone info for timezone-aware DateTime implementations
|
||||
*
|
||||
* Must be used as a (second) base for class implementing IDateType-interface.
|
||||
*/
|
||||
class TimezoneMixin
|
||||
{
|
||||
public:
|
||||
explicit TimezoneMixin(const String & time_zone_name = "");
|
||||
TimezoneMixin(const TimezoneMixin &) = default;
|
||||
|
||||
const DateLUTImpl & getTimeZone() const { return time_zone; }
|
||||
|
||||
protected:
|
||||
bool has_explicit_time_zone;
|
||||
const DateLUTImpl & time_zone;
|
||||
const DateLUTImpl & utc_time_zone;
|
||||
};
|
||||
|
||||
/** DateTime stores time as unix timestamp.
|
||||
* The value itself is independent of time zone.
|
||||
*
|
||||
* In binary format it is represented as unix timestamp.
|
||||
* In text format it is serialized to and parsed from YYYY-MM-DD hh:mm:ss format.
|
||||
* The text format is dependent of time zone.
|
||||
*
|
||||
* To cast from/to text format, time zone may be specified explicitly or implicit time zone may be used.
|
||||
*
|
||||
* Time zone may be specified explicitly as type parameter, example: DateTime('Europe/Moscow').
|
||||
* As it does not affect the internal representation of values,
|
||||
* all types with different time zones are equivalent and may be used interchangingly.
|
||||
* Time zone only affects parsing and displaying in text formats.
|
||||
*
|
||||
* If time zone is not specified (example: DateTime without parameter), then default time zone is used.
|
||||
* Default time zone is server time zone, if server is doing transformations
|
||||
* and if client is doing transformations, unless 'use_client_time_zone' setting is passed to client;
|
||||
* Server time zone is the time zone specified in 'timezone' parameter in configuration file,
|
||||
* or system time zone at the moment of server startup.
|
||||
*/
|
||||
class DataTypeDateTime final : public DataTypeNumberBase<UInt32>, public TimezoneMixin
|
||||
{
|
||||
public:
|
||||
explicit DataTypeDateTime(const String & time_zone_name = "");
|
||||
explicit DataTypeDateTime(const TimezoneMixin & time_zone);
|
||||
|
||||
static constexpr auto family_name = "DateTime";
|
||||
|
||||
const char * getFamilyName() const override { return family_name; }
|
||||
String doGetName() const override;
|
||||
TypeIndex getTypeId() const override { return TypeIndex::DateTime; }
|
||||
|
||||
bool canBeUsedAsVersion() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
};
|
||||
|
||||
}
|
||||
|
71
src/DataTypes/DataTypeDateTime64.cpp
Normal file
71
src/DataTypes/DataTypeDateTime64.cpp
Normal file
@ -0,0 +1,71 @@
|
||||
#include <DataTypes/DataTypeDateTime64.h>
|
||||
#include <DataTypes/Serializations/SerializationDateTime64.h>
|
||||
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <common/DateLUT.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/parseDateTimeBestEffort.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
#include <optional>
|
||||
#include <string>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
static constexpr UInt32 max_scale = 9;
|
||||
|
||||
DataTypeDateTime64::DataTypeDateTime64(UInt32 scale_, const std::string & time_zone_name)
|
||||
: DataTypeDecimalBase<DateTime64>(DecimalUtils::max_precision<DateTime64>, scale_),
|
||||
TimezoneMixin(time_zone_name)
|
||||
{
|
||||
if (scale > max_scale)
|
||||
throw Exception("Scale " + std::to_string(scale) + " is too large for DateTime64. Maximum is up to nanoseconds (9).",
|
||||
ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
}
|
||||
|
||||
DataTypeDateTime64::DataTypeDateTime64(UInt32 scale_, const TimezoneMixin & time_zone_info)
|
||||
: DataTypeDecimalBase<DateTime64>(DecimalUtils::max_precision<DateTime64>, scale_),
|
||||
TimezoneMixin(time_zone_info)
|
||||
{
|
||||
if (scale > max_scale)
|
||||
throw Exception("Scale " + std::to_string(scale) + " is too large for DateTime64. Maximum is up to nanoseconds (9).",
|
||||
ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
}
|
||||
|
||||
std::string DataTypeDateTime64::doGetName() const
|
||||
{
|
||||
if (!has_explicit_time_zone)
|
||||
return std::string(getFamilyName()) + "(" + std::to_string(this->scale) + ")";
|
||||
|
||||
WriteBufferFromOwnString out;
|
||||
out << "DateTime64(" << this->scale << ", " << quote << time_zone.getTimeZone() << ")";
|
||||
return out.str();
|
||||
}
|
||||
|
||||
bool DataTypeDateTime64::equals(const IDataType & rhs) const
|
||||
{
|
||||
if (const auto * ptype = typeid_cast<const DataTypeDateTime64 *>(&rhs))
|
||||
return this->scale == ptype->getScale();
|
||||
return false;
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeDateTime64::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationDateTime64>(time_zone, utc_time_zone, scale);
|
||||
}
|
||||
|
||||
}
|
104
src/DataTypes/DataTypeDateTime64.h
Normal file
104
src/DataTypes/DataTypeDateTime64.h
Normal file
@ -0,0 +1,104 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeDecimalBase.h>
|
||||
|
||||
class DateLUTImpl;
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** DateTime64 is same as DateTime, but it stores values as Int64 and has configurable sub-second part.
|
||||
*
|
||||
* `scale` determines number of decimal places for sub-second part of the DateTime64.
|
||||
*/
|
||||
class DataTypeDateTime64 final : public DataTypeDecimalBase<DateTime64>, public TimezoneMixin
|
||||
{
|
||||
public:
|
||||
using Base = DataTypeDecimalBase<DateTime64>;
|
||||
static constexpr UInt8 default_scale = 3;
|
||||
|
||||
static constexpr auto family_name = "DateTime64";
|
||||
static constexpr auto type_id = TypeIndex::DateTime64;
|
||||
|
||||
explicit DataTypeDateTime64(UInt32 scale_, const std::string & time_zone_name = "");
|
||||
|
||||
// reuse timezone from other DateTime/DateTime64
|
||||
DataTypeDateTime64(UInt32 scale_, const TimezoneMixin & time_zone_info);
|
||||
|
||||
const char * getFamilyName() const override { return family_name; }
|
||||
std::string doGetName() const override;
|
||||
TypeIndex getTypeId() const override { return type_id; }
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
bool canBePromoted() const override { return false; }
|
||||
|
||||
protected:
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
};
|
||||
|
||||
/** Tansform-type wrapper for DateTime64, applies given Transform to DateTime64 value or only to a whole part of it.
|
||||
*
|
||||
* Depending on what overloads of Transform::execute() are available, when called with DateTime64 value,
|
||||
* invokes Transform::execute() with:
|
||||
* * whole part of DateTime64 value, discarding fractional part.
|
||||
* * DateTime64 value and scale factor.
|
||||
*
|
||||
* Suitable Transfotm-types are commonly used in Date/DateTime manipulation functions,
|
||||
* and should implement static (or const) function with following signatures:
|
||||
* R execute(UInt32 whole_value, ... , const TimeZoneImpl &)
|
||||
* OR
|
||||
* R execute(DateTime64 value, Int64 scale_factor, ... , const TimeZoneImpl &)
|
||||
*
|
||||
* Where R and T could be arbitrary types.
|
||||
*/
|
||||
template <typename Transform>
|
||||
class TransformDateTime64 : public Transform
|
||||
{
|
||||
private:
|
||||
// Detect if Transform::execute is const or static method
|
||||
// with signature defined by template args (ignoring result type).
|
||||
template<typename = void, typename... Args>
|
||||
struct TransformHasExecuteOverload : std::false_type {};
|
||||
|
||||
template<typename... Args>
|
||||
struct TransformHasExecuteOverload<std::void_t<decltype(std::declval<Transform>().execute(std::declval<Args>()...))>, Args...>
|
||||
: std::true_type {};
|
||||
|
||||
template<typename... Args>
|
||||
static constexpr bool TransformHasExecuteOverload_v = TransformHasExecuteOverload<void, Args...>::value;
|
||||
|
||||
public:
|
||||
static constexpr auto name = Transform::name;
|
||||
|
||||
using Transform::execute;
|
||||
|
||||
// non-explicit constructor to allow creating from scale value (or with no scale at all), indispensable in some contexts.
|
||||
TransformDateTime64(UInt32 scale_ = 0)
|
||||
: scale_multiplier(DecimalUtils::scaleMultiplier<DateTime64::NativeType>(scale_))
|
||||
{}
|
||||
|
||||
template <typename ... Args>
|
||||
inline auto execute(const DateTime64 & t, Args && ... args) const
|
||||
{
|
||||
const auto transform = static_cast<const Transform *>(this);
|
||||
|
||||
if constexpr (TransformHasExecuteOverload_v<DateTime64, decltype(scale_multiplier), Args...>)
|
||||
{
|
||||
return transform->execute(t, scale_multiplier, std::forward<Args>(args)...);
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto components = DecimalUtils::splitWithScaleMultiplier(t, scale_multiplier);
|
||||
return transform->execute(static_cast<UInt32>(components.whole), std::forward<Args>(args)...);
|
||||
}
|
||||
}
|
||||
|
||||
private:
|
||||
DateTime64::NativeType scale_multiplier = 1;
|
||||
};
|
||||
|
||||
}
|
||||
|
52
src/DataTypes/DataTypeDecimalBase.cpp
Normal file
52
src/DataTypes/DataTypeDecimalBase.cpp
Normal file
@ -0,0 +1,52 @@
|
||||
#include <DataTypes/DataTypeDecimalBase.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
#include <type_traits>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
bool decimalCheckComparisonOverflow(const Context & context) { return context.getSettingsRef().decimal_check_overflow; }
|
||||
bool decimalCheckArithmeticOverflow(const Context & context) { return context.getSettingsRef().decimal_check_overflow; }
|
||||
|
||||
template <typename T>
|
||||
Field DataTypeDecimalBase<T>::getDefault() const
|
||||
{
|
||||
return DecimalField(T(0), scale);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
MutableColumnPtr DataTypeDecimalBase<T>::createColumn() const
|
||||
{
|
||||
return ColumnType::create(0, scale);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
T DataTypeDecimalBase<T>::getScaleMultiplier(UInt32 scale_)
|
||||
{
|
||||
return DecimalUtils::scaleMultiplier<typename T::NativeType>(scale_);
|
||||
}
|
||||
|
||||
|
||||
/// Explicit template instantiations.
|
||||
template class DataTypeDecimalBase<Decimal32>;
|
||||
template class DataTypeDecimalBase<Decimal64>;
|
||||
template class DataTypeDecimalBase<Decimal128>;
|
||||
template class DataTypeDecimalBase<Decimal256>;
|
||||
template class DataTypeDecimalBase<DateTime64>;
|
||||
|
||||
}
|
206
src/DataTypes/DataTypeDecimalBase.h
Normal file
206
src/DataTypes/DataTypeDecimalBase.h
Normal file
@ -0,0 +1,206 @@
|
||||
#pragma once
|
||||
#include <cmath>
|
||||
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
#include <Core/DecimalFunctions.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
#include <type_traits>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
class Context;
|
||||
bool decimalCheckComparisonOverflow(const Context & context);
|
||||
bool decimalCheckArithmeticOverflow(const Context & context);
|
||||
|
||||
inline UInt32 leastDecimalPrecisionFor(TypeIndex int_type)
|
||||
{
|
||||
switch (int_type)
|
||||
{
|
||||
case TypeIndex::Int8: [[fallthrough]];
|
||||
case TypeIndex::UInt8:
|
||||
return 3;
|
||||
case TypeIndex::Int16: [[fallthrough]];
|
||||
case TypeIndex::UInt16:
|
||||
return 5;
|
||||
case TypeIndex::Int32: [[fallthrough]];
|
||||
case TypeIndex::UInt32:
|
||||
return 10;
|
||||
case TypeIndex::Int64:
|
||||
return 19;
|
||||
case TypeIndex::UInt64:
|
||||
return 20;
|
||||
default:
|
||||
break;
|
||||
}
|
||||
return 0;
|
||||
}
|
||||
|
||||
/// Base class for decimals, like Decimal(P, S), where P is precision, S is scale.
|
||||
/// Maximum precisions for underlying types are:
|
||||
/// Int32 9
|
||||
/// Int64 18
|
||||
/// Int128 38
|
||||
/// Int256 76
|
||||
/// Operation between two decimals leads to Decimal(P, S), where
|
||||
/// P is one of (9, 18, 38, 76); equals to the maximum precision for the biggest underlying type of operands.
|
||||
/// S is maximum scale of operands. The allowed valuas are [0, precision]
|
||||
template <typename T>
|
||||
class DataTypeDecimalBase : public IDataType
|
||||
{
|
||||
static_assert(IsDecimalNumber<T>);
|
||||
|
||||
public:
|
||||
using FieldType = T;
|
||||
using ColumnType = ColumnDecimal<T>;
|
||||
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
static constexpr size_t maxPrecision() { return DecimalUtils::max_precision<T>; }
|
||||
|
||||
DataTypeDecimalBase(UInt32 precision_, UInt32 scale_)
|
||||
: precision(precision_),
|
||||
scale(scale_)
|
||||
{
|
||||
if (unlikely(precision < 1 || precision > maxPrecision()))
|
||||
throw Exception("Precision " + std::to_string(precision) + " is out of bounds", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
if (unlikely(scale > maxPrecision()))
|
||||
throw Exception("Scale " + std::to_string(scale) + " is out of bounds", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
}
|
||||
|
||||
TypeIndex getTypeId() const override { return TypeId<T>::value; }
|
||||
|
||||
Field getDefault() const override;
|
||||
MutableColumnPtr createColumn() const override;
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool shouldAlignRightInPrettyFormats() const override { return true; }
|
||||
bool textCanContainOnlyValidUTF8() const override { return true; }
|
||||
bool isComparable() const override { return true; }
|
||||
bool isValueRepresentedByNumber() const override { return true; }
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
bool haveMaximumSizeOfValue() const override { return true; }
|
||||
size_t getSizeOfValueInMemory() const override { return sizeof(T); }
|
||||
|
||||
bool isSummable() const override { return true; }
|
||||
bool canBeUsedInBooleanContext() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
||||
/// Decimal specific
|
||||
|
||||
UInt32 getPrecision() const { return precision; }
|
||||
UInt32 getScale() const { return scale; }
|
||||
T getScaleMultiplier() const { return getScaleMultiplier(scale); }
|
||||
|
||||
T wholePart(T x) const
|
||||
{
|
||||
return DecimalUtils::getWholePart(x, scale);
|
||||
}
|
||||
|
||||
T fractionalPart(T x) const
|
||||
{
|
||||
return DecimalUtils::getFractionalPart(x, scale);
|
||||
}
|
||||
|
||||
T maxWholeValue() const { return getScaleMultiplier(precision - scale) - T(1); }
|
||||
|
||||
template<typename U>
|
||||
bool canStoreWhole(U x) const
|
||||
{
|
||||
static_assert(std::is_signed_v<typename T::NativeType>);
|
||||
T max = maxWholeValue();
|
||||
if constexpr (std::is_signed_v<U>)
|
||||
return -max <= x && x <= max;
|
||||
else
|
||||
return x <= static_cast<std::make_unsigned_t<typename T::NativeType>>(max.value);
|
||||
}
|
||||
|
||||
/// @returns multiplier for U to become T with correct scale
|
||||
template <typename U>
|
||||
T scaleFactorFor(const DataTypeDecimalBase<U> & x, bool) const
|
||||
{
|
||||
if (getScale() < x.getScale())
|
||||
throw Exception("Decimal result's scale is less than argument's one", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
UInt32 scale_delta = getScale() - x.getScale(); /// scale_delta >= 0
|
||||
return getScaleMultiplier(scale_delta);
|
||||
}
|
||||
|
||||
template <typename U>
|
||||
T scaleFactorFor(const DataTypeNumber<U> & , bool is_multiply_or_divisor) const
|
||||
{
|
||||
if (is_multiply_or_divisor)
|
||||
return T(1);
|
||||
return getScaleMultiplier();
|
||||
}
|
||||
|
||||
static T getScaleMultiplier(UInt32 scale);
|
||||
|
||||
inline DecimalUtils::DataTypeDecimalTrait<T> getTrait() const
|
||||
{
|
||||
return {precision, scale};
|
||||
}
|
||||
|
||||
protected:
|
||||
const UInt32 precision;
|
||||
const UInt32 scale;
|
||||
};
|
||||
|
||||
|
||||
template <typename T>
|
||||
inline const DataTypeDecimalBase<T> * checkDecimalBase(const IDataType & data_type)
|
||||
{
|
||||
if (isColumnedAsDecimalT<T>(data_type))
|
||||
return static_cast<const DataTypeDecimalBase<T> *>(&data_type);
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
template <bool is_multiply, bool is_division, typename T, typename U, template <typename> typename DecimalType>
|
||||
inline auto decimalResultType(const DecimalType<T> & tx, const DecimalType<U> & ty)
|
||||
{
|
||||
const auto result_trait = DecimalUtils::binaryOpResult<is_multiply, is_division>(tx, ty);
|
||||
return DecimalType<typename decltype(result_trait)::FieldType>(result_trait.precision, result_trait.scale);
|
||||
}
|
||||
|
||||
template <bool is_multiply, bool is_division, typename T, typename U, template <typename> typename DecimalType>
|
||||
inline const DecimalType<T> decimalResultType(const DecimalType<T> & tx, const DataTypeNumber<U> & ty)
|
||||
{
|
||||
const auto result_trait = DecimalUtils::binaryOpResult<is_multiply, is_division>(tx, ty);
|
||||
return DecimalType<typename decltype(result_trait)::FieldType>(result_trait.precision, result_trait.scale);
|
||||
}
|
||||
|
||||
template <bool is_multiply, bool is_division, typename T, typename U, template <typename> typename DecimalType>
|
||||
inline const DecimalType<U> decimalResultType(const DataTypeNumber<T> & tx, const DecimalType<U> & ty)
|
||||
{
|
||||
const auto result_trait = DecimalUtils::binaryOpResult<is_multiply, is_division>(tx, ty);
|
||||
return DecimalType<typename decltype(result_trait)::FieldType>(result_trait.precision, result_trait.scale);
|
||||
}
|
||||
|
||||
template <template <typename> typename DecimalType>
|
||||
inline DataTypePtr createDecimal(UInt64 precision_value, UInt64 scale_value)
|
||||
{
|
||||
if (precision_value < DecimalUtils::min_precision || precision_value > DecimalUtils::max_precision<Decimal256>)
|
||||
throw Exception("Wrong precision", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
if (static_cast<UInt64>(scale_value) > precision_value)
|
||||
throw Exception("Negative scales and scales larger than precision are not supported", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
if (precision_value <= DecimalUtils::max_precision<Decimal32>)
|
||||
return std::make_shared<DecimalType<Decimal32>>(precision_value, scale_value);
|
||||
else if (precision_value <= DecimalUtils::max_precision<Decimal64>)
|
||||
return std::make_shared<DecimalType<Decimal64>>(precision_value, scale_value);
|
||||
else if (precision_value <= DecimalUtils::max_precision<Decimal128>)
|
||||
return std::make_shared<DecimalType<Decimal128>>(precision_value, scale_value);
|
||||
return std::make_shared<DecimalType<Decimal256>>(precision_value, scale_value);
|
||||
}
|
||||
|
||||
}
|
267
src/DataTypes/DataTypeEnum.cpp
Normal file
267
src/DataTypes/DataTypeEnum.cpp
Normal file
@ -0,0 +1,267 @@
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/Serializations/SerializationEnum.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/UTF8Helpers.h>
|
||||
#include <Poco/UTF8Encoding.h>
|
||||
|
||||
#include <limits>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_TYPE_OF_FIELD;
|
||||
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)
|
||||
{
|
||||
WriteBufferFromOwnString out;
|
||||
|
||||
writeString(EnumName<FieldType>::value, out);
|
||||
writeChar('(', out);
|
||||
|
||||
auto first = true;
|
||||
for (const auto & name_and_value : values)
|
||||
{
|
||||
if (!first)
|
||||
writeString(", ", out);
|
||||
|
||||
first = false;
|
||||
|
||||
writeQuotedString(name_and_value.first, out);
|
||||
writeString(" = ", out);
|
||||
writeText(name_and_value.second, out);
|
||||
}
|
||||
|
||||
writeChar(')', out);
|
||||
|
||||
return out.str();
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
DataTypeEnum<Type>::DataTypeEnum(const Values & values_)
|
||||
: EnumValues<Type>(values_)
|
||||
, type_name(generateName(this->getValues()))
|
||||
{
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
Field DataTypeEnum<Type>::getDefault() const
|
||||
{
|
||||
return this->getValues().front().second;
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
void DataTypeEnum<Type>::insertDefaultInto(IColumn & column) const
|
||||
{
|
||||
assert_cast<ColumnType &>(column).getData().push_back(this->getValues().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 : this->getValues())
|
||||
{
|
||||
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)
|
||||
{
|
||||
this->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 this->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 this->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);
|
||||
this->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);
|
||||
}
|
||||
|
||||
|
||||
template <typename Type>
|
||||
bool DataTypeEnum<Type>::contains(const IDataType & rhs) const
|
||||
{
|
||||
if (const auto * rhs_enum8 = typeid_cast<const DataTypeEnum8 *>(&rhs))
|
||||
return this->containsAll(rhs_enum8->getValues());
|
||||
if (const auto * rhs_enum16 = typeid_cast<const DataTypeEnum16 *>(&rhs))
|
||||
return this->containsAll(rhs_enum16->getValues());
|
||||
return false;
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
SerializationPtr DataTypeEnum<Type>::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationEnum<Type>>(this->getValues());
|
||||
}
|
||||
|
||||
|
||||
/// Explicit instantiations.
|
||||
template class DataTypeEnum<Int8>;
|
||||
template class DataTypeEnum<Int16>;
|
||||
|
||||
static void checkASTStructure(const ASTPtr & child)
|
||||
{
|
||||
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);
|
||||
}
|
||||
|
||||
template <typename DataTypeEnum>
|
||||
static DataTypePtr createExact(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)
|
||||
{
|
||||
checkASTStructure(child);
|
||||
|
||||
const auto * func = child->as<ASTFunction>();
|
||||
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>();
|
||||
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);
|
||||
}
|
||||
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->children.empty())
|
||||
throw Exception("Enum data type cannot be empty", ErrorCodes::EMPTY_DATA_PASSED);
|
||||
|
||||
/// Children must be functions 'equals' with string literal as left argument and numeric literal as right argument.
|
||||
for (const ASTPtr & child : arguments->children)
|
||||
{
|
||||
checkASTStructure(child);
|
||||
|
||||
const auto * func = child->as<ASTFunction>();
|
||||
const auto * value_literal = func->arguments->children[1]->as<ASTLiteral>();
|
||||
|
||||
if (!value_literal
|
||||
|| (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);
|
||||
|
||||
Int64 value = value_literal->value.get<Int64>();
|
||||
|
||||
if (value > std::numeric_limits<Int8>::max() || value < std::numeric_limits<Int8>::min())
|
||||
return createExact<DataTypeEnum16>(arguments);
|
||||
}
|
||||
|
||||
return createExact<DataTypeEnum8>(arguments);
|
||||
}
|
||||
|
||||
void registerDataTypeEnum(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerDataType("Enum8", createExact<DataTypeEnum<Int8>>);
|
||||
factory.registerDataType("Enum16", createExact<DataTypeEnum<Int16>>);
|
||||
factory.registerDataType("Enum", create);
|
||||
}
|
||||
|
||||
}
|
88
src/DataTypes/DataTypeEnum.h
Normal file
88
src/DataTypes/DataTypeEnum.h
Normal file
@ -0,0 +1,88 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/EnumValues.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <vector>
|
||||
#include <unordered_map>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IDataTypeEnum : public IDataType
|
||||
{
|
||||
public:
|
||||
virtual Field castToName(const Field & value_or_name) const = 0;
|
||||
virtual Field castToValue(const Field & value_or_name) const = 0;
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool isValueRepresentedByNumber() const override { return true; }
|
||||
bool isValueRepresentedByInteger() const override { return true; }
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
bool haveMaximumSizeOfValue() const override { return true; }
|
||||
bool isCategorial() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
bool isComparable() const override { return true; }
|
||||
};
|
||||
|
||||
|
||||
template <typename Type>
|
||||
class DataTypeEnum final : public IDataTypeEnum, public EnumValues<Type>
|
||||
{
|
||||
public:
|
||||
using FieldType = Type;
|
||||
using ColumnType = ColumnVector<FieldType>;
|
||||
using typename EnumValues<Type>::Values;
|
||||
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
private:
|
||||
std::string type_name;
|
||||
static std::string generateName(const Values & values);
|
||||
|
||||
public:
|
||||
explicit DataTypeEnum(const Values & values_);
|
||||
|
||||
std::string doGetName() const override { return type_name; }
|
||||
const char * getFamilyName() const override;
|
||||
|
||||
TypeIndex getTypeId() const override { return sizeof(FieldType) == 1 ? TypeIndex::Enum8 : TypeIndex::Enum16; }
|
||||
|
||||
FieldType readValue(ReadBuffer & istr) const
|
||||
{
|
||||
FieldType x;
|
||||
readText(x, istr);
|
||||
return this->findByValue(x)->first;
|
||||
}
|
||||
|
||||
Field castToName(const Field & value_or_name) const override;
|
||||
Field castToValue(const Field & value_or_name) const override;
|
||||
|
||||
MutableColumnPtr createColumn() const override { return ColumnType::create(); }
|
||||
|
||||
Field getDefault() const override;
|
||||
void insertDefaultInto(IColumn & column) const override;
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
bool textCanContainOnlyValidUTF8() const override;
|
||||
size_t getSizeOfValueInMemory() const override { return sizeof(FieldType); }
|
||||
|
||||
/// Check current Enum type extends another Enum type (contains all the same values and doesn't override name's with other values)
|
||||
/// Example:
|
||||
/// Enum('a' = 1, 'b' = 2) -> Enum('c' = 1, 'b' = 2, 'd' = 3) OK
|
||||
/// Enum('a' = 1, 'b' = 2) -> Enum('a' = 2, 'b' = 1) NOT OK
|
||||
bool contains(const IDataType & rhs) const;
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
};
|
||||
|
||||
|
||||
using DataTypeEnum8 = DataTypeEnum<Int8>;
|
||||
using DataTypeEnum16 = DataTypeEnum<Int16>;
|
||||
|
||||
}
|
@ -3,7 +3,7 @@
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Parsers/IAST_fwd.h>
|
||||
#include <Common/IFactoryWithAliases.h>
|
||||
#include <DataTypes/DataTypeCustom_fwd.h>
|
||||
#include <DataTypes/DataTypeCustom.h>
|
||||
|
||||
|
||||
#include <functional>
|
||||
@ -86,6 +86,5 @@ void registerDataTypeLowCardinality(DataTypeFactory & factory);
|
||||
void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory);
|
||||
void registerDataTypeDomainSimpleAggregateFunction(DataTypeFactory & factory);
|
||||
void registerDataTypeDomainGeo(DataTypeFactory & factory);
|
||||
void registerDataTypeOneElementTuple(DataTypeFactory & factory);
|
||||
|
||||
}
|
||||
|
79
src/DataTypes/DataTypeFixedString.cpp
Normal file
79
src/DataTypes/DataTypeFixedString.cpp
Normal file
@ -0,0 +1,79 @@
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/Serializations/SerializationFixedString.h>
|
||||
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/VarInt.h>
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int UNEXPECTED_AST_STRUCTURE;
|
||||
extern const int TOO_LARGE_STRING_SIZE;
|
||||
}
|
||||
|
||||
|
||||
std::string DataTypeFixedString::doGetName() const
|
||||
{
|
||||
return "FixedString(" + toString(n) + ")";
|
||||
}
|
||||
|
||||
MutableColumnPtr DataTypeFixedString::createColumn() const
|
||||
{
|
||||
return ColumnFixedString::create(n);
|
||||
}
|
||||
|
||||
Field DataTypeFixedString::getDefault() const
|
||||
{
|
||||
return String();
|
||||
}
|
||||
|
||||
bool DataTypeFixedString::equals(const IDataType & rhs) const
|
||||
{
|
||||
return typeid(rhs) == typeid(*this) && n == static_cast<const DataTypeFixedString &>(rhs).n;
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeFixedString::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationFixedString>(n);
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->children.size() != 1)
|
||||
throw Exception("FixedString data type family must have exactly one argument - size in bytes", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const auto * argument = arguments->children[0]->as<ASTLiteral>();
|
||||
if (!argument || argument->value.getType() != Field::Types::UInt64 || argument->value.get<UInt64>() == 0)
|
||||
throw Exception("FixedString data type family must have a number (positive integer) as its argument", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||
|
||||
return std::make_shared<DataTypeFixedString>(argument->value.get<UInt64>());
|
||||
}
|
||||
|
||||
|
||||
void registerDataTypeFixedString(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerDataType("FixedString", create);
|
||||
|
||||
/// Compatibility alias.
|
||||
factory.registerAlias("BINARY", "FixedString", DataTypeFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
69
src/DataTypes/DataTypeFixedString.h
Normal file
69
src/DataTypes/DataTypeFixedString.h
Normal file
@ -0,0 +1,69 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Common/PODArray_fwd.h>
|
||||
|
||||
#define MAX_FIXEDSTRING_SIZE 0xFFFFFF
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
|
||||
class DataTypeFixedString final : public IDataType
|
||||
{
|
||||
private:
|
||||
size_t n;
|
||||
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
DataTypeFixedString(size_t n_) : n(n_)
|
||||
{
|
||||
if (n == 0)
|
||||
throw Exception("FixedString size must be positive", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
if (n > MAX_FIXEDSTRING_SIZE)
|
||||
throw Exception("FixedString size is too large", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
}
|
||||
|
||||
std::string doGetName() const override;
|
||||
TypeIndex getTypeId() const override { return TypeIndex::FixedString; }
|
||||
|
||||
const char * getFamilyName() const override { return "FixedString"; }
|
||||
|
||||
size_t getN() const
|
||||
{
|
||||
return n;
|
||||
}
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool isComparable() const override { return true; }
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
bool isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion() const override { return true; }
|
||||
bool haveMaximumSizeOfValue() const override { return true; }
|
||||
size_t getSizeOfValueInMemory() const override { return n; }
|
||||
bool isCategorial() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
bool canBeInsideLowCardinality() const override { return true; }
|
||||
|
||||
/// Makes sure that the length of a newly inserted string to `chars` is equal to getN().
|
||||
/// If the length is less than getN() the function will add zero characters up to getN().
|
||||
/// If the length is greater than getN() the function will throw an exception.
|
||||
void alignStringLength(PaddedPODArray<UInt8> & chars, size_t old_size) const;
|
||||
};
|
||||
|
||||
}
|
167
src/DataTypes/DataTypeLowCardinality.cpp
Normal file
167
src/DataTypes/DataTypeLowCardinality.cpp
Normal file
@ -0,0 +1,167 @@
|
||||
#include <Columns/ColumnLowCardinality.h>
|
||||
#include <Columns/ColumnUnique.h>
|
||||
#include <Columns/ColumnFixedString.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Core/TypeListNumber.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/Serializations/SerializationLowCardinality.h>
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
DataTypeLowCardinality::DataTypeLowCardinality(DataTypePtr dictionary_type_)
|
||||
: dictionary_type(std::move(dictionary_type_))
|
||||
{
|
||||
auto inner_type = dictionary_type;
|
||||
if (dictionary_type->isNullable())
|
||||
inner_type = static_cast<const DataTypeNullable &>(*dictionary_type).getNestedType();
|
||||
|
||||
if (!inner_type->canBeInsideLowCardinality())
|
||||
throw Exception("DataTypeLowCardinality is supported only for numbers, strings, Date or DateTime, but got "
|
||||
+ dictionary_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
template <typename Creator>
|
||||
struct CreateColumnVector
|
||||
{
|
||||
MutableColumnUniquePtr & column;
|
||||
const IDataType & keys_type;
|
||||
const Creator & creator;
|
||||
|
||||
CreateColumnVector(MutableColumnUniquePtr & column_, const IDataType & keys_type_, const Creator & creator_)
|
||||
: column(column_), keys_type(keys_type_), creator(creator_)
|
||||
{
|
||||
}
|
||||
|
||||
template <typename T, size_t>
|
||||
void operator()()
|
||||
{
|
||||
if (typeid_cast<const DataTypeNumber<T> *>(&keys_type))
|
||||
column = creator(static_cast<ColumnVector<T> *>(nullptr));
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
template <typename Creator>
|
||||
MutableColumnUniquePtr DataTypeLowCardinality::createColumnUniqueImpl(const IDataType & keys_type,
|
||||
const Creator & creator)
|
||||
{
|
||||
const auto * type = &keys_type;
|
||||
if (const auto * nullable_type = typeid_cast<const DataTypeNullable *>(&keys_type))
|
||||
type = nullable_type->getNestedType().get();
|
||||
|
||||
WhichDataType which(type);
|
||||
|
||||
if (which.isString())
|
||||
return creator(static_cast<ColumnString *>(nullptr));
|
||||
else if (which.isFixedString())
|
||||
return creator(static_cast<ColumnFixedString *>(nullptr));
|
||||
else if (which.isDate())
|
||||
return creator(static_cast<ColumnVector<UInt16> *>(nullptr));
|
||||
else if (which.isDateTime())
|
||||
return creator(static_cast<ColumnVector<UInt32> *>(nullptr));
|
||||
else if (which.isInt() || which.isUInt() || which.isFloat())
|
||||
{
|
||||
MutableColumnUniquePtr column;
|
||||
TypeListNativeNumbers::forEach(CreateColumnVector(column, *type, creator));
|
||||
|
||||
if (!column)
|
||||
throw Exception("Unexpected numeric type: " + type->getName(), ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
return column;
|
||||
}
|
||||
|
||||
throw Exception("Unexpected dictionary type for DataTypeLowCardinality: " + type->getName(),
|
||||
ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
|
||||
MutableColumnUniquePtr DataTypeLowCardinality::createColumnUnique(const IDataType & keys_type)
|
||||
{
|
||||
auto creator = [&](auto x)
|
||||
{
|
||||
using ColumnType = typename std::remove_pointer<decltype(x)>::type;
|
||||
return ColumnUnique<ColumnType>::create(keys_type);
|
||||
};
|
||||
return createColumnUniqueImpl(keys_type, creator);
|
||||
}
|
||||
|
||||
MutableColumnUniquePtr DataTypeLowCardinality::createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys)
|
||||
{
|
||||
auto creator = [&](auto x)
|
||||
{
|
||||
using ColumnType = typename std::remove_pointer<decltype(x)>::type;
|
||||
return ColumnUnique<ColumnType>::create(std::move(keys), keys_type.isNullable());
|
||||
};
|
||||
return createColumnUniqueImpl(keys_type, creator);
|
||||
}
|
||||
|
||||
MutableColumnPtr DataTypeLowCardinality::createColumn() const
|
||||
{
|
||||
MutableColumnPtr indexes = DataTypeUInt8().createColumn();
|
||||
MutableColumnPtr dictionary = createColumnUnique(*dictionary_type);
|
||||
return ColumnLowCardinality::create(std::move(dictionary), std::move(indexes));
|
||||
}
|
||||
|
||||
Field DataTypeLowCardinality::getDefault() const
|
||||
{
|
||||
return dictionary_type->getDefault();
|
||||
}
|
||||
|
||||
bool DataTypeLowCardinality::equals(const IDataType & rhs) const
|
||||
{
|
||||
if (typeid(rhs) != typeid(*this))
|
||||
return false;
|
||||
|
||||
const auto & low_cardinality_rhs= static_cast<const DataTypeLowCardinality &>(rhs);
|
||||
return dictionary_type->equals(*low_cardinality_rhs.dictionary_type);
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeLowCardinality::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationLowCardinality>(dictionary_type);
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->children.size() != 1)
|
||||
throw Exception("LowCardinality data type family must have single argument - type of elements",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
return std::make_shared<DataTypeLowCardinality>(DataTypeFactory::instance().get(arguments->children[0]));
|
||||
}
|
||||
|
||||
void registerDataTypeLowCardinality(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerDataType("LowCardinality", create);
|
||||
}
|
||||
|
||||
|
||||
DataTypePtr removeLowCardinality(const DataTypePtr & type)
|
||||
{
|
||||
if (const auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(type.get()))
|
||||
return low_cardinality_type->getDictionaryType();
|
||||
return type;
|
||||
}
|
||||
|
||||
}
|
89
src/DataTypes/DataTypeLowCardinality.h
Normal file
89
src/DataTypes/DataTypeLowCardinality.h
Normal file
@ -0,0 +1,89 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Columns/IColumnUnique.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class DataTypeLowCardinality : public IDataType
|
||||
{
|
||||
private:
|
||||
DataTypePtr dictionary_type;
|
||||
|
||||
public:
|
||||
DataTypeLowCardinality(DataTypePtr dictionary_type_);
|
||||
|
||||
const DataTypePtr & getDictionaryType() const { return dictionary_type; }
|
||||
|
||||
String doGetName() const override
|
||||
{
|
||||
return "LowCardinality(" + dictionary_type->getName() + ")";
|
||||
}
|
||||
const char * getFamilyName() const override { return "LowCardinality"; }
|
||||
TypeIndex getTypeId() const override { return TypeIndex::LowCardinality; }
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return true; }
|
||||
bool cannotBeStoredInTables() const override { return dictionary_type->cannotBeStoredInTables(); }
|
||||
bool shouldAlignRightInPrettyFormats() const override { return dictionary_type->shouldAlignRightInPrettyFormats(); }
|
||||
bool textCanContainOnlyValidUTF8() const override { return dictionary_type->textCanContainOnlyValidUTF8(); }
|
||||
bool isComparable() const override { return dictionary_type->isComparable(); }
|
||||
bool canBeComparedWithCollation() const override { return dictionary_type->canBeComparedWithCollation(); }
|
||||
bool canBeUsedAsVersion() const override { return dictionary_type->canBeUsedAsVersion(); }
|
||||
bool isSummable() const override { return dictionary_type->isSummable(); }
|
||||
bool canBeUsedInBitOperations() const override { return dictionary_type->canBeUsedInBitOperations(); }
|
||||
bool canBeUsedInBooleanContext() const override { return dictionary_type->canBeUsedInBooleanContext(); }
|
||||
bool isValueRepresentedByNumber() const override { return dictionary_type->isValueRepresentedByNumber(); }
|
||||
bool isValueRepresentedByInteger() const override { return dictionary_type->isValueRepresentedByInteger(); }
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
bool haveMaximumSizeOfValue() const override { return dictionary_type->haveMaximumSizeOfValue(); }
|
||||
size_t getMaximumSizeOfValueInMemory() const override { return dictionary_type->getMaximumSizeOfValueInMemory(); }
|
||||
size_t getSizeOfValueInMemory() const override { return dictionary_type->getSizeOfValueInMemory(); }
|
||||
bool isCategorial() const override { return false; }
|
||||
bool isNullable() const override { return false; }
|
||||
bool onlyNull() const override { return false; }
|
||||
bool lowCardinality() const override { return true; }
|
||||
|
||||
static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type);
|
||||
static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys);
|
||||
|
||||
private:
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
|
||||
template <typename ... Params>
|
||||
using SerializeFunctionPtr = void (IDataType::*)(const IColumn &, size_t, Params ...) const;
|
||||
|
||||
template <typename... Params, typename... Args>
|
||||
void serializeImpl(const IColumn & column, size_t row_num, SerializeFunctionPtr<Params...> func, Args &&... args) const;
|
||||
|
||||
template <typename ... Params>
|
||||
using DeserializeFunctionPtr = void (IDataType::*)(IColumn &, Params ...) const;
|
||||
|
||||
template <typename ... Params, typename... Args>
|
||||
void deserializeImpl(IColumn & column, DeserializeFunctionPtr<Params...> func, Args &&... args) const;
|
||||
|
||||
template <typename Creator>
|
||||
static MutableColumnUniquePtr createColumnUniqueImpl(const IDataType & keys_type, const Creator & creator);
|
||||
};
|
||||
|
||||
/// Returns dictionary type if type is DataTypeLowCardinality, type otherwise.
|
||||
DataTypePtr removeLowCardinality(const DataTypePtr & type);
|
||||
|
||||
/// Remove LowCardinality recursively from all nested types.
|
||||
DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type);
|
||||
|
||||
/// Remove LowCardinality recursively from all nested columns.
|
||||
ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column);
|
||||
|
||||
/// Convert column of type from_type to type to_type by converting nested LowCardinality columns.
|
||||
ColumnPtr recursiveTypeConversion(const ColumnPtr & column, const DataTypePtr & from_type, const DataTypePtr & to_type);
|
||||
|
||||
}
|
125
src/DataTypes/DataTypeMap.cpp
Normal file
125
src/DataTypes/DataTypeMap.cpp
Normal file
@ -0,0 +1,125 @@
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Columns/ColumnMap.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/Serializations/SerializationMap.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTNameTypePair.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <ext/map.h>
|
||||
#include <ext/enumerate.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
|
||||
DataTypeMap::DataTypeMap(const DataTypes & elems_)
|
||||
{
|
||||
assert(elems_.size() == 2);
|
||||
key_type = elems_[0];
|
||||
value_type = elems_[1];
|
||||
|
||||
nested = std::make_shared<DataTypeArray>(
|
||||
std::make_shared<DataTypeTuple>(DataTypes{key_type, value_type}, Names{"keys", "values"}));
|
||||
}
|
||||
|
||||
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::string DataTypeMap::doGetName() const
|
||||
{
|
||||
WriteBufferFromOwnString s;
|
||||
s << "Map(" << key_type->getName() << "," << value_type->getName() << ")";
|
||||
|
||||
return s.str();
|
||||
}
|
||||
|
||||
static const IColumn & extractNestedColumn(const IColumn & column)
|
||||
{
|
||||
return assert_cast<const ColumnMap &>(column).getNestedColumn();
|
||||
}
|
||||
|
||||
DataTypePtr DataTypeMap::tryGetSubcolumnType(const String & subcolumn_name) const
|
||||
{
|
||||
return nested->tryGetSubcolumnType(subcolumn_name);
|
||||
}
|
||||
|
||||
ColumnPtr DataTypeMap::getSubcolumn(const String & subcolumn_name, const IColumn & column) const
|
||||
{
|
||||
return nested->getSubcolumn(subcolumn_name, extractNestedColumn(column));
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeMap::getSubcolumnSerialization(
|
||||
const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const
|
||||
{
|
||||
return nested->getSubcolumnSerialization(subcolumn_name, base_serialization_getter);
|
||||
}
|
||||
|
||||
MutableColumnPtr DataTypeMap::createColumn() const
|
||||
{
|
||||
return ColumnMap::create(nested->createColumn());
|
||||
}
|
||||
|
||||
Field DataTypeMap::getDefault() const
|
||||
{
|
||||
return Map();
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeMap::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationMap>(
|
||||
key_type->getDefaultSerialization(),
|
||||
value_type->getDefaultSerialization(),
|
||||
nested->getDefaultSerialization());
|
||||
}
|
||||
|
||||
bool DataTypeMap::equals(const IDataType & rhs) const
|
||||
{
|
||||
if (typeid(rhs) != typeid(*this))
|
||||
return false;
|
||||
|
||||
const DataTypeMap & rhs_map = static_cast<const DataTypeMap &>(rhs);
|
||||
return nested->equals(*rhs_map.nested);
|
||||
}
|
||||
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->children.size() != 2)
|
||||
throw Exception("Map data type family must have two arguments: key and value types", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
DataTypes nested_types;
|
||||
nested_types.reserve(arguments->children.size());
|
||||
|
||||
for (const ASTPtr & child : arguments->children)
|
||||
nested_types.emplace_back(DataTypeFactory::instance().get(child));
|
||||
|
||||
return std::make_shared<DataTypeMap>(nested_types);
|
||||
}
|
||||
|
||||
|
||||
void registerDataTypeMap(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerDataType("Map", create);
|
||||
}
|
||||
}
|
58
src/DataTypes/DataTypeMap.h
Normal file
58
src/DataTypes/DataTypeMap.h
Normal file
@ -0,0 +1,58 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Map data type.
|
||||
* Map is implemented as two arrays of keys and values.
|
||||
* Serialization of type 'Map(K, V)' is similar to serialization.
|
||||
* of 'Array(Tuple(keys K, values V))' or in other words of 'Nested(keys K, valuev V)'.
|
||||
*/
|
||||
class DataTypeMap final : public IDataType
|
||||
{
|
||||
private:
|
||||
DataTypePtr key_type;
|
||||
DataTypePtr value_type;
|
||||
|
||||
/// 'nested' is an Array(Tuple(key_type, value_type))
|
||||
DataTypePtr nested;
|
||||
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
DataTypeMap(const DataTypes & elems);
|
||||
DataTypeMap(const DataTypePtr & key_type_, const DataTypePtr & value_type_);
|
||||
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Map; }
|
||||
std::string doGetName() const override;
|
||||
const char * getFamilyName() const override { return "Map"; }
|
||||
|
||||
bool canBeInsideNullable() const override { return false; }
|
||||
|
||||
DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override;
|
||||
ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override;
|
||||
SerializationPtr getSubcolumnSerialization(
|
||||
const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const override;
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
bool isComparable() const override { return key_type->isComparable() && value_type->isComparable(); }
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return true; }
|
||||
|
||||
const DataTypePtr & getKeyType() const { return key_type; }
|
||||
const DataTypePtr & getValueType() const { return value_type; }
|
||||
DataTypes getKeyValueTypes() const { return {key_type, value_type}; }
|
||||
const DataTypePtr & getNestedType() const { return nested; }
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -57,7 +57,7 @@ static std::pair<DataTypePtr, DataTypeCustomDescPtr> create(const ASTPtr & argum
|
||||
auto data_type = std::make_shared<DataTypeArray>(std::make_shared<DataTypeTuple>(nested_types, nested_names));
|
||||
auto custom_name = std::make_unique<DataTypeNestedCustomName>(nested_types, nested_names);
|
||||
|
||||
return std::make_pair(std::move(data_type), std::make_unique<DataTypeCustomDesc>(std::move(custom_name), nullptr));
|
||||
return std::make_pair(std::move(data_type), std::make_unique<DataTypeCustomDesc>(std::move(custom_name)));
|
||||
}
|
||||
|
||||
void registerDataTypeNested(DataTypeFactory & factory)
|
||||
|
@ -1,7 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/DataTypeWithSimpleSerialization.h>
|
||||
#include <DataTypes/DataTypeCustom.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
34
src/DataTypes/DataTypeNothing.cpp
Normal file
34
src/DataTypes/DataTypeNothing.cpp
Normal file
@ -0,0 +1,34 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/Serializations/SerializationNothing.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Columns/ColumnNothing.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
MutableColumnPtr DataTypeNothing::createColumn() const
|
||||
{
|
||||
return ColumnNothing::create(0);
|
||||
}
|
||||
|
||||
bool DataTypeNothing::equals(const IDataType & rhs) const
|
||||
{
|
||||
return typeid(rhs) == typeid(*this);
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeNothing::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationNothing>();
|
||||
}
|
||||
|
||||
|
||||
void registerDataTypeNothing(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataType("Nothing", [] { return DataTypePtr(std::make_shared<DataTypeNothing>()); });
|
||||
}
|
||||
|
||||
}
|
34
src/DataTypes/DataTypeNothing.h
Normal file
34
src/DataTypes/DataTypeNothing.h
Normal file
@ -0,0 +1,34 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataTypeDummy.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Data type that cannot have any values.
|
||||
* Used to represent NULL of unknown type as Nullable(Nothing),
|
||||
* and possibly for empty array of unknown type as Array(Nothing).
|
||||
*/
|
||||
class DataTypeNothing final : public IDataTypeDummy
|
||||
{
|
||||
public:
|
||||
static constexpr bool is_parametric = false;
|
||||
|
||||
const char * getFamilyName() const override { return "Nothing"; }
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Nothing; }
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
bool isParametric() const override { return false; }
|
||||
bool textCanContainOnlyValidUTF8() const override { return true; }
|
||||
bool haveMaximumSizeOfValue() const override { return true; }
|
||||
size_t getSizeOfValueInMemory() const override { return 0; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
};
|
||||
|
||||
}
|
129
src/DataTypes/DataTypeNullable.cpp
Normal file
129
src/DataTypes/DataTypeNullable.cpp
Normal file
@ -0,0 +1,129 @@
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/Serializations/SerializationNullable.h>
|
||||
#include <DataTypes/Serializations/SerializationTupleElement.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Core/Field.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ConcatReadBuffer.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
|
||||
DataTypeNullable::DataTypeNullable(const DataTypePtr & nested_data_type_)
|
||||
: nested_data_type{nested_data_type_}
|
||||
{
|
||||
if (!nested_data_type->canBeInsideNullable())
|
||||
throw Exception("Nested type " + nested_data_type->getName() + " cannot be inside Nullable type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
|
||||
bool DataTypeNullable::onlyNull() const
|
||||
{
|
||||
return typeid_cast<const DataTypeNothing *>(nested_data_type.get());
|
||||
}
|
||||
|
||||
|
||||
MutableColumnPtr DataTypeNullable::createColumn() const
|
||||
{
|
||||
return ColumnNullable::create(nested_data_type->createColumn(), ColumnUInt8::create());
|
||||
}
|
||||
|
||||
Field DataTypeNullable::getDefault() const
|
||||
{
|
||||
return Null();
|
||||
}
|
||||
|
||||
size_t DataTypeNullable::getSizeOfValueInMemory() const
|
||||
{
|
||||
throw Exception("Value of type " + getName() + " in memory is not of fixed size.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
|
||||
bool DataTypeNullable::equals(const IDataType & rhs) const
|
||||
{
|
||||
return rhs.isNullable() && nested_data_type->equals(*static_cast<const DataTypeNullable &>(rhs).nested_data_type);
|
||||
}
|
||||
|
||||
DataTypePtr DataTypeNullable::tryGetSubcolumnType(const String & subcolumn_name) const
|
||||
{
|
||||
if (subcolumn_name == "null")
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
|
||||
return nested_data_type->tryGetSubcolumnType(subcolumn_name);
|
||||
}
|
||||
|
||||
ColumnPtr DataTypeNullable::getSubcolumn(const String & subcolumn_name, const IColumn & column) const
|
||||
{
|
||||
const auto & column_nullable = assert_cast<const ColumnNullable &>(column);
|
||||
if (subcolumn_name == "null")
|
||||
return column_nullable.getNullMapColumnPtr();
|
||||
|
||||
return nested_data_type->getSubcolumn(subcolumn_name, column_nullable.getNestedColumn());
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeNullable::getSubcolumnSerialization(
|
||||
const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const
|
||||
{
|
||||
if (subcolumn_name == "null")
|
||||
return std::make_shared<SerializationTupleElement>(base_serialization_getter(DataTypeUInt8()), subcolumn_name, false);
|
||||
|
||||
return nested_data_type->getSubcolumnSerialization(subcolumn_name, base_serialization_getter);
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeNullable::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationNullable>(nested_data_type->getDefaultSerialization());
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->children.size() != 1)
|
||||
throw Exception("Nullable data type family must have exactly one argument - nested type", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
DataTypePtr nested_type = DataTypeFactory::instance().get(arguments->children[0]);
|
||||
|
||||
return std::make_shared<DataTypeNullable>(nested_type);
|
||||
}
|
||||
|
||||
|
||||
void registerDataTypeNullable(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerDataType("Nullable", create);
|
||||
}
|
||||
|
||||
|
||||
DataTypePtr makeNullable(const DataTypePtr & type)
|
||||
{
|
||||
if (type->isNullable())
|
||||
return type;
|
||||
return std::make_shared<DataTypeNullable>(type);
|
||||
}
|
||||
|
||||
DataTypePtr removeNullable(const DataTypePtr & type)
|
||||
{
|
||||
if (type->isNullable())
|
||||
return static_cast<const DataTypeNullable &>(*type).getNestedType();
|
||||
return type;
|
||||
}
|
||||
|
||||
}
|
60
src/DataTypes/DataTypeNullable.h
Normal file
60
src/DataTypes/DataTypeNullable.h
Normal file
@ -0,0 +1,60 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// A nullable data type is an ordinary data type provided with a tag
|
||||
/// indicating that it also contains the NULL value. The following class
|
||||
/// embodies this concept.
|
||||
class DataTypeNullable final : public IDataType
|
||||
{
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
explicit DataTypeNullable(const DataTypePtr & nested_data_type_);
|
||||
std::string doGetName() const override { return "Nullable(" + nested_data_type->getName() + ")"; }
|
||||
const char * getFamilyName() const override { return "Nullable"; }
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Nullable; }
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return true; }
|
||||
bool cannotBeStoredInTables() const override { return nested_data_type->cannotBeStoredInTables(); }
|
||||
bool shouldAlignRightInPrettyFormats() const override { return nested_data_type->shouldAlignRightInPrettyFormats(); }
|
||||
bool textCanContainOnlyValidUTF8() const override { return nested_data_type->textCanContainOnlyValidUTF8(); }
|
||||
bool isComparable() const override { return nested_data_type->isComparable(); }
|
||||
bool canBeComparedWithCollation() const override { return nested_data_type->canBeComparedWithCollation(); }
|
||||
bool canBeUsedAsVersion() const override { return false; }
|
||||
bool isSummable() const override { return nested_data_type->isSummable(); }
|
||||
bool canBeUsedInBooleanContext() const override { return nested_data_type->canBeUsedInBooleanContext() || onlyNull(); }
|
||||
bool haveMaximumSizeOfValue() const override { return nested_data_type->haveMaximumSizeOfValue(); }
|
||||
size_t getMaximumSizeOfValueInMemory() const override { return 1 + nested_data_type->getMaximumSizeOfValueInMemory(); }
|
||||
bool isNullable() const override { return true; }
|
||||
size_t getSizeOfValueInMemory() const override;
|
||||
bool onlyNull() const override;
|
||||
bool canBeInsideLowCardinality() const override { return nested_data_type->canBeInsideLowCardinality(); }
|
||||
|
||||
DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override;
|
||||
ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override;
|
||||
SerializationPtr getSubcolumnSerialization(
|
||||
const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const override;
|
||||
|
||||
const DataTypePtr & getNestedType() const { return nested_data_type; }
|
||||
private:
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
|
||||
DataTypePtr nested_data_type;
|
||||
};
|
||||
|
||||
|
||||
DataTypePtr makeNullable(const DataTypePtr & type);
|
||||
DataTypePtr removeNullable(const DataTypePtr & type);
|
||||
|
||||
}
|
@ -13,196 +13,12 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
{
|
||||
writeText(assert_cast<const ColumnVector<T> &>(column).getData()[row_num], ostr);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
T x;
|
||||
|
||||
if constexpr (is_integer_v<T> && is_arithmetic_v<T>)
|
||||
readIntTextUnsafe(x, istr);
|
||||
else
|
||||
readText(x, istr);
|
||||
|
||||
assert_cast<ColumnVector<T> &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static inline void writeDenormalNumber(T x, WriteBuffer & ostr)
|
||||
{
|
||||
if constexpr (std::is_floating_point_v<T>)
|
||||
{
|
||||
if (std::signbit(x))
|
||||
{
|
||||
if (isNaN(x))
|
||||
writeCString("-nan", ostr);
|
||||
else
|
||||
writeCString("-inf", ostr);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (isNaN(x))
|
||||
writeCString("nan", ostr);
|
||||
else
|
||||
writeCString("inf", ostr);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// This function is not called for non floating point numbers.
|
||||
(void)x;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
auto x = assert_cast<const ColumnVector<T> &>(column).getData()[row_num];
|
||||
bool is_finite = isFinite(x);
|
||||
|
||||
const bool need_quote = (is_integer_v<T> && (sizeof(T) >= 8) && settings.json.quote_64bit_integers)
|
||||
|| (settings.json.quote_denormals && !is_finite);
|
||||
|
||||
if (need_quote)
|
||||
writeChar('"', ostr);
|
||||
|
||||
if (is_finite)
|
||||
writeText(x, ostr);
|
||||
else if (!settings.json.quote_denormals)
|
||||
writeCString("null", ostr);
|
||||
else
|
||||
writeDenormalNumber(x, ostr);
|
||||
|
||||
if (need_quote)
|
||||
writeChar('"', ostr);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
bool has_quote = false;
|
||||
if (!istr.eof() && *istr.position() == '"') /// We understand the number both in quotes and without.
|
||||
{
|
||||
has_quote = true;
|
||||
++istr.position();
|
||||
}
|
||||
|
||||
FieldType x;
|
||||
|
||||
/// null
|
||||
if (!has_quote && !istr.eof() && *istr.position() == 'n')
|
||||
{
|
||||
++istr.position();
|
||||
assertString("ull", istr);
|
||||
|
||||
x = NaNOrZero<T>();
|
||||
}
|
||||
else
|
||||
{
|
||||
static constexpr bool is_uint8 = std::is_same_v<T, UInt8>;
|
||||
static constexpr bool is_int8 = std::is_same_v<T, Int8>;
|
||||
|
||||
if (is_uint8 || is_int8)
|
||||
{
|
||||
// extra conditions to parse true/false strings into 1/0
|
||||
if (istr.eof())
|
||||
throwReadAfterEOF();
|
||||
if (*istr.position() == 't' || *istr.position() == 'f')
|
||||
{
|
||||
bool tmp = false;
|
||||
readBoolTextWord(tmp, istr);
|
||||
x = tmp;
|
||||
}
|
||||
else
|
||||
readText(x, istr);
|
||||
}
|
||||
else
|
||||
{
|
||||
readText(x, istr);
|
||||
}
|
||||
|
||||
if (has_quote)
|
||||
assertChar('"', istr);
|
||||
}
|
||||
|
||||
assert_cast<ColumnVector<T> &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
FieldType x;
|
||||
readCSV(x, istr);
|
||||
assert_cast<ColumnVector<T> &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
Field DataTypeNumberBase<T>::getDefault() const
|
||||
{
|
||||
return NearestFieldType<FieldType>();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::serializeBinary(const Field & field, WriteBuffer & ostr) const
|
||||
{
|
||||
/// ColumnVector<T>::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64
|
||||
typename ColumnVector<T>::ValueType x = get<NearestFieldType<FieldType>>(field);
|
||||
writeBinary(x, ostr);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::deserializeBinary(Field & field, ReadBuffer & istr) const
|
||||
{
|
||||
typename ColumnVector<T>::ValueType x;
|
||||
readBinary(x, istr);
|
||||
field = NearestFieldType<FieldType>(x);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
|
||||
{
|
||||
writeBinary(assert_cast<const ColumnVector<T> &>(column).getData()[row_num], ostr);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::deserializeBinary(IColumn & column, ReadBuffer & istr) const
|
||||
{
|
||||
typename ColumnVector<T>::ValueType x;
|
||||
readBinary(x, istr);
|
||||
assert_cast<ColumnVector<T> &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const
|
||||
{
|
||||
const typename ColumnVector<T>::Container & x = typeid_cast<const ColumnVector<T> &>(column).getData();
|
||||
|
||||
size_t size = x.size();
|
||||
|
||||
if (limit == 0 || offset + limit > size)
|
||||
limit = size - offset;
|
||||
|
||||
if (limit)
|
||||
ostr.write(reinterpret_cast<const char *>(&x[offset]), sizeof(typename ColumnVector<T>::ValueType) * limit);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeNumberBase<T>::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double /*avg_value_size_hint*/) const
|
||||
{
|
||||
typename ColumnVector<T>::Container & x = typeid_cast<ColumnVector<T> &>(column).getData();
|
||||
size_t initial_size = x.size();
|
||||
x.resize(initial_size + limit);
|
||||
size_t size = istr.readBig(reinterpret_cast<char*>(&x[initial_size]), sizeof(typename ColumnVector<T>::ValueType) * limit);
|
||||
x.resize(initial_size + size / sizeof(typename ColumnVector<T>::ValueType));
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
MutableColumnPtr DataTypeNumberBase<T>::createColumn() const
|
||||
{
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <Common/UInt128.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypeWithSimpleSerialization.h>
|
||||
#include <DataTypes/Serializations/SerializationNumber.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -14,7 +14,7 @@ class ColumnVector;
|
||||
/** Implements part of the IDataType interface, common to all numbers and for Date and DateTime.
|
||||
*/
|
||||
template <typename T>
|
||||
class DataTypeNumberBase : public DataTypeWithSimpleSerialization
|
||||
class DataTypeNumberBase : public IDataType
|
||||
{
|
||||
static_assert(IsNumber<T>);
|
||||
|
||||
@ -30,21 +30,8 @@ public:
|
||||
const char * getFamilyName() const override { return family_name; }
|
||||
TypeIndex getTypeId() const override { return type_id; }
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
Field getDefault() const override;
|
||||
|
||||
/** Format is platform-dependent. */
|
||||
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
|
||||
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
|
||||
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
|
||||
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
|
||||
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override;
|
||||
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
|
||||
bool isParametric() const override { return false; }
|
||||
@ -53,7 +40,7 @@ public:
|
||||
bool shouldAlignRightInPrettyFormats() const override
|
||||
{
|
||||
/// Just a number, without customizations. Counterexample: IPv4.
|
||||
return !custom_text_serialization;
|
||||
return !custom_serialization;
|
||||
}
|
||||
|
||||
bool textCanContainOnlyValidUTF8() const override { return true; }
|
||||
@ -66,6 +53,8 @@ public:
|
||||
size_t getSizeOfValueInMemory() const override { return sizeof(T); }
|
||||
bool isCategorial() const override { return isValueRepresentedByInteger(); }
|
||||
bool canBeInsideLowCardinality() const override { return true; }
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override { return std::make_shared<SerializationNumber<T>>(); }
|
||||
};
|
||||
|
||||
/// Prevent implicit template instantiation of DataTypeNumberBase for common numeric types
|
||||
|
112
src/DataTypes/DataTypeString.cpp
Normal file
112
src/DataTypes/DataTypeString.cpp
Normal file
@ -0,0 +1,112 @@
|
||||
#include <Core/Defines.h>
|
||||
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
#include <Core/Field.h>
|
||||
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/Serializations/SerializationString.h>
|
||||
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/VarInt.h>
|
||||
|
||||
#ifdef __SSE2__
|
||||
#include <emmintrin.h>
|
||||
#endif
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int UNEXPECTED_AST_STRUCTURE;
|
||||
}
|
||||
|
||||
Field DataTypeString::getDefault() const
|
||||
{
|
||||
return String();
|
||||
}
|
||||
|
||||
MutableColumnPtr DataTypeString::createColumn() const
|
||||
{
|
||||
return ColumnString::create();
|
||||
}
|
||||
|
||||
|
||||
bool DataTypeString::equals(const IDataType & rhs) const
|
||||
{
|
||||
return typeid(rhs) == typeid(*this);
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeString::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationString>();
|
||||
}
|
||||
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (arguments && !arguments->children.empty())
|
||||
{
|
||||
if (arguments->children.size() > 1)
|
||||
throw Exception("String data type family mustn't have more than one argument - size in characters", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const auto * argument = arguments->children[0]->as<ASTLiteral>();
|
||||
if (!argument || argument->value.getType() != Field::Types::UInt64)
|
||||
throw Exception("String data type family may have only a number (positive integer) as its argument", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
|
||||
}
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
}
|
||||
|
||||
|
||||
void registerDataTypeString(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerDataType("String", create);
|
||||
|
||||
/// These synonims are added for compatibility.
|
||||
|
||||
factory.registerAlias("CHAR", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("NCHAR", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("CHARACTER", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("VARCHAR", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("NVARCHAR", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("VARCHAR2", "String", DataTypeFactory::CaseInsensitive); /// Oracle
|
||||
factory.registerAlias("TEXT", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("TINYTEXT", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("MEDIUMTEXT", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("LONGTEXT", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("BLOB", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("CLOB", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("TINYBLOB", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("MEDIUMBLOB", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("LONGBLOB", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("BYTEA", "String", DataTypeFactory::CaseInsensitive); /// PostgreSQL
|
||||
|
||||
factory.registerAlias("CHARACTER LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("CHARACTER VARYING", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("CHAR LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("CHAR VARYING", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("NATIONAL CHAR", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("NATIONAL CHARACTER", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("NATIONAL CHARACTER LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("NATIONAL CHARACTER VARYING", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("NATIONAL CHAR VARYING", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("NCHAR VARYING", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("NCHAR LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("BINARY LARGE OBJECT", "String", DataTypeFactory::CaseInsensitive);
|
||||
factory.registerAlias("BINARY VARYING", "String", DataTypeFactory::CaseInsensitive);
|
||||
}
|
||||
}
|
43
src/DataTypes/DataTypeString.h
Normal file
43
src/DataTypes/DataTypeString.h
Normal file
@ -0,0 +1,43 @@
|
||||
#pragma once
|
||||
|
||||
#include <ostream>
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class DataTypeString final : public IDataType
|
||||
{
|
||||
public:
|
||||
using FieldType = String;
|
||||
static constexpr bool is_parametric = false;
|
||||
static constexpr auto type_id = TypeIndex::String;
|
||||
|
||||
const char * getFamilyName() const override
|
||||
{
|
||||
return "String";
|
||||
}
|
||||
|
||||
TypeIndex getTypeId() const override { return type_id; }
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
bool isParametric() const override { return false; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool isComparable() const override { return true; }
|
||||
bool canBeComparedWithCollation() const override { return true; }
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
bool isCategorial() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
bool canBeInsideLowCardinality() const override { return true; }
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
};
|
||||
|
||||
}
|
391
src/DataTypes/DataTypeTuple.cpp
Normal file
391
src/DataTypes/DataTypeTuple.cpp
Normal file
@ -0,0 +1,391 @@
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Core/Field.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/Serializations/SerializationInfo.h>
|
||||
#include <DataTypes/Serializations/SerializationTuple.h>
|
||||
#include <DataTypes/Serializations/SerializationTupleElement.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <Parsers/IAST.h>
|
||||
#include <Parsers/ASTNameTypePair.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/quoteString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/Operators.h>
|
||||
|
||||
#include <ext/map.h>
|
||||
#include <ext/enumerate.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
extern const int DUPLICATE_COLUMN;
|
||||
extern const int EMPTY_DATA_PASSED;
|
||||
extern const int NOT_FOUND_COLUMN_IN_BLOCK;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
||||
extern const int SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
|
||||
DataTypeTuple::DataTypeTuple(const DataTypes & elems_)
|
||||
: elems(elems_), have_explicit_names(false)
|
||||
{
|
||||
/// Automatically assigned names in form of '1', '2', ...
|
||||
size_t size = elems.size();
|
||||
names.resize(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
names[i] = toString(i + 1);
|
||||
}
|
||||
|
||||
static std::optional<Exception> checkTupleNames(const Strings & names)
|
||||
{
|
||||
std::unordered_set<String> names_set;
|
||||
for (const auto & name : names)
|
||||
{
|
||||
if (name.empty())
|
||||
return Exception("Names of tuple elements cannot be empty", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (isNumericASCII(name[0]))
|
||||
return Exception("Explicitly specified names of tuple elements cannot start with digit", ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (!names_set.insert(name).second)
|
||||
return Exception("Names of tuple elements must be unique", ErrorCodes::DUPLICATE_COLUMN);
|
||||
}
|
||||
|
||||
return {};
|
||||
}
|
||||
|
||||
DataTypeTuple::DataTypeTuple(const DataTypes & elems_, const Strings & names_, bool serialize_names_)
|
||||
: elems(elems_), names(names_), have_explicit_names(true), serialize_names(serialize_names_)
|
||||
{
|
||||
size_t size = elems.size();
|
||||
if (names.size() != size)
|
||||
throw Exception("Wrong number of names passed to constructor of DataTypeTuple", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (auto exception = checkTupleNames(names))
|
||||
throw std::move(*exception);
|
||||
}
|
||||
|
||||
bool DataTypeTuple::canBeCreatedWithNames(const Strings & names)
|
||||
{
|
||||
return checkTupleNames(names) == std::nullopt;
|
||||
}
|
||||
|
||||
std::string DataTypeTuple::doGetName() const
|
||||
{
|
||||
size_t size = elems.size();
|
||||
WriteBufferFromOwnString s;
|
||||
|
||||
s << "Tuple(";
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
s << ", ";
|
||||
|
||||
if (have_explicit_names && serialize_names)
|
||||
s << backQuoteIfNeed(names[i]) << ' ';
|
||||
|
||||
s << elems[i]->getName();
|
||||
}
|
||||
s << ")";
|
||||
|
||||
return s.str();
|
||||
}
|
||||
|
||||
|
||||
static inline IColumn & extractElementColumn(IColumn & column, size_t idx)
|
||||
{
|
||||
return assert_cast<ColumnTuple &>(column).getColumn(idx);
|
||||
}
|
||||
|
||||
static inline const IColumn & extractElementColumn(const IColumn & column, size_t idx)
|
||||
{
|
||||
return assert_cast<const ColumnTuple &>(column).getColumn(idx);
|
||||
}
|
||||
|
||||
template <typename F>
|
||||
static void addElementSafe(const DataTypes & elems, IColumn & column, F && impl)
|
||||
{
|
||||
/// We use the assumption that tuples of zero size do not exist.
|
||||
size_t old_size = column.size();
|
||||
|
||||
try
|
||||
{
|
||||
impl();
|
||||
|
||||
// Check that all columns now have the same size.
|
||||
size_t new_size = column.size();
|
||||
|
||||
for (auto i : ext::range(0, ext::size(elems)))
|
||||
{
|
||||
const auto & element_column = extractElementColumn(column, i);
|
||||
if (element_column.size() != new_size)
|
||||
{
|
||||
// This is not a logical error because it may work with
|
||||
// user-supplied data.
|
||||
throw Exception(ErrorCodes::SIZES_OF_COLUMNS_IN_TUPLE_DOESNT_MATCH,
|
||||
"Cannot read a tuple because not all elements are present");
|
||||
}
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
for (const auto & i : ext::range(0, ext::size(elems)))
|
||||
{
|
||||
auto & element_column = extractElementColumn(column, i);
|
||||
|
||||
if (element_column.size() > old_size)
|
||||
element_column.popBack(1);
|
||||
}
|
||||
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
MutableColumnPtr DataTypeTuple::createColumn() const
|
||||
{
|
||||
size_t size = elems.size();
|
||||
MutableColumns tuple_columns(size);
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
tuple_columns[i] = elems[i]->createColumn();
|
||||
return ColumnTuple::create(std::move(tuple_columns));
|
||||
}
|
||||
|
||||
Field DataTypeTuple::getDefault() const
|
||||
{
|
||||
return Tuple(ext::map<Tuple>(elems, [] (const DataTypePtr & elem) { return elem->getDefault(); }));
|
||||
}
|
||||
|
||||
void DataTypeTuple::insertDefaultInto(IColumn & column) const
|
||||
{
|
||||
addElementSafe(elems, column, [&]
|
||||
{
|
||||
for (const auto & i : ext::range(0, ext::size(elems)))
|
||||
elems[i]->insertDefaultInto(extractElementColumn(column, i));
|
||||
});
|
||||
}
|
||||
|
||||
bool DataTypeTuple::equals(const IDataType & rhs) const
|
||||
{
|
||||
if (typeid(rhs) != typeid(*this))
|
||||
return false;
|
||||
|
||||
const DataTypeTuple & rhs_tuple = static_cast<const DataTypeTuple &>(rhs);
|
||||
|
||||
size_t size = elems.size();
|
||||
if (size != rhs_tuple.elems.size())
|
||||
return false;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
if (!elems[i]->equals(*rhs_tuple.elems[i]))
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
|
||||
size_t DataTypeTuple::getPositionByName(const String & name) const
|
||||
{
|
||||
size_t size = elems.size();
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
if (names[i] == name)
|
||||
return i;
|
||||
throw Exception("Tuple doesn't have element with name '" + name + "'", ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK);
|
||||
}
|
||||
|
||||
|
||||
bool DataTypeTuple::textCanContainOnlyValidUTF8() const
|
||||
{
|
||||
return std::all_of(elems.begin(), elems.end(), [](auto && elem) { return elem->textCanContainOnlyValidUTF8(); });
|
||||
}
|
||||
|
||||
bool DataTypeTuple::haveMaximumSizeOfValue() const
|
||||
{
|
||||
return std::all_of(elems.begin(), elems.end(), [](auto && elem) { return elem->haveMaximumSizeOfValue(); });
|
||||
}
|
||||
|
||||
bool DataTypeTuple::isComparable() const
|
||||
{
|
||||
return std::all_of(elems.begin(), elems.end(), [](auto && elem) { return elem->isComparable(); });
|
||||
}
|
||||
|
||||
size_t DataTypeTuple::getMaximumSizeOfValueInMemory() const
|
||||
{
|
||||
size_t res = 0;
|
||||
for (const auto & elem : elems)
|
||||
res += elem->getMaximumSizeOfValueInMemory();
|
||||
return res;
|
||||
}
|
||||
|
||||
size_t DataTypeTuple::getSizeOfValueInMemory() const
|
||||
{
|
||||
size_t res = 0;
|
||||
for (const auto & elem : elems)
|
||||
res += elem->getSizeOfValueInMemory();
|
||||
return res;
|
||||
}
|
||||
|
||||
template <typename OnSuccess, typename OnContinue>
|
||||
auto DataTypeTuple::getSubcolumnEntity(const String & subcolumn_name,
|
||||
const OnSuccess & on_success, const OnContinue & on_continue) const
|
||||
{
|
||||
using ReturnType = decltype(on_success(0));
|
||||
for (size_t i = 0; i < names.size(); ++i)
|
||||
{
|
||||
if (startsWith(subcolumn_name, names[i]))
|
||||
{
|
||||
size_t name_length = names[i].size();
|
||||
|
||||
if (subcolumn_name.size() == name_length)
|
||||
return on_success(i);
|
||||
|
||||
if (subcolumn_name[name_length] == '.')
|
||||
return on_continue(i, subcolumn_name.substr(name_length + 1));
|
||||
}
|
||||
}
|
||||
|
||||
return ReturnType{};
|
||||
}
|
||||
|
||||
DataTypePtr DataTypeTuple::tryGetSubcolumnType(const String & subcolumn_name) const
|
||||
{
|
||||
if (subcolumn_name == MAIN_SUBCOLUMN_NAME)
|
||||
return shared_from_this();
|
||||
|
||||
auto on_success = [&](size_t pos) { return elems[pos]; };
|
||||
auto on_continue = [&](size_t pos, const String & next_subcolumn) { return elems[pos]->tryGetSubcolumnType(next_subcolumn); };
|
||||
|
||||
return getSubcolumnEntity(subcolumn_name, on_success, on_continue);
|
||||
}
|
||||
|
||||
ColumnPtr DataTypeTuple::getSubcolumn(const String & subcolumn_name, const IColumn & column) const
|
||||
{
|
||||
auto on_success = [&](size_t pos) { return extractElementColumn(column, pos).getPtr(); };
|
||||
auto on_continue = [&](size_t pos, const String & next_subcolumn)
|
||||
{
|
||||
return elems[pos]->getSubcolumn(next_subcolumn, extractElementColumn(column, pos));
|
||||
};
|
||||
|
||||
if (auto subcolumn = getSubcolumnEntity(subcolumn_name, on_success, on_continue))
|
||||
return subcolumn;
|
||||
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName());
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeTuple::getSubcolumnSerialization(
|
||||
const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const
|
||||
{
|
||||
auto on_success = [&](size_t pos)
|
||||
{
|
||||
return std::make_shared<SerializationTupleElement>(base_serialization_getter(*elems[pos]), names[pos]);
|
||||
};
|
||||
|
||||
auto on_continue = [&](size_t pos, const String & next_subcolumn)
|
||||
{
|
||||
auto next_serialization = elems[pos]->getSubcolumnSerialization(next_subcolumn, base_serialization_getter);
|
||||
return std::make_shared<SerializationTupleElement>(next_serialization, names[pos]);
|
||||
};
|
||||
|
||||
if (auto serialization = getSubcolumnEntity(subcolumn_name, on_success, on_continue))
|
||||
return serialization;
|
||||
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName());
|
||||
}
|
||||
|
||||
|
||||
SerializationPtr DataTypeTuple::doGetDefaultSerialization() const
|
||||
{
|
||||
SerializationTuple::ElementSerializations serializations(elems.size());
|
||||
for (size_t i = 0; i < elems.size(); ++i)
|
||||
{
|
||||
auto serialization = elems[i]->getDefaultSerialization();
|
||||
serializations[i] = std::make_shared<SerializationTupleElement>(serialization, names[i]);
|
||||
}
|
||||
|
||||
return std::make_shared<SerializationTuple>(std::move(serializations), have_explicit_names);
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeTuple::getSerialization(const String & column_name, const StreamExistenceCallback & callback) const
|
||||
{
|
||||
SerializationTuple::ElementSerializations serializations(elems.size());
|
||||
for (size_t i = 0; i < elems.size(); ++i)
|
||||
{
|
||||
auto subcolumn_name = Nested::concatenateName(column_name, names[i]);
|
||||
auto serializaion = elems[i]->getSerialization(subcolumn_name, callback);
|
||||
serializations[i] = std::make_shared<SerializationTupleElement>(serializaion, names[i]);
|
||||
}
|
||||
|
||||
return std::make_shared<SerializationTuple>(std::move(serializations), have_explicit_names);
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeTuple::getSerialization(const String & column_name, const SerializationInfo & info) const
|
||||
{
|
||||
SerializationTuple::ElementSerializations serializations(elems.size());
|
||||
for (size_t i = 0; i < elems.size(); ++i)
|
||||
{
|
||||
auto subcolumn_name = Nested::concatenateName(column_name, names[i]);
|
||||
|
||||
ISerialization::Settings settings =
|
||||
{
|
||||
.num_rows = info.getNumberOfRows(),
|
||||
.num_non_default_rows = info.getNumberOfNonDefaultValues(subcolumn_name),
|
||||
.min_ratio_for_dense_serialization = 10
|
||||
};
|
||||
|
||||
auto serializaion = elems[i]->getSerialization(settings);
|
||||
serializations[i] = std::make_shared<SerializationTupleElement>(serializaion, names[i]);
|
||||
}
|
||||
|
||||
return std::make_shared<SerializationTuple>(std::move(serializations), have_explicit_names);
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->children.empty())
|
||||
throw Exception("Tuple cannot be empty", ErrorCodes::EMPTY_DATA_PASSED);
|
||||
|
||||
DataTypes nested_types;
|
||||
nested_types.reserve(arguments->children.size());
|
||||
|
||||
Strings names;
|
||||
names.reserve(arguments->children.size());
|
||||
|
||||
for (const ASTPtr & child : arguments->children)
|
||||
{
|
||||
if (const auto * name_and_type_pair = child->as<ASTNameTypePair>())
|
||||
{
|
||||
nested_types.emplace_back(DataTypeFactory::instance().get(name_and_type_pair->type));
|
||||
names.emplace_back(name_and_type_pair->name);
|
||||
}
|
||||
else
|
||||
nested_types.emplace_back(DataTypeFactory::instance().get(child));
|
||||
}
|
||||
|
||||
if (names.empty())
|
||||
return std::make_shared<DataTypeTuple>(nested_types);
|
||||
else if (names.size() != nested_types.size())
|
||||
throw Exception("Names are specified not for all elements of Tuple type", ErrorCodes::BAD_ARGUMENTS);
|
||||
else
|
||||
return std::make_shared<DataTypeTuple>(nested_types, names);
|
||||
}
|
||||
|
||||
|
||||
void registerDataTypeTuple(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerDataType("Tuple", create);
|
||||
}
|
||||
|
||||
}
|
81
src/DataTypes/DataTypeTuple.h
Normal file
81
src/DataTypes/DataTypeTuple.h
Normal file
@ -0,0 +1,81 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Tuple data type.
|
||||
* Used as an intermediate result when evaluating expressions.
|
||||
* Also can be used as a column - the result of the query execution.
|
||||
*
|
||||
* Tuple elements can have names.
|
||||
* If an element is unnamed, it will have automatically assigned name like '1', '2', '3' corresponding to its position.
|
||||
* Manually assigned names must not begin with digit. Names must be unique.
|
||||
*
|
||||
* All tuples with same size and types of elements are equivalent for expressions, regardless to names of elements.
|
||||
*/
|
||||
class DataTypeTuple final : public IDataType
|
||||
{
|
||||
private:
|
||||
DataTypes elems;
|
||||
Strings names;
|
||||
bool have_explicit_names;
|
||||
bool serialize_names = true;
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
DataTypeTuple(const DataTypes & elems);
|
||||
DataTypeTuple(const DataTypes & elems, const Strings & names, bool serialize_names_ = true);
|
||||
|
||||
static bool canBeCreatedWithNames(const Strings & names);
|
||||
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Tuple; }
|
||||
std::string doGetName() const override;
|
||||
const char * getFamilyName() const override { return "Tuple"; }
|
||||
|
||||
bool canBeInsideNullable() const override { return false; }
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
void insertDefaultInto(IColumn & column) const override;
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return !elems.empty(); }
|
||||
bool isComparable() const override;
|
||||
bool textCanContainOnlyValidUTF8() const override;
|
||||
bool haveMaximumSizeOfValue() const override;
|
||||
size_t getMaximumSizeOfValueInMemory() const override;
|
||||
size_t getSizeOfValueInMemory() const override;
|
||||
|
||||
DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const override;
|
||||
ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const override;
|
||||
|
||||
SerializationPtr getSerialization(const String & column_name, const StreamExistenceCallback & callback) const override;
|
||||
SerializationPtr getSerialization(const String & column_name, const SerializationInfo & info) const override;
|
||||
|
||||
SerializationPtr getSubcolumnSerialization(
|
||||
const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const override;
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
|
||||
const DataTypes & getElements() const { return elems; }
|
||||
const Strings & getElementNames() const { return names; }
|
||||
|
||||
size_t getPositionByName(const String & name) const;
|
||||
|
||||
bool haveExplicitNames() const { return have_explicit_names; }
|
||||
bool serializeNames() const { return serialize_names; }
|
||||
|
||||
private:
|
||||
template <typename OnSuccess, typename OnContinue>
|
||||
auto getSubcolumnEntity(const String & subcolumn_name,
|
||||
const OnSuccess & on_success, const OnContinue & on_continue) const;
|
||||
};
|
||||
|
||||
}
|
||||
|
24
src/DataTypes/DataTypeUUID.cpp
Normal file
24
src/DataTypes/DataTypeUUID.cpp
Normal file
@ -0,0 +1,24 @@
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/Serializations/SerializationUUID.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool DataTypeUUID::equals(const IDataType & rhs) const
|
||||
{
|
||||
return typeid(rhs) == typeid(*this);
|
||||
}
|
||||
|
||||
SerializationPtr DataTypeUUID::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationUUID>();
|
||||
}
|
||||
|
||||
void registerDataTypeUUID(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataType("UUID", [] { return DataTypePtr(std::make_shared<DataTypeUUID>()); });
|
||||
}
|
||||
|
||||
}
|
28
src/DataTypes/DataTypeUUID.h
Normal file
28
src/DataTypes/DataTypeUUID.h
Normal file
@ -0,0 +1,28 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/UInt128.h>
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class DataTypeUUID final : public DataTypeNumberBase<UInt128>
|
||||
{
|
||||
|
||||
public:
|
||||
const char * getFamilyName() const override { return "UUID"; }
|
||||
TypeIndex getTypeId() const override { return TypeIndex::UUID; }
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
bool canBeUsedInBitOperations() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
bool canBeInsideLowCardinality() const override { return false; }
|
||||
|
||||
bool canBePromoted() const override { return false; }
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
};
|
||||
|
||||
}
|
@ -1,4 +1,5 @@
|
||||
#include <DataTypes/DataTypesDecimal.h>
|
||||
#include <DataTypes/Serializations/SerializationDecimal.h>
|
||||
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
@ -46,55 +47,6 @@ DataTypePtr DataTypeDecimal<T>::promoteNumericType() const
|
||||
return std::make_shared<PromotedType>(PromotedType::maxPrecision(), this->scale);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeDecimal<T>::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
{
|
||||
T value = assert_cast<const ColumnType &>(column).getData()[row_num];
|
||||
writeText(value, this->scale, ostr);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool DataTypeDecimal<T>::tryReadText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale)
|
||||
{
|
||||
UInt32 unread_scale = scale;
|
||||
if (!tryReadDecimalText(istr, x, precision, unread_scale))
|
||||
return false;
|
||||
|
||||
if (common::mulOverflow(x.value, DecimalUtils::scaleMultiplier<T>(unread_scale), x.value))
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeDecimal<T>::readText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale, bool csv)
|
||||
{
|
||||
UInt32 unread_scale = scale;
|
||||
if (csv)
|
||||
readCSVDecimalText(istr, x, precision, unread_scale);
|
||||
else
|
||||
readDecimalText(istr, x, precision, unread_scale);
|
||||
|
||||
if (common::mulOverflow(x.value, DecimalUtils::scaleMultiplier<T>(unread_scale), x.value))
|
||||
throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeDecimal<T>::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
T x;
|
||||
readText(x, istr);
|
||||
assert_cast<ColumnType &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void DataTypeDecimal<T>::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
T x;
|
||||
readText(x, istr, true);
|
||||
assert_cast<ColumnType &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
T DataTypeDecimal<T>::parseFromString(const String & str) const
|
||||
{
|
||||
@ -109,6 +61,12 @@ T DataTypeDecimal<T>::parseFromString(const String & str) const
|
||||
return x;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
SerializationPtr DataTypeDecimal<T>::doGetDefaultSerialization() const
|
||||
{
|
||||
return std::make_shared<SerializationDecimal<T>>(this->precision, this->scale);
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
|
@ -42,17 +42,9 @@ public:
|
||||
bool canBePromoted() const override { return true; }
|
||||
DataTypePtr promoteNumericType() const override;
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
T parseFromString(const String & str) const;
|
||||
void readText(T & x, ReadBuffer & istr, bool csv = false) const { readText(x, istr, this->precision, this->scale, csv); }
|
||||
|
||||
static void readText(T & x, ReadBuffer & istr, UInt32 precision_, UInt32 scale_, bool csv = false);
|
||||
static bool tryReadText(T & x, ReadBuffer & istr, UInt32 precision_, UInt32 scale_);
|
||||
SerializationPtr doGetDefaultSerialization() const override;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <type_traits>
|
||||
#include <Core/Field.h>
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
#include <DataTypes/Serializations/SerializationNumber.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -25,6 +26,11 @@ class DataTypeNumber final : public DataTypeNumberBase<T>
|
||||
using PromotedType = DataTypeNumber<NearestFieldType<T>>;
|
||||
return std::make_shared<PromotedType>();
|
||||
}
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override
|
||||
{
|
||||
return std::make_shared<SerializationNumber<T>>();
|
||||
}
|
||||
};
|
||||
|
||||
using DataTypeUInt8 = DataTypeNumber<UInt8>;
|
||||
|
77
src/DataTypes/EnumValues.cpp
Normal file
77
src/DataTypes/EnumValues.cpp
Normal file
@ -0,0 +1,77 @@
|
||||
#include <DataTypes/EnumValues.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int EMPTY_DATA_PASSED;
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
EnumValues<T>::EnumValues(const Values & values_)
|
||||
: values(values_)
|
||||
{
|
||||
if (values.empty())
|
||||
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;
|
||||
});
|
||||
|
||||
fillMaps();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void EnumValues<T>::fillMaps()
|
||||
{
|
||||
for (const auto & name_and_value : values)
|
||||
{
|
||||
const auto inserted_value = name_to_value_map.insert(
|
||||
{ StringRef{name_and_value.first}, name_and_value.second });
|
||||
|
||||
if (!inserted_value.second)
|
||||
throw Exception{"Duplicate names in enum: '" + name_and_value.first + "' = " + toString(name_and_value.second)
|
||||
+ " and " + toString(inserted_value.first->getMapped()),
|
||||
ErrorCodes::SYNTAX_ERROR};
|
||||
|
||||
const auto inserted_name = value_to_name_map.insert(
|
||||
{ name_and_value.second, StringRef{name_and_value.first} });
|
||||
|
||||
if (!inserted_name.second)
|
||||
throw Exception{"Duplicate values in enum: '" + name_and_value.first + "' = " + toString(name_and_value.second)
|
||||
+ " and '" + toString((*inserted_name.first).first) + "'",
|
||||
ErrorCodes::SYNTAX_ERROR};
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
T EnumValues<T>::getValue(StringRef field_name, bool try_treat_as_id) const
|
||||
{
|
||||
const auto it = name_to_value_map.find(field_name);
|
||||
if (!it)
|
||||
{
|
||||
/// It is used in CSV and TSV input formats. If we fail to find given string in
|
||||
/// enum names, we will try to treat it as enum id.
|
||||
if (try_treat_as_id)
|
||||
{
|
||||
T x;
|
||||
ReadBufferFromMemory tmp_buf(field_name.data, field_name.size);
|
||||
readText(x, tmp_buf);
|
||||
/// Check if we reached end of the tmp_buf (otherwise field_name is not a number)
|
||||
/// and try to find it in enum ids
|
||||
if (tmp_buf.eof() && value_to_name_map.find(x) != value_to_name_map.end())
|
||||
return x;
|
||||
}
|
||||
throw Exception{"Unknown element '" + field_name.toString() + "' for enum", ErrorCodes::BAD_ARGUMENTS};
|
||||
}
|
||||
return it->getMapped();
|
||||
}
|
||||
|
||||
template class EnumValues<Int8>;
|
||||
template class EnumValues<Int16>;
|
||||
|
||||
}
|
71
src/DataTypes/EnumValues.h
Normal file
71
src/DataTypes/EnumValues.h
Normal file
@ -0,0 +1,71 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/HashTable/HashMap.h>
|
||||
#include <unordered_map>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int BAD_ARGUMENTS;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
class EnumValues
|
||||
{
|
||||
public:
|
||||
using Value = std::pair<std::string, T>;
|
||||
using Values = std::vector<Value>;
|
||||
using NameToValueMap = HashMap<StringRef, T, StringRefHash>;
|
||||
using ValueToNameMap = std::unordered_map<T, StringRef>;
|
||||
|
||||
private:
|
||||
Values values;
|
||||
NameToValueMap name_to_value_map;
|
||||
ValueToNameMap value_to_name_map;
|
||||
|
||||
void fillMaps();
|
||||
|
||||
public:
|
||||
EnumValues(const Values & values_);
|
||||
|
||||
const Values & getValues() const { return values; }
|
||||
|
||||
auto findByValue(const T & value) const
|
||||
{
|
||||
const auto it = value_to_name_map.find(value);
|
||||
if (it == std::end(value_to_name_map))
|
||||
throw Exception{"Unexpected value " + toString(value) + " in enum", ErrorCodes::BAD_ARGUMENTS};
|
||||
|
||||
return it;
|
||||
}
|
||||
|
||||
const StringRef & getNameForValue(const T & value) const
|
||||
{
|
||||
return findByValue(value)->second;
|
||||
}
|
||||
|
||||
T getValue(StringRef field_name, bool try_treat_as_id = false) const;
|
||||
|
||||
template <typename TValues>
|
||||
bool containsAll(const TValues & rhs_values) const
|
||||
{
|
||||
auto check = [&](const auto & value)
|
||||
{
|
||||
auto it = name_to_value_map.find(value.first);
|
||||
/// If we don't have this name, than we have to be sure,
|
||||
/// that this value exists in enum
|
||||
if (it == name_to_value_map.end())
|
||||
return value_to_name_map.count(value.second) > 0;
|
||||
|
||||
/// If we have this name, than it should have the same value
|
||||
return it->value.second == value.second;
|
||||
};
|
||||
|
||||
return std::all_of(rhs_values.begin(), rhs_values.end(), check);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -11,6 +11,9 @@
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <DataTypes/DataTypeCustom.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <DataTypes/Serializations/SerializationSparse.h>
|
||||
#include <DataTypes/Serializations/SerializationInfo.h>
|
||||
#include <DataTypes/Serializations/SerializationTupleElement.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -18,54 +21,11 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int MULTIPLE_STREAMS_REQUIRED;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int DATA_TYPE_CANNOT_BE_PROMOTED;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
}
|
||||
|
||||
String IDataType::Substream::toString() const
|
||||
{
|
||||
switch (type)
|
||||
{
|
||||
case ArrayElements:
|
||||
return "ArrayElements";
|
||||
case ArraySizes:
|
||||
return "ArraySizes";
|
||||
case NullableElements:
|
||||
return "NullableElements";
|
||||
case NullMap:
|
||||
return "NullMap";
|
||||
case TupleElement:
|
||||
return "TupleElement(" + tuple_element_name + ", "
|
||||
+ std::to_string(escape_tuple_delimiter) + ")";
|
||||
case DictionaryKeys:
|
||||
return "DictionaryKeys";
|
||||
case DictionaryIndexes:
|
||||
return "DictionaryIndexes";
|
||||
}
|
||||
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
String IDataType::SubstreamPath::toString() const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
wb << "{";
|
||||
for (size_t i = 0; i < size(); ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
wb << ", ";
|
||||
wb << at(i).toString();
|
||||
}
|
||||
wb << "}";
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
IDataType::IDataType() : custom_name(nullptr), custom_text_serialization(nullptr), custom_streams(nullptr)
|
||||
{
|
||||
}
|
||||
|
||||
IDataType::~IDataType() = default;
|
||||
|
||||
String IDataType::getName() const
|
||||
@ -119,21 +79,19 @@ DataTypePtr IDataType::promoteNumericType() const
|
||||
throw Exception("Data type " + getName() + " can't be promoted.", ErrorCodes::DATA_TYPE_CANNOT_BE_PROMOTED);
|
||||
}
|
||||
|
||||
void IDataType::serializeBinaryBulk(const IColumn &, WriteBuffer &, size_t, size_t) const
|
||||
{
|
||||
throw Exception("Data type " + getName() + " must be serialized with multiple streams", ErrorCodes::MULTIPLE_STREAMS_REQUIRED);
|
||||
}
|
||||
|
||||
void IDataType::deserializeBinaryBulk(IColumn &, ReadBuffer &, size_t, double) const
|
||||
{
|
||||
throw Exception("Data type " + getName() + " must be deserialized with multiple streams", ErrorCodes::MULTIPLE_STREAMS_REQUIRED);
|
||||
}
|
||||
|
||||
size_t IDataType::getSizeOfValueInMemory() const
|
||||
{
|
||||
throw Exception("Value of type " + getName() + " in memory is not of fixed size.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
DataTypePtr IDataType::tryGetSubcolumnType(const String & subcolumn_name) const
|
||||
{
|
||||
if (subcolumn_name == MAIN_SUBCOLUMN_NAME)
|
||||
return shared_from_this();
|
||||
|
||||
return nullptr;
|
||||
}
|
||||
|
||||
DataTypePtr IDataType::getSubcolumnType(const String & subcolumn_name) const
|
||||
{
|
||||
auto subcolumn_type = tryGetSubcolumnType(subcolumn_name);
|
||||
@ -151,14 +109,14 @@ ColumnPtr IDataType::getSubcolumn(const String & subcolumn_name, const IColumn &
|
||||
Names IDataType::getSubcolumnNames() const
|
||||
{
|
||||
NameSet res;
|
||||
enumerateStreams([&res, this](const SubstreamPath & substream_path, const IDataType & /* substream_type */)
|
||||
getDefaultSerialization()->enumerateStreams([&res, this](const ISerialization::SubstreamPath & substream_path)
|
||||
{
|
||||
SubstreamPath new_path;
|
||||
ISerialization::SubstreamPath new_path;
|
||||
/// Iterate over path to try to get intermediate subcolumns for complex nested types.
|
||||
for (const auto & elem : substream_path)
|
||||
{
|
||||
new_path.push_back(elem);
|
||||
auto subcolumn_name = getSubcolumnNameForStream(new_path);
|
||||
auto subcolumn_name = ISerialization::getSubcolumnNameForStream(new_path);
|
||||
if (!subcolumn_name.empty() && tryGetSubcolumnType(subcolumn_name))
|
||||
res.insert(subcolumn_name);
|
||||
}
|
||||
@ -167,287 +125,117 @@ Names IDataType::getSubcolumnNames() const
|
||||
return Names(std::make_move_iterator(res.begin()), std::make_move_iterator(res.end()));
|
||||
}
|
||||
|
||||
static String getNameForSubstreamPath(
|
||||
String stream_name,
|
||||
const IDataType::SubstreamPath & path,
|
||||
bool escape_tuple_delimiter)
|
||||
{
|
||||
size_t array_level = 0;
|
||||
for (const auto & elem : path)
|
||||
{
|
||||
if (elem.type == IDataType::Substream::NullMap)
|
||||
stream_name += ".null";
|
||||
else if (elem.type == IDataType::Substream::ArraySizes)
|
||||
stream_name += ".size" + toString(array_level);
|
||||
else if (elem.type == IDataType::Substream::ArrayElements)
|
||||
++array_level;
|
||||
else if (elem.type == IDataType::Substream::DictionaryKeys)
|
||||
stream_name += ".dict";
|
||||
else if (elem.type == IDataType::Substream::TupleElement)
|
||||
{
|
||||
/// For compatibility reasons, we use %2E (escaped dot) instead of dot.
|
||||
/// Because nested data may be represented not by Array of Tuple,
|
||||
/// but by separate Array columns with names in a form of a.b,
|
||||
/// and name is encoded as a whole.
|
||||
stream_name += (escape_tuple_delimiter && elem.escape_tuple_delimiter ?
|
||||
escapeForFileName(".") : ".") + escapeForFileName(elem.tuple_element_name);
|
||||
}
|
||||
}
|
||||
|
||||
return stream_name;
|
||||
}
|
||||
|
||||
String IDataType::getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path)
|
||||
{
|
||||
auto name_in_storage = column.getNameInStorage();
|
||||
auto nested_storage_name = Nested::extractTableName(name_in_storage);
|
||||
|
||||
if (name_in_storage != nested_storage_name && (path.size() == 1 && path[0].type == IDataType::Substream::ArraySizes))
|
||||
name_in_storage = nested_storage_name;
|
||||
|
||||
auto stream_name = escapeForFileName(name_in_storage);
|
||||
return getNameForSubstreamPath(std::move(stream_name), path, true);
|
||||
}
|
||||
|
||||
String IDataType::getSubcolumnNameForStream(const SubstreamPath & path)
|
||||
{
|
||||
auto subcolumn_name = getNameForSubstreamPath("", path, false);
|
||||
if (!subcolumn_name.empty())
|
||||
subcolumn_name = subcolumn_name.substr(1); // It starts with a dot.
|
||||
|
||||
return subcolumn_name;
|
||||
}
|
||||
|
||||
bool IDataType::isSpecialCompressionAllowed(const SubstreamPath & path)
|
||||
{
|
||||
for (const Substream & elem : path)
|
||||
{
|
||||
if (elem.type == Substream::NullMap
|
||||
|| elem.type == Substream::ArraySizes
|
||||
|| elem.type == Substream::DictionaryIndexes)
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
void IDataType::insertDefaultInto(IColumn & column) const
|
||||
{
|
||||
column.insertDefault();
|
||||
}
|
||||
|
||||
void IDataType::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const
|
||||
{
|
||||
if (custom_streams)
|
||||
custom_streams->enumerateStreams(callback, path);
|
||||
else
|
||||
enumerateStreamsImpl(callback, path);
|
||||
}
|
||||
|
||||
void IDataType::serializeBinaryBulkStatePrefix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
if (custom_streams)
|
||||
custom_streams->serializeBinaryBulkStatePrefix(settings, state);
|
||||
else
|
||||
serializeBinaryBulkStatePrefixImpl(settings, state);
|
||||
}
|
||||
|
||||
void IDataType::serializeBinaryBulkStateSuffix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
if (custom_streams)
|
||||
custom_streams->serializeBinaryBulkStateSuffix(settings, state);
|
||||
else
|
||||
serializeBinaryBulkStateSuffixImpl(settings, state);
|
||||
}
|
||||
|
||||
void IDataType::deserializeBinaryBulkStatePrefix(
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
if (custom_streams)
|
||||
custom_streams->deserializeBinaryBulkStatePrefix(settings, state);
|
||||
else
|
||||
deserializeBinaryBulkStatePrefixImpl(settings, state);
|
||||
}
|
||||
|
||||
void IDataType::serializeBinaryBulkWithMultipleStreams(
|
||||
const IColumn & column,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
if (custom_streams)
|
||||
custom_streams->serializeBinaryBulkWithMultipleStreams(column, offset, limit, settings, state);
|
||||
else
|
||||
serializeBinaryBulkWithMultipleStreamsImpl(column, offset, limit, settings, state);
|
||||
}
|
||||
|
||||
void IDataType::deserializeBinaryBulkWithMultipleStreamsImpl(
|
||||
IColumn & column,
|
||||
size_t limit,
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & /* state */,
|
||||
SubstreamsCache * /* cache */) const
|
||||
{
|
||||
if (ReadBuffer * stream = settings.getter(settings.path))
|
||||
deserializeBinaryBulk(column, *stream, limit, settings.avg_value_size_hint);
|
||||
}
|
||||
|
||||
|
||||
void IDataType::deserializeBinaryBulkWithMultipleStreams(
|
||||
ColumnPtr & column,
|
||||
size_t limit,
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state,
|
||||
SubstreamsCache * cache) const
|
||||
{
|
||||
if (custom_streams)
|
||||
{
|
||||
custom_streams->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, state, cache);
|
||||
return;
|
||||
}
|
||||
|
||||
/// Do not cache complex type, because they can be constructed
|
||||
/// from their subcolumns, which are in cache.
|
||||
if (!haveSubtypes())
|
||||
{
|
||||
auto cached_column = getFromSubstreamsCache(cache, settings.path);
|
||||
if (cached_column)
|
||||
{
|
||||
column = cached_column;
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
auto mutable_column = column->assumeMutable();
|
||||
deserializeBinaryBulkWithMultipleStreamsImpl(*mutable_column, limit, settings, state, cache);
|
||||
column = std::move(mutable_column);
|
||||
|
||||
if (!haveSubtypes())
|
||||
addToSubstreamsCache(cache, settings.path, column);
|
||||
}
|
||||
|
||||
void IDataType::serializeAsTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
if (custom_text_serialization)
|
||||
custom_text_serialization->serializeTextEscaped(column, row_num, ostr, settings);
|
||||
else
|
||||
serializeTextEscaped(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void IDataType::deserializeAsTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
if (custom_text_serialization)
|
||||
custom_text_serialization->deserializeTextEscaped(column, istr, settings);
|
||||
else
|
||||
deserializeTextEscaped(column, istr, settings);
|
||||
}
|
||||
|
||||
void IDataType::serializeAsTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
if (custom_text_serialization)
|
||||
custom_text_serialization->serializeTextQuoted(column, row_num, ostr, settings);
|
||||
else
|
||||
serializeTextQuoted(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void IDataType::deserializeAsTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
if (custom_text_serialization)
|
||||
custom_text_serialization->deserializeTextQuoted(column, istr, settings);
|
||||
else
|
||||
deserializeTextQuoted(column, istr, settings);
|
||||
}
|
||||
|
||||
void IDataType::serializeAsTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
if (custom_text_serialization)
|
||||
custom_text_serialization->serializeTextCSV(column, row_num, ostr, settings);
|
||||
else
|
||||
serializeTextCSV(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void IDataType::deserializeAsTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
if (custom_text_serialization)
|
||||
custom_text_serialization->deserializeTextCSV(column, istr, settings);
|
||||
else
|
||||
deserializeTextCSV(column, istr, settings);
|
||||
}
|
||||
|
||||
void IDataType::serializeAsText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
if (custom_text_serialization)
|
||||
custom_text_serialization->serializeText(column, row_num, ostr, settings);
|
||||
else
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void IDataType::deserializeAsWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
if (custom_text_serialization)
|
||||
custom_text_serialization->deserializeWholeText(column, istr, settings);
|
||||
else
|
||||
deserializeWholeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void IDataType::serializeAsTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
if (custom_text_serialization)
|
||||
custom_text_serialization->serializeTextJSON(column, row_num, ostr, settings);
|
||||
else
|
||||
serializeTextJSON(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void IDataType::deserializeAsTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
if (custom_text_serialization)
|
||||
custom_text_serialization->deserializeTextJSON(column, istr, settings);
|
||||
else
|
||||
deserializeTextJSON(column, istr, settings);
|
||||
}
|
||||
|
||||
void IDataType::serializeAsTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
if (custom_text_serialization)
|
||||
custom_text_serialization->serializeTextXML(column, row_num, ostr, settings);
|
||||
else
|
||||
serializeTextXML(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void IDataType::setCustomization(DataTypeCustomDescPtr custom_desc_) const
|
||||
{
|
||||
/// replace only if not null
|
||||
if (custom_desc_->name)
|
||||
custom_name = std::move(custom_desc_->name);
|
||||
|
||||
if (custom_desc_->text_serialization)
|
||||
custom_text_serialization = std::move(custom_desc_->text_serialization);
|
||||
|
||||
if (custom_desc_->streams)
|
||||
custom_streams = std::move(custom_desc_->streams);
|
||||
if (custom_desc_->serialization)
|
||||
custom_serialization = std::move(custom_desc_->serialization);
|
||||
}
|
||||
|
||||
void IDataType::addToSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path, ColumnPtr column)
|
||||
SerializationPtr IDataType::getDefaultSerialization() const
|
||||
{
|
||||
if (cache && !path.empty())
|
||||
cache->emplace(getSubcolumnNameForStream(path), column);
|
||||
if (custom_serialization)
|
||||
return custom_serialization;
|
||||
|
||||
return doGetDefaultSerialization();
|
||||
}
|
||||
|
||||
ColumnPtr IDataType::getFromSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path)
|
||||
SerializationPtr IDataType::getSparseSerialization() const
|
||||
{
|
||||
if (!cache || path.empty())
|
||||
return nullptr;
|
||||
return std::make_shared<SerializationSparse>(getDefaultSerialization());
|
||||
}
|
||||
|
||||
auto it = cache->find(getSubcolumnNameForStream(path));
|
||||
if (it == cache->end())
|
||||
return nullptr;
|
||||
SerializationPtr IDataType::getSubcolumnSerialization(const String & subcolumn_name, const BaseSerializationGetter &) const
|
||||
{
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "There is no subcolumn {} in type {}", subcolumn_name, getName());
|
||||
}
|
||||
|
||||
return it->second;
|
||||
SerializationPtr IDataType::getSerialization(const String & column_name, const SerializationInfo & info) const
|
||||
{
|
||||
ISerialization::Settings settings =
|
||||
{
|
||||
.num_rows = info.getNumberOfRows(),
|
||||
.num_non_default_rows = info.getNumberOfNonDefaultValues(column_name),
|
||||
.min_ratio_for_dense_serialization = 10
|
||||
};
|
||||
|
||||
return getSerialization(settings);
|
||||
}
|
||||
|
||||
SerializationPtr IDataType::getSerialization(const IColumn & column) const
|
||||
{
|
||||
ISerialization::Settings settings =
|
||||
{
|
||||
.num_rows = column.size(),
|
||||
.num_non_default_rows = column.getNumberOfNonDefaultValues(),
|
||||
.min_ratio_for_dense_serialization = 10
|
||||
};
|
||||
|
||||
return getSerialization(settings);
|
||||
}
|
||||
|
||||
SerializationPtr IDataType::getSerialization(const ISerialization::Settings & settings) const
|
||||
{
|
||||
// if (settings.num_non_default_rows * settings.min_ratio_for_dense_serialization < settings.num_rows)
|
||||
// return getSparseSerialization();
|
||||
|
||||
UNUSED(settings);
|
||||
|
||||
return getDefaultSerialization();
|
||||
}
|
||||
|
||||
// static
|
||||
SerializationPtr IDataType::getSerialization(const NameAndTypePair & column, const IDataType::StreamExistenceCallback & callback)
|
||||
{
|
||||
if (column.isSubcolumn())
|
||||
{
|
||||
auto base_serialization_getter = [&](const IDataType & subcolumn_type)
|
||||
{
|
||||
return subcolumn_type.getSerialization(column.name, callback);
|
||||
};
|
||||
|
||||
auto type_in_storage = column.getTypeInStorage();
|
||||
return type_in_storage->getSubcolumnSerialization(column.getSubcolumnName(), base_serialization_getter);
|
||||
}
|
||||
|
||||
return column.type->getSerialization(column.name, callback);
|
||||
}
|
||||
|
||||
SerializationPtr IDataType::getSerialization(const String & column_name, const StreamExistenceCallback & callback) const
|
||||
{
|
||||
auto sparse_idx_name = escapeForFileName(column_name) + ".sparse.idx";
|
||||
if (callback(sparse_idx_name))
|
||||
return getSparseSerialization();
|
||||
|
||||
UNUSED(column_name);
|
||||
UNUSED(callback);
|
||||
|
||||
return getDefaultSerialization();
|
||||
}
|
||||
|
||||
DataTypePtr IDataType::getTypeForSubstream(const ISerialization::SubstreamPath & substream_path) const
|
||||
{
|
||||
auto type = tryGetSubcolumnType(ISerialization::getSubcolumnNameForStream(substream_path));
|
||||
if (type)
|
||||
return type->getSubcolumnType(MAIN_SUBCOLUMN_NAME);
|
||||
|
||||
return getSubcolumnType(MAIN_SUBCOLUMN_NAME);
|
||||
}
|
||||
|
||||
void IDataType::enumerateStreams(const SerializationPtr & serialization, const StreamCallbackWithType & callback, ISerialization::SubstreamPath & path) const
|
||||
{
|
||||
serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path)
|
||||
{
|
||||
callback(substream_path, *getTypeForSubstream(substream_path));
|
||||
}, path);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -5,7 +5,7 @@
|
||||
#include <boost/noncopyable.hpp>
|
||||
#include <Core/Names.h>
|
||||
#include <Core/Types.h>
|
||||
#include <DataTypes/DataTypeCustom_fwd.h>
|
||||
#include <DataTypes/DataTypeCustom.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -27,6 +27,7 @@ using DataTypePtr = std::shared_ptr<const IDataType>;
|
||||
using DataTypes = std::vector<DataTypePtr>;
|
||||
|
||||
struct NameAndTypePair;
|
||||
class SerializationInfo;
|
||||
|
||||
|
||||
/** Properties of data type.
|
||||
@ -36,10 +37,10 @@ struct NameAndTypePair;
|
||||
*
|
||||
* DataType is totally immutable object. You can always share them.
|
||||
*/
|
||||
class IDataType : private boost::noncopyable
|
||||
class IDataType : private boost::noncopyable, public std::enable_shared_from_this<IDataType>
|
||||
{
|
||||
public:
|
||||
IDataType();
|
||||
IDataType() = default;
|
||||
virtual ~IDataType();
|
||||
|
||||
/// Compile time flag. If false, then if C++ types are the same, then SQL types are also the same.
|
||||
@ -57,275 +58,49 @@ public:
|
||||
/// Data type id. It's used for runtime type checks.
|
||||
virtual TypeIndex getTypeId() const = 0;
|
||||
|
||||
/** Binary serialization for range of values in column - for writing to disk/network, etc.
|
||||
*
|
||||
* Some data types are represented in multiple streams while being serialized.
|
||||
* Example:
|
||||
* - Arrays are represented as stream of all elements and stream of array sizes.
|
||||
* - Nullable types are represented as stream of values (with unspecified values in place of NULLs) and stream of NULL flags.
|
||||
*
|
||||
* Different streams are identified by "path".
|
||||
* If the data type require single stream (it's true for most of data types), the stream will have empty path.
|
||||
* Otherwise, the path can have components like "array elements", "array sizes", etc.
|
||||
*
|
||||
* For multidimensional arrays, path can have arbitrary length.
|
||||
* As an example, for 2-dimensional arrays of numbers we have at least three streams:
|
||||
* - array sizes; (sizes of top level arrays)
|
||||
* - array elements / array sizes; (sizes of second level (nested) arrays)
|
||||
* - array elements / array elements; (the most deep elements, placed contiguously)
|
||||
*
|
||||
* Descendants must override either serializeBinaryBulk, deserializeBinaryBulk methods (for simple cases with single stream)
|
||||
* or serializeBinaryBulkWithMultipleStreams, deserializeBinaryBulkWithMultipleStreams, enumerateStreams methods (for cases with multiple streams).
|
||||
*
|
||||
* Default implementations of ...WithMultipleStreams methods will call serializeBinaryBulk, deserializeBinaryBulk for single stream.
|
||||
*/
|
||||
|
||||
struct Substream
|
||||
{
|
||||
enum Type
|
||||
{
|
||||
ArrayElements,
|
||||
ArraySizes,
|
||||
|
||||
NullableElements,
|
||||
NullMap,
|
||||
|
||||
TupleElement,
|
||||
|
||||
DictionaryKeys,
|
||||
DictionaryIndexes,
|
||||
};
|
||||
Type type;
|
||||
|
||||
/// Index of tuple element, starting at 1 or name.
|
||||
String tuple_element_name;
|
||||
|
||||
/// Do we need to escape a dot in filenames for tuple elements.
|
||||
bool escape_tuple_delimiter = true;
|
||||
|
||||
Substream(Type type_) : type(type_) {}
|
||||
|
||||
String toString() const;
|
||||
};
|
||||
|
||||
struct SubstreamPath : public std::vector<Substream>
|
||||
{
|
||||
String toString() const;
|
||||
};
|
||||
|
||||
/// Cache for common substreams of one type, but possible different its subcolumns.
|
||||
/// E.g. sizes of arrays of Nested data type.
|
||||
using SubstreamsCache = std::unordered_map<String, ColumnPtr>;
|
||||
|
||||
using StreamCallback = std::function<void(const SubstreamPath &, const IDataType &)>;
|
||||
|
||||
void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const;
|
||||
void enumerateStreams(const StreamCallback & callback, SubstreamPath && path) const { enumerateStreams(callback, path); }
|
||||
void enumerateStreams(const StreamCallback & callback) const { enumerateStreams(callback, {}); }
|
||||
|
||||
virtual DataTypePtr tryGetSubcolumnType(const String & /* subcolumn_name */) const { return nullptr; }
|
||||
static constexpr auto MAIN_SUBCOLUMN_NAME = "__main";
|
||||
virtual DataTypePtr tryGetSubcolumnType(const String & subcolumn_name) const;
|
||||
DataTypePtr getSubcolumnType(const String & subcolumn_name) const;
|
||||
virtual ColumnPtr getSubcolumn(const String & subcolumn_name, const IColumn & column) const;
|
||||
Names getSubcolumnNames() const;
|
||||
|
||||
using OutputStreamGetter = std::function<WriteBuffer*(const SubstreamPath &)>;
|
||||
using InputStreamGetter = std::function<ReadBuffer*(const SubstreamPath &)>;
|
||||
|
||||
struct SerializeBinaryBulkState
|
||||
{
|
||||
virtual ~SerializeBinaryBulkState() = default;
|
||||
};
|
||||
struct DeserializeBinaryBulkState
|
||||
{
|
||||
virtual ~DeserializeBinaryBulkState() = default;
|
||||
};
|
||||
SerializationPtr getDefaultSerialization() const;
|
||||
SerializationPtr getSparseSerialization() const;
|
||||
|
||||
using SerializeBinaryBulkStatePtr = std::shared_ptr<SerializeBinaryBulkState>;
|
||||
using DeserializeBinaryBulkStatePtr = std::shared_ptr<DeserializeBinaryBulkState>;
|
||||
/// Asks wether the stream with given name exists in table.
|
||||
/// If callback returned true for all streams, which are required for
|
||||
/// one of serialization types, that serialization will be chosen for reading.
|
||||
/// If callback always returned false, the default serialization will be chosen.
|
||||
using StreamExistenceCallback = std::function<bool(const String &)>;
|
||||
using BaseSerializationGetter = std::function<SerializationPtr(const IDataType &)>;
|
||||
|
||||
struct SerializeBinaryBulkSettings
|
||||
{
|
||||
OutputStreamGetter getter;
|
||||
SubstreamPath path;
|
||||
virtual SerializationPtr getSerialization(const String & column_name, const StreamExistenceCallback & callback) const;
|
||||
virtual SerializationPtr getSubcolumnSerialization(
|
||||
const String & subcolumn_name, const BaseSerializationGetter & base_serialization_getter) const;
|
||||
|
||||
size_t low_cardinality_max_dictionary_size = 0;
|
||||
bool low_cardinality_use_single_dictionary_for_part = true;
|
||||
static SerializationPtr getSerialization(
|
||||
const NameAndTypePair & column,
|
||||
const StreamExistenceCallback & callback = [](const String &) { return false; });
|
||||
|
||||
bool position_independent_encoding = true;
|
||||
};
|
||||
virtual SerializationPtr getSerialization(const String & column_name, const SerializationInfo & info) const;
|
||||
|
||||
struct DeserializeBinaryBulkSettings
|
||||
{
|
||||
InputStreamGetter getter;
|
||||
SubstreamPath path;
|
||||
SerializationPtr getSerialization(const ISerialization::Settings & settings) const;
|
||||
SerializationPtr getSerialization(const IColumn & column) const;
|
||||
|
||||
/// True if continue reading from previous positions in file. False if made fseek to the start of new granule.
|
||||
bool continuous_reading = true;
|
||||
using StreamCallbackWithType = std::function<void(const ISerialization::SubstreamPath &, const IDataType &)>;
|
||||
|
||||
bool position_independent_encoding = true;
|
||||
/// If not zero, may be used to avoid reallocations while reading column of String type.
|
||||
double avg_value_size_hint = 0;
|
||||
};
|
||||
|
||||
/// Call before serializeBinaryBulkWithMultipleStreams chain to write something before first mark.
|
||||
void serializeBinaryBulkStatePrefix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const;
|
||||
|
||||
/// Call after serializeBinaryBulkWithMultipleStreams chain to finish serialization.
|
||||
void serializeBinaryBulkStateSuffix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const;
|
||||
|
||||
/// Call before before deserializeBinaryBulkWithMultipleStreams chain to get DeserializeBinaryBulkStatePtr.
|
||||
void deserializeBinaryBulkStatePrefix(
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state) const;
|
||||
|
||||
/** 'offset' and 'limit' are used to specify range.
|
||||
* limit = 0 - means no limit.
|
||||
* offset must be not greater than size of column.
|
||||
* offset + limit could be greater than size of column
|
||||
* - in that case, column is serialized till the end.
|
||||
*/
|
||||
void serializeBinaryBulkWithMultipleStreams(
|
||||
const IColumn & column,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const;
|
||||
|
||||
/// Read no more than limit values and append them into column.
|
||||
void deserializeBinaryBulkWithMultipleStreams(
|
||||
ColumnPtr & column,
|
||||
size_t limit,
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state,
|
||||
SubstreamsCache * cache = nullptr) const;
|
||||
|
||||
/** Override these methods for data types that require just single stream (most of data types).
|
||||
*/
|
||||
virtual void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const;
|
||||
virtual void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const;
|
||||
|
||||
/** Serialization/deserialization of individual values.
|
||||
*
|
||||
* These are helper methods for implementation of various formats to input/output for user (like CSV, JSON, etc.).
|
||||
* There is no one-to-one correspondence between formats and these methods.
|
||||
* For example, TabSeparated and Pretty formats could use same helper method serializeTextEscaped.
|
||||
*
|
||||
* For complex data types (like arrays) binary serde for individual values may differ from bulk serde.
|
||||
* For example, if you serialize single array, it will be represented as its size and elements in single contiguous stream,
|
||||
* but if you bulk serialize column with arrays, then sizes and elements will be written to separate streams.
|
||||
*/
|
||||
|
||||
/// There is two variants for binary serde. First variant work with Field.
|
||||
virtual void serializeBinary(const Field & field, WriteBuffer & ostr) const = 0;
|
||||
virtual void deserializeBinary(Field & field, ReadBuffer & istr) const = 0;
|
||||
|
||||
/// Other variants takes a column, to avoid creating temporary Field object.
|
||||
/// Column must be non-constant.
|
||||
|
||||
/// Serialize one value of a column at specified row number.
|
||||
virtual void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const = 0;
|
||||
/// Deserialize one value and insert into a column.
|
||||
/// If method will throw an exception, then column will be in same state as before call to method.
|
||||
virtual void deserializeBinary(IColumn & column, ReadBuffer & istr) const = 0;
|
||||
|
||||
/** Text serialization with escaping but without quoting.
|
||||
*/
|
||||
void serializeAsTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
|
||||
|
||||
void deserializeAsTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
|
||||
|
||||
/** Text serialization as a literal that may be inserted into a query.
|
||||
*/
|
||||
void serializeAsTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
|
||||
|
||||
void deserializeAsTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
|
||||
|
||||
/** Text serialization for the CSV format.
|
||||
*/
|
||||
void serializeAsTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
|
||||
void deserializeAsTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
|
||||
|
||||
/** Text serialization for displaying on a terminal or saving into a text file, and the like.
|
||||
* Without escaping or quoting.
|
||||
*/
|
||||
void serializeAsText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
|
||||
|
||||
/** Text deserialization in case when buffer contains only one value, without any escaping and delimiters.
|
||||
*/
|
||||
void deserializeAsWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
|
||||
|
||||
/** Text serialization intended for using in JSON format.
|
||||
*/
|
||||
void serializeAsTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const;
|
||||
void deserializeAsTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const;
|
||||
|
||||
/** Text serialization for putting into the XML format.
|
||||
*/
|
||||
void serializeAsTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const;
|
||||
void enumerateStreams(const SerializationPtr & serialization, const StreamCallbackWithType & callback, ISerialization::SubstreamPath & path) const;
|
||||
void enumerateStreams(const SerializationPtr & serialization, const StreamCallbackWithType & callback, ISerialization::SubstreamPath && path) const { enumerateStreams(serialization, callback, path); }
|
||||
void enumerateStreams(const SerializationPtr & serialization, const StreamCallbackWithType & callback) const { enumerateStreams(serialization, callback, {}); }
|
||||
|
||||
protected:
|
||||
virtual String doGetName() const;
|
||||
virtual SerializationPtr doGetDefaultSerialization() const = 0;
|
||||
|
||||
virtual void enumerateStreamsImpl(const StreamCallback & callback, SubstreamPath & path) const
|
||||
{
|
||||
callback(path, *this);
|
||||
}
|
||||
|
||||
virtual void serializeBinaryBulkStatePrefixImpl(
|
||||
SerializeBinaryBulkSettings & /*settings*/,
|
||||
SerializeBinaryBulkStatePtr & /*state*/) const {}
|
||||
|
||||
virtual void serializeBinaryBulkStateSuffixImpl(
|
||||
SerializeBinaryBulkSettings & /*settings*/,
|
||||
SerializeBinaryBulkStatePtr & /*state*/) const {}
|
||||
|
||||
virtual void deserializeBinaryBulkStatePrefixImpl(
|
||||
DeserializeBinaryBulkSettings & /*settings*/,
|
||||
DeserializeBinaryBulkStatePtr & /*state*/) const {}
|
||||
|
||||
virtual void serializeBinaryBulkWithMultipleStreamsImpl(
|
||||
const IColumn & column,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & /*state*/) const
|
||||
{
|
||||
if (WriteBuffer * stream = settings.getter(settings.path))
|
||||
serializeBinaryBulk(column, *stream, offset, limit);
|
||||
}
|
||||
|
||||
virtual void deserializeBinaryBulkWithMultipleStreamsImpl(
|
||||
IColumn & column,
|
||||
size_t limit,
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state,
|
||||
SubstreamsCache * cache) const;
|
||||
|
||||
/// Default implementations of text serialization in case of 'custom_text_serialization' is not set.
|
||||
|
||||
virtual void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
virtual void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
virtual void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
virtual void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
virtual void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
virtual void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
virtual void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
virtual void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
virtual void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
virtual void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
virtual void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
DataTypePtr getTypeForSubstream(const ISerialization::SubstreamPath & substream_path) const;
|
||||
|
||||
public:
|
||||
static void addToSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path, ColumnPtr column);
|
||||
static ColumnPtr getFromSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path);
|
||||
|
||||
/** Create empty column for corresponding type.
|
||||
*/
|
||||
virtual MutableColumnPtr createColumn() const = 0;
|
||||
@ -357,7 +132,6 @@ public:
|
||||
/// Checks that two instances belong to the same type
|
||||
virtual bool equals(const IDataType & rhs) const = 0;
|
||||
|
||||
|
||||
/// Various properties on behaviour of data type.
|
||||
|
||||
/** The data type is dependent on parameters and types with different parameters are different.
|
||||
@ -483,27 +257,20 @@ public:
|
||||
/// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column.
|
||||
static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint);
|
||||
|
||||
static String getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path);
|
||||
static String getSubcolumnNameForStream(const SubstreamPath & path);
|
||||
|
||||
/// Substream path supports special compression methods like codec Delta.
|
||||
/// For all other substreams (like ArraySizes, NullMasks, etc.) we use only
|
||||
/// generic compression codecs like LZ4.
|
||||
static bool isSpecialCompressionAllowed(const SubstreamPath & path);
|
||||
protected:
|
||||
friend class DataTypeFactory;
|
||||
friend class AggregateFunctionSimpleState;
|
||||
|
||||
/// Customize this DataType
|
||||
void setCustomization(DataTypeCustomDescPtr custom_desc_) const;
|
||||
|
||||
/// This is mutable to allow setting custom name and serialization on `const IDataType` post construction.
|
||||
mutable DataTypeCustomNamePtr custom_name;
|
||||
mutable DataTypeCustomTextSerializationPtr custom_text_serialization;
|
||||
mutable DataTypeCustomStreamsPtr custom_streams;
|
||||
mutable SerializationPtr custom_serialization;
|
||||
|
||||
public:
|
||||
const IDataTypeCustomName * getCustomName() const { return custom_name.get(); }
|
||||
const IDataTypeCustomStreams * getCustomStreams() const { return custom_streams.get(); }
|
||||
const ISerialization * getCustomSerialization() const { return custom_serialization.get(); }
|
||||
};
|
||||
|
||||
|
||||
|
@ -1,6 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/DataTypeWithSimpleSerialization.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <Core/Field.h>
|
||||
|
||||
|
||||
@ -17,7 +17,7 @@ namespace ErrorCodes
|
||||
*
|
||||
* That is, this class is used just to distinguish the corresponding data type from the others.
|
||||
*/
|
||||
class IDataTypeDummy : public DataTypeWithSimpleSerialization
|
||||
class IDataTypeDummy : public IDataType
|
||||
{
|
||||
private:
|
||||
[[noreturn]] void throwNoSerialization() const
|
||||
@ -26,15 +26,6 @@ private:
|
||||
}
|
||||
|
||||
public:
|
||||
void serializeBinary(const Field &, WriteBuffer &) const override { throwNoSerialization(); }
|
||||
void deserializeBinary(Field &, ReadBuffer &) const override { throwNoSerialization(); }
|
||||
void serializeBinary(const IColumn &, size_t, WriteBuffer &) const override { throwNoSerialization(); }
|
||||
void deserializeBinary(IColumn &, ReadBuffer &) const override { throwNoSerialization(); }
|
||||
void serializeBinaryBulk(const IColumn &, WriteBuffer &, size_t, size_t) const override { throwNoSerialization(); }
|
||||
void deserializeBinaryBulk(IColumn &, ReadBuffer &, size_t, double) const override { throwNoSerialization(); }
|
||||
void serializeText(const IColumn &, size_t, WriteBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
|
||||
void deserializeText(IColumn &, ReadBuffer &, const FormatSettings &) const override { throwNoSerialization(); }
|
||||
|
||||
MutableColumnPtr createColumn() const override
|
||||
{
|
||||
throw Exception("Method createColumn() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
@ -52,6 +43,8 @@ public:
|
||||
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool cannotBeStoredInTables() const override { return true; }
|
||||
|
||||
SerializationPtr doGetDefaultSerialization() const override { throwNoSerialization(); }
|
||||
};
|
||||
|
||||
}
|
||||
|
197
src/DataTypes/Serializations/ISerialization.cpp
Normal file
197
src/DataTypes/Serializations/ISerialization.cpp
Normal file
@ -0,0 +1,197 @@
|
||||
#include <DataTypes/Serializations/ISerialization.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/Operators.h>
|
||||
#include <Common/escapeForFileName.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int MULTIPLE_STREAMS_REQUIRED;
|
||||
}
|
||||
|
||||
String ISerialization::Substream::toString() const
|
||||
{
|
||||
switch (type)
|
||||
{
|
||||
case ArrayElements:
|
||||
return "ArrayElements";
|
||||
case ArraySizes:
|
||||
return "ArraySizes";
|
||||
case NullableElements:
|
||||
return "NullableElements";
|
||||
case NullMap:
|
||||
return "NullMap";
|
||||
case TupleElement:
|
||||
return "TupleElement(" + tuple_element_name + ", "
|
||||
+ std::to_string(escape_tuple_delimiter) + ")";
|
||||
case DictionaryKeys:
|
||||
return "DictionaryKeys";
|
||||
case DictionaryIndexes:
|
||||
return "DictionaryIndexes";
|
||||
case SparseElements:
|
||||
return "SparseElements";
|
||||
case SparseOffsets:
|
||||
return "SparseOffsets";
|
||||
}
|
||||
|
||||
__builtin_unreachable();
|
||||
}
|
||||
|
||||
String ISerialization::SubstreamPath::toString() const
|
||||
{
|
||||
WriteBufferFromOwnString wb;
|
||||
wb << "{";
|
||||
for (size_t i = 0; i < size(); ++i)
|
||||
{
|
||||
if (i != 0)
|
||||
wb << ", ";
|
||||
wb << at(i).toString();
|
||||
}
|
||||
wb << "}";
|
||||
return wb.str();
|
||||
}
|
||||
|
||||
void ISerialization::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const
|
||||
{
|
||||
callback(path);
|
||||
}
|
||||
|
||||
void ISerialization::serializeBinaryBulk(const IColumn & column, WriteBuffer &, size_t, size_t) const
|
||||
{
|
||||
throw Exception(ErrorCodes::MULTIPLE_STREAMS_REQUIRED, "Column {} must be serialized with multiple streams", column.getName());
|
||||
}
|
||||
|
||||
void ISerialization::deserializeBinaryBulk(IColumn & column, ReadBuffer &, size_t, double) const
|
||||
{
|
||||
throw Exception(ErrorCodes::MULTIPLE_STREAMS_REQUIRED, "Column {} must be deserialized with multiple streams", column.getName());
|
||||
}
|
||||
|
||||
void ISerialization::serializeBinaryBulkWithMultipleStreams(
|
||||
const IColumn & column,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & /* state */) const
|
||||
{
|
||||
if (WriteBuffer * stream = settings.getter(settings.path))
|
||||
serializeBinaryBulk(column, *stream, offset, limit);
|
||||
}
|
||||
|
||||
void ISerialization::deserializeBinaryBulkWithMultipleStreams(
|
||||
ColumnPtr & column,
|
||||
size_t limit,
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & /* state */,
|
||||
SubstreamsCache * cache) const
|
||||
{
|
||||
auto cached_column = getFromSubstreamsCache(cache, settings.path);
|
||||
if (cached_column)
|
||||
{
|
||||
column = cached_column;
|
||||
}
|
||||
else if (ReadBuffer * stream = settings.getter(settings.path))
|
||||
{
|
||||
auto mutable_column = column->assumeMutable();
|
||||
deserializeBinaryBulk(*mutable_column, *stream, limit, settings.avg_value_size_hint);
|
||||
column = std::move(mutable_column);
|
||||
addToSubstreamsCache(cache, settings.path, column);
|
||||
}
|
||||
}
|
||||
|
||||
static String getNameForSubstreamPath(
|
||||
String stream_name,
|
||||
const ISerialization::SubstreamPath & path,
|
||||
bool escape_tuple_delimiter)
|
||||
{
|
||||
using Substream = ISerialization::Substream;
|
||||
|
||||
size_t array_level = 0;
|
||||
for (const auto & elem : path)
|
||||
{
|
||||
if (elem.type == Substream::NullMap)
|
||||
stream_name += ".null";
|
||||
else if (elem.type == Substream::ArraySizes)
|
||||
stream_name += ".size" + toString(array_level);
|
||||
else if (elem.type == Substream::ArrayElements)
|
||||
++array_level;
|
||||
else if (elem.type == Substream::DictionaryKeys)
|
||||
stream_name += ".dict";
|
||||
else if (elem.type == Substream::SparseOffsets)
|
||||
stream_name += ".sparse.idx";
|
||||
else if (elem.type == Substream::TupleElement)
|
||||
{
|
||||
/// For compatibility reasons, we use %2E (escaped dot) instead of dot.
|
||||
/// Because nested data may be represented not by Array of Tuple,
|
||||
/// but by separate Array columns with names in a form of a.b,
|
||||
/// and name is encoded as a whole.
|
||||
stream_name += (escape_tuple_delimiter && elem.escape_tuple_delimiter ?
|
||||
escapeForFileName(".") : ".") + escapeForFileName(elem.tuple_element_name);
|
||||
}
|
||||
}
|
||||
|
||||
return stream_name;
|
||||
}
|
||||
|
||||
String ISerialization::getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path)
|
||||
{
|
||||
return getFileNameForStream(column.getNameInStorage(), path);
|
||||
}
|
||||
|
||||
String ISerialization::getFileNameForStream(const String & name_in_storage, const SubstreamPath & path)
|
||||
{
|
||||
String stream_name;
|
||||
auto nested_storage_name = Nested::extractTableName(name_in_storage);
|
||||
if (name_in_storage != nested_storage_name && (path.size() == 1 && path[0].type == ISerialization::Substream::ArraySizes))
|
||||
stream_name = escapeForFileName(nested_storage_name);
|
||||
else
|
||||
stream_name = escapeForFileName(name_in_storage);
|
||||
|
||||
return getNameForSubstreamPath(std::move(stream_name), path, true);
|
||||
}
|
||||
|
||||
String ISerialization::getSubcolumnNameForStream(const SubstreamPath & path)
|
||||
{
|
||||
auto subcolumn_name = getNameForSubstreamPath("", path, false);
|
||||
if (!subcolumn_name.empty())
|
||||
subcolumn_name = subcolumn_name.substr(1); // It starts with a dot.
|
||||
|
||||
return subcolumn_name;
|
||||
}
|
||||
|
||||
void ISerialization::addToSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path, ColumnPtr column)
|
||||
{
|
||||
if (cache && !path.empty())
|
||||
cache->emplace(getSubcolumnNameForStream(path), column);
|
||||
}
|
||||
|
||||
ColumnPtr ISerialization::getFromSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path)
|
||||
{
|
||||
if (!cache || path.empty())
|
||||
return nullptr;
|
||||
|
||||
auto it = cache->find(getSubcolumnNameForStream(path));
|
||||
if (it == cache->end())
|
||||
return nullptr;
|
||||
|
||||
return it->second;
|
||||
}
|
||||
|
||||
bool ISerialization::isSpecialCompressionAllowed(const SubstreamPath & path)
|
||||
{
|
||||
for (const auto & elem : path)
|
||||
{
|
||||
if (elem.type == Substream::NullMap
|
||||
|| elem.type == Substream::ArraySizes
|
||||
|| elem.type == Substream::DictionaryIndexes
|
||||
|| elem.type == Substream::SparseOffsets)
|
||||
return false;
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
266
src/DataTypes/Serializations/ISerialization.h
Normal file
266
src/DataTypes/Serializations/ISerialization.h
Normal file
@ -0,0 +1,266 @@
|
||||
#pragma once
|
||||
|
||||
#include <Common/COW.h>
|
||||
#include <Core/Types.h>
|
||||
|
||||
#include <unordered_map>
|
||||
#include <memory>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class IDataType;
|
||||
|
||||
class ReadBuffer;
|
||||
class WriteBuffer;
|
||||
class ProtobufReader;
|
||||
class ProtobufWriter;
|
||||
|
||||
class IColumn;
|
||||
using ColumnPtr = COW<IColumn>::Ptr;
|
||||
using MutableColumnPtr = COW<IColumn>::MutablePtr;
|
||||
|
||||
class Field;
|
||||
|
||||
struct FormatSettings;
|
||||
struct NameAndTypePair;
|
||||
|
||||
class ISerialization
|
||||
{
|
||||
public:
|
||||
ISerialization() = default;
|
||||
virtual ~ISerialization() = default;
|
||||
|
||||
/** Binary serialization for range of values in column - for writing to disk/network, etc.
|
||||
*
|
||||
* Some data types are represented in multiple streams while being serialized.
|
||||
* Example:
|
||||
* - Arrays are represented as stream of all elements and stream of array sizes.
|
||||
* - Nullable types are represented as stream of values (with unspecified values in place of NULLs) and stream of NULL flags.
|
||||
*
|
||||
* Different streams are identified by "path".
|
||||
* If the data type require single stream (it's true for most of data types), the stream will have empty path.
|
||||
* Otherwise, the path can have components like "array elements", "array sizes", etc.
|
||||
*
|
||||
* For multidimensional arrays, path can have arbitrary length.
|
||||
* As an example, for 2-dimensional arrays of numbers we have at least three streams:
|
||||
* - array sizes; (sizes of top level arrays)
|
||||
* - array elements / array sizes; (sizes of second level (nested) arrays)
|
||||
* - array elements / array elements; (the most deep elements, placed contiguously)
|
||||
*
|
||||
* Descendants must override either serializeBinaryBulk, deserializeBinaryBulk methods (for simple cases with single stream)
|
||||
* or serializeBinaryBulkWithMultipleStreams, deserializeBinaryBulkWithMultipleStreams, enumerateStreams methods (for cases with multiple streams).
|
||||
*
|
||||
* Default implementations of ...WithMultipleStreams methods will call serializeBinaryBulk, deserializeBinaryBulk for single stream.
|
||||
*/
|
||||
|
||||
struct Substream
|
||||
{
|
||||
enum Type
|
||||
{
|
||||
ArrayElements,
|
||||
ArraySizes,
|
||||
|
||||
NullableElements,
|
||||
NullMap,
|
||||
|
||||
TupleElement,
|
||||
|
||||
DictionaryKeys,
|
||||
DictionaryIndexes,
|
||||
|
||||
SparseElements,
|
||||
SparseOffsets,
|
||||
};
|
||||
Type type;
|
||||
|
||||
/// Index of tuple element, starting at 1 or name.
|
||||
String tuple_element_name;
|
||||
|
||||
/// Do we need to escape a dot in filenames for tuple elements.
|
||||
bool escape_tuple_delimiter = true;
|
||||
|
||||
Substream(Type type_) : type(type_) {}
|
||||
|
||||
String toString() const;
|
||||
};
|
||||
|
||||
struct SubstreamPath : public std::vector<Substream>
|
||||
{
|
||||
String toString() const;
|
||||
};
|
||||
|
||||
/// Cache for common substreams of one type, but possible different its subcolumns.
|
||||
/// E.g. sizes of arrays of Nested data type.
|
||||
using SubstreamsCache = std::unordered_map<String, ColumnPtr>;
|
||||
|
||||
using StreamCallback = std::function<void(const SubstreamPath &)>;
|
||||
|
||||
virtual void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const;
|
||||
void enumerateStreams(const StreamCallback & callback, SubstreamPath && path) const { enumerateStreams(callback, path); }
|
||||
void enumerateStreams(const StreamCallback & callback) const { enumerateStreams(callback, {}); }
|
||||
|
||||
using OutputStreamGetter = std::function<WriteBuffer*(const SubstreamPath &)>;
|
||||
using InputStreamGetter = std::function<ReadBuffer*(const SubstreamPath &)>;
|
||||
|
||||
struct SerializeBinaryBulkState
|
||||
{
|
||||
virtual ~SerializeBinaryBulkState() = default;
|
||||
};
|
||||
|
||||
struct DeserializeBinaryBulkState
|
||||
{
|
||||
virtual ~DeserializeBinaryBulkState() = default;
|
||||
};
|
||||
|
||||
using SerializeBinaryBulkStatePtr = std::shared_ptr<SerializeBinaryBulkState>;
|
||||
using DeserializeBinaryBulkStatePtr = std::shared_ptr<DeserializeBinaryBulkState>;
|
||||
|
||||
struct SerializeBinaryBulkSettings
|
||||
{
|
||||
OutputStreamGetter getter;
|
||||
SubstreamPath path;
|
||||
|
||||
size_t low_cardinality_max_dictionary_size = 0;
|
||||
bool low_cardinality_use_single_dictionary_for_part = true;
|
||||
|
||||
bool position_independent_encoding = true;
|
||||
};
|
||||
|
||||
struct DeserializeBinaryBulkSettings
|
||||
{
|
||||
InputStreamGetter getter;
|
||||
SubstreamPath path;
|
||||
|
||||
/// True if continue reading from previous positions in file. False if made fseek to the start of new granule.
|
||||
bool continuous_reading = true;
|
||||
|
||||
bool position_independent_encoding = true;
|
||||
/// If not zero, may be used to avoid reallocations while reading column of String type.
|
||||
double avg_value_size_hint = 0;
|
||||
};
|
||||
|
||||
struct Settings
|
||||
{
|
||||
size_t num_rows;
|
||||
size_t num_non_default_rows;
|
||||
size_t min_ratio_for_dense_serialization;
|
||||
};
|
||||
|
||||
/// Call before serializeBinaryBulkWithMultipleStreams chain to write something before first mark.
|
||||
virtual void serializeBinaryBulkStatePrefix(
|
||||
SerializeBinaryBulkSettings & /*settings*/,
|
||||
SerializeBinaryBulkStatePtr & /*state*/) const {}
|
||||
|
||||
/// Call after serializeBinaryBulkWithMultipleStreams chain to finish serialization.
|
||||
virtual void serializeBinaryBulkStateSuffix(
|
||||
SerializeBinaryBulkSettings & /*settings*/,
|
||||
SerializeBinaryBulkStatePtr & /*state*/) const {}
|
||||
|
||||
/// Call before before deserializeBinaryBulkWithMultipleStreams chain to get DeserializeBinaryBulkStatePtr.
|
||||
virtual void deserializeBinaryBulkStatePrefix(
|
||||
DeserializeBinaryBulkSettings & /*settings*/,
|
||||
DeserializeBinaryBulkStatePtr & /*state*/) const {}
|
||||
|
||||
/** 'offset' and 'limit' are used to specify range.
|
||||
* limit = 0 - means no limit.
|
||||
* offset must be not greater than size of column.
|
||||
* offset + limit could be greater than size of column
|
||||
* - in that case, column is serialized till the end.
|
||||
*/
|
||||
virtual void serializeBinaryBulkWithMultipleStreams(
|
||||
const IColumn & column,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const;
|
||||
|
||||
/// Read no more than limit values and append them into column.
|
||||
virtual void deserializeBinaryBulkWithMultipleStreams(
|
||||
ColumnPtr & column,
|
||||
size_t limit,
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state,
|
||||
SubstreamsCache * cache = nullptr) const;
|
||||
|
||||
/** Override these methods for data types that require just single stream (most of data types).
|
||||
*/
|
||||
virtual void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const;
|
||||
virtual void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const;
|
||||
|
||||
/** Serialization/deserialization of individual values.
|
||||
*
|
||||
* These are helper methods for implementation of various formats to input/output for user (like CSV, JSON, etc.).
|
||||
* There is no one-to-one correspondence between formats and these methods.
|
||||
* For example, TabSeparated and Pretty formats could use same helper method serializeTextEscaped.
|
||||
*
|
||||
* For complex data types (like arrays) binary serde for individual values may differ from bulk serde.
|
||||
* For example, if you serialize single array, it will be represented as its size and elements in single contiguous stream,
|
||||
* but if you bulk serialize column with arrays, then sizes and elements will be written to separate streams.
|
||||
*/
|
||||
|
||||
/// There is two variants for binary serde. First variant work with Field.
|
||||
virtual void serializeBinary(const Field & field, WriteBuffer & ostr) const = 0;
|
||||
virtual void deserializeBinary(Field & field, ReadBuffer & istr) const = 0;
|
||||
|
||||
/// Other variants takes a column, to avoid creating temporary Field object.
|
||||
/// Column must be non-constant.
|
||||
|
||||
/// Serialize one value of a column at specified row number.
|
||||
virtual void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const = 0;
|
||||
/// Deserialize one value and insert into a column.
|
||||
/// If method will throw an exception, then column will be in same state as before call to method.
|
||||
virtual void deserializeBinary(IColumn & column, ReadBuffer & istr) const = 0;
|
||||
|
||||
/** Text serialization with escaping but without quoting.
|
||||
*/
|
||||
virtual void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
|
||||
virtual void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
|
||||
/** Text serialization as a literal that may be inserted into a query.
|
||||
*/
|
||||
virtual void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
|
||||
virtual void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
|
||||
/** Text serialization for the CSV format.
|
||||
*/
|
||||
virtual void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
virtual void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
|
||||
/** Text serialization for displaying on a terminal or saving into a text file, and the like.
|
||||
* Without escaping or quoting.
|
||||
*/
|
||||
virtual void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
|
||||
/** Text deserialization in case when buffer contains only one value, without any escaping and delimiters.
|
||||
*/
|
||||
virtual void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
|
||||
/** Text serialization intended for using in JSON format.
|
||||
*/
|
||||
virtual void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const = 0;
|
||||
virtual void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
|
||||
/** Text serialization for putting into the XML format.
|
||||
*/
|
||||
virtual void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
static String getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path);
|
||||
static String getFileNameForStream(const String & name_in_storage, const SubstreamPath & path);
|
||||
static String getSubcolumnNameForStream(const SubstreamPath & path);
|
||||
|
||||
static void addToSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path, ColumnPtr column);
|
||||
static ColumnPtr getFromSubstreamsCache(SubstreamsCache * cache, const SubstreamPath & path);
|
||||
|
||||
static bool isSpecialCompressionAllowed(const SubstreamPath & path);
|
||||
};
|
||||
|
||||
using SerializationPtr = std::shared_ptr<const ISerialization>;
|
||||
using Serializations = std::vector<SerializationPtr>;
|
||||
|
||||
}
|
@ -0,0 +1,97 @@
|
||||
#include <DataTypes/Serializations/SerializationCustomSimpleText.h>
|
||||
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
using namespace DB;
|
||||
|
||||
String serializeToString(const SerializationCustomSimpleText & domain, const IColumn & column, size_t row_num, const FormatSettings & settings)
|
||||
{
|
||||
WriteBufferFromOwnString buffer;
|
||||
domain.serializeText(column, row_num, buffer, settings);
|
||||
|
||||
return buffer.str();
|
||||
}
|
||||
|
||||
void deserializeFromString(const SerializationCustomSimpleText & domain, IColumn & column, const String & s, const FormatSettings & settings)
|
||||
{
|
||||
ReadBufferFromString istr(s);
|
||||
domain.deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
SerializationCustomSimpleText::SerializationCustomSimpleText(const SerializationPtr & nested_)
|
||||
: SerializationWrapper(nested_)
|
||||
{
|
||||
}
|
||||
|
||||
void SerializationCustomSimpleText::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
String str;
|
||||
readString(str, istr);
|
||||
deserializeFromString(*this, column, str, settings);
|
||||
}
|
||||
|
||||
void SerializationCustomSimpleText::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeEscapedString(serializeToString(*this, column, row_num, settings), ostr);
|
||||
}
|
||||
|
||||
void SerializationCustomSimpleText::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
String str;
|
||||
readEscapedString(str, istr);
|
||||
deserializeFromString(*this, column, str, settings);
|
||||
}
|
||||
|
||||
void SerializationCustomSimpleText::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeQuotedString(serializeToString(*this, column, row_num, settings), ostr);
|
||||
}
|
||||
|
||||
void SerializationCustomSimpleText::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
String str;
|
||||
readQuotedString(str, istr);
|
||||
deserializeFromString(*this, column, str, settings);
|
||||
}
|
||||
|
||||
void SerializationCustomSimpleText::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeCSVString(serializeToString(*this, column, row_num, settings), ostr);
|
||||
}
|
||||
|
||||
void SerializationCustomSimpleText::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
String str;
|
||||
readCSVString(str, istr, settings.csv);
|
||||
deserializeFromString(*this, column, str, settings);
|
||||
}
|
||||
|
||||
void SerializationCustomSimpleText::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeJSONString(serializeToString(*this, column, row_num, settings), ostr, settings);
|
||||
}
|
||||
|
||||
void SerializationCustomSimpleText::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
String str;
|
||||
readJSONString(str, istr);
|
||||
deserializeFromString(*this, column, str, settings);
|
||||
}
|
||||
|
||||
void SerializationCustomSimpleText::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeXMLStringForTextElement(serializeToString(*this, column, row_num, settings), ostr);
|
||||
}
|
||||
|
||||
}
|
57
src/DataTypes/Serializations/SerializationCustomSimpleText.h
Normal file
57
src/DataTypes/Serializations/SerializationCustomSimpleText.h
Normal file
@ -0,0 +1,57 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/SerializationWrapper.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ReadBuffer;
|
||||
class WriteBuffer;
|
||||
struct FormatSettings;
|
||||
class IColumn;
|
||||
|
||||
/** Simple IDataTypeCustomTextSerialization that uses serializeText/deserializeText
|
||||
* for all serialization and deserialization. */
|
||||
class SerializationCustomSimpleText : public SerializationWrapper
|
||||
{
|
||||
public:
|
||||
SerializationCustomSimpleText(const SerializationPtr & nested_);
|
||||
|
||||
// Methods that subclasses must override in order to get full serialization/deserialization support.
|
||||
virtual void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override = 0;
|
||||
virtual void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
|
||||
/** Text deserialization without quoting or escaping.
|
||||
*/
|
||||
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
/** Text serialization with escaping but without quoting.
|
||||
*/
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
/** Text serialization as a literal that may be inserted into a query.
|
||||
*/
|
||||
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
/** Text serialization for the CSV format.
|
||||
*/
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
/** delimiter - the delimiter we expect when reading a string value that is not double-quoted
|
||||
* (the delimiter is not consumed).
|
||||
*/
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
/** Text serialization intended for using in JSON format.
|
||||
* force_quoting_64bit_integers parameter forces to brace UInt64 and Int64 types into quotes.
|
||||
*/
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
/** Text serialization for putting into the XML format.
|
||||
*/
|
||||
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
};
|
||||
|
||||
}
|
74
src/DataTypes/Serializations/SerializationDecimal.cpp
Normal file
74
src/DataTypes/Serializations/SerializationDecimal.cpp
Normal file
@ -0,0 +1,74 @@
|
||||
#include <DataTypes/Serializations/SerializationDecimal.h>
|
||||
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Formats/ProtobufReader.h>
|
||||
#include <Formats/ProtobufWriter.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/readDecimalText.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int DECIMAL_OVERFLOW;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool SerializationDecimal<T>::tryReadText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale)
|
||||
{
|
||||
UInt32 unread_scale = scale;
|
||||
if (!tryReadDecimalText(istr, x, precision, unread_scale))
|
||||
return false;
|
||||
|
||||
if (common::mulOverflow(x.value, DecimalUtils::scaleMultiplier<T>(unread_scale), x.value))
|
||||
return false;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void SerializationDecimal<T>::readText(T & x, ReadBuffer & istr, UInt32 precision, UInt32 scale, bool csv)
|
||||
{
|
||||
UInt32 unread_scale = scale;
|
||||
if (csv)
|
||||
readCSVDecimalText(istr, x, precision, unread_scale);
|
||||
else
|
||||
readDecimalText(istr, x, precision, unread_scale);
|
||||
|
||||
if (common::mulOverflow(x.value, DecimalUtils::scaleMultiplier<T>(unread_scale), x.value))
|
||||
throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void SerializationDecimal<T>::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
{
|
||||
T value = assert_cast<const ColumnType &>(column).getData()[row_num];
|
||||
writeText(value, this->scale, ostr);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void SerializationDecimal<T>::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
T x;
|
||||
readText(x, istr);
|
||||
assert_cast<ColumnType &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void SerializationDecimal<T>::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
T x;
|
||||
readText(x, istr, true);
|
||||
assert_cast<ColumnType &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
template class SerializationDecimal<Decimal32>;
|
||||
template class SerializationDecimal<Decimal64>;
|
||||
template class SerializationDecimal<Decimal128>;
|
||||
template class SerializationDecimal<Decimal256>;
|
||||
|
||||
}
|
27
src/DataTypes/Serializations/SerializationDecimal.h
Normal file
27
src/DataTypes/Serializations/SerializationDecimal.h
Normal file
@ -0,0 +1,27 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/SerializationDecimalBase.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
class SerializationDecimal final : public SerializationDecimalBase<T>
|
||||
{
|
||||
public:
|
||||
using typename SerializationDecimalBase<T>::ColumnType;
|
||||
|
||||
SerializationDecimal(UInt32 precision_, UInt32 scale_)
|
||||
: SerializationDecimalBase<T>(precision_, scale_) {}
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
void readText(T & x, ReadBuffer & istr, bool csv = false) const { readText(x, istr, this->precision, this->scale, csv); }
|
||||
|
||||
static void readText(T & x, ReadBuffer & istr, UInt32 precision_, UInt32 scale_, bool csv = false);
|
||||
static bool tryReadText(T & x, ReadBuffer & istr, UInt32 precision_, UInt32 scale_);
|
||||
};
|
||||
|
||||
}
|
93
src/DataTypes/Serializations/SerializationIP.cpp
Normal file
93
src/DataTypes/Serializations/SerializationIP.cpp
Normal file
@ -0,0 +1,93 @@
|
||||
#include <DataTypes/Serializations/SerializationIP.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Common/formatIPv6.h>
|
||||
#include <Functions/FunctionsCoding.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING;
|
||||
}
|
||||
|
||||
SerializationIPv4::SerializationIPv4(const SerializationPtr & nested_)
|
||||
: SerializationCustomSimpleText(nested_)
|
||||
{
|
||||
}
|
||||
|
||||
void SerializationIPv4::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
{
|
||||
const auto * col = checkAndGetColumn<ColumnUInt32>(&column);
|
||||
if (!col)
|
||||
{
|
||||
throw Exception("IPv4 type can only serialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'};
|
||||
char * ptr = buffer;
|
||||
formatIPv4(reinterpret_cast<const unsigned char *>(&col->getData()[row_num]), ptr);
|
||||
|
||||
ostr.write(buffer, strlen(buffer));
|
||||
}
|
||||
|
||||
void SerializationIPv4::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
ColumnUInt32 * col = typeid_cast<ColumnUInt32 *>(&column);
|
||||
if (!col)
|
||||
{
|
||||
throw Exception("IPv4 type can only deserialize columns of type UInt32." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
char buffer[IPV4_MAX_TEXT_LENGTH + 1] = {'\0'};
|
||||
istr.read(buffer, sizeof(buffer) - 1);
|
||||
UInt32 ipv4_value = 0;
|
||||
if (!parseIPv4(buffer, reinterpret_cast<unsigned char *>(&ipv4_value)))
|
||||
{
|
||||
throw Exception("Invalid IPv4 value.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
|
||||
}
|
||||
|
||||
col->insert(ipv4_value);
|
||||
}
|
||||
|
||||
SerializationIPv6::SerializationIPv6(const SerializationPtr & nested_)
|
||||
: SerializationCustomSimpleText(nested_)
|
||||
{
|
||||
}
|
||||
void SerializationIPv6::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
{
|
||||
const auto * col = checkAndGetColumn<ColumnFixedString>(&column);
|
||||
if (!col)
|
||||
{
|
||||
throw Exception("IPv6 type domain can only serialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
char buffer[IPV6_MAX_TEXT_LENGTH + 1] = {'\0'};
|
||||
char * ptr = buffer;
|
||||
formatIPv6(reinterpret_cast<const unsigned char *>(col->getDataAt(row_num).data), ptr);
|
||||
|
||||
ostr.write(buffer, strlen(buffer));
|
||||
}
|
||||
|
||||
void SerializationIPv6::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
ColumnFixedString * col = typeid_cast<ColumnFixedString *>(&column);
|
||||
if (!col)
|
||||
{
|
||||
throw Exception("IPv6 type domain can only deserialize columns of type FixedString(16)." + column.getName(), ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
|
||||
char buffer[IPV6_MAX_TEXT_LENGTH + 1] = {'\0'};
|
||||
istr.read(buffer, sizeof(buffer) - 1);
|
||||
|
||||
std::string ipv6_value(IPV6_BINARY_LENGTH, '\0');
|
||||
if (!parseIPv6(buffer, reinterpret_cast<unsigned char *>(ipv6_value.data())))
|
||||
{
|
||||
throw Exception("Invalid IPv6 value.", ErrorCodes::CANNOT_PARSE_DOMAIN_VALUE_FROM_STRING);
|
||||
}
|
||||
|
||||
col->insertString(ipv6_value);
|
||||
}
|
||||
|
||||
}
|
26
src/DataTypes/Serializations/SerializationIP.h
Normal file
26
src/DataTypes/Serializations/SerializationIP.h
Normal file
@ -0,0 +1,26 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/SerializationCustomSimpleText.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class SerializationIPv4 final : public SerializationCustomSimpleText
|
||||
{
|
||||
public:
|
||||
SerializationIPv4(const SerializationPtr & nested_);
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
};
|
||||
|
||||
class SerializationIPv6 : public SerializationCustomSimpleText
|
||||
{
|
||||
public:
|
||||
SerializationIPv6(const SerializationPtr & nested_);
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
};
|
||||
|
||||
}
|
122
src/DataTypes/Serializations/SerializationInfo.cpp
Normal file
122
src/DataTypes/Serializations/SerializationInfo.cpp
Normal file
@ -0,0 +1,122 @@
|
||||
#include <DataTypes/Serializations/SerializationInfo.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Poco/JSON/JSON.h>
|
||||
#include <Poco/JSON/Object.h>
|
||||
#include <Poco/JSON/Stringifier.h>
|
||||
#include <Poco/JSON/Parser.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
void SerializationInfo::add(const Block & block)
|
||||
{
|
||||
number_of_rows += block.rows();
|
||||
for (const auto & elem : block)
|
||||
{
|
||||
non_default_values[elem.name] = elem.column->getNumberOfNonDefaultValues();
|
||||
for (const auto & subname : elem.type->getSubcolumnNames())
|
||||
{
|
||||
auto subcolumn = elem.type->getSubcolumn(subname, *elem.column);
|
||||
auto full_name = Nested::concatenateName(elem.name, subname);
|
||||
non_default_values[full_name] += subcolumn->getNumberOfNonDefaultValues();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void SerializationInfo::add(const SerializationInfo & other)
|
||||
{
|
||||
number_of_rows += other.number_of_rows;
|
||||
for (const auto & [name, num] : other.non_default_values)
|
||||
non_default_values[name] += num;
|
||||
}
|
||||
|
||||
size_t SerializationInfo::getNumberOfNonDefaultValues(const String & column_name) const
|
||||
{
|
||||
auto it = non_default_values.find(column_name);
|
||||
if (it == non_default_values.end())
|
||||
return 0;
|
||||
return it->second;
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
constexpr auto KEY_NUMBER_OF_ROWS = "number_of_rows";
|
||||
constexpr auto KEY_NUMBER_OF_NON_DEFAULT_VALUES = "number_of_non_default_values";
|
||||
constexpr auto KEY_NUMBER = "number";
|
||||
constexpr auto KEY_NAME = "name";
|
||||
constexpr auto KEY_VERSION = "version";
|
||||
|
||||
}
|
||||
|
||||
void SerializationInfo::fromJSON(const String & json_str)
|
||||
{
|
||||
Poco::JSON::Parser parser;
|
||||
auto object = parser.parse(json_str).extract<Poco::JSON::Object::Ptr>();
|
||||
|
||||
if (object->has(KEY_NUMBER_OF_ROWS))
|
||||
number_of_rows = object->getValue<size_t>(KEY_NUMBER_OF_ROWS);
|
||||
|
||||
if (object->has(KEY_NUMBER_OF_NON_DEFAULT_VALUES))
|
||||
{
|
||||
auto array = object->getArray(KEY_NUMBER_OF_NON_DEFAULT_VALUES);
|
||||
for (const auto & elem : *array)
|
||||
{
|
||||
auto elem_object = elem.extract<Poco::JSON::Object::Ptr>();
|
||||
if (!elem_object->has(KEY_NUMBER) || !elem_object->has(KEY_NAME))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Missed field 'name' or 'number' in SerializationInfo of columns");
|
||||
|
||||
auto name = elem_object->getValue<String>(KEY_NAME);
|
||||
auto number = elem_object->getValue<size_t>(KEY_NUMBER);
|
||||
non_default_values[name] = number;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
String SerializationInfo::toJSON() const
|
||||
{
|
||||
Poco::JSON::Object info;
|
||||
info.set(KEY_VERSION, version);
|
||||
info.set(KEY_NUMBER_OF_ROWS, number_of_rows);
|
||||
|
||||
Poco::JSON::Array column_infos;
|
||||
for (const auto & [name, num] : non_default_values)
|
||||
{
|
||||
Poco::JSON::Object column_info;
|
||||
column_info.set(KEY_NAME, name);
|
||||
column_info.set(KEY_NUMBER, num);
|
||||
column_infos.add(std::move(column_info));
|
||||
}
|
||||
|
||||
info.set(KEY_NUMBER_OF_NON_DEFAULT_VALUES, std::move(column_infos));
|
||||
|
||||
std::ostringstream oss; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
|
||||
oss.exceptions(std::ios::failbit);
|
||||
Poco::JSON::Stringifier::stringify(info, oss);
|
||||
|
||||
return oss.str();
|
||||
}
|
||||
|
||||
void SerializationInfo::read(ReadBuffer & in)
|
||||
{
|
||||
String json_str;
|
||||
readString(json_str, in);
|
||||
fromJSON(json_str);
|
||||
}
|
||||
|
||||
void SerializationInfo::write(WriteBuffer & out) const
|
||||
{
|
||||
writeString(toJSON(), out);
|
||||
}
|
||||
|
||||
}
|
32
src/DataTypes/Serializations/SerializationInfo.h
Normal file
32
src/DataTypes/Serializations/SerializationInfo.h
Normal file
@ -0,0 +1,32 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Block.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class SerializationInfo
|
||||
{
|
||||
public:
|
||||
static constexpr auto version = 1;
|
||||
|
||||
using NameToNumber = std::unordered_map<String, size_t>;
|
||||
|
||||
void add(const Block & block);
|
||||
void add(const SerializationInfo & other);
|
||||
|
||||
size_t getNumberOfNonDefaultValues(const String & column_name) const;
|
||||
size_t getNumberOfRows() const { return number_of_rows; }
|
||||
|
||||
void read(ReadBuffer & in);
|
||||
void write(WriteBuffer & out) const;
|
||||
|
||||
private:
|
||||
void fromJSON(const String & json_str);
|
||||
String toJSON() const;
|
||||
|
||||
size_t number_of_rows = 0;
|
||||
NameToNumber non_default_values;
|
||||
};
|
||||
|
||||
}
|
215
src/DataTypes/Serializations/SerializationNumber.cpp
Normal file
215
src/DataTypes/Serializations/SerializationNumber.cpp
Normal file
@ -0,0 +1,215 @@
|
||||
#include <DataTypes/Serializations/SerializationNumber.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Formats/ProtobufReader.h>
|
||||
#include <Formats/ProtobufWriter.h>
|
||||
#include <Core/Field.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
void SerializationNumber<T>::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
{
|
||||
writeText(assert_cast<const ColumnVector<T> &>(column).getData()[row_num], ostr);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void SerializationNumber<T>::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
T x;
|
||||
|
||||
if constexpr (is_integer_v<T> && is_arithmetic_v<T>)
|
||||
readIntTextUnsafe(x, istr);
|
||||
else
|
||||
readText(x, istr);
|
||||
|
||||
assert_cast<ColumnVector<T> &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static inline void writeDenormalNumber(T x, WriteBuffer & ostr)
|
||||
{
|
||||
if constexpr (std::is_floating_point_v<T>)
|
||||
{
|
||||
if (std::signbit(x))
|
||||
{
|
||||
if (isNaN(x))
|
||||
writeCString("-nan", ostr);
|
||||
else
|
||||
writeCString("-inf", ostr);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (isNaN(x))
|
||||
writeCString("nan", ostr);
|
||||
else
|
||||
writeCString("inf", ostr);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// This function is not called for non floating point numbers.
|
||||
(void)x;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
template <typename T>
|
||||
void SerializationNumber<T>::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
auto x = assert_cast<const ColumnVector<T> &>(column).getData()[row_num];
|
||||
bool is_finite = isFinite(x);
|
||||
|
||||
const bool need_quote = (is_integer_v<T> && (sizeof(T) >= 8) && settings.json.quote_64bit_integers)
|
||||
|| (settings.json.quote_denormals && !is_finite);
|
||||
|
||||
if (need_quote)
|
||||
writeChar('"', ostr);
|
||||
|
||||
if (is_finite)
|
||||
writeText(x, ostr);
|
||||
else if (!settings.json.quote_denormals)
|
||||
writeCString("null", ostr);
|
||||
else
|
||||
writeDenormalNumber(x, ostr);
|
||||
|
||||
if (need_quote)
|
||||
writeChar('"', ostr);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void SerializationNumber<T>::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
bool has_quote = false;
|
||||
if (!istr.eof() && *istr.position() == '"') /// We understand the number both in quotes and without.
|
||||
{
|
||||
has_quote = true;
|
||||
++istr.position();
|
||||
}
|
||||
|
||||
FieldType x;
|
||||
|
||||
/// null
|
||||
if (!has_quote && !istr.eof() && *istr.position() == 'n')
|
||||
{
|
||||
++istr.position();
|
||||
assertString("ull", istr);
|
||||
|
||||
x = NaNOrZero<T>();
|
||||
}
|
||||
else
|
||||
{
|
||||
static constexpr bool is_uint8 = std::is_same_v<T, UInt8>;
|
||||
static constexpr bool is_int8 = std::is_same_v<T, Int8>;
|
||||
|
||||
if (is_uint8 || is_int8)
|
||||
{
|
||||
// extra conditions to parse true/false strings into 1/0
|
||||
if (istr.eof())
|
||||
throwReadAfterEOF();
|
||||
if (*istr.position() == 't' || *istr.position() == 'f')
|
||||
{
|
||||
bool tmp = false;
|
||||
readBoolTextWord(tmp, istr);
|
||||
x = tmp;
|
||||
}
|
||||
else
|
||||
readText(x, istr);
|
||||
}
|
||||
else
|
||||
{
|
||||
readText(x, istr);
|
||||
}
|
||||
|
||||
if (has_quote)
|
||||
assertChar('"', istr);
|
||||
}
|
||||
|
||||
assert_cast<ColumnVector<T> &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void SerializationNumber<T>::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const
|
||||
{
|
||||
FieldType x;
|
||||
readCSV(x, istr);
|
||||
assert_cast<ColumnVector<T> &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void SerializationNumber<T>::serializeBinary(const Field & field, WriteBuffer & ostr) const
|
||||
{
|
||||
/// ColumnVector<T>::ValueType is a narrower type. For example, UInt8, when the Field type is UInt64
|
||||
typename ColumnVector<T>::ValueType x = get<NearestFieldType<FieldType>>(field);
|
||||
writeBinary(x, ostr);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void SerializationNumber<T>::deserializeBinary(Field & field, ReadBuffer & istr) const
|
||||
{
|
||||
typename ColumnVector<T>::ValueType x;
|
||||
readBinary(x, istr);
|
||||
field = NearestFieldType<FieldType>(x);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void SerializationNumber<T>::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
|
||||
{
|
||||
writeBinary(assert_cast<const ColumnVector<T> &>(column).getData()[row_num], ostr);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void SerializationNumber<T>::deserializeBinary(IColumn & column, ReadBuffer & istr) const
|
||||
{
|
||||
typename ColumnVector<T>::ValueType x;
|
||||
readBinary(x, istr);
|
||||
assert_cast<ColumnVector<T> &>(column).getData().push_back(x);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void SerializationNumber<T>::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const
|
||||
{
|
||||
const typename ColumnVector<T>::Container & x = typeid_cast<const ColumnVector<T> &>(column).getData();
|
||||
|
||||
size_t size = x.size();
|
||||
|
||||
if (limit == 0 || offset + limit > size)
|
||||
limit = size - offset;
|
||||
|
||||
if (limit)
|
||||
ostr.write(reinterpret_cast<const char *>(&x[offset]), sizeof(typename ColumnVector<T>::ValueType) * limit);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
void SerializationNumber<T>::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double /*avg_value_size_hint*/) const
|
||||
{
|
||||
typename ColumnVector<T>::Container & x = typeid_cast<ColumnVector<T> &>(column).getData();
|
||||
size_t initial_size = x.size();
|
||||
x.resize(initial_size + limit);
|
||||
size_t size = istr.readBig(reinterpret_cast<char*>(&x[initial_size]), sizeof(typename ColumnVector<T>::ValueType) * limit);
|
||||
x.resize(initial_size + size / sizeof(typename ColumnVector<T>::ValueType));
|
||||
}
|
||||
|
||||
template class SerializationNumber<UInt8>;
|
||||
template class SerializationNumber<UInt16>;
|
||||
template class SerializationNumber<UInt32>;
|
||||
template class SerializationNumber<UInt64>;
|
||||
template class SerializationNumber<UInt128>; // base for UUID
|
||||
template class SerializationNumber<UInt256>;
|
||||
template class SerializationNumber<Int8>;
|
||||
template class SerializationNumber<Int16>;
|
||||
template class SerializationNumber<Int32>;
|
||||
template class SerializationNumber<Int64>;
|
||||
template class SerializationNumber<Int128>;
|
||||
template class SerializationNumber<Int256>;
|
||||
template class SerializationNumber<Float32>;
|
||||
template class SerializationNumber<Float64>;
|
||||
|
||||
}
|
33
src/DataTypes/Serializations/SerializationNumber.h
Normal file
33
src/DataTypes/Serializations/SerializationNumber.h
Normal file
@ -0,0 +1,33 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/SimpleTextSerialization.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename T>
|
||||
class SerializationNumber : public SimpleTextSerialization
|
||||
{
|
||||
static_assert(IsNumber<T>);
|
||||
|
||||
public:
|
||||
using FieldType = T;
|
||||
using ColumnType = ColumnVector<T>;
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
||||
/** Format is platform-dependent. */
|
||||
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
|
||||
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
|
||||
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
|
||||
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
|
||||
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override;
|
||||
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
|
||||
};
|
||||
|
||||
}
|
155
src/DataTypes/Serializations/SerializationSparse.cpp
Normal file
155
src/DataTypes/Serializations/SerializationSparse.cpp
Normal file
@ -0,0 +1,155 @@
|
||||
#include <DataTypes/Serializations/SerializationSparse.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
void serializeOffsetsPositionIndependent(const IColumn::Offsets & offsets, WriteBuffer & ostr)
|
||||
{
|
||||
size_t size = offsets.size();
|
||||
IColumn::Offset prev_offset = 0;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
IColumn::Offset current_offset = offsets[i];
|
||||
writeIntBinary(current_offset - prev_offset, ostr);
|
||||
prev_offset = current_offset;
|
||||
}
|
||||
}
|
||||
|
||||
void deserializeOffsetsPositionIndependent(IColumn::Offsets & offsets, ReadBuffer & istr)
|
||||
{
|
||||
IColumn::Offset current_offset = 0;
|
||||
while (!istr.eof())
|
||||
{
|
||||
IColumn::Offset current_size = 0;
|
||||
readIntBinary(current_size, istr);
|
||||
current_offset += current_size;
|
||||
offsets.push_back(current_offset);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
SerializationSparse::SerializationSparse(const SerializationPtr & nested_serialization_)
|
||||
: SerializationWrapper(nested_serialization_)
|
||||
{
|
||||
}
|
||||
|
||||
void SerializationSparse::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const
|
||||
{
|
||||
path.push_back(Substream::SparseOffsets);
|
||||
callback(path);
|
||||
path.back() = Substream::SparseElements;
|
||||
nested_serialization->enumerateStreams(callback, path);
|
||||
path.pop_back();
|
||||
}
|
||||
|
||||
void SerializationSparse::serializeBinaryBulkStatePrefix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
settings.path.push_back(Substream::SparseElements);
|
||||
nested_serialization->serializeBinaryBulkStatePrefix(settings, state);
|
||||
settings.path.pop_back();
|
||||
}
|
||||
|
||||
void SerializationSparse::serializeBinaryBulkStateSuffix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
settings.path.push_back(Substream::SparseElements);
|
||||
nested_serialization->serializeBinaryBulkStateSuffix(settings, state);
|
||||
settings.path.pop_back();
|
||||
}
|
||||
|
||||
void SerializationSparse::deserializeBinaryBulkStatePrefix(
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
settings.path.push_back(Substream::SparseElements);
|
||||
nested_serialization->deserializeBinaryBulkStatePrefix(settings, state);
|
||||
settings.path.pop_back();
|
||||
}
|
||||
|
||||
void SerializationSparse::serializeBinaryBulkWithMultipleStreams(
|
||||
const IColumn & column,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
UNUSED(limit);
|
||||
UNUSED(offset);
|
||||
|
||||
/// TODO: inefficient.
|
||||
/// TODO: use limit and offset
|
||||
size_t size = column.size();
|
||||
|
||||
auto offsets_column = DataTypeNumber<IColumn::Offset>().createColumn();
|
||||
auto & offsets_data = assert_cast<ColumnVector<IColumn::Offset> &>(*offsets_column).getData();
|
||||
|
||||
column.getIndicesOfNonDefaultValues(offsets_data);
|
||||
auto values = column.index(*offsets_column, 0);
|
||||
offsets_data.push_back(size);
|
||||
|
||||
settings.path.push_back(Substream::SparseOffsets);
|
||||
if (auto * stream = settings.getter(settings.path))
|
||||
serializeOffsetsPositionIndependent(offsets_data, *stream);
|
||||
|
||||
settings.path.back() = Substream::SparseElements;
|
||||
nested_serialization->serializeBinaryBulkWithMultipleStreams(*values, 0, 0, settings, state);
|
||||
|
||||
settings.path.pop_back();
|
||||
}
|
||||
|
||||
void SerializationSparse::deserializeBinaryBulkWithMultipleStreams(
|
||||
ColumnPtr & column,
|
||||
size_t limit,
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state,
|
||||
SubstreamsCache * cache) const
|
||||
{
|
||||
settings.path.push_back(Substream::SparseOffsets);
|
||||
|
||||
auto offsets_column = DataTypeNumber<IColumn::Offset>().createColumn();
|
||||
auto & offsets_data = assert_cast<ColumnVector<IColumn::Offset> &>(*offsets_column).getData();
|
||||
|
||||
if (auto * stream = settings.getter(settings.path))
|
||||
deserializeOffsetsPositionIndependent(offsets_data, *stream);
|
||||
|
||||
settings.path.back() = Substream::SparseElements;
|
||||
|
||||
ColumnPtr values = column->cloneEmpty();
|
||||
nested_serialization->deserializeBinaryBulkWithMultipleStreams(values, limit, settings, state, cache);
|
||||
|
||||
auto mutable_column = column->assumeMutable();
|
||||
size_t size = values->size();
|
||||
ssize_t prev_offset = -1;
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
size_t offsets_diff = static_cast<ssize_t>(offsets_data[i]) - prev_offset;
|
||||
|
||||
if (offsets_diff > 1)
|
||||
mutable_column->insertManyDefaults(offsets_diff - 1);
|
||||
|
||||
mutable_column->insertFrom(*values, i);
|
||||
prev_offset = offsets_data[i];
|
||||
}
|
||||
|
||||
size_t offsets_diff = offsets_data[size] - prev_offset;
|
||||
if (offsets_diff > 1)
|
||||
mutable_column->insertManyDefaults(offsets_diff - 1);
|
||||
|
||||
settings.path.pop_back();
|
||||
}
|
||||
|
||||
}
|
42
src/DataTypes/Serializations/SerializationSparse.h
Normal file
42
src/DataTypes/Serializations/SerializationSparse.h
Normal file
@ -0,0 +1,42 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/SerializationWrapper.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class SerializationSparse final : public SerializationWrapper
|
||||
{
|
||||
public:
|
||||
SerializationSparse(const SerializationPtr & nested_);
|
||||
|
||||
void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override;
|
||||
|
||||
void serializeBinaryBulkStatePrefix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const override;
|
||||
|
||||
void serializeBinaryBulkStateSuffix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const override;
|
||||
|
||||
void deserializeBinaryBulkStatePrefix(
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state) const override;
|
||||
|
||||
void serializeBinaryBulkWithMultipleStreams(
|
||||
const IColumn & column,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const override;
|
||||
|
||||
void deserializeBinaryBulkWithMultipleStreams(
|
||||
ColumnPtr & column,
|
||||
size_t limit,
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state,
|
||||
SubstreamsCache * cache) const override;
|
||||
};
|
||||
|
||||
}
|
73
src/DataTypes/Serializations/SerializationTupleElement.cpp
Normal file
73
src/DataTypes/Serializations/SerializationTupleElement.cpp
Normal file
@ -0,0 +1,73 @@
|
||||
#include <DataTypes/Serializations/SerializationTupleElement.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void SerializationTupleElement::enumerateStreams(
|
||||
const StreamCallback & callback,
|
||||
SubstreamPath & path) const
|
||||
{
|
||||
addToPath(path);
|
||||
nested_serialization->enumerateStreams(callback, path);
|
||||
path.pop_back();
|
||||
}
|
||||
|
||||
void SerializationTupleElement::serializeBinaryBulkStatePrefix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
addToPath(settings.path);
|
||||
nested_serialization->serializeBinaryBulkStatePrefix(settings, state);
|
||||
settings.path.pop_back();
|
||||
}
|
||||
|
||||
void SerializationTupleElement::serializeBinaryBulkStateSuffix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
addToPath(settings.path);
|
||||
nested_serialization->serializeBinaryBulkStateSuffix(settings, state);
|
||||
settings.path.pop_back();
|
||||
}
|
||||
|
||||
void SerializationTupleElement::deserializeBinaryBulkStatePrefix(
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
addToPath(settings.path);
|
||||
nested_serialization->deserializeBinaryBulkStatePrefix(settings, state);
|
||||
settings.path.pop_back();
|
||||
}
|
||||
|
||||
void SerializationTupleElement::serializeBinaryBulkWithMultipleStreams(
|
||||
const IColumn & column,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
addToPath(settings.path);
|
||||
nested_serialization->serializeBinaryBulkWithMultipleStreams(column, offset, limit, settings, state);
|
||||
settings.path.pop_back();
|
||||
}
|
||||
|
||||
void SerializationTupleElement::deserializeBinaryBulkWithMultipleStreams(
|
||||
ColumnPtr & column,
|
||||
size_t limit,
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state,
|
||||
SubstreamsCache * cache) const
|
||||
{
|
||||
addToPath(settings.path);
|
||||
nested_serialization->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, state, cache);
|
||||
settings.path.pop_back();
|
||||
}
|
||||
|
||||
void SerializationTupleElement::addToPath(SubstreamPath & path) const
|
||||
{
|
||||
path.push_back(Substream::TupleElement);
|
||||
path.back().tuple_element_name = name;
|
||||
path.back().escape_tuple_delimiter = escape_delimiter;
|
||||
}
|
||||
|
||||
}
|
57
src/DataTypes/Serializations/SerializationTupleElement.h
Normal file
57
src/DataTypes/Serializations/SerializationTupleElement.h
Normal file
@ -0,0 +1,57 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/SerializationWrapper.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class SerializationTupleElement final : public SerializationWrapper
|
||||
{
|
||||
private:
|
||||
String name;
|
||||
bool escape_delimiter;
|
||||
|
||||
public:
|
||||
SerializationTupleElement(const SerializationPtr & nested_, const String & name_, bool escape_delimiter_ = true)
|
||||
: SerializationWrapper(nested_)
|
||||
, name(name_), escape_delimiter(escape_delimiter_)
|
||||
{
|
||||
}
|
||||
|
||||
const String & getElementName() const { return name; }
|
||||
|
||||
void enumerateStreams(
|
||||
const StreamCallback & callback,
|
||||
SubstreamPath & path) const override;
|
||||
|
||||
void serializeBinaryBulkStatePrefix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const override;
|
||||
|
||||
void serializeBinaryBulkStateSuffix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const override;
|
||||
|
||||
void deserializeBinaryBulkStatePrefix(
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state) const override;
|
||||
|
||||
void serializeBinaryBulkWithMultipleStreams(
|
||||
const IColumn & column,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const override;
|
||||
|
||||
void deserializeBinaryBulkWithMultipleStreams(
|
||||
ColumnPtr & column,
|
||||
size_t limit,
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state,
|
||||
SubstreamsCache * cache) const override;
|
||||
|
||||
private:
|
||||
void addToPath(SubstreamPath & path) const;
|
||||
};
|
||||
|
||||
}
|
140
src/DataTypes/Serializations/SerializationWrapper.cpp
Normal file
140
src/DataTypes/Serializations/SerializationWrapper.cpp
Normal file
@ -0,0 +1,140 @@
|
||||
#include <DataTypes/Serializations/SerializationWrapper.h>
|
||||
#include <Columns/IColumn.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
void SerializationWrapper::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const
|
||||
{
|
||||
nested_serialization->enumerateStreams(callback, path);
|
||||
}
|
||||
|
||||
void SerializationWrapper::serializeBinaryBulkStatePrefix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
nested_serialization->serializeBinaryBulkStatePrefix(settings, state);
|
||||
}
|
||||
|
||||
void SerializationWrapper::serializeBinaryBulkStateSuffix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
nested_serialization->serializeBinaryBulkStateSuffix(settings, state);
|
||||
}
|
||||
|
||||
void SerializationWrapper::deserializeBinaryBulkStatePrefix(
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
nested_serialization->deserializeBinaryBulkStatePrefix(settings, state);
|
||||
}
|
||||
|
||||
void SerializationWrapper::serializeBinaryBulkWithMultipleStreams(
|
||||
const IColumn & column,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const
|
||||
{
|
||||
|
||||
nested_serialization->serializeBinaryBulkWithMultipleStreams(column, offset, limit, settings, state);
|
||||
}
|
||||
|
||||
void SerializationWrapper::deserializeBinaryBulkWithMultipleStreams(
|
||||
ColumnPtr & column,
|
||||
size_t limit,
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state,
|
||||
SubstreamsCache * cache) const
|
||||
{
|
||||
|
||||
nested_serialization->deserializeBinaryBulkWithMultipleStreams(column, limit, settings, state, cache);
|
||||
}
|
||||
|
||||
void SerializationWrapper::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const
|
||||
{
|
||||
nested_serialization->serializeBinaryBulk(column, ostr, offset, limit);
|
||||
}
|
||||
|
||||
void SerializationWrapper::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const
|
||||
{
|
||||
nested_serialization->deserializeBinaryBulk(column, istr, limit, avg_value_size_hint);
|
||||
}
|
||||
|
||||
void SerializationWrapper::serializeBinary(const Field & field, WriteBuffer & ostr) const
|
||||
{
|
||||
nested_serialization->serializeBinary(field, ostr);
|
||||
}
|
||||
|
||||
void SerializationWrapper::deserializeBinary(Field & field, ReadBuffer & istr) const
|
||||
{
|
||||
nested_serialization->deserializeBinary(field, istr);
|
||||
}
|
||||
|
||||
void SerializationWrapper::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
|
||||
{
|
||||
nested_serialization->serializeBinary(column, row_num, ostr);
|
||||
}
|
||||
|
||||
void SerializationWrapper::deserializeBinary(IColumn & column, ReadBuffer & istr) const
|
||||
{
|
||||
nested_serialization->deserializeBinary(column, istr);
|
||||
}
|
||||
|
||||
void SerializationWrapper::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
nested_serialization->serializeTextEscaped(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void SerializationWrapper::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
nested_serialization->deserializeTextEscaped(column, istr, settings);
|
||||
}
|
||||
|
||||
void SerializationWrapper::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
nested_serialization->serializeTextQuoted(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void SerializationWrapper::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
nested_serialization->deserializeTextQuoted(column, istr, settings);
|
||||
}
|
||||
|
||||
void SerializationWrapper::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
nested_serialization->serializeTextCSV(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void SerializationWrapper::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
nested_serialization->deserializeTextCSV(column, istr, settings);
|
||||
}
|
||||
|
||||
void SerializationWrapper::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
nested_serialization->serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void SerializationWrapper::deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
nested_serialization->deserializeWholeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void SerializationWrapper::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
nested_serialization->serializeTextJSON(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void SerializationWrapper::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const
|
||||
{
|
||||
nested_serialization->deserializeTextJSON(column, istr, settings);
|
||||
}
|
||||
|
||||
void SerializationWrapper::serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
nested_serialization->serializeTextXML(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
}
|
72
src/DataTypes/Serializations/SerializationWrapper.h
Normal file
72
src/DataTypes/Serializations/SerializationWrapper.h
Normal file
@ -0,0 +1,72 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/Serializations/ISerialization.h>
|
||||
#include <Common/Exception.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class SerializationWrapper : public ISerialization
|
||||
{
|
||||
protected:
|
||||
SerializationPtr nested_serialization;
|
||||
|
||||
public:
|
||||
SerializationWrapper(const SerializationPtr & nested_serialization_) : nested_serialization(nested_serialization_) {}
|
||||
|
||||
void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override;
|
||||
|
||||
void serializeBinaryBulkStatePrefix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const override;
|
||||
|
||||
void serializeBinaryBulkStateSuffix(
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const override;
|
||||
|
||||
void deserializeBinaryBulkStatePrefix(
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state) const override;
|
||||
|
||||
void serializeBinaryBulkWithMultipleStreams(
|
||||
const IColumn & column,
|
||||
size_t offset,
|
||||
size_t limit,
|
||||
SerializeBinaryBulkSettings & settings,
|
||||
SerializeBinaryBulkStatePtr & state) const override;
|
||||
|
||||
void deserializeBinaryBulkWithMultipleStreams(
|
||||
ColumnPtr & column,
|
||||
size_t limit,
|
||||
DeserializeBinaryBulkSettings & settings,
|
||||
DeserializeBinaryBulkStatePtr & state,
|
||||
SubstreamsCache * cache) const override;
|
||||
|
||||
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override;
|
||||
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
|
||||
|
||||
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
|
||||
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
|
||||
|
||||
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
|
||||
void deserializeBinary(IColumn & column, ReadBuffer & istr) const override;
|
||||
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override;
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override;
|
||||
|
||||
void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override;
|
||||
};
|
||||
|
||||
}
|
63
src/DataTypes/Serializations/SimpleTextSerialization.h
Normal file
63
src/DataTypes/Serializations/SimpleTextSerialization.h
Normal file
@ -0,0 +1,63 @@
|
||||
#pragma once
|
||||
#include <DataTypes/Serializations/ISerialization.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Helper class to define same ISerialization text (de)serialization for all the variants (escaped, quoted, JSON, CSV).
|
||||
/// You need to define serializeText() and deserializeText() in derived class.
|
||||
class SimpleTextSerialization : public ISerialization
|
||||
{
|
||||
protected:
|
||||
SimpleTextSerialization()
|
||||
{}
|
||||
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override
|
||||
{
|
||||
serializeText(column, row_num, ostr, settings);
|
||||
}
|
||||
|
||||
void deserializeWholeText(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override
|
||||
{
|
||||
deserializeText(column, istr, settings);
|
||||
}
|
||||
|
||||
virtual void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0;
|
||||
};
|
||||
|
||||
}
|
@ -67,7 +67,7 @@ TEST_P(ParseDataTypeTest, parseStringValue)
|
||||
for (const auto & value : p.values)
|
||||
{
|
||||
ReadBuffer buffer(const_cast<char *>(value.data()), value.size(), 0);
|
||||
data_type->deserializeAsWholeText(*col, buffer, FormatSettings{});
|
||||
data_type->getDefaultSerialization()->deserializeWholeText(*col, buffer, FormatSettings{});
|
||||
}
|
||||
|
||||
ASSERT_EQ(p.expected_values.size(), col->size()) << "Actual items: " << *col;
|
||||
|
@ -32,15 +32,37 @@ SRCS(
|
||||
DataTypeNothing.cpp
|
||||
DataTypeNullable.cpp
|
||||
DataTypeNumberBase.cpp
|
||||
DataTypeOneElementTuple.cpp
|
||||
DataTypeString.cpp
|
||||
DataTypeTuple.cpp
|
||||
DataTypeUUID.cpp
|
||||
DataTypesDecimal.cpp
|
||||
DataTypesNumber.cpp
|
||||
EnumValues.cpp
|
||||
FieldToDataType.cpp
|
||||
IDataType.cpp
|
||||
NestedUtils.cpp
|
||||
Serializations/ISerialization.cpp
|
||||
Serializations/SerializationAggregateFunction.cpp
|
||||
Serializations/SerializationArray.cpp
|
||||
Serializations/SerializationDate.cpp
|
||||
Serializations/SerializationDateTime.cpp
|
||||
Serializations/SerializationDateTime64.cpp
|
||||
Serializations/SerializationDecimal.cpp
|
||||
Serializations/SerializationDecimalBase.cpp
|
||||
Serializations/SerializationEnum.cpp
|
||||
Serializations/SerializationFixedString.cpp
|
||||
Serializations/SerializationInfo.cpp
|
||||
Serializations/SerializationLowCardinality.cpp
|
||||
Serializations/SerializationMap.cpp
|
||||
Serializations/SerializationNothing.cpp
|
||||
Serializations/SerializationNullable.cpp
|
||||
Serializations/SerializationNumber.cpp
|
||||
Serializations/SerializationSparse.cpp
|
||||
Serializations/SerializationString.cpp
|
||||
Serializations/SerializationTuple.cpp
|
||||
Serializations/SerializationTupleElement.cpp
|
||||
Serializations/SerializationUUID.cpp
|
||||
Serializations/SerializationWrapper.cpp
|
||||
convertMySQLDataType.cpp
|
||||
getLeastSupertype.cpp
|
||||
getMostSubtype.cpp
|
||||
|
Loading…
Reference in New Issue
Block a user