mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Revert "Keep alias type name to metadata"
This commit is contained in:
parent
310e0cc144
commit
220188caca
@ -338,7 +338,7 @@ bool DataTypeAggregateFunction::equals(const IDataType & rhs) const
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const String & /*type_name*/, const ASTPtr & arguments)
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
String function_name;
|
||||
AggregateFunctionPtr function;
|
||||
|
@ -509,7 +509,7 @@ size_t DataTypeArray::getNumberOfDimensions() const
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const String & /*type_name*/, const ASTPtr & arguments)
|
||||
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);
|
||||
|
@ -101,16 +101,16 @@ public:
|
||||
|
||||
void registerDataTypeDomainIPv4AndIPv6(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataTypeCustom("IPv4", [&](const String & /*type_name*/)
|
||||
factory.registerSimpleDataTypeCustom("IPv4", []
|
||||
{
|
||||
return std::make_pair(DataTypeFactory::instance().get("UInt32"),
|
||||
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypeCustomFixedName>("IPv4"), std::make_unique<DataTypeCustomIPv4Serialization>()));
|
||||
});
|
||||
|
||||
factory.registerSimpleDataTypeCustom("IPv6", [&](const String & /*type_name*/)
|
||||
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>()));
|
||||
std::make_unique<DataTypeCustomDesc>(std::make_unique<DataTypeCustomFixedName>("IPv6"), std::make_unique<DataTypeCustomIPv6Serialization>()));
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -58,7 +58,7 @@ String DataTypeCustomSimpleAggregateFunction::getName() const
|
||||
}
|
||||
|
||||
|
||||
static std::pair<DataTypePtr, DataTypeCustomDescPtr> create(const String & /*type_name*/, const ASTPtr & arguments)
|
||||
static std::pair<DataTypePtr, DataTypeCustomDescPtr> create(const ASTPtr & arguments)
|
||||
{
|
||||
String function_name;
|
||||
AggregateFunctionPtr function;
|
||||
|
@ -113,12 +113,7 @@ bool DataTypeDate::equals(const IDataType & rhs) const
|
||||
|
||||
void registerDataTypeDate(DataTypeFactory & factory)
|
||||
{
|
||||
const auto & creator = [&](const String & /*type_name*/)
|
||||
{
|
||||
return DataTypePtr(std::make_shared<DataTypeDate>());
|
||||
};
|
||||
|
||||
factory.registerSimpleDataType("Date", creator, DataTypeFactory::CaseInsensitive);
|
||||
factory.registerSimpleDataType("Date", [] { return DataTypePtr(std::make_shared<DataTypeDate>()); }, DataTypeFactory::CaseInsensitive);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -43,8 +43,8 @@ TimezoneMixin::TimezoneMixin(const String & time_zone_name)
|
||||
utc_time_zone(DateLUT::instance("UTC"))
|
||||
{}
|
||||
|
||||
DataTypeDateTime::DataTypeDateTime(const String & time_zone_name, const String & type_name_)
|
||||
: TimezoneMixin(time_zone_name), type_name(type_name_)
|
||||
DataTypeDateTime::DataTypeDateTime(const String & time_zone_name)
|
||||
: TimezoneMixin(time_zone_name)
|
||||
{
|
||||
}
|
||||
|
||||
@ -55,10 +55,10 @@ DataTypeDateTime::DataTypeDateTime(const TimezoneMixin & time_zone_)
|
||||
String DataTypeDateTime::doGetName() const
|
||||
{
|
||||
if (!has_explicit_time_zone)
|
||||
return type_name;
|
||||
return "DateTime";
|
||||
|
||||
WriteBufferFromOwnString out;
|
||||
out << type_name << "(" << quote << time_zone.getTimeZone() << ")";
|
||||
out << "DateTime(" << quote << time_zone.getTimeZone() << ")";
|
||||
return out.str();
|
||||
}
|
||||
|
||||
@ -194,10 +194,10 @@ namespace ErrorCodes
|
||||
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
||||
}
|
||||
|
||||
static DataTypePtr create(const String & type_name, const ASTPtr & arguments)
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments)
|
||||
return std::make_shared<DataTypeDateTime>("", type_name);
|
||||
return std::make_shared<DataTypeDateTime>();
|
||||
|
||||
if (arguments->children.size() != 1)
|
||||
throw Exception("DateTime data type can optionally have only one argument - time zone name", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
@ -206,7 +206,7 @@ static DataTypePtr create(const String & type_name, const ASTPtr & arguments)
|
||||
if (!arg || arg->value.getType() != Field::Types::String)
|
||||
throw Exception("Parameter for DateTime data type must be string literal", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeDateTime>(arg->value.get<String>(), type_name);
|
||||
return std::make_shared<DataTypeDateTime>(arg->value.get<String>());
|
||||
}
|
||||
|
||||
void registerDataTypeDateTime(DataTypeFactory & factory)
|
||||
|
@ -49,7 +49,7 @@ protected:
|
||||
class DataTypeDateTime final : public DataTypeNumberBase<UInt32>, public TimezoneMixin
|
||||
{
|
||||
public:
|
||||
explicit DataTypeDateTime(const String & time_zone_name = "", const String & type_name_ = "DateTime");
|
||||
explicit DataTypeDateTime(const String & time_zone_name = "");
|
||||
explicit DataTypeDateTime(const TimezoneMixin & time_zone);
|
||||
|
||||
static constexpr auto family_name = "DateTime";
|
||||
@ -75,8 +75,6 @@ public:
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
private:
|
||||
const String type_name;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -233,7 +233,7 @@ getArgument(const ASTPtr & arguments, size_t argument_index, const char * argume
|
||||
return argument->value.get<NearestResultType>();
|
||||
}
|
||||
|
||||
static DataTypePtr create64(const String & /*type_name*/, const ASTPtr & arguments)
|
||||
static DataTypePtr create64(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->size() == 0)
|
||||
return std::make_shared<DataTypeDateTime64>(DataTypeDateTime64::default_scale);
|
||||
|
@ -193,7 +193,7 @@ const DecimalType<U> decimalResultType(const DataTypeNumber<T> &, const DecimalT
|
||||
}
|
||||
|
||||
template <template <typename> typename DecimalType>
|
||||
DataTypePtr createDecimal(UInt64 precision_value, UInt64 scale_value, const String & type_name = "Decimal", bool only_scale = false)
|
||||
DataTypePtr createDecimal(UInt64 precision_value, UInt64 scale_value)
|
||||
{
|
||||
if (precision_value < DecimalUtils::minPrecision() || precision_value > DecimalUtils::maxPrecision<Decimal128>())
|
||||
throw Exception("Wrong precision", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
@ -202,10 +202,10 @@ DataTypePtr createDecimal(UInt64 precision_value, UInt64 scale_value, const Stri
|
||||
throw Exception("Negative scales and scales larger than precision are not supported", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
if (precision_value <= DecimalUtils::maxPrecision<Decimal32>())
|
||||
return std::make_shared<DecimalType<Decimal32>>(precision_value, scale_value, type_name, only_scale);
|
||||
return std::make_shared<DecimalType<Decimal32>>(precision_value, scale_value);
|
||||
else if (precision_value <= DecimalUtils::maxPrecision<Decimal64>())
|
||||
return std::make_shared<DecimalType<Decimal64>>(precision_value, scale_value, type_name, only_scale);
|
||||
return std::make_shared<DecimalType<Decimal128>>(precision_value, scale_value, type_name, only_scale);
|
||||
return std::make_shared<DecimalType<Decimal64>>(precision_value, scale_value);
|
||||
return std::make_shared<DecimalType<Decimal128>>(precision_value, scale_value);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -365,7 +365,7 @@ static void checkASTStructure(const ASTPtr & child)
|
||||
}
|
||||
|
||||
template <typename DataTypeEnum>
|
||||
static DataTypePtr createExact(const String & /*type_name*/, const ASTPtr & arguments)
|
||||
static DataTypePtr createExact(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->children.empty())
|
||||
throw Exception("Enum data type cannot be empty", ErrorCodes::EMPTY_DATA_PASSED);
|
||||
@ -404,7 +404,7 @@ static DataTypePtr createExact(const String & /*type_name*/, const ASTPtr & argu
|
||||
return std::make_shared<DataTypeEnum>(values);
|
||||
}
|
||||
|
||||
static DataTypePtr create(const String & type_name, const ASTPtr & arguments)
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->children.empty())
|
||||
throw Exception("Enum data type cannot be empty", ErrorCodes::EMPTY_DATA_PASSED);
|
||||
@ -425,10 +425,10 @@ static DataTypePtr create(const String & type_name, const ASTPtr & arguments)
|
||||
Int64 value = value_literal->value.get<Int64>();
|
||||
|
||||
if (value > std::numeric_limits<Int8>::max() || value < std::numeric_limits<Int8>::min())
|
||||
return createExact<DataTypeEnum16>(type_name, arguments);
|
||||
return createExact<DataTypeEnum16>(arguments);
|
||||
}
|
||||
|
||||
return createExact<DataTypeEnum8>(type_name, arguments);
|
||||
return createExact<DataTypeEnum8>(arguments);
|
||||
}
|
||||
|
||||
void registerDataTypeEnum(DataTypeFactory & factory)
|
||||
|
@ -74,7 +74,7 @@ DataTypePtr DataTypeFactory::get(const String & family_name_param, const ASTPtr
|
||||
return get("LowCardinality", low_cardinality_params);
|
||||
}
|
||||
|
||||
return findCreatorByName(family_name)(family_name_param, parameters);
|
||||
return findCreatorByName(family_name)(parameters);
|
||||
}
|
||||
|
||||
|
||||
@ -107,30 +107,30 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator
|
||||
throw Exception("DataTypeFactory: the data type " + name + " has been provided "
|
||||
" a null constructor", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
registerDataType(name, [name, creator](const String & type_name, const ASTPtr & ast)
|
||||
registerDataType(name, [name, creator](const ASTPtr & ast)
|
||||
{
|
||||
if (ast)
|
||||
throw Exception("Data type " + name + " cannot have arguments", ErrorCodes::DATA_TYPE_CANNOT_HAVE_ARGUMENTS);
|
||||
return creator(type_name);
|
||||
return creator();
|
||||
}, case_sensitiveness);
|
||||
}
|
||||
|
||||
void DataTypeFactory::registerDataTypeCustom(const String & family_name, CreatorWithCustom creator, CaseSensitiveness case_sensitiveness)
|
||||
{
|
||||
registerDataType(family_name, [creator](const String & type_name, const ASTPtr & ast)
|
||||
registerDataType(family_name, [creator](const ASTPtr & ast)
|
||||
{
|
||||
auto res = creator(type_name, ast);
|
||||
auto res = creator(ast);
|
||||
res.first->setCustomization(std::move(res.second));
|
||||
|
||||
return res.first;
|
||||
}, case_sensitiveness);
|
||||
}
|
||||
|
||||
void DataTypeFactory::registerSimpleDataTypeCustom(const String & name, SimpleCreatorWithCustom creator, CaseSensitiveness case_sensitiveness)
|
||||
void DataTypeFactory::registerSimpleDataTypeCustom(const String &name, SimpleCreatorWithCustom creator, CaseSensitiveness case_sensitiveness)
|
||||
{
|
||||
registerDataTypeCustom(name, [creator](const String & type_name, const ASTPtr & /*ast*/)
|
||||
registerDataTypeCustom(name, [creator](const ASTPtr & /*ast*/)
|
||||
{
|
||||
return creator(type_name);
|
||||
return creator();
|
||||
}, case_sensitiveness);
|
||||
}
|
||||
|
||||
|
@ -16,15 +16,16 @@ namespace DB
|
||||
class IDataType;
|
||||
using DataTypePtr = std::shared_ptr<const IDataType>;
|
||||
|
||||
|
||||
/** Creates a data type by name of data type family and parameters.
|
||||
*/
|
||||
class DataTypeFactory final : private boost::noncopyable, public IFactoryWithAliases<std::function<DataTypePtr(const String & type_name, const ASTPtr & parameters)>>
|
||||
class DataTypeFactory final : private boost::noncopyable, public IFactoryWithAliases<std::function<DataTypePtr(const ASTPtr & parameters)>>
|
||||
{
|
||||
private:
|
||||
using SimpleCreator = std::function<DataTypePtr(const String & type_name)>;
|
||||
using SimpleCreator = std::function<DataTypePtr()>;
|
||||
using DataTypesDictionary = std::unordered_map<String, Creator>;
|
||||
using CreatorWithCustom = std::function<std::pair<DataTypePtr, DataTypeCustomDescPtr>(const String & type_name, const ASTPtr & parameters)>;
|
||||
using SimpleCreatorWithCustom = std::function<std::pair<DataTypePtr, DataTypeCustomDescPtr>(const String & type_name)>;
|
||||
using CreatorWithCustom = std::function<std::pair<DataTypePtr,DataTypeCustomDescPtr>(const ASTPtr & parameters)>;
|
||||
using SimpleCreatorWithCustom = std::function<std::pair<DataTypePtr,DataTypeCustomDescPtr>()>;
|
||||
|
||||
public:
|
||||
static DataTypeFactory & instance();
|
||||
|
@ -34,7 +34,7 @@ namespace ErrorCodes
|
||||
|
||||
std::string DataTypeFixedString::doGetName() const
|
||||
{
|
||||
return type_name + "(" + toString(n) + ")";
|
||||
return "FixedString(" + toString(n) + ")";
|
||||
}
|
||||
|
||||
|
||||
@ -279,7 +279,7 @@ bool DataTypeFixedString::equals(const IDataType & rhs) const
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const String & type_name, const ASTPtr & arguments)
|
||||
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);
|
||||
@ -288,7 +288,7 @@ static DataTypePtr create(const String & type_name, const ASTPtr & arguments)
|
||||
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>(), type_name);
|
||||
return std::make_shared<DataTypeFixedString>(argument->value.get<UInt64>());
|
||||
}
|
||||
|
||||
|
||||
|
@ -22,7 +22,7 @@ private:
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
|
||||
DataTypeFixedString(size_t n_, const String & type_name_ = "FixedString") : n(n_), type_name(type_name_)
|
||||
DataTypeFixedString(size_t n_) : n(n_)
|
||||
{
|
||||
if (n == 0)
|
||||
throw Exception("FixedString size must be positive", ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
@ -85,9 +85,6 @@ public:
|
||||
bool isCategorial() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
bool canBeInsideLowCardinality() const override { return true; }
|
||||
|
||||
private:
|
||||
const String type_name;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -10,22 +10,17 @@ bool DataTypeInterval::equals(const IDataType & rhs) const
|
||||
return typeid(rhs) == typeid(*this) && kind == static_cast<const DataTypeInterval &>(rhs).kind;
|
||||
}
|
||||
|
||||
template <IntervalKind::Kind kind>
|
||||
static DataTypePtr create(const String & /*type_name*/)
|
||||
{
|
||||
return DataTypePtr(std::make_shared<DataTypeInterval>(kind));
|
||||
}
|
||||
|
||||
void registerDataTypeInterval(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataType("IntervalSecond", create<IntervalKind::Second>);
|
||||
factory.registerSimpleDataType("IntervalMinute", create<IntervalKind::Minute>);
|
||||
factory.registerSimpleDataType("IntervalHour", create<IntervalKind::Hour>);
|
||||
factory.registerSimpleDataType("IntervalDay", create<IntervalKind::Day>);
|
||||
factory.registerSimpleDataType("IntervalWeek", create<IntervalKind::Week>);
|
||||
factory.registerSimpleDataType("IntervalMonth", create<IntervalKind::Month>);
|
||||
factory.registerSimpleDataType("IntervalQuarter", create<IntervalKind::Quarter>);
|
||||
factory.registerSimpleDataType("IntervalYear", create<IntervalKind::Year>);
|
||||
factory.registerSimpleDataType("IntervalSecond", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(IntervalKind::Second)); });
|
||||
factory.registerSimpleDataType("IntervalMinute", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(IntervalKind::Minute)); });
|
||||
factory.registerSimpleDataType("IntervalHour", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(IntervalKind::Hour)); });
|
||||
factory.registerSimpleDataType("IntervalDay", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(IntervalKind::Day)); });
|
||||
factory.registerSimpleDataType("IntervalWeek", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(IntervalKind::Week)); });
|
||||
factory.registerSimpleDataType("IntervalMonth", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(IntervalKind::Month)); });
|
||||
factory.registerSimpleDataType("IntervalQuarter", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(IntervalKind::Quarter)); });
|
||||
factory.registerSimpleDataType("IntervalYear", [] { return DataTypePtr(std::make_shared<DataTypeInterval>(IntervalKind::Year)); });
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -949,7 +949,7 @@ bool DataTypeLowCardinality::equals(const IDataType & rhs) const
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const String & /*type_name*/, const ASTPtr & arguments)
|
||||
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",
|
||||
|
@ -38,9 +38,7 @@ bool DataTypeNothing::equals(const IDataType & rhs) const
|
||||
|
||||
void registerDataTypeNothing(DataTypeFactory & factory)
|
||||
{
|
||||
const auto & creator = [&](const String & /*type_name*/) { return DataTypePtr(std::make_shared<DataTypeNothing>()); };
|
||||
|
||||
factory.registerSimpleDataType("Nothing", creator);
|
||||
factory.registerSimpleDataType("Nothing", [] { return DataTypePtr(std::make_shared<DataTypeNothing>()); });
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -507,7 +507,7 @@ bool DataTypeNullable::equals(const IDataType & rhs) const
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const String & /*type_name*/, const ASTPtr & arguments)
|
||||
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);
|
||||
|
@ -369,7 +369,7 @@ bool DataTypeString::equals(const IDataType & rhs) const
|
||||
|
||||
void registerDataTypeString(DataTypeFactory & factory)
|
||||
{
|
||||
const auto & creator = [&] (const String & type_name) { return std::make_shared<DataTypeString>(type_name); };
|
||||
auto creator = static_cast<DataTypePtr(*)()>([] { return DataTypePtr(std::make_shared<DataTypeString>()); });
|
||||
|
||||
factory.registerSimpleDataType("String", creator);
|
||||
|
||||
|
@ -14,10 +14,6 @@ public:
|
||||
using FieldType = String;
|
||||
static constexpr bool is_parametric = false;
|
||||
|
||||
DataTypeString(const String & type_name_ = "String") : type_name(type_name_) {}
|
||||
|
||||
String doGetName() const override { return type_name; }
|
||||
|
||||
const char * getFamilyName() const override
|
||||
{
|
||||
return "String";
|
||||
@ -67,9 +63,6 @@ public:
|
||||
bool isCategorial() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
bool canBeInsideLowCardinality() const override { return true; }
|
||||
|
||||
private:
|
||||
const String type_name;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -529,7 +529,7 @@ size_t DataTypeTuple::getSizeOfValueInMemory() const
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const String & /*type_name*/, const ASTPtr & arguments)
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->children.empty())
|
||||
throw Exception("Tuple cannot be empty", ErrorCodes::EMPTY_DATA_PASSED);
|
||||
@ -568,7 +568,7 @@ void registerDataTypeTuple(DataTypeFactory & factory)
|
||||
void registerDataTypeNested(DataTypeFactory & factory)
|
||||
{
|
||||
/// Nested(...) data type is just a sugar for Array(Tuple(...))
|
||||
factory.registerDataType("Nested", [&factory](const String & /*type_name*/, const ASTPtr & arguments)
|
||||
factory.registerDataType("Nested", [&factory](const ASTPtr & arguments)
|
||||
{
|
||||
return std::make_shared<DataTypeArray>(factory.get("Tuple", arguments));
|
||||
});
|
||||
|
@ -106,9 +106,7 @@ bool DataTypeUUID::equals(const IDataType & rhs) const
|
||||
|
||||
void registerDataTypeUUID(DataTypeFactory & factory)
|
||||
{
|
||||
const auto & creator = [&] (const String & /*type_name*/) { return std::make_shared<DataTypeUUID>(); };
|
||||
|
||||
factory.registerSimpleDataType("UUID", creator);
|
||||
factory.registerSimpleDataType("UUID", [] { return DataTypePtr(std::make_shared<DataTypeUUID>()); });
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -14,8 +14,6 @@
|
||||
#include <Parsers/IAST.h>
|
||||
|
||||
#include <type_traits>
|
||||
#include "DataTypesDecimal.h"
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -32,12 +30,7 @@ template <typename T>
|
||||
std::string DataTypeDecimal<T>::doGetName() const
|
||||
{
|
||||
std::stringstream ss;
|
||||
ss << type_name << "(";
|
||||
|
||||
if (!only_scale)
|
||||
ss << this->precision << ", ";
|
||||
|
||||
ss << this->scale << ")";
|
||||
ss << "Decimal(" << this->precision << ", " << this->scale << ")";
|
||||
return ss.str();
|
||||
}
|
||||
|
||||
@ -140,14 +133,8 @@ void DataTypeDecimal<T>::deserializeProtobuf(IColumn & column, ProtobufReader &
|
||||
container.back() = decimal;
|
||||
}
|
||||
|
||||
template<typename T>
|
||||
DataTypeDecimal<T>::DataTypeDecimal(UInt32 precision_, UInt32 scale_, const String & type_name_, bool only_scale_)
|
||||
: Base(precision_, scale_), type_name(type_name_), only_scale(only_scale_)
|
||||
{
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const String & type_name, const ASTPtr & arguments)
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->children.size() != 2)
|
||||
throw Exception("Decimal data type family must have exactly two arguments: precision and scale",
|
||||
@ -163,11 +150,11 @@ static DataTypePtr create(const String & type_name, const ASTPtr & arguments)
|
||||
UInt64 precision_value = precision->value.get<UInt64>();
|
||||
UInt64 scale_value = scale->value.get<UInt64>();
|
||||
|
||||
return createDecimal<DataTypeDecimal>(precision_value, scale_value, type_name);
|
||||
return createDecimal<DataTypeDecimal>(precision_value, scale_value);
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
static DataTypePtr createExact(const String & type_name, const ASTPtr & arguments)
|
||||
static DataTypePtr createExact(const ASTPtr & arguments)
|
||||
{
|
||||
if (!arguments || arguments->children.size() != 1)
|
||||
throw Exception("Decimal data type family must have exactly two arguments: precision and scale",
|
||||
@ -181,7 +168,7 @@ static DataTypePtr createExact(const String & type_name, const ASTPtr & argument
|
||||
UInt64 precision = DecimalUtils::maxPrecision<T>();
|
||||
UInt64 scale = scale_arg->value.get<UInt64>();
|
||||
|
||||
return createDecimal<DataTypeDecimal>(precision, scale, type_name, true);
|
||||
return createDecimal<DataTypeDecimal>(precision, scale);
|
||||
}
|
||||
|
||||
void registerDataTypeDecimal(DataTypeFactory & factory)
|
||||
|
@ -33,8 +33,6 @@ public:
|
||||
using typename Base::ColumnType;
|
||||
using Base::Base;
|
||||
|
||||
DataTypeDecimal(UInt32 precision_, UInt32 scale_, const String & type_name_ = "Decimal", bool only_scale_ = false);
|
||||
|
||||
static constexpr auto family_name = "Decimal";
|
||||
|
||||
const char * getFamilyName() const override { return family_name; }
|
||||
@ -57,12 +55,6 @@ public:
|
||||
|
||||
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_);
|
||||
|
||||
private:
|
||||
/// The name of data type how the user specified it. A single data type may be referenced by various synonims.
|
||||
const String type_name;
|
||||
/// If the user specified it only with scale parameter but without precision.
|
||||
bool only_scale = false;
|
||||
};
|
||||
|
||||
template <typename T>
|
||||
|
@ -5,26 +5,20 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
template <typename NumberType>
|
||||
static DataTypePtr create(const String & type_name)
|
||||
{
|
||||
return DataTypePtr(std::make_shared<NumberType>(type_name));
|
||||
}
|
||||
|
||||
void registerDataTypeNumbers(DataTypeFactory & factory)
|
||||
{
|
||||
factory.registerSimpleDataType("UInt8", create<DataTypeUInt8>);
|
||||
factory.registerSimpleDataType("UInt16", create<DataTypeUInt16>);
|
||||
factory.registerSimpleDataType("UInt32", create<DataTypeUInt32>);
|
||||
factory.registerSimpleDataType("UInt64", create<DataTypeUInt64>);
|
||||
factory.registerSimpleDataType("UInt8", [] { return DataTypePtr(std::make_shared<DataTypeUInt8>()); });
|
||||
factory.registerSimpleDataType("UInt16", [] { return DataTypePtr(std::make_shared<DataTypeUInt16>()); });
|
||||
factory.registerSimpleDataType("UInt32", [] { return DataTypePtr(std::make_shared<DataTypeUInt32>()); });
|
||||
factory.registerSimpleDataType("UInt64", [] { return DataTypePtr(std::make_shared<DataTypeUInt64>()); });
|
||||
|
||||
factory.registerSimpleDataType("Int8", create<DataTypeInt8>);
|
||||
factory.registerSimpleDataType("Int16", create<DataTypeInt16>);
|
||||
factory.registerSimpleDataType("Int32", create<DataTypeInt32>);
|
||||
factory.registerSimpleDataType("Int64", create<DataTypeInt64>);
|
||||
factory.registerSimpleDataType("Int8", [] { return DataTypePtr(std::make_shared<DataTypeInt8>()); });
|
||||
factory.registerSimpleDataType("Int16", [] { return DataTypePtr(std::make_shared<DataTypeInt16>()); });
|
||||
factory.registerSimpleDataType("Int32", [] { return DataTypePtr(std::make_shared<DataTypeInt32>()); });
|
||||
factory.registerSimpleDataType("Int64", [] { return DataTypePtr(std::make_shared<DataTypeInt64>()); });
|
||||
|
||||
factory.registerSimpleDataType("Float32", create<DataTypeFloat32>);
|
||||
factory.registerSimpleDataType("Float64", create<DataTypeFloat64>);
|
||||
factory.registerSimpleDataType("Float32", [] { return DataTypePtr(std::make_shared<DataTypeFloat32>()); });
|
||||
factory.registerSimpleDataType("Float64", [] { return DataTypePtr(std::make_shared<DataTypeFloat64>()); });
|
||||
|
||||
/// These synonyms are added for compatibility.
|
||||
|
||||
|
@ -25,13 +25,6 @@ class DataTypeNumber final : public DataTypeNumberBase<T>
|
||||
using PromotedType = DataTypeNumber<NearestFieldType<T>>;
|
||||
return std::make_shared<PromotedType>();
|
||||
}
|
||||
|
||||
public:
|
||||
DataTypeNumber(const String & type_name_ = TypeName<T>::get()) : type_name(type_name_) {}
|
||||
|
||||
String doGetName() const override { return type_name; }
|
||||
private:
|
||||
const String type_name;
|
||||
};
|
||||
|
||||
using DataTypeUInt8 = DataTypeNumber<UInt8>;
|
||||
|
@ -15,8 +15,8 @@
|
||||
1.0000 1.0000 0.33333333 0.33333333 0.20000000 0.20000000
|
||||
50.0000 50.0000 16.66666666 16.66666666 10.00000000 10.00000000
|
||||
-1.0000 -1.0000 -0.33333333 -0.33333333 -0.20000000 -0.20000000
|
||||
0.0000 0.00000000 0.00000000 Decimal128(8)
|
||||
-25.5000 -8.49999999 -5.10000000 Decimal128(8)
|
||||
0.0000 0.00000000 0.00000000 Decimal(38, 8)
|
||||
-25.5000 -8.49999999 -5.10000000 Decimal(38, 8)
|
||||
0.0000 0.00000000 0.00000000
|
||||
10.0000 3.33333333 2.00000000
|
||||
20.0000 6.66666666 4.00000000
|
||||
@ -26,8 +26,8 @@
|
||||
[-50.0000,-40.0000,-30.0000,-20.0000,-10.0000,0.0000,10.0000,20.0000,30.0000,40.0000,50.0000]
|
||||
[-16.66666666,-13.33333333,-10.00000000,-6.66666666,-3.33333333,0.00000000,3.33333333,6.66666666,10.00000000,13.33333333,16.66666666]
|
||||
[-10.00000000,-8.00000000,-6.00000000,-4.00000000,-2.00000000,0.00000000,2.00000000,4.00000000,6.00000000,8.00000000,10.00000000]
|
||||
0.0000 0.00000000 0.00000000 Decimal128(8)
|
||||
-25.0000 -8.33333333 -5.00000000 Decimal128(8)
|
||||
0.0000 0.00000000 0.00000000 Decimal(38, 8)
|
||||
-25.0000 -8.33333333 -5.00000000 Decimal(38, 8)
|
||||
0.0000 0.00000000 0.00000000
|
||||
10.0000 3.33333333 2.00000000
|
||||
20.0000 6.66666666 4.00000000
|
||||
@ -37,8 +37,8 @@
|
||||
[-50.0000,-40.0000,-30.0000,-20.0000,-10.0000,0.0000,10.0000,20.0000,30.0000,40.0000,50.0000]
|
||||
[-16.66666666,-13.33333333,-10.00000000,-6.66666666,-3.33333333,0.00000000,3.33333333,6.66666666,10.00000000,13.33333333,16.66666666]
|
||||
[-10.00000000,-8.00000000,-6.00000000,-4.00000000,-2.00000000,0.00000000,2.00000000,4.00000000,6.00000000,8.00000000,10.00000000]
|
||||
0.0000 0.00000000 0.00000000 Decimal128(8)
|
||||
-26.0000 -8.66666666 -5.20000000 Decimal128(8)
|
||||
0.0000 0.00000000 0.00000000 Decimal(38, 8)
|
||||
-26.0000 -8.66666666 -5.20000000 Decimal(38, 8)
|
||||
0.0000 0.00000000 0.00000000
|
||||
10.0000 3.33333333 2.00000000
|
||||
20.0000 6.66666666 4.00000000
|
||||
|
@ -1,8 +1,8 @@
|
||||
Array(Decimal32(3)) Array(Decimal64(3)) Array(Decimal128(3))
|
||||
Array(Decimal(9, 3)) Array(Decimal(18, 3)) Array(Decimal(38, 3))
|
||||
Array(Decimal(9, 2)) Array(Decimal(18, 2)) Array(Decimal(38, 2))
|
||||
Decimal32(3) Decimal64(3) Decimal128(3)
|
||||
Decimal(9, 3) Decimal(18, 3) Decimal(38, 3)
|
||||
Decimal(9, 2) Decimal(18, 2) Decimal(38, 2)
|
||||
Tuple(Decimal32(1), Decimal64(1), Decimal128(1)) Decimal32(1) Decimal64(1) Decimal128(1)
|
||||
Tuple(Decimal(9, 1), Decimal(18, 1), Decimal(38, 1)) Decimal(9, 1) Decimal(18, 1) Decimal(38, 1)
|
||||
0.100
|
||||
0.200
|
||||
0.300
|
||||
|
@ -1,6 +1,6 @@
|
||||
a DECIMAL(9, 4) DEFAULT CAST(0, \'DECIMAL(9, 4)\')
|
||||
b DECIMAL(18, 4) DEFAULT CAST(a / 2, \'DECIMAL(18, 4)\')
|
||||
c DECIMAL(38, 4) DEFAULT CAST(b / 3, \'DECIMAL(38, 4)\')
|
||||
a Decimal(9, 4) DEFAULT CAST(0, \'Decimal(9, 4)\')
|
||||
b Decimal(18, 4) DEFAULT CAST(a / 2, \'Decimal(18, 4)\')
|
||||
c Decimal(38, 4) DEFAULT CAST(b / 3, \'Decimal(38, 4)\')
|
||||
d Decimal(9, 4) MATERIALIZED a + toDecimal32(\'0.2\', 1)
|
||||
e Decimal(18, 4) ALIAS b * 2
|
||||
f Decimal(38, 4) ALIAS c * 6
|
||||
|
@ -9,7 +9,7 @@
|
||||
10003
|
||||
274972506.6
|
||||
9175437371954010821
|
||||
CREATE TABLE default.compression_codec_multiple_more_types (`id` Decimal128(13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)), `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)), `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)), `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
CREATE TABLE default.compression_codec_multiple_more_types (`id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)), `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)), `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)), `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))) ENGINE = MergeTree() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
1.5555555555555 hello world! [77] ['John']
|
||||
7.1000000000000 xxxxxxxxxxxx [127] ['Henry']
|
||||
!
|
||||
|
@ -20,7 +20,7 @@
|
||||
274972506.6
|
||||
9175437371954010821
|
||||
9175437371954010821
|
||||
CREATE TABLE test.compression_codec_multiple_more_types_replicated (`id` Decimal128(13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)), `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)), `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)), `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/compression_codec_multiple_more_types_replicated\', \'1\') ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
CREATE TABLE test.compression_codec_multiple_more_types_replicated (`id` Decimal(38, 13) CODEC(ZSTD(1), LZ4, ZSTD(1), ZSTD(1), Delta(2), Delta(4), Delta(1), LZ4HC(0)), `data` FixedString(12) CODEC(ZSTD(1), ZSTD(1), Delta(1), Delta(1), Delta(1), NONE, NONE, NONE, LZ4HC(0)), `ddd.age` Array(UInt8) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8)), `ddd.Name` Array(String) CODEC(LZ4, LZ4HC(0), NONE, NONE, NONE, ZSTD(1), Delta(8))) ENGINE = ReplicatedMergeTree(\'/clickhouse/tables/test/compression_codec_multiple_more_types_replicated\', \'1\') ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
1.5555555555555 hello world! [77] ['John']
|
||||
7.1000000000000 xxxxxxxxxxxx [127] ['Henry']
|
||||
!
|
||||
|
@ -18,7 +18,7 @@
|
||||
7 7
|
||||
8 8
|
||||
9 9
|
||||
SimpleAggregateFunction(sum, Double)
|
||||
SimpleAggregateFunction(sum, Float64)
|
||||
0 0
|
||||
1 2
|
||||
2 4
|
||||
|
@ -1,5 +1,5 @@
|
||||
CREATE TABLE default.ttl (`d` Date, `a` Int) ENGINE = MergeTree PARTITION BY toDayOfMonth(d) ORDER BY a TTL d + toIntervalDay(1) SETTINGS index_granularity = 8192
|
||||
CREATE TABLE default.ttl (`d` Date, `a` Int32) ENGINE = MergeTree PARTITION BY toDayOfMonth(d) ORDER BY a TTL d + toIntervalDay(1) SETTINGS index_granularity = 8192
|
||||
2100-10-10 3
|
||||
2100-10-10 4
|
||||
d Date
|
||||
a Int d + toIntervalDay(1)
|
||||
a Int32 d + toIntervalDay(1)
|
||||
|
@ -6,11 +6,11 @@
|
||||
2000-10-10 00:00:00 0
|
||||
2100-10-10 00:00:00 3
|
||||
2100-10-10 2
|
||||
CREATE TABLE default.ttl_00933_1 (`b` Int, `a` Int TTL now() - 1000) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL now() - 1000) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
1 0
|
||||
CREATE TABLE default.ttl_00933_1 (`b` Int, `a` Int TTL now() + 1000) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL now() + 1000) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
1 1
|
||||
CREATE TABLE default.ttl_00933_1 (`b` Int, `a` Int TTL today() - 1) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL today() - 1) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
1 0
|
||||
CREATE TABLE default.ttl_00933_1 (`b` Int, `a` Int TTL today() + 1) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
CREATE TABLE default.ttl_00933_1 (`b` Int32, `a` Int32 TTL today() + 1) ENGINE = MergeTree PARTITION BY tuple() ORDER BY tuple() SETTINGS index_granularity = 8192
|
||||
1 1
|
||||
|
@ -1,38 +0,0 @@
|
||||
binary BINARY(1)
|
||||
dec DEC(1, 1)
|
||||
tinyint TINYINT
|
||||
smallint SMALLINT
|
||||
int INT
|
||||
integer INTEGER
|
||||
bigint BIGINT
|
||||
float FLOAT
|
||||
double DOUBLE
|
||||
char CHAR
|
||||
varchar VARCHAR
|
||||
text TEXT
|
||||
tinytext TINYTEXT
|
||||
mediumtext MEDIUMTEXT
|
||||
longtext LONGTEXT
|
||||
blob BLOB
|
||||
tinyblob TINYBLOB
|
||||
mediumblob MEDIUMBLOB
|
||||
longblob LONGBLOB
|
||||
binary BINARY(1)
|
||||
dec DEC(1, 1)
|
||||
tinyint TINYINT
|
||||
smallint SMALLINT
|
||||
int INT
|
||||
integer INTEGER
|
||||
bigint BIGINT
|
||||
float FLOAT
|
||||
double DOUBLE
|
||||
char CHAR
|
||||
varchar VARCHAR
|
||||
text TEXT
|
||||
tinytext TINYTEXT
|
||||
mediumtext MEDIUMTEXT
|
||||
longtext LONGTEXT
|
||||
blob BLOB
|
||||
tinyblob TINYBLOB
|
||||
mediumblob MEDIUMBLOB
|
||||
longblob LONGBLOB
|
@ -1,32 +0,0 @@
|
||||
DROP TABLE IF EXISTS test_type_alias;
|
||||
|
||||
CREATE TABLE test_type_alias(
|
||||
binary BINARY(1),
|
||||
dec DEC(1, 1),
|
||||
tinyint TINYINT,
|
||||
smallint SMALLINT,
|
||||
int INT,
|
||||
integer INTEGER,
|
||||
bigint BIGINT,
|
||||
float FLOAT,
|
||||
double DOUBLE,
|
||||
char CHAR,
|
||||
varchar VARCHAR,
|
||||
text TEXT,
|
||||
tinytext TINYTEXT,
|
||||
mediumtext MEDIUMTEXT,
|
||||
longtext LONGTEXT,
|
||||
blob BLOB,
|
||||
tinyblob TINYBLOB,
|
||||
mediumblob MEDIUMBLOB,
|
||||
longblob LONGBLOB
|
||||
) ENGINE = Memory;
|
||||
|
||||
DESC test_type_alias;
|
||||
|
||||
DETACH TABLE test_type_alias;
|
||||
ATTACH TABLE test_type_alias;
|
||||
|
||||
DESC test_type_alias;
|
||||
|
||||
DROP TABLE IF EXISTS test_type_alias;
|
Loading…
Reference in New Issue
Block a user